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/12/11 15:33:25 UTC

[doris] branch master updated: [Enhancement](status) Unify error code and enable customed err msg for BE internal errors (#14744)

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 f3aea7f0f0 [Enhancement](status) Unify error code and enable customed err msg for BE internal errors (#14744)
f3aea7f0f0 is described below

commit f3aea7f0f08438c31c6781ad9263b3c17a825dab
Author: plat1ko <pl...@gmail.com>
AuthorDate: Sun Dec 11 23:33:18 2022 +0800

    [Enhancement](status) Unify error code and enable customed err msg for BE internal errors (#14744)
---
 be/src/agent/agent_server.cpp                      |   2 +-
 be/src/agent/heartbeat_server.cpp                  |   2 +-
 be/src/agent/task_worker_pool.cpp                  |   9 +-
 be/src/common/daemon.cpp                           |  14 +-
 be/src/common/status.cpp                           | 210 +-----
 be/src/common/status.h                             | 827 ++++++++++-----------
 be/src/env/env_posix.cpp                           |   7 +-
 be/src/exec/broker_scan_node.cpp                   |   3 +-
 be/src/exec/csv_scanner.cpp                        |   2 +
 be/src/exec/decompressor.cpp                       |   2 +
 be/src/exec/es/es_scan_reader.cpp                  |   4 +-
 be/src/exec/es_http_scan_node.cpp                  |   7 +-
 be/src/exec/exchange_node.cpp                      |   3 +-
 be/src/exec/hash_table.cpp                         |   2 +-
 be/src/exec/json_scanner.cpp                       |   7 +-
 be/src/exec/olap_scan_node.cpp                     |   7 +-
 be/src/exec/olap_scanner.cpp                       |   4 +-
 be/src/exec/parquet_scanner.cpp                    |   6 +-
 be/src/exec/parquet_writer.cpp                     |   6 +-
 .../schema_scanner/schema_backends_scanner.cpp     |   3 +-
 be/src/exec/tablet_sink.cpp                        |  34 +-
 be/src/exec/tablet_sink.h                          |   2 +-
 be/src/exprs/expr.cpp                              |   4 +-
 be/src/http/action/compaction_action.cpp           |   5 +-
 be/src/http/action/download_action.cpp             |   8 +-
 be/src/http/action/stream_load.cpp                 |  19 +-
 be/src/io/broker_reader.cpp                        |   8 +-
 be/src/io/broker_writer.cpp                        |   8 +-
 be/src/io/cache/file_cache.cpp                     |   3 +-
 be/src/io/cache/sub_file_cache.cpp                 |   8 +-
 be/src/io/cache/whole_file_cache.cpp               |   6 +-
 be/src/io/fs/hdfs_file_system.h                    |   2 +
 be/src/io/hdfs_writer.cpp                          |   1 +
 be/src/olap/base_compaction.cpp                    |  19 +-
 be/src/olap/base_tablet.cpp                        |   3 +-
 be/src/olap/byte_buffer.cpp                        |   9 +-
 be/src/olap/byte_buffer.h                          |  24 +-
 be/src/olap/collect_iterator.cpp                   |  19 +-
 be/src/olap/collect_iterator.h                     |   2 +-
 be/src/olap/compaction.cpp                         |   7 +-
 be/src/olap/comparison_predicate.h                 |   2 +-
 be/src/olap/compress.h                             |   8 +-
 be/src/olap/cumulative_compaction.cpp              |  17 +-
 be/src/olap/data_dir.cpp                           |  26 +-
 be/src/olap/decimal12.h                            |   2 +-
 be/src/olap/delete_handler.cpp                     |  15 +-
 be/src/olap/delete_handler.h                       |   4 +-
 be/src/olap/delta_writer.cpp                       |  15 +-
 be/src/olap/file_helper.cpp                        |  53 +-
 be/src/olap/file_helper.h                          |  36 +-
 be/src/olap/generic_iterators.cpp                  |   5 +-
 be/src/olap/in_list_predicate.h                    |   2 +-
 be/src/olap/memtable.cpp                           |   3 +-
 be/src/olap/memtable_flush_executor.cpp            |  17 +-
 be/src/olap/memtable_flush_executor.h              |   4 +-
 be/src/olap/merger.h                               |   2 +-
 be/src/olap/olap_meta.cpp                          |  15 +-
 be/src/olap/olap_server.cpp                        |   6 +-
 be/src/olap/options.cpp                            |  13 +-
 be/src/olap/push_handler.cpp                       |  84 +--
 be/src/olap/reader.cpp                             |  11 +-
 be/src/olap/reader.h                               |  10 +-
 be/src/olap/row_block.cpp                          |   3 +-
 be/src/olap/row_cursor.cpp                         |  19 +-
 be/src/olap/rowset/beta_rowset.cpp                 |  29 +-
 be/src/olap/rowset/beta_rowset_reader.cpp          |  33 +-
 be/src/olap/rowset/beta_rowset_writer.cpp          |  64 +-
 be/src/olap/rowset/beta_rowset_writer.h            |   2 +-
 be/src/olap/rowset/rowset.h                        |   8 +-
 be/src/olap/rowset/rowset_factory.cpp              |   9 +-
 be/src/olap/rowset/rowset_factory.h                |   4 +-
 be/src/olap/rowset/rowset_meta_manager.cpp         |  15 +-
 be/src/olap/rowset/rowset_reader.h                 |   4 +-
 be/src/olap/rowset/rowset_writer.h                 |  16 +-
 .../rowset/segment_v2/indexed_column_reader.cpp    |   5 +-
 be/src/olap/rowset/segment_v2/segment_iterator.cpp |   3 +-
 be/src/olap/rowset/vertical_beta_rowset_writer.cpp |   6 +-
 be/src/olap/schema_change.cpp                      |  91 +--
 be/src/olap/schema_change.h                        |   5 +-
 be/src/olap/snapshot_manager.cpp                   |  27 +-
 be/src/olap/storage_engine.cpp                     |  19 +-
 be/src/olap/storage_engine.h                       |   2 +-
 be/src/olap/stream_index_common.cpp                |   3 +-
 be/src/olap/tablet.cpp                             |  55 +-
 be/src/olap/tablet_manager.cpp                     |  37 +-
 be/src/olap/tablet_manager.h                       |   8 +-
 be/src/olap/tablet_meta.cpp                        |  17 +-
 be/src/olap/tablet_meta_manager.cpp                |   7 +-
 be/src/olap/task/engine_batch_load_task.cpp        |   3 +-
 be/src/olap/task/engine_batch_load_task.h          |   2 +-
 be/src/olap/task/engine_clone_task.cpp             |   3 +-
 be/src/olap/task/engine_publish_version_task.cpp   |  11 +-
 be/src/olap/tuple_reader.cpp                       |  11 +-
 be/src/olap/txn_manager.cpp                        |  19 +-
 be/src/olap/types.h                                |  49 +-
 be/src/olap/utils.cpp                              |  41 +-
 be/src/olap/utils.h                                |   2 +-
 be/src/olap/version_graph.cpp                      |   5 +-
 be/src/pipeline/exec/agg_context.h                 |   3 +
 be/src/pipeline/exec/exchange_sink_buffer.cpp      |   3 +-
 .../exec/streaming_aggregation_source_operator.h   |   2 +
 be/src/runtime/broker_mgr.cpp                      |   5 +-
 be/src/runtime/buffer_control_block.cpp            |   4 +-
 be/src/runtime/buffered_block_mgr2.cc              |  16 +-
 be/src/runtime/buffered_block_mgr2.h               |   1 +
 be/src/runtime/bufferpool/buffer_pool.cc           |   7 +-
 be/src/runtime/data_stream_sender.cpp              |   4 +-
 be/src/runtime/disk_io_mgr_reader_context.cc       |   4 +-
 be/src/runtime/exec_env_init.cpp                   |   2 +-
 be/src/runtime/fold_constant_executor.cpp          |   8 +-
 be/src/runtime/fragment_mgr.cpp                    |   6 +-
 be/src/runtime/load_path_mgr.cpp                   |   6 +-
 be/src/runtime/mem_pool.h                          |   2 +-
 be/src/runtime/plan_fragment_executor.cpp          |  15 +-
 be/src/runtime/routine_load/data_consumer.cpp      |   2 +-
 .../runtime/routine_load/data_consumer_group.cpp   |   2 +-
 .../routine_load/routine_load_task_executor.cpp    |  27 +-
 be/src/runtime/runtime_state.cpp                   |  15 +-
 be/src/runtime/runtime_state.h                     |   3 -
 be/src/runtime/small_file_mgr.cpp                  |   2 +-
 be/src/runtime/snapshot_loader.cpp                 |  15 +-
 be/src/runtime/stream_load/stream_load_context.cpp |  15 +-
 .../runtime/stream_load/stream_load_executor.cpp   |  27 +-
 be/src/runtime/tablets_channel.cpp                 |  33 +-
 be/src/runtime/tmp_file_mgr.cc                     |   8 +-
 be/src/runtime/user_function_cache.cpp             |   2 +-
 be/src/service/backend_options.cpp                 |   2 +-
 be/src/service/backend_service.cpp                 |   6 +-
 be/src/service/doris_main.cpp                      |   7 +-
 be/src/service/internal_service.cpp                |  26 +-
 be/src/tools/meta_tool.cpp                         |   2 +-
 be/src/util/broker_storage_backend.cpp             |   8 +-
 be/src/util/cgroup_util.cpp                        |   4 +-
 be/src/util/hdfs_util.cpp                          |   2 +-
 be/src/util/load_error_hub.cpp                     |   1 +
 be/src/util/storage_backend_mgr.cpp                |   3 +-
 be/src/util/threadpool.cpp                         |   3 +-
 be/src/util/thrift_rpc_helper.cpp                  |   5 +-
 be/src/util/zlib.cpp                               |   5 +-
 .../aggregate_function_java_udaf.h                 |   5 +-
 be/src/vec/exec/format/json/new_json_reader.cpp    |   7 +-
 be/src/vec/exec/scan/new_es_scan_node.cpp          |   6 +-
 be/src/vec/exec/scan/vfile_scanner.cpp             |   5 +-
 be/src/vec/exec/varrow_scanner.cpp                 |  10 +-
 be/src/vec/exec/vbroker_scan_node.cpp              |   3 +-
 be/src/vec/exec/vjdbc_connector.cpp                |   3 +-
 be/src/vec/exec/vjson_scanner.cpp                  |  13 +-
 be/src/vec/exprs/vexpr.cpp                         |   2 +-
 be/src/vec/functions/function_java_udf.h           |   2 +-
 be/src/vec/olap/block_reader.cpp                   |  15 +-
 be/src/vec/olap/block_reader.h                     |   2 +-
 be/src/vec/olap/vcollect_iterator.cpp              |  64 +-
 be/src/vec/olap/vcollect_iterator.h                |   4 +-
 be/src/vec/olap/vertical_block_reader.cpp          |  21 +-
 be/src/vec/olap/vertical_merge_iterator.cpp        |   9 +-
 be/src/vec/olap/vgeneric_iterators.cpp             |   5 +-
 be/src/vec/runtime/vorc_writer.cpp                 |   4 +-
 be/src/vec/sink/vdata_stream_sender.cpp            |   4 +-
 be/src/vec/sink/vtablet_sink.cpp                   |  12 +-
 be/test/agent/agent_server_test.cpp                |   4 +-
 be/test/common/config_test.cpp                     |  13 +-
 be/test/common/status_test.cpp                     |  13 +-
 be/test/env/env_posix_test.cpp                     |   5 +-
 be/test/exec/es_query_builder_test.cpp             |   8 +-
 be/test/exec/hdfs_file_reader_test.cpp             |   2 +-
 be/test/exec/s3_reader_test.cpp                    |   6 +-
 be/test/exec/tablet_info_test.cpp                  |   2 +-
 be/test/http/http_client_test.cpp                  |   2 +-
 be/test/olap/byte_buffer_test.cpp                  |  19 +-
 be/test/olap/delete_handler_test.cpp               |  55 +-
 be/test/olap/generic_iterators_test.cpp            |   9 +-
 be/test/olap/memtable_flush_executor_test.cpp      |   4 +-
 be/test/olap/olap_meta_test.cpp                    |   3 +-
 be/test/olap/ordered_data_compaction_test.cpp      |   3 +-
 be/test/olap/primary_key_index_test.cpp            |   3 +-
 be/test/olap/rowid_conversion_test.cpp             |   3 +-
 be/test/olap/rowset/beta_rowset_test.cpp           |   9 +-
 .../rowset/segment_v2/binary_prefix_page_test.cpp  |   3 +-
 .../rowset/segment_v2/bitshuffle_page_test.cpp     |   3 +-
 .../segment_v2/column_reader_writer_test.cpp       |   4 +-
 be/test/olap/rowset/segment_v2/plain_page_test.cpp |   3 +-
 be/test/olap/rowset/segment_v2/segment_test.cpp    |  15 +-
 be/test/olap/segcompaction_test.cpp                |   3 +-
 be/test/olap/tablet_meta_manager_test.cpp          |   3 +-
 be/test/olap/tablet_test.cpp                       |  20 +-
 be/test/runtime/buffered_tuple_stream2_test.cpp    |   2 +-
 be/test/runtime/data_stream_test.cpp               |   4 +-
 be/test/runtime/disk_io_mgr_test.cpp               |   4 +-
 be/test/runtime/load_channel_mgr_test.cpp          |   6 +-
 be/test/runtime/test_env.cc                        |   4 +-
 be/test/udf/uda_test.cpp                           |  26 +-
 be/test/util/broker_storage_backend_test.cpp       |   7 +-
 be/test/util/hdfs_storage_backend_test.cpp         |   5 +-
 be/test/util/s3_storage_backend_test.cpp           |   9 +-
 be/test/util/thread_test.cpp                       |   5 +-
 be/test/util/threadpool_test.cpp                   |  13 +-
 be/test/util/zip_util_test.cpp                     |   4 +-
 be/test/vec/exec/vgeneric_iterators_test.cpp       |   9 +-
 be/test/vec/olap/vertical_compaction_test.cpp      |   9 +-
 be/test/vec/runtime/vdata_stream_test.cpp          |   2 +-
 build.sh                                           |   5 +
 .../jsonb_p0/test_jsonb_load_and_function.groovy   |   2 +-
 .../test_jsonb_load_unique_key_and_function.groovy |   2 +-
 .../cast_function/test_cast_function.groovy        |   2 +-
 204 files changed, 1514 insertions(+), 1637 deletions(-)

diff --git a/be/src/agent/agent_server.cpp b/be/src/agent/agent_server.cpp
index d695200620..d57777b6e6 100644
--- a/be/src/agent/agent_server.cpp
+++ b/be/src/agent/agent_server.cpp
@@ -191,7 +191,7 @@ void AgentServer::submit_tasks(TAgentResult& agent_result,
 #undef HANDLE_TYPE
 
         if (!ret_st.ok()) {
-            LOG_WARNING("failed to submit task").tag("task", task).error(ret_st.get_error_msg());
+            LOG_WARNING("failed to submit task").tag("task", task).error(ret_st);
             // For now, all tasks in the batch share one status, so if any task
             // was failed to submit, we can only return error to FE(even when some
             // tasks have already been successfully submitted).
diff --git a/be/src/agent/heartbeat_server.cpp b/be/src/agent/heartbeat_server.cpp
index 5a412733eb..705f14ab9a 100644
--- a/be/src/agent/heartbeat_server.cpp
+++ b/be/src/agent/heartbeat_server.cpp
@@ -84,7 +84,7 @@ Status HeartbeatServer::_heartbeat(const TMasterInfo& master_info) {
         // write and update cluster id
         auto st = _olap_engine->set_cluster_id(master_info.cluster_id);
         if (!st.ok()) {
-            LOG(WARNING) << "fail to set cluster id. status=" << st.get_error_msg();
+            LOG(WARNING) << "fail to set cluster id. status=" << st;
             return Status::InternalError("fail to set cluster id.");
         } else {
             _master_info->cluster_id = master_info.cluster_id;
diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp
index cbf3ffa0ee..ca52ce7cf0 100644
--- a/be/src/agent/task_worker_pool.cpp
+++ b/be/src/agent/task_worker_pool.cpp
@@ -56,6 +56,7 @@
 #include "util/trace.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(agent_task_queue_size, MetricUnit::NOUNIT);
 
@@ -217,7 +218,7 @@ void TaskWorkerPool::start() {
 
     for (int i = 0; i < _worker_count; i++) {
         auto st = _thread_pool->submit_func(cb);
-        CHECK(st.ok()) << st.to_string();
+        CHECK(st.ok()) << st;
     }
 #endif
 }
@@ -554,7 +555,7 @@ void TaskWorkerPool::_alter_tablet(const TAgentTaskRequest& agent_task_req, int6
         }
     }
 
-    if (!status.ok() && status.code() != TStatusCode::NOT_IMPLEMENTED_ERROR) {
+    if (!status.ok() && !status.is<NOT_IMPLEMENTED_ERROR>()) {
         LOG_WARNING("failed to {}", process_name)
                 .tag("signature", agent_task_req.signature)
                 .tag("base_tablet_id", agent_task_req.alter_tablet_req_v2.base_tablet_id)
@@ -686,7 +687,7 @@ void TaskWorkerPool::_publish_version_worker_thread_callback() {
             status = _env->storage_engine()->execute_task(&engine_task);
             if (status.ok()) {
                 break;
-            } else if (status.precise_code() == OLAP_ERR_PUBLISH_VERSION_NOT_CONTINUOUS) {
+            } else if (status.is<PUBLISH_VERSION_NOT_CONTINUOUS>()) {
                 int64_t time_elapsed = time(nullptr) - agent_task_req.recv_time;
                 if (time_elapsed > PUBLISH_TIMEOUT_SEC) {
                     LOG(INFO) << "task elapsed " << time_elapsed
@@ -710,7 +711,7 @@ void TaskWorkerPool::_publish_version_worker_thread_callback() {
                 std::this_thread::sleep_for(std::chrono::seconds(1));
             }
         }
-        if (status.precise_code() == OLAP_ERR_PUBLISH_VERSION_NOT_CONTINUOUS && !is_task_timeout) {
+        if (status.is<PUBLISH_VERSION_NOT_CONTINUOUS>() && !is_task_timeout) {
             continue;
         }
 
diff --git a/be/src/common/daemon.cpp b/be/src/common/daemon.cpp
index 08a47b37f4..e0ac827fb3 100644
--- a/be/src/common/daemon.cpp
+++ b/be/src/common/daemon.cpp
@@ -318,12 +318,12 @@ static void init_doris_metrics(const std::vector<StorePath>& store_paths) {
     if (init_system_metrics) {
         auto st = DiskInfo::get_disk_devices(paths, &disk_devices);
         if (!st.ok()) {
-            LOG(WARNING) << "get disk devices failed, status=" << st.get_error_msg();
+            LOG(WARNING) << "get disk devices failed, status=" << st;
             return;
         }
         st = get_inet_interfaces(&network_interfaces);
         if (!st.ok()) {
-            LOG(WARNING) << "get inet interfaces failed, status=" << st.get_error_msg();
+            LOG(WARNING) << "get inet interfaces failed, status=" << st;
             return;
         }
     }
@@ -413,20 +413,20 @@ void Daemon::start() {
     st = Thread::create(
             "Daemon", "tcmalloc_gc_thread", [this]() { this->tcmalloc_gc_thread(); },
             &_tcmalloc_gc_thread);
-    CHECK(st.ok()) << st.to_string();
+    CHECK(st.ok()) << st;
     st = Thread::create(
             "Daemon", "buffer_pool_gc_thread", [this]() { this->buffer_pool_gc_thread(); },
             &_buffer_pool_gc_thread);
-    CHECK(st.ok()) << st.to_string();
+    CHECK(st.ok()) << st;
     st = Thread::create(
             "Daemon", "memory_maintenance_thread", [this]() { this->memory_maintenance_thread(); },
             &_memory_maintenance_thread);
-    CHECK(st.ok()) << st.to_string();
+    CHECK(st.ok()) << st;
     st = Thread::create(
             "Daemon", "load_channel_tracker_refresh_thread",
             [this]() { this->load_channel_tracker_refresh_thread(); },
             &_load_channel_tracker_refresh_thread);
-    CHECK(st.ok()) << st.to_string();
+    CHECK(st.ok()) << st;
 
     if (config::enable_metric_calculator) {
         CHECK(DorisMetrics::instance()->is_inited())
@@ -438,7 +438,7 @@ void Daemon::start() {
         st = Thread::create(
                 "Daemon", "calculate_metrics_thread",
                 [this]() { this->calculate_metrics_thread(); }, &_calculate_metrics_thread);
-        CHECK(st.ok()) << st.to_string();
+        CHECK(st.ok()) << st;
     }
 }
 
diff --git a/be/src/common/status.cpp b/be/src/common/status.cpp
index 9dc8b27a4b..32aafe35ed 100644
--- a/be/src/common/status.cpp
+++ b/be/src/common/status.cpp
@@ -8,89 +8,40 @@
 #include <rapidjson/stringbuffer.h>
 
 #include "gen_cpp/types.pb.h" // for PStatus
-#include "util/stack_util.h"
 
 namespace doris {
 
-constexpr int MAX_ERROR_NUM = 65536;
-struct ErrorCodeState {
-    int16_t error_code = 0;
-    bool stacktrace = true;
-    std::string description;
-    size_t count = 0; // Used for count the number of error happens
-    std::mutex mutex; // lock guard for count state
-};
-ErrorCodeState error_states[MAX_ERROR_NUM];
-class Initializer {
-public:
-    Initializer() {
-#define M(NAME, ERRORCODE, DESC, STACKTRACEENABLED) \
-    error_states[abs(ERRORCODE)].stacktrace = STACKTRACEENABLED;
-        APPLY_FOR_ERROR_CODES(M)
-#undef M
-// Currently, most of description is empty, so that we use NAME as description
-#define M(NAME, ERRORCODE, DESC, STACKTRACEENABLED) \
-    error_states[abs(ERRORCODE)].description = #NAME;
-        APPLY_FOR_ERROR_CODES(M)
-#undef M
-    }
-};
-Initializer init; // Used to init the error_states array
-
 Status::Status(const TStatus& s) {
-    _code = s.status_code;
-    if (_code != TStatusCode::OK) {
-        // It is ok to set precise code == 1 here, because we do not know the precise code
-        // just from thrift's TStatus
-        _precise_code = 1;
+    _code = static_cast<int>(s.status_code);
+    if (_code != ErrorCode::OK) {
         if (!s.error_msgs.empty()) {
-            _err_msg = s.error_msgs[0];
+            if (_err_msg == nullptr) {
+                _err_msg = std::make_unique<ErrMsg>();
+            }
+            _err_msg->_msg = s.error_msgs[0];
+#ifdef ENABLE_STACKTRACE
+            _err_msg->_stack = "";
+#endif
         }
+    } else {
+        _err_msg.reset();
     }
 }
 
-// TODO yiguolei, maybe should init PStatus's precise code because OLAPInternal Error may
-// tranfer precise code during BRPC
 Status::Status(const PStatus& s) {
-    _code = (TStatusCode::type)s.status_code();
-    if (_code != TStatusCode::OK) {
-        // It is ok to set precise code == 1 here, because we do not know the precise code
-        // just from thrift's TStatus
-        _precise_code = 1;
+    _code = s.status_code();
+    if (_code != ErrorCode::OK) {
         if (s.error_msgs_size() > 0) {
-            _err_msg = s.error_msgs(0);
-        }
-    }
-}
-
-// A wrapper for ErrorCode
-//      Precise code is for ErrorCode's enum value
-//      All Status Error is treated as Internal Error
-Status Status::OLAPInternalError(int16_t precise_code, std::string_view msg) {
-    return ConstructErrorStatus(TStatusCode::INTERNAL_ERROR, precise_code, msg);
-}
-
-Status Status::ConstructErrorStatus(TStatusCode::type tcode, int16_t precise_code,
-                                    std::string_view msg) {
-// This will print all error status's stack, it maybe too many, but it is just used for debug
-#ifdef PRINT_ALL_ERR_STATUS_STACKTRACE
-    LOG(WARNING) << "Error occurred, error code = " << precise_code << ", with message: " << msg
-                 << "\n caused by:" << get_stack_trace();
+            if (_err_msg == nullptr) {
+                _err_msg = std::make_unique<ErrMsg>();
+            }
+            _err_msg->_msg = s.error_msgs(0);
+#ifdef ENABLE_STACKTRACE
+            _err_msg->_stack = "";
 #endif
-    if (error_states[abs(precise_code)].stacktrace) {
-        // Add stacktrace as part of message, could use LOG(WARN) << "" << status will print both
-        // the error message and the stacktrace
-        if (msg.empty()) {
-            return Status(tcode, get_stack_trace(), precise_code);
-        } else {
-            return Status(tcode, std::string(msg) + "/n" + get_stack_trace(), precise_code);
         }
     } else {
-        if (msg.empty()) {
-            return Status(tcode, std::string_view(), precise_code);
-        } else {
-            return Status(tcode, msg, precise_code);
-        }
+        _err_msg.reset();
     }
 }
 
@@ -98,11 +49,12 @@ void Status::to_thrift(TStatus* s) const {
     s->error_msgs.clear();
     if (ok()) {
         s->status_code = TStatusCode::OK;
-    } else {
-        s->status_code = code();
-        s->error_msgs.push_back(_err_msg);
-        s->__isset.error_msgs = true;
+        return;
     }
+    s->status_code = (int16_t)_code > 0 ? (TStatusCode::type)_code : TStatusCode::INTERNAL_ERROR;
+    s->error_msgs.push_back(
+            fmt::format("[{}]{}", code_as_string(), _err_msg ? _err_msg->_msg : ""));
+    s->__isset.error_msgs = true;
 }
 
 TStatus Status::to_thrift() const {
@@ -113,102 +65,28 @@ TStatus Status::to_thrift() const {
 
 void Status::to_protobuf(PStatus* s) const {
     s->clear_error_msgs();
-    if (ok()) {
-        s->set_status_code((int)TStatusCode::OK);
-    } else {
-        s->set_status_code(code());
-        s->add_error_msgs(_err_msg);
-    }
-}
-
-const char* Status::code_as_string() const {
-    switch (code()) {
-    case TStatusCode::OK:
-        return "OK";
-    case TStatusCode::CANCELLED:
-        return "Cancelled";
-    case TStatusCode::NOT_IMPLEMENTED_ERROR:
-        return "Not supported";
-    case TStatusCode::RUNTIME_ERROR:
-        return "Runtime error";
-    case TStatusCode::MEM_LIMIT_EXCEEDED:
-        return "Memory limit exceeded";
-    case TStatusCode::INTERNAL_ERROR:
-        return "Internal error";
-    case TStatusCode::THRIFT_RPC_ERROR:
-        return "RPC error";
-    case TStatusCode::TIMEOUT:
-        return "Timeout";
-    case TStatusCode::MEM_ALLOC_FAILED:
-        return "Memory alloc failed";
-    case TStatusCode::BUFFER_ALLOCATION_FAILED:
-        return "Buffer alloc failed";
-    case TStatusCode::MINIMUM_RESERVATION_UNAVAILABLE:
-        return "Minimum reservation unavailable";
-    case TStatusCode::PUBLISH_TIMEOUT:
-        return "Publish timeout";
-    case TStatusCode::LABEL_ALREADY_EXISTS:
-        return "Label already exist";
-    case TStatusCode::END_OF_FILE:
-        return "End of file";
-    case TStatusCode::NOT_FOUND:
-        return "Not found";
-    case TStatusCode::CORRUPTION:
-        return "Corruption";
-    case TStatusCode::INVALID_ARGUMENT:
-        return "Invalid argument";
-    case TStatusCode::IO_ERROR:
-        return "IO error";
-    case TStatusCode::ALREADY_EXIST:
-        return "Already exist";
-    case TStatusCode::NETWORK_ERROR:
-        return "Network error";
-    case TStatusCode::ILLEGAL_STATE:
-        return "Illegal state";
-    case TStatusCode::NOT_AUTHORIZED:
-        return "Not authorized";
-    case TStatusCode::REMOTE_ERROR:
-        return "Remote error";
-    case TStatusCode::SERVICE_UNAVAILABLE:
-        return "Service unavailable";
-    case TStatusCode::UNINITIALIZED:
-        return "Uninitialized";
-    case TStatusCode::CONFIGURATION_ERROR:
-        return "Configuration error";
-    case TStatusCode::INCOMPLETE:
-        return "Incomplete";
-    case TStatusCode::DATA_QUALITY_ERROR:
-        return "Data quality error";
-    default: {
-        return "Unknown code";
-    }
-    }
-    return "Unknown code";
-}
-
-std::string Status::to_string() const {
-    std::string result(code_as_string());
-    if (ok()) {
-        return result;
-    }
-    if (precise_code() != 1) {
-        result.append(fmt::format("(error {})", precise_code()));
+    s->set_status_code((int)_code);
+    if (!ok() && _err_msg) {
+        s->add_error_msgs(_err_msg->_msg);
     }
-    result.append(": ");
-    result.append(_err_msg);
-    return result;
 }
 
 Status& Status::prepend(std::string_view msg) {
     if (!ok()) {
-        _err_msg = std::string(msg) + _err_msg;
+        if (_err_msg == nullptr) {
+            _err_msg = std::make_unique<ErrMsg>();
+        }
+        _err_msg->_msg = std::string(msg) + _err_msg->_msg;
     }
     return *this;
 }
 
 Status& Status::append(std::string_view msg) {
     if (!ok()) {
-        _err_msg.append(msg);
+        if (_err_msg == nullptr) {
+            _err_msg = std::make_unique<ErrMsg>();
+        }
+        _err_msg->_msg.append(msg);
     }
     return *this;
 }
@@ -216,25 +94,13 @@ Status& Status::append(std::string_view msg) {
 std::string Status::to_json() const {
     rapidjson::StringBuffer s;
     rapidjson::PrettyWriter<rapidjson::StringBuffer> writer(s);
-
     writer.StartObject();
     // status
     writer.Key("status");
-    writer.String(code_as_string());
+    writer.String(code_as_string().c_str());
     // msg
     writer.Key("msg");
-    if (ok()) {
-        writer.String("OK");
-    } else {
-        int16_t posix = precise_code();
-        if (posix != 1) {
-            char buf[64];
-            snprintf(buf, sizeof(buf), " (error %d)", posix);
-            writer.String((_err_msg + buf).c_str());
-        } else {
-            writer.String(_err_msg.c_str());
-        }
-    }
+    ok() ? writer.String("OK") : writer.String(_err_msg ? _err_msg->_msg.c_str() : "");
     writer.EndObject();
     return s.GetString();
 }
diff --git a/be/src/common/status.h b/be/src/common/status.h
index 46c3a66e89..c0f903aff7 100644
--- a/be/src/common/status.h
+++ b/be/src/common/status.h
@@ -12,249 +12,283 @@
 #include <string_view>
 
 #include "common/compiler_util.h"
-#include "common/logging.h"
 #include "gen_cpp/Status_types.h" // for TStatus
+#ifdef ENABLE_STACKTRACE
+#include "util/stack_util.h"
+#endif
 
 namespace doris {
 
 class PStatus;
 
-// ErrorName, ErrorCode, String Description, Should print stacktrace
-#define APPLY_FOR_ERROR_CODES(M)                                         \
-    M(OLAP_SUCCESS, 0, "", false)                                        \
-    M(OLAP_ERR_OTHER_ERROR, -1, "", true)                                \
-    M(OLAP_REQUEST_FAILED, -2, "", false)                                \
-    M(OLAP_ERR_OS_ERROR, -100, "", true)                                 \
-    M(OLAP_ERR_DIR_NOT_EXIST, -101, "", true)                            \
-    M(OLAP_ERR_FILE_NOT_EXIST, -102, "", true)                           \
-    M(OLAP_ERR_CREATE_FILE_ERROR, -103, "", true)                        \
-    M(OLAP_ERR_MALLOC_ERROR, -104, "", true)                             \
-    M(OLAP_ERR_STL_ERROR, -105, "", true)                                \
-    M(OLAP_ERR_IO_ERROR, -106, "", true)                                 \
-    M(OLAP_ERR_MUTEX_ERROR, -107, "", true)                              \
-    M(OLAP_ERR_PTHREAD_ERROR, -108, "", true)                            \
-    M(OLAP_ERR_NETWORK_ERROR, -109, "", true)                            \
-    M(OLAP_ERR_UB_FUNC_ERROR, -110, "", true)                            \
-    M(OLAP_ERR_COMPRESS_ERROR, -111, "", true)                           \
-    M(OLAP_ERR_DECOMPRESS_ERROR, -112, "", true)                         \
-    M(OLAP_ERR_UNKNOWN_COMPRESSION_TYPE, -113, "", true)                 \
-    M(OLAP_ERR_MMAP_ERROR, -114, "", true)                               \
-    M(OLAP_ERR_RWLOCK_ERROR, -115, "", true)                             \
-    M(OLAP_ERR_READ_UNENOUGH, -116, "", true)                            \
-    M(OLAP_ERR_CANNOT_CREATE_DIR, -117, "", true)                        \
-    M(OLAP_ERR_UB_NETWORK_ERROR, -118, "", true)                         \
-    M(OLAP_ERR_FILE_FORMAT_ERROR, -119, "", true)                        \
-    M(OLAP_ERR_EVAL_CONJUNCTS_ERROR, -120, "", true)                     \
-    M(OLAP_ERR_COPY_FILE_ERROR, -121, "", true)                          \
-    M(OLAP_ERR_FILE_ALREADY_EXIST, -122, "", true)                       \
-    M(OLAP_ERR_NOT_INITED, -200, "", true)                               \
-    M(OLAP_ERR_FUNC_NOT_IMPLEMENTED, -201, "", true)                     \
-    M(OLAP_ERR_CALL_SEQUENCE_ERROR, -202, "", true)                      \
-    M(OLAP_ERR_INPUT_PARAMETER_ERROR, -203, "", true)                    \
-    M(OLAP_ERR_BUFFER_OVERFLOW, -204, "", true)                          \
-    M(OLAP_ERR_CONFIG_ERROR, -205, "", true)                             \
-    M(OLAP_ERR_INIT_FAILED, -206, "", true)                              \
-    M(OLAP_ERR_INVALID_SCHEMA, -207, "", true)                           \
-    M(OLAP_ERR_CHECKSUM_ERROR, -208, "", true)                           \
-    M(OLAP_ERR_SIGNATURE_ERROR, -209, "", true)                          \
-    M(OLAP_ERR_CATCH_EXCEPTION, -210, "", true)                          \
-    M(OLAP_ERR_PARSE_PROTOBUF_ERROR, -211, "", true)                     \
-    M(OLAP_ERR_SERIALIZE_PROTOBUF_ERROR, -212, "", true)                 \
-    M(OLAP_ERR_WRITE_PROTOBUF_ERROR, -213, "", true)                     \
-    M(OLAP_ERR_VERSION_NOT_EXIST, -214, "", true)                        \
-    M(OLAP_ERR_TABLE_NOT_FOUND, -215, "", true)                          \
-    M(OLAP_ERR_TRY_LOCK_FAILED, -216, "", true)                          \
-    M(OLAP_ERR_OUT_OF_BOUND, -218, "", true)                             \
-    M(OLAP_ERR_UNDERFLOW, -219, "", true)                                \
-    M(OLAP_ERR_FILE_DATA_ERROR, -220, "", true)                          \
-    M(OLAP_ERR_TEST_FILE_ERROR, -221, "", true)                          \
-    M(OLAP_ERR_INVALID_ROOT_PATH, -222, "", true)                        \
-    M(OLAP_ERR_NO_AVAILABLE_ROOT_PATH, -223, "", true)                   \
-    M(OLAP_ERR_CHECK_LINES_ERROR, -224, "", true)                        \
-    M(OLAP_ERR_INVALID_CLUSTER_INFO, -225, "", true)                     \
-    M(OLAP_ERR_TRANSACTION_NOT_EXIST, -226, "", true)                    \
-    M(OLAP_ERR_DISK_FAILURE, -227, "", true)                             \
-    M(OLAP_ERR_TRANSACTION_ALREADY_COMMITTED, -228, "", true)            \
-    M(OLAP_ERR_TRANSACTION_ALREADY_VISIBLE, -229, "", true)              \
-    M(OLAP_ERR_VERSION_ALREADY_MERGED, -230, "", true)                   \
-    M(OLAP_ERR_LZO_DISABLED, -231, "", true)                             \
-    M(OLAP_ERR_DISK_REACH_CAPACITY_LIMIT, -232, "", true)                \
-    M(OLAP_ERR_TOO_MANY_TRANSACTIONS, -233, "", true)                    \
-    M(OLAP_ERR_INVALID_SNAPSHOT_VERSION, -234, "", true)                 \
-    M(OLAP_ERR_TOO_MANY_VERSION, -235, "", true)                         \
-    M(OLAP_ERR_NOT_INITIALIZED, -236, "", true)                          \
-    M(OLAP_ERR_ALREADY_CANCELLED, -237, "", true)                        \
-    M(OLAP_ERR_TOO_MANY_SEGMENTS, -238, "", true)                        \
-    M(OLAP_ERR_CE_CMD_PARAMS_ERROR, -300, "", true)                      \
-    M(OLAP_ERR_CE_BUFFER_TOO_SMALL, -301, "", true)                      \
-    M(OLAP_ERR_CE_CMD_NOT_VALID, -302, "", true)                         \
-    M(OLAP_ERR_CE_LOAD_TABLE_ERROR, -303, "", true)                      \
-    M(OLAP_ERR_CE_NOT_FINISHED, -304, "", true)                          \
-    M(OLAP_ERR_CE_TABLET_ID_EXIST, -305, "", true)                       \
-    M(OLAP_ERR_CE_TRY_CE_LOCK_ERROR, -306, "", false)                    \
-    M(OLAP_ERR_TABLE_VERSION_DUPLICATE_ERROR, -400, "", true)            \
-    M(OLAP_ERR_TABLE_VERSION_INDEX_MISMATCH_ERROR, -401, "", true)       \
-    M(OLAP_ERR_TABLE_INDEX_VALIDATE_ERROR, -402, "", true)               \
-    M(OLAP_ERR_TABLE_INDEX_FIND_ERROR, -403, "", true)                   \
-    M(OLAP_ERR_TABLE_CREATE_FROM_HEADER_ERROR, -404, "", true)           \
-    M(OLAP_ERR_TABLE_CREATE_META_ERROR, -405, "", true)                  \
-    M(OLAP_ERR_TABLE_ALREADY_DELETED_ERROR, -406, "", false)             \
-    M(OLAP_ERR_ENGINE_INSERT_EXISTS_TABLE, -500, "", true)               \
-    M(OLAP_ERR_ENGINE_DROP_NOEXISTS_TABLE, -501, "", true)               \
-    M(OLAP_ERR_ENGINE_LOAD_INDEX_TABLE_ERROR, -502, "", true)            \
-    M(OLAP_ERR_TABLE_INSERT_DUPLICATION_ERROR, -503, "", true)           \
-    M(OLAP_ERR_DELETE_VERSION_ERROR, -504, "", true)                     \
-    M(OLAP_ERR_GC_SCAN_PATH_ERROR, -505, "", true)                       \
-    M(OLAP_ERR_ENGINE_INSERT_OLD_TABLET, -506, "", true)                 \
-    M(OLAP_ERR_FETCH_OTHER_ERROR, -600, "", true)                        \
-    M(OLAP_ERR_FETCH_TABLE_NOT_EXIST, -601, "", true)                    \
-    M(OLAP_ERR_FETCH_VERSION_ERROR, -602, "", true)                      \
-    M(OLAP_ERR_FETCH_SCHEMA_ERROR, -603, "", true)                       \
-    M(OLAP_ERR_FETCH_COMPRESSION_ERROR, -604, "", true)                  \
-    M(OLAP_ERR_FETCH_CONTEXT_NOT_EXIST, -605, "", true)                  \
-    M(OLAP_ERR_FETCH_GET_READER_PARAMS_ERR, -606, "", true)              \
-    M(OLAP_ERR_FETCH_SAVE_SESSION_ERR, -607, "", true)                   \
-    M(OLAP_ERR_FETCH_MEMORY_EXCEEDED, -608, "", true)                    \
-    M(OLAP_ERR_READER_IS_UNINITIALIZED, -700, "", true)                  \
-    M(OLAP_ERR_READER_GET_ITERATOR_ERROR, -701, "", true)                \
-    M(OLAP_ERR_CAPTURE_ROWSET_READER_ERROR, -702, "", true)              \
-    M(OLAP_ERR_READER_READING_ERROR, -703, "", true)                     \
-    M(OLAP_ERR_READER_INITIALIZE_ERROR, -704, "", true)                  \
-    M(OLAP_ERR_BE_VERSION_NOT_MATCH, -800, "", true)                     \
-    M(OLAP_ERR_BE_REPLACE_VERSIONS_ERROR, -801, "", true)                \
-    M(OLAP_ERR_BE_MERGE_ERROR, -802, "", true)                           \
-    M(OLAP_ERR_CAPTURE_ROWSET_ERROR, -804, "", true)                     \
-    M(OLAP_ERR_BE_SAVE_HEADER_ERROR, -805, "", true)                     \
-    M(OLAP_ERR_BE_INIT_OLAP_DATA, -806, "", true)                        \
-    M(OLAP_ERR_BE_TRY_OBTAIN_VERSION_LOCKS, -807, "", true)              \
-    M(OLAP_ERR_BE_NO_SUITABLE_VERSION, -808, "", false)                  \
-    M(OLAP_ERR_BE_TRY_BE_LOCK_ERROR, -809, "", true)                     \
-    M(OLAP_ERR_BE_INVALID_NEED_MERGED_VERSIONS, -810, "", true)          \
-    M(OLAP_ERR_BE_ERROR_DELETE_ACTION, -811, "", true)                   \
-    M(OLAP_ERR_BE_SEGMENTS_OVERLAPPING, -812, "", true)                  \
-    M(OLAP_ERR_BE_CLONE_OCCURRED, -813, "", true)                        \
-    M(OLAP_ERR_PUSH_INIT_ERROR, -900, "", true)                          \
-    M(OLAP_ERR_PUSH_DELTA_FILE_EOF, -901, "", false)                     \
-    M(OLAP_ERR_PUSH_VERSION_INCORRECT, -902, "", true)                   \
-    M(OLAP_ERR_PUSH_SCHEMA_MISMATCH, -903, "", true)                     \
-    M(OLAP_ERR_PUSH_CHECKSUM_ERROR, -904, "", true)                      \
-    M(OLAP_ERR_PUSH_ACQUIRE_DATASOURCE_ERROR, -905, "", true)            \
-    M(OLAP_ERR_PUSH_CREAT_CUMULATIVE_ERROR, -906, "", true)              \
-    M(OLAP_ERR_PUSH_BUILD_DELTA_ERROR, -907, "", true)                   \
-    M(OLAP_ERR_PUSH_VERSION_ALREADY_EXIST, -908, "", true)               \
-    M(OLAP_ERR_PUSH_TABLE_NOT_EXIST, -909, "", true)                     \
-    M(OLAP_ERR_PUSH_INPUT_DATA_ERROR, -910, "", true)                    \
-    M(OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST, -911, "", true)           \
-    M(OLAP_ERR_PUSH_BATCH_PROCESS_REMOVED, -912, "", true)               \
-    M(OLAP_ERR_PUSH_COMMIT_ROWSET, -913, "", true)                       \
-    M(OLAP_ERR_PUSH_ROWSET_NOT_FOUND, -914, "", true)                    \
-    M(OLAP_ERR_INDEX_LOAD_ERROR, -1000, "", true)                        \
-    M(OLAP_ERR_INDEX_EOF, -1001, "", false)                              \
-    M(OLAP_ERR_INDEX_CHECKSUM_ERROR, -1002, "", true)                    \
-    M(OLAP_ERR_INDEX_DELTA_PRUNING, -1003, "", true)                     \
-    M(OLAP_ERR_DATA_ROW_BLOCK_ERROR, -1100, "", true)                    \
-    M(OLAP_ERR_DATA_FILE_TYPE_ERROR, -1101, "", true)                    \
-    M(OLAP_ERR_DATA_EOF, -1102, "", false)                               \
-    M(OLAP_ERR_WRITER_INDEX_WRITE_ERROR, -1200, "", true)                \
-    M(OLAP_ERR_WRITER_DATA_WRITE_ERROR, -1201, "", true)                 \
-    M(OLAP_ERR_WRITER_ROW_BLOCK_ERROR, -1202, "", true)                  \
-    M(OLAP_ERR_WRITER_SEGMENT_NOT_FINALIZED, -1203, "", true)            \
-    M(OLAP_ERR_ROWBLOCK_DECOMPRESS_ERROR, -1300, "", true)               \
-    M(OLAP_ERR_ROWBLOCK_FIND_ROW_EXCEPTION, -1301, "", true)             \
-    M(OLAP_ERR_ROWBLOCK_READ_INFO_ERROR, -1302, "", true)                \
-    M(OLAP_ERR_HEADER_ADD_VERSION, -1400, "", true)                      \
-    M(OLAP_ERR_HEADER_DELETE_VERSION, -1401, "", true)                   \
-    M(OLAP_ERR_HEADER_ADD_PENDING_DELTA, -1402, "", true)                \
-    M(OLAP_ERR_HEADER_ADD_INCREMENTAL_VERSION, -1403, "", true)          \
-    M(OLAP_ERR_HEADER_INVALID_FLAG, -1404, "", true)                     \
-    M(OLAP_ERR_HEADER_PUT, -1405, "", true)                              \
-    M(OLAP_ERR_HEADER_DELETE, -1406, "", true)                           \
-    M(OLAP_ERR_HEADER_GET, -1407, "", true)                              \
-    M(OLAP_ERR_HEADER_LOAD_INVALID_KEY, -1408, "", true)                 \
-    M(OLAP_ERR_HEADER_FLAG_PUT, -1409, "", true)                         \
-    M(OLAP_ERR_HEADER_LOAD_JSON_HEADER, -1410, "", true)                 \
-    M(OLAP_ERR_HEADER_INIT_FAILED, -1411, "", true)                      \
-    M(OLAP_ERR_HEADER_PB_PARSE_FAILED, -1412, "", true)                  \
-    M(OLAP_ERR_HEADER_HAS_PENDING_DATA, -1413, "", false)                \
-    M(OLAP_ERR_SCHEMA_SCHEMA_INVALID, -1500, "", false)                  \
-    M(OLAP_ERR_SCHEMA_SCHEMA_FIELD_INVALID, -1501, "", true)             \
-    M(OLAP_ERR_ALTER_MULTI_TABLE_ERR, -1600, "", true)                   \
-    M(OLAP_ERR_ALTER_DELTA_DOES_NOT_EXISTS, -1601, "", true)             \
-    M(OLAP_ERR_ALTER_STATUS_ERR, -1602, "", true)                        \
-    M(OLAP_ERR_PREVIOUS_SCHEMA_CHANGE_NOT_FINISHED, -1603, "", true)     \
-    M(OLAP_ERR_SCHEMA_CHANGE_INFO_INVALID, -1604, "", true)              \
-    M(OLAP_ERR_QUERY_SPLIT_KEY_ERR, -1605, "", true)                     \
-    M(OLAP_ERR_DATA_QUALITY_ERR, -1606, "", true)                        \
-    M(OLAP_ERR_COLUMN_DATA_LOAD_BLOCK, -1700, "", true)                  \
-    M(OLAP_ERR_COLUMN_DATA_RECORD_INDEX, -1701, "", true)                \
-    M(OLAP_ERR_COLUMN_DATA_MAKE_FILE_HEADER, -1702, "", true)            \
-    M(OLAP_ERR_COLUMN_DATA_READ_VAR_INT, -1703, "", true)                \
-    M(OLAP_ERR_COLUMN_DATA_PATCH_LIST_NUM, -1704, "", true)              \
-    M(OLAP_ERR_COLUMN_STREAM_EOF, -1705, "", false)                      \
-    M(OLAP_ERR_COLUMN_READ_STREAM, -1706, "", true)                      \
-    M(OLAP_ERR_COLUMN_STREAM_NOT_EXIST, -1716, "", true)                 \
-    M(OLAP_ERR_COLUMN_VALUE_NULL, -1717, "", true)                       \
-    M(OLAP_ERR_COLUMN_SEEK_ERROR, -1719, "", true)                       \
-    M(OLAP_ERR_DELETE_INVALID_CONDITION, -1900, "", true)                \
-    M(OLAP_ERR_DELETE_UPDATE_HEADER_FAILED, -1901, "", true)             \
-    M(OLAP_ERR_DELETE_SAVE_HEADER_FAILED, -1902, "", true)               \
-    M(OLAP_ERR_DELETE_INVALID_PARAMETERS, -1903, "", true)               \
-    M(OLAP_ERR_DELETE_INVALID_VERSION, -1904, "", true)                  \
-    M(OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION, -2000, "", false)         \
-    M(OLAP_ERR_CUMULATIVE_REPEAT_INIT, -2001, "", true)                  \
-    M(OLAP_ERR_CUMULATIVE_INVALID_PARAMETERS, -2002, "", true)           \
-    M(OLAP_ERR_CUMULATIVE_FAILED_ACQUIRE_DATA_SOURCE, -2003, "", true)   \
-    M(OLAP_ERR_CUMULATIVE_INVALID_NEED_MERGED_VERSIONS, -2004, "", true) \
-    M(OLAP_ERR_CUMULATIVE_ERROR_DELETE_ACTION, -2005, "", true)          \
-    M(OLAP_ERR_CUMULATIVE_MISS_VERSION, -2006, "", true)                 \
-    M(OLAP_ERR_CUMULATIVE_CLONE_OCCURRED, -2007, "", true)               \
-    M(OLAP_ERR_META_INVALID_ARGUMENT, -3000, "", true)                   \
-    M(OLAP_ERR_META_OPEN_DB, -3001, "", true)                            \
-    M(OLAP_ERR_META_KEY_NOT_FOUND, -3002, "", true)                      \
-    M(OLAP_ERR_META_GET, -3003, "", true)                                \
-    M(OLAP_ERR_META_PUT, -3004, "", true)                                \
-    M(OLAP_ERR_META_ITERATOR, -3005, "", true)                           \
-    M(OLAP_ERR_META_DELETE, -3006, "", true)                             \
-    M(OLAP_ERR_META_ALREADY_EXIST, -3007, "", true)                      \
-    M(OLAP_ERR_ROWSET_WRITER_INIT, -3100, "", true)                      \
-    M(OLAP_ERR_ROWSET_SAVE_FAILED, -3101, "", true)                      \
-    M(OLAP_ERR_ROWSET_GENERATE_ID_FAILED, -3102, "", true)               \
-    M(OLAP_ERR_ROWSET_DELETE_FILE_FAILED, -3103, "", true)               \
-    M(OLAP_ERR_ROWSET_BUILDER_INIT, -3104, "", true)                     \
-    M(OLAP_ERR_ROWSET_TYPE_NOT_FOUND, -3105, "", true)                   \
-    M(OLAP_ERR_ROWSET_ALREADY_EXIST, -3106, "", true)                    \
-    M(OLAP_ERR_ROWSET_CREATE_READER, -3107, "", true)                    \
-    M(OLAP_ERR_ROWSET_INVALID, -3108, "", true)                          \
-    M(OLAP_ERR_ROWSET_LOAD_FAILED, -3109, "", true)                      \
-    M(OLAP_ERR_ROWSET_READER_INIT, -3110, "", true)                      \
-    M(OLAP_ERR_ROWSET_READ_FAILED, -3111, "", true)                      \
-    M(OLAP_ERR_ROWSET_INVALID_STATE_TRANSITION, -3112, "", true)         \
-    M(OLAP_ERR_STRING_OVERFLOW_IN_VEC_ENGINE, -3113, "", true)           \
-    M(OLAP_ERR_ROWSET_ADD_MIGRATION_V2, -3114, "", true)                 \
-    M(OLAP_ERR_PUBLISH_VERSION_NOT_CONTINUOUS, -3115, "", false)         \
-    M(OLAP_ERR_ROWSET_RENAME_FILE_FAILED, -3116, "", false)              \
-    M(OLAP_ERR_SEGCOMPACTION_INIT_READER, -3117, "", false)              \
-    M(OLAP_ERR_SEGCOMPACTION_INIT_WRITER, -3118, "", false)              \
-    M(OLAP_ERR_SEGCOMPACTION_FAILED, -3119, "", false)
-
-enum ErrorCode {
-#define M(NAME, ERRORCODE, DESC, STACKTRACEENABLED) NAME = ERRORCODE,
-    APPLY_FOR_ERROR_CODES(M)
-#undef M
-};
+namespace ErrorCode {
+#define E(name, code) static constexpr int name = code
+E(OK, 0);
+#define TStatusError(name) E(name, TStatusCode::name)
+// Errors defined in TStatus
+TStatusError(PUBLISH_TIMEOUT);
+TStatusError(MEM_ALLOC_FAILED);
+TStatusError(BUFFER_ALLOCATION_FAILED);
+TStatusError(INVALID_ARGUMENT);
+TStatusError(MINIMUM_RESERVATION_UNAVAILABLE);
+TStatusError(CORRUPTION);
+TStatusError(IO_ERROR);
+TStatusError(NOT_FOUND);
+TStatusError(ALREADY_EXIST);
+TStatusError(NOT_IMPLEMENTED_ERROR);
+TStatusError(END_OF_FILE);
+TStatusError(INTERNAL_ERROR);
+TStatusError(RUNTIME_ERROR);
+TStatusError(CANCELLED);
+TStatusError(MEM_LIMIT_EXCEEDED);
+TStatusError(THRIFT_RPC_ERROR);
+TStatusError(TIMEOUT);
+TStatusError(TOO_MANY_TASKS);
+TStatusError(SERVICE_UNAVAILABLE);
+TStatusError(UNINITIALIZED);
+TStatusError(ABORTED);
+TStatusError(DATA_QUALITY_ERROR);
+TStatusError(LABEL_ALREADY_EXISTS);
+#undef TStatusError
+// BE internal errors
+E(OS_ERROR, -100);
+E(DIR_NOT_EXIST, -101);
+E(FILE_NOT_EXIST, -102);
+E(CREATE_FILE_ERROR, -103);
+E(STL_ERROR, -105);
+E(MUTEX_ERROR, -107);
+E(PTHREAD_ERROR, -108);
+E(NETWORK_ERROR, -109);
+E(UB_FUNC_ERROR, -110);
+E(COMPRESS_ERROR, -111);
+E(DECOMPRESS_ERROR, -112);
+E(UNKNOWN_COMPRESSION_TYPE, -113);
+E(MMAP_ERROR, -114);
+E(READ_UNENOUGH, -116);
+E(CANNOT_CREATE_DIR, -117);
+E(UB_NETWORK_ERROR, -118);
+E(FILE_FORMAT_ERROR, -119);
+E(EVAL_CONJUNCTS_ERROR, -120);
+E(COPY_FILE_ERROR, -121);
+E(FILE_ALREADY_EXIST, -122);
+E(CALL_SEQUENCE_ERROR, -202);
+E(BUFFER_OVERFLOW, -204);
+E(CONFIG_ERROR, -205);
+E(INIT_FAILED, -206);
+E(INVALID_SCHEMA, -207);
+E(CHECKSUM_ERROR, -208);
+E(SIGNATURE_ERROR, -209);
+E(CATCH_EXCEPTION, -210);
+E(PARSE_PROTOBUF_ERROR, -211);
+E(SERIALIZE_PROTOBUF_ERROR, -212);
+E(WRITE_PROTOBUF_ERROR, -213);
+E(VERSION_NOT_EXIST, -214);
+E(TABLE_NOT_FOUND, -215);
+E(TRY_LOCK_FAILED, -216);
+E(OUT_OF_BOUND, -218);
+E(FILE_DATA_ERROR, -220);
+E(TEST_FILE_ERROR, -221);
+E(INVALID_ROOT_PATH, -222);
+E(NO_AVAILABLE_ROOT_PATH, -223);
+E(CHECK_LINES_ERROR, -224);
+E(INVALID_CLUSTER_INFO, -225);
+E(TRANSACTION_NOT_EXIST, -226);
+E(DISK_FAILURE, -227);
+E(TRANSACTION_ALREADY_COMMITTED, -228);
+E(TRANSACTION_ALREADY_VISIBLE, -229);
+E(VERSION_ALREADY_MERGED, -230);
+E(LZO_DISABLED, -231);
+E(DISK_REACH_CAPACITY_LIMIT, -232);
+E(TOO_MANY_TRANSACTIONS, -233);
+E(INVALID_SNAPSHOT_VERSION, -234);
+E(TOO_MANY_VERSION, -235);
+E(NOT_INITIALIZED, -236);
+E(ALREADY_CANCELLED, -237);
+E(TOO_MANY_SEGMENTS, -238);
+E(CE_CMD_PARAMS_ERROR, -300);
+E(CE_BUFFER_TOO_SMALL, -301);
+E(CE_CMD_NOT_VALID, -302);
+E(CE_LOAD_TABLE_ERROR, -303);
+E(CE_NOT_FINISHED, -304);
+E(CE_TABLET_ID_EXIST, -305);
+E(TABLE_VERSION_DUPLICATE_ERROR, -400);
+E(TABLE_VERSION_INDEX_MISMATCH_ERROR, -401);
+E(TABLE_INDEX_VALIDATE_ERROR, -402);
+E(TABLE_INDEX_FIND_ERROR, -403);
+E(TABLE_CREATE_FROM_HEADER_ERROR, -404);
+E(TABLE_CREATE_META_ERROR, -405);
+E(TABLE_ALREADY_DELETED_ERROR, -406);
+E(ENGINE_INSERT_EXISTS_TABLE, -500);
+E(ENGINE_DROP_NOEXISTS_TABLE, -501);
+E(ENGINE_LOAD_INDEX_TABLE_ERROR, -502);
+E(TABLE_INSERT_DUPLICATION_ERROR, -503);
+E(DELETE_VERSION_ERROR, -504);
+E(GC_SCAN_PATH_ERROR, -505);
+E(ENGINE_INSERT_OLD_TABLET, -506);
+E(FETCH_OTHER_ERROR, -600);
+E(FETCH_TABLE_NOT_EXIST, -601);
+E(FETCH_VERSION_ERROR, -602);
+E(FETCH_SCHEMA_ERROR, -603);
+E(FETCH_COMPRESSION_ERROR, -604);
+E(FETCH_CONTEXT_NOT_EXIST, -605);
+E(FETCH_GET_READER_PARAMS_ERR, -606);
+E(FETCH_SAVE_SESSION_ERR, -607);
+E(FETCH_MEMORY_EXCEEDED, -608);
+E(READER_IS_UNINITIALIZED, -700);
+E(READER_GET_ITERATOR_ERROR, -701);
+E(CAPTURE_ROWSET_READER_ERROR, -702);
+E(READER_READING_ERROR, -703);
+E(READER_INITIALIZE_ERROR, -704);
+E(BE_VERSION_NOT_MATCH, -800);
+E(BE_REPLACE_VERSIONS_ERROR, -801);
+E(BE_MERGE_ERROR, -802);
+E(CAPTURE_ROWSET_ERROR, -804);
+E(BE_SAVE_HEADER_ERROR, -805);
+E(BE_INIT_OLAP_DATA, -806);
+E(BE_TRY_OBTAIN_VERSION_LOCKS, -807);
+E(BE_NO_SUITABLE_VERSION, -808);
+E(BE_INVALID_NEED_MERGED_VERSIONS, -810);
+E(BE_ERROR_DELETE_ACTION, -811);
+E(BE_SEGMENTS_OVERLAPPING, -812);
+E(BE_CLONE_OCCURRED, -813);
+E(PUSH_INIT_ERROR, -900);
+E(PUSH_VERSION_INCORRECT, -902);
+E(PUSH_SCHEMA_MISMATCH, -903);
+E(PUSH_CHECKSUM_ERROR, -904);
+E(PUSH_ACQUIRE_DATASOURCE_ERROR, -905);
+E(PUSH_CREAT_CUMULATIVE_ERROR, -906);
+E(PUSH_BUILD_DELTA_ERROR, -907);
+E(PUSH_VERSION_ALREADY_EXIST, -908);
+E(PUSH_TABLE_NOT_EXIST, -909);
+E(PUSH_INPUT_DATA_ERROR, -910);
+E(PUSH_TRANSACTION_ALREADY_EXIST, -911);
+E(PUSH_BATCH_PROCESS_REMOVED, -912);
+E(PUSH_COMMIT_ROWSET, -913);
+E(PUSH_ROWSET_NOT_FOUND, -914);
+E(INDEX_LOAD_ERROR, -1000);
+E(INDEX_CHECKSUM_ERROR, -1002);
+E(INDEX_DELTA_PRUNING, -1003);
+E(DATA_ROW_BLOCK_ERROR, -1100);
+E(DATA_FILE_TYPE_ERROR, -1101);
+E(WRITER_INDEX_WRITE_ERROR, -1200);
+E(WRITER_DATA_WRITE_ERROR, -1201);
+E(WRITER_ROW_BLOCK_ERROR, -1202);
+E(WRITER_SEGMENT_NOT_FINALIZED, -1203);
+E(ROWBLOCK_DECOMPRESS_ERROR, -1300);
+E(ROWBLOCK_FIND_ROW_EXCEPTION, -1301);
+E(ROWBLOCK_READ_INFO_ERROR, -1302);
+E(HEADER_ADD_VERSION, -1400);
+E(HEADER_DELETE_VERSION, -1401);
+E(HEADER_ADD_PENDING_DELTA, -1402);
+E(HEADER_ADD_INCREMENTAL_VERSION, -1403);
+E(HEADER_INVALID_FLAG, -1404);
+E(HEADER_LOAD_INVALID_KEY, -1408);
+E(HEADER_LOAD_JSON_HEADER, -1410);
+E(HEADER_INIT_FAILED, -1411);
+E(HEADER_PB_PARSE_FAILED, -1412);
+E(HEADER_HAS_PENDING_DATA, -1413);
+E(SCHEMA_SCHEMA_INVALID, -1500);
+E(SCHEMA_SCHEMA_FIELD_INVALID, -1501);
+E(ALTER_MULTI_TABLE_ERR, -1600);
+E(ALTER_DELTA_DOES_NOT_EXISTS, -1601);
+E(ALTER_STATUS_ERR, -1602);
+E(PREVIOUS_SCHEMA_CHANGE_NOT_FINISHED, -1603);
+E(SCHEMA_CHANGE_INFO_INVALID, -1604);
+E(QUERY_SPLIT_KEY_ERR, -1605);
+E(DATA_QUALITY_ERR, -1606);
+E(COLUMN_DATA_LOAD_BLOCK, -1700);
+E(COLUMN_DATA_RECORD_INDEX, -1701);
+E(COLUMN_DATA_MAKE_FILE_HEADER, -1702);
+E(COLUMN_DATA_READ_VAR_INT, -1703);
+E(COLUMN_DATA_PATCH_LIST_NUM, -1704);
+E(COLUMN_READ_STREAM, -1706);
+E(COLUMN_STREAM_NOT_EXIST, -1716);
+E(COLUMN_VALUE_NULL, -1717);
+E(COLUMN_SEEK_ERROR, -1719);
+E(DELETE_INVALID_CONDITION, -1900);
+E(DELETE_UPDATE_HEADER_FAILED, -1901);
+E(DELETE_SAVE_HEADER_FAILED, -1902);
+E(DELETE_INVALID_PARAMETERS, -1903);
+E(DELETE_INVALID_VERSION, -1904);
+E(CUMULATIVE_NO_SUITABLE_VERSION, -2000);
+E(CUMULATIVE_REPEAT_INIT, -2001);
+E(CUMULATIVE_INVALID_PARAMETERS, -2002);
+E(CUMULATIVE_FAILED_ACQUIRE_DATA_SOURCE, -2003);
+E(CUMULATIVE_INVALID_NEED_MERGED_VERSIONS, -2004);
+E(CUMULATIVE_ERROR_DELETE_ACTION, -2005);
+E(CUMULATIVE_MISS_VERSION, -2006);
+E(CUMULATIVE_CLONE_OCCURRED, -2007);
+E(META_INVALID_ARGUMENT, -3000);
+E(META_OPEN_DB_ERROR, -3001);
+E(META_KEY_NOT_FOUND, -3002);
+E(META_GET_ERROR, -3003);
+E(META_PUT_ERROR, -3004);
+E(META_ITERATOR_ERROR, -3005);
+E(META_DELETE_ERROR, -3006);
+E(META_ALREADY_EXIST, -3007);
+E(ROWSET_WRITER_INIT, -3100);
+E(ROWSET_SAVE_FAILED, -3101);
+E(ROWSET_GENERATE_ID_FAILED, -3102);
+E(ROWSET_DELETE_FILE_FAILED, -3103);
+E(ROWSET_BUILDER_INIT, -3104);
+E(ROWSET_TYPE_NOT_FOUND, -3105);
+E(ROWSET_ALREADY_EXIST, -3106);
+E(ROWSET_CREATE_READER, -3107);
+E(ROWSET_INVALID, -3108);
+E(ROWSET_LOAD_FAILED, -3109);
+E(ROWSET_READER_INIT, -3110);
+E(ROWSET_READ_FAILED, -3111);
+E(ROWSET_INVALID_STATE_TRANSITION, -3112);
+E(STRING_OVERFLOW_IN_VEC_ENGINE, -3113);
+E(ROWSET_ADD_MIGRATION_V2, -3114);
+E(PUBLISH_VERSION_NOT_CONTINUOUS, -3115);
+E(ROWSET_RENAME_FILE_FAILED, -3116);
+E(SEGCOMPACTION_INIT_READER, -3117);
+E(SEGCOMPACTION_INIT_WRITER, -3118);
+E(SEGCOMPACTION_FAILED, -3119);
+#undef E
+}; // namespace ErrorCode
+
+// clang-format off
+// whether to capture stacktrace
+template <int code>
+static constexpr bool capture_stacktrace() {
+    return code != ErrorCode::OK
+        && code != ErrorCode::END_OF_FILE
+        && code != ErrorCode::MEM_LIMIT_EXCEEDED
+        && code != ErrorCode::TRY_LOCK_FAILED
+        && code != ErrorCode::TOO_MANY_SEGMENTS
+        && code != ErrorCode::TOO_MANY_VERSION
+        && code != ErrorCode::ALREADY_CANCELLED
+        && code != ErrorCode::PUSH_TRANSACTION_ALREADY_EXIST
+        && code != ErrorCode::BE_NO_SUITABLE_VERSION
+        && code != ErrorCode::CUMULATIVE_NO_SUITABLE_VERSION
+        && code != ErrorCode::PUBLISH_VERSION_NOT_CONTINUOUS
+        && code != ErrorCode::ROWSET_RENAME_FILE_FAILED
+        && code != ErrorCode::SEGCOMPACTION_INIT_READER
+        && code != ErrorCode::SEGCOMPACTION_INIT_WRITER
+        && code != ErrorCode::SEGCOMPACTION_FAILED;
+}
+// clang-format on
 
 class Status {
 public:
-    Status() : _code(TStatusCode::OK), _precise_code(0) {}
+    Status() : _code(ErrorCode::OK) {}
 
     // copy c'tor makes copy of error detail so Status can be returned by value
-    Status(const Status& rhs) = default;
+    Status(const Status& rhs) { *this = rhs; }
 
     // move c'tor
     Status(Status&& rhs) noexcept = default;
 
     // same as copy c'tor
-    Status& operator=(const Status& rhs) = default;
+    Status& operator=(const Status& rhs) {
+        _code = rhs._code;
+        if (rhs._err_msg) {
+            _err_msg = std::make_unique<ErrMsg>(*rhs._err_msg);
+        }
+        return *this;
+    }
 
     // move assign
     Status& operator=(Status&& rhs) noexcept = default;
@@ -264,183 +298,100 @@ public:
 
     Status(const PStatus& pstatus);
 
-    // Not allow user create status using constructors, could only use util methods
-private:
-    Status(TStatusCode::type code, std::string_view msg, int16_t precise_code = 1)
-            : _code(code), _precise_code(precise_code), _err_msg(msg) {}
-
-    Status(TStatusCode::type code, std::string&& msg, int16_t precise_code = 1)
-            : _code(code), _precise_code(precise_code), _err_msg(std::move(msg)) {}
-
-    template <typename... Args>
-    static Status ErrorFmt(TStatusCode::type code, std::string_view fmt, Args&&... args) {
-        // In some cases, fmt contains '{}' but there are no args.
+    template <int code, bool stacktrace = true, typename... Args>
+    Status static Error(std::string_view msg, Args&&... args) {
+        Status status;
+        status._code = code;
+        status._err_msg = std::make_unique<ErrMsg>();
         if constexpr (sizeof...(args) == 0) {
-            return Status(code, fmt);
+            status._err_msg->_msg = msg;
         } else {
-            return Status(code, fmt::format(fmt, std::forward<Args>(args)...));
+            status._err_msg->_msg = fmt::format(msg, std::forward<Args>(args)...);
         }
-    }
-
-    template <typename... Args>
-    static Status ErrorFmtWithStackTrace(TStatusCode::type code, std::string_view fmt,
-                                         Args&&... args) {
-        // In some cases, fmt contains '{}' but there are no args.
-        if constexpr (sizeof...(args) == 0) {
-            return ConstructErrorStatus(code, -1, fmt);
-        } else {
-            return ConstructErrorStatus(code, -1, fmt::format(fmt, std::forward<Args>(args)...));
+#ifdef ENABLE_STACKTRACE
+        if constexpr (stacktrace && capture_stacktrace<code>()) {
+            status._err_msg->_stack = get_stack_trace();
         }
+#endif
+        return status;
     }
 
-public:
-    static Status OK() { return Status(); }
-
-    template <typename... Args>
-    static Status PublishTimeout(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::PUBLISH_TIMEOUT, fmt, std::forward<Args>(args)...);
-    }
-
-    template <typename... Args>
-    static Status MemoryAllocFailed(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::MEM_ALLOC_FAILED, fmt, std::forward<Args>(args)...);
-    }
-
-    template <typename... Args>
-    static Status BufferAllocFailed(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::BUFFER_ALLOCATION_FAILED, fmt, std::forward<Args>(args)...);
-    }
-
-    template <typename... Args>
-    static Status InvalidArgument(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::INVALID_ARGUMENT, fmt, std::forward<Args>(args)...);
-    }
-    template <typename... Args>
-    static Status MinimumReservationUnavailable(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::MINIMUM_RESERVATION_UNAVAILABLE, fmt,
-                        std::forward<Args>(args)...);
-    }
-
-    template <typename... Args>
-    static Status Corruption(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::CORRUPTION, fmt, std::forward<Args>(args)...);
-    }
-
-    template <typename... Args>
-    static Status IOError(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::IO_ERROR, fmt, std::forward<Args>(args)...);
-    }
-
-    template <typename... Args>
-    static Status NotFound(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::NOT_FOUND, fmt, std::forward<Args>(args)...);
-    }
-    template <typename... Args>
-    static Status AlreadyExist(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::ALREADY_EXIST, fmt, std::forward<Args>(args)...);
-    }
-    template <typename... Args>
-    static Status NotSupported(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::NOT_IMPLEMENTED_ERROR, fmt, std::forward<Args>(args)...);
-    }
-    template <typename... Args>
-    static Status EndOfFile(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::END_OF_FILE, fmt, std::forward<Args>(args)...);
-    }
-
-    template <typename... Args>
-    static Status InternalError(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::INTERNAL_ERROR, fmt, std::forward<Args>(args)...);
-    }
-
-    template <typename... Args>
-    static Status RuntimeError(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::RUNTIME_ERROR, fmt, std::forward<Args>(args)...);
-    }
-    template <typename... Args>
-    static Status Cancelled(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::CANCELLED, fmt, std::forward<Args>(args)...);
-    }
-
-    template <typename... Args>
-    static Status MemoryLimitExceeded(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::MEM_LIMIT_EXCEEDED, fmt, std::forward<Args>(args)...);
-    }
-
-    template <typename... Args>
-    static Status RpcError(std::string_view fmt, Args&&... args) {
-        return ErrorFmtWithStackTrace(TStatusCode::THRIFT_RPC_ERROR, fmt,
-                                      std::forward<Args>(args)...);
-    }
-
-    template <typename... Args>
-    static Status TimedOut(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::TIMEOUT, fmt, std::forward<Args>(args)...);
+    template <int code, bool stacktrace = true>
+    Status static Error() {
+        Status status;
+        status._code = code;
+#ifdef ENABLE_STACKTRACE
+        if constexpr (stacktrace && capture_stacktrace<code>()) {
+            status._err_msg = std::make_unique<ErrMsg>();
+            status._err_msg->_stack = get_stack_trace();
+        }
+#endif
+        return status;
     }
 
     template <typename... Args>
-    static Status TooManyTasks(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::TOO_MANY_TASKS, fmt, std::forward<Args>(args)...);
+    Status static Error(int code, std::string_view msg, Args&&... args) {
+        Status status;
+        status._code = code;
+        status._err_msg = std::make_unique<ErrMsg>();
+        if constexpr (sizeof...(args) == 0) {
+            status._err_msg->_msg = msg;
+        } else {
+            status._err_msg->_msg = fmt::format(msg, std::forward<Args>(args)...);
+        }
+        return status;
     }
 
-    template <typename... Args>
-    static Status ServiceUnavailable(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::SERVICE_UNAVAILABLE, fmt, std::forward<Args>(args)...);
+    Status static Error(int code) {
+        Status status;
+        status._code = code;
+        return status;
     }
 
-    template <typename... Args>
-    static Status Uninitialized(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::UNINITIALIZED, fmt, std::forward<Args>(args)...);
-    }
+    static Status OK() { return Status(); }
 
-    template <typename... Args>
-    static Status Aborted(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::ABORTED, fmt, std::forward<Args>(args)...);
+#define ERROR_CTOR(name, code)                                                  \
+    template <typename... Args>                                                 \
+    static Status name(std::string_view msg, Args&&... args) {                  \
+        return Error<ErrorCode::code, false>(msg, std::forward<Args>(args)...); \
     }
-
-    template <typename... Args>
-    static Status DataQualityError(std::string_view fmt, Args&&... args) {
-        return ErrorFmt(TStatusCode::DATA_QUALITY_ERROR, fmt, std::forward<Args>(args)...);
+    ERROR_CTOR(PublishTimeout, PUBLISH_TIMEOUT)
+    ERROR_CTOR(MemoryAllocFailed, MEM_ALLOC_FAILED)
+    ERROR_CTOR(BufferAllocFailed, BUFFER_ALLOCATION_FAILED)
+    ERROR_CTOR(InvalidArgument, INVALID_ARGUMENT)
+    ERROR_CTOR(MinimumReservationUnavailable, MINIMUM_RESERVATION_UNAVAILABLE)
+    ERROR_CTOR(Corruption, CORRUPTION)
+    ERROR_CTOR(IOError, IO_ERROR)
+    ERROR_CTOR(NotFound, NOT_FOUND)
+    ERROR_CTOR(AlreadyExist, ALREADY_EXIST)
+    ERROR_CTOR(NotSupported, NOT_IMPLEMENTED_ERROR)
+    ERROR_CTOR(EndOfFile, END_OF_FILE)
+    ERROR_CTOR(InternalError, INTERNAL_ERROR)
+    ERROR_CTOR(RuntimeError, RUNTIME_ERROR)
+    ERROR_CTOR(Cancelled, CANCELLED)
+    ERROR_CTOR(MemoryLimitExceeded, MEM_LIMIT_EXCEEDED)
+    ERROR_CTOR(RpcError, THRIFT_RPC_ERROR)
+    ERROR_CTOR(TimedOut, TIMEOUT)
+    ERROR_CTOR(TooManyTasks, TOO_MANY_TASKS)
+    ERROR_CTOR(ServiceUnavailable, SERVICE_UNAVAILABLE)
+    ERROR_CTOR(Uninitialized, UNINITIALIZED)
+    ERROR_CTOR(Aborted, ABORTED)
+    ERROR_CTOR(DataQualityError, DATA_QUALITY_ERROR)
+#undef ERROR_CTOR
+
+    template <int code>
+    bool is() const {
+        return code == _code;
     }
 
-    // A wrapper for ErrorCode
-    //      Precise code is for ErrorCode's enum value
-    //      All Status Error is treated as Internal Error
-    static Status OLAPInternalError(int16_t precise_code, std::string_view msg = "");
-
-    static Status ConstructErrorStatus(TStatusCode::type tcode, int16_t precise_code,
-                                       std::string_view msg);
-
-    bool ok() const { return _code == TStatusCode::OK; }
+    bool ok() const { return _code == ErrorCode::OK; }
 
-    bool is_cancelled() const { return code() == TStatusCode::CANCELLED; }
-    bool is_mem_limit_exceeded() const { return code() == TStatusCode::MEM_LIMIT_EXCEEDED; }
-    bool is_end_of_file() const { return code() == TStatusCode::END_OF_FILE; }
-    bool is_not_found() const { return code() == TStatusCode::NOT_FOUND; }
-    bool is_already_exist() const { return code() == TStatusCode::ALREADY_EXIST; }
     bool is_io_error() const {
-        auto p_code = precise_code();
-        return code() == TStatusCode::IO_ERROR ||
-               ((OLAP_ERR_IO_ERROR == p_code || OLAP_ERR_READ_UNENOUGH == p_code) &&
-                errno == EIO) ||
-               OLAP_ERR_CHECKSUM_ERROR == p_code || OLAP_ERR_FILE_DATA_ERROR == p_code ||
-               OLAP_ERR_TEST_FILE_ERROR == p_code || OLAP_ERR_ROWBLOCK_READ_INFO_ERROR == p_code;
+        return ErrorCode::IO_ERROR == _code || ErrorCode::READ_UNENOUGH == _code ||
+               ErrorCode::CHECKSUM_ERROR == _code || ErrorCode::FILE_DATA_ERROR == _code ||
+               ErrorCode::TEST_FILE_ERROR == _code || ErrorCode::ROWBLOCK_READ_INFO_ERROR == _code;
     }
 
-    /// @return @c true if the status indicates Uninitialized.
-    bool is_uninitialized() const { return code() == TStatusCode::UNINITIALIZED; }
-
-    // @return @c true if the status indicates an Aborted error.
-    bool is_aborted() const { return code() == TStatusCode::ABORTED; }
-
-    /// @return @c true if the status indicates an InvalidArgument error.
-    bool is_invalid_argument() const { return code() == TStatusCode::INVALID_ARGUMENT; }
-
-    // @return @c true if the status indicates ServiceUnavailable.
-    bool is_service_unavailable() const { return code() == TStatusCode::SERVICE_UNAVAILABLE; }
-
-    bool is_data_quality_error() const { return code() == TStatusCode::DATA_QUALITY_ERROR; }
-
     // Convert into TStatus. Call this if 'status_container' contains an optional
     // TStatus field named 'status'. This also sets __isset.status.
     template <typename T>
@@ -454,24 +405,17 @@ public:
     TStatus to_thrift() const;
     void to_protobuf(PStatus* status) const;
 
-    const std::string& get_error_msg() const { return _err_msg; }
+    std::string code_as_string() const {
+        return (int)_code >= 0 ? doris::to_string(static_cast<TStatusCode::type>(_code))
+                               : fmt::format("E{}", (int16_t)_code);
+    }
 
-    /// @return A string representation of this status suitable for printing.
-    ///   Returns the string "OK" for success.
     std::string to_string() const;
 
     /// @return A json representation of this status.
     std::string to_json() const;
 
-    /// @return A string representation of the status code, without the message
-    ///   text or sub code information.
-    const char* code_as_string() const;
-
-    TStatusCode::type code() const {
-        return ok() ? TStatusCode::OK : static_cast<TStatusCode::type>(_code);
-    }
-
-    int16_t precise_code() const { return ok() ? 0 : _precise_code; }
+    int code() const { return _code; }
 
     /// Clone this status and add the specified prefix to the message.
     ///
@@ -497,25 +441,38 @@ public:
     // Used like if (res == Status::OK())
     // if the state is ok, then both code and precise code is not initialized properly, so that should check ok state
     // ignore error messages during comparison
-    bool operator==(const Status& st) const {
-        return ok() ? st.ok() : code() == st.code() && precise_code() == st.precise_code();
-    }
+    bool operator==(const Status& st) const { return _code == st._code; }
 
     // Used like if (res != Status::OK())
-    bool operator!=(const Status& st) const {
-        return ok() ? !st.ok() : code() != st.code() || precise_code() != st.precise_code();
-    }
+    bool operator!=(const Status& st) const { return _code != st._code; }
+
+    friend std::ostream& operator<<(std::ostream& ostr, const Status& status);
 
 private:
-    TStatusCode::type _code;
-    int16_t _precise_code;
-    std::string _err_msg;
+    int _code;
+    struct ErrMsg {
+        std::string _msg;
+#ifdef ENABLE_STACKTRACE
+        std::string _stack;
+#endif
+    };
+    std::unique_ptr<ErrMsg> _err_msg;
 };
 
-// Override the << operator, it is used during LOG(INFO) << "xxxx" << status;
-// Add inline here to dedup many includes
-inline std::ostream& operator<<(std::ostream& ostr, const Status& param) {
-    return ostr << param.to_string();
+inline std::ostream& operator<<(std::ostream& ostr, const Status& status) {
+    ostr << '[' << status.code_as_string() << ']' << (status._err_msg ? status._err_msg->_msg : "");
+#ifdef ENABLE_STACKTRACE
+    if (status->_err_msg && !status->_err_msg._stack.empty()) {
+        ostr << '\n' << status->_err_msg._stack;
+    }
+#endif
+    return ostr;
+}
+
+inline std::string Status::to_string() const {
+    std::stringstream ss;
+    ss << *this;
+    return ss.str();
 }
 
 // some generally useful macros
@@ -548,40 +505,40 @@ inline std::ostream& operator<<(std::ostream& ostr, const Status& param) {
         }                                    \
     } while (false)
 
-#define EXIT_IF_ERROR(stmt)                         \
-    do {                                            \
-        Status _status_ = (stmt);                   \
-        if (UNLIKELY(!_status_.ok())) {             \
-            LOG(ERROR) << _status_.get_error_msg(); \
-            exit(1);                                \
-        }                                           \
+#define EXIT_IF_ERROR(stmt)             \
+    do {                                \
+        Status _status_ = (stmt);       \
+        if (UNLIKELY(!_status_.ok())) { \
+            LOG(ERROR) << _status_;     \
+            exit(1);                    \
+        }                               \
     } while (false)
 
 /// @brief Emit a warning if @c to_call returns a bad status.
-#define WARN_IF_ERROR(to_call, warning_prefix)                          \
-    do {                                                                \
-        Status _s = (to_call);                                          \
-        if (UNLIKELY(!_s.ok())) {                                       \
-            LOG(WARNING) << (warning_prefix) << ": " << _s.to_string(); \
-        }                                                               \
+#define WARN_IF_ERROR(to_call, warning_prefix)              \
+    do {                                                    \
+        Status _s = (to_call);                              \
+        if (UNLIKELY(!_s.ok())) {                           \
+            LOG(WARNING) << (warning_prefix) << ": " << _s; \
+        }                                                   \
     } while (false);
 
-#define RETURN_WITH_WARN_IF_ERROR(stmt, ret_code, warning_prefix)              \
-    do {                                                                       \
-        Status _s = (stmt);                                                    \
-        if (UNLIKELY(!_s.ok())) {                                              \
-            LOG(WARNING) << (warning_prefix) << ", error: " << _s.to_string(); \
-            return ret_code;                                                   \
-        }                                                                      \
+#define RETURN_WITH_WARN_IF_ERROR(stmt, ret_code, warning_prefix)  \
+    do {                                                           \
+        Status _s = (stmt);                                        \
+        if (UNLIKELY(!_s.ok())) {                                  \
+            LOG(WARNING) << (warning_prefix) << ", error: " << _s; \
+            return ret_code;                                       \
+        }                                                          \
     } while (false);
 
-#define RETURN_NOT_OK_STATUS_WITH_WARN(stmt, warning_prefix)                   \
-    do {                                                                       \
-        Status _s = (stmt);                                                    \
-        if (UNLIKELY(!_s.ok())) {                                              \
-            LOG(WARNING) << (warning_prefix) << ", error: " << _s.to_string(); \
-            return _s;                                                         \
-        }                                                                      \
+#define RETURN_NOT_OK_STATUS_WITH_WARN(stmt, warning_prefix)       \
+    do {                                                           \
+        Status _s = (stmt);                                        \
+        if (UNLIKELY(!_s.ok())) {                                  \
+            LOG(WARNING) << (warning_prefix) << ", error: " << _s; \
+            return _s;                                             \
+        }                                                          \
     } while (false);
 } // namespace doris
 #ifdef WARN_UNUSED_RESULT
diff --git a/be/src/env/env_posix.cpp b/be/src/env/env_posix.cpp
index 9df22e2d45..3407f223e5 100644
--- a/be/src/env/env_posix.cpp
+++ b/be/src/env/env_posix.cpp
@@ -38,6 +38,7 @@
 #include "util/slice.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 using std::string;
 using strings::Substitute;
@@ -352,7 +353,7 @@ public:
         if (_sync_on_close) {
             Status sync_status = sync();
             if (!sync_status.ok()) {
-                LOG(ERROR) << "Unable to Sync " << _filename << ": " << sync_status.to_string();
+                LOG(ERROR) << "Unable to Sync " << _filename << ": " << sync_status;
                 if (s.ok()) {
                     s = sync_status;
                 }
@@ -461,7 +462,7 @@ public:
         if (_sync_on_close) {
             s = sync();
             if (!s.ok()) {
-                LOG(ERROR) << "Unable to Sync " << _filename << ": " << s.to_string();
+                LOG(ERROR) << "Unable to Sync " << _filename << ": " << s;
             }
         }
 
@@ -600,7 +601,7 @@ Status PosixEnv::create_dir_if_missing(const string& dirname, bool* created) {
     }
 
     // Check that dirname is actually a directory.
-    if (s.is_already_exist()) {
+    if (s.is<ALREADY_EXIST>()) {
         bool is_dir = false;
         RETURN_IF_ERROR(is_directory(dirname, &is_dir));
         if (is_dir) {
diff --git a/be/src/exec/broker_scan_node.cpp b/be/src/exec/broker_scan_node.cpp
index 5448e059e1..580a126279 100644
--- a/be/src/exec/broker_scan_node.cpp
+++ b/be/src/exec/broker_scan_node.cpp
@@ -390,8 +390,7 @@ void BrokerScanNode::scanner_worker(int start_idx, int length) {
                 _scan_ranges[start_idx + i].scan_range.broker_scan_range;
         status = scanner_scan(scan_range, scanner_expr_ctxs, &counter);
         if (!status.ok()) {
-            LOG(WARNING) << "Scanner[" << start_idx + i
-                         << "] process failed. status=" << status.get_error_msg();
+            LOG(WARNING) << "Scanner[" << start_idx + i << "] process failed. status=" << status;
         }
     }
 
diff --git a/be/src/exec/csv_scanner.cpp b/be/src/exec/csv_scanner.cpp
index 59c480d650..bdba765c83 100644
--- a/be/src/exec/csv_scanner.cpp
+++ b/be/src/exec/csv_scanner.cpp
@@ -19,6 +19,8 @@
 
 #include <boost/algorithm/string.hpp>
 
+#include "common/logging.h"
+
 namespace doris {
 CsvScanner::CsvScanner(const std::vector<std::string>& csv_file_paths)
         : _is_open(false),
diff --git a/be/src/exec/decompressor.cpp b/be/src/exec/decompressor.cpp
index 5d7dee008e..881c46c98b 100644
--- a/be/src/exec/decompressor.cpp
+++ b/be/src/exec/decompressor.cpp
@@ -17,6 +17,8 @@
 
 #include "exec/decompressor.h"
 
+#include "common/logging.h"
+
 namespace doris {
 
 Status Decompressor::create_decompressor(CompressType type, Decompressor** decompressor) {
diff --git a/be/src/exec/es/es_scan_reader.cpp b/be/src/exec/es/es_scan_reader.cpp
index 8470350dd0..e262020a51 100644
--- a/be/src/exec/es/es_scan_reader.cpp
+++ b/be/src/exec/es/es_scan_reader.cpp
@@ -127,7 +127,7 @@ Status ESScanReader::open() {
     Status status = _network_client.execute_post_request(_query, &_cached_response);
     if (!status.ok() || _network_client.get_http_status() != 200) {
         std::stringstream ss;
-        ss << "Failed to connect to ES server, errmsg is: " << status.get_error_msg();
+        ss << "Failed to connect to ES server, errmsg is: " << status;
         LOG(WARNING) << ss.str();
         return Status::InternalError(ss.str());
     }
@@ -180,7 +180,7 @@ Status ESScanReader::get_next(bool* scan_eos, std::unique_ptr<ScrollParser>& scr
     Status status = scroll_parser->parse(response, _exactly_once);
     if (!status.ok()) {
         _eos = true;
-        LOG(WARNING) << status.get_error_msg();
+        LOG(WARNING) << status;
         return status;
     }
 
diff --git a/be/src/exec/es_http_scan_node.cpp b/be/src/exec/es_http_scan_node.cpp
index 73ee88b018..478f85b1a7 100644
--- a/be/src/exec/es_http_scan_node.cpp
+++ b/be/src/exec/es_http_scan_node.cpp
@@ -107,10 +107,10 @@ Status EsHttpScanNode::build_conjuncts_list() {
         } else {
             _conjunct_to_predicate[i] = -1;
 
-            VLOG_CRITICAL << status.get_error_msg();
+            VLOG_CRITICAL << status;
             status = predicate->get_es_query_status();
             if (!status.ok()) {
-                LOG(WARNING) << status.get_error_msg();
+                LOG(WARNING) << status;
                 return status;
             }
         }
@@ -461,8 +461,7 @@ void EsHttpScanNode::scanner_worker(int start_idx, int length, std::promise<Stat
                               scanner_expr_ctxs, &counter, doc_value_mode));
     status = scanner_scan(std::move(scanner), scanner_expr_ctxs, &counter);
     if (!status.ok()) {
-        LOG(WARNING) << "Scanner[" << start_idx
-                     << "] process failed. status=" << status.get_error_msg();
+        LOG(WARNING) << "Scanner[" << start_idx << "] process failed. status=" << status;
     }
 
     // scanner is going to finish
diff --git a/be/src/exec/exchange_node.cpp b/be/src/exec/exchange_node.cpp
index f9b9caa6fb..5bde6bc324 100644
--- a/be/src/exec/exchange_node.cpp
+++ b/be/src/exec/exchange_node.cpp
@@ -29,6 +29,7 @@
 #include "util/runtime_profile.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 ExchangeNode::ExchangeNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)
         : ExecNode(pool, tnode, descs),
@@ -125,7 +126,7 @@ Status ExchangeNode::fill_input_row_batch(RuntimeState* state) {
     }
     VLOG_FILE << "exch: has batch=" << (_input_batch == nullptr ? "false" : "true")
               << " #rows=" << (_input_batch != nullptr ? _input_batch->num_rows() : 0)
-              << " is_cancelled=" << (ret_status.is_cancelled() ? "true" : "false")
+              << " is_cancelled=" << (ret_status.is<CANCELLED>() ? "true" : "false")
               << " instance_id=" << state->fragment_instance_id();
     return ret_status;
 }
diff --git a/be/src/exec/hash_table.cpp b/be/src/exec/hash_table.cpp
index 820e15fba2..4bb795954a 100644
--- a/be/src/exec/hash_table.cpp
+++ b/be/src/exec/hash_table.cpp
@@ -177,7 +177,7 @@ Status HashTable::resize_buckets(int64_t num_buckets) {
     int64_t delta_bytes = (num_buckets - old_num_buckets) * sizeof(Bucket);
     Status st = thread_context()->thread_mem_tracker()->check_limit(delta_bytes);
     if (!st) {
-        LOG_EVERY_N(WARNING, 100) << "resize bucket failed: " << st.to_string();
+        LOG_EVERY_N(WARNING, 100) << "resize bucket failed: " << st;
         return st;
     }
     _mem_tracker->consume(delta_bytes);
diff --git a/be/src/exec/json_scanner.cpp b/be/src/exec/json_scanner.cpp
index 646fedef6a..5ee5bac8f2 100644
--- a/be/src/exec/json_scanner.cpp
+++ b/be/src/exec/json_scanner.cpp
@@ -28,6 +28,7 @@
 #include "runtime/runtime_state.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 JsonScanner::JsonScanner(RuntimeState* state, RuntimeProfile* profile,
                          const TBrokerScanRangeParams& params,
@@ -623,7 +624,7 @@ Status JsonReader::_handle_simple_json(Tuple* tuple, const std::vector<SlotDescr
         if (_next_line >= _total_lines) { // parse json and generic document
             size_t size = 0;
             Status st = _parse_json_doc(&size, eof);
-            if (st.is_data_quality_error()) {
+            if (st.is<DATA_QUALITY_ERROR>()) {
                 continue; // continue to read next
             }
             RETURN_IF_ERROR(st);     // terminate if encounter other errors
@@ -765,7 +766,7 @@ Status JsonReader::_handle_nested_complex_json(Tuple* tuple,
     while (true) {
         size_t size = 0;
         Status st = _parse_json_doc(&size, eof);
-        if (st.is_data_quality_error()) {
+        if (st.is<DATA_QUALITY_ERROR>()) {
             continue; // continue to read next
         }
         RETURN_IF_ERROR(st);
@@ -804,7 +805,7 @@ Status JsonReader::_handle_flat_array_complex_json(Tuple* tuple,
         if (_next_line >= _total_lines) {
             size_t size = 0;
             Status st = _parse_json_doc(&size, eof);
-            if (st.is_data_quality_error()) {
+            if (st.is<DATA_QUALITY_ERROR>()) {
                 continue; // continue to read next
             }
             RETURN_IF_ERROR(st);     // terminate if encounter other errors
diff --git a/be/src/exec/olap_scan_node.cpp b/be/src/exec/olap_scan_node.cpp
index e1b62781c8..8abca00c84 100644
--- a/be/src/exec/olap_scan_node.cpp
+++ b/be/src/exec/olap_scan_node.cpp
@@ -293,7 +293,7 @@ Status OlapScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eo
         Status status = start_scan(state);
 
         if (!status.ok()) {
-            LOG(ERROR) << "StartScan Failed cause " << status.get_error_msg();
+            LOG(ERROR) << "StartScan Failed cause " << status;
             *eos = true;
             return status;
         }
@@ -1577,8 +1577,7 @@ void OlapScanNode::transfer_thread(RuntimeState* state) {
                     COUNTER_UPDATE(_scanner_sched_counter, 1);
                     olap_scanners.erase(iter++);
                 } else {
-                    LOG(FATAL) << "Failed to assign scanner task to thread pool! "
-                               << s.get_error_msg();
+                    LOG(FATAL) << "Failed to assign scanner task to thread pool! " << s;
                 }
                 ++_total_assign_num;
             }
@@ -1751,7 +1750,7 @@ void OlapScanNode::scanner_thread(OlapScanner* scanner) {
         row_batch->set_scanner_id(scanner->id());
         status = scanner->get_batch(_runtime_state, row_batch, &eos);
         if (!status.ok()) {
-            LOG(WARNING) << "Scan thread read OlapScanner failed: " << status.to_string();
+            LOG(WARNING) << "Scan thread read OlapScanner failed: " << status;
             eos = true;
             break;
         }
diff --git a/be/src/exec/olap_scanner.cpp b/be/src/exec/olap_scanner.cpp
index 06fb0abc60..72769f7769 100644
--- a/be/src/exec/olap_scanner.cpp
+++ b/be/src/exec/olap_scanner.cpp
@@ -371,8 +371,8 @@ Status OlapScanner::get_batch(RuntimeState* state, RowBatch* batch, bool* eof) {
                                                                  &tmp_object_pool, eof);
             if (!res.ok()) {
                 return Status::InternalError(
-                        "Internal Error: read storage fail. res={}, tablet={}, backend={}", res,
-                        _tablet->full_name(), BackendOptions::get_localhost());
+                        "Internal Error: read storage fail. res={}, tablet={}, backend={}",
+                        res.to_string(), _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 e6da51f71d..074f7d35a7 100644
--- a/be/src/exec/parquet_scanner.cpp
+++ b/be/src/exec/parquet_scanner.cpp
@@ -24,6 +24,7 @@
 #include "runtime/stream_load/stream_load_pipe.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 ParquetScanner::ParquetScanner(RuntimeState* state, RuntimeProfile* profile,
                                const TBrokerScanRangeParams& params,
@@ -110,12 +111,11 @@ Status ParquetScanner::open_next_reader() {
         auto tuple_desc = _state->desc_tbl().get_tuple_descriptor(_tupleId);
         Status status =
                 _cur_file_reader->init_reader(tuple_desc, _conjunct_ctxs, _state->timezone());
-        if (status.is_end_of_file()) {
+        if (status.is<END_OF_FILE>()) {
             continue;
         } else {
             if (!status.ok()) {
-                return Status::InternalError("file: {}, error:{}", range.path,
-                                             status.get_error_msg());
+                return Status::InternalError("file: {}, error:{}", range.path, status.to_string());
             } else {
                 RETURN_IF_ERROR(_cur_file_reader->init_parquet_type());
                 return status;
diff --git a/be/src/exec/parquet_writer.cpp b/be/src/exec/parquet_writer.cpp
index b0d3fdf2ff..b21a0b869f 100644
--- a/be/src/exec/parquet_writer.cpp
+++ b/be/src/exec/parquet_writer.cpp
@@ -166,7 +166,7 @@ arrow::Status ParquetOutputStream::Write(const void* data, int64_t nbytes) {
     size_t written_len = 0;
     Status st = _file_writer->write(static_cast<const uint8_t*>(data), nbytes, &written_len);
     if (!st.ok()) {
-        return arrow::Status::IOError(st.get_error_msg());
+        return arrow::Status::IOError(st.to_string());
     }
     _cur_pos += written_len;
     _written_len += written_len;
@@ -183,8 +183,8 @@ arrow::Status ParquetOutputStream::Close() {
     }
     Status st = _file_writer->close();
     if (!st.ok()) {
-        LOG(WARNING) << "close parquet output stream failed: " << st.get_error_msg();
-        return arrow::Status::IOError(st.get_error_msg());
+        LOG(WARNING) << "close parquet output stream failed: " << st;
+        return arrow::Status::IOError(st.to_string());
     }
     _is_closed = true;
     return arrow::Status::OK();
diff --git a/be/src/exec/schema_scanner/schema_backends_scanner.cpp b/be/src/exec/schema_scanner/schema_backends_scanner.cpp
index ec29eb60a6..50684ebbcb 100644
--- a/be/src/exec/schema_scanner/schema_backends_scanner.cpp
+++ b/be/src/exec/schema_scanner/schema_backends_scanner.cpp
@@ -122,8 +122,7 @@ Status SchemaBackendsScanner::_fetch_backends_info() {
 
     Status status(result.status);
     if (!status.ok()) {
-        LOG(WARNING) << "fetch schema table data from master failed, errmsg="
-                     << status.get_error_msg();
+        LOG(WARNING) << "fetch schema table data from master failed, errmsg=" << status;
         return status;
     }
     _batch_data = std::move(result.data_batch);
diff --git a/be/src/exec/tablet_sink.cpp b/be/src/exec/tablet_sink.cpp
index 7c0051c191..6e835228da 100644
--- a/be/src/exec/tablet_sink.cpp
+++ b/be/src/exec/tablet_sink.cpp
@@ -211,7 +211,7 @@ Status NodeChannel::open_wait() {
                                            -1);
             Status st = _index_channel->check_intolerable_failure();
             if (!st.ok()) {
-                _cancel_with_msg(fmt::format("{}, err: {}", channel_info(), st.get_error_msg()));
+                _cancel_with_msg(fmt::format("{}, err: {}", channel_info(), st.to_string()));
             } else if (is_last_rpc) {
                 // if this is last rpc, will must set _add_batches_finished. otherwise, node channel's close_wait
                 // will be blocked.
@@ -240,7 +240,7 @@ Status NodeChannel::open_wait() {
 
                 Status st = _index_channel->check_intolerable_failure();
                 if (!st.ok()) {
-                    _cancel_with_msg(st.get_error_msg());
+                    _cancel_with_msg(st.to_string());
                 } else if (is_last_rpc) {
                     for (auto& tablet : result.tablet_vec()) {
                         TTabletCommitInfo commit_info;
@@ -279,7 +279,7 @@ Status NodeChannel::open_wait() {
             } else {
                 _cancel_with_msg(
                         fmt::format("{}, add batch req success but status isn't ok, err: {}",
-                                    channel_info(), status.get_error_msg()));
+                                    channel_info(), status.to_string()));
             }
 
             if (result.has_execution_time_us()) {
@@ -556,7 +556,7 @@ void NodeChannel::try_send_batch(RuntimeState* state) {
         Status st = row_batch->serialize(request.mutable_row_batch(), &uncompressed_bytes,
                                          &compressed_bytes, _parent->_transfer_large_data_by_brpc);
         if (!st.ok()) {
-            cancel(fmt::format("{}, err: {}", channel_info(), st.get_error_msg()));
+            cancel(fmt::format("{}, err: {}", channel_info(), st.to_string()));
             _add_batch_closure->clear_in_flight();
             return;
         }
@@ -626,7 +626,7 @@ void NodeChannel::try_send_batch(RuntimeState* state) {
                 PTabletWriterAddBatchRequest, ReusableClosure<PTabletWriterAddBatchResult>>(
                 &request, _add_batch_closure);
         if (!st.ok()) {
-            cancel(fmt::format("{}, err: {}", channel_info(), st.get_error_msg()));
+            cancel(fmt::format("{}, err: {}", channel_info(), st.to_string()));
             _add_batch_closure->clear_in_flight();
             return;
         }
@@ -990,10 +990,10 @@ Status OlapTableSink::open(RuntimeState* state) {
                 // The open() phase is mainly to generate DeltaWriter instances on the nodes corresponding to each node channel.
                 // This phase will not fail due to a single tablet.
                 // Therefore, if the open() phase fails, all tablets corresponding to the node need to be marked as failed.
-                index_channel->mark_as_failed(ch->node_id(), ch->host(),
-                                              fmt::format("{}, open failed, err: {}",
-                                                          ch->channel_info(), st.get_error_msg()),
-                                              -1);
+                index_channel->mark_as_failed(
+                        ch->node_id(), ch->host(),
+                        fmt::format("{}, open failed, err: {}", ch->channel_info(), st.to_string()),
+                        -1);
             }
         });
 
@@ -1133,13 +1133,13 @@ Status OlapTableSink::close(RuntimeState* state, Status close_status) {
                          &total_add_batch_num](const std::shared_ptr<NodeChannel>& ch) {
                             auto s = ch->close_wait(state);
                             if (!s.ok()) {
-                                index_channel->mark_as_failed(ch->node_id(), ch->host(),
-                                                              s.get_error_msg(), -1);
+                                auto err_msg = s.to_string();
+                                index_channel->mark_as_failed(ch->node_id(), ch->host(), err_msg,
+                                                              -1);
                                 // cancel the node channel in best effort
-                                ch->cancel(s.get_error_msg());
-                                LOG(WARNING)
-                                        << ch->channel_info()
-                                        << ", close channel failed, err: " << s.get_error_msg();
+                                ch->cancel(err_msg);
+                                LOG(WARNING) << ch->channel_info()
+                                             << ", close channel failed, err: " << err_msg;
                             }
                             ch->time_report(&node_add_batch_counter_map, &serialize_batch_ns,
                                             &mem_exceeded_block_ns, &queue_push_lock_ns,
@@ -1197,12 +1197,12 @@ Status OlapTableSink::close(RuntimeState* state, Status close_status) {
     } else {
         for (auto channel : _channels) {
             channel->for_each_node_channel([&status](const std::shared_ptr<NodeChannel>& ch) {
-                ch->cancel(status.get_error_msg());
+                ch->cancel(status.to_string());
             });
         }
         LOG(INFO) << "finished to close olap table sink. load_id=" << print_id(_load_id)
                   << ", txn_id=" << _txn_id
-                  << ", canceled all node channels due to error: " << status.get_error_msg();
+                  << ", canceled all node channels due to error: " << status;
     }
 
     // Sender join() must put after node channels mark_close/cancel.
diff --git a/be/src/exec/tablet_sink.h b/be/src/exec/tablet_sink.h
index fb820d6c42..a177db8a9b 100644
--- a/be/src/exec/tablet_sink.h
+++ b/be/src/exec/tablet_sink.h
@@ -409,7 +409,7 @@ void IndexChannel::add_row(const Row& tuple, int64_t tablet_id) {
         // if this node channel is already failed, this add_row will be skipped
         auto st = channel->add_row(tuple, tablet_id);
         if (!st.ok()) {
-            mark_as_failed(channel->node_id(), channel->host(), st.get_error_msg(), tablet_id);
+            mark_as_failed(channel->node_id(), channel->host(), st.to_string(), tablet_id);
             // continue add row to other node, the error will be checked for every batch outside
         }
     }
diff --git a/be/src/exprs/expr.cpp b/be/src/exprs/expr.cpp
index 864bcf748d..89e3077e4b 100644
--- a/be/src/exprs/expr.cpp
+++ b/be/src/exprs/expr.cpp
@@ -262,7 +262,7 @@ Status Expr::create_expr_tree(ObjectPool* pool, const TExpr& texpr, ExprContext*
     }
     if (!status.ok()) {
         LOG(ERROR) << "Could not construct expr tree.\n"
-                   << status.get_error_msg() << "\n"
+                   << status << "\n"
                    << apache::thrift::ThriftDebugString(texpr);
     }
     return status;
@@ -931,7 +931,7 @@ Status Expr::create_tree(const TExpr& texpr, ObjectPool* pool, Expr* root) {
         Status status = create_tree_internal(texpr.nodes, pool, root, &child_node_idx);
         if (UNLIKELY(!status.ok())) {
             LOG(ERROR) << "Could not construct expr tree.\n"
-                       << status.get_error_msg() << "\n"
+                       << status << "\n"
                        << apache::thrift::ThriftDebugString(texpr);
             return status;
         }
diff --git a/be/src/http/action/compaction_action.cpp b/be/src/http/action/compaction_action.cpp
index ea51c81e04..86d2548946 100644
--- a/be/src/http/action/compaction_action.cpp
+++ b/be/src/http/action/compaction_action.cpp
@@ -35,6 +35,7 @@
 #include "olap/storage_engine.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 const static std::string HEADER_JSON = "application/json";
 
@@ -192,7 +193,7 @@ Status CompactionAction::_execute_compaction_callback(TabletSharedPtr tablet,
         BaseCompaction base_compaction(tablet);
         res = base_compaction.compact();
         if (!res) {
-            if (res.precise_code() == OLAP_ERR_BE_NO_SUITABLE_VERSION) {
+            if (res.is<BE_NO_SUITABLE_VERSION>()) {
                 // Ignore this error code.
                 VLOG_NOTICE << "failed to init base compaction due to no suitable version, tablet="
                             << tablet->full_name();
@@ -206,7 +207,7 @@ Status CompactionAction::_execute_compaction_callback(TabletSharedPtr tablet,
         CumulativeCompaction cumulative_compaction(tablet);
         res = cumulative_compaction.compact();
         if (!res) {
-            if (res.precise_code() == OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION) {
+            if (res.is<CUMULATIVE_NO_SUITABLE_VERSION>()) {
                 // Ignore this error code.
                 VLOG_NOTICE << "failed to init cumulative compaction due to no suitable version,"
                             << "tablet=" << tablet->full_name();
diff --git a/be/src/http/action/download_action.cpp b/be/src/http/action/download_action.cpp
index d75beb4cd1..031f31f129 100644
--- a/be/src/http/action/download_action.cpp
+++ b/be/src/http/action/download_action.cpp
@@ -64,16 +64,14 @@ void DownloadAction::handle_normal(HttpRequest* req, const std::string& file_par
     if (config::enable_token_check) {
         status = check_token(req);
         if (!status.ok()) {
-            std::string error_msg = status.get_error_msg();
-            HttpChannel::send_reply(req, error_msg);
+            HttpChannel::send_reply(req, status.to_string());
             return;
         }
     }
 
     status = check_path_is_allowed(file_param);
     if (!status.ok()) {
-        std::string error_msg = status.get_error_msg();
-        HttpChannel::send_reply(req, error_msg);
+        HttpChannel::send_reply(req, status.to_string());
         return;
     }
 
@@ -89,7 +87,7 @@ void DownloadAction::handle_error_log(HttpRequest* req, const std::string& file_
 
     Status status = check_log_path_is_allowed(absolute_path);
     if (!status.ok()) {
-        std::string error_msg = status.get_error_msg();
+        std::string error_msg = status.to_string();
         HttpChannel::send_reply(req, error_msg);
         return;
     }
diff --git a/be/src/http/action/stream_load.cpp b/be/src/http/action/stream_load.cpp
index 69bfeabe48..3b2b289146 100644
--- a/be/src/http/action/stream_load.cpp
+++ b/be/src/http/action/stream_load.cpp
@@ -62,6 +62,7 @@
 #include "util/uid_util.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(streaming_load_requests_total, MetricUnit::REQUESTS);
 DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(streaming_load_duration_ms, MetricUnit::MILLISECONDS);
@@ -151,20 +152,20 @@ void StreamLoadAction::handle(HttpRequest* req) {
     // status already set to fail
     if (ctx->status.ok()) {
         ctx->status = _handle(ctx);
-        if (!ctx->status.ok() && ctx->status.code() != TStatusCode::PUBLISH_TIMEOUT) {
+        if (!ctx->status.ok() && !ctx->status.is<PUBLISH_TIMEOUT>()) {
             LOG(WARNING) << "handle streaming load failed, id=" << ctx->id
-                         << ", errmsg=" << ctx->status.get_error_msg();
+                         << ", errmsg=" << ctx->status;
         }
     }
     ctx->load_cost_millis = UnixMillis() - ctx->start_millis;
 
-    if (!ctx->status.ok() && ctx->status.code() != TStatusCode::PUBLISH_TIMEOUT) {
+    if (!ctx->status.ok() && !ctx->status.is<PUBLISH_TIMEOUT>()) {
         if (ctx->need_rollback) {
             _exec_env->stream_load_executor()->rollback_txn(ctx);
             ctx->need_rollback = false;
         }
         if (ctx->body_sink.get() != nullptr) {
-            ctx->body_sink->cancel(ctx->status.get_error_msg());
+            ctx->body_sink->cancel(ctx->status.to_string());
         }
     }
 
@@ -240,13 +241,13 @@ int StreamLoadAction::on_header(HttpRequest* req) {
 
     auto st = _on_header(req, ctx);
     if (!st.ok()) {
-        ctx->status = st;
+        ctx->status = std::move(st);
         if (ctx->need_rollback) {
             _exec_env->stream_load_executor()->rollback_txn(ctx);
             ctx->need_rollback = false;
         }
         if (ctx->body_sink.get() != nullptr) {
-            ctx->body_sink->cancel(st.get_error_msg());
+            ctx->body_sink->cancel(ctx->status.to_string());
         }
         auto str = ctx->to_json();
         // add new line at end
@@ -357,8 +358,7 @@ void StreamLoadAction::on_chunk_data(HttpRequest* req) {
         bb->flip();
         auto st = ctx->body_sink->append(bb);
         if (!st.ok()) {
-            LOG(WARNING) << "append body content failed. errmsg=" << st.get_error_msg() << ", "
-                         << ctx->brief();
+            LOG(WARNING) << "append body content failed. errmsg=" << st << ", " << ctx->brief();
             ctx->status = st;
             return;
         }
@@ -580,8 +580,7 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext*
 #endif
     Status plan_status(ctx->put_result.status);
     if (!plan_status.ok()) {
-        LOG(WARNING) << "plan streaming load failed. errmsg=" << plan_status.get_error_msg()
-                     << ctx->brief();
+        LOG(WARNING) << "plan streaming load failed. errmsg=" << plan_status << ctx->brief();
         return plan_status;
     }
     VLOG_NOTICE << "params is " << apache::thrift::ThriftDebugString(ctx->put_result.params);
diff --git a/be/src/io/broker_reader.cpp b/be/src/io/broker_reader.cpp
index d203653228..1073fd3ac6 100644
--- a/be/src/io/broker_reader.cpp
+++ b/be/src/io/broker_reader.cpp
@@ -85,7 +85,7 @@ Status BrokerReader::open() {
                                        config::thrift_rpc_timeout_ms, &status);
         if (!status.ok()) {
             LOG(WARNING) << "Create broker client failed. broker=" << broker_addr
-                         << ", status=" << status.get_error_msg();
+                         << ", status=" << status;
             return status;
         }
 
@@ -153,7 +153,7 @@ Status BrokerReader::readat(int64_t position, int64_t nbytes, int64_t* bytes_rea
                                        config::thrift_rpc_timeout_ms, &status);
         if (!status.ok()) {
             LOG(WARNING) << "Create broker client failed. broker=" << broker_addr
-                         << ", status=" << status.get_error_msg();
+                         << ", status=" << status;
             return status;
         }
 
@@ -228,7 +228,7 @@ void BrokerReader::close() {
                                        config::thrift_rpc_timeout_ms, &status);
         if (!status.ok()) {
             LOG(WARNING) << "Create broker client failed. broker=" << broker_addr
-                         << ", status=" << status.get_error_msg();
+                         << ", status=" << status;
             return;
         }
 
@@ -239,7 +239,7 @@ void BrokerReader::close() {
             status = client.reopen();
             if (!status.ok()) {
                 LOG(WARNING) << "Close broker reader failed. broker=" << broker_addr
-                             << ", status=" << status.get_error_msg();
+                             << ", status=" << status;
                 return;
             }
             client->closeReader(response, request);
diff --git a/be/src/io/broker_writer.cpp b/be/src/io/broker_writer.cpp
index c2f0e4dee8..398b172c07 100644
--- a/be/src/io/broker_writer.cpp
+++ b/be/src/io/broker_writer.cpp
@@ -85,7 +85,7 @@ Status BrokerWriter::open() {
                                        config::thrift_rpc_timeout_ms, &status);
         if (!status.ok()) {
             LOG(WARNING) << "Create broker writer client failed. "
-                         << "broker=" << broker_addr << ", status=" << status.get_error_msg();
+                         << "broker=" << broker_addr << ", status=" << status;
             return status;
         }
 
@@ -140,7 +140,7 @@ Status BrokerWriter::write(const uint8_t* buf, size_t buf_len, size_t* written_l
                                        config::thrift_rpc_timeout_ms, &status);
         if (!status.ok()) {
             LOG(WARNING) << "Create broker write client failed. "
-                         << "broker=" << broker_addr << ", status=" << status.get_error_msg();
+                         << "broker=" << broker_addr << ", status=" << status;
             return status;
         }
 
@@ -195,7 +195,7 @@ Status BrokerWriter::close() {
         BrokerServiceConnection client(client_cache(_env), broker_addr, 20000, &status);
         if (!status.ok()) {
             LOG(WARNING) << "Create broker write client failed. broker=" << broker_addr
-                         << ", status=" << status.get_error_msg();
+                         << ", status=" << status;
             return status;
         }
 
@@ -207,7 +207,7 @@ Status BrokerWriter::close() {
             status = client.reopen();
             if (!status.ok()) {
                 LOG(WARNING) << "Reopen broker writer failed. broker=" << broker_addr
-                             << ", status=" << status.get_error_msg();
+                             << ", status=" << status;
                 return status;
             }
             client->closeWriter(response, request);
diff --git a/be/src/io/cache/file_cache.cpp b/be/src/io/cache/file_cache.cpp
index b016ee2aa5..023da2f8f2 100644
--- a/be/src/io/cache/file_cache.cpp
+++ b/be/src/io/cache/file_cache.cpp
@@ -22,6 +22,7 @@
 #include "olap/iterators.h"
 
 namespace doris {
+using namespace ErrorCode;
 namespace io {
 
 Status FileCache::download_cache_to_local(const Path& cache_file, const Path& cache_done_file,
@@ -58,7 +59,7 @@ Status FileCache::download_cache_to_local(const Path& cache_file, const Path& ca
                 LOG(ERROR) << "read remote file failed: " << remote_file_reader->path().native()
                            << ", bytes read: " << bytes_read
                            << " vs need read size: " << need_req_size;
-                return Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+                return Status::Error<OS_ERROR>();
             }
             count_bytes_read += bytes_read;
             RETURN_NOT_OK_STATUS_WITH_WARN(
diff --git a/be/src/io/cache/sub_file_cache.cpp b/be/src/io/cache/sub_file_cache.cpp
index 34b509bc9e..d847bc4033 100644
--- a/be/src/io/cache/sub_file_cache.cpp
+++ b/be/src/io/cache/sub_file_cache.cpp
@@ -28,6 +28,7 @@
 #include "olap/iterators.h"
 
 namespace doris {
+using namespace ErrorCode;
 namespace io {
 
 using std::vector;
@@ -95,7 +96,7 @@ Status SubFileCache::read_at(size_t offset, Slice result, const IOContext& io_ct
             if (_cache_file_readers.find(*iter) == _cache_file_readers.end()) {
                 LOG(ERROR) << "Local cache file reader can't be found: " << offset_begin << ", "
                            << offset_begin;
-                return Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+                return Status::Error<OS_ERROR>();
             }
             if (offset_begin < offset) {
                 offset_begin = offset;
@@ -115,7 +116,7 @@ Status SubFileCache::read_at(size_t offset, Slice result, const IOContext& io_ct
                 LOG(ERROR) << "read local cache file failed: "
                            << _cache_file_readers[*iter]->path().native()
                            << ", bytes read: " << sub_bytes_read << " vs req size: " << req_size;
-                return Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+                return Status::Error<OS_ERROR>();
             }
             *bytes_read += sub_bytes_read;
             _last_match_times[*iter] = time(nullptr);
@@ -175,8 +176,7 @@ Status SubFileCache::_generate_cache_reader(size_t offset, size_t req_size) {
                 download_st.set_value(func());
             });
             if (!st.ok()) {
-                LOG(FATAL) << "Failed to submit download cache task to thread pool! "
-                           << st.get_error_msg();
+                LOG(FATAL) << "Failed to submit download cache task to thread pool! " << st;
             }
         } else {
             return Status::InternalError("Failed to get download cache thread token");
diff --git a/be/src/io/cache/whole_file_cache.cpp b/be/src/io/cache/whole_file_cache.cpp
index 613215c93c..3c7ccf45af 100644
--- a/be/src/io/cache/whole_file_cache.cpp
+++ b/be/src/io/cache/whole_file_cache.cpp
@@ -21,6 +21,7 @@
 #include "olap/iterators.h"
 
 namespace doris {
+using namespace ErrorCode;
 namespace io {
 
 const static std::string WHOLE_FILE_CACHE_NAME = "WHOLE_FILE_CACHE";
@@ -49,7 +50,7 @@ Status WholeFileCache::read_at(size_t offset, Slice result, const IOContext& io_
     if (*bytes_read != result.size) {
         LOG(ERROR) << "read cache file failed: " << _cache_file_reader->path().native()
                    << ", bytes read: " << bytes_read << " vs required size: " << result.size;
-        return Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+        return Status::Error<OS_ERROR>();
     }
     return Status::OK();
 }
@@ -112,8 +113,7 @@ Status WholeFileCache::_generate_cache_reader(size_t offset, size_t req_size) {
                 download_st.set_value(func());
             });
             if (!st.ok()) {
-                LOG(FATAL) << "Failed to submit download cache task to thread pool! "
-                           << st.get_error_msg();
+                LOG(FATAL) << "Failed to submit download cache task to thread pool! " << st;
                 return st;
             }
         } else {
diff --git a/be/src/io/fs/hdfs_file_system.h b/be/src/io/fs/hdfs_file_system.h
index 99d442e431..f86e73d1d3 100644
--- a/be/src/io/fs/hdfs_file_system.h
+++ b/be/src/io/fs/hdfs_file_system.h
@@ -20,6 +20,8 @@
 #include <gen_cpp/PlanNodes_types.h>
 #include <hdfs/hdfs.h>
 
+#include <atomic>
+
 #include "io/fs/remote_file_system.h"
 namespace doris {
 
diff --git a/be/src/io/hdfs_writer.cpp b/be/src/io/hdfs_writer.cpp
index 51c9fed7d7..46349015c7 100644
--- a/be/src/io/hdfs_writer.cpp
+++ b/be/src/io/hdfs_writer.cpp
@@ -19,6 +19,7 @@
 
 #include <filesystem>
 
+#include "common/logging.h"
 #include "service/backend_options.h"
 
 namespace doris {
diff --git a/be/src/olap/base_compaction.cpp b/be/src/olap/base_compaction.cpp
index ba71d2fd5d..ea4188788a 100644
--- a/be/src/olap/base_compaction.cpp
+++ b/be/src/olap/base_compaction.cpp
@@ -21,6 +21,7 @@
 #include "util/trace.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 BaseCompaction::BaseCompaction(TabletSharedPtr tablet)
         : Compaction(tablet, "BaseCompaction:" + std::to_string(tablet->tablet_id())) {}
@@ -29,13 +30,13 @@ BaseCompaction::~BaseCompaction() {}
 
 Status BaseCompaction::prepare_compact() {
     if (!_tablet->init_succeeded()) {
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     std::unique_lock<std::mutex> lock(_tablet->get_base_compaction_lock(), std::try_to_lock);
     if (!lock.owns_lock()) {
         LOG(WARNING) << "another base compaction is running. tablet=" << _tablet->full_name();
-        return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR);
+        return Status::Error<TRY_LOCK_FAILED>();
     }
     TRACE("got base compaction lock");
 
@@ -57,7 +58,7 @@ Status BaseCompaction::execute_compact_impl() {
     std::unique_lock<std::mutex> lock(_tablet->get_base_compaction_lock(), std::try_to_lock);
     if (!lock.owns_lock()) {
         LOG(WARNING) << "another base compaction is running. tablet=" << _tablet->full_name();
-        return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR);
+        return Status::Error<TRY_LOCK_FAILED>();
     }
     TRACE("got base compaction lock");
 
@@ -65,7 +66,7 @@ Status BaseCompaction::execute_compact_impl() {
     // for compaction may change. In this case, current compaction task should not be executed.
     if (_tablet->get_clone_occurred()) {
         _tablet->set_clone_occurred(false);
-        return Status::OLAPInternalError(OLAP_ERR_BE_CLONE_OCCURRED);
+        return Status::Error<BE_CLONE_OCCURRED>();
     }
 
     SCOPED_ATTACH_TASK(_mem_tracker);
@@ -113,7 +114,7 @@ Status BaseCompaction::pick_rowsets_to_compact() {
     RETURN_NOT_OK(_check_rowset_overlapping(_input_rowsets));
     _filter_input_rowset();
     if (_input_rowsets.size() <= 1) {
-        return Status::OLAPInternalError(OLAP_ERR_BE_NO_SUITABLE_VERSION);
+        return Status::Error<BE_NO_SUITABLE_VERSION>();
     }
 
     // If there are delete predicate rowsets in tablet, start_version > 0 implies some rowsets before
@@ -132,14 +133,14 @@ Status BaseCompaction::pick_rowsets_to_compact() {
             LOG(WARNING)
                     << "Some rowsets cannot apply delete predicates in base compaction. tablet_id="
                     << _tablet->tablet_id();
-            return Status::OLAPInternalError(OLAP_ERR_BE_NO_SUITABLE_VERSION);
+            return Status::Error<BE_NO_SUITABLE_VERSION>();
         }
     }
 
     if (_input_rowsets.size() == 2 && _input_rowsets[0]->end_version() == 1) {
         // the tablet is with rowset: [0-1], [2-y]
         // and [0-1] has no data. in this situation, no need to do base compaction.
-        return Status::OLAPInternalError(OLAP_ERR_BE_NO_SUITABLE_VERSION);
+        return Status::Error<BE_NO_SUITABLE_VERSION>();
     }
 
     // 1. cumulative rowset must reach base_compaction_num_cumulative_deltas threshold
@@ -192,7 +193,7 @@ Status BaseCompaction::pick_rowsets_to_compact() {
                 << ", num_cumulative_rowsets=" << _input_rowsets.size() - 1
                 << ", cumulative_base_ratio=" << cumulative_base_ratio
                 << ", interval_since_last_base_compaction=" << interval_since_last_base_compaction;
-    return Status::OLAPInternalError(OLAP_ERR_BE_NO_SUITABLE_VERSION);
+    return Status::Error<BE_NO_SUITABLE_VERSION>();
 }
 
 Status BaseCompaction::_check_rowset_overlapping(const std::vector<RowsetSharedPtr>& rowsets) {
@@ -202,7 +203,7 @@ Status BaseCompaction::_check_rowset_overlapping(const std::vector<RowsetSharedP
                          << "rowset version=" << rs->start_version() << "-" << rs->end_version()
                          << ", cumulative point=" << _tablet->cumulative_layer_point()
                          << ", tablet=" << _tablet->full_name();
-            return Status::OLAPInternalError(OLAP_ERR_BE_SEGMENTS_OVERLAPPING);
+            return Status::Error<BE_SEGMENTS_OVERLAPPING>();
         }
     }
     return Status::OK();
diff --git a/be/src/olap/base_tablet.cpp b/be/src/olap/base_tablet.cpp
index b69c6947d7..ee2e9539ab 100644
--- a/be/src/olap/base_tablet.cpp
+++ b/be/src/olap/base_tablet.cpp
@@ -24,6 +24,7 @@
 #include "util/path_util.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 extern MetricPrototype METRIC_query_scan_bytes;
 extern MetricPrototype METRIC_query_scan_rows;
@@ -54,7 +55,7 @@ BaseTablet::~BaseTablet() {
 Status BaseTablet::set_tablet_state(TabletState state) {
     if (_tablet_meta->tablet_state() == TABLET_SHUTDOWN && state != TABLET_SHUTDOWN) {
         LOG(WARNING) << "could not change tablet state from shutdown to " << state;
-        return Status::OLAPInternalError(OLAP_ERR_META_INVALID_ARGUMENT);
+        return Status::Error<META_INVALID_ARGUMENT>();
     }
     _tablet_meta->set_tablet_state(state);
     _state = state;
diff --git a/be/src/olap/byte_buffer.cpp b/be/src/olap/byte_buffer.cpp
index a90638a1ce..8be7373754 100644
--- a/be/src/olap/byte_buffer.cpp
+++ b/be/src/olap/byte_buffer.cpp
@@ -23,6 +23,7 @@
 #include "runtime/thread_context.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 StorageByteBuffer::StorageByteBuffer()
         : _array(nullptr), _capacity(0), _limit(0), _position(0), _is_mmap(false) {}
@@ -171,7 +172,7 @@ Status StorageByteBuffer::put(char src) {
         return Status::OK();
     }
 
-    return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
+    return Status::Error<BUFFER_OVERFLOW>();
 }
 
 Status StorageByteBuffer::put(uint64_t index, char src) {
@@ -180,19 +181,19 @@ Status StorageByteBuffer::put(uint64_t index, char src) {
         return Status::OK();
     }
 
-    return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
+    return Status::Error<BUFFER_OVERFLOW>();
 }
 
 Status StorageByteBuffer::put(const char* src, uint64_t src_size, uint64_t offset,
                               uint64_t length) {
     //没有足够的空间可以写
     if (length > remaining()) {
-        return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
+        return Status::Error<BUFFER_OVERFLOW>();
     }
 
     //src不够大
     if (offset + length > src_size) {
-        return Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND);
+        return Status::Error<OUT_OF_BOUND>();
     }
 
     memory_copy(&_array[_position], &src[offset], length);
diff --git a/be/src/olap/byte_buffer.h b/be/src/olap/byte_buffer.h
index b43e7fcccc..a0fe460243 100644
--- a/be/src/olap/byte_buffer.h
+++ b/be/src/olap/byte_buffer.h
@@ -73,23 +73,23 @@ public:
 
     uint64_t position() const { return _position; }
     // Set the position of the internal pointer
-    // If the new position is greater than or equal to limit, return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR)
+    // If the new position is greater than or equal to limit, return Status::Error<ErrorCode::INVALID_ARGUMENT>()
     Status set_position(uint64_t new_position) {
         if (new_position <= _limit) {
             _position = new_position;
             return Status::OK();
         } else {
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<ErrorCode::INVALID_ARGUMENT>();
         }
     }
 
     uint64_t limit() const { return _limit; }
     //set new limit
-    //If limit is greater than capacity, return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR)
+    //If limit is greater than capacity, return Status::Error<ErrorCode::INVALID_ARGUMENT>()
     //If position is greater than the new limit, set position equal to limit
     Status set_limit(uint64_t new_limit) {
         if (new_limit > _capacity) {
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<ErrorCode::INVALID_ARGUMENT>();
         }
 
         _limit = new_limit;
@@ -120,7 +120,7 @@ public:
             *result = _array[_position++];
             return Status::OK();
         } else {
-            return Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND);
+            return Status::Error<ErrorCode::OUT_OF_BOUND>();
         }
     }
 
@@ -130,7 +130,7 @@ public:
             *result = _array[index];
             return Status::OK();
         } else {
-            return Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND);
+            return Status::Error<ErrorCode::OUT_OF_BOUND>();
         }
     }
 
@@ -138,12 +138,12 @@ public:
     Status get(char* dst, uint64_t dst_size, uint64_t length) {
         // Not enough data to read
         if (OLAP_UNLIKELY(length > remaining())) {
-            return Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND);
+            return Status::Error<ErrorCode::OUT_OF_BOUND>();
         }
 
         // dst is not big enough
         if (OLAP_UNLIKELY(length > dst_size)) {
-            return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
+            return Status::Error<ErrorCode::BUFFER_OVERFLOW>();
         }
 
         memory_copy(dst, &_array[_position], length);
@@ -155,18 +155,18 @@ public:
     Status get(char* dst, uint64_t dst_size) { return get(dst, dst_size, dst_size); }
 
     // Write a byte, increment position when done
-    // If position >= limit before writing, return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW)
+    // If position >= limit before writing, return Status::Error<ErrorCode::BUFFER_OVERFLOW>()
     Status put(char src);
 
     // Write data at the index position without changing the position
     // Returns:
-    //   Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW) : index >= limit
+    //   Status::Error<ErrorCode::BUFFER_OVERFLOW>() : index >= limit
     Status put(uint64_t index, char src);
 
     // Read length bytes from &src[offset], write to buffer, and increase position after completion
     // Returns:
-    //   Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW): remaining() < length
-    //   Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND): offset + length > src_size
+    //   Status::Error<ErrorCode::BUFFER_OVERFLOW>(): remaining() < length
+    //   Status::Error<ErrorCode::OUT_OF_BOUND>(): offset + length > src_size
     Status put(const char* src, uint64_t src_size, uint64_t offset, uint64_t length);
 
     // write a set of data
diff --git a/be/src/olap/collect_iterator.cpp b/be/src/olap/collect_iterator.cpp
index 4b088fbeb6..8301fa299b 100644
--- a/be/src/olap/collect_iterator.cpp
+++ b/be/src/olap/collect_iterator.cpp
@@ -26,6 +26,7 @@
 #include "olap/rowset/beta_rowset_reader.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 CollectIterator::~CollectIterator() = default;
 
@@ -45,7 +46,7 @@ Status CollectIterator::add_child(RowsetReaderSharedPtr rs_reader) {
     std::unique_ptr<LevelIterator> child(new Level0Iterator(rs_reader, _reader));
     RETURN_NOT_OK(child->init());
     if (child->current_row() == nullptr) {
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+        return Status::Error<END_OF_FILE>();
     }
 
     _children.push_back(child.release());
@@ -185,7 +186,7 @@ Status CollectIterator::next(const RowCursor** row, bool* delete_flag) {
     if (LIKELY(_inner_iter)) {
         return _inner_iter->next(row, delete_flag);
     } else {
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+        return Status::Error<END_OF_FILE>();
     }
 }
 
@@ -236,7 +237,7 @@ Status CollectIterator::Level0Iterator::_refresh_current_row_v2() {
         }
     } while (_row_block != nullptr);
     _current_row = nullptr;
-    return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+    return Status::Error<END_OF_FILE>();
 }
 
 Status CollectIterator::Level0Iterator::next(const RowCursor** row, bool* delete_flag) {
@@ -285,11 +286,11 @@ CollectIterator::Level1Iterator::~Level1Iterator() {
 
 // Read next row into *row.
 // Returns
-//      Status::OLAPInternalError(OLAP_ERR_DATA_EOF) and set *row to nullptr when EOF is reached.
+//      Status::Error<END_OF_FILE>() and set *row to nullptr when EOF is reached.
 //      Others when error happens
 Status CollectIterator::Level1Iterator::next(const RowCursor** row, bool* delete_flag) {
     if (UNLIKELY(_cur_child == nullptr)) {
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+        return Status::Error<END_OF_FILE>();
     }
     if (_merge) {
         return _merge_next(row, delete_flag);
@@ -360,14 +361,14 @@ inline Status CollectIterator::Level1Iterator::_merge_next(const RowCursor** row
     if (LIKELY(res.ok())) {
         _heap->push(_cur_child);
         _cur_child = _heap->top();
-    } else if (res.precise_code() == OLAP_ERR_DATA_EOF) {
+    } else if (res.is<END_OF_FILE>()) {
         // current child has been read, to read next
         delete _cur_child;
         if (!_heap->empty()) {
             _cur_child = _heap->top();
         } else {
             _cur_child = nullptr;
-            return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+            return Status::Error<END_OF_FILE>();
         }
     } else {
         _cur_child = nullptr;
@@ -389,7 +390,7 @@ inline Status CollectIterator::Level1Iterator::_normal_next(const RowCursor** ro
     auto res = _cur_child->next(row, delete_flag);
     if (LIKELY(res.ok())) {
         return Status::OK();
-    } else if (res.precise_code() == OLAP_ERR_DATA_EOF) {
+    } else if (res.is<END_OF_FILE>()) {
         // current child has been read, to read next
         delete _cur_child;
         _children.pop_front();
@@ -399,7 +400,7 @@ inline Status CollectIterator::Level1Iterator::_normal_next(const RowCursor** ro
             return Status::OK();
         } else {
             _cur_child = nullptr;
-            return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+            return Status::Error<END_OF_FILE>();
         }
     } else {
         _cur_child = nullptr;
diff --git a/be/src/olap/collect_iterator.h b/be/src/olap/collect_iterator.h
index f5bcc3aa7b..ac9bed60e2 100644
--- a/be/src/olap/collect_iterator.h
+++ b/be/src/olap/collect_iterator.h
@@ -45,7 +45,7 @@ public:
 
     // Read next row into *row.
     // Returns
-    //      Status::OLAPInternalError(OLAP_ERR_DATA_EOF) and set *row to nullptr when EOF is reached.
+    //      Status::Error<END_OF_FILE>() and set *row to nullptr when EOF is reached.
     //      Others when error happens
     Status next(const RowCursor** row, bool* delete_flag);
 
diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp
index 8b2875afbd..fa47754cea 100644
--- a/be/src/olap/compaction.cpp
+++ b/be/src/olap/compaction.cpp
@@ -30,6 +30,7 @@
 using std::vector;
 
 namespace doris {
+using namespace ErrorCode;
 
 Compaction::Compaction(TabletSharedPtr tablet, const std::string& label)
         : _tablet(tablet),
@@ -305,7 +306,7 @@ Status Compaction::do_compaction_impl(int64_t permits) {
     if (_output_rowset == nullptr) {
         LOG(WARNING) << "rowset writer build failed. writer version:"
                      << ", output_version=" << _output_version;
-        return Status::OLAPInternalError(OLAP_ERR_ROWSET_BUILDER_INIT);
+        return Status::Error<ROWSET_BUILDER_INIT>();
     }
     TRACE_COUNTER_INCREMENT("output_rowset_data_size", _output_rowset->data_disk_size());
     TRACE_COUNTER_INCREMENT("output_row_num", _output_rowset->num_rows());
@@ -444,7 +445,7 @@ Status Compaction::check_version_continuity(const std::vector<RowsetSharedPtr>&
                          << prev_rowset->end_version()
                          << ", rowset version=" << rowset->start_version() << "-"
                          << rowset->end_version();
-            return Status::OLAPInternalError(OLAP_ERR_CUMULATIVE_MISS_VERSION);
+            return Status::Error<CUMULATIVE_MISS_VERSION>();
         }
         prev_rowset = rowset;
     }
@@ -460,7 +461,7 @@ Status Compaction::check_correctness(const Merger::Statistics& stats) {
                      << ", merged_row_num=" << stats.merged_rows
                      << ", filtered_row_num=" << stats.filtered_rows
                      << ", output_row_num=" << _output_rowset->num_rows();
-        return Status::OLAPInternalError(OLAP_ERR_CHECK_LINES_ERROR);
+        return Status::Error<CHECK_LINES_ERROR>();
     }
     return Status::OK();
 }
diff --git a/be/src/olap/comparison_predicate.h b/be/src/olap/comparison_predicate.h
index 8856fc4ade..ee1a60f10d 100644
--- a/be/src/olap/comparison_predicate.h
+++ b/be/src/olap/comparison_predicate.h
@@ -424,7 +424,7 @@ private:
                            roaring::Roaring* bitmap) const {
         roaring::Roaring roaring;
 
-        if (status.is_not_found()) {
+        if (status.is<ErrorCode::NOT_FOUND>()) {
             if constexpr (PT == PredicateType::EQ || PT == PredicateType::GT ||
                           PT == PredicateType::GE) {
                 *bitmap &= roaring; // set bitmap to empty
diff --git a/be/src/olap/compress.h b/be/src/olap/compress.h
index 4a3250ee27..255d5a415c 100644
--- a/be/src/olap/compress.h
+++ b/be/src/olap/compress.h
@@ -30,8 +30,8 @@ class StorageByteBuffer;
 //     out - output buffer,The space from position to limit can be used to store data
 //     smaller - Whether the compressed data size is smaller than the data size before compression
 // Returns:
-//     Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW) - Insufficient space left in output buffer
-//     Status::OLAPInternalError(OLAP_ERR_COMPRESS_ERROR) - Compression error
+//     Status::Error<BUFFER_OVERFLOW>() - Insufficient space left in output buffer
+//     Status::Error<COMPRESS_ERROR>() - Compression error
 typedef Status (*Compressor)(StorageByteBuffer* in, StorageByteBuffer* out, bool* smaller);
 
 // Define a decompression function to decompress the remaining memory in the input buffer
@@ -40,8 +40,8 @@ typedef Status (*Compressor)(StorageByteBuffer* in, StorageByteBuffer* out, bool
 //     in - input buffer,Decompress memory from position to limit
 //     out - output buffer,The space from position to limit can be used to store data
 // Returns:
-//     Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW) - Insufficient space left in output buffer
-//     Status::OLAPInternalError(OLAP_ERR_DECOMPRESS_ERROR) - decompression error
+//     Status::Error<BUFFER_OVERFLOW>() - Insufficient space left in output buffer
+//     Status::Error<DECOMPRESS_ERROR>() - decompression error
 typedef Status (*Decompressor)(StorageByteBuffer* in, StorageByteBuffer* out);
 
 #ifdef DORIS_WITH_LZO
diff --git a/be/src/olap/cumulative_compaction.cpp b/be/src/olap/cumulative_compaction.cpp
index b2fca016c7..6feb8c5c1c 100644
--- a/be/src/olap/cumulative_compaction.cpp
+++ b/be/src/olap/cumulative_compaction.cpp
@@ -22,6 +22,7 @@
 #include "util/trace.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 CumulativeCompaction::CumulativeCompaction(TabletSharedPtr tablet)
         : Compaction(tablet, "CumulativeCompaction:" + std::to_string(tablet->tablet_id())) {}
@@ -30,13 +31,13 @@ CumulativeCompaction::~CumulativeCompaction() {}
 
 Status CumulativeCompaction::prepare_compact() {
     if (!_tablet->init_succeeded()) {
-        return Status::OLAPInternalError(OLAP_ERR_CUMULATIVE_INVALID_PARAMETERS);
+        return Status::Error<CUMULATIVE_INVALID_PARAMETERS>();
     }
 
     std::unique_lock<std::mutex> lock(_tablet->get_cumulative_compaction_lock(), std::try_to_lock);
     if (!lock.owns_lock()) {
         LOG(INFO) << "The tablet is under cumulative compaction. tablet=" << _tablet->full_name();
-        return Status::OLAPInternalError(OLAP_ERR_CE_TRY_CE_LOCK_ERROR);
+        return Status::Error<TRY_LOCK_FAILED>();
     }
     TRACE("got cumulative compaction lock");
 
@@ -59,7 +60,7 @@ Status CumulativeCompaction::execute_compact_impl() {
     std::unique_lock<std::mutex> lock(_tablet->get_cumulative_compaction_lock(), std::try_to_lock);
     if (!lock.owns_lock()) {
         LOG(INFO) << "The tablet is under cumulative compaction. tablet=" << _tablet->full_name();
-        return Status::OLAPInternalError(OLAP_ERR_CE_TRY_CE_LOCK_ERROR);
+        return Status::Error<TRY_LOCK_FAILED>();
     }
     TRACE("got cumulative compaction lock");
 
@@ -67,7 +68,7 @@ Status CumulativeCompaction::execute_compact_impl() {
     // for compaction may change. In this case, current compaction task should not be executed.
     if (_tablet->get_clone_occurred()) {
         _tablet->set_clone_occurred(false);
-        return Status::OLAPInternalError(OLAP_ERR_CUMULATIVE_CLONE_OCCURRED);
+        return Status::Error<CUMULATIVE_CLONE_OCCURRED>();
     }
 
     SCOPED_ATTACH_TASK(_mem_tracker);
@@ -100,7 +101,7 @@ Status CumulativeCompaction::pick_rowsets_to_compact() {
     _tablet->pick_candidate_rowsets_to_cumulative_compaction(&candidate_rowsets, rdlock);
 
     if (candidate_rowsets.empty()) {
-        return Status::OLAPInternalError(OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION);
+        return Status::Error<CUMULATIVE_NO_SUITABLE_VERSION>();
     }
 
     // candidate_rowsets may not be continuous
@@ -122,14 +123,14 @@ Status CumulativeCompaction::pick_rowsets_to_compact() {
             &compaction_score);
 
     // Cumulative compaction will process with at least 1 rowset.
-    // So when there is no rowset being chosen, we should return Status::OLAPInternalError(OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION):
+    // So when there is no rowset being chosen, we should return Status::Error<CUMULATIVE_NO_SUITABLE_VERSION>():
     if (_input_rowsets.empty()) {
         if (_last_delete_version.first != -1) {
             // we meet a delete version, should increase the cumulative point to let base compaction handle the delete version.
             // plus 1 to skip the delete version.
             // NOTICE: after that, the cumulative point may be larger than max version of this tablet, but it doesn't matter.
             _tablet->set_cumulative_layer_point(_last_delete_version.first + 1);
-            return Status::OLAPInternalError(OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION);
+            return Status::Error<CUMULATIVE_NO_SUITABLE_VERSION>();
         }
 
         // we did not meet any delete version. which means compaction_score is not enough to do cumulative compaction.
@@ -172,7 +173,7 @@ Status CumulativeCompaction::pick_rowsets_to_compact() {
             }
         }
 
-        return Status::OLAPInternalError(OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION);
+        return Status::Error<CUMULATIVE_NO_SUITABLE_VERSION>();
     }
 
     return Status::OK();
diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp
index a745f6e43c..0b70977721 100644
--- a/be/src/olap/data_dir.cpp
+++ b/be/src/olap/data_dir.cpp
@@ -57,6 +57,7 @@
 using strings::Substitute;
 
 namespace doris {
+using namespace ErrorCode;
 
 DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(disks_total_capacity, MetricUnit::BYTES);
 DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(disks_avail_capacity, MetricUnit::BYTES);
@@ -147,7 +148,7 @@ Status DataDir::read_cluster_id(Env* env, const std::string& cluster_id_path, in
         } else {
             *cluster_id = -1;
         }
-    } else if (exist_status.is_not_found()) {
+    } else if (exist_status.is<NOT_FOUND>()) {
         *cluster_id = -1;
     } else {
         RETURN_NOT_OK_STATUS_WITH_WARN(
@@ -176,7 +177,7 @@ Status DataDir::_init_capacity() {
     auto data_path = fmt::format("{}/{}", _path, DATA_PREFIX);
     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())) {
+        (!exist_status.is<NOT_FOUND>() || !Env::Default()->create_dirs(data_path).ok())) {
         RETURN_NOT_OK_STATUS_WITH_WARN(
                 Status::IOError("failed to create data root path {}", data_path),
                 "create_dirs failed");
@@ -257,7 +258,7 @@ Status DataDir::get_shard(uint64_t* shard) {
     }
     auto shard_path = fmt::format("{}/{}/{}", _path, DATA_PREFIX, next_shard);
     RETURN_WITH_WARN_IF_ERROR(Env::Default()->create_dirs(shard_path),
-                              Status::OLAPInternalError(OLAP_ERR_CANNOT_CREATE_DIR),
+                              Status::Error<CANNOT_CREATE_DIR>(),
                               "fail to create path. path=" + shard_path);
 
     *shard = next_shard;
@@ -401,9 +402,9 @@ Status DataDir::load() {
                                     const std::string& value) -> bool {
         Status status = _tablet_manager->load_tablet_from_meta(this, tablet_id, schema_hash, value,
                                                                false, false, false, false);
-        if (!status.ok() && status.precise_code() != OLAP_ERR_TABLE_ALREADY_DELETED_ERROR &&
-            status.precise_code() != OLAP_ERR_ENGINE_INSERT_OLD_TABLET) {
-            // load_tablet_from_meta() may return Status::OLAPInternalError(OLAP_ERR_TABLE_ALREADY_DELETED_ERROR)
+        if (!status.ok() && !status.is<TABLE_ALREADY_DELETED_ERROR>() &&
+            !status.is<ENGINE_INSERT_OLD_TABLET>()) {
+            // load_tablet_from_meta() may return Status::Error<TABLE_ALREADY_DELETED_ERROR>()
             // which means the tablet status is DELETED
             // This may happen when the tablet was just deleted before the BE restarted,
             // but it has not been cleared from rocksdb. At this time, restarting the BE
@@ -411,7 +412,7 @@ Status DataDir::load() {
             // added to the garbage collection queue and will be automatically deleted afterwards.
             // Therefore, we believe that this situation is not a failure.
 
-            // Besides, load_tablet_from_meta() may return Status::OLAPInternalError(OLAP_ERR_ENGINE_INSERT_OLD_TABLET)
+            // Besides, load_tablet_from_meta() may return Status::Error<ENGINE_INSERT_OLD_TABLET>()
             // when BE is restarting and the older tablet have been added to the
             // garbage collection queue but not deleted yet.
             // In this case, since the data_dirs are parallel loaded, a later loaded tablet
@@ -488,9 +489,7 @@ Status DataDir::load() {
                     _meta, rowset_meta->partition_id(), rowset_meta->txn_id(),
                     rowset_meta->tablet_id(), rowset_meta->tablet_schema_hash(),
                     rowset_meta->tablet_uid(), rowset_meta->load_id(), rowset, true);
-            if (!commit_txn_status &&
-                commit_txn_status !=
-                        Status::OLAPInternalError(OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST)) {
+            if (!commit_txn_status && !commit_txn_status.is<PUSH_TRANSACTION_ALREADY_EXIST>()) {
                 LOG(WARNING) << "failed to add committed rowset: " << rowset_meta->rowset_id()
                              << " to tablet: " << rowset_meta->tablet_id()
                              << " for txn: " << rowset_meta->txn_id();
@@ -508,8 +507,7 @@ Status DataDir::load() {
                                         rowset_meta->get_rowset_pb());
             }
             Status publish_status = tablet->add_rowset(rowset);
-            if (!publish_status &&
-                publish_status.precise_code() != OLAP_ERR_PUSH_VERSION_ALREADY_EXIST) {
+            if (!publish_status && !publish_status.is<PUSH_VERSION_ALREADY_EXIST>()) {
                 LOG(WARNING) << "add visible rowset to tablet failed rowset_id:"
                              << rowset->rowset_id() << " tablet id: " << rowset_meta->tablet_id()
                              << " txn id:" << rowset_meta->txn_id()
@@ -797,7 +795,7 @@ Status DataDir::move_to_trash(const std::string& tablet_path) {
         !FileUtils::create_dir(trash_tablet_parent).ok()) {
         LOG(WARNING) << "delete file failed. due to mkdir failed. [file=" << tablet_path
                      << " new_dir=" << trash_tablet_parent << "]";
-        return Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+        return Status::Error<OS_ERROR>();
     }
 
     // 4. move tablet to trash
@@ -805,7 +803,7 @@ Status DataDir::move_to_trash(const std::string& tablet_path) {
     if (rename(tablet_path.c_str(), trash_tablet_path.c_str()) < 0) {
         LOG(WARNING) << "move file to trash failed. [file=" << tablet_path << " target='"
                      << trash_tablet_path << "' err='" << Errno::str() << "']";
-        return Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+        return Status::Error<OS_ERROR>();
     }
 
     // 5. check parent dir of source file, delete it when empty
diff --git a/be/src/olap/decimal12.h b/be/src/olap/decimal12.h
index 34658746ec..01bbec1b13 100644
--- a/be/src/olap/decimal12.h
+++ b/be/src/olap/decimal12.h
@@ -97,7 +97,7 @@ struct decimal12_t {
 
         if (sign != nullptr) {
             if (sign != value_string) {
-                return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+                return Status::Error<ErrorCode::INVALID_ARGUMENT>();
             } else {
                 ++value_string;
             }
diff --git a/be/src/olap/delete_handler.cpp b/be/src/olap/delete_handler.cpp
index 017cef5ecc..0b755ae890 100644
--- a/be/src/olap/delete_handler.cpp
+++ b/be/src/olap/delete_handler.cpp
@@ -47,6 +47,7 @@ using std::smatch;
 using google::protobuf::RepeatedPtrField;
 
 namespace doris {
+using namespace ErrorCode;
 
 Status DeleteHandler::generate_delete_predicate(const TabletSchema& schema,
                                                 const std::vector<TCondition>& conditions,
@@ -54,14 +55,14 @@ Status DeleteHandler::generate_delete_predicate(const TabletSchema& schema,
     if (conditions.empty()) {
         LOG(WARNING) << "invalid parameters for store_cond."
                      << " condition_size=" << conditions.size();
-        return Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_PARAMETERS);
+        return Status::Error<DELETE_INVALID_PARAMETERS>();
     }
 
     // Check whether the delete condition meets the requirements
     for (const TCondition& condition : conditions) {
         if (check_condition_valid(schema, condition) != Status::OK()) {
             LOG(WARNING) << "invalid condition. condition=" << ThriftDebugString(condition);
-            return Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION);
+            return Status::Error<DELETE_INVALID_CONDITION>();
         }
     }
 
@@ -168,7 +169,7 @@ Status DeleteHandler::check_condition_valid(const TabletSchema& schema, const TC
     int32_t field_index = schema.field_index(cond.column_name);
     if (field_index < 0) {
         LOG(WARNING) << "field is not existent. [field_index=" << field_index << "]";
-        return Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION);
+        return Status::Error<DELETE_INVALID_CONDITION>();
     }
 
     // Delete condition should only applied on key columns or duplicate key table, and
@@ -179,7 +180,7 @@ Status DeleteHandler::check_condition_valid(const TabletSchema& schema, const TC
         column.type() == OLAP_FIELD_TYPE_DOUBLE || column.type() == OLAP_FIELD_TYPE_FLOAT) {
         LOG(WARNING) << "field is not key column, or storage model is not duplicate, or data type "
                         "is float or double.";
-        return Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION);
+        return Status::Error<DELETE_INVALID_CONDITION>();
     }
 
     // Check operator and operands size are matched.
@@ -187,14 +188,14 @@ Status DeleteHandler::check_condition_valid(const TabletSchema& schema, const TC
         cond.condition_values.size() != 1) {
         LOG(WARNING) << "invalid condition value size. [size=" << cond.condition_values.size()
                      << "]";
-        return Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION);
+        return Status::Error<DELETE_INVALID_CONDITION>();
     }
 
     // Check each operand is valid
     for (const auto& condition_value : cond.condition_values) {
         if (!is_condition_value_valid(column, cond.condition_op, condition_value)) {
             LOG(WARNING) << "invalid condition value. [value=" << condition_value << "]";
-            return Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION);
+            return Status::Error<DELETE_INVALID_CONDITION>();
         }
     }
 
@@ -257,7 +258,7 @@ Status DeleteHandler::init(TabletSchemaSPtr tablet_schema,
             TCondition condition;
             if (!_parse_condition(sub_predicate, &condition)) {
                 LOG(WARNING) << "fail to parse condition. [condition=" << sub_predicate << "]";
-                return Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_PARAMETERS);
+                return Status::Error<DELETE_INVALID_PARAMETERS>();
             }
             condition.__set_column_unique_id(
                     delete_pred_related_schema->column(condition.column_name).unique_id());
diff --git a/be/src/olap/delete_handler.h b/be/src/olap/delete_handler.h
index 3dc8e5b3b7..815635ec53 100644
--- a/be/src/olap/delete_handler.h
+++ b/be/src/olap/delete_handler.h
@@ -88,8 +88,8 @@ public:
     //     * schema: tablet's schema, the delete conditions and data rows are in this schema
     //     * version: maximum version
     // return:
-    //     * Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_PARAMETERS): input parameters are not valid
-    //     * Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR): alloc memory failed
+    //     * Status::Error<DELETE_INVALID_PARAMETERS>(): input parameters are not valid
+    //     * Status::Error<MEM_ALLOC_FAILED>(): alloc memory failed
     Status init(TabletSchemaSPtr tablet_schema,
                 const std::vector<RowsetMetaSharedPtr>& delete_conditions, int64_t version);
 
diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp
index ad3f884797..856a9af261 100644
--- a/be/src/olap/delta_writer.cpp
+++ b/be/src/olap/delta_writer.cpp
@@ -33,6 +33,7 @@
 #include "util/ref_count_closure.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 Status DeltaWriter::open(WriteRequest* req, DeltaWriter** writer, const UniqueId& load_id,
                          bool is_vec) {
@@ -100,7 +101,7 @@ Status DeltaWriter::init() {
     if (_tablet == nullptr) {
         LOG(WARNING) << "fail to find tablet. tablet_id=" << _req.tablet_id
                      << ", schema_hash=" << _req.schema_hash;
-        return Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND);
+        return Status::Error<TABLE_NOT_FOUND>();
     }
 
     // get rowset ids snapshot
@@ -121,14 +122,14 @@ Status DeltaWriter::init() {
                          << _tablet->version_count()
                          << ", exceed limit: " << config::max_tablet_version_num
                          << ". tablet: " << _tablet->full_name();
-            return Status::OLAPInternalError(OLAP_ERR_TOO_MANY_VERSION);
+            return Status::Error<TOO_MANY_VERSION>();
         }
     }
 
     {
         std::shared_lock base_migration_rlock(_tablet->get_migration_lock(), std::try_to_lock);
         if (!base_migration_rlock.owns_lock()) {
-            return Status::OLAPInternalError(OLAP_ERR_RWLOCK_ERROR);
+            return Status::Error<TRY_LOCK_FAILED>();
         }
         std::lock_guard<std::mutex> push_lock(_tablet->get_push_lock());
         RETURN_NOT_OK(_storage_engine->txn_manager()->prepare_txn(_req.partition_id, _tablet,
@@ -243,7 +244,7 @@ Status DeltaWriter::flush_memtable_and_wait(bool need_wait) {
     std::lock_guard<std::mutex> l(_lock);
     if (!_is_init) {
         // This writer is not initialized before flushing. Do nothing
-        // But we return OLAP_SUCCESS instead of Status::OLAPInternalError(OLAP_ERR_ALREADY_CANCELLED),
+        // But we return OK instead of Status::Error<ALREADY_CANCELLED>(),
         // Because this method maybe called when trying to reduce mem consumption,
         // and at that time, the writer may not be initialized yet and that is a normal case.
         return Status::OK();
@@ -272,7 +273,7 @@ Status DeltaWriter::flush_memtable_and_wait(bool need_wait) {
 Status DeltaWriter::wait_flush() {
     std::lock_guard<std::mutex> l(_lock);
     if (!_is_init) {
-        // return OLAP_SUCCESS instead of Status::OLAPInternalError(OLAP_ERR_ALREADY_CANCELLED) for same reason
+        // return OK instead of Status::Error<ALREADY_CANCELLED>() for same reason
         // as described in flush_memtable_and_wait()
         return Status::OK();
     }
@@ -361,11 +362,11 @@ Status DeltaWriter::close_wait(const PSlaveTabletNodes& slave_tablet_nodes,
     _cur_rowset = _rowset_writer->build();
     if (_cur_rowset == nullptr) {
         LOG(WARNING) << "fail to build rowset";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+        return Status::Error<MEM_ALLOC_FAILED>();
     }
     Status res = _storage_engine->txn_manager()->commit_txn(_req.partition_id, _tablet, _req.txn_id,
                                                             _req.load_id, _cur_rowset, false);
-    if (!res && res != Status::OLAPInternalError(OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST)) {
+    if (!res && !res.is<PUSH_TRANSACTION_ALREADY_EXIST>()) {
         LOG(WARNING) << "Failed to commit txn: " << _req.txn_id
                      << " for rowset: " << _cur_rowset->rowset_id();
         return res;
diff --git a/be/src/olap/file_helper.cpp b/be/src/olap/file_helper.cpp
index ecbe6afa89..fa1bcb576c 100644
--- a/be/src/olap/file_helper.cpp
+++ b/be/src/olap/file_helper.cpp
@@ -34,6 +34,7 @@
 using std::string;
 
 namespace doris {
+using namespace ErrorCode;
 
 FileHandler::FileHandler() : _fd(-1), _wr_length(0), _file_name("") {}
 
@@ -47,7 +48,7 @@ Status FileHandler::open(const string& file_name, int flag) {
     }
 
     if (!this->close()) {
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     _fd = ::open(file_name.c_str(), flag);
@@ -57,9 +58,9 @@ Status FileHandler::open(const string& file_name, int flag) {
         LOG(WARNING) << "failed to open file. [err=" << strerror_r(errno, errmsg, 64)
                      << ", file_name='" << file_name << "' flag=" << flag << "]";
         if (errno == EEXIST) {
-            return Status::OLAPInternalError(OLAP_ERR_FILE_ALREADY_EXIST);
+            return Status::Error<FILE_ALREADY_EXIST>();
         }
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     VLOG_NOTICE << "success to open file. file_name=" << file_name << ", mode=" << flag
@@ -74,7 +75,7 @@ Status FileHandler::open_with_mode(const string& file_name, int flag, int mode)
     }
 
     if (!this->close()) {
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     _fd = ::open(file_name.c_str(), flag, mode);
@@ -84,9 +85,9 @@ Status FileHandler::open_with_mode(const string& file_name, int flag, int mode)
         LOG(WARNING) << "failed to open file. [err=" << strerror_r(errno, err_buf, 64)
                      << " file_name='" << file_name << "' flag=" << flag << " mode=" << mode << "]";
         if (errno == EEXIST) {
-            return Status::OLAPInternalError(OLAP_ERR_FILE_ALREADY_EXIST);
+            return Status::Error<FILE_ALREADY_EXIST>();
         }
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     VLOG_NOTICE << "success to open file. file_name=" << file_name << ", mode=" << mode
@@ -119,7 +120,7 @@ Status FileHandler::close() {
         char errmsg[64];
         LOG(WARNING) << "failed to close file. [err= " << strerror_r(errno, errmsg, 64)
                      << " file_name='" << _file_name << "' fd=" << _fd << "]";
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     VLOG_NOTICE << "finished to close file. "
@@ -141,13 +142,13 @@ Status FileHandler::pread(void* buf, size_t size, size_t offset) {
             LOG(WARNING) << "failed to pread from file. [err= " << strerror_r(errno, errmsg, 64)
                          << " file_name='" << _file_name << "' fd=" << _fd << " size=" << size
                          << " offset=" << offset << "]";
-            return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+            return Status::Error<IO_ERROR>();
         } else if (0 == rd_size) {
             char errmsg[64];
             LOG(WARNING) << "read unenough from file. [err= " << strerror_r(errno, errmsg, 64)
                          << " file_name='" << _file_name << "' fd=" << _fd << " size=" << size
                          << " offset=" << offset << "]";
-            return Status::OLAPInternalError(OLAP_ERR_READ_UNENOUGH);
+            return Status::Error<READ_UNENOUGH>();
         }
 
         size -= rd_size;
@@ -169,13 +170,13 @@ Status FileHandler::write(const void* buf, size_t buf_size) {
             LOG(WARNING) << "failed to write to file. [err= " << strerror_r(errno, errmsg, 64)
                          << " file_name='" << _file_name << "' fd=" << _fd << " size=" << buf_size
                          << "]";
-            return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+            return Status::Error<IO_ERROR>();
         } else if (0 == wr_size) {
             char errmsg[64];
             LOG(WARNING) << "write unenough to file. [err=" << strerror_r(errno, errmsg, 64)
                          << " file_name='" << _file_name << "' fd=" << _fd << " size=" << buf_size
                          << "]";
-            return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+            return Status::Error<IO_ERROR>();
         }
 
         buf_size -= wr_size;
@@ -208,13 +209,13 @@ Status FileHandler::pwrite(const void* buf, size_t buf_size, size_t offset) {
             LOG(WARNING) << "failed to pwrite to file. [err= " << strerror_r(errno, errmsg, 64)
                          << " file_name='" << _file_name << "' fd=" << _fd << " size=" << buf_size
                          << " offset=" << offset << "]";
-            return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+            return Status::Error<IO_ERROR>();
         } else if (0 == wr_size) {
             char errmsg[64];
             LOG(WARNING) << "pwrite unenough to file. [err= " << strerror_r(errno, errmsg, 64)
                          << " file_name='" << _file_name << "' fd=" << _fd << " size=" << buf_size
                          << " offset=" << offset << "]";
-            return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+            return Status::Error<IO_ERROR>();
         }
 
         buf_size -= wr_size;
@@ -249,7 +250,7 @@ Status FileHandlerWithBuf::open(const string& file_name, const char* mode) {
     }
 
     if (!this->close()) {
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     _fp = ::fopen(file_name.c_str(), mode);
@@ -259,9 +260,9 @@ Status FileHandlerWithBuf::open(const string& file_name, const char* mode) {
         LOG(WARNING) << "failed to open file. [err= " << strerror_r(errno, errmsg, 64)
                      << " file_name='" << file_name << "' flag='" << mode << "']";
         if (errno == EEXIST) {
-            return Status::OLAPInternalError(OLAP_ERR_FILE_ALREADY_EXIST);
+            return Status::Error<FILE_ALREADY_EXIST>();
         }
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     VLOG_NOTICE << "success to open file. "
@@ -284,7 +285,7 @@ Status FileHandlerWithBuf::close() {
         char errmsg[64];
         LOG(WARNING) << "failed to close file. [err= " << strerror_r(errno, errmsg, 64)
                      << " file_name='" << _file_name << "']";
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     _fp = nullptr;
@@ -295,7 +296,7 @@ Status FileHandlerWithBuf::close() {
 Status FileHandlerWithBuf::read(void* buf, size_t size) {
     if (OLAP_UNLIKELY(nullptr == _fp)) {
         LOG(WARNING) << "Fail to write, fp is nullptr!";
-        return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
+        return Status::Error<UNINITIALIZED>();
     }
 
     size_t rd_size = ::fread(buf, 1, size, _fp);
@@ -307,19 +308,19 @@ Status FileHandlerWithBuf::read(void* buf, size_t size) {
         LOG(WARNING) << "read unenough from file. [err=" << strerror_r(errno, errmsg, 64)
                      << " file_name='" << _file_name << "' size=" << size << " rd_size=" << rd_size
                      << "]";
-        return Status::OLAPInternalError(OLAP_ERR_READ_UNENOUGH);
+        return Status::Error<READ_UNENOUGH>();
     } else {
         char errmsg[64];
         LOG(WARNING) << "failed to read from file. [err=" << strerror_r(errno, errmsg, 64)
                      << " file_name='" << _file_name << "' size=" << size << "]";
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 }
 
 Status FileHandlerWithBuf::pread(void* buf, size_t size, size_t offset) {
     if (OLAP_UNLIKELY(nullptr == _fp)) {
         LOG(WARNING) << "Fail to write, fp is nullptr!";
-        return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
+        return Status::Error<UNINITIALIZED>();
     }
 
     if (0 != ::fseek(_fp, offset, SEEK_SET)) {
@@ -327,7 +328,7 @@ Status FileHandlerWithBuf::pread(void* buf, size_t size, size_t offset) {
         LOG(WARNING) << "failed to seek file. [err= " << strerror_r(errno, errmsg, 64)
                      << " file_name='" << _file_name << "' size=" << size << " offset=" << offset
                      << "]";
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     return this->read(buf, size);
@@ -336,7 +337,7 @@ Status FileHandlerWithBuf::pread(void* buf, size_t size, size_t offset) {
 Status FileHandlerWithBuf::write(const void* buf, size_t buf_size) {
     if (OLAP_UNLIKELY(nullptr == _fp)) {
         LOG(WARNING) << "Fail to write, fp is nullptr!";
-        return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
+        return Status::Error<UNINITIALIZED>();
     }
 
     size_t wr_size = ::fwrite(buf, 1, buf_size, _fp);
@@ -345,7 +346,7 @@ Status FileHandlerWithBuf::write(const void* buf, size_t buf_size) {
         char errmsg[64];
         LOG(WARNING) << "failed to write to file. [err= " << strerror_r(errno, errmsg, 64)
                      << " file_name='" << _file_name << "' size=" << buf_size << "]";
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     return Status::OK();
@@ -354,7 +355,7 @@ Status FileHandlerWithBuf::write(const void* buf, size_t buf_size) {
 Status FileHandlerWithBuf::pwrite(const void* buf, size_t buf_size, size_t offset) {
     if (OLAP_UNLIKELY(nullptr == _fp)) {
         LOG(WARNING) << "Fail to write, fp is nullptr!";
-        return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
+        return Status::Error<UNINITIALIZED>();
     }
 
     if (0 != ::fseek(_fp, offset, SEEK_SET)) {
@@ -362,7 +363,7 @@ Status FileHandlerWithBuf::pwrite(const void* buf, size_t buf_size, size_t offse
         LOG(WARNING) << "failed to seek file. [err= " << strerror_r(errno, errmsg, 64)
                      << " file_name='" << _file_name << "' size=" << buf_size
                      << " offset=" << offset << "]";
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     return this->write(buf, buf_size);
diff --git a/be/src/olap/file_helper.h b/be/src/olap/file_helper.h
index 132d4b0f10..c01602ee24 100644
--- a/be/src/olap/file_helper.h
+++ b/be/src/olap/file_helper.h
@@ -236,7 +236,7 @@ private:
 template <typename MessageType, typename ExtraType, typename FileHandlerType>
 Status FileHeader<MessageType, ExtraType, FileHandlerType>::prepare(FileHandlerType* file_handler) {
     if (nullptr == file_handler) {
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<ErrorCode::INVALID_ARGUMENT>();
     }
 
     // Use the file name as Signature to prevent problems caused by some misoperations
@@ -246,12 +246,12 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::prepare(FileHandlerT
         if (!_proto.SerializeToString(&_proto_string)) {
             LOG(WARNING) << "serialize file header to string error. [path='"
                          << file_handler->file_name() << "']";
-            return Status::OLAPInternalError(OLAP_ERR_SERIALIZE_PROTOBUF_ERROR);
+            return Status::Error<ErrorCode::SERIALIZE_PROTOBUF_ERROR>();
         }
     } catch (...) {
         LOG(WARNING) << "serialize file header to string error. [path='"
                      << file_handler->file_name() << "']";
-        return Status::OLAPInternalError(OLAP_ERR_SERIALIZE_PROTOBUF_ERROR);
+        return Status::Error<ErrorCode::SERIALIZE_PROTOBUF_ERROR>();
     }
 
     _fixed_file_header.protobuf_checksum =
@@ -270,7 +270,7 @@ template <typename MessageType, typename ExtraType, typename FileHandlerType>
 Status FileHeader<MessageType, ExtraType, FileHandlerType>::serialize(
         FileHandlerType* file_handler) {
     if (nullptr == file_handler) {
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<ErrorCode::INVALID_ARGUMENT>();
     }
 
     // write to file
@@ -278,7 +278,7 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::serialize(
         char errmsg[64];
         LOG(WARNING) << "fail to write fixed header to file. [file='" << file_handler->file_name()
                      << "' err=" << strerror_r(errno, errmsg, 64) << "]";
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<ErrorCode::IO_ERROR>();
     }
 
     if (!file_handler->pwrite(&_extra_fixed_header, sizeof(_extra_fixed_header),
@@ -287,7 +287,7 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::serialize(
         LOG(WARNING) << "fail to write extra fixed header to file. [file='"
                      << file_handler->file_name() << "' err=" << strerror_r(errno, errmsg, 64)
                      << "]";
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<ErrorCode::IO_ERROR>();
     }
 
     if (!file_handler->pwrite(_proto_string.c_str(), _proto_string.size(),
@@ -295,7 +295,7 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::serialize(
         char errmsg[64];
         LOG(WARNING) << "fail to write proto header to file. [file='" << file_handler->file_name()
                      << "' err='" << strerror_r(errno, errmsg, 64) << "']";
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<ErrorCode::IO_ERROR>();
     }
 
     return Status::OK();
@@ -305,7 +305,7 @@ template <typename MessageType, typename ExtraType, typename FileHandlerType>
 Status FileHeader<MessageType, ExtraType, FileHandlerType>::unserialize(
         FileHandlerType* file_handler) {
     if (nullptr == file_handler) {
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<ErrorCode::INVALID_ARGUMENT>();
     }
 
     off_t real_file_length = 0;
@@ -315,7 +315,7 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::unserialize(
         char errmsg[64];
         LOG(WARNING) << "fail to load header structure from file. file="
                      << file_handler->file_name() << ", error=" << strerror_r(errno, errmsg, 64);
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<ErrorCode::IO_ERROR>();
     }
 
     if (_fixed_file_header.magic_number != OLAP_FIX_HEADER_MAGIC_NUMBER) {
@@ -327,7 +327,7 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::unserialize(
             LOG(WARNING) << "fail to load header structure from file. file="
                          << file_handler->file_name()
                          << ", error=" << strerror_r(errno, errmsg, 64);
-            return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+            return Status::Error<ErrorCode::IO_ERROR>();
         }
 
         _fixed_file_header.file_length = tmp_header.file_length;
@@ -351,7 +351,7 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::unserialize(
         char errmsg[64];
         LOG(WARNING) << "fail to load extra fixed header from file. file="
                      << file_handler->file_name() << ", error=" << strerror_r(errno, errmsg, 64);
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<ErrorCode::IO_ERROR>();
     }
 
     std::unique_ptr<char[]> buf(new (std::nothrow) char[_fixed_file_header.protobuf_length]);
@@ -360,7 +360,7 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::unserialize(
         char errmsg[64];
         LOG(WARNING) << "malloc protobuf buf error. file=" << file_handler->file_name()
                      << ", error=" << strerror_r(errno, errmsg, 64);
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+        return Status::Error<ErrorCode::MEM_ALLOC_FAILED>();
     }
 
     if (!file_handler->pread(buf.get(), _fixed_file_header.protobuf_length,
@@ -368,7 +368,7 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::unserialize(
         char errmsg[64];
         LOG(WARNING) << "fail to load protobuf from file. file=" << file_handler->file_name()
                      << ", error=" << strerror_r(errno, errmsg, 64);
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<ErrorCode::IO_ERROR>();
     }
 
     real_file_length = file_handler->length();
@@ -377,7 +377,7 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::unserialize(
         LOG(WARNING) << "file length is not match. file=" << file_handler->file_name()
                      << ", file_length=" << file_length()
                      << ", real_file_length=" << real_file_length;
-        return Status::OLAPInternalError(OLAP_ERR_FILE_DATA_ERROR);
+        return Status::Error<ErrorCode::FILE_DATA_ERROR>();
     }
 
     // check proto checksum
@@ -388,7 +388,7 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::unserialize(
         LOG(WARNING) << "checksum is not match. file=" << file_handler->file_name()
                      << ", expect=" << _fixed_file_header.protobuf_checksum
                      << ", actual=" << real_protobuf_checksum;
-        return Status::OLAPInternalError(OLAP_ERR_CHECKSUM_ERROR);
+        return Status::Error<ErrorCode::CHECKSUM_ERROR>();
     }
 
     try {
@@ -397,11 +397,11 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::unserialize(
         if (!_proto.ParseFromString(protobuf_str)) {
             LOG(WARNING) << "fail to parse file content to protobuf object. file="
                          << file_handler->file_name();
-            return Status::OLAPInternalError(OLAP_ERR_PARSE_PROTOBUF_ERROR);
+            return Status::Error<ErrorCode::PARSE_PROTOBUF_ERROR>();
         }
     } catch (...) {
         LOG(WARNING) << "fail to load protobuf. file='" << file_handler->file_name();
-        return Status::OLAPInternalError(OLAP_ERR_PARSE_PROTOBUF_ERROR);
+        return Status::Error<ErrorCode::PARSE_PROTOBUF_ERROR>();
     }
 
     return Status::OK();
@@ -416,7 +416,7 @@ Status FileHeader<MessageType, ExtraType, FileHandlerType>::validate(const std::
         char errmsg[64];
         LOG(WARNING) << "fail to open file. [file='" << filename
                      << "' err=" << strerror_r(errno, errmsg, 64) << "]";
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<ErrorCode::IO_ERROR>();
     }
 
     if (!(res = unserialize(&file_handler))) {
diff --git a/be/src/olap/generic_iterators.cpp b/be/src/olap/generic_iterators.cpp
index f8be5dc02b..eec7e30477 100644
--- a/be/src/olap/generic_iterators.cpp
+++ b/be/src/olap/generic_iterators.cpp
@@ -24,6 +24,7 @@
 #include "olap/row_cursor_cell.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 // This iterator will generate ordered data. For example for schema
 // (int, int) this iterator will generator data like
@@ -197,7 +198,7 @@ Status MergeIteratorContext::_load_next_block() {
         Status st = _iter->next_batch(&_block);
         if (!st.ok()) {
             _valid = false;
-            if (st.is_end_of_file()) {
+            if (st.is<END_OF_FILE>()) {
                 return Status::OK();
             } else {
                 return st;
@@ -378,7 +379,7 @@ Status UnionIterator::init(const StorageReadOptions& opts) {
 Status UnionIterator::next_batch(RowBlockV2* block) {
     while (_cur_iter != nullptr) {
         auto st = _cur_iter->next_batch(block);
-        if (st.is_end_of_file()) {
+        if (st.is<END_OF_FILE>()) {
             delete _cur_iter;
             _cur_iter = nullptr;
             _origin_iters.pop_front();
diff --git a/be/src/olap/in_list_predicate.h b/be/src/olap/in_list_predicate.h
index eed5260a6a..e1bc5316f2 100644
--- a/be/src/olap/in_list_predicate.h
+++ b/be/src/olap/in_list_predicate.h
@@ -221,7 +221,7 @@ public:
             bool exact_match;
             Status s = iterator->seek_dictionary(&value, &exact_match);
             rowid_t seeked_ordinal = iterator->current_ordinal();
-            if (!s.is_not_found()) {
+            if (!s.is<ErrorCode::NOT_FOUND>()) {
                 if (!s.ok()) {
                     return s;
                 }
diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp
index 892592b797..10c35c19f6 100644
--- a/be/src/olap/memtable.cpp
+++ b/be/src/olap/memtable.cpp
@@ -31,6 +31,7 @@
 #include "vec/core/field.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 MemTable::MemTable(TabletSharedPtr tablet, Schema* schema, const TabletSchema* tablet_schema,
                    const std::vector<SlotDescriptor*>* slot_descs, TupleDescriptor* tuple_desc,
@@ -459,7 +460,7 @@ Status MemTable::_do_flush(int64_t& duration_ns) {
     SCOPED_RAW_TIMER(&duration_ns);
     if (_skip_list) {
         Status st = _rowset_writer->flush_single_memtable(this, &_flush_size);
-        if (st.precise_code() == OLAP_ERR_FUNC_NOT_IMPLEMENTED) {
+        if (st.is<NOT_IMPLEMENTED_ERROR>()) {
             // For alpha rowset, we do not implement "flush_single_memtable".
             // Flush the memtable like the old way.
             Table::Iterator it(_skip_list.get());
diff --git a/be/src/olap/memtable_flush_executor.cpp b/be/src/olap/memtable_flush_executor.cpp
index 1dd4859e5a..52a5ffdda2 100644
--- a/be/src/olap/memtable_flush_executor.cpp
+++ b/be/src/olap/memtable_flush_executor.cpp
@@ -25,6 +25,7 @@
 #include "util/time.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 class MemtableFlushTask final : public Runnable {
 public:
@@ -58,9 +59,9 @@ std::ostream& operator<<(std::ostream& os, const FlushStatistic& stat) {
 }
 
 Status FlushToken::submit(std::unique_ptr<MemTable> mem_table) {
-    ErrorCode s = _flush_status.load();
-    if (s != OLAP_SUCCESS) {
-        return Status::OLAPInternalError(s);
+    auto s = _flush_status.load();
+    if (s != OK) {
+        return Status::Error(s);
     }
     int64_t submit_task_time = MonotonicNanos();
     auto task = std::make_shared<MemtableFlushTask>(this, std::move(mem_table), submit_task_time);
@@ -74,14 +75,14 @@ void FlushToken::cancel() {
 
 Status FlushToken::wait() {
     _flush_token->wait();
-    ErrorCode s = _flush_status.load();
-    return s == OLAP_SUCCESS ? Status::OK() : Status::OLAPInternalError(s);
+    auto s = _flush_status.load();
+    return s == OK ? Status::OK() : Status::Error(s);
 }
 
 void FlushToken::_flush_memtable(MemTable* memtable, int64_t submit_task_time) {
     _stats.flush_wait_time_ns += (MonotonicNanos() - submit_task_time);
     // If previous flush has failed, return directly
-    if (_flush_status.load() != OLAP_SUCCESS) {
+    if (_flush_status.load() != OK) {
         return;
     }
 
@@ -92,8 +93,8 @@ void FlushToken::_flush_memtable(MemTable* memtable, int64_t submit_task_time) {
         LOG(WARNING) << "Flush memtable failed with res = " << s;
     }
     // If s is not ok, ignore the code, just use other code is ok
-    _flush_status.store(s.ok() ? OLAP_SUCCESS : OLAP_ERR_OTHER_ERROR);
-    if (_flush_status.load() != OLAP_SUCCESS) {
+    _flush_status.store(s.ok() ? OK : ErrorCode::INTERNAL_ERROR);
+    if (_flush_status.load() != OK) {
         return;
     }
 
diff --git a/be/src/olap/memtable_flush_executor.h b/be/src/olap/memtable_flush_executor.h
index c558439af2..ff5e1e08e8 100644
--- a/be/src/olap/memtable_flush_executor.h
+++ b/be/src/olap/memtable_flush_executor.h
@@ -56,7 +56,7 @@ std::ostream& operator<<(std::ostream& os, const FlushStatistic& stat);
 class FlushToken {
 public:
     explicit FlushToken(std::unique_ptr<ThreadPoolToken> flush_pool_token)
-            : _flush_token(std::move(flush_pool_token)), _flush_status(OLAP_SUCCESS) {}
+            : _flush_token(std::move(flush_pool_token)), _flush_status(ErrorCode::OK) {}
 
     Status submit(std::unique_ptr<MemTable> mem_table);
 
@@ -79,7 +79,7 @@ private:
 
     // Records the current flush status of the tablet.
     // Note: Once its value is set to Failed, it cannot return to SUCCESS.
-    std::atomic<ErrorCode> _flush_status;
+    std::atomic<int> _flush_status;
 
     FlushStatistic _stats;
 };
diff --git a/be/src/olap/merger.h b/be/src/olap/merger.h
index aff9a741c4..734c6a866c 100644
--- a/be/src/olap/merger.h
+++ b/be/src/olap/merger.h
@@ -39,7 +39,7 @@ public:
     };
 
     // merge rows from `src_rowset_readers` and write into `dst_rowset_writer`.
-    // return OLAP_SUCCESS and set statistics into `*stats_output`.
+    // return OK and set statistics into `*stats_output`.
     // return others on error
     static Status merge_rowsets(TabletSharedPtr tablet, ReaderType reader_type,
                                 TabletSchemaSPtr cur_tablet_schema,
diff --git a/be/src/olap/olap_meta.cpp b/be/src/olap/olap_meta.cpp
index 963822b480..92b85b1d5b 100644
--- a/be/src/olap/olap_meta.cpp
+++ b/be/src/olap/olap_meta.cpp
@@ -42,6 +42,7 @@ using rocksdb::kDefaultColumnFamilyName;
 using rocksdb::NewFixedPrefixTransform;
 
 namespace doris {
+using namespace ErrorCode;
 const std::string META_POSTFIX = "/meta";
 const size_t PREFIX_LENGTH = 4;
 
@@ -77,7 +78,7 @@ Status OlapMeta::init() {
     rocksdb::Status s = DB::Open(options, db_path, column_families, &_handles, &_db);
     if (!s.ok() || _db == nullptr) {
         LOG(WARNING) << "rocks db open failed, reason:" << s.ToString();
-        return Status::OLAPInternalError(OLAP_ERR_META_OPEN_DB);
+        return Status::Error<META_OPEN_DB_ERROR>();
     }
     return Status::OK();
 }
@@ -93,10 +94,10 @@ Status OlapMeta::get(const int column_family_index, const std::string& key, std:
     }
     DorisMetrics::instance()->meta_read_request_duration_us->increment(duration_ns / 1000);
     if (s.IsNotFound()) {
-        return Status::OLAPInternalError(OLAP_ERR_META_KEY_NOT_FOUND);
+        return Status::Error<META_KEY_NOT_FOUND>();
     } else if (!s.ok()) {
         LOG(WARNING) << "rocks db get key:" << key << " failed, reason:" << s.ToString();
-        return Status::OLAPInternalError(OLAP_ERR_META_GET);
+        return Status::Error<META_GET_ERROR>();
     }
     return Status::OK();
 }
@@ -131,7 +132,7 @@ Status OlapMeta::put(const int column_family_index, const std::string& key,
     DorisMetrics::instance()->meta_write_request_duration_us->increment(duration_ns / 1000);
     if (!s.ok()) {
         LOG(WARNING) << "rocks db put key:" << key << " failed, reason:" << s.ToString();
-        return Status::OLAPInternalError(OLAP_ERR_META_PUT);
+        return Status::Error<META_PUT_ERROR>();
     }
     return Status::OK();
 }
@@ -150,7 +151,7 @@ Status OlapMeta::remove(const int column_family_index, const std::string& key) {
     DorisMetrics::instance()->meta_write_request_duration_us->increment(duration_ns / 1000);
     if (!s.ok()) {
         LOG(WARNING) << "rocks db delete key:" << key << " failed, reason:" << s.ToString();
-        return Status::OLAPInternalError(OLAP_ERR_META_DELETE);
+        return Status::Error<META_DELETE_ERROR>();
     }
     return Status::OK();
 }
@@ -167,7 +168,7 @@ Status OlapMeta::iterate(const int column_family_index, const std::string& prefi
     rocksdb::Status status = it->status();
     if (!status.ok()) {
         LOG(WARNING) << "rocksdb seek failed. reason:" << status.ToString();
-        return Status::OLAPInternalError(OLAP_ERR_META_ITERATOR);
+        return Status::Error<META_ITERATOR_ERROR>();
     }
     for (; it->Valid(); it->Next()) {
         if (prefix != "") {
@@ -184,7 +185,7 @@ Status OlapMeta::iterate(const int column_family_index, const std::string& prefi
     }
     if (!it->status().ok()) {
         LOG(WARNING) << "rocksdb iterator failed. reason:" << status.ToString();
-        return Status::OLAPInternalError(OLAP_ERR_META_ITERATOR);
+        return Status::Error<META_ITERATOR_ERROR>();
     }
     return Status::OK();
 }
diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp
index a266d643b3..01b9581403 100644
--- a/be/src/olap/olap_server.cpp
+++ b/be/src/olap/olap_server.cpp
@@ -467,7 +467,7 @@ void StorageEngine::_compaction_tasks_producer_callback() {
                 Status st = _submit_compaction_task(tablet, compaction_type);
                 if (!st.ok()) {
                     LOG(WARNING) << "failed to submit compaction task for tablet: "
-                                 << tablet->tablet_id() << ", err: " << st.get_error_msg();
+                                 << tablet->tablet_id() << ", err: " << st;
                 }
             }
             interval = config::generate_compaction_tasks_interval_ms;
@@ -652,7 +652,7 @@ Status StorageEngine::_submit_compaction_task(TabletSharedPtr tablet,
                     "tablet_id={}, compaction_type={}, "
                     "permit={}, current_permit={}, status={}",
                     tablet->tablet_id(), compaction_type, permits, _permit_limiter.usage(),
-                    st.get_error_msg());
+                    st.to_string());
         }
         return st;
     }
@@ -737,7 +737,7 @@ void StorageEngine::_cooldown_tasks_producer_callback() {
             });
 
             if (!st.ok()) {
-                LOG(INFO) << "failed to submit cooldown task, err msg: " << st.get_error_msg();
+                LOG(INFO) << "failed to submit cooldown task, err msg: " << st;
             }
         }
     } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval)));
diff --git a/be/src/olap/options.cpp b/be/src/olap/options.cpp
index 32eacf195a..49665555e9 100644
--- a/be/src/olap/options.cpp
+++ b/be/src/olap/options.cpp
@@ -29,6 +29,7 @@
 #include "util/path_util.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 using std::string;
 using std::vector;
@@ -59,14 +60,14 @@ Status parse_root_path(const string& root_path, StorePath* path) {
     tmp_vec[0].erase(tmp_vec[0].find_last_not_of("/") + 1);
     if (tmp_vec[0].empty() || tmp_vec[0][0] != '/') {
         LOG(WARNING) << "invalid store path. path=" << tmp_vec[0];
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     string canonicalized_path;
     Status status = Env::Default()->canonicalize(tmp_vec[0], &canonicalized_path);
     if (!status.ok()) {
         LOG(WARNING) << "path can not be canonicalized. may be not exist. path=" << tmp_vec[0];
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
     path->path = tmp_vec[0];
 
@@ -105,7 +106,7 @@ Status parse_root_path(const string& root_path, StorePath* path) {
             medium_str = to_upper(value);
         } else {
             LOG(WARNING) << "invalid property of store path, " << tmp_vec[i];
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<INVALID_ARGUMENT>();
         }
     }
 
@@ -114,7 +115,7 @@ Status parse_root_path(const string& root_path, StorePath* path) {
         if (!valid_signed_number<int64_t>(capacity_str) ||
             strtol(capacity_str.c_str(), nullptr, 10) < 0) {
             LOG(WARNING) << "invalid capacity of store path, capacity=" << capacity_str;
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<INVALID_ARGUMENT>();
         }
         path->capacity_bytes = strtol(capacity_str.c_str(), nullptr, 10) * GB_EXCHANGE_BYTE;
     }
@@ -129,7 +130,7 @@ Status parse_root_path(const string& root_path, StorePath* path) {
             path->storage_medium = TStorageMedium::REMOTE_CACHE;
         } else {
             LOG(WARNING) << "invalid storage medium. medium=" << medium_str;
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<INVALID_ARGUMENT>();
         }
     }
 
@@ -149,7 +150,7 @@ Status parse_conf_store_paths(const string& config_path, std::vector<StorePath>*
     }
     if (paths->empty() || (path_vec.size() != paths->size() && !config::ignore_broken_disk)) {
         LOG(WARNING) << "fail to parse storage_root_path config. value=[" << config_path << "]";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
     return Status::OK();
 }
diff --git a/be/src/olap/push_handler.cpp b/be/src/olap/push_handler.cpp
index e43092a359..b99e1f4952 100644
--- a/be/src/olap/push_handler.cpp
+++ b/be/src/olap/push_handler.cpp
@@ -35,6 +35,7 @@
 #include "runtime/exec_env.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 // Process push command, the main logical is as follows:
 //    a. related tablets not exist:
@@ -84,12 +85,12 @@ Status PushHandler::_do_streaming_ingestion(TabletSharedPtr tablet, const TPushR
     // add transaction in engine, then check sc status
     // lock, prevent sc handler checking transaction concurrently
     if (tablet == nullptr) {
-        return Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND);
+        return Status::Error<TABLE_NOT_FOUND>();
     }
 
     std::shared_lock base_migration_rlock(tablet->get_migration_lock(), std::try_to_lock);
     if (!base_migration_rlock.owns_lock()) {
-        return Status::OLAPInternalError(OLAP_ERR_RWLOCK_ERROR);
+        return Status::Error<TRY_LOCK_FAILED>();
     }
     PUniqueId load_id;
     load_id.set_hi(0);
@@ -131,7 +132,7 @@ Status PushHandler::_do_streaming_ingestion(TabletSharedPtr tablet, const TPushR
         LOG(WARNING) << "failed to push data. version count: " << tablet->version_count()
                      << ", exceed limit: " << config::max_tablet_version_num
                      << ". tablet: " << tablet->full_name();
-        return Status::OLAPInternalError(OLAP_ERR_TOO_MANY_VERSION);
+        return Status::Status::Error<TOO_MANY_VERSION>();
     }
     auto tablet_schema = std::make_shared<TabletSchema>();
     tablet_schema->copy_from(*tablet->tablet_schema());
@@ -172,8 +173,7 @@ Status PushHandler::_do_streaming_ingestion(TabletSharedPtr tablet, const TPushR
     }
     Status commit_status = StorageEngine::instance()->txn_manager()->commit_txn(
             request.partition_id, tablet, request.transaction_id, load_id, rowset_to_add, false);
-    if (commit_status != Status::OK() &&
-        commit_status.precise_code() != OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST) {
+    if (commit_status != Status::OK() && !commit_status.is<PUSH_TRANSACTION_ALREADY_EXIST>()) {
         res = commit_status;
     }
     return res;
@@ -215,7 +215,7 @@ Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, RowsetSharedPtr* cur
             std::unique_ptr<PushBrokerReader> reader(new (std::nothrow) PushBrokerReader());
             if (reader == nullptr) {
                 LOG(WARNING) << "fail to create reader. tablet=" << cur_tablet->full_name();
-                res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+                res = Status::Error<MEM_ALLOC_FAILED>();
                 break;
             }
 
@@ -223,7 +223,7 @@ Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, RowsetSharedPtr* cur
             std::unique_ptr<Schema> schema(new (std::nothrow) Schema(tablet_schema));
             if (schema == nullptr) {
                 LOG(WARNING) << "fail to create schema. tablet=" << cur_tablet->full_name();
-                res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+                res = Status::Error<MEM_ALLOC_FAILED>();
                 break;
             }
 
@@ -232,7 +232,7 @@ Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, RowsetSharedPtr* cur
                                      _request.desc_tbl))) {
                 LOG(WARNING) << "fail to init reader. res=" << res
                              << ", tablet=" << cur_tablet->full_name();
-                res = Status::OLAPInternalError(OLAP_ERR_PUSH_INIT_ERROR);
+                res = Status::Error<PUSH_INIT_ERROR>();
                 break;
             }
 
@@ -278,7 +278,7 @@ Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, RowsetSharedPtr* cur
         *cur_rowset = rowset_writer->build();
         if (*cur_rowset == nullptr) {
             LOG(WARNING) << "fail to build rowset";
-            res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+            res = Status::Error<MEM_ALLOC_FAILED>();
             break;
         }
 
@@ -312,7 +312,7 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, RowsetSharedPtr* cur_ro
             if (!(res = raw_file.init(_request.http_file_path.c_str()))) {
                 LOG(WARNING) << "failed to read raw file. res=" << res
                              << ", file=" << _request.http_file_path;
-                res = Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+                res = Status::Error<INVALID_ARGUMENT>();
                 break;
             }
 
@@ -325,7 +325,7 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, RowsetSharedPtr* cur_ro
 #ifndef DORIS_WITH_LZO
             if (need_decompress) {
                 // if lzo is disabled, compressed data is not allowed here
-                res = Status::OLAPInternalError(OLAP_ERR_VERSION_ALREADY_MERGED);
+                res = Status::Error<VERSION_ALREADY_MERGED>();
                 break;
             }
 #endif
@@ -334,7 +334,7 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, RowsetSharedPtr* cur_ro
             if (reader == nullptr) {
                 LOG(WARNING) << "fail to create reader. tablet=" << cur_tablet->full_name()
                              << ", file=" << _request.http_file_path;
-                res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+                res = Status::Error<MEM_ALLOC_FAILED>();
                 break;
             }
 
@@ -343,7 +343,7 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, RowsetSharedPtr* cur_ro
                 LOG(WARNING) << "fail to init reader. res=" << res
                              << ", tablet=" << cur_tablet->full_name()
                              << ", file=" << _request.http_file_path;
-                res = Status::OLAPInternalError(OLAP_ERR_PUSH_INIT_ERROR);
+                res = Status::Error<PUSH_INIT_ERROR>();
                 break;
             }
         }
@@ -393,7 +393,7 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, RowsetSharedPtr* cur_ro
 
             if (!reader->validate_checksum()) {
                 LOG(WARNING) << "pushed delta file has wrong checksum.";
-                res = Status::OLAPInternalError(OLAP_ERR_PUSH_BUILD_DELTA_ERROR);
+                res = Status::Error<PUSH_BUILD_DELTA_ERROR>();
                 break;
             }
         }
@@ -406,7 +406,7 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, RowsetSharedPtr* cur_ro
 
         if (*cur_rowset == nullptr) {
             LOG(WARNING) << "fail to build rowset";
-            res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+            res = Status::Error<MEM_ALLOC_FAILED>();
             break;
         }
 
@@ -424,14 +424,14 @@ Status BinaryFile::init(const char* path) {
     // open file
     if (!open(path, "rb")) {
         LOG(WARNING) << "fail to open file. file=" << path;
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     // load header
     if (!_header.unserialize(this)) {
         LOG(WARNING) << "fail to read file header. file=" << path;
         close();
-        return Status::OLAPInternalError(OLAP_ERR_PUSH_INIT_ERROR);
+        return Status::Error<PUSH_INIT_ERROR>();
     }
 
     return Status::OK();
@@ -462,13 +462,13 @@ Status BinaryReader::init(TabletSchemaSPtr tablet_schema, BinaryFile* file) {
         _row_buf = new (std::nothrow) char[_row_buf_size];
         if (_row_buf == nullptr) {
             LOG(WARNING) << "fail to malloc one row buf. size=" << _row_buf_size;
-            res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+            res = Status::Error<MEM_ALLOC_FAILED>();
             break;
         }
 
         if (-1 == _file->seek(_file->header_size(), SEEK_SET)) {
             LOG(WARNING) << "skip header, seek fail.";
-            res = Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+            res = Status::Error<IO_ERROR>();
             break;
         }
 
@@ -493,7 +493,7 @@ Status BinaryReader::next(RowCursor* row) {
 
     if (!_ready || nullptr == row) {
         // Here i assume _ready means all states were set up correctly
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     const TabletSchema& schema = *_tablet_schema;
@@ -539,7 +539,7 @@ Status BinaryReader::next(RowCursor* row) {
                 LOG(WARNING) << "invalid data length for VARCHAR! "
                              << "max_len=" << column.length() - sizeof(VarcharLengthType)
                              << ", real_len=" << field_size;
-                return Status::OLAPInternalError(OLAP_ERR_PUSH_INPUT_DATA_ERROR);
+                return Status::Error<PUSH_INPUT_DATA_ERROR>();
             }
         } else if (column.type() == OLAP_FIELD_TYPE_STRING) {
             // Read string length buffer first
@@ -555,7 +555,7 @@ Status BinaryReader::next(RowCursor* row) {
                 LOG(WARNING) << "invalid data length for string! "
                              << "max_len=" << column.length() - sizeof(StringLengthType)
                              << ", real_len=" << field_size;
-                return Status::OLAPInternalError(OLAP_ERR_PUSH_INPUT_DATA_ERROR);
+                return Status::Error<PUSH_INPUT_DATA_ERROR>();
             }
         } else {
             field_size = column.length();
@@ -604,13 +604,13 @@ Status LzoBinaryReader::init(TabletSchemaSPtr tablet_schema, BinaryFile* file) {
         _row_info_buf = new (std::nothrow) char[row_info_buf_size];
         if (_row_info_buf == nullptr) {
             LOG(WARNING) << "fail to malloc rows info buf. size=" << row_info_buf_size;
-            res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+            res = Status::Error<MEM_ALLOC_FAILED>();
             break;
         }
 
         if (-1 == _file->seek(_file->header_size(), SEEK_SET)) {
             LOG(WARNING) << "skip header, seek fail.";
-            res = Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+            res = Status::Error<IO_ERROR>();
             break;
         }
 
@@ -637,7 +637,7 @@ Status LzoBinaryReader::next(RowCursor* row) {
 
     if (!_ready || nullptr == row) {
         // Here i assume _ready means all states were set up correctly
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     if (_row_num == 0) {
@@ -681,7 +681,7 @@ Status LzoBinaryReader::next(RowCursor* row) {
                 LOG(WARNING) << "invalid data length for VARCHAR! "
                              << "max_len=" << column.length() - sizeof(VarcharLengthType)
                              << ", real_len=" << field_size;
-                return Status::OLAPInternalError(OLAP_ERR_PUSH_INPUT_DATA_ERROR);
+                return Status::Error<PUSH_INPUT_DATA_ERROR>();
             }
         } else if (column.type() == OLAP_FIELD_TYPE_STRING) {
             // Get string field size
@@ -692,7 +692,7 @@ Status LzoBinaryReader::next(RowCursor* row) {
                 LOG(WARNING) << "invalid data length for string! "
                              << "max_len=" << column.length() - sizeof(StringLengthType)
                              << ", real_len=" << field_size;
-                return Status::OLAPInternalError(OLAP_ERR_PUSH_INPUT_DATA_ERROR);
+                return Status::Error<PUSH_INPUT_DATA_ERROR>();
             }
         } else {
             field_size = column.length();
@@ -742,7 +742,7 @@ Status LzoBinaryReader::_next_block() {
         _row_buf = new (std::nothrow) char[_max_row_buf_size];
         if (_row_buf == nullptr) {
             LOG(WARNING) << "fail to malloc rows buf. size=" << _max_row_buf_size;
-            res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+            res = Status::Error<MEM_ALLOC_FAILED>();
             return res;
         }
     }
@@ -755,7 +755,7 @@ Status LzoBinaryReader::_next_block() {
         _row_compressed_buf = new (std::nothrow) char[_max_compressed_buf_size];
         if (_row_compressed_buf == nullptr) {
             LOG(WARNING) << "fail to malloc rows compressed buf. size=" << _max_compressed_buf_size;
-            res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+            res = Status::Error<MEM_ALLOC_FAILED>();
             return res;
         }
     }
@@ -803,14 +803,14 @@ Status PushBrokerReader::init(const Schema* schema, const TBrokerScanRange& t_sc
     DescriptorTbl* desc_tbl = nullptr;
     Status status = DescriptorTbl::create(_runtime_state->obj_pool(), t_desc_tbl, &desc_tbl);
     if (UNLIKELY(!status.ok())) {
-        LOG(WARNING) << "Failed to create descriptor table, msg: " << status.get_error_msg();
-        return Status::OLAPInternalError(OLAP_ERR_PUSH_INIT_ERROR);
+        LOG(WARNING) << "Failed to create descriptor table, msg: " << status;
+        return Status::Error<PUSH_INIT_ERROR>();
     }
     _runtime_state->set_desc_tbl(desc_tbl);
     status = _runtime_state->init_mem_trackers(dummy_id);
     if (UNLIKELY(!status.ok())) {
-        LOG(WARNING) << "Failed to init mem trackers, msg: " << status.get_error_msg();
-        return Status::OLAPInternalError(OLAP_ERR_PUSH_INIT_ERROR);
+        LOG(WARNING) << "Failed to init mem trackers, msg: " << status;
+        return Status::Error<PUSH_INIT_ERROR>();
     }
     _runtime_profile = _runtime_state->runtime_profile();
     _runtime_profile->set_name("PushBrokerReader");
@@ -827,13 +827,13 @@ Status PushBrokerReader::init(const Schema* schema, const TBrokerScanRange& t_sc
         break;
     default:
         LOG(WARNING) << "Unsupported file format type: " << t_scan_range.ranges[0].format_type;
-        return Status::OLAPInternalError(OLAP_ERR_PUSH_INIT_ERROR);
+        return Status::Error<PUSH_INIT_ERROR>();
     }
     _scanner.reset(scanner);
     status = _scanner->open();
     if (UNLIKELY(!status.ok())) {
-        LOG(WARNING) << "Failed to open scanner, msg: " << status.get_error_msg();
-        return Status::OLAPInternalError(OLAP_ERR_PUSH_INIT_ERROR);
+        LOG(WARNING) << "Failed to open scanner, msg: " << status;
+        return Status::Error<PUSH_INIT_ERROR>();
     }
 
     // init tuple
@@ -842,14 +842,14 @@ Status PushBrokerReader::init(const Schema* schema, const TBrokerScanRange& t_sc
     if (_tuple_desc == nullptr) {
         std::stringstream ss;
         LOG(WARNING) << "Failed to get tuple descriptor, tuple_id: " << tuple_id;
-        return Status::OLAPInternalError(OLAP_ERR_PUSH_INIT_ERROR);
+        return Status::Error<PUSH_INIT_ERROR>();
     }
 
     int tuple_buffer_size = _tuple_desc->byte_size();
     void* tuple_buffer = _mem_pool->allocate(tuple_buffer_size);
     if (tuple_buffer == nullptr) {
         LOG(WARNING) << "Allocate memory for tuple failed";
-        return Status::OLAPInternalError(OLAP_ERR_PUSH_INIT_ERROR);
+        return Status::Error<PUSH_INIT_ERROR>();
     }
     _tuple = reinterpret_cast<Tuple*>(tuple_buffer);
 
@@ -917,7 +917,7 @@ Status PushBrokerReader::fill_field_row(RowCursorCell* dst, const char* src, boo
         break;
     }
     default:
-        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        return Status::Error<INVALID_SCHEMA>();
     }
 
     return Status::OK();
@@ -925,7 +925,7 @@ Status PushBrokerReader::fill_field_row(RowCursorCell* dst, const char* src, boo
 
 Status PushBrokerReader::next(ContiguousRow* row) {
     if (!_ready || row == nullptr) {
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     memset(_tuple, 0, _tuple_desc->num_null_bytes());
@@ -933,7 +933,7 @@ Status PushBrokerReader::next(ContiguousRow* row) {
     Status status = _scanner->get_next(_tuple, _mem_pool.get(), &_eof, &_fill_tuple);
     if (UNLIKELY(!status.ok())) {
         LOG(WARNING) << "Scanner get next tuple failed";
-        return Status::OLAPInternalError(OLAP_ERR_PUSH_INPUT_DATA_ERROR);
+        return Status::Error<PUSH_INPUT_DATA_ERROR>();
     }
     if (_eof || !_fill_tuple) {
         return Status::OK();
@@ -953,7 +953,7 @@ Status PushBrokerReader::next(ContiguousRow* row) {
         if (field_status != Status::OK()) {
             LOG(WARNING) << "fill field row failed in spark load, slot index: " << i
                          << ", type: " << type;
-            return Status::OLAPInternalError(OLAP_ERR_SCHEMA_SCHEMA_FIELD_INVALID);
+            return Status::Error<SCHEMA_SCHEMA_FIELD_INVALID>();
         }
     }
 
diff --git a/be/src/olap/reader.cpp b/be/src/olap/reader.cpp
index fcc2f9b6dc..1e42f0a040 100644
--- a/be/src/olap/reader.cpp
+++ b/be/src/olap/reader.cpp
@@ -32,6 +32,7 @@
 #include "runtime/mem_pool.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 void TabletReader::ReaderParams::check_validation() const {
     if (UNLIKELY(version.first == -1)) {
@@ -316,7 +317,7 @@ Status TabletReader::_init_return_columns(const ReaderParams& read_params) {
     } else {
         LOG(WARNING) << "fail to init return columns. [reader_type=" << read_params.reader_type
                      << " return_columns_size=" << read_params.return_columns.size() << "]";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     std::sort(_key_cids.begin(), _key_cids.end(), std::greater<uint32_t>());
@@ -342,7 +343,7 @@ Status TabletReader::_init_keys_param(const ReaderParams& read_params) {
                 << "Input param are invalid. Column count is bigger than num_columns of schema. "
                 << "column_count=" << scan_key_size
                 << ", schema.num_columns=" << _tablet_schema->num_columns();
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     std::vector<uint32_t> columns(scan_key_size);
@@ -355,7 +356,7 @@ Status TabletReader::_init_keys_param(const ReaderParams& read_params) {
             LOG(WARNING) << "The start_key.at(" << i
                          << ").size == " << read_params.start_key[i].size() << ", not equals the "
                          << scan_key_size;
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<INVALID_ARGUMENT>();
         }
 
         Status res = _keys_param.start_keys[i].init_scan_key(
@@ -378,7 +379,7 @@ Status TabletReader::_init_keys_param(const ReaderParams& read_params) {
         if (read_params.end_key[i].size() != scan_key_size) {
             LOG(WARNING) << "The end_key.at(" << i << ").size == " << read_params.end_key[i].size()
                          << ", not equals the " << scan_key_size;
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<INVALID_ARGUMENT>();
         }
 
         Status res = _keys_param.end_keys[i].init_scan_key(_tablet_schema,
@@ -421,7 +422,7 @@ Status TabletReader::_init_orderby_keys_param(const ReaderParams& read_params) {
             LOG(WARNING) << "read_orderby_key_num_prefix_columns != _orderby_key_columns.size "
                          << read_params.read_orderby_key_num_prefix_columns << " vs. "
                          << _orderby_key_columns.size();
-            return Status::OLAPInternalError(OLAP_ERR_OTHER_ERROR);
+            return Status::Error<ErrorCode::INTERNAL_ERROR>();
         }
     }
 
diff --git a/be/src/olap/reader.h b/be/src/olap/reader.h
index 131d04e028..0ea2e7c757 100644
--- a/be/src/olap/reader.h
+++ b/be/src/olap/reader.h
@@ -121,20 +121,20 @@ public:
     virtual Status init(const ReaderParams& read_params);
 
     // Read next row with aggregation.
-    // Return OLAP_SUCCESS and set `*eof` to false when next row is read into `row_cursor`.
-    // Return OLAP_SUCCESS and set `*eof` to true when no more rows can be read.
+    // Return OK and set `*eof` to false when next row is read into `row_cursor`.
+    // Return OK and set `*eof` to true when no more rows can be read.
     // Return others when unexpected error happens.
     virtual Status next_row_with_aggregation(RowCursor* row_cursor, MemPool* mem_pool,
                                              ObjectPool* agg_pool, bool* eof) = 0;
 
     // Read next block with aggregation.
-    // Return OLAP_SUCCESS and set `*eof` to false when next block is read
-    // Return OLAP_SUCCESS and set `*eof` to true when no more rows can be read.
+    // Return OK and set `*eof` to false when next block is read
+    // Return OK and set `*eof` to true when no more rows can be read.
     // Return others when unexpected error happens.
     // TODO: Rethink here we still need mem_pool and agg_pool?
     virtual Status next_block_with_aggregation(vectorized::Block* block, MemPool* mem_pool,
                                                ObjectPool* agg_pool, bool* eof) {
-        return Status::OLAPInternalError(OLAP_ERR_READER_INITIALIZE_ERROR);
+        return Status::Error<ErrorCode::READER_INITIALIZE_ERROR>();
     }
 
     virtual uint64_t merged_rows() const { return _merged_rows; }
diff --git a/be/src/olap/row_block.cpp b/be/src/olap/row_block.cpp
index 2d66985933..12e0dce044 100644
--- a/be/src/olap/row_block.cpp
+++ b/be/src/olap/row_block.cpp
@@ -36,6 +36,7 @@ using std::upper_bound;
 using std::vector;
 
 namespace doris {
+using namespace ErrorCode;
 
 RowBlock::RowBlock(TabletSchemaSPtr schema) : _capacity(0), _schema(schema) {
     _mem_pool.reset(new MemPool());
@@ -65,7 +66,7 @@ Status RowBlock::finalize(uint32_t row_num) {
         LOG(WARNING) << "Input row num is larger than internal row num."
                         "[row_num="
                      << row_num << "; _info.row_num=" << _info.row_num << "]";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
     _info.row_num = row_num;
     return Status::OK();
diff --git a/be/src/olap/row_cursor.cpp b/be/src/olap/row_cursor.cpp
index 48233830c6..e9fe2e9396 100644
--- a/be/src/olap/row_cursor.cpp
+++ b/be/src/olap/row_cursor.cpp
@@ -28,6 +28,7 @@ using std::string;
 using std::vector;
 
 namespace doris {
+using namespace ErrorCode;
 RowCursor::RowCursor()
         : _fixed_len(0), _variable_len(0), _string_field_count(0), _long_text_buf(nullptr) {}
 
@@ -47,7 +48,7 @@ Status RowCursor::_init(const std::vector<uint32_t>& columns) {
     for (auto cid : columns) {
         if (_schema->column(cid) == nullptr) {
             LOG(WARNING) << "Fail to create field.";
-            return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+            return Status::Error<INIT_FAILED>();
         }
         _variable_len += column_schema(cid)->get_variable_len();
         if (_schema->column(cid)->type() == OLAP_FIELD_TYPE_STRING) {
@@ -59,7 +60,7 @@ Status RowCursor::_init(const std::vector<uint32_t>& columns) {
     _fixed_buf = new (nothrow) char[_fixed_len]();
     if (_fixed_buf == nullptr) {
         LOG(WARNING) << "Fail to malloc _fixed_buf.";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+        return Status::Error<MEM_ALLOC_FAILED>();
     }
     _owned_fixed_buf = _fixed_buf;
 
@@ -140,7 +141,7 @@ Status RowCursor::init(TabletSchemaSPtr schema, size_t column_count) {
                 << "Input param are invalid. Column count is bigger than num_columns of schema. "
                 << "column_count=" << column_count
                 << ", schema.num_columns=" << schema->num_columns();
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     std::vector<uint32_t> columns;
@@ -156,7 +157,7 @@ Status RowCursor::init(const std::vector<TabletColumn>& schema, size_t column_co
         LOG(WARNING)
                 << "Input param are invalid. Column count is bigger than num_columns of schema. "
                 << "column_count=" << column_count << ", schema.num_columns=" << schema.size();
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     std::vector<uint32_t> columns;
@@ -180,7 +181,7 @@ Status RowCursor::init_scan_key(TabletSchemaSPtr schema,
                 << "Input param are invalid. Column count is bigger than num_columns of schema. "
                 << "column_count=" << scan_key_size
                 << ", schema.num_columns=" << schema->num_columns();
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     std::vector<uint32_t> columns(scan_key_size);
@@ -258,7 +259,7 @@ Status RowCursor::from_tuple(const OlapTuple& tuple) {
     if (tuple.size() != _schema->num_column_ids()) {
         LOG(WARNING) << "column count does not match. tuple_size=" << tuple.size()
                      << ", field_count=" << _schema->num_column_ids();
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     for (size_t i = 0; i < tuple.size(); ++i) {
@@ -328,19 +329,19 @@ Status RowCursor::_alloc_buf() {
     _variable_buf = new (nothrow) char[_variable_len]();
     if (_variable_buf == nullptr) {
         LOG(WARNING) << "Fail to malloc _variable_buf.";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+        return Status::Error<MEM_ALLOC_FAILED>();
     }
     if (_string_field_count > 0) {
         _long_text_buf = (char**)malloc(_string_field_count * sizeof(char*));
         if (_long_text_buf == nullptr) {
             LOG(WARNING) << "Fail to malloc _long_text_buf.";
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+            return Status::Error<MEM_ALLOC_FAILED>();
         }
         for (int i = 0; i < _string_field_count; ++i) {
             _long_text_buf[i] = (char*)malloc(DEFAULT_TEXT_LENGTH * sizeof(char));
             if (_long_text_buf[i] == nullptr) {
                 LOG(WARNING) << "Fail to malloc _long_text_buf.";
-                return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+                return Status::Error<MEM_ALLOC_FAILED>();
             }
         }
     }
diff --git a/be/src/olap/rowset/beta_rowset.cpp b/be/src/olap/rowset/beta_rowset.cpp
index 6fbe708090..1af1ddb50f 100644
--- a/be/src/olap/rowset/beta_rowset.cpp
+++ b/be/src/olap/rowset/beta_rowset.cpp
@@ -34,6 +34,7 @@
 #include "util/doris_metrics.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 using io::FileCacheManager;
 
@@ -112,7 +113,7 @@ Status BetaRowset::do_load(bool /*use_cache*/) {
 Status BetaRowset::get_segments_size(std::vector<size_t>* segments_size) {
     auto fs = _rowset_meta->fs();
     if (!fs || _schema == nullptr) {
-        return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+        return Status::Error<INIT_FAILED>();
     }
     for (int seg_id = 0; seg_id < num_segments(); ++seg_id) {
         auto seg_path = segment_file_path(seg_id);
@@ -125,7 +126,7 @@ Status BetaRowset::get_segments_size(std::vector<size_t>* segments_size) {
 Status BetaRowset::load_segments(std::vector<segment_v2::SegmentSharedPtr>* segments) {
     auto fs = _rowset_meta->fs();
     if (!fs || _schema == nullptr) {
-        return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+        return Status::Error<INIT_FAILED>();
     }
     for (int seg_id = 0; seg_id < num_segments(); ++seg_id) {
         auto seg_path = segment_file_path(seg_id);
@@ -136,7 +137,7 @@ Status BetaRowset::load_segments(std::vector<segment_v2::SegmentSharedPtr>* segm
         if (!s.ok()) {
             LOG(WARNING) << "failed to open segment. " << seg_path << " under rowset "
                          << unique_id() << " : " << s.to_string();
-            return Status::OLAPInternalError(OLAP_ERR_ROWSET_LOAD_FAILED);
+            return Status::Error<ROWSET_LOAD_FAILED>();
         }
         segments->push_back(std::move(segment));
     }
@@ -147,7 +148,7 @@ Status BetaRowset::load_segment(int64_t seg_id, segment_v2::SegmentSharedPtr* se
     DCHECK(seg_id >= 0);
     auto fs = _rowset_meta->fs();
     if (!fs || _schema == nullptr) {
-        return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+        return Status::Error<INIT_FAILED>();
     }
     auto seg_path = segment_file_path(seg_id);
     auto cache_path = segment_cache_path(seg_id);
@@ -156,7 +157,7 @@ Status BetaRowset::load_segment(int64_t seg_id, segment_v2::SegmentSharedPtr* se
     if (!s.ok()) {
         LOG(WARNING) << "failed to open segment. " << seg_path << " under rowset " << unique_id()
                      << " : " << s.to_string();
-        return Status::OLAPInternalError(OLAP_ERR_ROWSET_LOAD_FAILED);
+        return Status::Error<ROWSET_LOAD_FAILED>();
     }
     return Status::OK();
 }
@@ -182,7 +183,7 @@ Status BetaRowset::remove() {
                 << ", tabletid:" << _rowset_meta->tablet_id();
     auto fs = _rowset_meta->fs();
     if (!fs) {
-        return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+        return Status::Error<INIT_FAILED>();
     }
     bool success = true;
     Status st;
@@ -201,7 +202,7 @@ Status BetaRowset::remove() {
     }
     if (!success) {
         LOG(WARNING) << "failed to remove files in rowset " << unique_id();
-        return Status::OLAPInternalError(OLAP_ERR_ROWSET_DELETE_FILE_FAILED);
+        return Status::Error<ROWSET_DELETE_FILE_FAILED>();
     }
     return Status::OK();
 }
@@ -215,7 +216,7 @@ Status BetaRowset::link_files_to(const std::string& dir, RowsetId new_rowset_id,
     DCHECK(is_local());
     auto fs = _rowset_meta->fs();
     if (!fs) {
-        return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+        return Status::Error<INIT_FAILED>();
     }
     for (int i = 0; i < num_segments(); ++i) {
         auto dst_path = segment_file_path(dir, new_rowset_id, i + new_rowset_start_seg_id);
@@ -223,7 +224,7 @@ Status BetaRowset::link_files_to(const std::string& dir, RowsetId new_rowset_id,
         bool dst_path_exist = false;
         if (!fs->exists(dst_path, &dst_path_exist).ok() || dst_path_exist) {
             LOG(WARNING) << "failed to create hard link, file already exist: " << dst_path;
-            return Status::OLAPInternalError(OLAP_ERR_FILE_ALREADY_EXIST);
+            return Status::Error<FILE_ALREADY_EXIST>();
         }
         auto src_path = segment_file_path(i);
         // TODO(lingbin): how external storage support link?
@@ -231,7 +232,7 @@ Status BetaRowset::link_files_to(const std::string& dir, RowsetId new_rowset_id,
         if (!fs->link_file(src_path, dst_path).ok()) {
             LOG(WARNING) << "fail to create hard link. from=" << src_path << ", "
                          << "to=" << dst_path << ", errno=" << Errno::no();
-            return Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+            return Status::Error<OS_ERROR>();
         }
     }
     return Status::OK();
@@ -244,17 +245,17 @@ Status BetaRowset::copy_files_to(const std::string& dir, const RowsetId& new_row
         Status status = Env::Default()->path_exists(dst_path);
         if (status.ok()) {
             LOG(WARNING) << "file already exist: " << dst_path;
-            return Status::OLAPInternalError(OLAP_ERR_FILE_ALREADY_EXIST);
+            return Status::Error<FILE_ALREADY_EXIST>();
         }
-        if (!status.is_not_found()) {
+        if (!status.is<NOT_FOUND>()) {
             LOG(WARNING) << "file check exist error: " << dst_path;
-            return Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+            return Status::Error<OS_ERROR>();
         }
         auto src_path = segment_file_path(i);
         if (!Env::Default()->copy_path(src_path, dst_path).ok()) {
             LOG(WARNING) << "fail to copy file. from=" << src_path << ", to=" << dst_path
                          << ", errno=" << Errno::no();
-            return Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+            return Status::Error<OS_ERROR>();
         }
     }
     return Status::OK();
diff --git a/be/src/olap/rowset/beta_rowset_reader.cpp b/be/src/olap/rowset/beta_rowset_reader.cpp
index eae3f403d1..f70d84ae6f 100644
--- a/be/src/olap/rowset/beta_rowset_reader.cpp
+++ b/be/src/olap/rowset/beta_rowset_reader.cpp
@@ -30,6 +30,7 @@
 #include "vec/olap/vgeneric_iterators.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 BetaRowsetReader::BetaRowsetReader(BetaRowsetSharedPtr rowset)
         : _context(nullptr), _rowset(std::move(rowset)), _stats(&_owned_stats) {
@@ -173,7 +174,7 @@ Status BetaRowsetReader::get_segment_iterators(RowsetReaderContext* read_context
         auto s = seg_ptr->new_iterator(*_input_schema, _read_options, &iter);
         if (!s.ok()) {
             LOG(WARNING) << "failed to create iterator[" << seg_ptr->id() << "]: " << s.to_string();
-            return Status::OLAPInternalError(OLAP_ERR_ROWSET_READER_INIT);
+            return Status::Error<ROWSET_READER_INIT>();
         }
         seg_iterators.push_back(std::move(iter));
     }
@@ -182,7 +183,7 @@ Status BetaRowsetReader::get_segment_iterators(RowsetReaderContext* read_context
         auto st = owned_it->init(_read_options);
         if (!st.ok()) {
             LOG(WARNING) << "failed to init iterator: " << st.to_string();
-            return Status::OLAPInternalError(OLAP_ERR_ROWSET_READER_INIT);
+            return Status::Error<ROWSET_READER_INIT>();
         }
         // transfer ownership of segment iterator to `_iterator`
         out_iters->push_back(owned_it.release());
@@ -223,7 +224,7 @@ Status BetaRowsetReader::init(RowsetReaderContext* read_context) {
     auto s = final_iterator->init(_read_options);
     if (!s.ok()) {
         LOG(WARNING) << "failed to init iterator: " << s.to_string();
-        return Status::OLAPInternalError(OLAP_ERR_ROWSET_READER_INIT);
+        return Status::Error<ROWSET_READER_INIT>();
     }
     _iterator.reset(final_iterator);
 
@@ -272,12 +273,12 @@ Status BetaRowsetReader::next_block(RowBlock** block) {
     {
         auto s = _iterator->next_batch(_input_block.get());
         if (!s.ok()) {
-            if (s.is_end_of_file()) {
+            if (s.is<END_OF_FILE>()) {
                 *block = nullptr;
-                return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+                return Status::Error<END_OF_FILE>();
             }
             LOG(WARNING) << "failed to read next block: " << s.to_string();
-            return Status::OLAPInternalError(OLAP_ERR_ROWSET_READ_FAILED);
+            return Status::Error<ROWSET_READ_FAILED>();
         }
     }
 
@@ -297,11 +298,11 @@ Status BetaRowsetReader::next_block(vectorized::Block* block) {
         do {
             auto s = _iterator->next_batch(block);
             if (!s.ok()) {
-                if (s.is_end_of_file()) {
-                    return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+                if (s.is<END_OF_FILE>()) {
+                    return Status::Error<END_OF_FILE>();
                 } else {
                     LOG(WARNING) << "failed to read next block: " << s.to_string();
-                    return Status::OLAPInternalError(OLAP_ERR_ROWSET_READ_FAILED);
+                    return Status::Error<ROWSET_READ_FAILED>();
                 }
             }
         } while (block->rows() == 0);
@@ -315,15 +316,15 @@ Status BetaRowsetReader::next_block(vectorized::Block* block) {
                 {
                     auto s = _iterator->next_batch(_input_block.get());
                     if (!s.ok()) {
-                        if (s.is_end_of_file()) {
+                        if (s.is<END_OF_FILE>()) {
                             if (is_first) {
-                                return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+                                return Status::Error<END_OF_FILE>();
                             } else {
                                 break;
                             }
                         } else {
                             LOG(WARNING) << "failed to read next block: " << s.to_string();
-                            return Status::OLAPInternalError(OLAP_ERR_ROWSET_READ_FAILED);
+                            return Status::Error<ROWSET_READ_FAILED>();
                         }
                     } else if (_input_block->selected_size() == 0) {
                         continue;
@@ -336,7 +337,7 @@ Status BetaRowsetReader::next_block(vectorized::Block* block) {
                 auto s = _input_block->convert_to_vec_block(block);
                 if (UNLIKELY(!s.ok())) {
                     LOG(WARNING) << "failed to read next block: " << s.to_string();
-                    return Status::OLAPInternalError(OLAP_ERR_STRING_OVERFLOW_IN_VEC_ENGINE);
+                    return Status::Error<STRING_OVERFLOW_IN_VEC_ENGINE>();
                 }
             }
             is_first = false;
@@ -353,11 +354,11 @@ Status BetaRowsetReader::next_block_view(vectorized::BlockView* block_view) {
         do {
             auto s = _iterator->next_block_view(block_view);
             if (!s.ok()) {
-                if (s.is_end_of_file()) {
-                    return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+                if (s.is<END_OF_FILE>()) {
+                    return Status::Error<END_OF_FILE>();
                 } else {
                     LOG(WARNING) << "failed to read next block: " << s.to_string();
-                    return Status::OLAPInternalError(OLAP_ERR_ROWSET_READ_FAILED);
+                    return Status::Error<ROWSET_READ_FAILED>();
                 }
             }
         } while (block_view->empty());
diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp b/be/src/olap/rowset/beta_rowset_writer.cpp
index 6f8bb656c5..ac5bf223e5 100644
--- a/be/src/olap/rowset/beta_rowset_writer.cpp
+++ b/be/src/olap/rowset/beta_rowset_writer.cpp
@@ -38,6 +38,7 @@
 #include "runtime/memory/mem_tracker_limiter.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 class StorageEngine;
 
@@ -52,7 +53,7 @@ BetaRowsetWriter::BetaRowsetWriter()
           _total_index_size(0),
           _raw_num_rows_written(0),
           _is_doing_segcompaction(false) {
-    _segcompaction_status.store(OLAP_SUCCESS);
+    _segcompaction_status.store(OK);
 }
 
 BetaRowsetWriter::~BetaRowsetWriter() {
@@ -176,7 +177,7 @@ std::unique_ptr<segment_v2::SegmentWriter> BetaRowsetWriter::_create_segcompacti
 Status BetaRowsetWriter::_delete_original_segments(uint32_t begin, uint32_t end) {
     auto fs = _rowset_meta->fs();
     if (!fs) {
-        return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+        return Status::Error<INIT_FAILED>();
     }
     for (uint32_t i = begin; i <= end; ++i) {
         auto seg_path = BetaRowset::segment_file_path(_context.rowset_dir, _context.rowset_id, i);
@@ -199,7 +200,7 @@ Status BetaRowsetWriter::_rename_compacted_segments(int64_t begin, int64_t end)
     if (ret) {
         LOG(WARNING) << "failed to rename " << src_seg_path << " to " << dst_seg_path
                      << ". ret:" << ret << " errno:" << errno;
-        return Status::OLAPInternalError(OLAP_ERR_ROWSET_RENAME_FILE_FAILED);
+        return Status::Error<ROWSET_RENAME_FILE_FAILED>();
     }
     return Status::OK();
 }
@@ -231,7 +232,7 @@ Status BetaRowsetWriter::_rename_compacted_segment_plain(uint64_t seg_id) {
     if (ret) {
         LOG(WARNING) << "failed to rename " << src_seg_path << " to " << dst_seg_path
                      << ". ret:" << ret << " errno:" << errno;
-        return Status::OLAPInternalError(OLAP_ERR_ROWSET_RENAME_FILE_FAILED);
+        return Status::Error<ROWSET_RENAME_FILE_FAILED>();
     }
     return Status::OK();
 }
@@ -260,14 +261,14 @@ Status BetaRowsetWriter::_check_correctness(std::unique_ptr<OlapReaderStatistics
     if (sum_target_row != stat_read_row) {
         LOG(WARNING) << "read row_num does not match. expect read row:" << sum_target_row
                      << " actual read row:" << stat_read_row;
-        return Status::OLAPInternalError(OLAP_ERR_CHECK_LINES_ERROR);
+        return Status::Error<CHECK_LINES_ERROR>();
     }
 
     uint64_t total_row = row_count + merged_row_stat;
     if (stat_read_row != total_row) {
         LOG(WARNING) << "total row_num does not match. expect total row:" << total_row
                      << " actual total row:" << stat_read_row;
-        return Status::OLAPInternalError(OLAP_ERR_CHECK_LINES_ERROR);
+        return Status::Error<CHECK_LINES_ERROR>();
     }
     return Status::OK();
 }
@@ -277,7 +278,7 @@ Status BetaRowsetWriter::_do_compact_segments(SegCompactionCandidatesSharedPtr s
     // throttle segcompaction task if memory depleted.
     if (MemTrackerLimiter::sys_mem_exceed_limit_check(GB_EXCHANGE_BYTE)) {
         LOG(WARNING) << "skip segcompaction due to memory shortage";
-        return Status::OLAPInternalError(OLAP_ERR_FETCH_MEMORY_EXCEEDED);
+        return Status::Error<FETCH_MEMORY_EXCEEDED>();
     }
     uint64_t begin = (*(segments->begin()))->id();
     uint64_t end = (*(segments->end() - 1))->id();
@@ -291,14 +292,14 @@ Status BetaRowsetWriter::_do_compact_segments(SegCompactionCandidatesSharedPtr s
             _get_segcompaction_reader(segments, schema, stat.get(), &merged_row_stat);
     if (UNLIKELY(reader == nullptr)) {
         LOG(WARNING) << "failed to get segcompaction reader";
-        return Status::OLAPInternalError(OLAP_ERR_SEGCOMPACTION_INIT_READER);
+        return Status::Error<SEGCOMPACTION_INIT_READER>();
     }
     std::unique_ptr<vectorized::VMergeIterator> reader_ptr;
     reader_ptr.reset(reader);
     auto writer = _create_segcompaction_writer(begin, end);
     if (UNLIKELY(writer == nullptr)) {
         LOG(WARNING) << "failed to get segcompaction writer";
-        return Status::OLAPInternalError(OLAP_ERR_SEGCOMPACTION_INIT_WRITER);
+        return Status::Error<SEGCOMPACTION_INIT_WRITER>();
     }
     uint64_t row_count = 0;
     vectorized::Block block = _context.tablet_schema->create_block();
@@ -306,7 +307,7 @@ Status BetaRowsetWriter::_do_compact_segments(SegCompactionCandidatesSharedPtr s
         auto status = reader_ptr->next_batch(&block);
         row_count += block.rows();
         if (status != Status::OK()) {
-            if (LIKELY(status.is_end_of_file())) {
+            if (LIKELY(status.is<END_OF_FILE>())) {
                 RETURN_NOT_OK_LOG(_add_block_for_segcompaction(&block, &writer),
                                   "write block failed");
                 break;
@@ -358,11 +359,11 @@ Status BetaRowsetWriter::_do_compact_segments(SegCompactionCandidatesSharedPtr s
 void BetaRowsetWriter::compact_segments(SegCompactionCandidatesSharedPtr segments) {
     Status status = _do_compact_segments(segments);
     if (!status.ok()) {
-        int16_t errcode = status.precise_code();
+        int16_t errcode = status.code();
         switch (errcode) {
-        case OLAP_ERR_FETCH_MEMORY_EXCEEDED:
-        case OLAP_ERR_SEGCOMPACTION_INIT_READER:
-        case OLAP_ERR_SEGCOMPACTION_INIT_WRITER:
+        case FETCH_MEMORY_EXCEEDED:
+        case SEGCOMPACTION_INIT_READER:
+        case SEGCOMPACTION_INIT_WRITER:
             LOG(WARNING) << "segcompaction failed, try next time:" << status;
             return;
         default:
@@ -370,7 +371,7 @@ void BetaRowsetWriter::compact_segments(SegCompactionCandidatesSharedPtr segment
                          << " tablet_id:" << _context.tablet_id
                          << " rowset_id:" << _context.rowset_id << " status:" << status;
             // status will be checked by the next trigger of segcompaction or the final wait
-            _segcompaction_status.store(OLAP_ERR_OTHER_ERROR);
+            _segcompaction_status.store(ErrorCode::INTERNAL_ERROR);
         }
     }
     DCHECK_EQ(_is_doing_segcompaction, true);
@@ -385,7 +386,7 @@ Status BetaRowsetWriter::_load_noncompacted_segments(
         std::vector<segment_v2::SegmentSharedPtr>* segments) {
     auto fs = _rowset_meta->fs();
     if (!fs) {
-        return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+        return Status::Error<INIT_FAILED>();
     }
     for (int seg_id = _segcompacted_point; seg_id < _num_segment; ++seg_id) {
         auto seg_path =
@@ -397,7 +398,7 @@ Status BetaRowsetWriter::_load_noncompacted_segments(
                                            _context.tablet_schema, &segment);
         if (!s.ok()) {
             LOG(WARNING) << "failed to open segment. " << seg_path << ":" << s.to_string();
-            return Status::OLAPInternalError(OLAP_ERR_ROWSET_LOAD_FAILED);
+            return Status::Error<ROWSET_LOAD_FAILED>();
         }
         segments->push_back(std::move(segment));
     }
@@ -499,8 +500,8 @@ Status BetaRowsetWriter::_segcompaction_if_necessary() {
         !_check_and_set_is_doing_segcompaction()) {
         return status;
     }
-    if (_segcompaction_status.load() != OLAP_SUCCESS) {
-        status = Status::OLAPInternalError(OLAP_ERR_SEGCOMPACTION_FAILED);
+    if (_segcompaction_status.load() != OK) {
+        status = Status::Error<SEGCOMPACTION_FAILED>();
     } else if ((_num_segment - _segcompacted_point) >=
                config::segcompaction_threshold_segment_num) {
         SegCompactionCandidatesSharedPtr segments = std::make_shared<SegCompactionCandidates>();
@@ -529,8 +530,8 @@ Status BetaRowsetWriter::_segcompaction_ramaining_if_necessary() {
     if (!config::enable_segcompaction || !config::enable_storage_vectorization) {
         return Status::OK();
     }
-    if (_segcompaction_status.load() != OLAP_SUCCESS) {
-        return Status::OLAPInternalError(OLAP_ERR_SEGCOMPACTION_FAILED);
+    if (_segcompaction_status.load() != OK) {
+        return Status::Error<SEGCOMPACTION_FAILED>();
     }
     if (!_is_segcompacted() || _segcompacted_point == _num_segment) {
         // no need if never segcompact before or all segcompacted
@@ -573,7 +574,7 @@ Status BetaRowsetWriter::_add_block(const vectorized::Block* block,
         auto s = (*segment_writer)->append_block(block, row_offset, input_row_num);
         if (UNLIKELY(!s.ok())) {
             LOG(WARNING) << "failed to append block: " << s.to_string();
-            return Status::OLAPInternalError(OLAP_ERR_WRITER_DATA_WRITE_ERROR);
+            return Status::Error<WRITER_DATA_WRITE_ERROR>();
         }
         row_offset += input_row_num;
     } while (row_offset < block_row_num);
@@ -588,7 +589,7 @@ Status BetaRowsetWriter::_add_block_for_segcompaction(
     auto s = (*segment_writer)->append_block(block, 0, block->rows());
     if (UNLIKELY(!s.ok())) {
         LOG(WARNING) << "failed to append block: " << s.to_string();
-        return Status::OLAPInternalError(OLAP_ERR_WRITER_DATA_WRITE_ERROR);
+        return Status::Error<WRITER_DATA_WRITE_ERROR>();
     }
     return Status::OK();
 }
@@ -602,7 +603,7 @@ Status BetaRowsetWriter::_add_row(const RowType& row) {
     auto s = _segment_writer->append_row(row);
     if (PREDICT_FALSE(!s.ok())) {
         LOG(WARNING) << "failed to append row: " << s.to_string();
-        return Status::OLAPInternalError(OLAP_ERR_WRITER_DATA_WRITE_ERROR);
+        return Status::Error<WRITER_DATA_WRITE_ERROR>();
     }
     if (PREDICT_FALSE(_segment_writer->estimate_segment_size() >= MAX_SEGMENT_SIZE ||
                       _segment_writer->num_rows_written() >= _context.max_rows_per_segment)) {
@@ -661,7 +662,7 @@ Status BetaRowsetWriter::flush_single_memtable(MemTable* memtable, int64_t* flus
         auto s = writer->append_row(dst_row);
         if (PREDICT_FALSE(!s.ok())) {
             LOG(WARNING) << "failed to append row: " << s.to_string();
-            return Status::OLAPInternalError(OLAP_ERR_WRITER_DATA_WRITE_ERROR);
+            return Status::Error<WRITER_DATA_WRITE_ERROR>();
         }
 
         if (PREDICT_FALSE(writer->estimate_segment_size() >= MAX_SEGMENT_SIZE ||
@@ -710,8 +711,8 @@ Status BetaRowsetWriter::_wait_flying_segcompaction() {
     if (elapsed >= MICROS_PER_SEC) {
         LOG(INFO) << "wait flying segcompaction finish time:" << elapsed << "us";
     }
-    if (_segcompaction_status.load() != OLAP_SUCCESS) {
-        return Status::OLAPInternalError(OLAP_ERR_SEGCOMPACTION_FAILED);
+    if (_segcompaction_status.load() != OK) {
+        return Status::Error<SEGCOMPACTION_FAILED>();
     }
     return Status::OK();
 }
@@ -897,13 +898,12 @@ Status BetaRowsetWriter::_do_create_segment_writer(
     }
     auto fs = _rowset_meta->fs();
     if (!fs) {
-        return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+        return Status::Error<INIT_FAILED>();
     }
     io::FileWriterPtr file_writer;
     Status st = fs->create_file(path, &file_writer);
     if (!st.ok()) {
-        LOG(WARNING) << "failed to create writable file. path=" << path
-                     << ", err: " << st.get_error_msg();
+        LOG(WARNING) << "failed to create writable file. path=" << path << ", err: " << st;
         return st;
     }
 
@@ -948,7 +948,7 @@ Status BetaRowsetWriter::_create_segment_writer(
                    << " _num_segment:" << _num_segment
                    << " _segcompacted_point:" << _segcompacted_point
                    << " _num_segcompacted:" << _num_segcompacted;
-        return Status::OLAPInternalError(OLAP_ERR_TOO_MANY_SEGMENTS);
+        return Status::Error<TOO_MANY_SEGMENTS>();
     } else {
         return _do_create_segment_writer(writer, false, -1, -1);
     }
@@ -972,7 +972,7 @@ Status BetaRowsetWriter::_flush_segment_writer(std::unique_ptr<segment_v2::Segme
     Status s = (*writer)->finalize(&segment_size, &index_size);
     if (!s.ok()) {
         LOG(WARNING) << "failed to finalize segment: " << s.to_string();
-        return Status::OLAPInternalError(OLAP_ERR_WRITER_DATA_WRITE_ERROR);
+        return Status::Error<WRITER_DATA_WRITE_ERROR>();
     }
     KeyBoundsPB key_bounds;
     Slice min_key = (*writer)->min_encoded_key();
diff --git a/be/src/olap/rowset/beta_rowset_writer.h b/be/src/olap/rowset/beta_rowset_writer.h
index 7e1438761b..aa098cc0b8 100644
--- a/be/src/olap/rowset/beta_rowset_writer.h
+++ b/be/src/olap/rowset/beta_rowset_writer.h
@@ -175,7 +175,7 @@ protected:
     std::mutex _is_doing_segcompaction_lock;
     std::condition_variable _segcompacting_cond;
 
-    std::atomic<ErrorCode> _segcompaction_status;
+    std::atomic<int> _segcompaction_status;
 
     fmt::memory_buffer vlog_buffer;
 };
diff --git a/be/src/olap/rowset/rowset.h b/be/src/olap/rowset/rowset.h
index 1e06b4b425..46f99bb723 100644
--- a/be/src/olap/rowset/rowset.h
+++ b/be/src/olap/rowset/rowset.h
@@ -65,7 +65,7 @@ public:
             break;
 
         default:
-            return Status::OLAPInternalError(OLAP_ERR_ROWSET_INVALID_STATE_TRANSITION);
+            return Status::Error<ErrorCode::ROWSET_INVALID_STATE_TRANSITION>();
         }
         return Status::OK();
     }
@@ -81,7 +81,7 @@ public:
             break;
 
         default:
-            return Status::OLAPInternalError(OLAP_ERR_ROWSET_INVALID_STATE_TRANSITION);
+            return Status::Error<ErrorCode::ROWSET_INVALID_STATE_TRANSITION>();
         }
         return Status::OK();
     }
@@ -93,7 +93,7 @@ public:
             break;
 
         default:
-            return Status::OLAPInternalError(OLAP_ERR_ROWSET_INVALID_STATE_TRANSITION);
+            return Status::Error<ErrorCode::ROWSET_INVALID_STATE_TRANSITION>();
         }
         return Status::OK();
     }
@@ -115,7 +115,7 @@ public:
     // Derived class implements the load logic by overriding the `do_load_once()` method.
     Status load(bool use_cache = true);
 
-    // returns Status::OLAPInternalError(OLAP_ERR_ROWSET_CREATE_READER) when failed to create reader
+    // returns Status::Error<ErrorCode::ROWSET_CREATE_READER>() when failed to create reader
     virtual Status create_reader(std::shared_ptr<RowsetReader>* result) = 0;
 
     // Split range denoted by `start_key` and `end_key` into sub-ranges, each contains roughly
diff --git a/be/src/olap/rowset/rowset_factory.cpp b/be/src/olap/rowset/rowset_factory.cpp
index fcbfb5fd61..7657e5c4ac 100644
--- a/be/src/olap/rowset/rowset_factory.cpp
+++ b/be/src/olap/rowset/rowset_factory.cpp
@@ -26,23 +26,24 @@
 #include "olap/rowset/vertical_beta_rowset_writer.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 Status RowsetFactory::create_rowset(TabletSchemaSPtr schema, const std::string& tablet_path,
                                     RowsetMetaSharedPtr rowset_meta, RowsetSharedPtr* rowset) {
     if (rowset_meta->rowset_type() == ALPHA_ROWSET) {
-        return Status::OLAPInternalError(OLAP_ERR_ROWSET_INVALID);
+        return Status::Error<ROWSET_INVALID>();
     }
     if (rowset_meta->rowset_type() == BETA_ROWSET) {
         rowset->reset(new BetaRowset(schema, tablet_path, rowset_meta));
         return (*rowset)->init();
     }
-    return Status::OLAPInternalError(OLAP_ERR_ROWSET_TYPE_NOT_FOUND); // should never happen
+    return Status::Error<ROWSET_TYPE_NOT_FOUND>(); // should never happen
 }
 
 Status RowsetFactory::create_rowset_writer(const RowsetWriterContext& context, bool is_vertical,
                                            std::unique_ptr<RowsetWriter>* output) {
     if (context.rowset_type == ALPHA_ROWSET) {
-        return Status::OLAPInternalError(OLAP_ERR_ROWSET_INVALID);
+        return Status::Error<ROWSET_INVALID>();
     }
     if (context.rowset_type == BETA_ROWSET) {
         if (is_vertical) {
@@ -52,7 +53,7 @@ Status RowsetFactory::create_rowset_writer(const RowsetWriterContext& context, b
         output->reset(new BetaRowsetWriter);
         return (*output)->init(context);
     }
-    return Status::OLAPInternalError(OLAP_ERR_ROWSET_TYPE_NOT_FOUND);
+    return Status::Error<ROWSET_TYPE_NOT_FOUND>();
 }
 
 } // namespace doris
diff --git a/be/src/olap/rowset/rowset_factory.h b/be/src/olap/rowset/rowset_factory.h
index 72d2e76cf8..ea7106935b 100644
--- a/be/src/olap/rowset/rowset_factory.h
+++ b/be/src/olap/rowset/rowset_factory.h
@@ -29,13 +29,13 @@ struct RowsetWriterContext;
 
 class RowsetFactory {
 public:
-    // return OLAP_SUCCESS and set inited rowset in `*rowset`.
+    // return OK and set inited rowset in `*rowset`.
     // return others if failed to create or init rowset.
     static Status create_rowset(TabletSchemaSPtr schema, const std::string& tablet_path,
                                 RowsetMetaSharedPtr rowset_meta, RowsetSharedPtr* rowset);
 
     // create and init rowset writer.
-    // return OLAP_SUCCESS and set `*output` to inited rowset writer.
+    // return OK and set `*output` to inited rowset writer.
     // return others if failed
     static Status create_rowset_writer(const RowsetWriterContext& context, bool is_vertical,
                                        std::unique_ptr<RowsetWriter>* output);
diff --git a/be/src/olap/rowset/rowset_meta_manager.cpp b/be/src/olap/rowset/rowset_meta_manager.cpp
index 795a0fb38b..7a58fa9cb6 100644
--- a/be/src/olap/rowset/rowset_meta_manager.cpp
+++ b/be/src/olap/rowset/rowset_meta_manager.cpp
@@ -31,6 +31,7 @@
 #include "olap/utils.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 const std::string ROWSET_PREFIX = "rst_";
 
@@ -47,19 +48,19 @@ Status RowsetMetaManager::get_rowset_meta(OlapMeta* meta, TabletUid tablet_uid,
     std::string key = ROWSET_PREFIX + tablet_uid.to_string() + "_" + rowset_id.to_string();
     std::string value;
     Status s = meta->get(META_COLUMN_FAMILY_INDEX, key, &value);
-    if (s.precise_code() == OLAP_ERR_META_KEY_NOT_FOUND) {
+    if (s.is<META_KEY_NOT_FOUND>()) {
         std::string error_msg = "rowset id:" + key + " not found.";
         LOG(WARNING) << error_msg;
-        return Status::OLAPInternalError(OLAP_ERR_META_KEY_NOT_FOUND);
+        return Status::Error<META_KEY_NOT_FOUND>();
     } else if (!s.ok()) {
         std::string error_msg = "load rowset id:" + key + " failed.";
         LOG(WARNING) << error_msg;
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
     bool ret = rowset_meta->init(value);
     if (!ret) {
         std::string error_msg = "parse rowset meta failed. rowset id:" + key;
-        return Status::OLAPInternalError(OLAP_ERR_SERIALIZE_PROTOBUF_ERROR);
+        return Status::Error<SERIALIZE_PROTOBUF_ERROR>();
     }
     return Status::OK();
 }
@@ -75,7 +76,7 @@ Status RowsetMetaManager::get_json_rowset_meta(OlapMeta* meta, TabletUid tablet_
     bool ret = rowset_meta_ptr->json_rowset_meta(json_rowset_meta);
     if (!ret) {
         std::string error_msg = "get json rowset meta failed. rowset id:" + rowset_id.to_string();
-        return Status::OLAPInternalError(OLAP_ERR_SERIALIZE_PROTOBUF_ERROR);
+        return Status::Error<SERIALIZE_PROTOBUF_ERROR>();
     }
     return Status::OK();
 }
@@ -88,7 +89,7 @@ Status RowsetMetaManager::save(OlapMeta* meta, TabletUid tablet_uid, const Rowse
     if (!ret) {
         std::string error_msg = "serialize rowset pb failed. rowset id:" + key;
         LOG(WARNING) << error_msg;
-        return Status::OLAPInternalError(OLAP_ERR_SERIALIZE_PROTOBUF_ERROR);
+        return Status::Error<SERIALIZE_PROTOBUF_ERROR>();
     }
     Status status = meta->put(META_COLUMN_FAMILY_INDEX, key, value);
     return status;
@@ -141,7 +142,7 @@ Status RowsetMetaManager::load_json_rowset_meta(OlapMeta* meta,
     if (!ret) {
         std::string error_msg = "parse json rowset meta failed.";
         LOG(WARNING) << error_msg;
-        return Status::OLAPInternalError(OLAP_ERR_SERIALIZE_PROTOBUF_ERROR);
+        return Status::Error<SERIALIZE_PROTOBUF_ERROR>();
     }
     RowsetId rowset_id = rowset_meta.rowset_id();
     TabletUid tablet_uid = rowset_meta.tablet_uid();
diff --git a/be/src/olap/rowset/rowset_reader.h b/be/src/olap/rowset/rowset_reader.h
index a189ef73dc..99fcf0da58 100644
--- a/be/src/olap/rowset/rowset_reader.h
+++ b/be/src/olap/rowset/rowset_reader.h
@@ -50,8 +50,8 @@ public:
 
     // read next block data into *block.
     // Returns
-    //      OLAP_SUCCESS when read successfully.
-    //      Status::OLAPInternalError(OLAP_ERR_DATA_EOF) and set *block to null when there is no more block.
+    //      OK when read successfully.
+    //      Status::Error<END_OF_FILE>() and set *block to null when there is no more block.
     //      Others when error happens.
     virtual Status next_block(RowBlock** block) = 0;
 
diff --git a/be/src/olap/rowset/rowset_writer.h b/be/src/olap/rowset/rowset_writer.h
index 531dd103c8..54371d9b41 100644
--- a/be/src/olap/rowset/rowset_writer.h
+++ b/be/src/olap/rowset/rowset_writer.h
@@ -43,11 +43,11 @@ public:
     virtual Status add_row(const ContiguousRow& row) = 0;
 
     virtual Status add_block(const vectorized::Block* block) {
-        return Status::OLAPInternalError(OLAP_ERR_FUNC_NOT_IMPLEMENTED);
+        return Status::Error<ErrorCode::NOT_IMPLEMENTED_ERROR>();
     }
     virtual Status add_columns(const vectorized::Block* block, const std::vector<uint32_t>& col_ids,
                                bool is_key, uint32_t max_rows_per_segment) {
-        return Status::OLAPInternalError(OLAP_ERR_FUNC_NOT_IMPLEMENTED);
+        return Status::Error<ErrorCode::NOT_IMPLEMENTED_ERROR>();
     }
 
     // Precondition: the input `rowset` should have the same type of the rowset we're building
@@ -59,18 +59,14 @@ public:
     // explicit flush all buffered rows into segment file.
     // note that `add_row` could also trigger flush when certain conditions are met
     virtual Status flush() = 0;
-    virtual Status flush_columns() {
-        return Status::OLAPInternalError(OLAP_ERR_FUNC_NOT_IMPLEMENTED);
-    }
-    virtual Status final_flush() {
-        return Status::OLAPInternalError(OLAP_ERR_FUNC_NOT_IMPLEMENTED);
-    }
+    virtual Status flush_columns() { return Status::Error<ErrorCode::NOT_IMPLEMENTED_ERROR>(); }
+    virtual Status final_flush() { return Status::Error<ErrorCode::NOT_IMPLEMENTED_ERROR>(); }
 
     virtual Status flush_single_memtable(MemTable* memtable, int64_t* flush_size) {
-        return Status::OLAPInternalError(OLAP_ERR_FUNC_NOT_IMPLEMENTED);
+        return Status::Error<ErrorCode::NOT_IMPLEMENTED_ERROR>();
     }
     virtual Status flush_single_memtable(const vectorized::Block* block) {
-        return Status::OLAPInternalError(OLAP_ERR_FUNC_NOT_IMPLEMENTED);
+        return Status::Error<ErrorCode::NOT_IMPLEMENTED_ERROR>();
     }
 
     // finish building and return pointer to the built rowset (guaranteed to be inited).
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 6bcf6055d9..424e1b7141 100644
--- a/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp
@@ -25,6 +25,7 @@
 #include "olap/rowset/segment_v2/page_io.h"
 
 namespace doris {
+using namespace ErrorCode;
 namespace segment_v2 {
 
 using strings::Substitute;
@@ -174,7 +175,7 @@ Status IndexedColumnIterator::seek_at_or_after(const void* key, bool* exact_matc
         std::string encoded_key;
         _reader->_value_key_coder->full_encode_ascending(key, &encoded_key);
         Status st = _value_iter.seek_at_or_before(encoded_key);
-        if (st.is_not_found()) {
+        if (st.is<NOT_FOUND>()) {
             // all keys in page is greater than `encoded_key`, point to the first page.
             // otherwise, we may missing some pages.
             // For example, the predicate is `col1 > 2`, and the index page is [3,5,7].
@@ -203,7 +204,7 @@ Status IndexedColumnIterator::seek_at_or_after(const void* key, bool* exact_matc
     // seek inside data page
     Status st = _data_page.data_decoder->seek_at_or_after_value(key, exact_match);
     // return the first row of next page when not found
-    if (st.is_not_found() && _reader->_has_index_page) {
+    if (st.is<NOT_FOUND>() && _reader->_has_index_page) {
         if (_value_iter.has_next()) {
             _seeked = true;
             *exact_match = false;
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
index 5f5419073a..fe03a21f2a 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
@@ -34,6 +34,7 @@
 #include "util/simd/bits.h"
 
 namespace doris {
+using namespace ErrorCode;
 namespace segment_v2 {
 
 // A fast range iterator for roaring bitmap. Output ranges use closed-open form, like [from, to).
@@ -519,7 +520,7 @@ Status SegmentIterator::_lookup_ordinal_from_pk_index(const RowCursor& key, bool
     Status status = index_iterator->seek_at_or_after(&index_key, &exact_match);
     if (UNLIKELY(!status.ok())) {
         *rowid = num_rows();
-        if (status.is_not_found()) {
+        if (status.is<NOT_FOUND>()) {
             return Status::OK();
         }
         return status;
diff --git a/be/src/olap/rowset/vertical_beta_rowset_writer.cpp b/be/src/olap/rowset/vertical_beta_rowset_writer.cpp
index 50324c8dd3..890f80e97d 100644
--- a/be/src/olap/rowset/vertical_beta_rowset_writer.cpp
+++ b/be/src/olap/rowset/vertical_beta_rowset_writer.cpp
@@ -20,6 +20,7 @@
 #include "olap/rowset/beta_rowset.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 VerticalBetaRowsetWriter::~VerticalBetaRowsetWriter() {
     if (!_already_built) {
@@ -132,13 +133,12 @@ Status VerticalBetaRowsetWriter::_create_segment_writer(
             BetaRowset::segment_file_path(_context.rowset_dir, _context.rowset_id, _num_segment++);
     auto fs = _rowset_meta->fs();
     if (!fs) {
-        return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+        return Status::Error<INIT_FAILED>();
     }
     io::FileWriterPtr file_writer;
     Status st = fs->create_file(path, &file_writer);
     if (!st.ok()) {
-        LOG(WARNING) << "failed to create writable file. path=" << path
-                     << ", err: " << st.get_error_msg();
+        LOG(WARNING) << "failed to create writable file. path=" << path << ", err: " << st;
         return st;
     }
 
diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp
index 8af54320b2..4c9b65f704 100644
--- a/be/src/olap/schema_change.cpp
+++ b/be/src/olap/schema_change.cpp
@@ -43,6 +43,7 @@
 using std::nothrow;
 
 namespace doris {
+using namespace ErrorCode;
 
 constexpr int ALTER_TABLE_BATCH_SIZE = 4096;
 
@@ -315,7 +316,7 @@ ColumnMapping* RowBlockChanger::get_mutable_column_mapping(size_t column_index)
                          << " origin_type="                                                      \
                          << ref_block->tablet_schema()->column(ref_column).type()                \
                          << ", alter_type=" << mutable_block->tablet_schema()->column(i).type(); \
-            return Status::OLAPInternalError(OLAP_ERR_SCHEMA_CHANGE_INFO_INVALID);               \
+            return Status::Error<SCHEMA_CHANGE_INFO_INVALID>();                                  \
         }                                                                                        \
         break;                                                                                   \
     }
@@ -580,19 +581,19 @@ Status RowBlockChanger::change_row_block(const RowBlock* ref_block, int32_t data
                                          const uint64_t* filtered_rows) const {
     if (mutable_block == nullptr) {
         LOG(FATAL) << "mutable block is uninitialized.";
-        return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
+        return Status::Error<UNINITIALIZED>();
     } else if (mutable_block->tablet_schema()->num_columns() != _schema_mapping.size()) {
         LOG(WARNING) << "mutable block does not match with schema mapping rules. "
                      << "block_schema_size=" << mutable_block->tablet_schema()->num_columns()
                      << ", mapping_schema_size=" << _schema_mapping.size();
-        return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
+        return Status::Error<UNINITIALIZED>();
     }
 
     if (mutable_block->capacity() < ref_block->row_block_info().row_num) {
         LOG(WARNING) << "mutable block is not large enough for storing the changed block. "
                      << "mutable_block_size=" << mutable_block->capacity()
                      << ", ref_block_row_num=" << ref_block->row_block_info().row_num;
-        return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
+        return Status::Error<UNINITIALIZED>();
     }
 
     mutable_block->clear();
@@ -600,13 +601,13 @@ Status RowBlockChanger::change_row_block(const RowBlock* ref_block, int32_t data
     RowCursor write_helper;
     if (write_helper.init(mutable_block->tablet_schema()) != Status::OK()) {
         LOG(WARNING) << "fail to init rowcursor.";
-        return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
+        return Status::Error<UNINITIALIZED>();
     }
 
     RowCursor read_helper;
     if (read_helper.init(ref_block->tablet_schema()) != Status::OK()) {
         LOG(WARNING) << "fail to init rowcursor.";
-        return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
+        return Status::Error<UNINITIALIZED>();
     }
 
     // a.1 First determine whether the data needs to be filtered, and finally only those marked as 1 are left as needed
@@ -637,7 +638,7 @@ Status RowBlockChanger::change_row_block(const RowBlock* ref_block, int32_t data
                 } else {
                     LOG(WARNING) << "error materialized view function : "
                                  << _schema_mapping[i].materialized_function;
-                    return Status::OLAPInternalError(OLAP_ERR_SCHEMA_CHANGE_INFO_INVALID);
+                    return Status::Error<SCHEMA_CHANGE_INFO_INVALID>();
                 }
                 VLOG_NOTICE << "_schema_mapping[" << i << "].materialized_function : "
                             << _schema_mapping[i].materialized_function;
@@ -649,7 +650,7 @@ Status RowBlockChanger::change_row_block(const RowBlock* ref_block, int32_t data
                     if (!_do_materialized_transform(&read_helper, &write_helper,
                                                     ref_block->tablet_schema()->column(ref_column),
                                                     i, _schema_mapping[i].ref_column, mem_pool)) {
-                        return Status::OLAPInternalError(OLAP_ERR_DATA_QUALITY_ERR);
+                        return Status::Error<DATA_QUALITY_ERR>();
                     }
                 }
                 continue;
@@ -738,7 +739,7 @@ Status RowBlockChanger::change_row_block(const RowBlock* ref_block, int32_t data
                     LOG(WARNING) << "the column type which was altered from was unsupported."
                                  << " from_type="
                                  << ref_block->tablet_schema()->column(ref_column).type();
-                    return Status::OLAPInternalError(OLAP_ERR_SCHEMA_CHANGE_INFO_INVALID);
+                    return Status::Error<SCHEMA_CHANGE_INFO_INVALID>();
                 }
 
                 if (newtype < reftype) {
@@ -784,7 +785,7 @@ Status RowBlockChanger::change_block(vectorized::Block* ref_block,
         LOG(WARNING) << "block does not match with schema mapping rules. "
                      << "block_schema_size=" << new_block->columns()
                      << ", mapping_schema_size=" << _schema_mapping.size();
-        return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
+        return Status::Error<UNINITIALIZED>();
     }
 
     ObjectPool pool;
@@ -1017,7 +1018,7 @@ Status RowBlockAllocator::allocate(RowBlock** row_block, size_t num_rows, bool n
 
     if (_memory_limitation > 0 && _tracker->consumption() + row_block_size > _memory_limitation) {
         *row_block = nullptr;
-        return Status::OLAPInternalError(OLAP_ERR_FETCH_MEMORY_EXCEEDED);
+        return Status::Error<FETCH_MEMORY_EXCEEDED>();
     }
 
     // TODO(lijiao) : Why abandon the original m_row_block_buffer
@@ -1025,7 +1026,7 @@ Status RowBlockAllocator::allocate(RowBlock** row_block, size_t num_rows, bool n
 
     if (*row_block == nullptr) {
         LOG(WARNING) << "failed to malloc RowBlock. size=" << sizeof(RowBlock);
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+        return Status::Error<MEM_ALLOC_FAILED>();
     }
 
     RowBlockInfo row_block_info(0U, num_rows);
@@ -1263,7 +1264,7 @@ Status SchemaChangeDirectly::_inner_process(RowsetReaderSharedPtr rowset_reader,
         _row_block_allocator = new RowBlockAllocator(new_tablet->tablet_schema(), 0);
         if (_row_block_allocator == nullptr) {
             LOG(FATAL) << "failed to malloc RowBlockAllocator. size=" << sizeof(RowBlockAllocator);
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<INVALID_ARGUMENT>();
         }
     }
 
@@ -1271,12 +1272,12 @@ Status SchemaChangeDirectly::_inner_process(RowsetReaderSharedPtr rowset_reader,
         _cursor = new (nothrow) RowCursor();
         if (nullptr == _cursor) {
             LOG(WARNING) << "fail to allocate row cursor.";
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<INVALID_ARGUMENT>();
         }
 
         if (!_cursor->init(new_tablet->tablet_schema())) {
             LOG(WARNING) << "fail to init row cursor.";
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<INVALID_ARGUMENT>();
         }
     }
 
@@ -1309,7 +1310,7 @@ Status SchemaChangeDirectly::_inner_process(RowsetReaderSharedPtr rowset_reader,
         _add_filtered_rows(filtered_rows);
 
         if (!_write_row_block(rowset_writer, new_row_block.get())) {
-            res = Status::OLAPInternalError(OLAP_ERR_SCHEMA_CHANGE_INFO_INVALID);
+            res = Status::Error<SCHEMA_CHANGE_INFO_INVALID>();
             LOG(WARNING) << "failed to write row block.";
             return res;
         }
@@ -1319,7 +1320,7 @@ Status SchemaChangeDirectly::_inner_process(RowsetReaderSharedPtr rowset_reader,
     }
 
     if (!rowset_writer->flush()) {
-        return Status::OLAPInternalError(OLAP_ERR_ALTER_STATUS_ERR);
+        return Status::Error<ALTER_STATUS_ERR>();
     }
 
     return res;
@@ -1344,7 +1345,7 @@ Status VSchemaChangeDirectly::_inner_process(RowsetReaderSharedPtr rowset_reader
     } while (true);
 
     if (!rowset_writer->flush()) {
-        return Status::OLAPInternalError(OLAP_ERR_ALTER_STATUS_ERR);
+        return Status::Error<ALTER_STATUS_ERR>();
     }
 
     return Status::OK();
@@ -1380,7 +1381,7 @@ Status SchemaChangeWithSorting::_inner_process(RowsetReaderSharedPtr rowset_read
                 new (nothrow) RowBlockAllocator(new_tablet->tablet_schema(), _memory_limitation);
         if (_row_block_allocator == nullptr) {
             LOG(FATAL) << "failed to malloc RowBlockAllocator. size=" << sizeof(RowBlockAllocator);
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<INVALID_ARGUMENT>();
         }
     }
 
@@ -1424,8 +1425,8 @@ Status SchemaChangeWithSorting::_inner_process(RowsetReaderSharedPtr rowset_read
         // that mean RowBlockAllocator::alocate() memory exceeded.
         // But we can flush row_block_arr if row_block_arr is not empty.
         // Don't return directly.
-        if (OLAP_ERR_MALLOC_ERROR == st.precise_code()) {
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+        if (st.is<MEM_ALLOC_FAILED>()) {
+            return st;
         } else if (st) {
             // do memory check for sorting, in case schema change task fail at row block sorting because of
             // not doing internal sorting first
@@ -1444,7 +1445,7 @@ Status SchemaChangeWithSorting::_inner_process(RowsetReaderSharedPtr rowset_read
                              << "memory_limitation=" << _memory_limitation
                              << ". You can increase the memory by changing the config: "
                              << "memory_limitation_per_thread_for_schema_change_bytes";
-                return Status::OLAPInternalError(OLAP_ERR_FETCH_MEMORY_EXCEEDED);
+                return Status::Error<FETCH_MEMORY_EXCEEDED>();
             }
 
             // enter here while memory limitation is reached.
@@ -1455,7 +1456,7 @@ Status SchemaChangeWithSorting::_inner_process(RowsetReaderSharedPtr rowset_read
                         oldest_write_timestamp, newest_write_timestamp, new_tablet,
                         segments_overlap, &rowset)) {
                 LOG(WARNING) << "failed to sorting internally.";
-                return Status::OLAPInternalError(OLAP_ERR_ALTER_STATUS_ERR);
+                return Status::Error<ALTER_STATUS_ERR>();
             }
 
             src_rowsets.push_back(rowset);
@@ -1485,7 +1486,7 @@ Status SchemaChangeWithSorting::_inner_process(RowsetReaderSharedPtr rowset_read
             if (!row_block_sorter.sort(&new_row_block)) {
                 row_block_arr.push_back(new_row_block);
                 LOG(WARNING) << "failed to sort row block.";
-                return Status::OLAPInternalError(OLAP_ERR_ALTER_STATUS_ERR);
+                return Status::Error<ALTER_STATUS_ERR>();
             }
             row_block_arr.push_back(new_row_block);
         } else {
@@ -1507,7 +1508,7 @@ Status SchemaChangeWithSorting::_inner_process(RowsetReaderSharedPtr rowset_read
                                oldest_write_timestamp, newest_write_timestamp, new_tablet,
                                segments_overlap, &rowset)) {
             LOG(WARNING) << "failed to sorting internally.";
-            return Status::OLAPInternalError(OLAP_ERR_ALTER_STATUS_ERR);
+            return Status::Error<ALTER_STATUS_ERR>();
         }
 
         src_rowsets.push_back(rowset);
@@ -1528,11 +1529,11 @@ Status SchemaChangeWithSorting::_inner_process(RowsetReaderSharedPtr rowset_read
             LOG(WARNING) << "create empty version for schema change failed."
                          << " version=" << rowset_writer->version().first << "-"
                          << rowset_writer->version().second;
-            return Status::OLAPInternalError(OLAP_ERR_ALTER_STATUS_ERR);
+            return Status::Error<ALTER_STATUS_ERR>();
         }
     } else if (!_external_sorting(src_rowsets, rowset_writer, new_tablet)) {
         LOG(WARNING) << "failed to sorting externally.";
-        return Status::OLAPInternalError(OLAP_ERR_ALTER_STATUS_ERR);
+        return Status::Error<ALTER_STATUS_ERR>();
     }
 
     return res;
@@ -1603,7 +1604,7 @@ Status VSchemaChangeWithSorting::_inner_process(RowsetReaderSharedPtr rowset_rea
                              << " _memory_limitation=" << _memory_limitation
                              << ", new_block->allocated_bytes()=" << new_block->allocated_bytes()
                              << ", consumption=" << _mem_tracker->consumption();
-                return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+                return Status::Error<INVALID_ARGUMENT>();
             }
         }
         _mem_tracker->consume(new_block->allocated_bytes());
@@ -1729,9 +1730,9 @@ Status VSchemaChangeWithSorting::_external_sorting(vector<RowsetSharedPtr>& src_
 
 Status SchemaChangeHandler::process_alter_tablet_v2(const TAlterTabletReqV2& request) {
     if (!request.__isset.desc_tbl) {
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR)
-                .append("desc_tbl is not set. Maybe the FE version is not equal to the BE "
-                        "version.");
+        return Status::Error<INVALID_ARGUMENT>().append(
+                "desc_tbl is not set. Maybe the FE version is not equal to the BE "
+                "version.");
     }
 
     LOG(INFO) << "begin to do request alter tablet: base_tablet_id=" << request.base_tablet_id
@@ -1742,7 +1743,7 @@ Status SchemaChangeHandler::process_alter_tablet_v2(const TAlterTabletReqV2& req
             StorageEngine::instance()->tablet_manager()->get_tablet(request.base_tablet_id);
     if (base_tablet == nullptr) {
         LOG(WARNING) << "fail to find base tablet. base_tablet=" << request.base_tablet_id;
-        return Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND);
+        return Status::Error<TABLE_NOT_FOUND>();
     }
     // Lock schema_change_lock util schema change info is stored in tablet header
     std::unique_lock<std::mutex> schema_change_lock(base_tablet->get_schema_change_lock(),
@@ -1750,7 +1751,7 @@ Status SchemaChangeHandler::process_alter_tablet_v2(const TAlterTabletReqV2& req
     if (!schema_change_lock.owns_lock()) {
         LOG(WARNING) << "failed to obtain schema change lock. "
                      << "base_tablet=" << request.base_tablet_id;
-        return Status::OLAPInternalError(OLAP_ERR_TRY_LOCK_FAILED);
+        return Status::Error<TRY_LOCK_FAILED>();
     }
 
     Status res = _do_process_alter_tablet_v2(request);
@@ -1774,7 +1775,7 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2&
             StorageEngine::instance()->tablet_manager()->get_tablet(request.base_tablet_id);
     if (base_tablet == nullptr) {
         LOG(WARNING) << "fail to find base tablet. base_tablet=" << request.base_tablet_id;
-        return Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND);
+        return Status::Error<TABLE_NOT_FOUND>();
     }
 
     // new tablet has to exist
@@ -1783,7 +1784,7 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2&
     if (new_tablet == nullptr) {
         LOG(WARNING) << "fail to find new tablet."
                      << " new_tablet=" << request.new_tablet_id;
-        return Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND);
+        return Status::Error<TABLE_NOT_FOUND>();
     }
 
     // check if tablet's state is not_ready, if it is ready, it means the tablet already finished
@@ -1803,11 +1804,11 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2&
 
     std::shared_lock base_migration_rlock(base_tablet->get_migration_lock(), std::try_to_lock);
     if (!base_migration_rlock.owns_lock()) {
-        return Status::OLAPInternalError(OLAP_ERR_RWLOCK_ERROR);
+        return Status::Error<TRY_LOCK_FAILED>();
     }
     std::shared_lock new_migration_rlock(new_tablet->get_migration_lock(), std::try_to_lock);
     if (!new_migration_rlock.owns_lock()) {
-        return Status::OLAPInternalError(OLAP_ERR_RWLOCK_ERROR);
+        return Status::Error<TRY_LOCK_FAILED>();
     }
 
     std::vector<Version> versions_to_be_changed;
@@ -1886,7 +1887,7 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2&
                     LOG(WARNING) << "New tablet has a version " << pair.first
                                  << " crossing base tablet's max_version="
                                  << max_rowset->end_version();
-                    return Status::OLAPInternalError(OLAP_ERR_VERSION_ALREADY_MERGED);
+                    return Status::Error<VERSION_ALREADY_MERGED>();
                 }
             }
             std::vector<RowsetSharedPtr> empty_vec;
@@ -1912,7 +1913,7 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2&
                 LOG(WARNING) << "fail to acquire all data sources. "
                              << "version_num=" << versions_to_be_changed.size()
                              << ", data_source_num=" << rs_readers.size();
-                res = Status::OLAPInternalError(OLAP_ERR_ALTER_DELTA_DOES_NOT_EXISTS);
+                res = Status::Error<ALTER_DELTA_DOES_NOT_EXISTS>();
                 break;
             }
             auto& all_del_preds = base_tablet->delete_predicates();
@@ -2116,7 +2117,7 @@ Status SchemaChangeHandler::_get_versions_to_be_changed(
     RowsetSharedPtr rowset = base_tablet->rowset_with_max_version();
     if (rowset == nullptr) {
         LOG(WARNING) << "Tablet has no version. base_tablet=" << base_tablet->full_name();
-        return Status::OLAPInternalError(OLAP_ERR_ALTER_DELTA_DOES_NOT_EXISTS);
+        return Status::Error<ALTER_DELTA_DOES_NOT_EXISTS>();
     }
     *max_rowset = rowset;
 
@@ -2193,7 +2194,7 @@ Status SchemaChangeHandler::_convert_historical_rowsets(const SchemaChangeParams
                 rs_reader->oldest_write_timestamp(), rs_reader->newest_write_timestamp(),
                 rs_reader->rowset()->rowset_meta()->fs(), &rowset_writer);
         if (!status.ok()) {
-            res = Status::OLAPInternalError(OLAP_ERR_ROWSET_BUILDER_INIT);
+            res = Status::Error<ROWSET_BUILDER_INIT>();
             return process_alter_exit();
         }
 
@@ -2218,7 +2219,7 @@ Status SchemaChangeHandler::_convert_historical_rowsets(const SchemaChangeParams
             return process_alter_exit();
         }
         res = sc_params.new_tablet->add_rowset(new_rowset);
-        if (res.precise_code() == OLAP_ERR_PUSH_VERSION_ALREADY_EXIST) {
+        if (res.is<PUSH_VERSION_ALREADY_EXIST>()) {
             LOG(WARNING) << "version already exist, version revert occurred. "
                          << "tablet=" << sc_params.new_tablet->full_name() << ", version='"
                          << rs_reader->version().first << "-" << rs_reader->version().second;
@@ -2277,7 +2278,7 @@ Status SchemaChangeHandler::_parse_request(const SchemaChangeParams& sc_params,
                 LOG(WARNING) << "referenced column was missing. "
                              << "[column=" << column_name << " referenced_column=" << column_index
                              << "]";
-                return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR);
+                return Status::Error<CE_CMD_PARAMS_ERROR>();
             }
         }
 
@@ -2400,7 +2401,7 @@ Status SchemaChangeHandler::_init_column_mapping(ColumnMapping* column_mapping,
     column_mapping->default_value = WrapperField::create(column_schema);
 
     if (column_mapping->default_value == nullptr) {
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+        return Status::Error<MEM_ALLOC_FAILED>();
     }
 
     if (column_schema.is_nullable() && value.length() == 0) {
@@ -2433,7 +2434,7 @@ Status SchemaChangeHandler::_validate_alter_result(TabletSharedPtr new_tablet,
     for (auto& pair : version_rowsets) {
         RowsetSharedPtr rowset = pair.second;
         if (!rowset->check_file_exist()) {
-            return Status::OLAPInternalError(OLAP_ERR_FILE_NOT_EXIST);
+            return Status::Error<FILE_NOT_EXIST>();
         }
     }
     return Status::OK();
diff --git a/be/src/olap/schema_change.h b/be/src/olap/schema_change.h
index 00e2f79bad..fd9b45cf8e 100644
--- a/be/src/olap/schema_change.h
+++ b/be/src/olap/schema_change.h
@@ -91,8 +91,7 @@ public:
                            TabletSchemaSPtr base_tablet_schema) {
         if (rowset_reader->rowset()->empty() || rowset_reader->rowset()->num_rows() == 0) {
             RETURN_WITH_WARN_IF_ERROR(
-                    rowset_writer->flush(),
-                    Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR),
+                    rowset_writer->flush(), Status::Error<ErrorCode::INVALID_ARGUMENT>(),
                     fmt::format("create empty version for schema change failed. version= {}-{}",
                                 rowset_writer->version().first, rowset_writer->version().second));
 
@@ -108,7 +107,7 @@ public:
 
         // Check row num changes
         if (config::row_nums_check && !_check_row_nums(rowset_reader, *rowset_writer)) {
-            return Status::OLAPInternalError(OLAP_ERR_ALTER_STATUS_ERR);
+            return Status::Error<ErrorCode::ALTER_STATUS_ERR>();
         }
 
         LOG(INFO) << "all row nums. source_rows=" << rowset_reader->rowset()->num_rows()
diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp
index 7a4093e912..da629d91f0 100644
--- a/be/src/olap/snapshot_manager.cpp
+++ b/be/src/olap/snapshot_manager.cpp
@@ -49,6 +49,7 @@ using std::vector;
 using std::list;
 
 namespace doris {
+using namespace ErrorCode;
 
 SnapshotManager* SnapshotManager::_s_instance = nullptr;
 std::mutex SnapshotManager::_mlock;
@@ -69,14 +70,14 @@ Status SnapshotManager::make_snapshot(const TSnapshotRequest& request, string* s
     Status res = Status::OK();
     if (snapshot_path == nullptr) {
         LOG(WARNING) << "output parameter cannot be null";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     TabletSharedPtr ref_tablet =
             StorageEngine::instance()->tablet_manager()->get_tablet(request.tablet_id);
     if (ref_tablet == nullptr) {
         LOG(WARNING) << "failed to get tablet. tablet=" << request.tablet_id;
-        return Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND);
+        return Status::Error<TABLE_NOT_FOUND>();
     }
 
     res = _create_snapshot_files(ref_tablet, request, snapshot_path, allow_incremental_clone);
@@ -98,7 +99,7 @@ Status SnapshotManager::release_snapshot(const string& snapshot_path) {
     for (auto store : stores) {
         std::string abs_path;
         RETURN_WITH_WARN_IF_ERROR(Env::Default()->canonicalize(store->path(), &abs_path),
-                                  Status::OLAPInternalError(OLAP_ERR_DIR_NOT_EXIST),
+                                  Status::Error<DIR_NOT_EXIST>(),
                                   "canonical path " + store->path() + "failed");
 
         if (snapshot_path.compare(0, abs_path.size(), abs_path) == 0 &&
@@ -112,7 +113,7 @@ Status SnapshotManager::release_snapshot(const string& snapshot_path) {
     }
 
     LOG(WARNING) << "released snapshot path illegal. [path='" << snapshot_path << "']";
-    return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR);
+    return Status::Error<CE_CMD_PARAMS_ERROR>();
 }
 
 Status SnapshotManager::convert_rowset_ids(const std::string& clone_dir, int64_t tablet_id,
@@ -121,7 +122,7 @@ Status SnapshotManager::convert_rowset_ids(const std::string& clone_dir, int64_t
     Status res = Status::OK();
     // check clone dir existed
     if (!FileUtils::check_exist(clone_dir)) {
-        res = Status::OLAPInternalError(OLAP_ERR_DIR_NOT_EXIST);
+        res = Status::Error<DIR_NOT_EXIST>();
         LOG(WARNING) << "clone dir not existed when convert rowsetids. clone_dir=" << clone_dir;
         return res;
     }
@@ -277,7 +278,7 @@ Status SnapshotManager::_rename_rowset_id(const RowsetMetaPB& rs_meta_pb,
     RowsetSharedPtr new_rowset = rs_writer->build();
     if (new_rowset == nullptr) {
         LOG(WARNING) << "failed to build rowset when rename rowset id";
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+        return Status::Error<MEM_ALLOC_FAILED>();
     }
     RETURN_NOT_OK(new_rowset->load(false));
     new_rowset->rowset_meta()->to_rowset_pb(new_rs_meta_pb);
@@ -292,7 +293,7 @@ Status SnapshotManager::_calc_snapshot_id_path(const TabletSharedPtr& tablet, in
     Status res = Status::OK();
     if (out_path == nullptr) {
         LOG(WARNING) << "output parameter cannot be null";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     // get current timestamp string
@@ -348,7 +349,7 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet
     Status res = Status::OK();
     if (snapshot_path == nullptr) {
         LOG(WARNING) << "output parameter cannot be null";
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     // snapshot_id_path:
@@ -379,19 +380,19 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet
     }
 
     RETURN_WITH_WARN_IF_ERROR(FileUtils::create_dir(schema_full_path),
-                              Status::OLAPInternalError(OLAP_ERR_CANNOT_CREATE_DIR),
+                              Status::Error<CANNOT_CREATE_DIR>(),
                               "create path " + schema_full_path + " failed");
 
     string snapshot_id;
     RETURN_WITH_WARN_IF_ERROR(FileUtils::canonicalize(snapshot_id_path, &snapshot_id),
-                              Status::OLAPInternalError(OLAP_ERR_CANNOT_CREATE_DIR),
+                              Status::Error<CANNOT_CREATE_DIR>(),
                               "canonicalize path " + snapshot_id_path + " failed");
 
     do {
         TabletMetaSharedPtr new_tablet_meta(new (nothrow) TabletMeta());
         if (new_tablet_meta == nullptr) {
             LOG(WARNING) << "fail to malloc TabletMeta.";
-            res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+            res = Status::Error<MEM_ALLOC_FAILED>();
             break;
         }
         std::vector<RowsetSharedPtr> consistent_rowsets;
@@ -449,7 +450,7 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet
                         LOG(WARNING) << "invalid make snapshot request. "
                                      << " version=" << last_version->end_version()
                                      << " req_version=" << request.version;
-                        res = Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+                        res = Status::Error<INVALID_ARGUMENT>();
                         break;
                     }
                     version = request.version;
@@ -524,7 +525,7 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet
                 res = new_tablet_meta->save_as_json(json_header_path, ref_tablet->data_dir());
             }
         } else {
-            res = Status::OLAPInternalError(OLAP_ERR_INVALID_SNAPSHOT_VERSION);
+            res = Status::Error<INVALID_SNAPSHOT_VERSION>();
         }
 
         if (!res.ok()) {
diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp
index 05a6b24a1e..6d3edd3633 100644
--- a/be/src/olap/storage_engine.cpp
+++ b/be/src/olap/storage_engine.cpp
@@ -79,6 +79,7 @@ using std::vector;
 using strings::Substitute;
 
 namespace doris {
+using namespace ErrorCode;
 
 DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(unused_rowsets_count, MetricUnit::ROWSETS);
 
@@ -645,7 +646,7 @@ Status StorageEngine::start_trash_sweep(double* usage, bool ignore_guard) {
     local_tm_now.tm_isdst = 0;
     if (localtime_r(&now, &local_tm_now) == nullptr) {
         LOG(WARNING) << "fail to localtime_r time. time=" << now;
-        return Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+        return Status::Error<OS_ERROR>();
     }
     const time_t local_now = mktime(&local_tm_now); //得到当地日历时间
 
@@ -805,7 +806,7 @@ Status StorageEngine::_do_sweep(const std::string& scan_root, const time_t& loca
             local_tm_create.tm_isdst = 0;
             if (strptime(str_time.c_str(), "%Y%m%d%H%M%S", &local_tm_create) == nullptr) {
                 LOG(WARNING) << "fail to strptime time. [time=" << str_time << "]";
-                res = Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+                res = Status::Error<OS_ERROR>();
                 continue;
             }
 
@@ -824,7 +825,7 @@ Status StorageEngine::_do_sweep(const std::string& scan_root, const time_t& loca
                 if (!ret.ok()) {
                     LOG(WARNING) << "fail to remove file or directory. path_desc: " << scan_root
                                  << ", error=" << ret.to_string();
-                    res = Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+                    res = Status::Error<OS_ERROR>();
                     continue;
                 }
             } else {
@@ -834,7 +835,7 @@ Status StorageEngine::_do_sweep(const std::string& scan_root, const time_t& loca
         }
     } catch (...) {
         LOG(WARNING) << "Exception occur when scan directory. path_desc=" << scan_root;
-        res = Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        res = Status::Error<IO_ERROR>();
     }
 
     return res;
@@ -912,7 +913,7 @@ Status StorageEngine::create_tablet(const TCreateTabletReq& request) {
     stores = get_stores_for_create_tablet(request.storage_medium);
     if (stores.empty()) {
         LOG(WARNING) << "there is no available disk that can be used to create tablet.";
-        return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR);
+        return Status::Error<CE_CMD_PARAMS_ERROR>();
     }
     TRACE("got data directory for create tablet");
     return _tablet_manager->create_tablet(request, stores);
@@ -924,13 +925,13 @@ Status StorageEngine::obtain_shard_path(TStorageMedium::type storage_medium,
 
     if (shard_path == nullptr) {
         LOG(WARNING) << "invalid output parameter which is null pointer.";
-        return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR);
+        return Status::Error<CE_CMD_PARAMS_ERROR>();
     }
 
     auto stores = get_stores_for_create_tablet(storage_medium);
     if (stores.empty()) {
         LOG(WARNING) << "no available disk can be used to create tablet.";
-        return Status::OLAPInternalError(OLAP_ERR_NO_AVAILABLE_ROOT_PATH);
+        return Status::Error<NO_AVAILABLE_ROOT_PATH>();
     }
 
     Status res = Status::OK();
@@ -965,11 +966,11 @@ Status StorageEngine::load_header(const string& shard_path, const TCloneReq& req
             store = get_store(store_path);
             if (store == nullptr) {
                 LOG(WARNING) << "invalid shard path, path=" << shard_path;
-                return Status::OLAPInternalError(OLAP_ERR_INVALID_ROOT_PATH);
+                return Status::Error<INVALID_ROOT_PATH>();
             }
         } catch (...) {
             LOG(WARNING) << "invalid shard path, path=" << shard_path;
-            return Status::OLAPInternalError(OLAP_ERR_INVALID_ROOT_PATH);
+            return Status::Error<INVALID_ROOT_PATH>();
         }
     }
 
diff --git a/be/src/olap/storage_engine.h b/be/src/olap/storage_engine.h
index 3e9c5d5857..d937957717 100644
--- a/be/src/olap/storage_engine.h
+++ b/be/src/olap/storage_engine.h
@@ -124,7 +124,7 @@ public:
     // @param [in] root_path specify root path of new tablet
     // @param [in] request specify new tablet info
     // @param [in] restore whether we're restoring a tablet from trash
-    // @return OLAP_SUCCESS if load tablet success
+    // @return OK if load tablet success
     Status load_header(const std::string& shard_path, const TCloneReq& request,
                        bool restore = false);
 
diff --git a/be/src/olap/stream_index_common.cpp b/be/src/olap/stream_index_common.cpp
index 9a9469f755..1fc5f92e82 100644
--- a/be/src/olap/stream_index_common.cpp
+++ b/be/src/olap/stream_index_common.cpp
@@ -21,6 +21,7 @@
 #include "olap/wrapper_field.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 ColumnStatistics::ColumnStatistics()
         : _minimum(nullptr), _maximum(nullptr), _ignored(true), _null_supported(false) {}
@@ -106,7 +107,7 @@ Status ColumnStatistics::write_to_buffer(char* buffer, size_t size) {
     }
 
     if (size < this->size()) {
-        return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
+        return Status::Error<BUFFER_OVERFLOW>();
     }
 
     // TODO(zc): too ugly
diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp
index 93b4e67525..c670e75b4a 100644
--- a/be/src/olap/tablet.cpp
+++ b/be/src/olap/tablet.cpp
@@ -67,6 +67,7 @@
 #include "util/trace.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 using std::pair;
 using std::nothrow;
@@ -335,12 +336,12 @@ Status Tablet::modify_rowsets(std::vector<RowsetSharedPtr>& to_add,
             if (find_rs == _rs_version_map.end()) {
                 LOG(WARNING) << "try to delete not exist version " << rs->version() << " from "
                              << full_name();
-                return Status::OLAPInternalError(OLAP_ERR_DELETE_VERSION_ERROR);
+                return Status::Error<DELETE_VERSION_ERROR>();
             } else if (find_rs->second->rowset_id() != rs->rowset_id()) {
                 LOG(WARNING) << "try to delete version " << rs->version() << " from " << full_name()
                              << ", but rowset id changed, delete rowset id is " << rs->rowset_id()
                              << ", exists rowsetid is" << find_rs->second->rowset_id();
-                return Status::OLAPInternalError(OLAP_ERR_DELETE_VERSION_ERROR);
+                return Status::Error<DELETE_VERSION_ERROR>();
             }
         }
     }
@@ -664,7 +665,7 @@ Status Tablet::capture_consistent_versions(const Version& spec_version,
                 LOG(WARNING) << "tablet:" << full_name()
                              << ", version already has been merged. spec_version: " << spec_version;
             }
-            status = Status::OLAPInternalError(OLAP_ERR_VERSION_ALREADY_MERGED);
+            status = Status::Error<VERSION_ALREADY_MERGED>();
         } else {
             if (version_path != nullptr) {
                 LOG(WARNING) << "status:" << status << ", tablet:" << full_name()
@@ -740,7 +741,7 @@ Status Tablet::_capture_consistent_rowsets_unlocked(const std::vector<Version>&
         if (!is_find) {
             LOG(WARNING) << "fail to find Rowset for version. tablet=" << full_name()
                          << ", version='" << version;
-            return Status::OLAPInternalError(OLAP_ERR_CAPTURE_ROWSET_ERROR);
+            return Status::Error<CAPTURE_ROWSET_ERROR>();
         }
     }
     return Status::OK();
@@ -768,14 +769,14 @@ Status Tablet::capture_rs_readers(const std::vector<Version>& version_path,
                 LOG(WARNING) << "fail to find Rowset in stale_rs_version for version. tablet="
                              << full_name() << ", version='" << version.first << "-"
                              << version.second;
-                return Status::OLAPInternalError(OLAP_ERR_CAPTURE_ROWSET_READER_ERROR);
+                return Status::Error<CAPTURE_ROWSET_READER_ERROR>();
             }
         }
         RowsetReaderSharedPtr rs_reader;
         auto res = it->second->create_reader(&rs_reader);
         if (!res.ok()) {
             LOG(WARNING) << "failed to create reader for rowset:" << it->second->rowset_id();
-            return Status::OLAPInternalError(OLAP_ERR_CAPTURE_ROWSET_READER_ERROR);
+            return Status::Error<CAPTURE_ROWSET_READER_ERROR>();
         }
         rs_readers->push_back(std::move(rs_reader));
     }
@@ -961,18 +962,18 @@ Status Tablet::split_range(const OlapTuple& start_key_strings, const OlapTuple&
     if (start_key_strings.size() > 0) {
         if (start_key.init_scan_key(_schema, start_key_strings.values()) != Status::OK()) {
             LOG(WARNING) << "fail to initial key strings with RowCursor type.";
-            return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+            return Status::Error<INIT_FAILED>();
         }
 
         if (start_key.from_tuple(start_key_strings) != Status::OK()) {
             LOG(WARNING) << "init end key failed";
-            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+            return Status::Error<INVALID_SCHEMA>();
         }
         key_num = start_key_strings.size();
     } else {
         if (start_key.init(_schema, num_short_key_columns()) != Status::OK()) {
             LOG(WARNING) << "fail to initial key strings with RowCursor type.";
-            return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+            return Status::Error<INIT_FAILED>();
         }
 
         start_key.allocate_memory_for_string_type(_schema);
@@ -985,17 +986,17 @@ Status Tablet::split_range(const OlapTuple& start_key_strings, const OlapTuple&
     if (end_key_strings.size() > 0) {
         if (!end_key.init_scan_key(_schema, end_key_strings.values())) {
             LOG(WARNING) << "fail to parse strings to key with RowCursor type.";
-            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+            return Status::Error<INVALID_SCHEMA>();
         }
 
         if (end_key.from_tuple(end_key_strings) != Status::OK()) {
             LOG(WARNING) << "init end key failed";
-            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+            return Status::Error<INVALID_SCHEMA>();
         }
     } else {
         if (end_key.init(_schema, num_short_key_columns()) != Status::OK()) {
             LOG(WARNING) << "fail to initial key strings with RowCursor type.";
-            return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+            return Status::Error<INIT_FAILED>();
         }
 
         end_key.allocate_memory_for_string_type(_schema);
@@ -1108,7 +1109,7 @@ Status Tablet::_contains_version(const Version& version) {
             CHECK(it.second != nullptr) << "there exist a version=" << it.first
                                         << " contains the input rs with version=" << version
                                         << ", but the related rs is null";
-            return Status::OLAPInternalError(OLAP_ERR_PUSH_VERSION_ALREADY_EXIST);
+            return Status::Error<PUSH_VERSION_ALREADY_EXIST>();
         }
     }
 
@@ -1465,7 +1466,7 @@ Status Tablet::prepare_compaction_and_calculate_permits(CompactionType compactio
         if (!res.ok()) {
             set_last_cumu_compaction_failure_time(UnixMillis());
             *permits = 0;
-            if (res.precise_code() != OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION) {
+            if (!res.is<CUMULATIVE_NO_SUITABLE_VERSION>()) {
                 DorisMetrics::instance()->cumulative_compaction_request_failed->increment(1);
                 return Status::InternalError("prepare cumulative compaction with err: {}", res);
             }
@@ -1494,7 +1495,7 @@ Status Tablet::prepare_compaction_and_calculate_permits(CompactionType compactio
         if (!res.ok()) {
             set_last_base_compaction_failure_time(UnixMillis());
             *permits = 0;
-            if (res.precise_code() != OLAP_ERR_BE_NO_SUITABLE_VERSION) {
+            if (!res.is<BE_NO_SUITABLE_VERSION>()) {
                 DorisMetrics::instance()->base_compaction_request_failed->increment(1);
                 return Status::InternalError("prepare base compaction with err: {}", res);
             }
@@ -1573,7 +1574,7 @@ Status Tablet::create_initial_rowset(const int64_t req_version) {
     Status res = Status::OK();
     if (req_version < 1) {
         LOG(WARNING) << "init version of tablet should at least 1. req.ver=" << req_version;
-        return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR);
+        return Status::Error<CE_CMD_PARAMS_ERROR>();
     }
     Version version(0, req_version);
     RowsetSharedPtr new_rowset;
@@ -1702,22 +1703,22 @@ Status Tablet::cooldown() {
     std::unique_lock schema_change_lock(_schema_change_lock, std::try_to_lock);
     if (!schema_change_lock.owns_lock()) {
         LOG(WARNING) << "Failed to own schema_change_lock. tablet=" << tablet_id();
-        return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR);
+        return Status::Error<TRY_LOCK_FAILED>();
     }
     // Check executing serially with compaction task.
     std::unique_lock base_compaction_lock(_base_compaction_lock, std::try_to_lock);
     if (!base_compaction_lock.owns_lock()) {
         LOG(WARNING) << "Failed to own base_compaction_lock. tablet=" << tablet_id();
-        return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR);
+        return Status::Error<TRY_LOCK_FAILED>();
     }
     std::unique_lock cumu_compaction_lock(_cumulative_compaction_lock, std::try_to_lock);
     if (!cumu_compaction_lock.owns_lock()) {
         LOG(WARNING) << "Failed to own cumu_compaction_lock. tablet=" << tablet_id();
-        return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR);
+        return Status::Error<TRY_LOCK_FAILED>();
     }
     auto dest_fs = io::FileSystemMap::instance()->get(storage_policy());
     if (!dest_fs) {
-        return Status::OLAPInternalError(OLAP_ERR_NOT_INITED);
+        return Status::Error<UNINITIALIZED>();
     }
     DCHECK(dest_fs->type() == io::FileSystemType::S3);
     auto old_rowset = pick_cooldown_rowset();
@@ -1919,7 +1920,7 @@ Status Tablet::lookup_row_key(const Slice& encoded_key, const RowsetIdUnorderedS
         auto& segments = segment_cache_handle.get_segments();
         DCHECK_GT(segments.size(), rs.second);
         Status s = segments[rs.second]->lookup_row_key(encoded_key, &loc);
-        if (s.is_not_found()) {
+        if (s.is<NOT_FOUND>()) {
             continue;
         }
         if (!s.ok()) {
@@ -2005,21 +2006,21 @@ Status Tablet::calc_delete_bitmap(RowsetId rowset_id,
                                            loc.row_id);
                         ++row_id;
                         continue;
-                    } else if (st.is_already_exist()) {
+                    } else if (st.is<ALREADY_EXIST>()) {
                         delete_bitmap->add({rowset_id, seg->id(), dummy_version.first}, row_id);
                         ++row_id;
                         continue;
                     }
                 }
                 auto st = lookup_row_key(*key, specified_rowset_ids, &loc, dummy_version.first - 1);
-                CHECK(st.ok() || st.is_not_found() || st.is_already_exist());
-                if (st.is_not_found()) {
+                CHECK(st.ok() || st.is<NOT_FOUND>() || st.is<ALREADY_EXIST>());
+                if (st.is<NOT_FOUND>()) {
                     ++row_id;
                     continue;
                 }
 
                 // sequence id smaller than the previous one, so delete current row
-                if (st.is_already_exist()) {
+                if (st.is<ALREADY_EXIST>()) {
                     loc.rowset_id = rowset_id;
                     loc.segment_id = seg->id();
                     loc.row_id = row_id;
@@ -2047,8 +2048,8 @@ Status Tablet::_check_pk_in_pre_segments(
         const Version& version, DeleteBitmapPtr delete_bitmap, RowLocation* loc) {
     for (auto it = pre_segments.rbegin(); it != pre_segments.rend(); ++it) {
         auto st = (*it)->lookup_row_key(key, loc);
-        CHECK(st.ok() || st.is_not_found() || st.is_already_exist());
-        if (st.is_not_found()) {
+        CHECK(st.ok() || st.is<NOT_FOUND>() || st.is<ALREADY_EXIST>());
+        if (st.is<NOT_FOUND>()) {
             continue;
         } else if (st.ok() && _schema->has_sequence_col() &&
                    delete_bitmap->contains({loc->rowset_id, loc->segment_id, version.first},
diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp
index f8e7200d10..03d6b0f4d1 100644
--- a/be/src/olap/tablet_manager.cpp
+++ b/be/src/olap/tablet_manager.cpp
@@ -68,6 +68,7 @@ using std::vector;
 using strings::Substitute;
 
 namespace doris {
+using namespace ErrorCode;
 
 DEFINE_GAUGE_METRIC_PROTOTYPE_5ARG(tablet_meta_mem_consumption, MetricUnit::BYTES, "",
                                    mem_consumption, Labels({{"type", "tablet_meta"}}));
@@ -110,11 +111,11 @@ Status TabletManager::_add_tablet_unlocked(TTabletId tablet_id, const TabletShar
         if (existed_tablet->tablet_path() == tablet->tablet_path()) {
             LOG(WARNING) << "add the same tablet twice! tablet_id=" << tablet_id
                          << ", tablet_path=" << tablet->tablet_path();
-            return Status::OLAPInternalError(OLAP_ERR_ENGINE_INSERT_EXISTS_TABLE);
+            return Status::Error<ENGINE_INSERT_EXISTS_TABLE>();
         }
         if (existed_tablet->data_dir() == tablet->data_dir()) {
             LOG(WARNING) << "add tablet with same data dir twice! tablet_id=" << tablet_id;
-            return Status::OLAPInternalError(OLAP_ERR_ENGINE_INSERT_EXISTS_TABLE);
+            return Status::Error<ENGINE_INSERT_EXISTS_TABLE>();
         }
     }
 
@@ -134,7 +135,7 @@ Status TabletManager::_add_tablet_unlocked(TTabletId tablet_id, const TabletShar
             // it could prevent error when log level is changed in the future.
             LOG(FATAL) << "new tablet is empty and old tablet exists. it should not happen."
                        << " tablet_id=" << tablet_id;
-            return Status::OLAPInternalError(OLAP_ERR_ENGINE_INSERT_EXISTS_TABLE);
+            return Status::Error<ENGINE_INSERT_EXISTS_TABLE>();
         }
         old_time = old_rowset == nullptr ? -1 : old_rowset->creation_time();
         new_time = new_rowset->creation_time();
@@ -169,7 +170,7 @@ Status TabletManager::_add_tablet_unlocked(TTabletId tablet_id, const TabletShar
                   << "tablet_id=" << tablet->tablet_id()
                   << ", tablet_path=" << tablet->tablet_path();
 
-        res = Status::OLAPInternalError(OLAP_ERR_ENGINE_INSERT_OLD_TABLET);
+        res = Status::Error<ENGINE_INSERT_OLD_TABLET>();
     }
     LOG(WARNING) << "add duplicated tablet. force=" << force << ", res=" << res
                  << ", tablet_id=" << tablet_id << ", old_version=" << old_version
@@ -255,7 +256,7 @@ Status TabletManager::create_tablet(const TCreateTabletReq& request, std::vector
                          << "new_tablet_id=" << tablet_id
                          << ", base_tablet_id=" << request.base_tablet_id;
             DorisMetrics::instance()->create_tablet_requests_failed->increment(1);
-            return Status::OLAPInternalError(OLAP_ERR_TABLE_CREATE_META_ERROR);
+            return Status::Error<TABLE_CREATE_META_ERROR>();
         }
         // If we are doing schema-change, we should use the same data dir
         // TODO(lingbin): A litter trick here, the directory should be determined before
@@ -272,7 +273,7 @@ Status TabletManager::create_tablet(const TCreateTabletReq& request, std::vector
     if (tablet == nullptr) {
         LOG(WARNING) << "fail to create tablet. tablet_id=" << request.tablet_id;
         DorisMetrics::instance()->create_tablet_requests_failed->increment(1);
-        return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR);
+        return Status::Error<CE_CMD_PARAMS_ERROR>();
     }
     TRACE("succeed to create tablet");
 
@@ -344,7 +345,7 @@ TabletSharedPtr TabletManager::_internal_create_tablet_unlocked(
         // Because if _add_tablet_unlocked() return OK, we must can get it from map.
         TabletSharedPtr tablet_ptr = _get_tablet_unlocked(new_tablet_id);
         if (tablet_ptr == nullptr) {
-            res = Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND);
+            res = Status::Error<TABLE_NOT_FOUND>();
             LOG(WARNING) << "fail to get tablet. res=" << res;
             break;
         }
@@ -723,7 +724,7 @@ Status TabletManager::load_tablet_from_meta(DataDir* data_dir, TTabletId tablet_
         LOG(WARNING) << "fail to load tablet because can not parse meta_binary string. "
                      << "tablet_id=" << tablet_id << ", schema_hash=" << schema_hash
                      << ", path=" << data_dir->path();
-        return Status::OLAPInternalError(OLAP_ERR_HEADER_PB_PARSE_FAILED);
+        return Status::Error<HEADER_PB_PARSE_FAILED>();
     }
     tablet_meta->init_rs_metas_fs(data_dir->fs());
 
@@ -734,12 +735,12 @@ Status TabletManager::load_tablet_from_meta(DataDir* data_dir, TTabletId tablet_
                      << ", schema_hash=" << schema_hash << ")"
                      << ", but meet tablet=" << tablet_meta->full_name()
                      << ", path=" << data_dir->path();
-        return Status::OLAPInternalError(OLAP_ERR_HEADER_PB_PARSE_FAILED);
+        return Status::Error<HEADER_PB_PARSE_FAILED>();
     }
     if (tablet_meta->tablet_uid().hi == 0 && tablet_meta->tablet_uid().lo == 0) {
         LOG(WARNING) << "fail to load tablet because its uid == 0. "
                      << "tablet=" << tablet_meta->full_name() << ", path=" << data_dir->path();
-        return Status::OLAPInternalError(OLAP_ERR_HEADER_PB_PARSE_FAILED);
+        return Status::Error<HEADER_PB_PARSE_FAILED>();
     }
 
     if (restore) {
@@ -751,7 +752,7 @@ Status TabletManager::load_tablet_from_meta(DataDir* data_dir, TTabletId tablet_
     if (tablet == nullptr) {
         LOG(WARNING) << "fail to load tablet. tablet_id=" << tablet_id
                      << ", schema_hash:" << schema_hash;
-        return Status::OLAPInternalError(OLAP_ERR_TABLE_CREATE_FROM_HEADER_ERROR);
+        return Status::Error<TABLE_CREATE_FROM_HEADER_ERROR>();
     }
 
     // NOTE: method load_tablet_from_meta could be called by two cases as below
@@ -765,7 +766,7 @@ Status TabletManager::load_tablet_from_meta(DataDir* data_dir, TTabletId tablet_
     if (check_path && !Env::Default()->path_exists(tablet->tablet_path()).ok()) {
         LOG(WARNING) << "tablet path not exists, create tablet failed, path="
                      << tablet->tablet_path();
-        return Status::OLAPInternalError(OLAP_ERR_TABLE_ALREADY_DELETED_ERROR);
+        return Status::Error<TABLE_ALREADY_DELETED_ERROR>();
     }
 
     if (tablet_meta->tablet_state() == TABLET_SHUTDOWN) {
@@ -775,7 +776,7 @@ Status TabletManager::load_tablet_from_meta(DataDir* data_dir, TTabletId tablet_
             std::lock_guard<std::shared_mutex> shutdown_tablets_wrlock(_shutdown_tablets_lock);
             _shutdown_tablets.push_back(tablet);
         }
-        return Status::OLAPInternalError(OLAP_ERR_TABLE_ALREADY_DELETED_ERROR);
+        return Status::Error<TABLE_ALREADY_DELETED_ERROR>();
     }
     // NOTE: We do not check tablet's initial version here, because if BE restarts when
     // one tablet is doing schema-change, we may meet empty tablet.
@@ -783,7 +784,7 @@ Status TabletManager::load_tablet_from_meta(DataDir* data_dir, TTabletId tablet_
         LOG(WARNING) << "fail to load tablet. it is in running state but without delta. "
                      << "tablet=" << tablet->full_name() << ", path=" << data_dir->path();
         // tablet state is invalid, drop tablet
-        return Status::OLAPInternalError(OLAP_ERR_TABLE_INDEX_VALIDATE_ERROR);
+        return Status::Error<TABLE_INDEX_VALIDATE_ERROR>();
     }
 
     RETURN_NOT_OK_LOG(tablet->init(),
@@ -819,13 +820,13 @@ Status TabletManager::load_tablet_from_dir(DataDir* store, TTabletId tablet_id,
 
     if (!Env::Default()->path_exists(header_path).ok()) {
         LOG(WARNING) << "fail to find header file. [header_path=" << header_path << "]";
-        return Status::OLAPInternalError(OLAP_ERR_FILE_NOT_EXIST);
+        return Status::Error<FILE_NOT_EXIST>();
     }
 
     TabletMetaSharedPtr tablet_meta(new TabletMeta());
     if (tablet_meta->create_from_file(header_path) != Status::OK()) {
         LOG(WARNING) << "fail to load tablet_meta. file_path=" << header_path;
-        return Status::OLAPInternalError(OLAP_ERR_ENGINE_LOAD_INDEX_TABLE_ERROR);
+        return Status::Error<ENGINE_LOAD_INDEX_TABLE_ERROR>();
     }
     // has to change shard id here, because meta file maybe copied from other source
     // its shard is different from local shard
@@ -849,7 +850,7 @@ Status TabletManager::report_tablet_info(TTabletInfo* tablet_info) {
     TabletSharedPtr tablet = get_tablet(tablet_info->tablet_id);
     if (tablet == nullptr) {
         LOG(WARNING) << "can't find tablet=" << tablet_info->tablet_id;
-        return Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND);
+        return Status::Error<TABLE_NOT_FOUND>();
     }
 
     tablet->build_tablet_report_info(tablet_info);
@@ -1170,7 +1171,7 @@ Status TabletManager::_create_tablet_meta_unlocked(const TCreateTabletReq& reque
             (*tablet_meta)->set_preferred_rowset_type(BETA_ROWSET);
         } else {
             LOG(FATAL) << "invalid TStorageFormat: " << request.storage_format;
-            return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR);
+            return Status::Error<CE_CMD_PARAMS_ERROR>();
         }
     }
     return res;
diff --git a/be/src/olap/tablet_manager.h b/be/src/olap/tablet_manager.h
index 1bd62381cf..58a60cbee7 100644
--- a/be/src/olap/tablet_manager.h
+++ b/be/src/olap/tablet_manager.h
@@ -110,8 +110,8 @@ public:
 
     // 获取所有tables的名字
     //
-    // Return OLAP_SUCCESS, if run ok
-    //        Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR), if tables is null
+    // Return OK, if run ok
+    //        Status::Error<INVALID_ARGUMENT>(), if tables is null
     Status report_tablet_info(TTabletInfo* tablet_info);
 
     Status build_all_report_tablets_info(std::map<TTabletId, TTablet>* tablets_info);
@@ -146,9 +146,9 @@ private:
     // Add a tablet pointer to StorageEngine
     // If force, drop the existing tablet add this new one
     //
-    // Return OLAP_SUCCESS, if run ok
+    // Return OK, if run ok
     //        OLAP_ERR_TABLE_INSERT_DUPLICATION_ERROR, if find duplication
-    //        Status::OLAPInternalError(OLAP_ERR_NOT_INITED), if not inited
+    //        Status::Error<UNINITIALIZED>(), if not inited
     Status _add_tablet_unlocked(TTabletId tablet_id, const TabletSharedPtr& tablet,
                                 bool update_meta, bool force);
 
diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp
index 010b2217c6..99c7523250 100644
--- a/be/src/olap/tablet_meta.cpp
+++ b/be/src/olap/tablet_meta.cpp
@@ -33,6 +33,7 @@ using std::unordered_map;
 using std::vector;
 
 namespace doris {
+using namespace ErrorCode;
 
 Status TabletMeta::create(const TCreateTabletReq& request, const TabletUid& tablet_uid,
                           uint64_t shard_id, uint32_t next_unique_id,
@@ -295,13 +296,13 @@ Status TabletMeta::create_from_file(const string& file_path) {
 
     if (file_handler.open(file_path, O_RDONLY) != Status::OK()) {
         LOG(WARNING) << "fail to open ordinal file. file=" << file_path;
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     // In file_header.unserialize(), it validates file length, signature, checksum of protobuf.
     if (file_header.unserialize(&file_handler) != Status::OK()) {
         LOG(WARNING) << "fail to unserialize tablet_meta. file='" << file_path;
-        return Status::OLAPInternalError(OLAP_ERR_PARSE_PROTOBUF_ERROR);
+        return Status::Error<PARSE_PROTOBUF_ERROR>();
     }
 
     TabletMetaPB tablet_meta_pb;
@@ -309,7 +310,7 @@ Status TabletMeta::create_from_file(const string& file_path) {
         tablet_meta_pb.CopyFrom(file_header.message());
     } catch (...) {
         LOG(WARNING) << "fail to copy protocol buffer object. file='" << file_path;
-        return Status::OLAPInternalError(OLAP_ERR_PARSE_PROTOBUF_ERROR);
+        return Status::Error<PARSE_PROTOBUF_ERROR>();
     }
 
     init_from_pb(tablet_meta_pb);
@@ -371,20 +372,20 @@ Status TabletMeta::save(const string& file_path, const TabletMetaPB& tablet_meta
 
     if (!file_handler.open_with_mode(file_path, O_CREAT | O_WRONLY | O_TRUNC, S_IRUSR | S_IWUSR)) {
         LOG(WARNING) << "fail to open header file. file='" << file_path;
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
 
     try {
         file_header.mutable_message()->CopyFrom(tablet_meta_pb);
     } catch (...) {
         LOG(WARNING) << "fail to copy protocol buffer object. file='" << file_path;
-        return Status::OLAPInternalError(OLAP_ERR_OTHER_ERROR);
+        return Status::Error<ErrorCode::INTERNAL_ERROR>();
     }
 
     if (file_header.prepare(&file_handler) != Status::OK() ||
         file_header.serialize(&file_handler) != Status::OK()) {
         LOG(WARNING) << "fail to serialize to file header. file='" << file_path;
-        return Status::OLAPInternalError(OLAP_ERR_SERIALIZE_PROTOBUF_ERROR);
+        return Status::Error<SERIALIZE_PROTOBUF_ERROR>();
     }
 
     return Status::OK();
@@ -426,7 +427,7 @@ Status TabletMeta::deserialize(const string& meta_binary) {
     bool parsed = tablet_meta_pb.ParseFromString(meta_binary);
     if (!parsed) {
         LOG(WARNING) << "parse tablet meta failed";
-        return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
+        return Status::Error<INIT_FAILED>();
     }
     init_from_pb(tablet_meta_pb);
     return Status::OK();
@@ -640,7 +641,7 @@ Status TabletMeta::add_rs_meta(const RowsetMetaSharedPtr& rs_meta) {
             if (rs->rowset_id() != rs_meta->rowset_id()) {
                 LOG(WARNING) << "version already exist. rowset_id=" << rs->rowset_id()
                              << " version=" << rs->version() << ", tablet=" << full_name();
-                return Status::OLAPInternalError(OLAP_ERR_PUSH_VERSION_ALREADY_EXIST);
+                return Status::Error<PUSH_VERSION_ALREADY_EXIST>();
             } else {
                 // rowsetid,version is equal, it is a duplicate req, skip it
                 return Status::OK();
diff --git a/be/src/olap/tablet_meta_manager.cpp b/be/src/olap/tablet_meta_manager.cpp
index 4193c5588f..336cd1d592 100644
--- a/be/src/olap/tablet_meta_manager.cpp
+++ b/be/src/olap/tablet_meta_manager.cpp
@@ -45,6 +45,7 @@ using rocksdb::Status;
 using rocksdb::kDefaultColumnFamilyName;
 
 namespace doris {
+using namespace ErrorCode;
 
 // should use tablet->generate_tablet_meta_copy() method to get a copy of current tablet meta
 // there are some rowset meta in local meta store and in in-memory tablet meta
@@ -57,10 +58,10 @@ Status TabletMetaManager::get_meta(DataDir* store, TTabletId tablet_id, TSchemaH
     std::string key = key_stream.str();
     std::string value;
     Status s = meta->get(META_COLUMN_FAMILY_INDEX, key, &value);
-    if (s.precise_code() == OLAP_ERR_META_KEY_NOT_FOUND) {
+    if (s.is<META_KEY_NOT_FOUND>()) {
         LOG(WARNING) << "tablet_id:" << tablet_id << ", schema_hash:" << schema_hash
                      << " not found.";
-        return Status::OLAPInternalError(OLAP_ERR_META_KEY_NOT_FOUND);
+        return Status::Error<META_KEY_NOT_FOUND>();
     } else if (!s.ok()) {
         LOG(WARNING) << "load tablet_id:" << tablet_id << ", schema_hash:" << schema_hash
                      << " failed.";
@@ -151,7 +152,7 @@ Status TabletMetaManager::load_json_meta(DataDir* store, const std::string& meta
     bool ret = json2pb::JsonToProtoMessage(json_meta, &tablet_meta_pb, &error);
     if (!ret) {
         LOG(ERROR) << "JSON to protobuf message failed: " << error;
-        return Status::OLAPInternalError(OLAP_ERR_HEADER_LOAD_JSON_HEADER);
+        return Status::Error<HEADER_LOAD_JSON_HEADER>();
     }
 
     std::string meta_binary;
diff --git a/be/src/olap/task/engine_batch_load_task.cpp b/be/src/olap/task/engine_batch_load_task.cpp
index 59a9cf16bc..bd3abf3efc 100644
--- a/be/src/olap/task/engine_batch_load_task.cpp
+++ b/be/src/olap/task/engine_batch_load_task.cpp
@@ -45,6 +45,7 @@ using std::string;
 using std::vector;
 
 namespace doris {
+using namespace ErrorCode;
 
 EngineBatchLoadTask::EngineBatchLoadTask(TPushReq& push_req, std::vector<TTabletInfo>* tablet_infos)
         : _push_req(push_req), _tablet_infos(tablet_infos) {
@@ -232,7 +233,7 @@ Status EngineBatchLoadTask::_process() {
         status = _push(_push_req, _tablet_infos);
         time_t push_finish = time(nullptr);
         LOG(INFO) << "Push finish, cost time: " << (push_finish - push_begin);
-        if (status.precise_code() == OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST) {
+        if (status.is<PUSH_TRANSACTION_ALREADY_EXIST>()) {
             status = Status::OK();
         }
     }
diff --git a/be/src/olap/task/engine_batch_load_task.h b/be/src/olap/task/engine_batch_load_task.h
index a37d6033b2..45c42b8298 100644
--- a/be/src/olap/task/engine_batch_load_task.h
+++ b/be/src/olap/task/engine_batch_load_task.h
@@ -60,7 +60,7 @@ private:
     // @param [in] request specify tablet and delete conditions
     // @param [out] tablet_info_vec return tablet last status, which
     //              include version info, row count, data size, etc
-    // @return OLAP_SUCCESS if submit delete_data success
+    // @return OK if submit delete_data success
     virtual Status _delete_data(const TPushReq& request, vector<TTabletInfo>* tablet_info_vec);
 
     Status _get_tmp_file_dir(const std::string& root_path, std::string* local_path);
diff --git a/be/src/olap/task/engine_clone_task.cpp b/be/src/olap/task/engine_clone_task.cpp
index 4bb86501b0..b77373ec6b 100644
--- a/be/src/olap/task/engine_clone_task.cpp
+++ b/be/src/olap/task/engine_clone_task.cpp
@@ -41,6 +41,7 @@ using strings::Split;
 using strings::SkipWhitespace;
 
 namespace doris {
+using namespace ErrorCode;
 
 const std::string HTTP_REQUEST_PREFIX = "/api/_tablet/_download?";
 const std::string HTTP_REQUEST_TOKEN_PARAM = "token=";
@@ -83,7 +84,7 @@ Status EngineCloneTask::_do_clone() {
     if (tablet != nullptr) {
         std::shared_lock migration_rlock(tablet->get_migration_lock(), std::try_to_lock);
         if (!migration_rlock.owns_lock()) {
-            return Status::OLAPInternalError(OLAP_ERR_RWLOCK_ERROR);
+            return Status::Error<TRY_LOCK_FAILED>();
         }
 
         // get download path
diff --git a/be/src/olap/task/engine_publish_version_task.cpp b/be/src/olap/task/engine_publish_version_task.cpp
index 25b55955c6..d8687f9839 100644
--- a/be/src/olap/task/engine_publish_version_task.cpp
+++ b/be/src/olap/task/engine_publish_version_task.cpp
@@ -27,6 +27,8 @@
 
 namespace doris {
 
+using namespace ErrorCode;
+
 using std::map;
 
 EnginePublishVersionTask::EnginePublishVersionTask(TPublishVersionRequest& publish_version_req,
@@ -97,7 +99,7 @@ Status EnginePublishVersionTask::finish() {
                 LOG(WARNING) << "could not find related rowset for tablet " << tablet_info.tablet_id
                              << " txn id " << transaction_id;
                 _error_tablet_ids->push_back(tablet_info.tablet_id);
-                res = Status::OLAPInternalError(OLAP_ERR_PUSH_ROWSET_NOT_FOUND);
+                res = Status::Error<PUSH_ROWSET_NOT_FOUND>();
                 continue;
             }
             TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet(
@@ -106,7 +108,7 @@ Status EnginePublishVersionTask::finish() {
                 LOG(WARNING) << "can't get tablet when publish version. tablet_id="
                              << tablet_info.tablet_id << " schema_hash=" << tablet_info.schema_hash;
                 _error_tablet_ids->push_back(tablet_info.tablet_id);
-                res = Status::OLAPInternalError(OLAP_ERR_PUSH_TABLE_NOT_EXIST);
+                res = Status::Error<PUSH_TABLE_NOT_EXIST>();
                 continue;
             }
             // in uniq key model with merge-on-write, we should see all
@@ -134,7 +136,7 @@ Status EnginePublishVersionTask::finish() {
                     // publish failed
                     if (!tablet->check_version_exist(version)) {
                         add_error_tablet_id(tablet_info.tablet_id);
-                        res = Status::OLAPInternalError(OLAP_ERR_PUBLISH_VERSION_NOT_CONTINUOUS);
+                        res = Status::Error<PUBLISH_VERSION_NOT_CONTINUOUS>();
                     }
                     continue;
                 }
@@ -218,8 +220,7 @@ void TabletPublishTxnTask::handle() {
 
     // add visible rowset to tablet
     publish_status = _tablet->add_inc_rowset(_rowset);
-    if (publish_status != Status::OK() &&
-        publish_status.precise_code() != OLAP_ERR_PUSH_VERSION_ALREADY_EXIST) {
+    if (publish_status != Status::OK() && !publish_status.is<PUSH_VERSION_ALREADY_EXIST>()) {
         LOG(WARNING) << "fail to add visible rowset to tablet. rowset_id=" << _rowset->rowset_id()
                      << ", tablet_id=" << _tablet_info.tablet_id << ", txn_id=" << _transaction_id
                      << ", res=" << publish_status;
diff --git a/be/src/olap/tuple_reader.cpp b/be/src/olap/tuple_reader.cpp
index 146249fb79..cdf25b2eb1 100644
--- a/be/src/olap/tuple_reader.cpp
+++ b/be/src/olap/tuple_reader.cpp
@@ -33,6 +33,7 @@ using std::set;
 using std::vector;
 
 namespace doris {
+using namespace ErrorCode;
 
 Status TupleReader::_init_collect_iter(const ReaderParams& read_params,
                                        std::vector<RowsetReaderSharedPtr>* valid_rs_readers) {
@@ -51,7 +52,7 @@ Status TupleReader::_init_collect_iter(const ReaderParams& read_params,
     for (auto& rs_reader : rs_readers) {
         RETURN_NOT_OK(rs_reader->init(&_reader_context));
         Status res = _collect_iter.add_child(rs_reader);
-        if (!res.ok() && res.precise_code() != OLAP_ERR_DATA_EOF) {
+        if (!res.ok() && !res.is<END_OF_FILE>()) {
             LOG(WARNING) << "failed to add child to iterator, err=" << res;
             return res;
         }
@@ -110,7 +111,7 @@ Status TupleReader::_direct_next_row(RowCursor* row_cursor, MemPool* mem_pool, O
     }
     direct_copy_row(row_cursor, *_next_key);
     auto res = _collect_iter.next(&_next_key, &_next_delete_flag);
-    if (UNLIKELY(!res.ok() && res.precise_code() != OLAP_ERR_DATA_EOF)) {
+    if (UNLIKELY(!res.ok() && !res.is<END_OF_FILE>())) {
         return res;
     }
     return Status::OK();
@@ -124,7 +125,7 @@ Status TupleReader::_direct_agg_key_next_row(RowCursor* row_cursor, MemPool* mem
     }
     init_row_with_others(row_cursor, *_next_key, mem_pool, agg_pool);
     auto res = _collect_iter.next(&_next_key, &_next_delete_flag);
-    if (UNLIKELY(!res.ok() && res.precise_code() != OLAP_ERR_DATA_EOF)) {
+    if (UNLIKELY(!res.ok() && !res.is<END_OF_FILE>())) {
         return res;
     }
     if (_need_agg_finalize) {
@@ -143,7 +144,7 @@ Status TupleReader::_agg_key_next_row(RowCursor* row_cursor, MemPool* mem_pool,
     int64_t merged_count = 0;
     do {
         auto res = _collect_iter.next(&_next_key, &_next_delete_flag);
-        if (UNLIKELY(res.precise_code() == OLAP_ERR_DATA_EOF)) {
+        if (UNLIKELY(res.is<END_OF_FILE>())) {
             break;
         }
 
@@ -189,7 +190,7 @@ Status TupleReader::_unique_key_next_row(RowCursor* row_cursor, MemPool* mem_poo
         while (_next_key) {
             // skip the lower version rows;
             auto res = _collect_iter.next(&_next_key, &_next_delete_flag);
-            if (LIKELY(res.precise_code() != OLAP_ERR_DATA_EOF)) {
+            if (LIKELY(!res.is<END_OF_FILE>())) {
                 if (UNLIKELY(!res.ok())) {
                     LOG(WARNING) << "next failed: " << res;
                     return res;
diff --git a/be/src/olap/txn_manager.cpp b/be/src/olap/txn_manager.cpp
index ad652e31ad..3907d7ed12 100644
--- a/be/src/olap/txn_manager.cpp
+++ b/be/src/olap/txn_manager.cpp
@@ -70,6 +70,7 @@ using std::stringstream;
 using std::vector;
 
 namespace doris {
+using namespace ErrorCode;
 
 TxnManager::TxnManager(int32_t txn_map_shard_size, int32_t txn_shard_size)
         : _txn_map_shard_size(txn_map_shard_size), _txn_shard_size(txn_shard_size) {
@@ -152,7 +153,7 @@ Status TxnManager::prepare_txn(TPartitionId partition_id, TTransactionId transac
     if (txn_partition_map.size() > config::max_runnings_transactions_per_txn_map) {
         LOG(WARNING) << "too many transactions: " << txn_tablet_map.size()
                      << ", limit: " << config::max_runnings_transactions_per_txn_map;
-        return Status::OLAPInternalError(OLAP_ERR_TOO_MANY_TRANSACTIONS);
+        return Status::Error<TOO_MANY_TRANSACTIONS>();
     }
 
     // not found load id
@@ -209,7 +210,7 @@ Status TxnManager::commit_txn(OlapMeta* meta, TPartitionId partition_id,
         LOG(WARNING) << "could not commit txn because rowset ptr is null. "
                      << "partition_id: " << key.first << ", transaction_id: " << key.second
                      << ", tablet: " << tablet_info.to_string();
-        return Status::OLAPInternalError(OLAP_ERR_ROWSET_INVALID);
+        return Status::Error<ROWSET_INVALID>();
     }
 
     std::unique_lock<std::mutex> txn_lock(_get_txn_lock(transaction_id));
@@ -246,7 +247,7 @@ Status TxnManager::commit_txn(OlapMeta* meta, TPartitionId partition_id,
                                  << ", tablet: " << tablet_info.to_string()
                                  << ", exist rowset_id: " << load_info.rowset->rowset_id()
                                  << ", new rowset_id: " << rowset_ptr->rowset_id();
-                    return Status::OLAPInternalError(OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST);
+                    return Status::Error<PUSH_TRANSACTION_ALREADY_EXIST>();
                 }
             }
         }
@@ -262,7 +263,7 @@ Status TxnManager::commit_txn(OlapMeta* meta, TPartitionId partition_id,
             LOG(WARNING) << "save committed rowset failed. when commit txn rowset_id:"
                          << rowset_ptr->rowset_id() << "tablet id: " << tablet_id
                          << "txn id:" << transaction_id;
-            return Status::OLAPInternalError(OLAP_ERR_ROWSET_SAVE_FAILED);
+            return Status::Error<ROWSET_SAVE_FAILED>();
         }
     }
 
@@ -333,10 +334,10 @@ Status TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id,
                 LOG(WARNING) << "save committed rowset failed. when publish txn rowset_id:"
                              << rowset_ptr->rowset_id() << ", tablet id: " << tablet_id
                              << ", txn id:" << transaction_id;
-                return Status::OLAPInternalError(OLAP_ERR_ROWSET_SAVE_FAILED);
+                return Status::Error<ROWSET_SAVE_FAILED>();
             }
         } else {
-            return Status::OLAPInternalError(OLAP_ERR_TRANSACTION_NOT_EXIST);
+            return Status::Error<TRANSACTION_NOT_EXIST>();
         }
     }
     {
@@ -380,7 +381,7 @@ Status TxnManager::rollback_txn(TPartitionId partition_id, TTransactionId transa
             if (load_info.rowset != nullptr) {
                 // if rowset is not null, it means other thread may commit the rowset
                 // should not delete txn any more
-                return Status::OLAPInternalError(OLAP_ERR_TRANSACTION_ALREADY_COMMITTED);
+                return Status::Error<TRANSACTION_ALREADY_COMMITTED>();
             }
         }
         it->second.erase(tablet_info);
@@ -406,7 +407,7 @@ Status TxnManager::delete_txn(OlapMeta* meta, TPartitionId partition_id,
     txn_tablet_map_t& txn_tablet_map = _get_txn_tablet_map(transaction_id);
     auto it = txn_tablet_map.find(key);
     if (it == txn_tablet_map.end()) {
-        return Status::OLAPInternalError(OLAP_ERR_TRANSACTION_NOT_EXIST);
+        return Status::Error<TRANSACTION_NOT_EXIST>();
     }
     auto load_itr = it->second.find(tablet_info);
     if (load_itr != it->second.end()) {
@@ -422,7 +423,7 @@ Status TxnManager::delete_txn(OlapMeta* meta, TPartitionId partition_id,
                              << ", tablet: " << tablet_info.to_string()
                              << ", rowset id: " << load_info.rowset->rowset_id()
                              << ", version: " << load_info.rowset->version().first;
-                return Status::OLAPInternalError(OLAP_ERR_TRANSACTION_ALREADY_COMMITTED);
+                return Status::Error<TRANSACTION_ALREADY_COMMITTED>();
             } else {
                 RowsetMetaManager::remove(meta, tablet_uid, load_info.rowset->rowset_id());
 #ifndef BE_TEST
diff --git a/be/src/olap/types.h b/be/src/olap/types.h
index 3c784d8804..c580f2ea53 100644
--- a/be/src/olap/types.h
+++ b/be/src/olap/types.h
@@ -373,12 +373,12 @@ public:
 
     Status convert_from(void* dest, const void* src, const TypeInfo* src_type, MemPool* mem_pool,
                         size_t variable_len = 0) const override {
-        return Status::OLAPInternalError(OLAP_ERR_FUNC_NOT_IMPLEMENTED);
+        return Status::Error<ErrorCode::NOT_IMPLEMENTED_ERROR>();
     }
 
     Status from_string(void* buf, const std::string& scan_key, const int precision = 0,
                        const int scale = 0) const override {
-        return Status::OLAPInternalError(OLAP_ERR_FUNC_NOT_IMPLEMENTED);
+        return Status::Error<ErrorCode::NOT_IMPLEMENTED_ERROR>();
     }
 
     std::string to_string(const void* src) const override {
@@ -626,7 +626,7 @@ struct BaseFieldtypeTraits : public CppTypeTraits<field_type> {
 
     static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
                                MemPool* mem_pool, size_t variable_len = 0) {
-        return Status::OLAPInternalError(OLAP_ERR_FUNC_NOT_IMPLEMENTED);
+        return Status::Error<ErrorCode::NOT_IMPLEMENTED_ERROR>();
     }
 
     static inline void set_to_max(void* buf) {
@@ -686,7 +686,7 @@ Status arithmetic_convert_from_varchar(void* dest, const void* src) {
     //TODO: use C++17 if constexpr to replace label assignment
     auto result = convert_from_varchar<T>(src_value, parse_res, std::is_integral<T>());
     if (UNLIKELY(parse_res != StringParser::PARSE_SUCCESS)) {
-        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        return Status::Error<ErrorCode::INVALID_SCHEMA>();
     }
     memcpy(dest, &result, sizeof(T));
     return Status::OK();
@@ -716,7 +716,7 @@ struct NumericFieldtypeTraits : public BaseFieldtypeTraits<fieldType> {
         } else if (src_type->type() == OLAP_FIELD_TYPE_CHAR) {
             return numeric_convert_from_char<CppType>(dest, src);
         }
-        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        return Status::Error<ErrorCode::INVALID_SCHEMA>();
     }
 };
 
@@ -951,7 +951,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL32>
                                                                  9, scale, &result);
 
         if (result == StringParser::PARSE_FAILURE) {
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<ErrorCode::INVALID_ARGUMENT>();
         }
         *reinterpret_cast<int32_t*>(buf) = (int32_t)value;
         return Status::OK();
@@ -975,7 +975,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL64>
         int64_t value = StringParser::string_to_decimal<int64_t>(scan_key.c_str(), scan_key.size(),
                                                                  18, scale, &result);
         if (result == StringParser::PARSE_FAILURE) {
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<ErrorCode::INVALID_ARGUMENT>();
         }
         *reinterpret_cast<int64_t*>(buf) = (int64_t)value;
         return Status::OK();
@@ -999,7 +999,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DECIMAL128I>
         int128_t value = StringParser::string_to_decimal<int128_t>(
                 scan_key.c_str(), scan_key.size(), 38, scale, &result);
         if (result == StringParser::PARSE_FAILURE) {
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<ErrorCode::INVALID_ARGUMENT>();
         }
         *reinterpret_cast<PackedInt128*>(buf) = value;
         return Status::OK();
@@ -1071,7 +1071,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATE> : public BaseFieldtypeTraits<OLAP_F
             SrcType src_value = *reinterpret_cast<const SrcType*>(src);
             DateTimeValue dt;
             if (!dt.from_date_int64(src_value)) {
-                return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+                return Status::Error<ErrorCode::INVALID_SCHEMA>();
             }
             CppType year = static_cast<CppType>(src_value / 10000);
             CppType month = static_cast<CppType>((src_value % 10000) / 100);
@@ -1097,10 +1097,10 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATE> : public BaseFieldtypeTraits<OLAP_F
                     return Status::OK();
                 }
             }
-            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+            return Status::Error<ErrorCode::INVALID_SCHEMA>();
         }
 
-        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        return Status::Error<ErrorCode::INVALID_SCHEMA>();
     }
     static void set_to_max(void* buf) {
         // max is 9999 * 16 * 32 + 12 * 32 + 31;
@@ -1177,7 +1177,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
             SrcType src_value = *reinterpret_cast<const SrcType*>(src);
             doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType> dt;
             if (!dt.from_date_int64(src_value)) {
-                return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+                return Status::Error<ErrorCode::INVALID_SCHEMA>();
             }
             CppType year = static_cast<CppType>(src_value / 10000);
             CppType month = static_cast<CppType>((src_value % 10000) / 100);
@@ -1203,10 +1203,10 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
                     return Status::OK();
                 }
             }
-            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+            return Status::Error<ErrorCode::INVALID_SCHEMA>();
         }
 
-        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        return Status::Error<ErrorCode::INVALID_SCHEMA>();
     }
     static void set_to_max(void* buf) {
         // max is 9999 * 16 * 32 + 12 * 32 + 31;
@@ -1287,7 +1287,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATETIMEV2>
                                                 << doris::vectorized::TIME_PART_LENGTH;
             return Status::OK();
         }
-        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        return Status::Error<ErrorCode::INVALID_SCHEMA>();
     }
     static void set_to_max(void* buf) {
         // max is 9999 * 16 * 32 + 12 * 32 + 31;
@@ -1370,7 +1370,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATETIME>
             *reinterpret_cast<CppType*>(dest) = to_value.to_datetime_int64();
             return Status::OK();
         }
-        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        return Status::Error<ErrorCode::INVALID_SCHEMA>();
     }
     static void set_to_max(void* buf) {
         // 设置为最大时间,其含义为:9999-12-31 23:59:59
@@ -1397,7 +1397,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_CHAR> : public BaseFieldtypeTraits<OLAP_F
         if (value_len > OLAP_VARCHAR_MAX_LENGTH) {
             LOG(WARNING) << "the len of value string is too long, len=" << value_len
                          << ", max_len=" << OLAP_VARCHAR_MAX_LENGTH;
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<ErrorCode::INVALID_ARGUMENT>();
         }
 
         auto slice = reinterpret_cast<Slice*>(buf);
@@ -1471,7 +1471,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> : public FieldTypeTraits<OLAP_FI
         if (value_len > OLAP_VARCHAR_MAX_LENGTH) {
             LOG(WARNING) << "the len of value string is too long, len=" << value_len
                          << ", max_len=" << OLAP_VARCHAR_MAX_LENGTH;
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<ErrorCode::INVALID_ARGUMENT>();
         }
 
         auto slice = reinterpret_cast<Slice*>(buf);
@@ -1493,8 +1493,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> : public FieldTypeTraits<OLAP_FI
         case OLAP_FIELD_TYPE_DOUBLE:
         case OLAP_FIELD_TYPE_DECIMAL: {
             auto result = src_type->to_string(src);
-            if (result.size() > variable_len)
-                return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            if (result.size() > variable_len) return Status::Error<ErrorCode::INVALID_ARGUMENT>();
             auto slice = reinterpret_cast<Slice*>(dest);
             slice->data = reinterpret_cast<char*>(mem_pool->allocate(result.size()));
             memcpy(slice->data, result.c_str(), result.size());
@@ -1506,7 +1505,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_VARCHAR> : public FieldTypeTraits<OLAP_FI
             deep_copy(dest, src, mem_pool);
             return Status::OK();
         default:
-            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+            return Status::Error<ErrorCode::INVALID_SCHEMA>();
         }
     }
 
@@ -1524,7 +1523,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
         if (value_len > config::string_type_length_soft_limit_bytes) {
             LOG(WARNING) << "the len of value string is too long, len=" << value_len
                          << ", max_len=" << config::string_type_length_soft_limit_bytes;
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+            return Status::Error<ErrorCode::INVALID_ARGUMENT>();
         }
 
         auto slice = reinterpret_cast<Slice*>(buf);
@@ -1557,7 +1556,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_STRING> : public FieldTypeTraits<OLAP_FIE
             deep_copy(dest, src, mem_pool);
             return Status::OK();
         default:
-            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+            return Status::Error<ErrorCode::INVALID_SCHEMA>();
         }
     }
 
@@ -1577,13 +1576,13 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_JSONB> : public FieldTypeTraits<OLAP_FIEL
     static Status from_string(void* buf, const std::string& scan_key, const int precision,
                               const int scale) {
         // TODO support schema change
-        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        return Status::Error<ErrorCode::INVALID_SCHEMA>();
     }
 
     static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
                                MemPool* mem_pool, size_t variable_len = 0) {
         // TODO support schema change
-        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
+        return Status::Error<ErrorCode::INVALID_SCHEMA>();
     }
 
     static void set_to_min(void* buf) {
diff --git a/be/src/olap/utils.cpp b/be/src/olap/utils.cpp
index 367b3d6bda..86b231a64a 100644
--- a/be/src/olap/utils.cpp
+++ b/be/src/olap/utils.cpp
@@ -57,6 +57,7 @@ using std::set;
 using std::vector;
 
 namespace doris {
+using namespace ErrorCode;
 
 Status olap_compress(const char* src_buf, size_t src_len, char* dest_buf, size_t dest_len,
                      size_t* written_len, OLAPCompressionType compression_type) {
@@ -64,7 +65,7 @@ Status olap_compress(const char* src_buf, size_t src_len, char* dest_buf, size_t
         LOG(WARNING) << "input param with nullptr pointer. [src_buf=" << src_buf
                      << " dest_buf=" << dest_buf << " written_len=" << written_len << "]";
 
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     *written_len = dest_len;
@@ -80,12 +81,12 @@ Status olap_compress(const char* src_buf, size_t src_len, char* dest_buf, size_t
             LOG(WARNING) << "compress failed. src_len=" << src_len << "; dest_len= " << dest_len
                          << "; written_len=" << *written_len << "; lzo_res=" << lzo_res;
 
-            return Status::OLAPInternalError(OLAP_ERR_COMPRESS_ERROR);
+            return Status::Error<COMPRESS_ERROR>();
         } else if (*written_len > dest_len) {
             VLOG_NOTICE << "buffer overflow when compressing. "
                         << "dest_len=" << dest_len << ", written_len=" << *written_len;
 
-            return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
+            return Status::Error<BUFFER_OVERFLOW>();
         }
         break;
     }
@@ -99,12 +100,12 @@ Status olap_compress(const char* src_buf, size_t src_len, char* dest_buf, size_t
             LOG(WARNING) << "compress failed. src_len=" << src_len << "; dest_len= " << dest_len
                          << "; written_len=" << *written_len << "; lzo_res=" << lzo_res;
 
-            return Status::OLAPInternalError(OLAP_ERR_COMPRESS_ERROR);
+            return Status::Error<COMPRESS_ERROR>();
         } else if (*written_len > dest_len) {
             VLOG_NOTICE << "buffer overflow when compressing. "
                         << ", dest_len=" << dest_len << ", written_len=" << *written_len;
 
-            return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
+            return Status::Error<BUFFER_OVERFLOW>();
         }
         break;
     }
@@ -117,7 +118,7 @@ Status olap_compress(const char* src_buf, size_t src_len, char* dest_buf, size_t
         if (0 == lz4_res) {
             VLOG_TRACE << "compress failed. src_len=" << src_len << ", dest_len=" << dest_len
                        << ", written_len=" << *written_len << ", lz4_res=" << lz4_res;
-            return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
+            return Status::Error<BUFFER_OVERFLOW>();
         }
         break;
     }
@@ -134,7 +135,7 @@ Status olap_decompress(const char* src_buf, size_t src_len, char* dest_buf, size
         LOG(WARNING) << "input param with nullptr pointer. [src_buf=" << src_buf
                      << " dest_buf=" << dest_buf << " written_len=" << written_len << "]";
 
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     *written_len = dest_len;
@@ -147,11 +148,11 @@ Status olap_decompress(const char* src_buf, size_t src_len, char* dest_buf, size
         if (LZO_E_OK != lzo_res) {
             LOG(WARNING) << "decompress failed. src_len=" << src_len << "; dest_len= " << dest_len
                          << "; written_len=" << *written_len << "; lzo_res=" << lzo_res;
-            return Status::OLAPInternalError(OLAP_ERR_DECOMPRESS_ERROR);
+            return Status::Error<DECOMPRESS_ERROR>();
         } else if (*written_len > dest_len) {
             LOG(WARNING) << "buffer overflow when decompressing. [dest_len=" << dest_len
                          << " written_len=" << *written_len << "]";
-            return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
+            return Status::Error<BUFFER_OVERFLOW>();
         }
         break;
     }
@@ -162,11 +163,11 @@ Status olap_decompress(const char* src_buf, size_t src_len, char* dest_buf, size
         if (LZO_E_OK != lzo_res) {
             LOG(WARNING) << "compress failed. src_len=" << src_len << "; dest_len= " << dest_len
                          << "; written_len=" << *written_len << "; lzo_res=" << lzo_res;
-            return Status::OLAPInternalError(OLAP_ERR_DECOMPRESS_ERROR);
+            return Status::Error<DECOMPRESS_ERROR>();
         } else if (*written_len > dest_len) {
             LOG(WARNING) << "buffer overflow when decompressing. [dest_len=" << dest_len
                          << " written_len=" << *written_len << "]";
-            return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
+            return Status::Error<BUFFER_OVERFLOW>();
         }
         break;
     }
@@ -178,7 +179,7 @@ Status olap_decompress(const char* src_buf, size_t src_len, char* dest_buf, size
         if (lz4_res < 0) {
             LOG(WARNING) << "decompress failed. src_len=" << src_len << "; dest_len= " << dest_len
                          << "; written_len=" << *written_len << "; lzo_res=" << lz4_res;
-            return Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW);
+            return Status::Error<BUFFER_OVERFLOW>();
         }
         break;
     }
@@ -553,12 +554,12 @@ Status gen_timestamp_string(string* out_string) {
 
     if (localtime_r(&now, &local_tm) == nullptr) {
         LOG(WARNING) << "fail to localtime_r time. [time=" << now << "]";
-        return Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+        return Status::Error<OS_ERROR>();
     }
     char time_suffix[16] = {0}; // Example: 20150706111404, 长度是15个字符
     if (strftime(time_suffix, sizeof(time_suffix), "%Y%m%d%H%M%S", &local_tm) == 0) {
         LOG(WARNING) << "fail to strftime time. [time=" << now << "]";
-        return Status::OLAPInternalError(OLAP_ERR_OS_ERROR);
+        return Status::Error<OS_ERROR>();
     }
 
     *out_string = time_suffix;
@@ -576,7 +577,7 @@ Status read_write_test_file(const string& test_file_path) {
             LOG(WARNING) << "fail to delete test file. "
                          << "path=" << test_file_path << ", errno=" << errno
                          << ", err=" << strerror_r(errno, errmsg, 64);
-            return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+            return Status::Error<IO_ERROR>();
         }
     } else {
         if (errno != ENOENT) {
@@ -584,7 +585,7 @@ Status read_write_test_file(const string& test_file_path) {
             LOG(WARNING) << "fail to access test file. "
                          << "path=" << test_file_path << ", errno=" << errno
                          << ", err=" << strerror_r(errno, errmsg, 64);
-            return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+            return Status::Error<IO_ERROR>();
         }
     }
     Status res = Status::OK();
@@ -600,12 +601,12 @@ Status read_write_test_file(const string& test_file_path) {
     char* read_test_buff = nullptr;
     if (posix_memalign((void**)&write_test_buff, DIRECT_IO_ALIGNMENT, TEST_FILE_BUF_SIZE) != 0) {
         LOG(WARNING) << "fail to allocate write buffer memory. size=" << TEST_FILE_BUF_SIZE;
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+        return Status::Error<MEM_ALLOC_FAILED>();
     }
     std::unique_ptr<char, decltype(&std::free)> write_buff(write_test_buff, &std::free);
     if (posix_memalign((void**)&read_test_buff, DIRECT_IO_ALIGNMENT, TEST_FILE_BUF_SIZE) != 0) {
         LOG(WARNING) << "fail to allocate read buffer memory. size=" << TEST_FILE_BUF_SIZE;
-        return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+        return Status::Error<MEM_ALLOC_FAILED>();
     }
     std::unique_ptr<char, decltype(&std::free)> read_buff(read_test_buff, &std::free);
     // generate random numbers
@@ -625,7 +626,7 @@ Status read_write_test_file(const string& test_file_path) {
     if (memcmp(write_buff.get(), read_buff.get(), TEST_FILE_BUF_SIZE) != 0) {
         LOG(WARNING) << "the test file write_buf and read_buf not equal, [file_name = "
                      << test_file_path << "]";
-        return Status::OLAPInternalError(OLAP_ERR_TEST_FILE_ERROR);
+        return Status::Error<TEST_FILE_ERROR>();
     }
     if ((res = file_handler.close()) != Status::OK()) {
         LOG(WARNING) << "fail to close test file. [file_name=" << test_file_path << "]";
@@ -635,7 +636,7 @@ Status read_write_test_file(const string& test_file_path) {
         char errmsg[64];
         VLOG_NOTICE << "fail to delete test file. [err='" << strerror_r(errno, errmsg, 64)
                     << "' path='" << test_file_path << "']";
-        return Status::OLAPInternalError(OLAP_ERR_IO_ERROR);
+        return Status::Error<IO_ERROR>();
     }
     return res;
 }
diff --git a/be/src/olap/utils.h b/be/src/olap/utils.h
index e6f9c7a5db..8998a82a42 100644
--- a/be/src/olap/utils.h
+++ b/be/src/olap/utils.h
@@ -78,7 +78,7 @@ private:
 template <typename T>
 Status split_string(const std::string& base, const T separator, std::vector<std::string>* result) {
     if (!result) {
-        return Status::OLAPInternalError(OLAP_ERR_OTHER_ERROR);
+        return Status::Error<ErrorCode::INVALID_ARGUMENT>();
     }
 
     // 处理base为空的情况
diff --git a/be/src/olap/version_graph.cpp b/be/src/olap/version_graph.cpp
index 3747050d5b..bae1d289a2 100644
--- a/be/src/olap/version_graph.cpp
+++ b/be/src/olap/version_graph.cpp
@@ -26,6 +26,7 @@
 #include "util/time.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 void TimestampedVersionTracker::_construct_versioned_tracker(
         const std::vector<RowsetMetaSharedPtr>& rs_metas) {
@@ -517,7 +518,7 @@ Status VersionGraph::delete_version_from_graph(const Version& version) {
         _vertex_index_map.find(end_vertex_value) == _vertex_index_map.end()) {
         LOG(WARNING) << "vertex for version does not exists. "
                      << "version=" << version.first << "-" << version.second;
-        return Status::OLAPInternalError(OLAP_ERR_HEADER_DELETE_VERSION);
+        return Status::Error<HEADER_DELETE_VERSION>();
     }
 
     int64_t start_vertex_index = _vertex_index_map[start_vertex_value];
@@ -563,7 +564,7 @@ Status VersionGraph::capture_consistent_versions(const Version& spec_version,
     if (spec_version.first > spec_version.second) {
         LOG(WARNING) << "invalid specified version. "
                      << "spec_version=" << spec_version.first << "-" << spec_version.second;
-        return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
+        return Status::Error<INVALID_ARGUMENT>();
     }
 
     int64_t cur_idx = -1;
diff --git a/be/src/pipeline/exec/agg_context.h b/be/src/pipeline/exec/agg_context.h
index fae8d74231..c7ddbdeb48 100644
--- a/be/src/pipeline/exec/agg_context.h
+++ b/be/src/pipeline/exec/agg_context.h
@@ -16,6 +16,9 @@
 // under the License.
 #pragma once
 
+#include <atomic>
+#include <mutex>
+
 #include "common/status.h"
 
 namespace doris {
diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp
index 689faefa55..9f6fbdf891 100644
--- a/be/src/pipeline/exec/exchange_sink_buffer.cpp
+++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp
@@ -183,8 +183,7 @@ Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) {
                                     const PTransmitDataResult& result) {
         Status s = Status(result.status());
         if (!s.ok()) {
-            _failed(id,
-                    fmt::format("exchange req success but status isn't ok: {}", s.get_error_msg()));
+            _failed(id, fmt::format("exchange req success but status isn't ok: {}", s.to_string()));
         } else if (eos) {
             _ended(id);
         } else {
diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.h b/be/src/pipeline/exec/streaming_aggregation_source_operator.h
index fba6fab2a4..751d0b8e92 100644
--- a/be/src/pipeline/exec/streaming_aggregation_source_operator.h
+++ b/be/src/pipeline/exec/streaming_aggregation_source_operator.h
@@ -16,6 +16,8 @@
 // under the License.
 #pragma once
 
+#include <atomic>
+
 #include "agg_context.h"
 #include "operator.h"
 
diff --git a/be/src/runtime/broker_mgr.cpp b/be/src/runtime/broker_mgr.cpp
index 73300a9667..5e57922399 100644
--- a/be/src/runtime/broker_mgr.cpp
+++ b/be/src/runtime/broker_mgr.cpp
@@ -75,8 +75,7 @@ void BrokerMgr::ping(const TNetworkAddress& addr) {
         BrokerServiceConnection client(_exec_env->broker_client_cache(), addr,
                                        config::thrift_rpc_timeout_ms, &status);
         if (!status.ok()) {
-            LOG(WARNING) << "Create broker client failed. broker=" << addr
-                         << ", status=" << status.get_error_msg();
+            LOG(WARNING) << "Create broker client failed. broker=" << addr << ", status=" << status;
             return;
         }
 
@@ -86,7 +85,7 @@ void BrokerMgr::ping(const TNetworkAddress& addr) {
             status = client.reopen();
             if (!status.ok()) {
                 LOG(WARNING) << "Create broker client failed. broker=" << addr
-                             << ", status=" << status.get_error_msg();
+                             << ", status=" << status;
                 return;
             }
             client->ping(response, request);
diff --git a/be/src/runtime/buffer_control_block.cpp b/be/src/runtime/buffer_control_block.cpp
index a3d765f3c2..c745bc8750 100644
--- a/be/src/runtime/buffer_control_block.cpp
+++ b/be/src/runtime/buffer_control_block.cpp
@@ -26,7 +26,7 @@
 namespace doris {
 
 void GetResultBatchCtx::on_failure(const Status& status) {
-    DCHECK(!status.ok()) << "status is ok, errmsg=" << status.get_error_msg();
+    DCHECK(!status.ok()) << "status is ok, errmsg=" << status;
     status.to_protobuf(result->mutable_status());
     {
         // call by result sink
@@ -64,7 +64,7 @@ void GetResultBatchCtx::on_data(const std::unique_ptr<TFetchDataResult>& t_resul
             result->set_packet_seq(packet_seq);
             result->set_eos(eos);
         } else {
-            LOG(WARNING) << "TFetchDataResult serialize failed, errmsg=" << st.get_error_msg();
+            LOG(WARNING) << "TFetchDataResult serialize failed, errmsg=" << st;
         }
     } else {
         result->set_empty_batch(true);
diff --git a/be/src/runtime/buffered_block_mgr2.cc b/be/src/runtime/buffered_block_mgr2.cc
index 0a17ad0b3a..1754be4dc1 100644
--- a/be/src/runtime/buffered_block_mgr2.cc
+++ b/be/src/runtime/buffered_block_mgr2.cc
@@ -48,6 +48,7 @@ using std::shared_ptr;
 using std::unique_lock;
 
 namespace doris {
+using namespace ErrorCode;
 
 BufferedBlockMgr2::BlockMgrsMap BufferedBlockMgr2::_s_query_to_block_mgrs;
 SpinLock BufferedBlockMgr2::_s_block_mgrs_lock;
@@ -715,9 +716,9 @@ Status BufferedBlockMgr2::allocate_scratch_space(int64_t block_size, TmpFileMgr:
         }
         // Log error and try other files if there was a problem. Problematic files will be
         // blacklisted so we will not repeatedly log the same error.
-        LOG(WARNING) << "Error while allocating temporary file range: " << status.get_error_msg()
+        LOG(WARNING) << "Error while allocating temporary file range: " << status
                      << ". Will try another temporary file.";
-        errs.emplace_back(status.get_error_msg());
+        errs.emplace_back(status.to_string());
     }
     Status err_status = Status::InternalError(
             "No usable temporary files: space could not be allocated on any temporary device.");
@@ -781,21 +782,20 @@ void BufferedBlockMgr2::write_complete(Block* block, const Status& write_status)
         VLOG_FILE << "Query: " << _query_id
                   << ". Write did not complete successfully: "
                      "write_status="
-                  << write_status.get_error_msg() << ", status=" << status.get_error_msg()
-                  << ". _is_cancelled=" << _is_cancelled;
+                  << write_status << ", status=" << status << ". _is_cancelled=" << _is_cancelled;
 
         // If the instance is already cancelled, don't confuse things with these errors.
-        if (!write_status.is_cancelled() && !state->is_cancelled()) {
+        if (!write_status.is<CANCELLED>() && !state->is_cancelled()) {
             if (!write_status.ok()) {
                 // Report but do not attempt to recover from write error.
                 DCHECK(block->_tmp_file != nullptr);
-                block->_tmp_file->report_io_error(write_status.get_error_msg());
+                block->_tmp_file->report_io_error(write_status.to_string());
                 VLOG_QUERY << "Query: " << _query_id << " write complete callback with error.";
-                state->log_error(write_status.get_error_msg());
+                state->log_error(write_status.to_string());
             }
             if (!status.ok()) {
                 VLOG_QUERY << "Query: " << _query_id << " error while writing unpinned blocks.";
-                state->log_error(status.get_error_msg());
+                state->log_error(status.to_string());
             }
         }
         // Set cancelled and wake up waiting threads if an error occurred.  Note that in
diff --git a/be/src/runtime/buffered_block_mgr2.h b/be/src/runtime/buffered_block_mgr2.h
index 6c7a63378f..9b423cf7f1 100644
--- a/be/src/runtime/buffered_block_mgr2.h
+++ b/be/src/runtime/buffered_block_mgr2.h
@@ -24,6 +24,7 @@
 
 #include "runtime/disk_io_mgr.h"
 #include "runtime/tmp_file_mgr.h"
+#include "util/uid_util.h"
 
 namespace doris {
 
diff --git a/be/src/runtime/bufferpool/buffer_pool.cc b/be/src/runtime/bufferpool/buffer_pool.cc
index 26c43f2b9e..9d11c0f58d 100644
--- a/be/src/runtime/bufferpool/buffer_pool.cc
+++ b/be/src/runtime/bufferpool/buffer_pool.cc
@@ -157,7 +157,7 @@ void BufferPool::DestroyPage(ClientHandle* client, PageHandle* handle) {
         // of cleaning up the page, freeing the buffer and updating reservations correctly.
         BufferHandle buffer;
         Status status = ExtractBuffer(client, handle, &buffer);
-        DCHECK(status.ok()) << status.get_error_msg();
+        DCHECK(status.ok()) << status;
         FreeBuffer(client, &buffer);
     } else {
         // In the unpinned case, no reservations are used so we just clean up the page.
@@ -597,9 +597,8 @@ string BufferPool::Client::DebugString() {
     std::lock_guard<std::mutex> lock(lock_);
     std::stringstream ss;
     ss << "<BufferPool::Client> " << this << " name: " << name_
-       << " write_status: " << write_status_.get_error_msg() << " buffers allocated "
-       << buffers_allocated_bytes_ << " num_pages: " << num_pages_
-       << " pinned_bytes: " << pinned_pages_.bytes()
+       << " write_status: " << write_status_ << " buffers allocated " << buffers_allocated_bytes_
+       << " num_pages: " << num_pages_ << " pinned_bytes: " << pinned_pages_.bytes()
        << " dirty_unpinned_bytes: " << dirty_unpinned_pages_.bytes()
        << " in_flight_write_bytes: " << in_flight_write_pages_.bytes();
     ss << "\n  " << pinned_pages_.size() << " pinned pages: ";
diff --git a/be/src/runtime/data_stream_sender.cpp b/be/src/runtime/data_stream_sender.cpp
index df1495b750..1394ee8875 100644
--- a/be/src/runtime/data_stream_sender.cpp
+++ b/be/src/runtime/data_stream_sender.cpp
@@ -280,7 +280,7 @@ Status DataStreamSender::Channel::close_internal() {
 Status DataStreamSender::Channel::close(RuntimeState* state) {
     Status st = close_internal();
     if (!st.ok()) {
-        state->log_error(st.get_error_msg());
+        state->log_error(st.to_string());
     }
     return st;
 }
@@ -289,7 +289,7 @@ Status DataStreamSender::Channel::close_wait(RuntimeState* state) {
     if (_need_close) {
         Status st = _wait_last_brpc();
         if (!st.ok()) {
-            state->log_error(st.get_error_msg());
+            state->log_error(st.to_string());
         }
         _need_close = false;
         return st;
diff --git a/be/src/runtime/disk_io_mgr_reader_context.cc b/be/src/runtime/disk_io_mgr_reader_context.cc
index 011b21615f..4ef71f1c94 100644
--- a/be/src/runtime/disk_io_mgr_reader_context.cc
+++ b/be/src/runtime/disk_io_mgr_reader_context.cc
@@ -198,8 +198,8 @@ string DiskIoMgr::RequestContext::debug_string() const {
     if (_state == RequestContext::Cancelled) ss << "Cancelled";
     if (_state == RequestContext::Active) ss << "Active";
     if (_state != RequestContext::Inactive) {
-        ss << " _status=" << (_status.ok() ? "OK" : _status.get_error_msg())
-           << " #ready_buffers=" << _num_ready_buffers << " #used_buffers=" << _num_used_buffers
+        ss << " _status=" << _status << " #ready_buffers=" << _num_ready_buffers
+           << " #used_buffers=" << _num_used_buffers
            << " #num_buffers_in_reader=" << _num_buffers_in_reader
            << " #finished_scan_ranges=" << _num_finished_ranges
            << " #disk_with_ranges=" << _num_disks_with_ranges
diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp
index d84b40c670..411aff1680 100644
--- a/be/src/runtime/exec_env_init.cpp
+++ b/be/src/runtime/exec_env_init.cpp
@@ -158,7 +158,7 @@ Status ExecEnv::_init(const std::vector<StorePath>& store_paths) {
     _cgroups_mgr->init_cgroups();
     Status status = _load_path_mgr->init();
     if (!status.ok()) {
-        LOG(ERROR) << "load path mgr init failed." << status.get_error_msg();
+        LOG(ERROR) << "load path mgr init failed." << status;
         exit(-1);
     }
     _broker_mgr->init();
diff --git a/be/src/runtime/fold_constant_executor.cpp b/be/src/runtime/fold_constant_executor.cpp
index 59d820c1ad..a782024eb1 100644
--- a/be/src/runtime/fold_constant_executor.cpp
+++ b/be/src/runtime/fold_constant_executor.cpp
@@ -167,14 +167,14 @@ Status FoldConstantExecutor::_init(const TQueryGlobals& query_globals) {
     Status status =
             DescriptorTbl::create(_runtime_state->obj_pool(), TDescriptorTable(), &desc_tbl);
     if (UNLIKELY(!status.ok())) {
-        LOG(WARNING) << "Failed to create descriptor table, msg: " << status.get_error_msg();
-        return Status::Uninitialized(status.get_error_msg());
+        LOG(WARNING) << "Failed to create descriptor table, msg: " << status;
+        return status;
     }
     _runtime_state->set_desc_tbl(desc_tbl);
     status = _runtime_state->init_mem_trackers(FoldConstantExecutor::_dummy_id);
     if (UNLIKELY(!status.ok())) {
-        LOG(WARNING) << "Failed to init mem trackers, msg: " << status.get_error_msg();
-        return Status::Uninitialized(status.get_error_msg());
+        LOG(WARNING) << "Failed to init mem trackers, msg: " << status;
+        return status;
     }
 
     _runtime_profile = _runtime_state->runtime_profile();
diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp
index e051a71bf7..1ed44838a4 100644
--- a/be/src/runtime/fragment_mgr.cpp
+++ b/be/src/runtime/fragment_mgr.cpp
@@ -743,7 +743,7 @@ Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params, Fi
                                "push plan fragment to thread pool failed");
             return Status::InternalError(strings::Substitute(
                     "push plan fragment $0 to thread pool failed. err = $1, BE: $2",
-                    print_id(params.params.fragment_instance_id), st.get_error_msg(),
+                    print_id(params.params.fragment_instance_id), st.to_string(),
                     BackendOptions::get_localhost()));
         }
     } else {
@@ -771,8 +771,8 @@ Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params, Fi
         if (!st.ok()) {
             context->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, "submit context fail");
             remove_pipeline_context(context);
-            return Status::InternalError("Submit pipeline failed. err = {}, BE: {}",
-                                         st.get_error_msg(), BackendOptions::get_localhost());
+            return Status::InternalError("Submit pipeline failed. err = {}, BE: {}", st.to_string(),
+                                         BackendOptions::get_localhost());
         }
     }
 
diff --git a/be/src/runtime/load_path_mgr.cpp b/be/src/runtime/load_path_mgr.cpp
index 4bcbc06eba..510512f2f5 100644
--- a/be/src/runtime/load_path_mgr.cpp
+++ b/be/src/runtime/load_path_mgr.cpp
@@ -32,6 +32,7 @@
 #include "util/file_utils.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 static const uint32_t MAX_SHARD_NUM = 1024;
 static const std::string SHARD_PREFIX = "__shard_";
@@ -98,8 +99,7 @@ Status LoadPathMgr::allocate_dir(const std::string& db, const std::string& label
             *prefix = path;
             return Status::OK();
         } else {
-            LOG(WARNING) << "create dir failed:" << path
-                         << ", error msg:" << status.get_error_msg();
+            LOG(WARNING) << "create dir failed:" << path << ", error msg:" << status;
         }
     }
 
@@ -179,7 +179,7 @@ void LoadPathMgr::clean_one_path(const std::string& path) {
     std::vector<std::string> dbs;
     Status status = FileUtils::list_files(env, path, &dbs);
     // path may not exist
-    if (!status.ok() && !status.is_not_found()) {
+    if (!status.ok() && !status.is<NOT_FOUND>()) {
         LOG(WARNING) << "scan one path to delete directory failed. path=" << path;
         return;
     }
diff --git a/be/src/runtime/mem_pool.h b/be/src/runtime/mem_pool.h
index 8a08f16797..e3b602e077 100644
--- a/be/src/runtime/mem_pool.h
+++ b/be/src/runtime/mem_pool.h
@@ -277,7 +277,7 @@ private:
     Status ALWAYS_INLINE allocate_safely(int64_t size, int alignment, uint8_t*& ret) {
         uint8_t* result = allocate<CHECK_LIMIT_FIRST>(size, alignment);
         if (result == nullptr) {
-            return Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
+            return Status::Error<ErrorCode::MEM_ALLOC_FAILED>();
         }
         ret = result;
         return Status::OK();
diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp
index d7ba70403e..b21ce4c28f 100644
--- a/be/src/runtime/plan_fragment_executor.cpp
+++ b/be/src/runtime/plan_fragment_executor.cpp
@@ -52,6 +52,7 @@
 #include "vec/runtime/vdata_stream_mgr.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 PlanFragmentExecutor::PlanFragmentExecutor(ExecEnv* exec_env,
                                            const report_status_callback& report_status_cb)
@@ -252,13 +253,13 @@ Status PlanFragmentExecutor::open() {
         status = open_internal();
     }
 
-    if (!status.ok() && !status.is_cancelled() && _runtime_state->log_has_space()) {
+    if (!status.ok() && !status.is<CANCELLED>() && _runtime_state->log_has_space()) {
         // Log error message in addition to returning in Status. Queries that do not
         // fetch results (e.g. insert) may not receive the message directly and can
         // only retrieve the log.
-        _runtime_state->log_error(status.get_error_msg());
+        _runtime_state->log_error(status.to_string());
     }
-    if (status.is_cancelled()) {
+    if (status.is<CANCELLED>()) {
         if (_cancel_reason == PPlanFragmentCancelReason::CALL_RPC_ERROR) {
             status = Status::RuntimeError(_cancel_msg);
         } else if (_cancel_reason == PPlanFragmentCancelReason::MEMORY_LIMIT_EXCEED) {
@@ -309,7 +310,7 @@ Status PlanFragmentExecutor::open_vectorized_internal() {
             }
 
             auto st = _sink->send(runtime_state(), block);
-            if (st.is_end_of_file()) {
+            if (st.is<END_OF_FILE>()) {
                 break;
             }
             RETURN_IF_ERROR(st);
@@ -407,7 +408,7 @@ Status PlanFragmentExecutor::open_internal() {
             _collect_query_statistics();
         }
         const Status& st = _sink->send(runtime_state(), batch);
-        if (st.is_end_of_file()) {
+        if (st.is<END_OF_FILE>()) {
             break;
         }
         RETURN_IF_ERROR(st);
@@ -609,8 +610,8 @@ void PlanFragmentExecutor::update_status(const Status& new_status) {
         std::lock_guard<std::mutex> l(_status_lock);
         // if current `_status` is ok, set it to `new_status` to record the error.
         if (_status.ok()) {
-            if (new_status.is_mem_limit_exceeded()) {
-                _runtime_state->set_mem_limit_exceeded(new_status.get_error_msg());
+            if (new_status.is<MEM_LIMIT_EXCEEDED>()) {
+                _runtime_state->set_mem_limit_exceeded(new_status.to_string());
             }
             _status = new_status;
             if (_runtime_state->query_type() == TQueryType::EXTERNAL) {
diff --git a/be/src/runtime/routine_load/data_consumer.cpp b/be/src/runtime/routine_load/data_consumer.cpp
index 60a0f93a44..ebd714ecd5 100644
--- a/be/src/runtime/routine_load/data_consumer.cpp
+++ b/be/src/runtime/routine_load/data_consumer.cpp
@@ -95,7 +95,7 @@ Status KafkaDataConsumer::init(StreamLoadContext* ctx) {
             Status st = ctx->exec_env()->small_file_mgr()->get_file(file_id, parts[2], &file_path);
             if (!st.ok()) {
                 return Status::InternalError("PAUSE: failed to get file for config: {}, error: {}",
-                                             item.first, st.get_error_msg());
+                                             item.first, st.to_string());
             }
             RETURN_IF_ERROR(set_conf(item.first, file_path));
         } else {
diff --git a/be/src/runtime/routine_load/data_consumer_group.cpp b/be/src/runtime/routine_load/data_consumer_group.cpp
index bdf3225a53..0640591ece 100644
--- a/be/src/runtime/routine_load/data_consumer_group.cpp
+++ b/be/src/runtime/routine_load/data_consumer_group.cpp
@@ -140,7 +140,7 @@ Status KafkaDataConsumerGroup::start_all(StreamLoadContext* ctx) {
             _thread_pool.shutdown();
             _thread_pool.join();
             if (!result_st.ok()) {
-                kafka_pipe->cancel(result_st.get_error_msg());
+                kafka_pipe->cancel(result_st.to_string());
                 return result_st;
             }
             kafka_pipe->finish();
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 b67ff3ea58..a5535199e2 100644
--- a/be/src/runtime/routine_load/routine_load_task_executor.cpp
+++ b/be/src/runtime/routine_load/routine_load_task_executor.cpp
@@ -31,6 +31,7 @@
 #include "util/uid_util.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(routine_load_task_count, MetricUnit::NOUNIT);
 
@@ -225,8 +226,8 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) {
                                                 std::unique_lock<std::mutex> l(_lock);
                                                 _task_map.erase(ctx->id);
                                                 LOG(INFO) << "finished routine load task "
-                                                          << ctx->brief() << ", status: "
-                                                          << ctx->status.get_error_msg()
+                                                          << ctx->brief()
+                                                          << ", status: " << ctx->status
                                                           << ", current tasks num: "
                                                           << _task_map.size();
                                                 if (ctx->unref()) {
@@ -249,14 +250,14 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) {
 
 void RoutineLoadTaskExecutor::exec_task(StreamLoadContext* ctx, DataConsumerPool* consumer_pool,
                                         ExecFinishCallback cb) {
-#define HANDLE_ERROR(stmt, err_msg)                                                        \
-    do {                                                                                   \
-        Status _status_ = (stmt);                                                          \
-        if (UNLIKELY(!_status_.ok() && _status_.code() != TStatusCode::PUBLISH_TIMEOUT)) { \
-            err_handler(ctx, _status_, err_msg);                                           \
-            cb(ctx);                                                                       \
-            return;                                                                        \
-        }                                                                                  \
+#define HANDLE_ERROR(stmt, err_msg)                                        \
+    do {                                                                   \
+        Status _status_ = (stmt);                                          \
+        if (UNLIKELY(!_status_.ok() && !_status_.is<PUBLISH_TIMEOUT>())) { \
+            err_handler(ctx, _status_, err_msg);                           \
+            cb(ctx);                                                       \
+            return;                                                        \
+        }                                                                  \
     } while (false);
 
     LOG(INFO) << "begin to execute routine load task: " << ctx->brief();
@@ -273,7 +274,7 @@ void RoutineLoadTaskExecutor::exec_task(StreamLoadContext* ctx, DataConsumerPool
         Status st = std::static_pointer_cast<KafkaDataConsumerGroup>(consumer_grp)
                             ->assign_topic_partitions(ctx);
         if (!st.ok()) {
-            err_handler(ctx, st, st.get_error_msg());
+            err_handler(ctx, st, st.to_string());
             cb(ctx);
             return;
         }
@@ -324,7 +325,7 @@ void RoutineLoadTaskExecutor::exec_task(StreamLoadContext* ctx, DataConsumerPool
         if (!st.ok()) {
             // Kafka Offset Commit is idempotent, Failure should not block the normal process
             // So just print a warning
-            LOG(WARNING) << st.get_error_msg();
+            LOG(WARNING) << st;
             break;
         }
 
@@ -339,7 +340,7 @@ void RoutineLoadTaskExecutor::exec_task(StreamLoadContext* ctx, DataConsumerPool
         if (!st.ok()) {
             // Kafka Offset Commit is idempotent, Failure should not block the normal process
             // So just print a warning
-            LOG(WARNING) << st.get_error_msg();
+            LOG(WARNING) << st;
         }
         _data_consumer_pool.return_consumer(consumer);
 
diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp
index b18671a9b8..9688bbf441 100644
--- a/be/src/runtime/runtime_state.cpp
+++ b/be/src/runtime/runtime_state.cpp
@@ -42,6 +42,7 @@
 #include "util/uid_util.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 // for ut only
 RuntimeState::RuntimeState(const TUniqueId& fragment_instance_id,
@@ -250,14 +251,6 @@ bool RuntimeState::log_error(const std::string& error) {
     return false;
 }
 
-void RuntimeState::log_error(const Status& status) {
-    if (status.ok()) {
-        return;
-    }
-
-    log_error(status.get_error_msg());
-}
-
 void RuntimeState::get_unreported_errors(std::vector<std::string>* new_errors) {
     std::lock_guard<std::mutex> l(_error_log_lock);
 
@@ -274,7 +267,7 @@ Status RuntimeState::set_mem_limit_exceeded(const std::string& msg) {
             _process_status = Status::MemoryLimitExceeded(msg);
         }
     }
-    DCHECK(_process_status.is_mem_limit_exceeded());
+    DCHECK(_process_status.is<MEM_LIMIT_EXCEEDED>());
     return _process_status;
 }
 
@@ -335,7 +328,7 @@ Status RuntimeState::append_error_msg_to_file(std::function<std::string()> line,
     if (_error_log_file == nullptr) {
         Status status = create_error_log_file();
         if (!status.ok()) {
-            LOG(WARNING) << "Create error file log failed. because: " << status.get_error_msg();
+            LOG(WARNING) << "Create error file log failed. because: " << status;
             if (_error_log_file != nullptr) {
                 _error_log_file->close();
                 delete _error_log_file;
@@ -388,7 +381,7 @@ void RuntimeState::export_load_error(const std::string& err_msg) {
             Status st = LoadErrorHub::create_hub(_exec_env, _load_error_hub_info.get(),
                                                  _error_log_file_path, &_error_hub);
             if (!st.ok()) {
-                LOG(WARNING) << "failed to create load error hub: " << st.get_error_msg();
+                LOG(WARNING) << "failed to create load error hub: " << st;
                 return;
             }
         }
diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h
index 016f0089fd..5af2f7df55 100644
--- a/be/src/runtime/runtime_state.h
+++ b/be/src/runtime/runtime_state.h
@@ -157,9 +157,6 @@ public:
     // Appends error to the _error_log if there is space
     bool log_error(const std::string& error);
 
-    // If !status.ok(), appends the error to the _error_log
-    void log_error(const Status& status);
-
     // Returns true if the error log has not reached _max_errors.
     bool log_has_space() {
         std::lock_guard<std::mutex> l(_error_log_lock);
diff --git a/be/src/runtime/small_file_mgr.cpp b/be/src/runtime/small_file_mgr.cpp
index 3e542bb551..883cdbc5c8 100644
--- a/be/src/runtime/small_file_mgr.cpp
+++ b/be/src/runtime/small_file_mgr.cpp
@@ -63,7 +63,7 @@ Status SmallFileMgr::_load_local_files() {
         }
         auto st = _load_single_file(_local_path, file);
         if (!st.ok()) {
-            LOG(WARNING) << "load small file failed: " << st.get_error_msg();
+            LOG(WARNING) << "load small file failed: " << st;
         }
         return true;
     };
diff --git a/be/src/runtime/snapshot_loader.cpp b/be/src/runtime/snapshot_loader.cpp
index 11c8930d11..c4b2a390df 100644
--- a/be/src/runtime/snapshot_loader.cpp
+++ b/be/src/runtime/snapshot_loader.cpp
@@ -121,8 +121,7 @@ Status SnapshotLoader::upload(const std::map<std::string, std::string>& src_to_d
             status = FileUtils::md5sum(src_path + "/" + local_file, &md5sum);
             if (!status.ok()) {
                 std::stringstream ss;
-                ss << "failed to get md5sum of file: " << local_file << ": "
-                   << status.get_error_msg();
+                ss << "failed to get md5sum of file: " << local_file << ": " << status;
                 LOG(WARNING) << ss.str();
                 return Status::InternalError(ss.str());
             }
@@ -253,7 +252,7 @@ Status SnapshotLoader::download(const std::map<std::string, std::string>& src_to
                     Status st = FileUtils::md5sum(local_path + "/" + remote_file, &local_md5sum);
                     if (!st.ok()) {
                         LOG(WARNING) << "failed to get md5sum of local file: " << remote_file
-                                     << ". msg: " << st.get_error_msg() << ". download it";
+                                     << ". msg: " << st << ". download it";
                         need_download = true;
                     } else {
                         VLOG_CRITICAL << "get local file checksum: " << remote_file << ": "
@@ -295,8 +294,7 @@ Status SnapshotLoader::download(const std::map<std::string, std::string>& src_to
             status = FileUtils::md5sum(full_local_file, &downloaded_md5sum);
             if (!status.ok()) {
                 std::stringstream ss;
-                ss << "failed to get md5sum of file: " << full_local_file
-                   << ", err: " << status.get_error_msg();
+                ss << "failed to get md5sum of file: " << full_local_file << ", err: " << status;
                 LOG(WARNING) << ss.str();
                 return Status::InternalError(ss.str());
             }
@@ -323,8 +321,8 @@ Status SnapshotLoader::download(const std::map<std::string, std::string>& src_to
             std::string new_name;
             Status st = _replace_tablet_id(local_file, remote_tablet_id, &new_name);
             if (!st.ok()) {
-                LOG(WARNING) << "failed to replace tablet id. unknown local file: "
-                             << st.get_error_msg() << ". ignore it";
+                LOG(WARNING) << "failed to replace tablet id. unknown local file: " << st
+                             << ". ignore it";
                 continue;
             }
             VLOG_CRITICAL << "new file name after replace tablet id: " << new_name;
@@ -542,8 +540,7 @@ Status SnapshotLoader::_get_existing_files_from_local(const std::string& local_p
     Status status = FileUtils::list_files(Env::Default(), local_path, local_files);
     if (!status.ok()) {
         std::stringstream ss;
-        ss << "failed to list files in local path: " << local_path
-           << ", msg: " << status.get_error_msg();
+        ss << "failed to list files in local path: " << local_path << ", msg: " << status;
         LOG(WARNING) << ss.str();
         return status;
     }
diff --git a/be/src/runtime/stream_load/stream_load_context.cpp b/be/src/runtime/stream_load/stream_load_context.cpp
index 172b32ff6c..93544cfbe3 100644
--- a/be/src/runtime/stream_load/stream_load_context.cpp
+++ b/be/src/runtime/stream_load/stream_load_context.cpp
@@ -20,6 +20,7 @@
 #include <sstream>
 
 namespace doris {
+using namespace ErrorCode;
 
 std::string StreamLoadContext::to_json() const {
     rapidjson::StringBuffer s;
@@ -41,13 +42,13 @@ std::string StreamLoadContext::to_json() const {
     // status
     writer.Key("Status");
     switch (status.code()) {
-    case TStatusCode::OK:
+    case OK:
         writer.String("Success");
         break;
-    case TStatusCode::PUBLISH_TIMEOUT:
+    case PUBLISH_TIMEOUT:
         writer.String("Publish Timeout");
         break;
-    case TStatusCode::LABEL_ALREADY_EXISTS:
+    case LABEL_ALREADY_EXISTS:
         writer.String("Label Already Exists");
         writer.Key("ExistingJobStatus");
         writer.String(existing_job_status.c_str());
@@ -61,7 +62,7 @@ std::string StreamLoadContext::to_json() const {
     if (status.ok()) {
         writer.String("OK");
     } else {
-        writer.String(status.get_error_msg().c_str());
+        writer.String(status.to_string().c_str());
     }
     // number_load_rows
     writer.Key("NumberTotalRows");
@@ -267,10 +268,10 @@ std::string StreamLoadContext::to_json_for_mini_load() const {
     bool show_ok = true;
     writer.Key("status");
     switch (status.code()) {
-    case TStatusCode::OK:
+    case OK:
         writer.String("Success");
         break;
-    case TStatusCode::PUBLISH_TIMEOUT:
+    case PUBLISH_TIMEOUT:
         // treat PUBLISH_TIMEOUT as OK in mini load
         writer.String("Success");
         break;
@@ -284,7 +285,7 @@ std::string StreamLoadContext::to_json_for_mini_load() const {
     if (status.ok() || show_ok) {
         writer.String("OK");
     } else {
-        writer.String(status.get_error_msg().c_str());
+        writer.String(status.to_string().c_str());
     }
     writer.EndObject();
     return s.GetString();
diff --git a/be/src/runtime/stream_load/stream_load_executor.cpp b/be/src/runtime/stream_load/stream_load_executor.cpp
index 298cb1fae9..34336ed427 100644
--- a/be/src/runtime/stream_load/stream_load_executor.cpp
+++ b/be/src/runtime/stream_load/stream_load_executor.cpp
@@ -30,6 +30,7 @@
 #include "util/thrift_rpc_helper.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 #ifdef BE_TEST
 TLoadTxnBeginResult k_stream_load_begin_result;
@@ -80,10 +81,10 @@ Status StreamLoadExecutor::execute_plan_fragment(StreamLoadContext* ctx) {
                     LOG(WARNING) << "fragment execute failed"
                                  << ", query_id="
                                  << UniqueId(ctx->put_result.params.params.query_id)
-                                 << ", err_msg=" << status->get_error_msg() << ", " << ctx->brief();
+                                 << ", err_msg=" << status->to_string() << ", " << ctx->brief();
                     // cancel body_sink, make sender known it
                     if (ctx->body_sink != nullptr) {
-                        ctx->body_sink->cancel(status->get_error_msg());
+                        ctx->body_sink->cancel(status->to_string());
                     }
 
                     switch (ctx->load_src_type) {
@@ -105,7 +106,7 @@ Status StreamLoadExecutor::execute_plan_fragment(StreamLoadContext* ctx) {
                     // However, the http connection may still be sending data to stream_load_pipe
                     // and waiting for it to be consumed.
                     // Therefore, we need to actively cancel to end the pipe.
-                    ctx->body_sink->cancel(status->get_error_msg());
+                    ctx->body_sink->cancel(status->to_string());
                 }
 
                 if (ctx->need_commit_self && ctx->body_sink != nullptr) {
@@ -164,8 +165,7 @@ Status StreamLoadExecutor::begin_txn(StreamLoadContext* ctx) {
         status = Status(result.status);
     }
     if (!status.ok()) {
-        LOG(WARNING) << "begin transaction failed, errmsg=" << status.get_error_msg()
-                     << ctx->brief();
+        LOG(WARNING) << "begin transaction failed, errmsg=" << status << ctx->brief();
         if (result.__isset.job_status) {
             ctx->existing_job_status = result.job_status;
         }
@@ -201,9 +201,8 @@ Status StreamLoadExecutor::pre_commit_txn(StreamLoadContext* ctx) {
     // rollback this transaction
     Status status(result.status);
     if (!status.ok()) {
-        LOG(WARNING) << "precommit transaction failed, errmsg=" << status.get_error_msg()
-                     << ctx->brief();
-        if (status.code() == TStatusCode::PUBLISH_TIMEOUT) {
+        LOG(WARNING) << "precommit transaction failed, errmsg=" << status << ctx->brief();
+        if (status.is<PUBLISH_TIMEOUT>()) {
             ctx->need_rollback = false;
         }
         return status;
@@ -231,7 +230,7 @@ Status StreamLoadExecutor::operate_txn_2pc(StreamLoadContext* ctx) {
             config::txn_commit_rpc_timeout_ms));
     Status status(result.status);
     if (!status.ok()) {
-        LOG(WARNING) << "2PC commit transaction failed, errmsg=" << status.get_error_msg();
+        LOG(WARNING) << "2PC commit transaction failed, errmsg=" << status;
         return status;
     }
     return Status::OK();
@@ -283,9 +282,8 @@ Status StreamLoadExecutor::commit_txn(StreamLoadContext* ctx) {
     // rollback this transaction
     Status status(result.status);
     if (!status.ok()) {
-        LOG(WARNING) << "commit transaction failed, errmsg=" << status.get_error_msg() << ", "
-                     << ctx->brief();
-        if (status.code() == TStatusCode::PUBLISH_TIMEOUT) {
+        LOG(WARNING) << "commit transaction failed, errmsg=" << status << ", " << ctx->brief();
+        if (status.is<PUBLISH_TIMEOUT>()) {
             ctx->need_rollback = false;
         }
         return status;
@@ -308,7 +306,7 @@ void StreamLoadExecutor::rollback_txn(StreamLoadContext* ctx) {
     }
     request.tbl = ctx->table;
     request.txnId = ctx->txn_id;
-    request.__set_reason(ctx->status.get_error_msg());
+    request.__set_reason(ctx->status.to_string());
 
     // set attachment if has
     TTxnCommitAttachment attachment;
@@ -325,8 +323,7 @@ void StreamLoadExecutor::rollback_txn(StreamLoadContext* ctx) {
                 client->loadTxnRollback(result, request);
             });
     if (!rpc_st.ok()) {
-        LOG(WARNING) << "transaction rollback failed. errmsg=" << rpc_st.get_error_msg()
-                     << ctx->brief();
+        LOG(WARNING) << "transaction rollback failed. errmsg=" << rpc_st << ctx->brief();
     }
 #else
     result = k_stream_load_rollback_result;
diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp
index d6c81fb845..0624bcb94a 100644
--- a/be/src/runtime/tablets_channel.cpp
+++ b/be/src/runtime/tablets_channel.cpp
@@ -190,9 +190,9 @@ void TabletsChannel::_close_wait(DeltaWriter* writer,
     } else {
         PTabletError* tablet_error = tablet_errors->Add();
         tablet_error->set_tablet_id(writer->tablet_id());
-        tablet_error->set_msg(st.get_error_msg());
+        tablet_error->set_msg(st.to_string());
         VLOG_PROGRESS << "close wait failed tablet " << writer->tablet_id() << " transaction_id "
-                      << _txn_id << "err msg " << st.get_error_msg();
+                      << _txn_id << "err msg " << st;
     }
 }
 
@@ -284,12 +284,11 @@ void TabletsChannel::reduce_mem_usage() {
             for (int i = 0; i < counter; i++) {
                 Status st = writers[i]->flush_memtable_and_wait(false);
                 if (!st.ok()) {
-                    auto err_msg = strings::Substitute(
-                            "tablet writer failed to reduce mem consumption by flushing memtable, "
-                            "tablet_id=$0, txn_id=$1, err=$2, errcode=$3, msg:$4",
-                            writers[i]->tablet_id(), _txn_id, st.code(), st.precise_code(),
-                            st.get_error_msg());
-                    LOG(WARNING) << err_msg;
+                    LOG_WARNING(
+                            "tablet writer failed to reduce mem consumption by flushing memtable")
+                            .tag("tablet_id", writers[i]->tablet_id())
+                            .tag("txn_id", _txn_id)
+                            .error(st);
                     writers[i]->cancel_with_status(st);
                     _broken_tablets.insert(writers[i]->tablet_id());
                 }
@@ -312,11 +311,11 @@ void TabletsChannel::reduce_mem_usage() {
     for (auto writer : writers_to_wait_flush) {
         Status st = writer->wait_flush();
         if (!st.ok()) {
-            auto err_msg = strings::Substitute(
-                    "tablet writer failed to reduce mem consumption by waiting flush memtable, "
-                    "tablet_id=$0, txn_id=$1, err=$2, errcode=$3, msg:$4",
-                    writer->tablet_id(), _txn_id, st.code(), st.precise_code(), st.get_error_msg());
-            LOG(WARNING) << err_msg;
+            LOG_WARNING(
+                    "tablet writer failed to reduce mem consumption by waiting flushing memtable")
+                    .tag("tablet_id", writer->tablet_id())
+                    .tag("txn_id", _txn_id)
+                    .error(st);
             writer->cancel_with_status(st);
             _broken_tablets.insert(writer->tablet_id());
         }
@@ -478,11 +477,9 @@ Status TabletsChannel::add_batch(const TabletWriterAddRequest& request,
 
         Status st = tablet_writer_it->second->write(&send_data, tablet_to_rowidxs_it.second);
         if (!st.ok()) {
-            auto err_msg = strings::Substitute(
-                    "tablet writer write failed, tablet_id=$0, txn_id=$1, err=$2"
-                    ", errcode=$3, msg:$4",
-                    tablet_to_rowidxs_it.first, _txn_id, st.code(), st.precise_code(),
-                    st.get_error_msg());
+            auto err_msg =
+                    fmt::format("tablet writer write failed, tablet_id={}, txn_id={}, err={}",
+                                tablet_to_rowidxs_it.first, _txn_id, st);
             LOG(WARNING) << err_msg;
             PTabletError* error = tablet_errors->Add();
             error->set_tablet_id(tablet_to_rowidxs_it.first);
diff --git a/be/src/runtime/tmp_file_mgr.cc b/be/src/runtime/tmp_file_mgr.cc
index 639d411b13..2a38b1663e 100644
--- a/be/src/runtime/tmp_file_mgr.cc
+++ b/be/src/runtime/tmp_file_mgr.cc
@@ -88,7 +88,7 @@ Status TmpFileMgr::init_custom(const vector<string>& tmp_dirs, bool one_dir_per_
         Status status = FileSystemUtil::verify_is_directory(tmp_path.string());
         if (!status.ok()) {
             LOG(WARNING) << "Cannot use directory " << tmp_path.string()
-                         << " for scratch: " << status.get_error_msg();
+                         << " for scratch: " << status;
             continue;
         }
         // Find the disk id of tmp_path. Add the scratch directory if there isn't another
@@ -116,7 +116,7 @@ Status TmpFileMgr::init_custom(const vector<string>& tmp_dirs, bool one_dir_per_
             } else {
                 LOG(WARNING) << "Could not remove and recreate directory "
                              << scratch_subdir_path.string() << ": cannot use it for scratch. "
-                             << "Error was: " << status.get_error_msg();
+                             << "Error was: " << status;
             }
         }
     }
@@ -228,7 +228,7 @@ Status TmpFileMgr::File::allocate_space(int64_t write_size, int64_t* offset) {
         // First call to AllocateSpace. Create the file.
         status = FileSystemUtil::create_file(_path);
         if (!status.ok()) {
-            report_io_error(status.get_error_msg());
+            report_io_error(status.to_string());
             return status;
         }
         _disk_id = DiskInfo::disk_id(_path.c_str());
@@ -236,7 +236,7 @@ Status TmpFileMgr::File::allocate_space(int64_t write_size, int64_t* offset) {
     int64_t new_size = _current_size + write_size;
     status = FileSystemUtil::resize_file(_path, new_size);
     if (!status.ok()) {
-        report_io_error(status.get_error_msg());
+        report_io_error(status.to_string());
         return status;
     }
     *offset = _current_size;
diff --git a/be/src/runtime/user_function_cache.cpp b/be/src/runtime/user_function_cache.cpp
index 5fdd1b16f8..d90e8eddac 100644
--- a/be/src/runtime/user_function_cache.cpp
+++ b/be/src/runtime/user_function_cache.cpp
@@ -436,7 +436,7 @@ Status UserFunctionCache::check_jar(int64_t fid, const std::string& url,
         return Status::InternalError(
                 "Java UDAF has error, maybe you should check the path about java impl jar, because "
                 "{}",
-                st.get_error_msg());
+                st.to_string());
     }
     return Status::OK();
 }
diff --git a/be/src/service/backend_options.cpp b/be/src/service/backend_options.cpp
index a0b03e35bd..bfbb95763d 100644
--- a/be/src/service/backend_options.cpp
+++ b/be/src/service/backend_options.cpp
@@ -41,7 +41,7 @@ bool BackendOptions::init() {
     Status status = get_hosts_v4(&hosts);
 
     if (!status.ok()) {
-        LOG(FATAL) << status.get_error_msg();
+        LOG(FATAL) << status;
         return false;
     }
 
diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp
index b584668f58..fd34f81487 100644
--- a/be/src/service/backend_service.cpp
+++ b/be/src/service/backend_service.cpp
@@ -142,7 +142,7 @@ void BackendService::transmit_data(TTransmitDataResult& return_val,
         //        params.sender_id,
         //        params.be_number);
         //VLOG_ROW << "params.eos: " << (params.eos ? "true" : "false")
-        //        << " close_sender status: " << status.get_error_msg();
+        //        << " close_sender status: " << status;
         //status.set_t_status(&return_val);
     }
 }
@@ -164,7 +164,7 @@ void BackendService::submit_export_task(TStatus& t_status, const TExportTaskRequ
     //    } else {
     //        VLOG_RPC << "start export task failed id="
     //            << request.params.params.fragment_instance_id
-    //            << " and err_msg=" << status.get_error_msg();
+    //            << " and err_msg=" << status;
     //    }
     //    status.to_thrift(&t_status);
 }
@@ -193,7 +193,7 @@ void BackendService::erase_export_task(TStatus& t_status, const TUniqueId& task_
     //    Status status = _exec_env->export_task_mgr()->erase_task(task_id);
     //    if (!status.ok()) {
     //        LOG(WARNING) << "delete export task failed. because "
-    //            << status.get_error_msg() << " with task_id " << task_id;
+    //            << status << " with task_id " << task_id;
     //    } else {
     //        VLOG_RPC << "delete export task successful with task_id " << task_id;
     //    }
diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp
index 11460be3ee..a02d290e0d 100644
--- a/be/src/service/doris_main.cpp
+++ b/be/src/service/doris_main.cpp
@@ -379,7 +379,7 @@ int main(int argc, char** argv) {
         // Init jni
         status = doris::JniUtil::Init();
         if (!status.ok()) {
-            LOG(WARNING) << "Failed to initialize JNI: " << status.get_error_msg();
+            LOG(WARNING) << "Failed to initialize JNI: " << status;
             exit(1);
         }
     }
@@ -405,7 +405,7 @@ int main(int argc, char** argv) {
     doris::StorageEngine* engine = nullptr;
     auto st = doris::StorageEngine::open(options, &engine);
     if (!st.ok()) {
-        LOG(FATAL) << "fail to open StorageEngine, res=" << st.get_error_msg();
+        LOG(FATAL) << "fail to open StorageEngine, res=" << st;
         exit(-1);
     }
     exec_env->set_storage_engine(engine);
@@ -488,8 +488,7 @@ int main(int argc, char** argv) {
 
     status = heartbeat_thrift_server->start();
     if (!status.ok()) {
-        LOG(ERROR) << "Doris BE HeartBeat Service did not start correctly, exiting: "
-                   << status.get_error_msg();
+        LOG(ERROR) << "Doris BE HeartBeat Service did not start correctly, exiting: " << status;
         doris::shutdown_logging();
         exit(1);
     }
diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp
index c33db5a884..3aa142d401 100644
--- a/be/src/service/internal_service.cpp
+++ b/be/src/service/internal_service.cpp
@@ -57,6 +57,7 @@
 #include "vec/runtime/vdata_stream_mgr.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 const uint32_t DOWNLOAD_FILE_MAX_RETRY = 3;
 
@@ -152,7 +153,7 @@ void PInternalServiceImpl::_transmit_data(google::protobuf::RpcController* cntl_
     if (extract_st.ok()) {
         st = _exec_env->stream_mgr()->transmit_data(request, &done);
         if (!st.ok()) {
-            LOG(WARNING) << "transmit_data failed, message=" << st.get_error_msg()
+            LOG(WARNING) << "transmit_data failed, message=" << st
                          << ", fragment_instance_id=" << print_id(request->finst_id())
                          << ", node=" << request->node_id();
         }
@@ -174,9 +175,8 @@ void PInternalServiceImpl::tablet_writer_open(google::protobuf::RpcController* c
     brpc::ClosureGuard closure_guard(done);
     auto st = _exec_env->load_channel_mgr()->open(*request);
     if (!st.ok()) {
-        LOG(WARNING) << "load channel open failed, message=" << st.get_error_msg()
-                     << ", id=" << request->id() << ", index_id=" << request->index_id()
-                     << ", txn_id=" << request->txn_id();
+        LOG(WARNING) << "load channel open failed, message=" << st << ", id=" << request->id()
+                     << ", index_id=" << request->index_id() << ", txn_id=" << request->txn_id();
     }
     st.to_protobuf(response->mutable_status());
 }
@@ -194,7 +194,7 @@ void PInternalServiceImpl::exec_plan_fragment(google::protobuf::RpcController* c
             request->has_version() ? request->version() : PFragmentRequestVersion::VERSION_1;
     st = _exec_plan_fragment(request->request(), version, compact);
     if (!st.ok()) {
-        LOG(WARNING) << "exec plan fragment failed, errmsg=" << st.get_error_msg();
+        LOG(WARNING) << "exec plan fragment failed, errmsg=" << st;
     }
     st.to_protobuf(response->mutable_status());
 }
@@ -262,7 +262,7 @@ void PInternalServiceImpl::_tablet_writer_add_block(google::protobuf::RpcControl
 
             auto st = _exec_env->load_channel_mgr()->add_batch(*request, response);
             if (!st.ok()) {
-                LOG(WARNING) << "tablet writer add block failed, message=" << st.get_error_msg()
+                LOG(WARNING) << "tablet writer add block failed, message=" << st
                              << ", id=" << request->id() << ", index_id=" << request->index_id()
                              << ", sender_id=" << request->sender_id()
                              << ", backend id=" << request->backend_id();
@@ -322,7 +322,7 @@ void PInternalServiceImpl::_tablet_writer_add_batch(google::protobuf::RpcControl
 
             auto st = _exec_env->load_channel_mgr()->add_batch(*request, response);
             if (!st.ok()) {
-                LOG(WARNING) << "tablet writer add batch failed, message=" << st.get_error_msg()
+                LOG(WARNING) << "tablet writer add batch failed, message=" << st
                              << ", id=" << request->id() << ", index_id=" << request->index_id()
                              << ", sender_id=" << request->sender_id()
                              << ", backend id=" << request->backend_id();
@@ -423,7 +423,7 @@ void PInternalServiceImpl::fetch_table_schema(google::protobuf::RpcController* c
         uint32_t len = request->file_scan_range().size();
         st = deserialize_thrift_msg(buf, &len, false, &file_scan_range);
         if (!st.ok()) {
-            LOG(WARNING) << "fetch table schema failed, errmsg=" << st.get_error_msg();
+            LOG(WARNING) << "fetch table schema failed, errmsg=" << st;
             st.to_protobuf(result->mutable_status());
             return;
         }
@@ -479,7 +479,7 @@ void PInternalServiceImpl::fetch_table_schema(google::protobuf::RpcController* c
     std::vector<TypeDescriptor> col_types;
     st = reader->get_parsered_schema(&col_names, &col_types);
     if (!st.ok()) {
-        LOG(WARNING) << "fetch table schema failed, errmsg=" << st.get_error_msg();
+        LOG(WARNING) << "fetch table schema failed, errmsg=" << st;
         st.to_protobuf(result->mutable_status());
         return;
     }
@@ -676,7 +676,7 @@ void PInternalServiceImpl::fold_constant_expr(google::protobuf::RpcController* c
         st = _fold_constant_expr(cntl->request_attachment().to_string(), response);
     }
     if (!st.ok()) {
-        LOG(WARNING) << "exec fold constant expr failed, errmsg=" << st.get_error_msg();
+        LOG(WARNING) << "exec fold constant expr failed, errmsg=" << st;
     }
     st.to_protobuf(response->mutable_status());
 }
@@ -740,7 +740,7 @@ void PInternalServiceImpl::_transmit_block(google::protobuf::RpcController* cntl
     if (extract_st.ok()) {
         st = _exec_env->vstream_mgr()->transmit_block(request, &done);
         if (!st.ok()) {
-            LOG(WARNING) << "transmit_block failed, message=" << st.get_error_msg()
+            LOG(WARNING) << "transmit_block failed, message=" << st
                          << ", fragment_instance_id=" << print_id(request->finst_id())
                          << ", node=" << request->node_id();
         }
@@ -959,9 +959,7 @@ void PInternalServiceImpl::request_slave_tablet_pull_rowset(
                 tablet->data_dir()->get_meta(), rowset_meta->partition_id(), rowset_meta->txn_id(),
                 rowset_meta->tablet_id(), rowset_meta->tablet_schema_hash(), tablet->tablet_uid(),
                 rowset_meta->load_id(), rowset, true);
-        if (!commit_txn_status &&
-            commit_txn_status !=
-                    Status::OLAPInternalError(OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST)) {
+        if (!commit_txn_status && !commit_txn_status.is<PUSH_TRANSACTION_ALREADY_EXIST>()) {
             LOG(WARNING) << "failed to add committed rowset for slave replica. rowset_id="
                          << rowset_meta->rowset_id() << ", tablet_id=" << rowset_meta->tablet_id()
                          << ", txn_id=" << rowset_meta->txn_id();
diff --git a/be/src/tools/meta_tool.cpp b/be/src/tools/meta_tool.cpp
index 376f059759..1da3ef4787 100644
--- a/be/src/tools/meta_tool.cpp
+++ b/be/src/tools/meta_tool.cpp
@@ -113,7 +113,7 @@ void get_meta(DataDir* data_dir) {
     std::string value;
     Status s =
             TabletMetaManager::get_json_meta(data_dir, FLAGS_tablet_id, FLAGS_schema_hash, &value);
-    if (s.precise_code() == doris::OLAP_ERR_META_KEY_NOT_FOUND) {
+    if (s.is<doris::ErrorCode::META_KEY_NOT_FOUND>()) {
         std::cout << "no tablet meta for tablet_id:" << FLAGS_tablet_id
                   << ", schema_hash:" << FLAGS_schema_hash << std::endl;
         return;
diff --git a/be/src/util/broker_storage_backend.cpp b/be/src/util/broker_storage_backend.cpp
index a574c68165..2aa8ead58a 100644
--- a/be/src/util/broker_storage_backend.cpp
+++ b/be/src/util/broker_storage_backend.cpp
@@ -154,7 +154,7 @@ Status BrokerStorageBackend::rename(const std::string& orig_name, const std::str
     if (!status.ok()) {
         std::stringstream ss;
         ss << "failed to get broker client. "
-           << "broker addr: " << _broker_addr << ". msg: " << status.get_error_msg();
+           << "broker addr: " << _broker_addr << ". msg: " << status;
         LOG(WARNING) << ss.str();
         return Status::InternalError(ss.str());
     }
@@ -204,7 +204,7 @@ Status BrokerStorageBackend::list(const std::string& remote_path, bool contain_m
     if (!status.ok()) {
         std::stringstream ss;
         ss << "failed to get broker client. "
-           << "broker addr: " << _broker_addr << ". msg: " << status.get_error_msg();
+           << "broker addr: " << _broker_addr << ". msg: " << status;
         LOG(WARNING) << ss.str();
         return Status::InternalError(ss.str());
     }
@@ -290,7 +290,7 @@ Status BrokerStorageBackend::rm(const std::string& remote) {
     if (!status.ok()) {
         std::stringstream ss;
         ss << "failed to get broker client. "
-           << "broker addr: " << _broker_addr << ". msg: " << status.get_error_msg();
+           << "broker addr: " << _broker_addr << ". msg: " << status;
         LOG(WARNING) << ss.str();
         return Status::InternalError(ss.str());
     }
@@ -352,7 +352,7 @@ Status BrokerStorageBackend::exist(const std::string& path) {
     if (!status.ok()) {
         std::stringstream ss;
         ss << "failed to get broker client. "
-           << "broker addr: " << _broker_addr << ". msg: " << status.get_error_msg();
+           << "broker addr: " << _broker_addr << ". msg: " << status;
         LOG(WARNING) << ss.str();
         return Status::InternalError(ss.str());
     }
diff --git a/be/src/util/cgroup_util.cpp b/be/src/util/cgroup_util.cpp
index 63b3f61e63..0e3be6cde5 100644
--- a/be/src/util/cgroup_util.cpp
+++ b/be/src/util/cgroup_util.cpp
@@ -203,7 +203,7 @@ std::string CGroupUtil::debug_string() {
     if (status.ok()) {
         mem_limit_str = strings::Substitute("$0", mem_limit);
     } else {
-        mem_limit_str = status.get_error_msg();
+        mem_limit_str = status.to_string();
     }
     string cpu_limit_str;
     float cpu_limit;
@@ -217,7 +217,7 @@ std::string CGroupUtil::debug_string() {
             cpu_limit_str = "unlimited";
         }
     } else {
-        cpu_limit_str = status.get_error_msg();
+        cpu_limit_str = status.to_string();
     }
     return strings::Substitute("Process CGroup Info: memory.limit_in_bytes=$0, cpu cfs limits: $1",
                                mem_limit_str, cpu_limit_str);
diff --git a/be/src/util/hdfs_util.cpp b/be/src/util/hdfs_util.cpp
index 5ffa09b966..fcfaf2c5fc 100644
--- a/be/src/util/hdfs_util.cpp
+++ b/be/src/util/hdfs_util.cpp
@@ -33,7 +33,7 @@ hdfsFS HDFSHandle::create_hdfs_fs(HDFSCommonBuilder& hdfs_builder) {
     if (hdfs_builder.is_need_kinit()) {
         Status status = hdfs_builder.run_kinit();
         if (!status.ok()) {
-            LOG(WARNING) << status.get_error_msg();
+            LOG(WARNING) << status;
             return nullptr;
         }
     }
diff --git a/be/src/util/load_error_hub.cpp b/be/src/util/load_error_hub.cpp
index 1149656d8a..801401632b 100644
--- a/be/src/util/load_error_hub.cpp
+++ b/be/src/util/load_error_hub.cpp
@@ -19,6 +19,7 @@
 
 #include <thrift/protocol/TDebugProtocol.h>
 
+#include "common/logging.h"
 #include "gen_cpp/PaloInternalService_types.h"
 #include "util/broker_load_error_hub.h"
 #include "util/mysql_load_error_hub.h"
diff --git a/be/src/util/storage_backend_mgr.cpp b/be/src/util/storage_backend_mgr.cpp
index 5179a7f31a..4d940faf05 100644
--- a/be/src/util/storage_backend_mgr.cpp
+++ b/be/src/util/storage_backend_mgr.cpp
@@ -28,6 +28,7 @@
 #include "util/storage_backend.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 Status StorageBackendMgr::init(const std::string& storage_param_dir) {
     if (_is_inited) {
@@ -35,7 +36,7 @@ Status StorageBackendMgr::init(const std::string& storage_param_dir) {
     }
     Status exist_status = Env::Default()->path_exists(storage_param_dir);
     if (!exist_status.ok() &&
-        (!exist_status.is_not_found() || !Env::Default()->create_dirs(storage_param_dir).ok())) {
+        (!exist_status.is<NOT_FOUND>() || !Env::Default()->create_dirs(storage_param_dir).ok())) {
         RETURN_NOT_OK_STATUS_WITH_WARN(
                 Status::IOError("failed to create remote storage_param root path {}",
                                 storage_param_dir),
diff --git a/be/src/util/threadpool.cpp b/be/src/util/threadpool.cpp
index 76bc7e2171..036eb35250 100644
--- a/be/src/util/threadpool.cpp
+++ b/be/src/util/threadpool.cpp
@@ -34,6 +34,7 @@
 #include "util/thread.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 using std::string;
 using strings::Substitute;
@@ -253,7 +254,7 @@ ThreadPool::~ThreadPool() {
 }
 
 Status ThreadPool::init() {
-    if (!_pool_status.is_uninitialized()) {
+    if (!_pool_status.is<UNINITIALIZED>()) {
         return Status::NotSupported("The thread pool {} is already initialized", _name);
     }
     _pool_status = Status::OK();
diff --git a/be/src/util/thrift_rpc_helper.cpp b/be/src/util/thrift_rpc_helper.cpp
index 4241a27fdf..d6889cd258 100644
--- a/be/src/util/thrift_rpc_helper.cpp
+++ b/be/src/util/thrift_rpc_helper.cpp
@@ -51,8 +51,7 @@ Status ThriftRpcHelper::rpc(const std::string& ip, const int32_t port,
     Status status;
     ClientConnection<T> client(_s_exec_env->get_client_cache<T>(), address, timeout_ms, &status);
     if (!status.ok()) {
-        LOG(WARNING) << "Connect frontend failed, address=" << address
-                     << ", status=" << status.get_error_msg();
+        LOG(WARNING) << "Connect frontend failed, address=" << address << ", status=" << status;
         return status;
     }
     try {
@@ -67,7 +66,7 @@ Status ThriftRpcHelper::rpc(const std::string& ip, const int32_t port,
             status = client.reopen(timeout_ms);
             if (!status.ok()) {
                 LOG(WARNING) << "client reopen failed. address=" << address
-                             << ", status=" << status.get_error_msg();
+                             << ", status=" << status;
                 return status;
             }
             callback(client);
diff --git a/be/src/util/zlib.cpp b/be/src/util/zlib.cpp
index c23ca009fb..c4143ca232 100644
--- a/be/src/util/zlib.cpp
+++ b/be/src/util/zlib.cpp
@@ -34,6 +34,7 @@ using std::string;
 #define ZRETURN_NOT_OK(call) RETURN_IF_ERROR(ZlibResultToStatus(call))
 
 namespace doris {
+using namespace ErrorCode;
 namespace zlib {
 
 namespace {
@@ -83,7 +84,7 @@ Status CompressLevel(Slice input, int level, ostream* out) {
         zs.next_out = chunk.get();
         flush = (zs.avail_in == 0) ? Z_FINISH : Z_NO_FLUSH;
         Status s = ZlibResultToStatus(deflate(&zs, flush));
-        if (!s.ok() && !s.is_end_of_file()) {
+        if (!s.ok() && !s.is<END_OF_FILE>()) {
             return s;
         }
         int out_size = zs.next_out - chunk.get();
@@ -109,7 +110,7 @@ Status Uncompress(Slice compressed, std::ostream* out) {
         zs.avail_out = arraysize(buf);
         flush = zs.avail_in > 0 ? Z_NO_FLUSH : Z_FINISH;
         s = ZlibResultToStatus(inflate(&zs, flush));
-        if (!s.ok() && !s.is_end_of_file()) {
+        if (!s.ok() && !s.is<END_OF_FILE>()) {
             return s;
         }
         out->write(reinterpret_cast<char*>(buf), zs.next_out - buf);
diff --git a/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h b/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h
index 113277498c..cde7c844cb 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h
@@ -250,9 +250,8 @@ private:
             func_id = env->GetMethodID(executor_cl, func_name, func_sign);
             Status s = JniUtil::GetJniExceptionMsg(env);
             if (!s.ok()) {
-                return Status::InternalError(
-                        strings::Substitute("Java-Udaf register_func_id meet error and error is $0",
-                                            s.get_error_msg()));
+                return Status::InternalError(strings::Substitute(
+                        "Java-Udaf register_func_id meet error and error is $0", s.to_string()));
             }
             return s;
         };
diff --git a/be/src/vec/exec/format/json/new_json_reader.cpp b/be/src/vec/exec/format/json/new_json_reader.cpp
index 2228f9701d..7e1e4c93c2 100644
--- a/be/src/vec/exec/format/json/new_json_reader.cpp
+++ b/be/src/vec/exec/format/json/new_json_reader.cpp
@@ -26,6 +26,7 @@
 #include "vec/core/block.h"
 #include "vec/exec/scan/vscanner.h"
 namespace doris::vectorized {
+using namespace ErrorCode;
 
 NewJsonReader::NewJsonReader(RuntimeState* state, RuntimeProfile* profile, ScannerCounter* counter,
                              const TFileScanRangeParams& params, const TFileRangeDesc& range,
@@ -355,7 +356,7 @@ Status NewJsonReader::_vhandle_simple_json(std::vector<MutableColumnPtr>& column
         bool valid = false;
         if (_next_row >= _total_rows) { // parse json and generic document
             Status st = _parse_json(is_empty_row, eof);
-            if (st.is_data_quality_error()) {
+            if (st.is<DATA_QUALITY_ERROR>()) {
                 continue; // continue to read next
             }
             RETURN_IF_ERROR(st);
@@ -426,7 +427,7 @@ Status NewJsonReader::_vhandle_flat_array_complex_json(
     do {
         if (_next_row >= _total_rows) {
             Status st = _parse_json(is_empty_row, eof);
-            if (st.is_data_quality_error()) {
+            if (st.is<DATA_QUALITY_ERROR>()) {
                 continue; // continue to read next
             }
             RETURN_IF_ERROR(st);
@@ -456,7 +457,7 @@ Status NewJsonReader::_vhandle_nested_complex_json(std::vector<MutableColumnPtr>
                                                    bool* is_empty_row, bool* eof) {
     while (true) {
         Status st = _parse_json(is_empty_row, eof);
-        if (st.is_data_quality_error()) {
+        if (st.is<DATA_QUALITY_ERROR>()) {
             continue; // continue to read next
         }
         RETURN_IF_ERROR(st);
diff --git a/be/src/vec/exec/scan/new_es_scan_node.cpp b/be/src/vec/exec/scan/new_es_scan_node.cpp
index cf492cc29d..605cece627 100644
--- a/be/src/vec/exec/scan/new_es_scan_node.cpp
+++ b/be/src/vec/exec/scan/new_es_scan_node.cpp
@@ -54,7 +54,7 @@ NewEsScanNode::NewEsScanNode(ObjectPool* pool, const TPlanNode& tnode, const Des
 }
 
 std::string NewEsScanNode::get_name() {
-    return fmt::format("VNewEsScanNode");
+    return "VNewEsScanNode";
 }
 
 Status NewEsScanNode::init(const TPlanNode& tnode, RuntimeState* state) {
@@ -228,10 +228,10 @@ Status NewEsScanNode::build_conjuncts_list() {
         } else {
             _conjunct_to_predicate[i] = -1;
 
-            VLOG_CRITICAL << status.get_error_msg();
+            VLOG_CRITICAL << status;
             status = predicate->get_es_query_status();
             if (!status.ok()) {
-                LOG(WARNING) << status.get_error_msg();
+                LOG(WARNING) << status;
                 return status;
             }
         }
diff --git a/be/src/vec/exec/scan/vfile_scanner.cpp b/be/src/vec/exec/scan/vfile_scanner.cpp
index ee1605a6e0..75107b4d1d 100644
--- a/be/src/vec/exec/scan/vfile_scanner.cpp
+++ b/be/src/vec/exec/scan/vfile_scanner.cpp
@@ -40,6 +40,7 @@
 #include "vec/functions/simple_function_factory.h"
 
 namespace doris::vectorized {
+using namespace ErrorCode;
 
 VFileScanner::VFileScanner(RuntimeState* state, NewFileScanNode* parent, int64_t limit,
                            const TFileScanRange& scan_range, RuntimeProfile* profile)
@@ -527,11 +528,11 @@ Status VFileScanner::_get_next_reader() {
             return Status::InternalError("Not supported file format: {}", _params.format_type);
         }
 
-        if (init_status.is_end_of_file()) {
+        if (init_status.is<END_OF_FILE>()) {
             continue;
         } else if (!init_status.ok()) {
             return Status::InternalError("failed to init reader for file {}, err: {}", range.path,
-                                         init_status.get_error_msg());
+                                         init_status.to_string());
         }
 
         _name_to_col_type.clear();
diff --git a/be/src/vec/exec/varrow_scanner.cpp b/be/src/vec/exec/varrow_scanner.cpp
index 56bc06abf8..2c2a7d6973 100644
--- a/be/src/vec/exec/varrow_scanner.cpp
+++ b/be/src/vec/exec/varrow_scanner.cpp
@@ -26,6 +26,7 @@
 #include "vec/utils/arrow_column_to_doris_column.h"
 
 namespace doris::vectorized {
+using namespace ErrorCode;
 
 VArrowScanner::VArrowScanner(RuntimeState* state, RuntimeProfile* profile,
                              const TBrokerScanRangeParams& params,
@@ -83,12 +84,11 @@ Status VArrowScanner::_open_next_reader() {
         Status status =
                 _cur_file_reader->init_reader(tuple_desc, _conjunct_ctxs, _state->timezone());
 
-        if (status.is_end_of_file()) {
+        if (status.is<END_OF_FILE>()) {
             continue;
         } else {
             if (!status.ok()) {
-                return Status::InternalError(" file: {} error:{}", range.path,
-                                             status.get_error_msg());
+                return Status::InternalError(" file: {} error:{}", range.path, status.to_string());
             } else {
                 update_profile(_cur_file_reader->statistics());
                 return status;
@@ -200,7 +200,7 @@ Status VArrowScanner::get_next(vectorized::Block* block, bool* eof) {
     {
         Status st = _init_arrow_batch_if_necessary();
         if (!st.ok()) {
-            if (!st.is_end_of_file()) {
+            if (!st.is<END_OF_FILE>()) {
                 return st;
             }
             *eof = true;
@@ -224,7 +224,7 @@ Status VArrowScanner::get_next(vectorized::Block* block, bool* eof) {
             continue;
         }
         // return error if not EOF
-        if (!status.is_end_of_file()) {
+        if (!status.is<END_OF_FILE>()) {
             return status;
         }
         _cur_file_eof = true;
diff --git a/be/src/vec/exec/vbroker_scan_node.cpp b/be/src/vec/exec/vbroker_scan_node.cpp
index 68bdca68d0..e79d3ce104 100644
--- a/be/src/vec/exec/vbroker_scan_node.cpp
+++ b/be/src/vec/exec/vbroker_scan_node.cpp
@@ -282,8 +282,7 @@ void VBrokerScanNode::scanner_worker(int start_idx, int length) {
                 _scan_ranges[start_idx + i].scan_range.broker_scan_range;
         status = scanner_scan(scan_range, &counter);
         if (!status.ok()) {
-            LOG(WARNING) << "Scanner[" << start_idx + i
-                         << "] process failed. status=" << status.get_error_msg();
+            LOG(WARNING) << "Scanner[" << start_idx + i << "] process failed. status=" << status;
         }
     }
 
diff --git a/be/src/vec/exec/vjdbc_connector.cpp b/be/src/vec/exec/vjdbc_connector.cpp
index ce171ad569..5a656392a8 100644
--- a/be/src/vec/exec/vjdbc_connector.cpp
+++ b/be/src/vec/exec/vjdbc_connector.cpp
@@ -347,8 +347,7 @@ Status JdbcConnector::_register_func_id(JNIEnv* env) {
         Status s = JniUtil::GetJniExceptionMsg(env);
         if (!s.ok()) {
             return Status::InternalError(strings::Substitute(
-                    "Jdbc connector _register_func_id meet error and error is $0",
-                    s.get_error_msg()));
+                    "Jdbc connector _register_func_id meet error and error is $0", s.to_string()));
         }
         return s;
     };
diff --git a/be/src/vec/exec/vjson_scanner.cpp b/be/src/vec/exec/vjson_scanner.cpp
index 26053181c3..cf5722f739 100644
--- a/be/src/vec/exec/vjson_scanner.cpp
+++ b/be/src/vec/exec/vjson_scanner.cpp
@@ -27,6 +27,7 @@
 #include "vec/data_types/data_type_string.h"
 
 namespace doris::vectorized {
+using namespace ErrorCode;
 
 template <typename JsonReader>
 VJsonScanner<JsonReader>::VJsonScanner(RuntimeState* state, RuntimeProfile* profile,
@@ -157,7 +158,7 @@ Status VJsonReader::_vhandle_simple_json(std::vector<MutableColumnPtr>& columns,
         bool valid = false;
         if (_next_line >= _total_lines) { // parse json and generic document
             Status st = _parse_json(is_empty_row, eof);
-            if (st.is_data_quality_error()) {
+            if (st.is<DATA_QUALITY_ERROR>()) {
                 continue; // continue to read next
             }
             RETURN_IF_ERROR(st);
@@ -380,7 +381,7 @@ Status VJsonReader::_vhandle_flat_array_complex_json(std::vector<MutableColumnPt
     do {
         if (_next_line >= _total_lines) {
             Status st = _parse_json(is_empty_row, eof);
-            if (st.is_data_quality_error()) {
+            if (st.is<DATA_QUALITY_ERROR>()) {
                 continue; // continue to read next
             }
             RETURN_IF_ERROR(st);
@@ -410,7 +411,7 @@ Status VJsonReader::_vhandle_nested_complex_json(std::vector<MutableColumnPtr>&
                                                  bool* is_empty_row, bool* eof) {
     while (true) {
         Status st = _parse_json(is_empty_row, eof);
-        if (st.is_data_quality_error()) {
+        if (st.is<DATA_QUALITY_ERROR>()) {
             continue; // continue to read next
         }
         RETURN_IF_ERROR(st);
@@ -869,7 +870,7 @@ Status VSIMDJsonReader::_vhandle_simple_json(Block& block,
         try {
             if (_next_line >= _total_lines) { // parse json and generic document
                 Status st = _parse_json(is_empty_row, eof);
-                if (st.is_data_quality_error()) {
+                if (st.is<DATA_QUALITY_ERROR>()) {
                     continue; // continue to read next
                 }
                 RETURN_IF_ERROR(st);
@@ -947,7 +948,7 @@ Status VSIMDJsonReader::_vhandle_flat_array_complex_json(
         try {
             if (_next_line >= _total_lines) {
                 Status st = _parse_json(is_empty_row, eof);
-                if (st.is_data_quality_error()) {
+                if (st.is<DATA_QUALITY_ERROR>()) {
                     continue; // continue to read next
                 }
                 RETURN_IF_ERROR(st);
@@ -985,7 +986,7 @@ Status VSIMDJsonReader::_vhandle_nested_complex_json(Block& block,
     while (true) {
         try {
             Status st = _parse_json(is_empty_row, eof);
-            if (st.is_data_quality_error()) {
+            if (st.is<DATA_QUALITY_ERROR>()) {
                 continue; // continue to read next
             }
             RETURN_IF_ERROR(st);
diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp
index b1dbe2015f..12af55f060 100644
--- a/be/src/vec/exprs/vexpr.cpp
+++ b/be/src/vec/exprs/vexpr.cpp
@@ -209,7 +209,7 @@ Status VExpr::create_expr_tree(doris::ObjectPool* pool, const doris::TExpr& texp
     }
     if (!status.ok()) {
         LOG(ERROR) << "Could not construct expr tree.\n"
-                   << status.get_error_msg() << "\n"
+                   << status << "\n"
                    << apache::thrift::ThriftDebugString(texpr);
     }
     return status;
diff --git a/be/src/vec/functions/function_java_udf.h b/be/src/vec/functions/function_java_udf.h
index abb06e4d38..3f88b4226e 100644
--- a/be/src/vec/functions/function_java_udf.h
+++ b/be/src/vec/functions/function_java_udf.h
@@ -115,7 +115,7 @@ private:
             env->CallNonvirtualVoidMethodA(executor, parent->executor_cl_,
                                            parent->executor_close_id_, NULL);
             Status s = JniUtil::GetJniExceptionMsg(env);
-            if (!s.ok()) LOG(WARNING) << s.get_error_msg();
+            if (!s.ok()) LOG(WARNING) << s;
             env->DeleteGlobalRef(executor);
         }
 
diff --git a/be/src/vec/olap/block_reader.cpp b/be/src/vec/olap/block_reader.cpp
index 655e59895b..04cea468e5 100644
--- a/be/src/vec/olap/block_reader.cpp
+++ b/be/src/vec/olap/block_reader.cpp
@@ -25,6 +25,7 @@
 #include "vec/olap/vcollect_iterator.h"
 
 namespace doris::vectorized {
+using namespace ErrorCode;
 
 BlockReader::~BlockReader() {
     for (int i = 0; i < _agg_functions.size(); ++i) {
@@ -53,7 +54,7 @@ Status BlockReader::_init_collect_iter(const ReaderParams& read_params,
     for (auto& rs_reader : rs_readers) {
         RETURN_NOT_OK(rs_reader->init(&_reader_context));
         Status res = _vcollect_iter.add_child(rs_reader);
-        if (!res.ok() && res.precise_code() != OLAP_ERR_DATA_EOF) {
+        if (!res.ok() && !res.is<END_OF_FILE>()) {
             LOG(WARNING) << "failed to add child to iterator, err=" << res;
             return res;
         }
@@ -65,7 +66,7 @@ Status BlockReader::_init_collect_iter(const ReaderParams& read_params,
     RETURN_IF_ERROR(_vcollect_iter.build_heap(*valid_rs_readers));
     if (_vcollect_iter.is_merge()) {
         auto status = _vcollect_iter.current_row(&_next_row);
-        _eof = status.precise_code() == OLAP_ERR_DATA_EOF;
+        _eof = status.is<END_OF_FILE>();
     }
 
     return Status::OK();
@@ -165,14 +166,14 @@ Status BlockReader::init(const ReaderParams& read_params) {
 Status BlockReader::_direct_next_block(Block* block, MemPool* mem_pool, ObjectPool* agg_pool,
                                        bool* eof) {
     auto res = _vcollect_iter.next(block);
-    if (UNLIKELY(!res.ok() && res.precise_code() != OLAP_ERR_DATA_EOF)) {
+    if (UNLIKELY(!res.ok() && !res.is<END_OF_FILE>())) {
         return res;
     }
-    *eof = res.precise_code() == OLAP_ERR_DATA_EOF;
+    *eof = res.is<END_OF_FILE>();
     _eof = *eof;
     if (UNLIKELY(_reader_context.record_rowids)) {
         res = _vcollect_iter.current_block_row_locations(&_block_row_locations);
-        if (UNLIKELY(!res.ok() && res != Status::OLAPInternalError(OLAP_ERR_DATA_EOF))) {
+        if (UNLIKELY(!res.ok() && res != Status::Error<END_OF_FILE>())) {
             return res;
         }
         DCHECK_EQ(_block_row_locations.size(), block->rows());
@@ -202,7 +203,7 @@ Status BlockReader::_agg_key_next_block(Block* block, MemPool* mem_pool, ObjectP
 
     while (true) {
         auto res = _vcollect_iter.next(&_next_row);
-        if (UNLIKELY(res.precise_code() == OLAP_ERR_DATA_EOF)) {
+        if (UNLIKELY(res.is<END_OF_FILE>())) {
             _eof = true;
             *eof = true;
             break;
@@ -260,7 +261,7 @@ Status BlockReader::_unique_key_next_block(Block* block, MemPool* mem_pool, Obje
         // in UNIQUE_KEY highest version is the final result, there is no need to
         // merge the lower versions
         auto res = _vcollect_iter.next(&_next_row);
-        if (UNLIKELY(res.precise_code() == OLAP_ERR_DATA_EOF)) {
+        if (UNLIKELY(res.is<END_OF_FILE>())) {
             _eof = true;
             *eof = true;
             if (UNLIKELY(_reader_context.record_rowids)) {
diff --git a/be/src/vec/olap/block_reader.h b/be/src/vec/olap/block_reader.h
index 356a5a1ab4..b5a17c80e5 100644
--- a/be/src/vec/olap/block_reader.h
+++ b/be/src/vec/olap/block_reader.h
@@ -37,7 +37,7 @@ public:
 
     Status next_row_with_aggregation(RowCursor* row_cursor, MemPool* mem_pool, ObjectPool* agg_pool,
                                      bool* eof) override {
-        return Status::OLAPInternalError(OLAP_ERR_READER_INITIALIZE_ERROR);
+        return Status::Error<ErrorCode::READER_INITIALIZE_ERROR>();
     }
 
     Status next_block_with_aggregation(Block* block, MemPool* mem_pool, ObjectPool* agg_pool,
diff --git a/be/src/vec/olap/vcollect_iterator.cpp b/be/src/vec/olap/vcollect_iterator.cpp
index b9160514ad..fafb653715 100644
--- a/be/src/vec/olap/vcollect_iterator.cpp
+++ b/be/src/vec/olap/vcollect_iterator.cpp
@@ -21,14 +21,16 @@
 #include "util/defer_op.h"
 
 namespace doris {
+using namespace ErrorCode;
+
 namespace vectorized {
 
-#define RETURN_IF_NOT_EOF_AND_OK(stmt)                                                  \
-    do {                                                                                \
-        const Status& _status_ = (stmt);                                                \
-        if (UNLIKELY(!_status_.ok() && _status_.precise_code() != OLAP_ERR_DATA_EOF)) { \
-            return _status_;                                                            \
-        }                                                                               \
+#define RETURN_IF_NOT_EOF_AND_OK(stmt)                                 \
+    do {                                                               \
+        const Status& _status_ = (stmt);                               \
+        if (UNLIKELY(!_status_.ok() && !_status_.is<END_OF_FILE>())) { \
+            return _status_;                                           \
+        }                                                              \
     } while (false)
 
 VCollectIterator::~VCollectIterator() {
@@ -79,7 +81,7 @@ Status VCollectIterator::build_heap(std::vector<RowsetReaderSharedPtr>& rs_reade
                 delete (*c_iter);
                 c_iter = _children.erase(c_iter);
                 r_iter = rs_readers.erase(r_iter);
-                if (s.precise_code() != OLAP_ERR_DATA_EOF) {
+                if (!s.is<END_OF_FILE>()) {
                     return s;
                 }
             } else {
@@ -166,19 +168,19 @@ Status VCollectIterator::current_row(IteratorRowRef* ref) const {
     if (LIKELY(_inner_iter)) {
         *ref = *_inner_iter->current_row_ref();
         if (ref->row_pos == -1) {
-            return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+            return Status::Error<END_OF_FILE>();
         } else {
             return Status::OK();
         }
     }
-    return Status::OLAPInternalError(OLAP_ERR_DATA_ROW_BLOCK_ERROR);
+    return Status::Error<DATA_ROW_BLOCK_ERROR>();
 }
 
 Status VCollectIterator::next(IteratorRowRef* ref) {
     if (LIKELY(_inner_iter)) {
         return _inner_iter->next(ref);
     } else {
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+        return Status::Error<END_OF_FILE>();
     }
 }
 
@@ -186,7 +188,7 @@ Status VCollectIterator::next(Block* block) {
     if (LIKELY(_inner_iter)) {
         return _inner_iter->next(block);
     } else {
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+        return Status::Error<END_OF_FILE>();
     }
 }
 
@@ -223,10 +225,10 @@ Status VCollectIterator::Level0Iterator::_refresh_current_row() {
         } else {
             _reset();
             auto res = _refresh();
-            if (!res.ok() && res.precise_code() != OLAP_ERR_DATA_EOF) {
+            if (!res.ok() && !res.is<END_OF_FILE>()) {
                 return res;
             }
-            if (res.precise_code() == OLAP_ERR_DATA_EOF && _is_empty()) {
+            if (res.is<END_OF_FILE>() && _is_empty()) {
                 break;
             }
 
@@ -237,7 +239,7 @@ Status VCollectIterator::Level0Iterator::_refresh_current_row() {
     } while (!_is_empty());
     _ref.row_pos = -1;
     _current = -1;
-    return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+    return Status::Error<END_OF_FILE>();
 }
 
 Status VCollectIterator::Level0Iterator::next(IteratorRowRef* ref) {
@@ -265,11 +267,11 @@ Status VCollectIterator::Level0Iterator::next(Block* block) {
         return Status::OK();
     } else {
         auto res = _rs_reader->next_block(block);
-        if (!res.ok() && res.precise_code() != OLAP_ERR_DATA_EOF) {
+        if (!res.ok() && !res.is<END_OF_FILE>()) {
             return res;
         }
-        if (res.precise_code() == OLAP_ERR_DATA_EOF && block->rows() == 0) {
-            return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+        if (res.is<END_OF_FILE>() && block->rows() == 0) {
+            return Status::Error<END_OF_FILE>();
         }
         if (UNLIKELY(_reader->_reader_context.record_rowids)) {
             RETURN_NOT_OK(_rs_reader->current_block_row_locations(&_block_row_locations));
@@ -329,13 +331,13 @@ VCollectIterator::Level1Iterator::~Level1Iterator() {
 
 // Read next row into *row.
 // Returns
-//      OLAP_SUCCESS when read successfully.
-//      Status::OLAPInternalError(OLAP_ERR_DATA_EOF) and set *row to nullptr when EOF is reached.
+//      OK when read successfully.
+//      Status::Error<END_OF_FILE>() and set *row to nullptr when EOF is reached.
 //      Others when error happens
 Status VCollectIterator::Level1Iterator::next(IteratorRowRef* ref) {
     if (UNLIKELY(_cur_child == nullptr)) {
         _ref.reset();
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+        return Status::Error<END_OF_FILE>();
     }
     if (_merge) {
         return _merge_next(ref);
@@ -346,12 +348,12 @@ Status VCollectIterator::Level1Iterator::next(IteratorRowRef* ref) {
 
 // Read next block
 // Returns
-//      OLAP_SUCCESS when read successfully.
-//      Status::OLAPInternalError(OLAP_ERR_DATA_EOF) and set *row to nullptr when EOF is reached.
+//      OK when read successfully.
+//      Status::Error<END_OF_FILE>() and set *row to nullptr when EOF is reached.
 //      Others when error happens
 Status VCollectIterator::Level1Iterator::next(Block* block) {
     if (UNLIKELY(_cur_child == nullptr)) {
-        return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+        return Status::Error<END_OF_FILE>();
     }
     if (_merge) {
         return _merge_next(block);
@@ -405,7 +407,7 @@ Status VCollectIterator::Level1Iterator::_merge_next(IteratorRowRef* ref) {
     if (LIKELY(res.ok())) {
         _heap->push(_cur_child);
         _cur_child = _heap->top();
-    } else if (res.precise_code() == OLAP_ERR_DATA_EOF) {
+    } else if (res.is<END_OF_FILE>()) {
         // current child has been read, to read next
         delete _cur_child;
         if (!_heap->empty()) {
@@ -413,7 +415,7 @@ Status VCollectIterator::Level1Iterator::_merge_next(IteratorRowRef* ref) {
         } else {
             _ref.reset();
             _cur_child = nullptr;
-            return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+            return Status::Error<END_OF_FILE>();
         }
     } else {
         _ref.reset();
@@ -441,7 +443,7 @@ Status VCollectIterator::Level1Iterator::_normal_next(IteratorRowRef* ref) {
     if (LIKELY(res.ok())) {
         _ref = *ref;
         return Status::OK();
-    } else if (res.precise_code() == OLAP_ERR_DATA_EOF) {
+    } else if (res.is<END_OF_FILE>()) {
         // current child has been read, to read next
         delete _cur_child;
         _children.pop_front();
@@ -450,7 +452,7 @@ Status VCollectIterator::Level1Iterator::_normal_next(IteratorRowRef* ref) {
             return _normal_next(ref);
         } else {
             _cur_child = nullptr;
-            return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+            return Status::Error<END_OF_FILE>();
         }
     } else {
         _cur_child = nullptr;
@@ -488,7 +490,7 @@ Status VCollectIterator::Level1Iterator::_merge_next(Block* block) {
             pre_row_ref.reset();
         }
         auto res = _merge_next(&cur_row);
-        if (UNLIKELY(res.precise_code() == OLAP_ERR_DATA_EOF)) {
+        if (UNLIKELY(res.is<END_OF_FILE>())) {
             if (UNLIKELY(_reader->_reader_context.record_rowids)) {
                 _block_row_locations.resize(target_block_row);
             }
@@ -533,7 +535,7 @@ Status VCollectIterator::Level1Iterator::_normal_next(Block* block) {
     auto res = _cur_child->next(block);
     if (LIKELY(res.ok())) {
         return Status::OK();
-    } else if (res.precise_code() == OLAP_ERR_DATA_EOF) {
+    } else if (res.is<END_OF_FILE>()) {
         // current child has been read, to read next
         delete _cur_child;
         _children.pop_front();
@@ -542,7 +544,7 @@ Status VCollectIterator::Level1Iterator::_normal_next(Block* block) {
             return _normal_next(block);
         } else {
             _cur_child = nullptr;
-            return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+            return Status::Error<END_OF_FILE>();
         }
     } else {
         _cur_child = nullptr;
@@ -556,7 +558,7 @@ Status VCollectIterator::Level1Iterator::current_block_row_locations(
     if (!_merge) {
         if (UNLIKELY(_cur_child == nullptr)) {
             block_row_locations->clear();
-            return Status::OLAPInternalError(OLAP_ERR_DATA_EOF);
+            return Status::Error<END_OF_FILE>();
         }
         return _cur_child->current_block_row_locations(block_row_locations);
     } else {
diff --git a/be/src/vec/olap/vcollect_iterator.h b/be/src/vec/olap/vcollect_iterator.h
index b16f752bc0..313764d44f 100644
--- a/be/src/vec/olap/vcollect_iterator.h
+++ b/be/src/vec/olap/vcollect_iterator.h
@@ -49,8 +49,8 @@ public:
 
     // Read nest order row in Block.
     // Returns
-    //      OLAP_SUCCESS when read successfully.
-    //      Status::OLAPInternalError(OLAP_ERR_DATA_EOF) and set *row to nullptr when EOF is reached.
+    //      OK when read successfully.
+    //      Status::Error<END_OF_FILE>() and set *row to nullptr when EOF is reached.
     //      Others when error happens
     Status next(IteratorRowRef* ref);
 
diff --git a/be/src/vec/olap/vertical_block_reader.cpp b/be/src/vec/olap/vertical_block_reader.cpp
index a5c02fb176..02dafbefcc 100644
--- a/be/src/vec/olap/vertical_block_reader.cpp
+++ b/be/src/vec/olap/vertical_block_reader.cpp
@@ -27,6 +27,7 @@
 #include "vec/olap/vertical_merge_iterator.h"
 
 namespace doris::vectorized {
+using namespace ErrorCode;
 
 VerticalBlockReader::~VerticalBlockReader() {
     for (int i = 0; i < _agg_functions.size(); ++i) {
@@ -84,7 +85,7 @@ Status VerticalBlockReader::_init_collect_iter(const ReaderParams& read_params)
     // In dup keys value columns compact, get first row for _init_agg_state
     if (!read_params.is_key_column_group && read_params.tablet->keys_type() == KeysType::AGG_KEYS) {
         auto st = _vcollect_iter->next_row(&_next_row);
-        _eof = st.is_end_of_file();
+        _eof = st.is<END_OF_FILE>();
     }
 
     return Status::OK();
@@ -160,10 +161,10 @@ Status VerticalBlockReader::init(const ReaderParams& read_params) {
 Status VerticalBlockReader::_direct_next_block(Block* block, MemPool* mem_pool,
                                                ObjectPool* agg_pool, bool* eof) {
     auto res = _vcollect_iter->next_batch(block);
-    if (UNLIKELY(!res.ok() && !res.is_end_of_file())) {
+    if (UNLIKELY(!res.ok() && !res.is<END_OF_FILE>())) {
         return res;
     }
-    *eof = (res.is_end_of_file());
+    *eof = (res.is<END_OF_FILE>());
     _eof = *eof;
     return Status::OK();
 }
@@ -266,10 +267,10 @@ Status VerticalBlockReader::_agg_key_next_block(Block* block, MemPool* mem_pool,
     if (_reader_context.is_key_column_group) {
         // collect_iter will filter agg keys
         auto res = _vcollect_iter->next_batch(block);
-        if (UNLIKELY(!res.ok() && !res.is_end_of_file())) {
+        if (UNLIKELY(!res.ok() && !res.is<END_OF_FILE>())) {
             return res;
         }
-        *eof = (res.is_end_of_file());
+        *eof = (res.is<END_OF_FILE>());
         _eof = *eof;
         return Status::OK();
     }
@@ -288,7 +289,7 @@ Status VerticalBlockReader::_agg_key_next_block(Block* block, MemPool* mem_pool,
     do {
         Status res = _vcollect_iter->next_row(&_next_row);
         if (UNLIKELY(!res.ok())) {
-            if (UNLIKELY(res.is_end_of_file())) {
+            if (UNLIKELY(res.is<END_OF_FILE>())) {
                 *eof = true;
                 _eof = true;
                 break;
@@ -325,14 +326,14 @@ Status VerticalBlockReader::_unique_key_next_block(Block* block, MemPool* mem_po
         auto row_source_idx = _row_sources_buffer->buffered_size();
 
         auto res = _vcollect_iter->next_batch(block);
-        if (UNLIKELY(!res.ok() && !res.is_end_of_file())) {
+        if (UNLIKELY(!res.ok() && !res.is<END_OF_FILE>())) {
             return res;
         }
         auto block_rows = block->rows();
         if (_filter_delete && block_rows > 0) {
             int ori_delete_sign_idx = _reader_context.tablet_schema->field_index(DELETE_SIGN);
             if (ori_delete_sign_idx < 0) {
-                *eof = (res.is_end_of_file());
+                *eof = (res.is<END_OF_FILE>());
                 _eof = *eof;
                 return Status::OK();
             }
@@ -365,7 +366,7 @@ Status VerticalBlockReader::_unique_key_next_block(Block* block, MemPool* mem_po
             _stats.rows_del_filtered += block_rows - block->rows();
             DCHECK(block->try_get_by_name("__DORIS_COMPACTION_FILTER__") == nullptr);
         }
-        *eof = (res.is_end_of_file());
+        *eof = (res.is<END_OF_FILE>());
         _eof = *eof;
         return Status::OK();
     }
@@ -375,7 +376,7 @@ Status VerticalBlockReader::_unique_key_next_block(Block* block, MemPool* mem_po
     do {
         Status res = _vcollect_iter->unique_key_next_row(&_next_row);
         if (UNLIKELY(!res.ok())) {
-            if (UNLIKELY(res.is_end_of_file())) {
+            if (UNLIKELY(res.is<END_OF_FILE>())) {
                 *eof = true;
                 _eof = true;
                 break;
diff --git a/be/src/vec/olap/vertical_merge_iterator.cpp b/be/src/vec/olap/vertical_merge_iterator.cpp
index 3a5bf46f62..5e683d3bce 100644
--- a/be/src/vec/olap/vertical_merge_iterator.cpp
+++ b/be/src/vec/olap/vertical_merge_iterator.cpp
@@ -18,6 +18,7 @@
 #include "vec/olap/vertical_merge_iterator.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 namespace vectorized {
 
@@ -314,7 +315,7 @@ Status VerticalMergeIteratorContext::_load_next_block() {
         Status st = _iter->next_batch(_block.get());
         if (!st.ok()) {
             _valid = false;
-            if (st.is_end_of_file()) {
+            if (st.is<END_OF_FILE>()) {
                 return Status::OK();
             } else {
                 return st;
@@ -418,7 +419,7 @@ Status VerticalMaskMergeIterator::next_row(vectorized::IteratorRowRef* ref) {
     DCHECK(_row_sources_buf);
     auto st = _row_sources_buf->has_remaining();
     if (!st.ok()) {
-        if (st.is_end_of_file()) {
+        if (st.is<END_OF_FILE>()) {
             for (auto iter : _origin_iter_ctx) {
                 RETURN_IF_ERROR(iter->advance());
                 DCHECK(!iter->valid());
@@ -470,7 +471,7 @@ Status VerticalMaskMergeIterator::unique_key_next_row(vectorized::IteratorRowRef
         }
         st = _row_sources_buf->has_remaining();
     }
-    if (st.is_end_of_file()) {
+    if (st.is<END_OF_FILE>()) {
         for (auto iter : _origin_iter_ctx) {
             RETURN_IF_ERROR(iter->advance());
             DCHECK(!iter->valid());
@@ -498,7 +499,7 @@ Status VerticalMaskMergeIterator::next_batch(Block* block) {
         rows += same_source_cnt;
         st = _row_sources_buf->has_remaining();
     }
-    if (st.is_end_of_file()) {
+    if (st.is<END_OF_FILE>()) {
         for (auto iter : _origin_iter_ctx) {
             RETURN_IF_ERROR(iter->advance());
             DCHECK(!iter->valid());
diff --git a/be/src/vec/olap/vgeneric_iterators.cpp b/be/src/vec/olap/vgeneric_iterators.cpp
index 5939c05d8d..c559fa47d2 100644
--- a/be/src/vec/olap/vgeneric_iterators.cpp
+++ b/be/src/vec/olap/vgeneric_iterators.cpp
@@ -28,6 +28,7 @@
 #include "vec/core/block.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 namespace vectorized {
 VStatisticsIterator::~VStatisticsIterator() {
@@ -292,7 +293,7 @@ Status VMergeIteratorContext::_load_next_block() {
         Status st = _iter->next_batch(_block.get());
         if (!st.ok()) {
             _valid = false;
-            if (st.is_end_of_file()) {
+            if (st.is<END_OF_FILE>()) {
                 return Status::OK();
             } else {
                 return st;
@@ -381,7 +382,7 @@ Status VUnionIterator::init(const StorageReadOptions& opts) {
 Status VUnionIterator::next_batch(Block* block) {
     while (_cur_iter != nullptr) {
         auto st = _cur_iter->next_batch(block);
-        if (st.is_end_of_file()) {
+        if (st.is<END_OF_FILE>()) {
             delete _cur_iter;
             _origin_iters.pop_front();
             if (!_origin_iters.empty()) {
diff --git a/be/src/vec/runtime/vorc_writer.cpp b/be/src/vec/runtime/vorc_writer.cpp
index 40d2560d28..4129a86831 100644
--- a/be/src/vec/runtime/vorc_writer.cpp
+++ b/be/src/vec/runtime/vorc_writer.cpp
@@ -41,7 +41,7 @@ void VOrcOutputStream::close() {
     if (!_is_closed) {
         Status st = _file_writer->close();
         if (!st.ok()) {
-            LOG(WARNING) << "close orc output stream failed: " << st.get_error_msg();
+            LOG(WARNING) << "close orc output stream failed: " << st;
         }
         _is_closed = true;
     }
@@ -52,7 +52,7 @@ void VOrcOutputStream::write(const void* data, size_t length) {
         size_t written_len = 0;
         Status st = _file_writer->write(static_cast<const uint8_t*>(data), length, &written_len);
         if (!st.ok()) {
-            LOG(WARNING) << "Write to ORC file failed: " << st.get_error_msg();
+            LOG(WARNING) << "Write to ORC file failed: " << st;
             return;
         }
         _cur_pos += written_len;
diff --git a/be/src/vec/sink/vdata_stream_sender.cpp b/be/src/vec/sink/vdata_stream_sender.cpp
index 794b1b0323..69f2d4d99b 100644
--- a/be/src/vec/sink/vdata_stream_sender.cpp
+++ b/be/src/vec/sink/vdata_stream_sender.cpp
@@ -219,7 +219,7 @@ Status Channel::close_wait(RuntimeState* state) {
     if (_need_close) {
         Status st = _wait_last_brpc();
         if (!st.ok()) {
-            state->log_error(st.get_error_msg());
+            state->log_error(st.to_string());
         }
         _need_close = false;
         return st;
@@ -248,7 +248,7 @@ Status Channel::close_internal() {
 Status Channel::close(RuntimeState* state) {
     Status st = close_internal();
     if (!st.ok()) {
-        state->log_error(st.get_error_msg());
+        state->log_error(st.to_string());
     }
     return st;
 }
diff --git a/be/src/vec/sink/vtablet_sink.cpp b/be/src/vec/sink/vtablet_sink.cpp
index d46dfe956b..de59e0b5a7 100644
--- a/be/src/vec/sink/vtablet_sink.cpp
+++ b/be/src/vec/sink/vtablet_sink.cpp
@@ -101,7 +101,7 @@ Status VNodeChannel::open_wait() {
                                        -1);
         Status st = _index_channel->check_intolerable_failure();
         if (!st.ok()) {
-            _cancel_with_msg(fmt::format("{}, err: {}", channel_info(), st.get_error_msg()));
+            _cancel_with_msg(fmt::format("{}, err: {}", channel_info(), st.to_string()));
         } else if (is_last_rpc) {
             // if this is last rpc, will must set _add_batches_finished. otherwise, node channel's close_wait
             // will be blocked.
@@ -128,7 +128,7 @@ Status VNodeChannel::open_wait() {
 
             Status st = _index_channel->check_intolerable_failure();
             if (!st.ok()) {
-                _cancel_with_msg(st.get_error_msg());
+                _cancel_with_msg(st.to_string());
             } else if (is_last_rpc) {
                 for (auto& tablet : result.tablet_vec()) {
                     TTabletCommitInfo commit_info;
@@ -160,7 +160,7 @@ Status VNodeChannel::open_wait() {
             }
         } else {
             _cancel_with_msg(fmt::format("{}, add batch req success but status isn't ok, err: {}",
-                                         channel_info(), status.get_error_msg()));
+                                         channel_info(), status.to_string()));
         }
 
         if (result.has_execution_time_us()) {
@@ -282,7 +282,7 @@ void VNodeChannel::try_send_block(RuntimeState* state) {
                                     state->fragement_transmission_compression_type(),
                                     _parent->_transfer_large_data_by_brpc);
         if (!st.ok()) {
-            cancel(fmt::format("{}, err: {}", channel_info(), st.get_error_msg()));
+            cancel(fmt::format("{}, err: {}", channel_info(), st.to_string()));
             _add_block_closure->clear_in_flight();
             return;
         }
@@ -349,7 +349,7 @@ void VNodeChannel::try_send_block(RuntimeState* state) {
                 PTabletWriterAddBlockRequest, ReusableClosure<PTabletWriterAddBlockResult>>(
                 &request, _add_block_closure);
         if (!st.ok()) {
-            cancel(fmt::format("{}, err: {}", channel_info(), st.get_error_msg()));
+            cancel(fmt::format("{}, err: {}", channel_info(), st.to_string()));
             _add_block_closure->clear_in_flight();
             return;
         }
@@ -594,7 +594,7 @@ Status VOlapTableSink::send(RuntimeState* state, vectorized::Block* input_block,
                 auto st = entry.first->add_block(&block, entry.second);
                 if (!st.ok()) {
                     _channels[i]->mark_as_failed(entry.first->node_id(), entry.first->host(),
-                                                 st.get_error_msg());
+                                                 st.to_string());
                 }
             }
         }
diff --git a/be/test/agent/agent_server_test.cpp b/be/test/agent/agent_server_test.cpp
index 6340737b31..40988b2879 100644
--- a/be/test/agent/agent_server_test.cpp
+++ b/be/test/agent/agent_server_test.cpp
@@ -135,7 +135,7 @@ TEST(MakeSnapshotTest, TestMakeSnapshot) {
     TAgentResult return_value2;
     EXPECT_CALL(mock_command_executor, make_snapshot(_, _))
             .Times(1)
-            .WillOnce(Return(Status::OLAPInternalError(OLAP_ERR_WRITE_PROTOBUF_ERROR)));
+            .WillOnce(Return(Status::Error<WRITE_PROTOBUF_ERROR>()));
     agent_server.make_snapshot(return_value2, snapshot_request);
 
     EXPECT_EQ(TStatusCode::RUNTIME_ERROR, return_value2.status.status_code);
@@ -166,7 +166,7 @@ TEST(ReleaseSnapshotTest, TestReleaseSnapshot) {
     TAgentResult return_value2;
     EXPECT_CALL(mock_command_executor, release_snapshot(snapshot_path))
             .Times(1)
-            .WillOnce(Return(Status::OLAPInternalError(OLAP_ERR_WRITE_PROTOBUF_ERROR)));
+            .WillOnce(Return(Status::Error<WRITE_PROTOBUF_ERROR>()));
     agent_server.release_snapshot(return_value2, snapshot_path);
 
     EXPECT_EQ(TStatusCode::RUNTIME_ERROR, return_value2.status.status_code);
diff --git a/be/test/common/config_test.cpp b/be/test/common/config_test.cpp
index e835d8313b..4176fedcfd 100644
--- a/be/test/common/config_test.cpp
+++ b/be/test/common/config_test.cpp
@@ -99,36 +99,37 @@ TEST_F(ConfigTest, UpdateConfigs) {
     // not exist
     Status s = config::set_config("cfg_not_exist", "123");
     EXPECT_FALSE(s.ok());
-    EXPECT_EQ(s.to_string(), "Not found: 'cfg_not_exist' is not found");
+    EXPECT_EQ(s.to_string(), "[NOT_FOUND]'cfg_not_exist' is not found");
 
     // immutable
     EXPECT_TRUE(cfg_bool_immutable);
     s = config::set_config("cfg_bool_immutable", "false");
     EXPECT_FALSE(s.ok());
-    EXPECT_EQ(s.to_string(), "Not supported: 'cfg_bool_immutable' is not support to modify");
+    EXPECT_EQ(s.to_string(),
+              "[NOT_IMPLEMENTED_ERROR]'cfg_bool_immutable' is not support to modify");
     EXPECT_TRUE(cfg_bool_immutable);
 
     // convert error
     s = config::set_config("cfg_bool", "falseeee");
     EXPECT_FALSE(s.ok());
-    EXPECT_EQ(s.to_string(), "Invalid argument: convert 'falseeee' as bool failed");
+    EXPECT_EQ(s.to_string(), "[INVALID_ARGUMENT]convert 'falseeee' as bool failed");
     EXPECT_TRUE(cfg_bool);
 
     s = config::set_config("cfg_double", "");
     EXPECT_FALSE(s.ok());
-    EXPECT_EQ(s.to_string(), "Invalid argument: convert '' as double failed");
+    EXPECT_EQ(s.to_string(), "[INVALID_ARGUMENT]convert '' as double failed");
     EXPECT_EQ(cfg_double, 654.321);
 
     // convert error
     s = config::set_config("cfg_int32_t", "4294967296124");
     EXPECT_FALSE(s.ok());
-    EXPECT_EQ(s.to_string(), "Invalid argument: convert '4294967296124' as int32_t failed");
+    EXPECT_EQ(s.to_string(), "[INVALID_ARGUMENT]convert '4294967296124' as int32_t failed");
     EXPECT_EQ(cfg_int32_t, 65536124);
 
     // not support
     s = config::set_config("cfg_std_string", "test");
     EXPECT_FALSE(s.ok());
-    EXPECT_EQ(s.to_string(), "Not supported: 'cfg_std_string' is not support to modify");
+    EXPECT_EQ(s.to_string(), "[NOT_IMPLEMENTED_ERROR]'cfg_std_string' is not support to modify");
     EXPECT_EQ(cfg_std_string, "doris_config_test_string");
 }
 
diff --git a/be/test/common/status_test.cpp b/be/test/common/status_test.cpp
index dbeb4ad9b3..ee7cb910b1 100644
--- a/be/test/common/status_test.cpp
+++ b/be/test/common/status_test.cpp
@@ -29,8 +29,7 @@ TEST_F(StatusTest, OK) {
     // default
     Status st;
     EXPECT_TRUE(st.ok());
-    EXPECT_EQ("", st.get_error_msg());
-    EXPECT_EQ("OK", st.to_string());
+    EXPECT_EQ("[OK]", st.to_string());
     // copy
     {
         Status other = st;
@@ -50,24 +49,22 @@ TEST_F(StatusTest, Error) {
     // default
     Status st = Status::InternalError("123");
     EXPECT_FALSE(st.ok());
-    EXPECT_EQ("123", st.get_error_msg());
-    EXPECT_EQ("Internal error: 123", st.to_string());
+    EXPECT_EQ("[INTERNAL_ERROR]123", st.to_string());
     // copy
     {
         Status other = st;
         EXPECT_FALSE(other.ok());
-        EXPECT_EQ("123", st.get_error_msg());
+        EXPECT_EQ("[INTERNAL_ERROR]123", other.to_string());
     }
     // move assign
     st = Status::InternalError("456");
     EXPECT_FALSE(st.ok());
-    EXPECT_EQ("456", st.get_error_msg());
+    EXPECT_EQ("[INTERNAL_ERROR]456", st.to_string());
     // move construct
     {
         Status other = std::move(st);
         EXPECT_FALSE(other.ok());
-        EXPECT_EQ("456", other.get_error_msg());
-        EXPECT_EQ("Internal error: 456", other.to_string());
+        EXPECT_EQ("[INTERNAL_ERROR]456", other.to_string());
     }
 }
 
diff --git a/be/test/env/env_posix_test.cpp b/be/test/env/env_posix_test.cpp
index b325e66ad9..fd900cb398 100644
--- a/be/test/env/env_posix_test.cpp
+++ b/be/test/env/env_posix_test.cpp
@@ -24,6 +24,7 @@
 #include "util/file_utils.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 class EnvPosixTest : public testing::Test {
 public:
@@ -111,7 +112,7 @@ TEST_F(EnvPosixTest, random_access) {
 
         // end of file
         st = rfile->read_at(114, &slice4);
-        EXPECT_EQ(TStatusCode::END_OF_FILE, st.code());
+        EXPECT_EQ(END_OF_FILE, st.code());
         LOG(INFO) << "st=" << st.to_string();
     }
 }
@@ -179,7 +180,7 @@ TEST_F(EnvPosixTest, random_rw) {
 
         // end of file
         st = rfile->read_at(102, slice4);
-        EXPECT_EQ(TStatusCode::END_OF_FILE, st.code());
+        EXPECT_EQ(END_OF_FILE, st.code());
         LOG(INFO) << "st=" << st.to_string();
     }
 }
diff --git a/be/test/exec/es_query_builder_test.cpp b/be/test/exec/es_query_builder_test.cpp
index 6b8a172dcb..020c36a145 100644
--- a/be/test/exec/es_query_builder_test.cpp
+++ b/be/test/exec/es_query_builder_test.cpp
@@ -402,7 +402,7 @@ TEST_F(BooleanQueryBuilderTest, validate_esquery) {
     ExtFunction empty_es_query(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols,
                                empty_query_values);
     st = BooleanQueryBuilder::check_es_query(empty_es_query);
-    EXPECT_STREQ(st.get_error_msg().c_str(), "esquery must only one root");
+    EXPECT_STREQ(st.to_string().c_str(), "[INVALID_ARGUMENT]esquery must only one root");
     //LOG(INFO) <<"error msg:" << st1.get_error_msg();
     char malformed_query[] = "{\"bool\": {\"must_not\": {\"exists\": {";
     int malformed_query_length = (int)strlen(malformed_query);
@@ -412,7 +412,7 @@ TEST_F(BooleanQueryBuilderTest, validate_esquery) {
     ExtFunction malformed_es_query(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols,
                                    malformed_query_values);
     st = BooleanQueryBuilder::check_es_query(malformed_es_query);
-    EXPECT_STREQ(st.get_error_msg().c_str(), "malformed esquery json");
+    EXPECT_STREQ(st.to_string().c_str(), "[INVALID_ARGUMENT]malformed esquery json");
     char illegal_query[] = "{\"term\": {\"k1\" : \"2\"},\"match\": {\"k1\": \"3\"}}";
     int illegal_query_length = (int)strlen(illegal_query);
     StringValue illegal_query_value(illegal_query, illegal_query_length);
@@ -421,7 +421,7 @@ TEST_F(BooleanQueryBuilderTest, validate_esquery) {
     ExtFunction illegal_es_query(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols,
                                  illegal_query_values);
     st = BooleanQueryBuilder::check_es_query(illegal_es_query);
-    EXPECT_STREQ(st.get_error_msg().c_str(), "esquery must only one root");
+    EXPECT_STREQ(st.to_string().c_str(), "[INVALID_ARGUMENT]esquery must only one root");
     char illegal_key_query[] = "[\"22\"]";
     int illegal_key_query_length = (int)strlen(illegal_key_query);
     StringValue illegal_key_query_value(illegal_key_query, illegal_key_query_length);
@@ -430,7 +430,7 @@ TEST_F(BooleanQueryBuilderTest, validate_esquery) {
     ExtFunction illegal_key_es_query(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols,
                                      illegal_key_query_values);
     st = BooleanQueryBuilder::check_es_query(illegal_key_es_query);
-    EXPECT_STREQ(st.get_error_msg().c_str(), "esquery must be a object");
+    EXPECT_STREQ(st.to_string().c_str(), "[INVALID_ARGUMENT]esquery must be a object");
 }
 
 TEST_F(BooleanQueryBuilderTest, validate_partial) {
diff --git a/be/test/exec/hdfs_file_reader_test.cpp b/be/test/exec/hdfs_file_reader_test.cpp
index 4c9bdafba4..382c09f14d 100644
--- a/be/test/exec/hdfs_file_reader_test.cpp
+++ b/be/test/exec/hdfs_file_reader_test.cpp
@@ -38,7 +38,7 @@ TEST_F(HdfsFileReaderTest, test_connect_fail) {
     hdfsParams.__set_hdfs_conf(confs);
     HdfsFileReader hdfs_file_reader(hdfsParams, "/user/foo/test.data", 0);
     Status status = hdfs_file_reader.open();
-    EXPECT_EQ(TStatusCode::INTERNAL_ERROR, status.code());
+    EXPECT_EQ(ErrorCode::INTERNAL_ERROR, status.code());
     hdfs_file_reader.close();
 }
 
diff --git a/be/test/exec/s3_reader_test.cpp b/be/test/exec/s3_reader_test.cpp
index d41a78975a..7e00751f52 100644
--- a/be/test/exec/s3_reader_test.cpp
+++ b/be/test/exec/s3_reader_test.cpp
@@ -28,9 +28,11 @@
 #include <string>
 #include <vector>
 
+#include "common/status.h"
 #include "io/s3_writer.h"
 
 namespace doris {
+using namespace ErrorCode;
 static const std::string AK = "";
 static const std::string SK = "";
 static const std::string ENDPOINT = "http://s3.bj.bcebos.com";
@@ -88,13 +90,13 @@ TEST_F(S3ReaderTest, normal) {
     EXPECT_TRUE(st.ok());
     std::unique_ptr<S3Writer> writer1(new S3Writer(_aws_properties, path, 0));
     st = writer1->open();
-    EXPECT_TRUE(st.is_already_exist());
+    EXPECT_TRUE(st.is<ALREADY_EXIST>());
     std::unique_ptr<S3Reader> reader(new S3Reader(_aws_properties, path, 0));
     st = reader->open();
     EXPECT_TRUE(st.ok());
     std::unique_ptr<S3Reader> reader1(new S3Reader(_aws_properties, path + "xx", 0));
     st = reader1->open();
-    EXPECT_TRUE(st.is_not_found());
+    EXPECT_TRUE(st.is<NOT_FOUND>());
     EXPECT_EQ(_content.length(), reader->size());
     std::string verification_contents;
     verification_contents.resize(_content.length());
diff --git a/be/test/exec/tablet_info_test.cpp b/be/test/exec/tablet_info_test.cpp
index 93144476c6..8c86759fab 100644
--- a/be/test/exec/tablet_info_test.cpp
+++ b/be/test/exec/tablet_info_test.cpp
@@ -537,7 +537,7 @@ TEST_F(OlapTablePartitionParamTest, multi_list_partition) {
 
     OlapTablePartitionParam part(schema, t_partition_param);
     st = part.init();
-    LOG(INFO) << st.get_error_msg();
+    LOG(INFO) << st;
     EXPECT_TRUE(st.ok());
     LOG(INFO) << part.debug_string();
 
diff --git a/be/test/http/http_client_test.cpp b/be/test/http/http_client_test.cpp
index e7346158fc..8dceb8d494 100644
--- a/be/test/http/http_client_test.cpp
+++ b/be/test/http/http_client_test.cpp
@@ -169,7 +169,7 @@ TEST_F(HttpClientTest, post_failed) {
     st = client.execute_post_request(request_body, &response);
     EXPECT_FALSE(st.ok());
     std::string not_found = "404";
-    EXPECT_TRUE(boost::algorithm::contains(st.get_error_msg(), not_found));
+    EXPECT_TRUE(boost::algorithm::contains(st.to_string(), not_found));
 }
 
 } // namespace doris
diff --git a/be/test/olap/byte_buffer_test.cpp b/be/test/olap/byte_buffer_test.cpp
index 18c6cbc190..b5699cadc7 100644
--- a/be/test/olap/byte_buffer_test.cpp
+++ b/be/test/olap/byte_buffer_test.cpp
@@ -26,6 +26,7 @@
 #include "olap/file_helper.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 class TestByteBuffer : public testing::Test {
 public:
@@ -53,7 +54,7 @@ TEST_F(TestByteBuffer, TestReadWrite) {
     }
 
     // 参数错误的指定写
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND), buf1->put(in, sizeof(in), 5, 10));
+    EXPECT_EQ(Status::Error<OUT_OF_BOUND>(), buf1->put(in, sizeof(in), 5, 10));
 
     for (int i = 0; i < 50; i++) {
         EXPECT_EQ(Status::OK(), buf1->put(i));
@@ -62,8 +63,8 @@ TEST_F(TestByteBuffer, TestReadWrite) {
     }
 
     // 再写就失败了
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW), buf1->put(0));
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW), buf1->put(in, sizeof(in)));
+    EXPECT_EQ(Status::Error<BUFFER_OVERFLOW>(), buf1->put(0));
+    EXPECT_EQ(Status::Error<BUFFER_OVERFLOW>(), buf1->put(in, sizeof(in)));
 
     // 转为读模式
     buf1->flip();
@@ -78,8 +79,8 @@ TEST_F(TestByteBuffer, TestReadWrite) {
         }
     }
     char buf[50];
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND), buf1->get(buf, 100));
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_BUFFER_OVERFLOW), buf1->get(buf, 10, 50));
+    EXPECT_EQ(Status::Error<OUT_OF_BOUND>(), buf1->get(buf, 100));
+    EXPECT_EQ(Status::Error<BUFFER_OVERFLOW>(), buf1->get(buf, 10, 50));
     EXPECT_EQ(Status::OK(), buf1->get(buf, sizeof(buf)));
     EXPECT_EQ(0u, buf1->remaining());
     EXPECT_EQ(100u, buf1->position());
@@ -88,8 +89,8 @@ TEST_F(TestByteBuffer, TestReadWrite) {
         EXPECT_EQ(i, buf[i]);
     }
     char byte;
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND), buf1->get(&byte));
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_OUT_OF_BOUND), buf1->get(&byte, 1));
+    EXPECT_EQ(Status::Error<OUT_OF_BOUND>(), buf1->get(&byte));
+    EXPECT_EQ(Status::Error<OUT_OF_BOUND>(), buf1->get(&byte, 1));
 
     EXPECT_EQ(Status::OK(), buf1->put(10, 'x'));
     EXPECT_EQ(Status::OK(), buf1->get(10, &byte));
@@ -98,11 +99,11 @@ TEST_F(TestByteBuffer, TestReadWrite) {
     EXPECT_EQ(Status::OK(), buf1->set_limit(11));
     EXPECT_EQ(11u, buf1->limit());
     EXPECT_EQ(11u, buf1->position());
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR), buf1->set_limit(101));
+    EXPECT_EQ(Status::Error<INVALID_ARGUMENT>(), buf1->set_limit(101));
     EXPECT_EQ(Status::OK(), buf1->set_position(10));
     EXPECT_EQ(Status::OK(), buf1->get(&byte));
     EXPECT_EQ('x', byte);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR), buf1->set_position(12));
+    EXPECT_EQ(Status::Error<INVALID_ARGUMENT>(), buf1->set_position(12));
 
     SAFE_DELETE(buf1);
 }
diff --git a/be/test/olap/delete_handler_test.cpp b/be/test/olap/delete_handler_test.cpp
index 2bf6d74714..d47efff4e3 100644
--- a/be/test/olap/delete_handler_test.cpp
+++ b/be/test/olap/delete_handler_test.cpp
@@ -41,6 +41,7 @@ using namespace doris;
 using google::protobuf::RepeatedPtrField;
 
 namespace doris {
+using namespace ErrorCode;
 
 static const uint32_t MAX_PATH_LEN = 1024;
 static StorageEngine* k_engine = nullptr;
@@ -366,7 +367,7 @@ TEST_F(TestDeleteConditionHandler, StoreCondInvalidParameters) {
     Status failed_res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(),
                                                                  conditions, &del_pred);
     ;
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_PARAMETERS), failed_res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_PARAMETERS>(), failed_res);
 }
 
 // 检测过滤条件中指定的列不存在,或者列不符合要求
@@ -383,7 +384,7 @@ TEST_F(TestDeleteConditionHandler, StoreCondNonexistentColumn) {
     Status failed_res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(),
                                                                  conditions, &del_pred);
     ;
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), failed_res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), failed_res);
 
     // 'v'是value列
     conditions.clear();
@@ -396,7 +397,7 @@ TEST_F(TestDeleteConditionHandler, StoreCondNonexistentColumn) {
     failed_res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                           &del_pred);
     ;
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), failed_res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), failed_res);
 
     // value column in duplicate model can be deleted;
     conditions.clear();
@@ -588,7 +589,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_1;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_1);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k1的值越下界,k1类型为int8
     conditions[0].condition_values.clear();
@@ -596,7 +597,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_2;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_2);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k2的值越上界,k2类型为int16
     conditions[0].condition_values.clear();
@@ -605,7 +606,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_3;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_3);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k2的值越下界,k2类型为int16
     conditions[0].condition_values.clear();
@@ -613,7 +614,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_4;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_4);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k3的值越上界,k3类型为int32
     conditions[0].condition_values.clear();
@@ -622,7 +623,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_5;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_5);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k3的值越下界,k3类型为int32
     conditions[0].condition_values.clear();
@@ -630,7 +631,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_6;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_6);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k4的值越上界,k2类型为int64
     conditions[0].condition_values.clear();
@@ -639,7 +640,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_7;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_7);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k4的值越下界,k1类型为int64
     conditions[0].condition_values.clear();
@@ -647,7 +648,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_8;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_8);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k5的值越上界,k5类型为int128
     conditions[0].condition_values.clear();
@@ -656,7 +657,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_9;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_9);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k5的值越下界,k5类型为int128
     conditions[0].condition_values.clear();
@@ -664,7 +665,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_10;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_10);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k9整数部分长度过长,k9类型为decimal, precision=6, frac=3
     conditions[0].condition_values.clear();
@@ -673,7 +674,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_11;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_11);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k9小数部分长度过长,k9类型为decimal, precision=6, frac=3
     conditions[0].condition_values.clear();
@@ -681,7 +682,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_12;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_12);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k9没有小数部分,但包含小数点
     conditions[0].condition_values.clear();
@@ -689,7 +690,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_13;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_13);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k10类型的过滤值不符合对应格式,k10为date
     conditions[0].condition_values.clear();
@@ -698,21 +699,21 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_14;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_14);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     conditions[0].condition_values.clear();
     conditions[0].condition_values.push_back("2013-64-01");
     DeletePredicatePB del_pred_15;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_15);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     conditions[0].condition_values.clear();
     conditions[0].condition_values.push_back("2013-01-40");
     DeletePredicatePB del_pred_16;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_16);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k11类型的过滤值不符合对应格式,k11为datetime
     conditions[0].condition_values.clear();
@@ -721,42 +722,42 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_17;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_17);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     conditions[0].condition_values.clear();
     conditions[0].condition_values.push_back("2013-64-01 00:00:00");
     DeletePredicatePB del_pred_18;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_18);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     conditions[0].condition_values.clear();
     conditions[0].condition_values.push_back("2013-01-40 00:00:00");
     DeletePredicatePB del_pred_19;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_19);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     conditions[0].condition_values.clear();
     conditions[0].condition_values.push_back("2013-01-01 24:00:00");
     DeletePredicatePB del_pred_20;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_20);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     conditions[0].condition_values.clear();
     conditions[0].condition_values.push_back("2013-01-01 00:60:00");
     DeletePredicatePB del_pred_21;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_21);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     conditions[0].condition_values.clear();
     conditions[0].condition_values.push_back("2013-01-01 00:00:60");
     DeletePredicatePB del_pred_22;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_22);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     // 测试k12和k13类型的过滤值过长,k12,k13类型分别为string(64), varchar(64)
     conditions[0].condition_values.clear();
@@ -768,7 +769,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_23;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_23);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 
     conditions[0].condition_values.clear();
     conditions[0].column_name = "k13";
@@ -779,7 +780,7 @@ TEST_F(TestDeleteConditionHandler2, InvalidConditionValue) {
     DeletePredicatePB del_pred_24;
     res = DeleteHandler::generate_delete_predicate(*tablet->tablet_schema(), conditions,
                                                    &del_pred_24);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DELETE_INVALID_CONDITION), res);
+    EXPECT_EQ(Status::Error<DELETE_INVALID_CONDITION>(), res);
 }
 
 class TestDeleteHandler : public testing::Test {
diff --git a/be/test/olap/generic_iterators_test.cpp b/be/test/olap/generic_iterators_test.cpp
index 8de11658aa..9e982f4eef 100644
--- a/be/test/olap/generic_iterators_test.cpp
+++ b/be/test/olap/generic_iterators_test.cpp
@@ -28,6 +28,7 @@
 #include "util/slice.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 class GenericIteratorsTest : public testing::Test {
 public:
@@ -69,7 +70,7 @@ TEST(GenericIteratorsTest, AutoIncrement) {
             row_count++;
         }
     } while (st.ok());
-    EXPECT_TRUE(st.is_end_of_file());
+    EXPECT_TRUE(st.is<END_OF_FILE>());
     EXPECT_EQ(500, row_count);
 
     delete iter;
@@ -108,7 +109,7 @@ TEST(GenericIteratorsTest, Union) {
             row_count++;
         }
     } while (st.ok());
-    EXPECT_TRUE(st.is_end_of_file());
+    EXPECT_TRUE(st.is<END_OF_FILE>());
     EXPECT_EQ(600, row_count);
 
     delete iter;
@@ -150,7 +151,7 @@ TEST(GenericIteratorsTest, MergeAgg) {
             row_count++;
         }
     } while (st.ok());
-    EXPECT_TRUE(st.is_end_of_file());
+    EXPECT_TRUE(st.is<END_OF_FILE>());
     EXPECT_EQ(600, row_count);
 
     delete iter;
@@ -184,7 +185,7 @@ TEST(GenericIteratorsTest, MergeUnique) {
             row_count++;
         }
     } while (st.ok());
-    EXPECT_TRUE(st.is_end_of_file());
+    EXPECT_TRUE(st.is<END_OF_FILE>());
     EXPECT_EQ(300, row_count);
 
     delete iter;
diff --git a/be/test/olap/memtable_flush_executor_test.cpp b/be/test/olap/memtable_flush_executor_test.cpp
index df2e42e03a..f20e4b072c 100644
--- a/be/test/olap/memtable_flush_executor_test.cpp
+++ b/be/test/olap/memtable_flush_executor_test.cpp
@@ -103,13 +103,13 @@ TEST_F(TestMemTableFlushExecutor, create_flush_handler) {
     EXPECT_EQ(1, flush_handler->get_stats().flush_count);
 
     FlushResult res2;
-    res2.flush_status = Status::OLAPInternalError(OLAP_ERR_OTHER_ERROR);
+    res2.flush_status = Status::Error<OTHER_ERROR>();
     flush_handler->on_flush_finished(res2);
     EXPECT_TRUE(flush_handler->is_cancelled());
     EXPECT_EQ(100, flush_handler->get_stats().flush_time_ns);
     EXPECT_EQ(1, flush_handler->get_stats().flush_count);
 
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_OTHER_ERROR), flush_handler->wait());
+    EXPECT_EQ(Status::Error<OTHER_ERROR>(), flush_handler->wait());
 }
 
 } // namespace doris
diff --git a/be/test/olap/olap_meta_test.cpp b/be/test/olap/olap_meta_test.cpp
index c871b367d6..43de818fec 100644
--- a/be/test/olap/olap_meta_test.cpp
+++ b/be/test/olap/olap_meta_test.cpp
@@ -33,6 +33,7 @@
 using std::string;
 
 namespace doris {
+using namespace ErrorCode;
 
 class OlapMetaTest : public testing::Test {
 public:
@@ -75,7 +76,7 @@ TEST_F(OlapMetaTest, TestPutAndGet) {
 
     // abnormal cases
     s = _meta->get(META_COLUMN_FAMILY_INDEX, "key_not_exist", &value_get);
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_META_KEY_NOT_FOUND), s);
+    EXPECT_EQ(Status::Error<META_KEY_NOT_FOUND>(), s);
 }
 
 TEST_F(OlapMetaTest, TestRemove) {
diff --git a/be/test/olap/ordered_data_compaction_test.cpp b/be/test/olap/ordered_data_compaction_test.cpp
index 06b9ecf2ad..ad6813c657 100644
--- a/be/test/olap/ordered_data_compaction_test.cpp
+++ b/be/test/olap/ordered_data_compaction_test.cpp
@@ -37,6 +37,7 @@
 #include "vec/olap/vertical_merge_iterator.h"
 
 namespace doris {
+using namespace ErrorCode;
 namespace vectorized {
 
 static const uint32_t MAX_PATH_LEN = 1024;
@@ -459,7 +460,7 @@ TEST_F(OrderedDataCompactionTest, test_01) {
             output_data.emplace_back(columns[0].column->get_int(i), columns[1].column->get_int(i));
         }
     } while (s == Status::OK());
-    EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DATA_EOF), s);
+    EXPECT_EQ(Status::Error<END_OF_FILE>(), s);
     EXPECT_EQ(out_rowset->rowset_meta()->num_rows(), output_data.size());
     EXPECT_EQ(output_data.size(), num_input_rowset * num_segments * rows_per_segment);
     std::vector<uint32_t> segment_num_rows;
diff --git a/be/test/olap/primary_key_index_test.cpp b/be/test/olap/primary_key_index_test.cpp
index 84da33c4b7..aca1dc575e 100644
--- a/be/test/olap/primary_key_index_test.cpp
+++ b/be/test/olap/primary_key_index_test.cpp
@@ -28,6 +28,7 @@
 #include "util/key_util.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 class PrimaryKeyIndexTest : public testing::Test {
 public:
@@ -127,7 +128,7 @@ TEST_F(PrimaryKeyIndexTest, builder) {
         EXPECT_FALSE(exists);
         auto status = index_iterator->seek_at_or_after(&slice, &exact_match);
         EXPECT_FALSE(exact_match);
-        EXPECT_TRUE(status.is_not_found());
+        EXPECT_TRUE(status.is<NOT_FOUND>());
     }
 
     // read all key
diff --git a/be/test/olap/rowid_conversion_test.cpp b/be/test/olap/rowid_conversion_test.cpp
index a08f17f4c4..66d61044aa 100644
--- a/be/test/olap/rowid_conversion_test.cpp
+++ b/be/test/olap/rowid_conversion_test.cpp
@@ -37,6 +37,7 @@
 #include "util/file_utils.h"
 
 namespace doris {
+using namespace ErrorCode;
 
 static const uint32_t MAX_PATH_LEN = 1024;
 static StorageEngine* k_engine = nullptr;
@@ -405,7 +406,7 @@ protected:
                                          columns[1].column->get_int(i));
             }
         } while (s == Status::OK());
-        EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DATA_EOF), s);
+        EXPECT_EQ(Status::Error<END_OF_FILE>(), s);
         EXPECT_EQ(out_rowset->rowset_meta()->num_rows(), output_data.size());
         std::vector<uint32_t> segment_num_rows;
         EXPECT_TRUE(output_rs_reader->get_segment_num_rows(&segment_num_rows).ok());
diff --git a/be/test/olap/rowset/beta_rowset_test.cpp b/be/test/olap/rowset/beta_rowset_test.cpp
index ea8a3485fa..c880564542 100644
--- a/be/test/olap/rowset/beta_rowset_test.cpp
+++ b/be/test/olap/rowset/beta_rowset_test.cpp
@@ -45,6 +45,7 @@
 using std::string;
 
 namespace doris {
+using namespace ErrorCode;
 
 static const uint32_t MAX_PATH_LEN = 1024;
 StorageEngine* k_engine = nullptr;
@@ -251,7 +252,7 @@ TEST_F(BetaRowsetTest, BasicFunctionTest) {
                     num_rows_read++;
                 }
             }
-            EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DATA_EOF), s);
+            EXPECT_EQ(Status::Error<END_OF_FILE>(), s);
             EXPECT_TRUE(output_block == nullptr);
             EXPECT_EQ(rowset->rowset_meta()->num_rows(), num_rows_read);
             EXPECT_TRUE(rowset_reader->get_segment_num_rows(&segment_num_rows).ok());
@@ -293,7 +294,7 @@ TEST_F(BetaRowsetTest, BasicFunctionTest) {
                     num_rows_read++;
                 }
             }
-            EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DATA_EOF), s);
+            EXPECT_EQ(Status::Error<END_OF_FILE>(), s);
             EXPECT_TRUE(output_block == nullptr);
             EXPECT_EQ(1, num_rows_read);
             EXPECT_TRUE(rowset_reader->get_segment_num_rows(&segment_num_rows).ok());
@@ -335,7 +336,7 @@ TEST_F(BetaRowsetTest, BasicFunctionTest) {
                     num_rows_read++;
                 }
             }
-            EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DATA_EOF), s);
+            EXPECT_EQ(Status::Error<END_OF_FILE>(), s);
             EXPECT_TRUE(output_block == nullptr);
             EXPECT_EQ(rowset->rowset_meta()->num_rows(), num_rows_read);
             EXPECT_TRUE(rowset_reader->get_segment_num_rows(&segment_num_rows).ok());
@@ -374,7 +375,7 @@ TEST_F(BetaRowsetTest, BasicFunctionTest) {
                     num_rows_read++;
                 }
             }
-            EXPECT_EQ(Status::OLAPInternalError(OLAP_ERR_DATA_EOF), s);
+            EXPECT_EQ(Status::Error<END_OF_FILE>(), s);
             EXPECT_TRUE(output_block == nullptr);
             EXPECT_EQ(100, num_rows_read);
             delete predicate;
diff --git a/be/test/olap/rowset/segment_v2/binary_prefix_page_test.cpp b/be/test/olap/rowset/segment_v2/binary_prefix_page_test.cpp
index e84106d936..66ad144cb1 100644
--- a/be/test/olap/rowset/segment_v2/binary_prefix_page_test.cpp
+++ b/be/test/olap/rowset/segment_v2/binary_prefix_page_test.cpp
@@ -31,6 +31,7 @@
 #include "util/debug_util.h"
 
 namespace doris {
+using namespace ErrorCode;
 namespace segment_v2 {
 
 class BinaryPrefixPageTest : public testing::Test {
@@ -107,7 +108,7 @@ public:
         Slice v1 = Slice("1039");
         bool exact_match;
         ret = page_decoder->seek_at_or_after_value(&v1, &exact_match);
-        EXPECT_TRUE(ret.is_not_found());
+        EXPECT_TRUE(ret.is<NOT_FOUND>());
 
         Slice v2 = Slice("1000");
         ret = page_decoder->seek_at_or_after_value(&v2, &exact_match);
diff --git a/be/test/olap/rowset/segment_v2/bitshuffle_page_test.cpp b/be/test/olap/rowset/segment_v2/bitshuffle_page_test.cpp
index e7b3d4adc8..1158e6828f 100644
--- a/be/test/olap/rowset/segment_v2/bitshuffle_page_test.cpp
+++ b/be/test/olap/rowset/segment_v2/bitshuffle_page_test.cpp
@@ -31,6 +31,7 @@ using doris::segment_v2::PageBuilderOptions;
 using doris::OlapReaderStatistics;
 
 namespace doris {
+using namespace ErrorCode;
 
 class BitShufflePageTest : public testing::Test {
 public:
@@ -165,7 +166,7 @@ public:
         EXPECT_FALSE(exact_match);
 
         status = page_decoder.seek_at_or_after_value(bigger_than_biggest, &exact_match);
-        EXPECT_EQ(status.code(), TStatusCode::NOT_FOUND);
+        EXPECT_EQ(status.code(), NOT_FOUND);
     }
 };
 
diff --git a/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp b/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp
index 9178e64a3c..a0dd6c7ee5 100644
--- a/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp
+++ b/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp
@@ -86,7 +86,7 @@ void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows,
     {
         io::FileWriterPtr file_writer;
         Status st = fs->create_file(fname, &file_writer);
-        EXPECT_TRUE(st.ok()) << st.get_error_msg();
+        EXPECT_TRUE(st.ok()) << st;
 
         ColumnWriterOptions writer_opts;
         writer_opts.meta = &meta;
@@ -259,7 +259,7 @@ void test_array_nullable_data(CollectionValue* src_data, uint8_t* src_is_null, i
     {
         io::FileWriterPtr file_writer;
         Status st = fs->create_file(fname, &file_writer);
-        EXPECT_TRUE(st.ok()) << st.get_error_msg();
+        EXPECT_TRUE(st.ok()) << st;
 
         ColumnWriterOptions writer_opts;
         writer_opts.meta = &meta;
diff --git a/be/test/olap/rowset/segment_v2/plain_page_test.cpp b/be/test/olap/rowset/segment_v2/plain_page_test.cpp
index 4cd3e47ba9..9b0ae895ed 100644
--- a/be/test/olap/rowset/segment_v2/plain_page_test.cpp
+++ b/be/test/olap/rowset/segment_v2/plain_page_test.cpp
@@ -29,6 +29,7 @@
 #include "runtime/mem_pool.h"
 
 namespace doris {
+using namespace ErrorCode;
 namespace segment_v2 {
 
 class PlainPageTest : public testing::Test {
@@ -160,7 +161,7 @@ public:
 
         if (bigger_than_biggest != nullptr) {
             status = page_decoder.seek_at_or_after_value(bigger_than_biggest, &exact_match);
-            EXPECT_EQ(status.code(), TStatusCode::NOT_FOUND);
+            EXPECT_EQ(status.code(), NOT_FOUND);
         }
     }
 };
diff --git a/be/test/olap/rowset/segment_v2/segment_test.cpp b/be/test/olap/rowset/segment_v2/segment_test.cpp
index 5f9ff06d2b..9ce5b31415 100644
--- a/be/test/olap/rowset/segment_v2/segment_test.cpp
+++ b/be/test/olap/rowset/segment_v2/segment_test.cpp
@@ -44,6 +44,7 @@
 #include "util/key_util.h"
 
 namespace doris {
+using namespace ErrorCode;
 namespace segment_v2 {
 
 using std::string;
@@ -348,7 +349,7 @@ TEST_F(SegmentReaderWriterTest, normal) {
                     ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok());
 
                     RowBlockV2 block(schema, 100);
-                    EXPECT_TRUE(iter->next_batch(&block).is_end_of_file());
+                    EXPECT_TRUE(iter->next_batch(&block).is<END_OF_FILE>());
                     EXPECT_EQ(0, block.num_rows());
                 }
                 // test seek, key (-2, -1)
@@ -379,7 +380,7 @@ TEST_F(SegmentReaderWriterTest, normal) {
                     ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok());
 
                     RowBlockV2 block(schema, 100);
-                    EXPECT_TRUE(iter->next_batch(&block).is_end_of_file());
+                    EXPECT_TRUE(iter->next_batch(&block).is<END_OF_FILE>());
                     EXPECT_EQ(0, block.num_rows());
                 }
             }
@@ -831,7 +832,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) {
 
             RowBlockV2 block(schema, 100);
             st = iter->next_batch(&block);
-            EXPECT_TRUE(st.is_end_of_file());
+            EXPECT_TRUE(st.is<END_OF_FILE>());
             EXPECT_EQ(0, block.num_rows());
         }
 
@@ -865,7 +866,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) {
 
             RowBlockV2 block(schema, 100);
             st = iter->next_batch(&block);
-            EXPECT_TRUE(st.is_end_of_file());
+            EXPECT_TRUE(st.is<END_OF_FILE>());
             EXPECT_EQ(0, block.num_rows());
         }
     }
@@ -1076,7 +1077,7 @@ TEST_F(SegmentReaderWriterTest, TestLookupRowKey) {
             encode_key<RowCursor, true, true>(&encoded_key, row, tablet_schema->num_key_columns());
             RowLocation row_location;
             Status st = segment->lookup_row_key(encoded_key, &row_location);
-            EXPECT_EQ(st.is_not_found(), true);
+            EXPECT_EQ(st.is<NOT_FOUND>(), true);
         }
     }
 }
@@ -1139,7 +1140,7 @@ TEST_F(SegmentReaderWriterTest, TestLookupRowKeyWithSequenceCol) {
 
             RowLocation row_location;
             Status st = segment->lookup_row_key(encoded_key, &row_location);
-            EXPECT_EQ(st.is_not_found(), true);
+            EXPECT_EQ(st.is<NOT_FOUND>(), true);
         }
     }
 
@@ -1167,7 +1168,7 @@ TEST_F(SegmentReaderWriterTest, TestLookupRowKeyWithSequenceCol) {
 
         RowLocation row_location;
         Status st = segment->lookup_row_key(encoded_key, &row_location);
-        EXPECT_EQ(st.is_already_exist(), true);
+        EXPECT_EQ(st.is<ALREADY_EXIST>(), true);
     }
 }
 
diff --git a/be/test/olap/segcompaction_test.cpp b/be/test/olap/segcompaction_test.cpp
... 667 lines suppressed ...


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