[Config] change some static config to dynamic config and delete some unused config (#5158)
* change some BE static config to dynamic config Co-authored-by: weizuo <weizuo@xiaomi.com>
This commit is contained in:
@ -108,24 +108,12 @@ CONF_mInt32(report_task_interval_seconds, "10");
|
||||
CONF_mInt32(report_disk_state_interval_seconds, "60");
|
||||
// the interval time(seconds) for agent report olap table to FE
|
||||
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");
|
||||
// the timeout(seconds) for make snapshot
|
||||
// CONF_Int32(make_snapshot_timeout_seconds, "600");
|
||||
// the timeout(seconds) for release snapshot
|
||||
// CONF_Int32(release_snapshot_timeout_seconds, "600");
|
||||
// the max download speed(KB/s)
|
||||
CONF_mInt32(max_download_speed_kbps, "50000");
|
||||
// download low speed limit(KB/s)
|
||||
CONF_mInt32(download_low_speed_limit_kbps, "50");
|
||||
// download low speed time(seconds)
|
||||
CONF_mInt32(download_low_speed_time, "300");
|
||||
// curl verbose mode
|
||||
// CONF_Int64(curl_verbose_mode, "1");
|
||||
// seconds to sleep for each time check table status
|
||||
// CONF_Int32(check_status_sleep_time_seconds, "10");
|
||||
// sleep time for one second
|
||||
CONF_Int32(sleep_one_second, "1");
|
||||
// sleep time for five seconds
|
||||
@ -150,15 +138,8 @@ CONF_String(log_buffer_level, "");
|
||||
// Pull load task dir
|
||||
CONF_String(pull_load_task_dir, "${DORIS_HOME}/var/pull_load");
|
||||
|
||||
// the maximum number of bytes to display on the debug webserver's log page
|
||||
CONF_Int64(web_log_bytes, "1048576");
|
||||
// number of threads available to serve backend execution requests
|
||||
CONF_Int32(be_service_threads, "64");
|
||||
// key=value pair of default query options for Doris, separated by ','
|
||||
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");
|
||||
|
||||
// cgroups allocated for doris
|
||||
CONF_String(doris_cgroups, "");
|
||||
@ -169,12 +150,8 @@ CONF_String(doris_cgroups, "");
|
||||
CONF_Int32(num_threads_per_core, "3");
|
||||
// if true, compresses tuple data in Serialize
|
||||
CONF_Bool(compress_rowbatches, "true");
|
||||
// serialize and deserialize each returned row batch
|
||||
CONF_Bool(serialize_batch, "false");
|
||||
// interval between profile reports; in seconds
|
||||
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
|
||||
CONF_Int32(doris_scanner_thread_pool_thread_num, "48");
|
||||
// number of olap scanner thread pool queue size
|
||||
@ -186,7 +163,7 @@ CONF_Int32(etl_thread_pool_queue_size, "256");
|
||||
// port on which to run Doris test backend
|
||||
CONF_Int32(port, "20001");
|
||||
// default thrift client connect timeout(in seconds)
|
||||
CONF_Int32(thrift_connect_timeout_seconds, "3");
|
||||
CONF_mInt32(thrift_connect_timeout_seconds, "3");
|
||||
// default thrift client retry interval (in milliseconds)
|
||||
CONF_mInt64(thrift_client_retry_interval_ms, "1000");
|
||||
// max row count number for single scan range
|
||||
@ -204,24 +181,14 @@ CONF_mInt32(max_pushdown_conditions_per_column, "1024");
|
||||
CONF_mInt32(doris_max_pushdown_conjuncts_return_rate, "90");
|
||||
// (Advanced) Maximum size of per-query receive-side buffer
|
||||
CONF_mInt32(exchg_node_buffer_size_bytes, "10485760");
|
||||
// insert sort threshold for sorter
|
||||
// CONF_Int32(insertion_threshold, "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_mInt32(push_write_mbytes_per_sec, "10");
|
||||
|
||||
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");
|
||||
// if set, saves the generated IR to the output file.
|
||||
//CONF_String(module_output, "");
|
||||
// memory_limitation_per_thread_for_schema_change unit GB
|
||||
CONF_mInt32(memory_limitation_per_thread_for_schema_change, "2");
|
||||
|
||||
// CONF_Int64(max_unpacked_row_block_size, "104857600");
|
||||
|
||||
CONF_mInt32(file_descriptor_cache_clean_interval, "3600");
|
||||
CONF_mInt32(disk_stat_monitor_interval, "5");
|
||||
CONF_mInt32(unused_rowset_monitor_interval, "30");
|
||||
@ -234,7 +201,6 @@ CONF_Bool(storage_strict_check_incompatible_old_format, "true");
|
||||
|
||||
// BE process will exit if the percentage of error disk reach this value.
|
||||
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");
|
||||
// pending data policy
|
||||
CONF_mInt32(pending_data_expire_time_sec, "1800");
|
||||
@ -256,7 +222,6 @@ CONF_Int32(file_descriptor_cache_capacity, "32768");
|
||||
// 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");
|
||||
|
||||
// Cache for storage page size
|
||||
CONF_String(storage_page_cache_limit, "20%");
|
||||
@ -272,8 +237,6 @@ CONF_mBool(disable_auto_compaction, "false");
|
||||
// check the configuration of auto compaction in seconds when auto compaction disabled
|
||||
CONF_mInt32(check_auto_compaction_interval_seconds, "5");
|
||||
|
||||
// CONF_Int64(base_compaction_start_hour, "20");
|
||||
// CONF_Int64(base_compaction_end_hour, "7");
|
||||
CONF_mInt64(base_compaction_num_cumulative_deltas, "5");
|
||||
CONF_mDouble(base_cumulative_delta_ratio, "0.3");
|
||||
CONF_mInt64(base_compaction_interval_seconds_since_last_operation, "86400");
|
||||
@ -304,7 +267,6 @@ CONF_mInt64(cumulative_size_based_compaction_lower_size_mbytes, "64");
|
||||
CONF_mInt64(min_cumulative_compaction_num_singleton_deltas, "5");
|
||||
CONF_mInt64(max_cumulative_compaction_num_singleton_deltas, "1000");
|
||||
CONF_mInt64(cumulative_compaction_budgeted_bytes, "104857600");
|
||||
// CONF_Int32(cumulative_compaction_write_mbytes_per_sec, "100");
|
||||
// cumulative compaction skips recently published deltas in order to prevent
|
||||
// compacting a version that might be queried (in case the query planning phase took some time).
|
||||
// the following config set the window size
|
||||
@ -365,15 +327,10 @@ CONF_mInt64(streaming_load_json_max_mb, "100");
|
||||
// the alive time of a TabletsChannel.
|
||||
// If the channel does not receive any data till this time,
|
||||
// the channel will be removed.
|
||||
CONF_Int32(streaming_load_rpc_max_alive_time_sec, "1200");
|
||||
CONF_mInt32(streaming_load_rpc_max_alive_time_sec, "1200");
|
||||
// the timeout of a rpc to open the tablet writer in remote BE.
|
||||
// short operation time, can set a short timeout
|
||||
CONF_Int32(tablet_writer_open_rpc_timeout_sec, "60");
|
||||
// Deprecated, use query_timeout instead
|
||||
// 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_mInt32(tablet_writer_open_rpc_timeout_sec, "60");
|
||||
// OlapTableSink sender's send interval, should be less than the real response time of a tablet writer rpc.
|
||||
CONF_mInt32(olap_table_sink_send_interval_ms, "1");
|
||||
|
||||
@ -382,9 +339,6 @@ CONF_Int32(fragment_pool_thread_num_min, "64");
|
||||
CONF_Int32(fragment_pool_thread_num_max, "512");
|
||||
CONF_Int32(fragment_pool_queue_size, "2048");
|
||||
|
||||
//for cast
|
||||
// CONF_Bool(cast, "true");
|
||||
|
||||
// Spill to disk when query
|
||||
// Writable scratch directories, split by ";"
|
||||
CONF_String(query_scratch_dirs, "${DORIS_HOME}");
|
||||
@ -427,7 +381,6 @@ CONF_Bool(enable_quadratic_probing, "false");
|
||||
CONF_String(pprof_profile_dir, "${DORIS_HOME}/log");
|
||||
|
||||
// for partition
|
||||
// CONF_Bool(enable_partitioned_hash_join, "false")
|
||||
CONF_Bool(enable_partitioned_aggregation, "true");
|
||||
|
||||
// to forward compatibility, will be removed later
|
||||
@ -442,23 +395,10 @@ CONF_mBool(enable_prefetch, "true");
|
||||
// Otherwise, Doris will use all cores returned from "/proc/cpuinfo".
|
||||
CONF_Int32(num_cores, "0");
|
||||
|
||||
// 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");
|
||||
|
||||
// When BE start, If there is a broken disk, BE process will exit by default.
|
||||
// Otherwise, we will ignore the broken disk,
|
||||
CONF_Bool(ignore_broken_disk, "false");
|
||||
|
||||
// Writable scratch directories
|
||||
CONF_String(scratch_dirs, "/tmp");
|
||||
|
||||
// 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");
|
||||
|
||||
// linux transparent huge page
|
||||
CONF_Bool(madvise_huge_pages, "false");
|
||||
|
||||
@ -516,10 +456,6 @@ CONF_mInt32(max_consumer_num_per_group, "3");
|
||||
// this should be larger than FE config 'max_concurrent_task_num_per_be' (default 5)
|
||||
CONF_Int32(routine_load_thread_pool_size, "10");
|
||||
|
||||
// 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");
|
||||
|
||||
// max external scan cache batch count, means cache max_memory_cache_batch_count * batch_size row
|
||||
// default is 20, batch_size's default value is 1024 means 20 * 1024 rows will be cached
|
||||
CONF_mInt32(max_memory_sink_batch_count, "20");
|
||||
@ -532,7 +468,7 @@ CONF_mInt32(scan_context_gc_interval_min, "5");
|
||||
CONF_String(es_scroll_keepalive, "5m");
|
||||
|
||||
// HTTP connection timeout for es
|
||||
CONF_Int32(es_http_timeout_ms, "5000");
|
||||
CONF_mInt32(es_http_timeout_ms, "5000");
|
||||
|
||||
// the max client cache number per each host
|
||||
// There are variety of client cache in BE, but currently we use the
|
||||
@ -544,7 +480,7 @@ CONF_Int32(max_client_cache_size_per_host, "10");
|
||||
CONF_String(small_file_dir, "${DORIS_HOME}/lib/small_file/");
|
||||
// path gc
|
||||
CONF_Bool(path_gc_check, "true");
|
||||
CONF_Int32(path_gc_check_interval_second, "86400");
|
||||
CONF_mInt32(path_gc_check_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");
|
||||
@ -593,7 +529,7 @@ CONF_Int32(txn_shard_size, "1024");
|
||||
CONF_Bool(ignore_load_tablet_failure, "false");
|
||||
|
||||
// Whether to continue to start be when load tablet from header failed.
|
||||
CONF_Bool(ignore_rowset_stale_unconsistent_delete, "false");
|
||||
CONF_mBool(ignore_rowset_stale_unconsistent_delete, "false");
|
||||
|
||||
// Soft memory limit as a fraction of hard memory limit.
|
||||
CONF_Double(soft_mem_limit_frac, "0.9");
|
||||
|
||||
@ -325,8 +325,6 @@ In some deployment environments, the `conf/` directory may be overwritten due to
|
||||
* Description: Configure how many rows of data are contained in a single RowBlock.
|
||||
* Default value: 1024
|
||||
|
||||
### `default_query_options`
|
||||
|
||||
### `default_rowset_type`
|
||||
|
||||
* Type: string
|
||||
@ -508,8 +506,6 @@ Indicates how many tablets in this data directory failed to load. At the same ti
|
||||
|
||||
### `load_process_max_memory_limit_percent`
|
||||
|
||||
### `local_library_dir`
|
||||
|
||||
### `log_buffer_level`
|
||||
|
||||
### `madvise_huge_pages`
|
||||
@ -689,15 +685,6 @@ Indicates how many tablets in this data directory failed to load. At the same ti
|
||||
|
||||
### `scan_context_gc_interval_min`
|
||||
|
||||
### `scratch_dirs`
|
||||
|
||||
### `serialize_batch`
|
||||
|
||||
### `sleep_five_seconds`
|
||||
+ Type: int32
|
||||
+ Description: Global variables, used for BE thread sleep for 5 seconds, should not be modified
|
||||
+ Default value: 5
|
||||
|
||||
### `sleep_one_second`
|
||||
|
||||
+ Type: int32
|
||||
@ -708,8 +695,6 @@ Indicates how many tablets in this data directory failed to load. At the same ti
|
||||
|
||||
### `snapshot_expire_time_sec`
|
||||
|
||||
### `sorter_block_size`
|
||||
|
||||
### `status_report_interval`
|
||||
|
||||
### `storage_flood_stage_left_capacity_bytes`
|
||||
@ -862,8 +847,6 @@ If the parameter is `THREAD_POOL`, the model is a blocking I/O model.
|
||||
|
||||
### `user_function_dir`
|
||||
|
||||
### `web_log_bytes`
|
||||
|
||||
### `webserver_num_workers`
|
||||
|
||||
### `webserver_port`
|
||||
|
||||
@ -322,8 +322,6 @@ tablet_score = compaction_tablet_scan_frequency_factor * tablet_scan_frequency +
|
||||
* 描述:配置单个RowBlock之中包含多少行的数据。
|
||||
* 默认值:1024
|
||||
|
||||
### `default_query_options`
|
||||
|
||||
### `default_rowset_type`
|
||||
* 类型:string
|
||||
* 描述:标识BE默认选择的存储格式,可配置的参数为:"**ALPHA**", "**BETA**"。主要起以下两个作用
|
||||
@ -509,8 +507,6 @@ load tablets from header failed, failed tablets size: xxx, path=xxx
|
||||
|
||||
### `load_process_max_memory_limit_percent`
|
||||
|
||||
### `local_library_dir`
|
||||
|
||||
### `log_buffer_level`
|
||||
|
||||
### `madvise_huge_pages`
|
||||
@ -690,15 +686,8 @@ load tablets from header failed, failed tablets size: xxx, path=xxx
|
||||
|
||||
### `scan_context_gc_interval_min`
|
||||
|
||||
### `scratch_dirs`
|
||||
|
||||
### `serialize_batch`
|
||||
|
||||
### `sleep_five_seconds`
|
||||
+ 类型:int32
|
||||
+ 描述:全局变量,用于BE线程休眠5秒,不应该被修改
|
||||
+ 默认值:5
|
||||
|
||||
### `sleep_one_second`
|
||||
+ 类型:int32
|
||||
+ 描述:全局变量,用于BE线程休眠1秒,不应该被修改
|
||||
@ -708,8 +697,6 @@ load tablets from header failed, failed tablets size: xxx, path=xxx
|
||||
|
||||
### `snapshot_expire_time_sec`
|
||||
|
||||
### `sorter_block_size`
|
||||
|
||||
### `status_report_interval`
|
||||
|
||||
### `storage_flood_stage_left_capacity_bytes`
|
||||
@ -861,8 +848,6 @@ Stream Load 一般适用于导入几个GB以内的数据,不适合导入过大
|
||||
|
||||
### `user_function_dir`
|
||||
|
||||
### `web_log_bytes`
|
||||
|
||||
### `webserver_num_workers`
|
||||
|
||||
### `webserver_port`
|
||||
|
||||
Reference in New Issue
Block a user