You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by zh...@apache.org on 2020/04/08 03:17:57 UTC

[incubator-doris] branch master updated: [config] Support to modify configs when BE is running without restarting (#3264)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 8fc284d  [config] Support to modify configs when BE is running without restarting (#3264)
8fc284d is described below

commit 8fc284d5931d82558b03f90656b61c5b75bffa52
Author: Yingchun Lai <40...@qq.com>
AuthorDate: Tue Apr 7 22:17:47 2020 -0500

    [config] Support to modify configs when BE is running without restarting (#3264)
    
    In the past, when we want to modify some BE configs, we have to modify be.conf and then restart BE.
    This patch provides a way to modify configs in the type of 'threshold', 'interval', 'enable flag'
    when BE is running without restarting it.
    You can update a single config once by BE's http API: `be_host:be_http_port/api/update_config?config_name=new_value`
---
 be/src/common/config.h                             | 172 +++++++-------
 be/src/common/configbase.cpp                       | 261 +++++++++++++--------
 be/src/common/configbase.h                         | 101 ++++----
 be/src/common/logging.h                            |   1 -
 be/src/http/CMakeLists.txt                         |   1 +
 be/src/http/action/update_config_action.cpp        |  76 ++++++
 .../action/update_config_action.h}                 |  38 +--
 be/src/http/default_path_handlers.cpp              |   2 +-
 be/src/http/http_client.cpp                        |   1 +
 be/src/olap/olap_meta.cpp                          |  11 +-
 be/src/olap/olap_server.cpp                        |  96 ++++----
 be/src/plugin/plugin_mgr.cpp                       |   3 +-
 be/src/runtime/external_scan_context_mgr.cpp       |   4 +-
 be/src/runtime/external_scan_context_mgr.h         |   1 -
 be/src/runtime/load_channel_mgr.cpp                |   2 +-
 be/src/runtime/mem_pool.h                          |   1 +
 be/src/runtime/thread_resource_mgr.cpp             |   1 +
 be/src/service/backend_options.cpp                 |   1 +
 be/src/service/http_service.cpp                    |   4 +
 be/src/util/metrics.h                              |   1 +
 be/src/util/runtime_profile.cpp                    |   1 +
 be/src/util/thrift_rpc_helper.h                    |   1 +
 be/test/common/config_test.cpp                     |  77 +++++-
 be/test/common/resource_tls_test.cpp               |   1 +
 be/test/exprs/hybird_set_test.cpp                  |   1 +
 be/test/olap/bloom_filter_test.cpp                 |   1 +
 be/test/olap/byte_buffer_test.cpp                  |   1 +
 be/test/olap/file_helper_test.cpp                  |   1 +
 be/test/olap/file_utils_test.cpp                   |   1 +
 be/test/runtime/large_int_value_test.cpp           |   1 +
 be/test/util/cidr_test.cpp                         |   1 +
 be/test/util/filesystem_util_test.cpp              |   1 +
 be/test/util/internal_queue_test.cpp               |   1 +
 33 files changed, 536 insertions(+), 331 deletions(-)

diff --git a/be/src/common/config.h b/be/src/common/config.h
index ad7513d..9bbe0a9 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -40,9 +40,9 @@ namespace config {
     //// tcmalloc gc parameter
     ////
     // min memory for TCmalloc, when used memory is smaller than this, do not returned to OS
-    CONF_Int64(tc_use_memory_min, "10737418240");
+    CONF_mInt64(tc_use_memory_min, "10737418240");
     // free memory rate.[0-100]
-    CONF_Int64(tc_free_memory_rate, "20");
+    CONF_mInt64(tc_free_memory_rate, "20");
 
     // process memory limit specified as number of bytes
     // ('<int>[bB]?'), megabytes ('<float>[mM]'), gigabytes ('<float>[gG]'),
@@ -87,29 +87,29 @@ namespace config {
     // the count of thread to release snapshot
     CONF_Int32(release_snapshot_worker_count, "5");
     // the interval time(seconds) for agent report tasks signatrue to FE
-    CONF_Int32(report_task_interval_seconds, "10");
+    CONF_mInt32(report_task_interval_seconds, "10");
     // the interval time(seconds) for agent report disk state to FE
-    CONF_Int32(report_disk_state_interval_seconds, "60");
+    CONF_mInt32(report_disk_state_interval_seconds, "60");
     // the interval time(seconds) for agent report olap table to FE
-    CONF_Int32(report_tablet_interval_seconds, "60");
+    CONF_mInt32(report_tablet_interval_seconds, "60");
     // the interval time(seconds) for agent report plugin status to FE
     // CONF_Int32(report_plugin_interval_seconds, "120");
     // the timeout(seconds) for alter table
-    CONF_Int32(alter_tablet_timeout_seconds, "86400");
+    // CONF_Int32(alter_tablet_timeout_seconds, "86400");
     // the timeout(seconds) for make snapshot
-    CONF_Int32(make_snapshot_timeout_seconds, "600");
+    // CONF_Int32(make_snapshot_timeout_seconds, "600");
     // the timeout(seconds) for release snapshot
-    CONF_Int32(release_snapshot_timeout_seconds, "600");
+    // CONF_Int32(release_snapshot_timeout_seconds, "600");
     // the max download speed(KB/s)
-    CONF_Int32(max_download_speed_kbps, "50000");
+    CONF_mInt32(max_download_speed_kbps, "50000");
     // download low speed limit(KB/s)
-    CONF_Int32(download_low_speed_limit_kbps, "50");
+    CONF_mInt32(download_low_speed_limit_kbps, "50");
     // download low speed time(seconds)
-    CONF_Int32(download_low_speed_time, "300");
+    CONF_mInt32(download_low_speed_time, "300");
     // curl verbose mode
-    CONF_Int64(curl_verbose_mode, "1");
+    // CONF_Int64(curl_verbose_mode, "1");
     // seconds to sleep for each time check table status
-    CONF_Int32(check_status_sleep_time_seconds, "10");
+    // CONF_Int32(check_status_sleep_time_seconds, "10");
     // sleep time for one second
     CONF_Int32(sleep_one_second, "1");
     // sleep time for five seconds
@@ -142,7 +142,7 @@ namespace config {
     CONF_String(default_query_options, "");
 
     // If non-zero, Doris will output memory usage every log_mem_usage_interval'th fragment completion.
-    CONF_Int32(log_mem_usage_interval, "0");
+    // CONF_Int32(log_mem_usage_interval, "0");
 
     // cgroups allocated for doris
     CONF_String(doris_cgroups, "");
@@ -156,7 +156,7 @@ namespace config {
     // serialize and deserialize each returned row batch
     CONF_Bool(serialize_batch, "false");
     // interval between profile reports; in seconds
-    CONF_Int32(status_report_interval, "5");
+    CONF_mInt32(status_report_interval, "5");
     // Local directory to copy UDF libraries from HDFS into
     CONF_String(local_library_dir, "${UDF_RUNTIME_DIR}");
     // number of olap scanner thread pool size
@@ -172,63 +172,63 @@ namespace config {
     // default thrift client connect timeout(in seconds)
     CONF_Int32(thrift_connect_timeout_seconds, "3");
     // max row count number for single scan range
-    CONF_Int32(doris_scan_range_row_count, "524288");
+    CONF_mInt32(doris_scan_range_row_count, "524288");
     // size of scanner queue between scanner thread and compute thread
-    CONF_Int32(doris_scanner_queue_size, "1024");
+    CONF_mInt32(doris_scanner_queue_size, "1024");
     // single read execute fragment row size
-    CONF_Int32(doris_scanner_row_num, "16384");
+    CONF_mInt32(doris_scanner_row_num, "16384");
     // number of max scan keys
-    CONF_Int32(doris_max_scan_key_num, "1024");
+    CONF_mInt32(doris_max_scan_key_num, "1024");
     // return_row / total_row
-    CONF_Int32(doris_max_pushdown_conjuncts_return_rate, "90");
+    CONF_mInt32(doris_max_pushdown_conjuncts_return_rate, "90");
     // (Advanced) Maximum size of per-query receive-side buffer
-    CONF_Int32(exchg_node_buffer_size_bytes, "10485760");
+    CONF_mInt32(exchg_node_buffer_size_bytes, "10485760");
     // insert sort threadhold for sorter
-    CONF_Int32(insertion_threadhold, "16");
+    // CONF_Int32(insertion_threadhold, "16");
     // the block_size every block allocate for sorter
     CONF_Int32(sorter_block_size, "8388608");
     // push_write_mbytes_per_sec
     CONF_Int32(push_write_mbytes_per_sec, "10");
 
-    CONF_Int64(column_dictionary_key_ratio_threshold, "0");
-    CONF_Int64(column_dictionary_key_size_threshold, "0");
+    CONF_mInt64(column_dictionary_key_ratio_threshold, "0");
+    CONF_mInt64(column_dictionary_key_size_threshold, "0");
     // if true, output IR after optimization passes
-    CONF_Bool(dump_ir, "false");
+    // CONF_Bool(dump_ir, "false");
     // if set, saves the generated IR to the output file.
-    CONF_String(module_output, "");
+    //CONF_String(module_output, "");
     // memory_limitation_per_thread_for_schema_change unit GB
-    CONF_Int32(memory_limitation_per_thread_for_schema_change, "2");
+    CONF_mInt32(memory_limitation_per_thread_for_schema_change, "2");
 
-    CONF_Int64(max_unpacked_row_block_size, "104857600");
+    // CONF_Int64(max_unpacked_row_block_size, "104857600");
 
-    CONF_Int32(file_descriptor_cache_clean_interval, "3600");
-    CONF_Int32(disk_stat_monitor_interval, "5");
-    CONF_Int32(unused_rowset_monitor_interval, "30");
+    CONF_mInt32(file_descriptor_cache_clean_interval, "3600");
+    CONF_mInt32(disk_stat_monitor_interval, "5");
+    CONF_mInt32(unused_rowset_monitor_interval, "30");
     CONF_String(storage_root_path, "${DORIS_HOME}/storage");
     // BE process will exit if the percentage of error disk reach this value.
-    CONF_Int32(max_percentage_of_error_disk, "0");
-    CONF_Int32(default_num_rows_per_data_block, "1024");
-    CONF_Int32(default_num_rows_per_column_file_block, "1024");
+    CONF_mInt32(max_percentage_of_error_disk, "0");
+    // CONF_Int32(default_num_rows_per_data_block, "1024");
+    CONF_mInt32(default_num_rows_per_column_file_block, "1024");
     CONF_Int32(max_tablet_num_per_shard, "1024");
     // pending data policy
-    CONF_Int32(pending_data_expire_time_sec, "1800");
+    CONF_mInt32(pending_data_expire_time_sec, "1800");
     // inc_rowset expired interval
-    CONF_Int32(inc_rowset_expired_sec, "1800");
+    CONF_mInt32(inc_rowset_expired_sec, "1800");
     // garbage sweep policy
     CONF_Int32(max_garbage_sweep_interval, "3600");
     CONF_Int32(min_garbage_sweep_interval, "180");
-    CONF_Int32(snapshot_expire_time_sec, "172800");
+    CONF_mInt32(snapshot_expire_time_sec, "172800");
     // 仅仅是建议值,当磁盘空间不足时,trash下的文件保存期可不遵守这个参数
-    CONF_Int32(trash_file_expire_time_sec, "259200");
+    CONF_mInt32(trash_file_expire_time_sec, "259200");
     // check row nums for BE/CE and schema change. true is open, false is closed.
-    CONF_Bool(row_nums_check, "true")
+    CONF_mBool(row_nums_check, "true")
     //file descriptors cache, by default, cache 32768 descriptors
     CONF_Int32(file_descriptor_cache_capacity, "32768");
     // minimum file descriptor number
     // modify them upon necessity
     CONF_Int32(min_file_descriptor_number, "60000");
     CONF_Int64(index_stream_cache_capacity, "10737418240");
-    CONF_Int64(max_packed_row_block_size, "20971520");
+    // CONF_Int64(max_packed_row_block_size, "20971520");
 
     // Cache for stoage page size
     CONF_String(storage_page_cache_limit, "20G");
@@ -236,26 +236,26 @@ namespace config {
     CONF_Bool(disable_storage_page_cache, "false");
 
     // be policy
-    CONF_Int64(base_compaction_start_hour, "20");
-    CONF_Int64(base_compaction_end_hour, "7");
-    CONF_Int32(base_compaction_check_interval_seconds, "60");
-    CONF_Int64(base_compaction_num_cumulative_deltas, "5");
+    // CONF_Int64(base_compaction_start_hour, "20");
+    // CONF_Int64(base_compaction_end_hour, "7");
+    CONF_mInt32(base_compaction_check_interval_seconds, "60");
+    CONF_mInt64(base_compaction_num_cumulative_deltas, "5");
     CONF_Int32(base_compaction_num_threads_per_disk, "1");
-    CONF_Double(base_cumulative_delta_ratio, "0.3");
-    CONF_Int64(base_compaction_interval_seconds_since_last_operation, "86400");
-    CONF_Int32(base_compaction_write_mbytes_per_sec, "5");
+    CONF_mDouble(base_cumulative_delta_ratio, "0.3");
+    CONF_mInt64(base_compaction_interval_seconds_since_last_operation, "86400");
+    CONF_mInt32(base_compaction_write_mbytes_per_sec, "5");
 
     // cumulative compaction policy: max delta file's size unit:B
-    CONF_Int32(cumulative_compaction_check_interval_seconds, "10");
-    CONF_Int64(min_cumulative_compaction_num_singleton_deltas, "5");
-    CONF_Int64(max_cumulative_compaction_num_singleton_deltas, "1000");
+    CONF_mInt32(cumulative_compaction_check_interval_seconds, "10");
+    CONF_mInt64(min_cumulative_compaction_num_singleton_deltas, "5");
+    CONF_mInt64(max_cumulative_compaction_num_singleton_deltas, "1000");
     CONF_Int32(cumulative_compaction_num_threads_per_disk, "1");
-    CONF_Int64(cumulative_compaction_budgeted_bytes, "104857600");
-    CONF_Int32(cumulative_compaction_write_mbytes_per_sec, "100");
+    CONF_mInt64(cumulative_compaction_budgeted_bytes, "104857600");
+    // CONF_Int32(cumulative_compaction_write_mbytes_per_sec, "100");
 
     // if compaction of a tablet failed, this tablet should not be chosen to
     // compaction until this interval passes.
-    CONF_Int64(min_compaction_failure_interval_sec, "600") // 10 min
+    CONF_mInt64(min_compaction_failure_interval_sec, "600") // 10 min
     // Too many compaction tasks may run out of memory.
     // This config is to limit the max concurrency of running compaction tasks.
     // -1 means no limit, and the max concurrency will be:
@@ -269,17 +269,17 @@ namespace config {
     // Number of webserver workers
     CONF_Int32(webserver_num_workers, "5");
     // Period to update rate counters and sampling counters in ms.
-    CONF_Int32(periodic_counter_update_period_ms, "500");
+    CONF_mInt32(periodic_counter_update_period_ms, "500");
 
     // Used for mini Load. mini load data file will be removed after this time.
     CONF_Int64(load_data_reserve_hours, "4");
     // log error log will be removed after this time
-    CONF_Int64(load_error_log_reserve_hours, "48");
+    CONF_mInt64(load_error_log_reserve_hours, "48");
     // Deprecated, use streaming_load_max_mb instead
-    CONF_Int64(mini_load_max_mb, "2048");
+    // CONF_Int64(mini_load_max_mb, "2048");
     CONF_Int32(number_tablet_writer_threads, "16");
 
-    CONF_Int64(streaming_load_max_mb, "10240");
+    CONF_mInt64(streaming_load_max_mb, "10240");
     // the alive time of a TabletsChannel.
     // If the channel does not receive any data till this time,
     // the channel will be removed.
@@ -291,14 +291,14 @@ namespace config {
     // the timeout of a rpc to process one batch in tablet writer.
     // you may need to increase this timeout if using larger 'streaming_load_max_mb',
     // or encounter 'tablet writer write failed' error when loading.
-    CONF_Int32(tablet_writer_rpc_timeout_sec, "600");
+    // CONF_Int32(tablet_writer_rpc_timeout_sec, "600");
 
     // Fragment thread pool
     CONF_Int32(fragment_pool_thread_num, "64");
     CONF_Int32(fragment_pool_queue_size, "1024");
 
     //for cast
-    CONF_Bool(cast, "true");
+    // CONF_Bool(cast, "true");
 
     // Spill to disk when query
     // Writable scratch directories, splitted by ";"
@@ -342,26 +342,26 @@ namespace config {
     CONF_String(pprof_profile_dir, "${DORIS_HOME}/log")
 
     // for partition
-    CONF_Bool(enable_partitioned_hash_join, "false")
+    // CONF_Bool(enable_partitioned_hash_join, "false")
     CONF_Bool(enable_partitioned_aggregation, "true")
 
     // to forward compatibility, will be removed later
-    CONF_Bool(enable_token_check, "true");
+    CONF_mBool(enable_token_check, "true");
 
     // to open/close system metrics
     CONF_Bool(enable_system_metrics, "true");
 
-    CONF_Bool(enable_prefetch, "true");
+    CONF_mBool(enable_prefetch, "true");
 
     // Number of cores Doris will used, this will effect only when it's greater than 0.
     // Otherwise, Doris will use all cores returned from "/proc/cpuinfo".
     CONF_Int32(num_cores, "0");
 
-    CONF_Bool(thread_creation_fault_injection, "false");
+    // CONF_Bool(thread_creation_fault_injection, "false");
 
     // Set this to encrypt and perform an integrity
     // check on all data spilled to disk during a query
-    CONF_Bool(disk_spill_encryption, "false");
+    // CONF_Bool(disk_spill_encryption, "false");
 
     // When BE start, If there is a broken disk, BE process will exit by default.
     // Otherwise, we will ignore the broken disk,
@@ -372,7 +372,7 @@ namespace config {
 
     // If false and --scratch_dirs contains multiple directories on the same device,
     // then only the first writable directory is used
-    CONF_Bool(allow_multiple_scratch_dirs_per_device, "false");
+    // CONF_Bool(allow_multiple_scratch_dirs_per_device, "false");
 
     // linux transparent huge page
     CONF_Bool(madvise_huge_pages, "false");
@@ -387,13 +387,13 @@ namespace config {
     CONF_String(buffer_pool_clean_pages_limit, "20G");
 
     // Sleep time in seconds between memory maintenance iterations
-    CONF_Int64(memory_maintenance_sleep_time_s, "10");
+    CONF_mInt64(memory_maintenance_sleep_time_s, "10");
 
     // Aligement
     CONF_Int32(memory_max_alignment, "16");
 
     // write buffer size before flush
-    CONF_Int64(write_buffer_size, "104857600");
+    CONF_mInt64(write_buffer_size, "104857600");
 
     // following 2 configs limit the memory consumption of load process on a Backend.
     // eg: memory limit to 80% of mem limit config but up to 100GB(default)
@@ -404,31 +404,31 @@ namespace config {
     CONF_Int32(load_process_max_memory_limit_percent, "80");    // 80%
 
     // update interval of tablet stat cache
-    CONF_Int32(tablet_stat_cache_update_interval_second, "300");
+    CONF_mInt32(tablet_stat_cache_update_interval_second, "300");
 
     // result buffer cancelled time (unit: second)
-    CONF_Int32(result_buffer_cancelled_interval_time, "300");
+    CONF_mInt32(result_buffer_cancelled_interval_time, "300");
 
     // can perform recovering tablet
     CONF_Bool(force_recovery, "false");
 
     // the increased frequency of priority for remaining tasks in BlockingPriorityQueue
-    CONF_Int32(priority_queue_remaining_tasks_increased_frequency, "512");
+    CONF_mInt32(priority_queue_remaining_tasks_increased_frequency, "512");
 
     // sync tablet_meta when modifing meta
-    CONF_Bool(sync_tablet_meta, "false");
+    CONF_mBool(sync_tablet_meta, "false");
 
     // default thrift rpc timeout ms
-    CONF_Int32(thrift_rpc_timeout_ms, "5000");
+    CONF_mInt32(thrift_rpc_timeout_ms, "5000");
 
     // txn commit rpc timeout
-    CONF_Int32(txn_commit_rpc_timeout_ms, "10000");
+    CONF_mInt32(txn_commit_rpc_timeout_ms, "10000");
 
     // If set to true, metric calculator will run
     CONF_Bool(enable_metric_calculator, "true");
 
     // max consumer num in one data consumer group, for routine load
-    CONF_Int32(max_consumer_num_per_group, "3");
+    CONF_mInt32(max_consumer_num_per_group, "3");
 
     // the size of thread pool for routine load task.
     // this should be larger than FE config 'max_concurrent_task_num_per_be' (default 5)
@@ -436,15 +436,15 @@ namespace config {
 
     // Is set to true, index loading failure will not causing BE exit,
     // and the tablet will be marked as bad, so that FE will try to repair it.
-    CONF_Bool(auto_recover_index_loading_failure, "false");
+    // CONF_Bool(auto_recover_index_loading_failure, "false");
 
     // max external scan cache batch count, means cache max_memory_cache_batch_count * batch_size row
     // default is 20, batch_size's defualt value is 1024 means 20 * 1024 rows will be cached
-    CONF_Int32(max_memory_sink_batch_count, "20");
+    CONF_mInt32(max_memory_sink_batch_count, "20");
 
     // This configuration is used for the context gc thread schedule period
     // note: unit is minute, default is 5min
-    CONF_Int32(scan_context_gc_interval_min, "5");
+    CONF_mInt32(scan_context_gc_interval_min, "5");
 
     // es scroll keep-alive
     CONF_String(es_scroll_keepalive, "5m");
@@ -463,22 +463,22 @@ namespace config {
     // path gc
     CONF_Bool(path_gc_check, "true");
     CONF_Int32(path_gc_check_interval_second, "86400");
-    CONF_Int32(path_gc_check_step, "1000");
-    CONF_Int32(path_gc_check_step_interval_ms, "10");
-    CONF_Int32(path_scan_interval_second, "86400");
+    CONF_mInt32(path_gc_check_step, "1000");
+    CONF_mInt32(path_gc_check_step_interval_ms, "10");
+    CONF_mInt32(path_scan_interval_second, "86400");
 
     // The following 2 configs limit the max usage of disk capacity of a data dir.
     // If both of these 2 threshold reached, no more data can be writen into that data dir.
     // The percent of max used capacity of a data dir
-    CONF_Int32(storage_flood_stage_usage_percent, "95");    // 95%
+    CONF_mInt32(storage_flood_stage_usage_percent, "95");    // 95%
     // The min bytes that should be left of a data dir
-    CONF_Int64(storage_flood_stage_left_capacity_bytes, "1073741824")   // 1GB
+    CONF_mInt64(storage_flood_stage_left_capacity_bytes, "1073741824")   // 1GB
     // number of thread for flushing memtable per store
     CONF_Int32(flush_thread_num_per_store, "2");
 
     // config for tablet meta checkpoint
-    CONF_Int32(tablet_meta_checkpoint_min_new_rowsets_num, "10");
-    CONF_Int32(tablet_meta_checkpoint_min_interval_secs, "600");
+    CONF_mInt32(tablet_meta_checkpoint_min_new_rowsets_num, "10");
+    CONF_mInt32(tablet_meta_checkpoint_min_interval_secs, "600");
 
     // config for default rowset type
     // Valid configs: ALPHA, BETA
@@ -489,12 +489,12 @@ namespace config {
 
     // max number of txns in txn manager
     // this is a self protection to avoid too many txns saving in manager
-    CONF_Int64(max_runnings_transactions, "2000");
+    CONF_mInt64(max_runnings_transactions, "2000");
 
     // tablet_map_lock shard size, the value is 2^n, n=0,1,2,3,4
     // this is a an enhancement for better performance to manage tablet
     CONF_Int32(tablet_map_shard_size, "1");
-    
+
     CONF_String(plugin_path, "${DORIS_HOME}/plugin")
 
 } // namespace config
diff --git a/be/src/common/configbase.cpp b/be/src/common/configbase.cpp
index 9f3673b..115c2db 100644
--- a/be/src/common/configbase.cpp
+++ b/be/src/common/configbase.cpp
@@ -28,91 +28,19 @@
 #include "common/config.h"
 #undef __IN_CONFIGBASE_CPP__
 
+#include "common/status.h"
+#include "gutil/strings/substitute.h"
+
 namespace doris {
 namespace config {
 
-std::list<Register::Field>* Register::_s_fieldlist = nullptr;
-std::map<std::string, std::string>* confmap = nullptr;
+std::map<std::string, Register::Field>* Register::_s_field_map = nullptr;
+std::map<std::string, std::string>* full_conf_map = nullptr;
 
 Properties props;
 
-// load conf file
-bool Properties::load(const char* filename) {
-    // if filename is null, use the empty props
-    if (filename == nullptr) {
-        return true;
-    }
-
-    // open the conf file
-    std::ifstream input(filename);
-    if (!input.is_open()) {
-        std::cerr << "config::load() failed to open the file:" << filename << std::endl;
-        return false;
-    }
-
-    // load properties
-    std::string line;
-    std::string key;
-    std::string value;
-    line.reserve(512);
-    while (input) {
-        // read one line at a time
-        std::getline(input, line);
-
-        // remove left and right spaces
-        trim(line);
-
-        // ignore comments
-        if (line.empty() || line[0] == '#') {
-            continue;
-        }
-
-        // read key and value
-        splitkv(line, key, value);
-        trim(key);
-        trim(value);
-
-        // insert into propmap
-        propmap[key] = value;
-    }
-
-    // close the conf file
-    input.close();
-
-    return true;
-}
-
-template <typename T>
-bool Properties::get(const char* key, const char* defstr, T& retval) const {
-    std::map<std::string, std::string>::const_iterator it = propmap.find(std::string(key));
-    std::string valstr = it != propmap.end() ? it->second : std::string(defstr);
-    trim(valstr);
-    if (!replaceenv(valstr)) {
-        return false;
-    }
-    return strtox(valstr, retval);
-}
-
-template <typename T>
-bool Properties::strtox(const std::string& valstr, std::vector<T>& retval) {
-    std::stringstream ss(valstr);
-    std::string item;
-    T t;
-    while (std::getline(ss, item, ',')) {
-        if (!strtox(trim(item), t)) {
-            return false;
-        }
-        retval.push_back(t);
-    }
-    return true;
-}
-
-const std::map<std::string, std::string>& Properties::getmap() const {
-    return propmap;
-}
-
 // trim string
-std::string& Properties::trim(std::string& s) {
+std::string& trim(std::string& s) {
     // rtrim
     s.erase(std::find_if(s.rbegin(), s.rend(), std::not1(std::ptr_fun<int, int>(std::isspace)))
                     .base(),
@@ -124,7 +52,7 @@ std::string& Properties::trim(std::string& s) {
 }
 
 // split string by '='
-void Properties::splitkv(const std::string& s, std::string& k, std::string& v) {
+void splitkv(const std::string& s, std::string& k, std::string& v) {
     const char sep = '=';
     int start = 0;
     int end = 0;
@@ -138,7 +66,7 @@ void Properties::splitkv(const std::string& s, std::string& k, std::string& v) {
 }
 
 // replace env variables
-bool Properties::replaceenv(std::string& s) {
+bool replaceenv(std::string& s) {
     std::size_t pos = 0;
     std::size_t start = 0;
     while ((start = s.find("${", pos)) != std::string::npos) {
@@ -158,7 +86,28 @@ bool Properties::replaceenv(std::string& s) {
     return true;
 }
 
-bool Properties::strtox(const std::string& valstr, bool& retval) {
+bool strtox(const std::string& valstr, bool& retval);
+bool strtox(const std::string& valstr, int16_t& retval);
+bool strtox(const std::string& valstr, int32_t& retval);
+bool strtox(const std::string& valstr, int64_t& retval);
+bool strtox(const std::string& valstr, double& retval);
+bool strtox(const std::string& valstr, std::string& retval);
+
+template <typename T>
+bool strtox(const std::string& valstr, std::vector<T>& retval) {
+    std::stringstream ss(valstr);
+    std::string item;
+    T t;
+    while (std::getline(ss, item, ',')) {
+        if (!strtox(trim(item), t)) {
+            return false;
+        }
+        retval.push_back(t);
+    }
+    return true;
+}
+
+bool strtox(const std::string& valstr, bool& retval) {
     if (valstr.compare("true") == 0) {
         retval = true;
     } else if (valstr.compare("false") == 0) {
@@ -170,7 +119,7 @@ bool Properties::strtox(const std::string& valstr, bool& retval) {
 }
 
 template <typename T>
-bool Properties::strtointeger(const std::string& valstr, T& retval) {
+bool strtointeger(const std::string& valstr, T& retval) {
     if (valstr.length() == 0) {
         return false; // empty-string is only allowed for string type.
     }
@@ -181,26 +130,28 @@ bool Properties::strtointeger(const std::string& valstr, T& retval) {
     if (errno || end != valcstr + strlen(valcstr)) {
         return false; // bad parse
     }
+    T tmp = retval;
     retval = static_cast<T>(ret64);
     if (retval != ret64) {
+        retval = tmp;
         return false;
     }
     return true;
 }
 
-bool Properties::strtox(const std::string& valstr, int16_t& retval) {
+bool strtox(const std::string& valstr, int16_t& retval) {
     return strtointeger(valstr, retval);
 }
 
-bool Properties::strtox(const std::string& valstr, int32_t& retval) {
+bool strtox(const std::string& valstr, int32_t& retval) {
     return strtointeger(valstr, retval);
 }
 
-bool Properties::strtox(const std::string& valstr, int64_t& retval) {
+bool strtox(const std::string& valstr, int64_t& retval) {
     return strtointeger(valstr, retval);
 }
 
-bool Properties::strtox(const std::string& valstr, double& retval) {
+bool strtox(const std::string& valstr, double& retval) {
     if (valstr.length() == 0) {
         return false; // empty-string is only allowed for string type.
     }
@@ -214,11 +165,78 @@ bool Properties::strtox(const std::string& valstr, double& retval) {
     return true;
 }
 
-bool Properties::strtox(const std::string& valstr, std::string& retval) {
+bool strtox(const std::string& valstr, std::string& retval) {
     retval = valstr;
     return true;
 }
 
+// load conf file
+bool Properties::load(const char* filename) {
+    // if filename is null, use the empty props
+    if (filename == nullptr) {
+        return true;
+    }
+
+    // open the conf file
+    std::ifstream input(filename);
+    if (!input.is_open()) {
+        std::cerr << "config::load() failed to open the file:" << filename << std::endl;
+        return false;
+    }
+
+    // load properties
+    std::string line;
+    std::string key;
+    std::string value;
+    line.reserve(512);
+    while (input) {
+        // read one line at a time
+        std::getline(input, line);
+
+        // remove left and right spaces
+        trim(line);
+
+        // ignore comments
+        if (line.empty() || line[0] == '#') {
+            continue;
+        }
+
+        // read key and value
+        splitkv(line, key, value);
+        trim(key);
+        trim(value);
+
+        // insert into file_conf_map
+        file_conf_map[key] = value;
+    }
+
+    // close the conf file
+    input.close();
+
+    return true;
+}
+
+template <typename T>
+bool Properties::get(const char* key, const char* defstr, T& retval) const {
+    const auto& it = file_conf_map.find(std::string(key));
+    std::string valstr = it != file_conf_map.end() ? it->second : std::string(defstr);
+    trim(valstr);
+    if (!replaceenv(valstr)) {
+        return false;
+    }
+    return strtox(valstr, retval);
+}
+
+template <typename T>
+bool update(const std::string& value, T& retval) {
+    std::string valstr(value);
+    trim(valstr);
+    if (!replaceenv(valstr)) {
+        return false;
+    }
+    return strtox(valstr, retval);
+}
+
 template <typename T>
 std::ostream& operator<<(std::ostream& out, const std::vector<T>& v) {
     size_t last = v.size() - 1;
@@ -240,7 +258,7 @@ std::ostream& operator<<(std::ostream& out, const std::vector<T>& v) {
         if (FILL_CONFMAP) {                                                                        \
             std::ostringstream oss;                                                                \
             oss << (*reinterpret_cast<TYPE*>((FIELD).storage));                                    \
-            (*confmap)[(FIELD).name] = oss.str();                                                  \
+            (*full_conf_map)[(FIELD).name] = oss.str();                                            \
         }                                                                                          \
         continue;                                                                                  \
     }
@@ -251,29 +269,64 @@ bool init(const char* filename, bool fillconfmap) {
     if (!props.load(filename)) {
         return false;
     }
-    // fill confmap ?
-    if (fillconfmap && confmap == nullptr) {
-        confmap = new std::map<std::string, std::string>();
+    // fill full_conf_map ?
+    if (fillconfmap && full_conf_map == nullptr) {
+        full_conf_map = new std::map<std::string, std::string>();
     }
 
     // set conf fields
-    for (const auto& it : *Register::_s_fieldlist) {
-        SET_FIELD(it, bool, fillconfmap);
-        SET_FIELD(it, int16_t, fillconfmap);
-        SET_FIELD(it, int32_t, fillconfmap);
-        SET_FIELD(it, int64_t, fillconfmap);
-        SET_FIELD(it, double, fillconfmap);
-        SET_FIELD(it, std::string, fillconfmap);
-        SET_FIELD(it, std::vector<bool>, fillconfmap);
-        SET_FIELD(it, std::vector<int16_t>, fillconfmap);
-        SET_FIELD(it, std::vector<int32_t>, fillconfmap);
-        SET_FIELD(it, std::vector<int64_t>, fillconfmap);
-        SET_FIELD(it, std::vector<double>, fillconfmap);
-        SET_FIELD(it, std::vector<std::string>, fillconfmap);
+    for (const auto& it : *Register::_s_field_map) {
+        SET_FIELD(it.second, bool, fillconfmap);
+        SET_FIELD(it.second, int16_t, fillconfmap);
+        SET_FIELD(it.second, int32_t, fillconfmap);
+        SET_FIELD(it.second, int64_t, fillconfmap);
+        SET_FIELD(it.second, double, fillconfmap);
+        SET_FIELD(it.second, std::string, fillconfmap);
+        SET_FIELD(it.second, std::vector<bool>, fillconfmap);
+        SET_FIELD(it.second, std::vector<int16_t>, fillconfmap);
+        SET_FIELD(it.second, std::vector<int32_t>, fillconfmap);
+        SET_FIELD(it.second, std::vector<int64_t>, fillconfmap);
+        SET_FIELD(it.second, std::vector<double>, fillconfmap);
+        SET_FIELD(it.second, std::vector<std::string>, fillconfmap);
     }
 
     return true;
 }
 
+#define UPDATE_FIELD(FIELD, VALUE, TYPE)                                             \
+    if (strcmp((FIELD).type, #TYPE) == 0) {                                          \
+        if (!update((VALUE), *reinterpret_cast<TYPE*>((FIELD).storage))) {           \
+            return Status::InvalidArgument(                                          \
+                    strings::Substitute("convert '$0' as $1 failed", VALUE, #TYPE)); \
+        }                                                                            \
+        if (full_conf_map != nullptr) {                                              \
+            std::ostringstream oss;                                                  \
+            oss << (*reinterpret_cast<TYPE*>((FIELD).storage));                      \
+            (*full_conf_map)[(FIELD).name] = oss.str();                              \
+        }                                                                            \
+        return Status::OK();                                                         \
+    }
+
+Status set_config(const std::string& field, const std::string& value) {
+    auto it = Register::_s_field_map->find(field);
+    if (it == Register::_s_field_map->end()) {
+        return Status::NotFound(strings::Substitute("'$0' is not found", field));
+    }
+
+    if (!it->second.valmutable) {
+        return Status::NotSupported(strings::Substitute("'$0' is not support to modify", field));
+    }
+
+    UPDATE_FIELD(it->second, value, bool);
+    UPDATE_FIELD(it->second, value, int16_t);
+    UPDATE_FIELD(it->second, value, int32_t);
+    UPDATE_FIELD(it->second, value, int64_t);
+    UPDATE_FIELD(it->second, value, double);
+
+    // The other types are not thread safe to change dynamically.
+    return Status::NotSupported(strings::Substitute(
+            "'$0' is type of '$1' which is not support to modify", field, it->second.type));
+}
+
 } // namespace config
 } // namespace doris
diff --git a/be/src/common/configbase.h b/be/src/common/configbase.h
index a4057c6..9dd58bb 100644
--- a/be/src/common/configbase.h
+++ b/be/src/common/configbase.h
@@ -20,56 +20,72 @@
 
 #include <stdint.h>
 
-#include <list>
 #include <map>
+#include <string>
 #include <vector>
 
 namespace doris {
+class Status;
+
 namespace config {
 
 class Register {
 public:
     struct Field {
-        const char* type;
-        const char* name;
-        void* storage;
-        const char* defval;
-        Field(const char* ftype, const char* fname, void* fstorage, const char* fdefval)
-                : type(ftype), name(fname), storage(fstorage), defval(fdefval) {}
+        const char* type = nullptr;
+        const char* name = nullptr;
+        void* storage = nullptr;
+        const char* defval = nullptr;
+        bool valmutable = false;
+        Field(const char* ftype, const char* fname, void* fstorage, const char* fdefval,
+              bool fvalmutable)
+                : type(ftype),
+                  name(fname),
+                  storage(fstorage),
+                  defval(fdefval),
+                  valmutable(fvalmutable) {}
     };
 
 public:
-    static std::list<Field>* _s_fieldlist;
+    static std::map<std::string, Field>* _s_field_map;
 
 public:
-    Register(const char* ftype, const char* fname, void* fstorage, const char* fdefval) {
-        if (_s_fieldlist == NULL) {
-            _s_fieldlist = new std::list<Field>();
+    Register(const char* ftype, const char* fname, void* fstorage, const char* fdefval,
+             bool fvalmutable) {
+        if (_s_field_map == nullptr) {
+            _s_field_map = new std::map<std::string, Field>();
         }
-        Field field(ftype, fname, fstorage, fdefval);
-        _s_fieldlist->push_back(field);
+        Field field(ftype, fname, fstorage, fdefval, fvalmutable);
+        _s_field_map->insert(std::make_pair(std::string(fname), field));
     }
 };
 
-#define DEFINE_FIELD(FIELD_TYPE, FIELD_NAME, FIELD_DEFAULT) \
-    FIELD_TYPE FIELD_NAME;                                  \
-    static Register reg_##FIELD_NAME(#FIELD_TYPE, #FIELD_NAME, &FIELD_NAME, FIELD_DEFAULT);
+#define DEFINE_FIELD(FIELD_TYPE, FIELD_NAME, FIELD_DEFAULT, VALMUTABLE)                    \
+    FIELD_TYPE FIELD_NAME;                                                                 \
+    static Register reg_##FIELD_NAME(#FIELD_TYPE, #FIELD_NAME, &FIELD_NAME, FIELD_DEFAULT, \
+                                     VALMUTABLE);
 
 #define DECLARE_FIELD(FIELD_TYPE, FIELD_NAME) extern FIELD_TYPE FIELD_NAME;
 
 #ifdef __IN_CONFIGBASE_CPP__
-#define CONF_Bool(name, defaultstr) DEFINE_FIELD(bool, name, defaultstr)
-#define CONF_Int16(name, defaultstr) DEFINE_FIELD(int16_t, name, defaultstr)
-#define CONF_Int32(name, defaultstr) DEFINE_FIELD(int32_t, name, defaultstr)
-#define CONF_Int64(name, defaultstr) DEFINE_FIELD(int64_t, name, defaultstr)
-#define CONF_Double(name, defaultstr) DEFINE_FIELD(double, name, defaultstr)
-#define CONF_String(name, defaultstr) DEFINE_FIELD(std::string, name, defaultstr)
-#define CONF_Bools(name, defaultstr) DEFINE_FIELD(std::vector<bool>, name, defaultstr)
-#define CONF_Int16s(name, defaultstr) DEFINE_FIELD(std::vector<int16_t>, name, defaultstr)
-#define CONF_Int32s(name, defaultstr) DEFINE_FIELD(std::vector<int32_t>, name, defaultstr)
-#define CONF_Int64s(name, defaultstr) DEFINE_FIELD(std::vector<int64_t>, name, defaultstr)
-#define CONF_Doubles(name, defaultstr) DEFINE_FIELD(std::vector<double>, name, defaultstr)
-#define CONF_Strings(name, defaultstr) DEFINE_FIELD(std::vector<std::string>, name, defaultstr)
+#define CONF_Bool(name, defaultstr) DEFINE_FIELD(bool, name, defaultstr, false)
+#define CONF_Int16(name, defaultstr) DEFINE_FIELD(int16_t, name, defaultstr, false)
+#define CONF_Int32(name, defaultstr) DEFINE_FIELD(int32_t, name, defaultstr, false)
+#define CONF_Int64(name, defaultstr) DEFINE_FIELD(int64_t, name, defaultstr, false)
+#define CONF_Double(name, defaultstr) DEFINE_FIELD(double, name, defaultstr, false)
+#define CONF_String(name, defaultstr) DEFINE_FIELD(std::string, name, defaultstr, false)
+#define CONF_Bools(name, defaultstr) DEFINE_FIELD(std::vector<bool>, name, defaultstr, false)
+#define CONF_Int16s(name, defaultstr) DEFINE_FIELD(std::vector<int16_t>, name, defaultstr, false)
+#define CONF_Int32s(name, defaultstr) DEFINE_FIELD(std::vector<int32_t>, name, defaultstr, false)
+#define CONF_Int64s(name, defaultstr) DEFINE_FIELD(std::vector<int64_t>, name, defaultstr, false)
+#define CONF_Doubles(name, defaultstr) DEFINE_FIELD(std::vector<double>, name, defaultstr, false)
+#define CONF_Strings(name, defaultstr) \
+    DEFINE_FIELD(std::vector<std::string>, name, defaultstr, false)
+#define CONF_mBool(name, defaultstr) DEFINE_FIELD(bool, name, defaultstr, true)
+#define CONF_mInt16(name, defaultstr) DEFINE_FIELD(int16_t, name, defaultstr, true)
+#define CONF_mInt32(name, defaultstr) DEFINE_FIELD(int32_t, name, defaultstr, true)
+#define CONF_mInt64(name, defaultstr) DEFINE_FIELD(int64_t, name, defaultstr, true)
+#define CONF_mDouble(name, defaultstr) DEFINE_FIELD(double, name, defaultstr, true)
 #else
 #define CONF_Bool(name, defaultstr) DECLARE_FIELD(bool, name)
 #define CONF_Int16(name, defaultstr) DECLARE_FIELD(int16_t, name)
@@ -83,40 +99,33 @@ public:
 #define CONF_Int64s(name, defaultstr) DECLARE_FIELD(std::vector<int64_t>, name)
 #define CONF_Doubles(name, defaultstr) DECLARE_FIELD(std::vector<double>, name)
 #define CONF_Strings(name, defaultstr) DECLARE_FIELD(std::vector<std::string>, name)
+#define CONF_mBool(name, defaultstr) DECLARE_FIELD(bool, name)
+#define CONF_mInt16(name, defaultstr) DECLARE_FIELD(int16_t, name)
+#define CONF_mInt32(name, defaultstr) DECLARE_FIELD(int32_t, name)
+#define CONF_mInt64(name, defaultstr) DECLARE_FIELD(int64_t, name)
+#define CONF_mDouble(name, defaultstr) DECLARE_FIELD(double, name)
 #endif
 
+// configuration properties load from config file.
 class Properties {
 public:
     bool load(const char* filename);
     template <typename T>
     bool get(const char* key, const char* defstr, T& retval) const;
-    const std::map<std::string, std::string>& getmap() const;
-
-private:
-    template <typename T>
-    static bool strtox(const std::string& valstr, std::vector<T>& retval);
-    template <typename T>
-    static bool strtointeger(const std::string& valstr, T& retval);
-    static bool strtox(const std::string& valstr, bool& retval);
-    static bool strtox(const std::string& valstr, int16_t& retval);
-    static bool strtox(const std::string& valstr, int32_t& retval);
-    static bool strtox(const std::string& valstr, int64_t& retval);
-    static bool strtox(const std::string& valstr, double& retval);
-    static bool strtox(const std::string& valstr, std::string& retval);
-    static std::string& trim(std::string& s);
-    static void splitkv(const std::string& s, std::string& k, std::string& v);
-    static bool replaceenv(std::string& s);
 
 private:
-    std::map<std::string, std::string> propmap;
+    std::map<std::string, std::string> file_conf_map;
 };
 
 extern Properties props;
 
-extern std::map<std::string, std::string>* confmap;
+// full configurations.
+extern std::map<std::string, std::string>* full_conf_map;
 
 bool init(const char* filename, bool fillconfmap = false);
 
+Status set_config(const std::string& field, const std::string& value);
+
 } // namespace config
 } // namespace doris
 
diff --git a/be/src/common/logging.h b/be/src/common/logging.h
index b2021e6..cad58f3 100644
--- a/be/src/common/logging.h
+++ b/be/src/common/logging.h
@@ -42,7 +42,6 @@
 // This is including a glog internal file.  We want this to expose the
 // function to get the stack trace.
 #include <glog/logging.h>
-#include <common/config.h>
 #undef MutexLock
 #endif
 
diff --git a/be/src/http/CMakeLists.txt b/be/src/http/CMakeLists.txt
index 39f9863..0ee8d14 100644
--- a/be/src/http/CMakeLists.txt
+++ b/be/src/http/CMakeLists.txt
@@ -46,6 +46,7 @@ add_library(Webserver STATIC
   action/stream_load.cpp
   action/meta_action.cpp
   action/compaction_action.cpp
+  action/update_config_action.cpp
   #  action/multi_start.cpp
   #  action/multi_show.cpp
   #  action/multi_commit.cpp
diff --git a/be/src/http/action/update_config_action.cpp b/be/src/http/action/update_config_action.cpp
new file mode 100644
index 0000000..91eb6b0
--- /dev/null
+++ b/be/src/http/action/update_config_action.cpp
@@ -0,0 +1,76 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "http/action/update_config_action.h"
+
+#include <rapidjson/document.h>
+#include <rapidjson/prettywriter.h>
+#include <rapidjson/rapidjson.h>
+#include <rapidjson/stringbuffer.h>
+
+#include <string>
+
+#include "common/configbase.h"
+#include "common/logging.h"
+#include "common/status.h"
+#include "gutil/strings/substitute.h"
+#include "http/http_channel.h"
+#include "http/http_headers.h"
+#include "http/http_request.h"
+#include "http/http_response.h"
+#include "http/http_status.h"
+
+namespace doris {
+
+const static std::string HEADER_JSON = "application/json";
+
+void UpdateConfigAction::handle(HttpRequest* req) {
+    LOG(INFO) << req->debug_string();
+
+    Status s;
+    std::string msg;
+    if (req->params()->size() != 1) {
+        s = Status::InvalidArgument("");
+        msg = "Now only support to set a single config once, via 'config_name=new_value'";
+    } else {
+        DCHECK(req->params()->size() == 1);
+        const std::string& config = req->params()->begin()->first;
+        const std::string& new_value = req->params()->begin()->second;
+        s = config::set_config(config, new_value);
+        if (s.ok()) {
+            LOG(INFO) << "set_config " << config << "=" << new_value << " success";
+        } else {
+            LOG(WARNING) << "set_config " << config << "=" << new_value << " failed";
+            msg = strings::Substitute("set $0=$1 failed, reason: $2", config, new_value,
+                                      s.to_string());
+        }
+    }
+
+    std::string status(s.ok() ? "OK" : "BAD");
+    rapidjson::Document root;
+    root.SetObject();
+    root.AddMember("status", rapidjson::Value(status.c_str(), status.size()), root.GetAllocator());
+    root.AddMember("msg", rapidjson::Value(msg.c_str(), msg.size()), root.GetAllocator());
+    rapidjson::StringBuffer strbuf;
+    rapidjson::PrettyWriter<rapidjson::StringBuffer> writer(strbuf);
+    root.Accept(writer);
+
+    req->add_output_header(HttpHeaders::CONTENT_TYPE, HEADER_JSON.c_str());
+    HttpChannel::send_reply(req, HttpStatus::OK, strbuf.GetString());
+}
+
+} // namespace doris
diff --git a/be/src/util/thrift_rpc_helper.h b/be/src/http/action/update_config_action.h
similarity index 51%
copy from be/src/util/thrift_rpc_helper.h
copy to be/src/http/action/update_config_action.h
index 0b014f5..97965a2 100644
--- a/be/src/util/thrift_rpc_helper.h
+++ b/be/src/http/action/update_config_action.h
@@ -17,41 +17,17 @@
 
 #pragma once
 
-#include "common/status.h"
-#include "gen_cpp/FrontendService_types.h"
+#include "http/http_handler.h"
 
 namespace doris {
 
-class ExecEnv;
-class FrontendServiceClient;
-template <class T> 
-class ClientConnection;
-
-// this class is a helper for jni call. easy for unit test
-class ThriftRpcHelper {
+// Update BE config.
+class UpdateConfigAction : public HttpHandler {
 public:
-    static void setup(ExecEnv* exec_env);
-
-    // for default timeout
-    template<typename T>
-    static Status rpc(
-        const std::string& ip,
-        const int32_t port,
-        std::function<void (ClientConnection<T>&)> callback) {
-
-        return rpc(ip, port, callback, config::thrift_rpc_timeout_ms);
-    }
+    UpdateConfigAction() {}
+    virtual ~UpdateConfigAction() {}
 
-    template<typename T>
-    static Status rpc(
-        const std::string& ip,
-        const int32_t port,
-        std::function<void (ClientConnection<T>&)> callback,
-        int timeout_ms);
-
-private:
-    static ExecEnv* _s_exec_env;
+    void handle(HttpRequest* req) override;
 };
 
-}
-
+} // namespace doris
diff --git a/be/src/http/default_path_handlers.cpp b/be/src/http/default_path_handlers.cpp
index cdc3a31..77c8958 100644
--- a/be/src/http/default_path_handlers.cpp
+++ b/be/src/http/default_path_handlers.cpp
@@ -68,7 +68,7 @@ void logs_handler(const WebPageHandler::ArgumentMap& args, std::stringstream* ou
 void config_handler(const WebPageHandler::ArgumentMap& args, std::stringstream* output) {
     (*output) << "<h2>Configurations</h2>";
     (*output) << "<pre>";
-    for (const auto& it : *(config::confmap)) {
+    for (const auto& it : *(config::full_conf_map)) {
         (*output) << it.first << "=" << it.second << std::endl;
     }
     (*output) << "</pre>";
diff --git a/be/src/http/http_client.cpp b/be/src/http/http_client.cpp
index b1fdb66..0f09384 100644
--- a/be/src/http/http_client.cpp
+++ b/be/src/http/http_client.cpp
@@ -15,6 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "common/config.h"
 #include "http/http_client.h"
 
 namespace doris {
diff --git a/be/src/olap/olap_meta.cpp b/be/src/olap/olap_meta.cpp
index d1329c5..22f7d7e 100755
--- a/be/src/olap/olap_meta.cpp
+++ b/be/src/olap/olap_meta.cpp
@@ -38,7 +38,6 @@ using rocksdb::ReadOptions;
 using rocksdb::WriteOptions;
 using rocksdb::Slice;
 using rocksdb::Iterator;
-using rocksdb::Status;
 using rocksdb::kDefaultColumnFamilyName;
 using rocksdb::NewFixedPrefixTransform;
 
@@ -78,7 +77,7 @@ OLAPStatus OlapMeta::init() {
     ColumnFamilyOptions meta_column_family;
     meta_column_family.prefix_extractor.reset(NewFixedPrefixTransform(PREFIX_LENGTH));
     column_families.emplace_back(META_COLUMN_FAMILY, meta_column_family);
-    Status s = DB::Open(options, db_path, column_families, &_handles, &_db);
+    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 OLAP_ERR_META_OPEN_DB;
@@ -90,7 +89,7 @@ OLAPStatus OlapMeta::get(const int column_family_index, const std::string& key,
     DorisMetrics::meta_read_request_total.increment(1);
     rocksdb::ColumnFamilyHandle* handle = _handles[column_family_index];
     int64_t duration_ns = 0;
-    Status s = Status::OK();
+    rocksdb::Status s;
     {
         SCOPED_RAW_TIMER(&duration_ns);
         s = _db->Get(ReadOptions(), handle, Slice(key), value);
@@ -109,7 +108,7 @@ OLAPStatus OlapMeta::put(const int column_family_index, const std::string& key,
     DorisMetrics::meta_write_request_total.increment(1);
     rocksdb::ColumnFamilyHandle* handle = _handles[column_family_index];
     int64_t duration_ns = 0;
-    Status s = Status::OK();
+    rocksdb::Status s;
     {
         SCOPED_RAW_TIMER(&duration_ns);
         WriteOptions write_options;
@@ -127,7 +126,7 @@ OLAPStatus OlapMeta::put(const int column_family_index, const std::string& key,
 OLAPStatus OlapMeta::remove(const int column_family_index, const std::string& key) {
     DorisMetrics::meta_write_request_total.increment(1);
     rocksdb::ColumnFamilyHandle* handle = _handles[column_family_index];
-    Status s = Status::OK();
+    rocksdb::Status s;
     int64_t duration_ns = 0;
     {
         SCOPED_RAW_TIMER(&duration_ns);
@@ -152,7 +151,7 @@ OLAPStatus OlapMeta::iterate(const int column_family_index, const std::string& p
     } else {
         it->Seek(prefix);
     }
-    Status status = it->status();
+    rocksdb::Status status = it->status();
     if (!status.ok()) {
         LOG(WARNING) << "rocksdb seek failed. reason:" << status.ToString();
         return OLAP_ERR_META_ITERATOR;
diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp
index ec60b0b..6015f5e 100644
--- a/be/src/olap/olap_server.cpp
+++ b/be/src/olap/olap_server.cpp
@@ -160,13 +160,13 @@ void* StorageEngine::_fd_cache_clean_callback(void* arg) {
 #ifdef GOOGLE_PROFILER
     ProfilerRegisterThread();
 #endif
-    uint32_t interval = config::file_descriptor_cache_clean_interval;
-    if (interval <= 0) {
-        OLAP_LOG_WARNING("config of file descriptor clean interval is illegal: [%d], "
-                         "force set to 3600", interval);
-        interval = 3600;
-    }
     while (!_stop_bg_worker) {
+        int32_t interval = config::file_descriptor_cache_clean_interval;
+        if (interval <= 0) {
+            OLAP_LOG_WARNING("config of file descriptor clean interval is illegal: [%d], "
+                             "force set to 3600", interval);
+            interval = 3600;
+        }
         SLEEP_IN_BG_WORKER(interval);
 
         _start_clean_fd_cache();
@@ -179,13 +179,6 @@ void* StorageEngine::_base_compaction_thread_callback(void* arg, DataDir* data_d
 #ifdef GOOGLE_PROFILER
     ProfilerRegisterThread();
 #endif
-    uint32_t interval = config::base_compaction_check_interval_seconds;
-    if (interval <= 0) {
-        OLAP_LOG_WARNING("base compaction check interval config is illegal: [%d], "
-                         "force set to 1", interval);
-        interval = 1;
-    }
-
     //string last_base_compaction_fs;
     //TTabletId last_base_compaction_tablet_id = -1;
     while (!_stop_bg_worker) {
@@ -197,6 +190,12 @@ void* StorageEngine::_base_compaction_thread_callback(void* arg, DataDir* data_d
             _perform_base_compaction(data_dir);
         }
 
+        int32_t interval = config::base_compaction_check_interval_seconds;
+        if (interval <= 0) {
+            OLAP_LOG_WARNING("base compaction check interval config is illegal: [%d], "
+                             "force set to 1", interval);
+            interval = 1;
+        }
         SLEEP_IN_BG_WORKER(interval);
     }
 
@@ -253,17 +252,15 @@ void* StorageEngine::_disk_stat_monitor_thread_callback(void* arg) {
 #ifdef GOOGLE_PROFILER
     ProfilerRegisterThread();
 #endif
-
-    uint32_t interval = config::disk_stat_monitor_interval;
-
-    if (interval <= 0) {
-        LOG(WARNING) << "disk_stat_monitor_interval config is illegal: " << interval
-                << ", force set to 1";
-        interval = 1;
-    }
-
     while (!_stop_bg_worker) {
         _start_disk_stat_monitor();
+
+        int32_t interval = config::disk_stat_monitor_interval;
+        if (interval <= 0) {
+            LOG(WARNING) << "disk_stat_monitor_interval config is illegal: " << interval
+                         << ", force set to 1";
+            interval = 1;
+        }
         SLEEP_IN_BG_WORKER(interval);
     }
 
@@ -275,12 +272,6 @@ void* StorageEngine::_cumulative_compaction_thread_callback(void* arg, DataDir*
     ProfilerRegisterThread();
 #endif
     LOG(INFO) << "try to start cumulative compaction process!";
-    uint32_t interval = config::cumulative_compaction_check_interval_seconds;
-    if (interval <= 0) {
-        LOG(WARNING) << "cumulative compaction check interval config is illegal:" << interval
-            << "will be forced set to one";
-        interval = 1;
-    }
 
     while (!_stop_bg_worker) {
         // must be here, because this thread is start on start and
@@ -290,6 +281,13 @@ void* StorageEngine::_cumulative_compaction_thread_callback(void* arg, DataDir*
         if (!data_dir->reach_capacity_limit(0)) {
             _perform_cumulative_compaction(data_dir);
         }
+
+        int32_t interval = config::cumulative_compaction_check_interval_seconds;
+        if (interval <= 0) {
+            LOG(WARNING) << "cumulative compaction check interval config is illegal:" << interval
+                         << "will be forced set to one";
+            interval = 1;
+        }
         SLEEP_IN_BG_WORKER(interval);
     }
 
@@ -300,17 +298,15 @@ void* StorageEngine::_unused_rowset_monitor_thread_callback(void* arg) {
 #ifdef GOOGLE_PROFILER
     ProfilerRegisterThread();
 #endif
-
-    uint32_t interval = config::unused_rowset_monitor_interval;
-
-    if (interval <= 0) {
-        LOG(WARNING) << "unused_rowset_monitor_interval config is illegal: " << interval
-                << ", force set to 1";
-        interval = 1;
-    }
-
     while (!_stop_bg_worker) {
         start_delete_unused_rowset();
+
+        int32_t interval = config::unused_rowset_monitor_interval;
+        if (interval <= 0) {
+            LOG(WARNING) << "unused_rowset_monitor_interval config is illegal: " << interval
+                         << ", force set to 1";
+            interval = 1;
+        }
         SLEEP_IN_BG_WORKER(interval);
     }
 
@@ -325,17 +321,18 @@ void* StorageEngine::_path_gc_thread_callback(void* arg) {
 #endif
 
     LOG(INFO) << "try to start path gc thread!";
-    uint32_t interval = config::path_gc_check_interval_second;
-    if (interval <= 0) {
-        LOG(WARNING) << "path gc thread check interval config is illegal:" << interval
-                     << "will be forced set to half hour";
-        interval = 1800; // 0.5 hour
-    }
 
     while (!_stop_bg_worker) {
         LOG(INFO) << "try to perform path gc!";
         // perform path gc by rowset id
         ((DataDir*)arg)->perform_path_gc_by_rowsetid();
+
+        int32_t interval = config::path_gc_check_interval_second;
+        if (interval <= 0) {
+            LOG(WARNING) << "path gc thread check interval config is illegal:" << interval
+                         << "will be forced set to half hour";
+            interval = 1800; // 0.5 hour
+        }
         SLEEP_IN_BG_WORKER(interval);
     }
 
@@ -348,16 +345,17 @@ void* StorageEngine::_path_scan_thread_callback(void* arg) {
 #endif
 
     LOG(INFO) << "try to start path scan thread!";
-    uint32_t interval = config::path_scan_interval_second;
-    if (interval <= 0) {
-        LOG(WARNING) << "path gc thread check interval config is illegal:" << interval
-                     << "will be forced set to one day";
-        interval = 24 * 3600; // one day
-    }
 
     while (!_stop_bg_worker) {
         LOG(INFO) << "try to perform path scan!";
         ((DataDir*)arg)->perform_path_scan();
+
+        int32_t interval = config::path_scan_interval_second;
+        if (interval <= 0) {
+            LOG(WARNING) << "path gc thread check interval config is illegal:" << interval
+                         << "will be forced set to one day";
+            interval = 24 * 3600; // one day
+        }
         SLEEP_IN_BG_WORKER(interval);
     }
 
diff --git a/be/src/plugin/plugin_mgr.cpp b/be/src/plugin/plugin_mgr.cpp
index fc459da..295470a 100644
--- a/be/src/plugin/plugin_mgr.cpp
+++ b/be/src/plugin/plugin_mgr.cpp
@@ -17,6 +17,7 @@
 
 #include <boost/foreach.hpp>
 
+#include "common/config.h"
 #include "plugin/plugin_mgr.h"
 #include "gutil/strings/substitute.h"
 
@@ -159,4 +160,4 @@ Status PluginMgr::get_all_plugin_info(std::vector<TPluginInfo>* plugin_info_list
     return Status::OK();
 }
 
-}
\ No newline at end of file
+}
diff --git a/be/src/runtime/external_scan_context_mgr.cpp b/be/src/runtime/external_scan_context_mgr.cpp
index 57ad940..6583b02 100644
--- a/be/src/runtime/external_scan_context_mgr.cpp
+++ b/be/src/runtime/external_scan_context_mgr.cpp
@@ -27,7 +27,7 @@
 
 namespace doris {
 
-ExternalScanContextMgr::ExternalScanContextMgr(ExecEnv* exec_env) : _exec_env(exec_env), _is_stop(false), _scan_context_gc_interval_min(doris::config::scan_context_gc_interval_min) {
+ExternalScanContextMgr::ExternalScanContextMgr(ExecEnv* exec_env) : _exec_env(exec_env), _is_stop(false) {
     // start the reaper thread for gc the expired context
     _keep_alive_reaper.reset(
             new std::thread(
@@ -88,7 +88,7 @@ Status ExternalScanContextMgr::clear_scan_context(const std::string& context_id)
 
 void ExternalScanContextMgr::gc_expired_context() {
     while (!_is_stop) {
-        std::this_thread::sleep_for(std::chrono::seconds(_scan_context_gc_interval_min * 60));
+        std::this_thread::sleep_for(std::chrono::seconds(doris::config::scan_context_gc_interval_min * 60));
         time_t current_time = time(NULL);
         std::vector<std::shared_ptr<ScanContext>> expired_contexts;
         {
diff --git a/be/src/runtime/external_scan_context_mgr.h b/be/src/runtime/external_scan_context_mgr.h
index 691fdfc..9730e55 100644
--- a/be/src/runtime/external_scan_context_mgr.h
+++ b/be/src/runtime/external_scan_context_mgr.h
@@ -70,7 +70,6 @@ private:
     bool _is_stop;
     std::unique_ptr<std::thread> _keep_alive_reaper;
     std::mutex _lock;
-    u_int32_t _scan_context_gc_interval_min;
 };
 
 }
\ No newline at end of file
diff --git a/be/src/runtime/load_channel_mgr.cpp b/be/src/runtime/load_channel_mgr.cpp
index a801f50..0e31bb1 100644
--- a/be/src/runtime/load_channel_mgr.cpp
+++ b/be/src/runtime/load_channel_mgr.cpp
@@ -26,7 +26,7 @@
 
 namespace doris {
 
-// Calculate the totol memory limit of all load tasks on this BE
+// Calculate the total memory limit of all load tasks on this BE
 static int64_t calc_process_max_load_memory(int64_t process_mem_limit) {
     if (process_mem_limit == -1) {
         // no limit
diff --git a/be/src/runtime/mem_pool.h b/be/src/runtime/mem_pool.h
index 2b05fb7..a9da9cf 100644
--- a/be/src/runtime/mem_pool.h
+++ b/be/src/runtime/mem_pool.h
@@ -25,6 +25,7 @@
 #include <string>
 #include <vector>
 
+#include "common/config.h"
 #include "common/logging.h"
 #include "gutil/dynamic_annotations.h"
 #include "util/bit_util.h"
diff --git a/be/src/runtime/thread_resource_mgr.cpp b/be/src/runtime/thread_resource_mgr.cpp
index 3694f60..4b3a6ac 100644
--- a/be/src/runtime/thread_resource_mgr.cpp
+++ b/be/src/runtime/thread_resource_mgr.cpp
@@ -21,6 +21,7 @@
 
 #include <boost/algorithm/string.hpp>
 
+#include "common/config.h"
 #include "common/logging.h"
 #include "util/cpu_info.h"
 
diff --git a/be/src/service/backend_options.cpp b/be/src/service/backend_options.cpp
index 10c7867..6935920 100644
--- a/be/src/service/backend_options.cpp
+++ b/be/src/service/backend_options.cpp
@@ -21,6 +21,7 @@
 
 #include "gutil/strings/split.h"
 
+#include "common/config.h"
 #include "common/logging.h"
 #include "common/status.h"
 #include "util/network_util.h"
diff --git a/be/src/service/http_service.cpp b/be/src/service/http_service.cpp
index 0a2aeb4..caf6a2c 100644
--- a/be/src/service/http_service.cpp
+++ b/be/src/service/http_service.cpp
@@ -28,6 +28,7 @@
 #include "http/action/restore_tablet_action.h"
 #include "http/action/snapshot_action.h"
 #include "http/action/stream_load.h"
+#include "http/action/update_config_action.h"
 #include "http/default_path_handlers.h"
 #include "http/download_action.h"
 #include "http/ev_http_server.h"
@@ -119,6 +120,9 @@ Status HttpService::start() {
     CompactionAction* run_compaction_action = new CompactionAction(CompactionActionType::RUN_COMPACTION);
     _ev_http_server->register_handler(HttpMethod::POST, "/api/compaction/run", run_compaction_action);
 
+    UpdateConfigAction* update_config_action = new UpdateConfigAction();
+    _ev_http_server->register_handler(HttpMethod::POST, "/api/update_config", update_config_action);
+
     RETURN_IF_ERROR(_ev_http_server->start());
     return Status::OK();
 }
diff --git a/be/src/util/metrics.h b/be/src/util/metrics.h
index c101b6c..331c5c3 100644
--- a/be/src/util/metrics.h
+++ b/be/src/util/metrics.h
@@ -26,6 +26,7 @@
 #include <mutex>
 #include <iomanip>
 
+#include "common/config.h"
 #include "util/spinlock.h"
 #include "util/core_local.h"
 
diff --git a/be/src/util/runtime_profile.cpp b/be/src/util/runtime_profile.cpp
index e596064..51432f7 100644
--- a/be/src/util/runtime_profile.cpp
+++ b/be/src/util/runtime_profile.cpp
@@ -23,6 +23,7 @@
 #include <iomanip>
 #include <iostream>
 
+#include "common/config.h"
 #include "common/object_pool.h"
 #include "util/container_util.hpp"
 #include "util/cpu_info.h"
diff --git a/be/src/util/thrift_rpc_helper.h b/be/src/util/thrift_rpc_helper.h
index 0b014f5..543e43a 100644
--- a/be/src/util/thrift_rpc_helper.h
+++ b/be/src/util/thrift_rpc_helper.h
@@ -17,6 +17,7 @@
 
 #pragma once
 
+#include "common/config.h"
 #include "common/status.h"
 #include "gen_cpp/FrontendService_types.h"
 
diff --git a/be/test/common/config_test.cpp b/be/test/common/config_test.cpp
index f17f0bf..a6d36ae 100644
--- a/be/test/common/config_test.cpp
+++ b/be/test/common/config_test.cpp
@@ -20,12 +20,13 @@
 #undef __IN_CONFIGBASE_CPP__
 
 #include <gtest/gtest.h>
+#include "common/status.h"
 
 namespace doris {
 using namespace config;
 
 class ConfigTest : public testing::Test {
-    void SetUp() override { config::Register::_s_fieldlist->clear(); }
+    void SetUp() override { config::Register::_s_field_map->clear(); }
 };
 
 TEST_F(ConfigTest, DumpAllConfigs) {
@@ -45,7 +46,7 @@ TEST_F(ConfigTest, DumpAllConfigs) {
 
     config::init(nullptr, true);
     std::stringstream ss;
-    for (const auto& it : *(config::confmap)) {
+    for (const auto& it : *(config::full_conf_map)) {
         ss << it.first << "=" << it.second << std::endl;
     }
     ASSERT_EQ(
@@ -66,6 +67,78 @@ cfg_std_vector_std_string=doris, config, test, string
             ss.str());
 }
 
+TEST_F(ConfigTest, UpdateConfigs) {
+    CONF_Bool(cfg_bool_immutable, "true");
+    CONF_mBool(cfg_bool, "false");
+    CONF_mDouble(cfg_double, "123.456");
+    CONF_mInt16(cfg_int16_t, "2561");
+    CONF_mInt32(cfg_int32_t, "65536123");
+    CONF_mInt64(cfg_int64_t, "4294967296123");
+    CONF_String(cfg_std_string, "doris_config_test_string");
+
+    config::init(nullptr, true);
+
+    // bool
+    ASSERT_FALSE(cfg_bool);
+    ASSERT_TRUE(config::set_config("cfg_bool", "true").ok());
+    ASSERT_TRUE(cfg_bool);
+
+    // double
+    ASSERT_EQ(cfg_double, 123.456);
+    ASSERT_TRUE(config::set_config("cfg_double", "654.321").ok());
+    ASSERT_EQ(cfg_double, 654.321);
+
+    // int16
+    ASSERT_EQ(cfg_int16_t, 2561);
+    ASSERT_TRUE(config::set_config("cfg_int16_t", "2562").ok());
+    ASSERT_EQ(cfg_int16_t, 2562);
+
+    // int32
+    ASSERT_EQ(cfg_int32_t, 65536123);
+    ASSERT_TRUE(config::set_config("cfg_int32_t", "65536124").ok());
+    ASSERT_EQ(cfg_int32_t, 65536124);
+
+    // int64
+    ASSERT_EQ(cfg_int64_t, 4294967296123);
+    ASSERT_TRUE(config::set_config("cfg_int64_t", "4294967296124").ok());
+    ASSERT_EQ(cfg_int64_t, 4294967296124);
+
+    // not exist
+    Status s = config::set_config("cfg_not_exist", "123");
+    ASSERT_FALSE(s.ok());
+    ASSERT_EQ(s.to_string(), "Not found: 'cfg_not_exist' is not found");
+
+    // immutable
+    ASSERT_TRUE(cfg_bool_immutable);
+    s = config::set_config("cfg_bool_immutable", "false");
+    ASSERT_FALSE(s.ok());
+    ASSERT_EQ(s.to_string(), "Not supported: 'cfg_bool_immutable' is not support to modify");
+    ASSERT_TRUE(cfg_bool_immutable);
+
+    // convert error
+    s = config::set_config("cfg_bool", "falseeee");
+    ASSERT_FALSE(s.ok());
+    ASSERT_EQ(s.to_string(), "Invalid argument: convert 'falseeee' as bool failed");
+    ASSERT_TRUE(cfg_bool);
+
+    s = config::set_config("cfg_double", "");
+    ASSERT_FALSE(s.ok());
+    ASSERT_EQ(s.to_string(), "Invalid argument: convert '' as double failed");
+    ASSERT_EQ(cfg_double, 654.321);
+
+    // convert error
+    s = config::set_config("cfg_int32_t", "4294967296124");
+    ASSERT_FALSE(s.ok());
+    ASSERT_EQ(s.to_string(), "Invalid argument: convert '4294967296124' as int32_t failed");
+    ASSERT_EQ(cfg_int32_t, 65536124);
+
+    // not support
+    s = config::set_config("cfg_std_string", "test");
+    ASSERT_FALSE(s.ok());
+    ASSERT_EQ(s.to_string(), "Not supported: 'cfg_std_string' is not support to modify");
+    ASSERT_EQ(cfg_std_string, "doris_config_test_string");
+}
+
 } // namespace doris
 
 int main(int argc, char** argv) {
diff --git a/be/test/common/resource_tls_test.cpp b/be/test/common/resource_tls_test.cpp
index 567d324..e0e2250 100644
--- a/be/test/common/resource_tls_test.cpp
+++ b/be/test/common/resource_tls_test.cpp
@@ -20,6 +20,7 @@
 #include <gtest/gtest.h>
 
 #include "gen_cpp/Types_types.h"
+#include "common/configbase.h"
 #include "util/logging.h"
 
 namespace doris {
diff --git a/be/test/exprs/hybird_set_test.cpp b/be/test/exprs/hybird_set_test.cpp
index 53d1c32..f8a102c 100644
--- a/be/test/exprs/hybird_set_test.cpp
+++ b/be/test/exprs/hybird_set_test.cpp
@@ -19,6 +19,7 @@
 
 #include <string>
 #include <gtest/gtest.h>
+#include "common/configbase.h"
 #include "util/logging.h"
 
 namespace doris {
diff --git a/be/test/olap/bloom_filter_test.cpp b/be/test/olap/bloom_filter_test.cpp
index 42e689b..c77dff0 100644
--- a/be/test/olap/bloom_filter_test.cpp
+++ b/be/test/olap/bloom_filter_test.cpp
@@ -20,6 +20,7 @@
 #include <string>
 
 #include "olap/bloom_filter.hpp"
+#include "common/configbase.h"
 #include "util/logging.h"
 
 using std::string;
diff --git a/be/test/olap/byte_buffer_test.cpp b/be/test/olap/byte_buffer_test.cpp
index 80a9fb5..641b990 100755
--- a/be/test/olap/byte_buffer_test.cpp
+++ b/be/test/olap/byte_buffer_test.cpp
@@ -21,6 +21,7 @@
 #include "boost/filesystem.hpp"
 #include "olap/byte_buffer.h"
 #include "olap/file_helper.h"
+#include "common/configbase.h"
 #include "util/logging.h"
 
 namespace doris {
diff --git a/be/test/olap/file_helper_test.cpp b/be/test/olap/file_helper_test.cpp
index 90d8e46..667e7db 100644
--- a/be/test/olap/file_helper_test.cpp
+++ b/be/test/olap/file_helper_test.cpp
@@ -23,6 +23,7 @@
 #include "olap/olap_define.h"
 #include "olap/file_helper.h"
 #include "boost/filesystem.hpp"
+#include "common/configbase.h"
 #include "util/logging.h"
 
 #ifndef BE_TEST
diff --git a/be/test/olap/file_utils_test.cpp b/be/test/olap/file_utils_test.cpp
index 2c18f1d..0a17573 100644
--- a/be/test/olap/file_utils_test.cpp
+++ b/be/test/olap/file_utils_test.cpp
@@ -28,6 +28,7 @@
 #include "util/file_utils.h"
 #include "util/logging.h"
 #include "env/env.h"
+#include "common/configbase.h"
 
 #ifndef BE_TEST
 #define BE_TEST
diff --git a/be/test/runtime/large_int_value_test.cpp b/be/test/runtime/large_int_value_test.cpp
index ffbec1a..3777145 100644
--- a/be/test/runtime/large_int_value_test.cpp
+++ b/be/test/runtime/large_int_value_test.cpp
@@ -23,6 +23,7 @@
 #include <sstream>
 #include <gtest/gtest.h>
 
+#include "common/configbase.h"
 #include "common/logging.h"
 
 namespace doris {
diff --git a/be/test/util/cidr_test.cpp b/be/test/util/cidr_test.cpp
index 3db83d0..6b562d3 100644
--- a/be/test/util/cidr_test.cpp
+++ b/be/test/util/cidr_test.cpp
@@ -23,6 +23,7 @@
 
 #include <gtest/gtest.h>
 
+#include "common/configbase.h"
 #include "util/cpu_info.h"
 #include "util/logging.h"
 
diff --git a/be/test/util/filesystem_util_test.cpp b/be/test/util/filesystem_util_test.cpp
index ab38439..458f8a9 100644
--- a/be/test/util/filesystem_util_test.cpp
+++ b/be/test/util/filesystem_util_test.cpp
@@ -22,6 +22,7 @@
 #include <boost/filesystem.hpp>
 #include <gtest/gtest.h>
 
+#include "common/configbase.h"
 #include "util/logging.h"
 
 namespace doris {
diff --git a/be/test/util/internal_queue_test.cpp b/be/test/util/internal_queue_test.cpp
index 735c90c..ec82655 100644
--- a/be/test/util/internal_queue_test.cpp
+++ b/be/test/util/internal_queue_test.cpp
@@ -22,6 +22,7 @@
 #include <gtest/gtest.h>
 #include <unistd.h>
 
+#include "common/configbase.h"
 #include "util/logging.h"
 
 using std::vector;


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