Common Configuration
July 10, 2023About 27 min
Common Configuration
IoTDB common files for ConfigNode and DataNode are under conf.
iotdb-system.properties:IoTDB system configurations.
Effective
Different configuration parameters take effect in the following three ways:
- Only allowed to be modified in first start up: Can't be modified after first start, otherwise the ConfigNode/DataNode cannot start.
- After restarting system: Can be modified after the ConfigNode/DataNode first start, but take effect after restart.
- hot-load: Can be modified while the ConfigNode/DataNode is running, and trigger through sending the command(sql)
load configurationorset configuration 'key1' = 'value1'to the IoTDB server by client or session.
Configuration File
Replication Configuration
- config_node_consensus_protocol_class
| Name | config_node_consensus_protocol_class |
|---|---|
| Description | Consensus protocol of ConfigNode replicas, only support RatisConsensus |
| Type | String |
| Default | org.apache.iotdb.consensus.ratis.RatisConsensus |
| Effective | Only allowed to be modified in first start up |
- schema_replication_factor
| Name | schema_replication_factor |
|---|---|
| Description | Schema replication num |
| Type | int32 |
| Default | 1 |
| Effective | Take effect on new created Databases after restarting system |
- schema_region_consensus_protocol_class
| Name | schema_region_consensus_protocol_class |
|---|---|
| Description | Consensus protocol of schema replicas,larger than 1 replicas could only use RatisConsensus |
| Type | String |
| Default | org.apache.iotdb.consensus.ratis.RatisConsensus |
| Effective | Only allowed to be modified in first start up |
- data_replication_factor
| Name | data_replication_factor |
|---|---|
| Description | Data replication num |
| Type | int32 |
| Default | 1 |
| Effective | Take effect on new created Databases after restarting system |
- data_region_consensus_protocol_class
| Name | data_region_consensus_protocol_class |
|---|---|
| Description | Consensus protocol of data replicas,larger than 1 replicas could use IoTConsensus or RatisConsensus |
| Type | String |
| Default | org.apache.iotdb.consensus.simple.SimpleConsensus |
| Effective | Only allowed to be modified in first start up |
Load balancing Configuration
- series_partition_slot_num
| Name | series_slot_num |
|---|---|
| Description | Slot num of series partition |
| Type | int32 |
| Default | 10000 |
| Effective | Only allowed to be modified in first start up |
- series_partition_executor_class
| Name | series_partition_executor_class |
|---|---|
| Description | Series partition hash function |
| Type | String |
| Default | org.apache.iotdb.commons.partition.executor.hash.BKDRHashExecutor |
| Effective | Only allowed to be modified in first start up |
- schema_region_group_extension_policy
| Name | schema_region_group_extension_policy |
|---|---|
| Description | The extension policy of SchemaRegionGroup |
| Type | string |
| Default | AUTO |
| Effective | After restarting system |
- default_schema_region_group_num_per_database
| Name | default_schema_region_group_num_per_database |
|---|---|
| Description | The number of SchemaRegionGroups that each Database has when using the CUSTOM-SchemaRegionGroup extension policy. The least number of SchemaRegionGroups that each Database has when using the AUTO-SchemaRegionGroup extension policy. |
| Type | int |
| Default | 1 |
| Effective | After restarting system |
- schema_region_per_data_node
| Name | schema_region_per_data_node |
|---|---|
| Description | The maximum number of SchemaRegion expected to be managed by each DataNode |
| Type | double |
| Default | 1.0 |
| Effective | After restarting system |
- data_region_group_extension_policy
| Name | data_region_group_extension_policy |
|---|---|
| Description | The extension policy of DataRegionGroup |
| Type | string |
| Default | AUTO |
| Effective | After restarting system |
- default_data_region_group_num_per_database
| Name | default_data_region_group_num_per_database |
|---|---|
| Description | The number of DataRegionGroups that each Database has when using the CUSTOM-DataRegionGroup extension policy. The least number of DataRegionGroups that each Database has when using the AUTO-DataRegionGroup extension policy. |
| Type | int |
| Default | 1 |
| Effective | After restarting system |
- data_region_per_data_node
| Name | data_region_per_data_node |
|---|---|
| Description | The maximum number of DataRegion expected to be managed by each DataNode |
| Type | double |
| Default | Half of the CPU cores |
| Effective | After restarting system |
- enable_data_partition_inherit_policy
| Name | enable_data_partition_inherit_policy |
|---|---|
| Description | Whether to enable the DataPartition inherit policy |
| Type | Boolean |
| Default | false |
| Effective | After restarting system |
- leader_distribution_policy
| Name | leader_distribution_policy |
|---|---|
| Description | The policy of cluster RegionGroups' leader distribution |
| Type | String |
| Default | MIN_COST_FLOW |
| Effective | After restarting system |
- enable_auto_leader_balance_for_ratis
| Name | enable_auto_leader_balance_for_ratis_consensus |
|---|---|
| Description | Whether to enable auto leader balance for Ratis consensus protocol |
| Type | Boolean |
| Default | false |
| Effective | After restarting system |
- enable_auto_leader_balance_for_iot_consensus
| Name | enable_auto_leader_balance_for_iot_consensus |
|---|---|
| Description | Whether to enable auto leader balance for IoTConsensus protocol |
| Type | Boolean |
| Default | true |
| Effective | After restarting system |
Cluster Management
- cluster_name
| Name | cluster_name |
|---|---|
| Description | The name of cluster |
| Type | String |
| Default | default_cluster |
| Effective | Execute SQL in CLI: set configuration 'cluster_name'='xxx' (xxx is the new cluster name) |
| Attention | This change is distributed to each node through the network. In the event of network fluctuations or node downtime, it is not guaranteed that the modification will be successful on all nodes. Nodes that fail to modify will not be able to join the cluster upon restart. At this time, it is necessary to manually modify the cluster_name item in the configuration file of the node, and then restart. Under normal circumstances, it is not recommended to change the cluster name by manually modifying the configuration file, nor is it recommended to hot load through the load configuration method. |
- time_partition_interval
| Name | time_partition_interval |
|---|---|
| Description | Time partition interval of data when ConfigNode allocate data |
| Type | Long |
| Unit | ms |
| Default | 604800000 |
| Effective | Only allowed to be modified in first start up |
- heartbeat_interval_in_ms
| Name | heartbeat_interval_in_ms |
|---|---|
| Description | Heartbeat interval in the cluster nodes |
| Type | Long |
| Unit | ms |
| Default | 1000 |
| Effective | After restarting system |
- disk_space_warning_threshold
| Name | disk_space_warning_threshold |
|---|---|
| Description | Disk remaining threshold |
| Type | double(percentage) |
| Default | 0.05 |
| Effective | After restarting system |
Memory Control Configuration
- datanode_memory_proportion
| Name | datanode_memory_proportion |
|---|---|
| Description | Memory Allocation Ratio: StorageEngine, QueryEngine, SchemaEngine, Consensus, StreamingEngine, and Free Memory |
| Type | Ratio |
| Default | 3:3:1:1:1:1 |
| Effective | After restarting system |
- schema_memory_proportion
| Name | schema_memory_proportion |
|---|---|
| Description | Schema Memory Allocation Ratio: SchemaRegion, SchemaCache, and PartitionCache. |
| Type | Ratio |
| Default | 5:4:1 |
| Effective | After restarting system |
- storage_engine_memory_proportion
| Name | storage_engine_memory_proportion |
|---|---|
| Description | Memory allocation ratio in StorageEngine: Write, Compaction |
| Type | Ratio |
| Default | 8:2 |
| Effective | After restarting system |
- write_memory_proportion
| Name | write_memory_proportion |
|---|---|
| Description | Memory allocation ratio in writing: Memtable, TimePartitionInfo |
| Type | Ratio |
| Default | 19:1 |
| Effective | After restarting system |
- concurrent_writing_time_partition
| Name | concurrent_writing_time_partition |
|---|---|
| Description | This config decides how many time partitions in a database can be inserted concurrently For example, your partitionInterval is 86400 and you want to insert data in 5 different days, |
| Type | int32 |
| Default | 1 |
| Effective | After restarting system |
- primitive_array_size
| Name | primitive_array_size |
|---|---|
| Description | primitive array size (length of each array) in array pool |
| Type | Int32 |
| Default | 64 |
| Effective | After restart system |
- chunk_metadata_size_proportion
| Name | chunk_metadata_size_proportion |
|---|---|
| Description | size proportion for chunk metadata maintains in memory when writing tsfile |
| Type | Double |
| Default | 0.1 |
| Effective | After restart system |
- flush_proportion
| Name | flush_proportion |
|---|---|
| Description | Ratio of write memory for invoking flush disk, 0.4 by default If you have extremely high write load (like batch=1000), it can be set lower than the default value like 0.2 |
| Type | Double |
| Default | 0.4 |
| Effective | After restart system |
- buffered_arrays_memory_proportion
| Name | buffered_arrays_memory_proportion |
|---|---|
| Description | Ratio of write memory allocated for buffered arrays |
| Type | Double |
| Default | 0.6 |
| Effective | After restart system |
- reject_proportion
| Name | reject_proportion |
|---|---|
| Description | Ratio of write memory for rejecting insertion |
| Type | Double |
| Default | 0.8 |
| Effective | After restart system |
- write_memory_variation_report_proportion
| Name | write_memory_variation_report_proportion |
|---|---|
| Description | if memory cost of data region increased more than proportion of allocated memory for write, report to system |
| Type | Double |
| Default | 0.001 |
| Effective | After restarting system |
- check_period_when_insert_blocked
| Name | check_period_when_insert_blocked |
|---|---|
| Description | when an inserting is rejected, waiting period (in ms) to check system again |
| Type | Int32 |
| Default | 50 |
| Effective | After restart system |
- io_task_queue_size_for_flushing
| Name | io_task_queue_size_for_flushing |
|---|---|
| Description | size of ioTaskQueue. The default value is 10 |
| Type | Int32 |
| Default | 10 |
| Effective | After restart system |
- enable_query_memory_estimation
| Name | enable_query_memory_estimation |
|---|---|
| Description | If true, we will estimate each query's possible memory footprint before executing it and deny it if its estimated memory exceeds current free memory |
| Type | bool |
| Default | true |
| Effective | hot-load |
- partition_cache_size
| Name | partition_cache_size |
|---|---|
| Description | The max num of partition info record cached on DataNode. |
| Type | Int32 |
| Default | 1000 |
| Effective | After restarting system |
Schema Engine Configuration
- schema_engine_mode
| Name | schema_engine_mode |
|---|---|
| Description | Schema engine mode, supporting Memory and PBTree modes; PBTree mode support evict the timeseries schema temporarily not used in memory at runtime, and load it into memory from disk when needed. This parameter must be the same on all DataNodes in one cluster. |
| Type | string |
| Default | Memory |
| Effective | Only allowed to be modified in first start up |
- mlog_buffer_size
| Name | mlog_buffer_size |
|---|---|
| Description | size of log buffer in each metadata operation plan(in byte) |
| Type | int32 |
| Default | 1048576 |
| Effective | After restart system |
- sync_mlog_period_in_ms
| Name | sync_mlog_period_in_ms |
|---|---|
| Description | The cycle when metadata log is periodically forced to be written to disk(in milliseconds). If force_mlog_period_in_ms = 0 it means force metadata log to be written to disk after each refreshment |
| Type | Int64 |
| Default | 100 |
| Effective | After restarting system |
- tag_attribute_flush_interval
| Name | tag_attribute_flush_interval |
|---|---|
| Description | interval num for tag and attribute records when force flushing to disk. When a certain amount of tag and attribute records is reached, they will be force flushed to disk. It is possible to lose at most tag_attribute_flush_interval records |
| Type | int32 |
| Default | 1000 |
| Effective | Only allowed to be modified in first start up |
- tag_attribute_total_size
| Name | tag_attribute_total_size |
|---|---|
| Description | The maximum persistence size of tags and attributes of each time series. |
| Type | int32 |
| Default | 700 |
| Effective | Only allowed to be modified in first start up |
- schema_region_device_node_cache_size
| Name | schema_region_device_node_cache_size |
|---|---|
| Description | The max num of device node, used for speeding up device query, cached in schemaRegion. |
| Type | Int32 |
| Default | 10000 |
| Effective | After restarting system |
- max_measurement_num_of_internal_request
| Name | max_measurement_num_of_internal_request |
|---|---|
| Description | When there's too many measurements in one create timeseries plan, the plan will be split to several sub plan, with measurement num no more than this param. |
| Type | Int32 |
| Default | 10000 |
| Effective | After restarting system |
Configurations for creating schema automatically
- enable_auto_create_schema
| Name | enable_auto_create_schema |
|---|---|
| Description | whether auto create the time series when a non-existed time series data comes |
| Type | true or false |
| Default | true |
| Effective | After restarting system |
- default_storage_group_level
| Name | default_storage_group_level |
|---|---|
| Description | Database level when creating schema automatically is enabled. For example, if we receives a data point from root.sg0.d1.s2, we will set root.sg0 as the database if database level is 1. (root is level 0) |
| Type | integer |
| Default | 1 |
| Effective | After restarting system |
- boolean_string_infer_type
| Name | boolean_string_infer_type |
|---|---|
| Description | To which type the values "true" and "false" should be reslved |
| Type | BOOLEAN or TEXT |
| Default | BOOLEAN |
| Effective | After restarting system |
- integer_string_infer_type
| Name | integer_string_infer_type |
|---|---|
| Description | To which type an integer string like "67" in a query should be resolved |
| Type | INT32, INT64, DOUBLE, FLOAT or TEXT |
| Default | DOUBLE |
| Effective | After restarting system |
- floating_string_infer_type
| Name | floating_string_infer_type |
|---|---|
| Description | To which type a floating number string like "6.7" in a query should be resolved |
| Type | DOUBLE, FLOAT or TEXT |
| Default | DOUBLE |
| Effective | After restarting system |
- nan_string_infer_type
| Name | nan_string_infer_type |
|---|---|
| Description | To which type the value NaN in a query should be resolved |
| Type | DOUBLE, FLOAT or TEXT |
| Default | FLOAT |
| Effective | After restarting system |
Query Configurations
- read_consistency_level
| Name | mpp_data_exchange_core_pool_size |
|---|---|
| Description | The read consistency level, 1. strong(Default, read from the leader replica) 2. weak(Read from a random replica) |
| Type | string |
| Default | strong |
| Effective | After restarting system |
- meta_data_cache_enable
| Name | meta_data_cache_enable |
|---|---|
| Description | Whether to cache meta data(BloomFilter, ChunkMetadata and TimeSeriesMetadata) or not. |
| Type | Boolean |
| Default | true |
| Effective | After restarting system |
- chunk_timeseriesmeta_free_memory_proportion
| Name | chunk_timeseriesmeta_free_memory_proportion |
|---|---|
| Description | Read memory Allocation Ratio: BloomFilterCache : ChunkCache : TimeSeriesMetadataCache : Coordinator : Operators : DataExchange : timeIndex in TsFileResourceList : others. |
| Default | 1 : 100 : 200 : 300 : 400 |
| Effective | After restarting system |
- enable_last_cache
| Name | enable_last_cache |
|---|---|
| Description | Whether to enable LAST cache. |
| Type | Boolean |
| Default | true |
| Effective | After restarting system |
- max_deduplicated_path_num
| Name | max_deduplicated_path_num |
|---|---|
| Description | allowed max numbers of deduplicated path in one query. |
| Type | Int32 |
| Default | 1000 |
| Effective | After restarting system |
- mpp_data_exchange_core_pool_size
| Name | mpp_data_exchange_core_pool_size |
|---|---|
| Description | Core size of ThreadPool of MPP data exchange |
| Type | int32 |
| Default | 10 |
| Effective | After restarting system |
- mpp_data_exchange_max_pool_size
| Name | mpp_data_exchange_max_pool_size |
|---|---|
| Description | Max size of ThreadPool of MPP data exchange |
| Type | int32 |
| Default | 10 |
| Effective | After restarting system |
- mpp_data_exchange_keep_alive_time_in_ms
| Name | mpp_data_exchange_keep_alive_time_in_ms |
|---|---|
| Description | Max waiting time for MPP data exchange |
| Type | long |
| Default | 1000 |
| Effective | After restarting system |
- driver_task_execution_time_slice_in_ms
| Name | driver_task_execution_time_slice_in_ms |
|---|---|
| Description | Maximum execution time of a DriverTask |
| Type | int32 |
| Default | 100 |
| Effective | After restarting system |
- max_tsblock_size_in_bytes
| Name | max_tsblock_size_in_bytes |
|---|---|
| Description | Maximum capacity of a TsBlock |
| Type | int32 |
| Default | 1024 * 1024 (1 MB) |
| Effective | After restarting system |
- max_tsblock_line_numbers
| Name | max_tsblock_line_numbers |
|---|---|
| Description | Maximum number of lines in a single TsBlock |
| Type | int32 |
| Default | 1000 |
| Effective | After restarting system |
- slow_query_threshold
| Name | slow_query_threshold |
|---|---|
| Description | Time cost(ms) threshold for slow query. |
| Type | Int32 |
| Default | 30000 |
| Effective | Trigger |
- query_timeout_threshold
| Name | query_timeout_threshold |
|---|---|
| Description | The max executing time of query. unit: ms |
| Type | Int32 |
| Default | 60000 |
| Effective | After restarting system |
- max_allowed_concurrent_queries
| Name | max_allowed_concurrent_queries |
|---|---|
| Description | The maximum allowed concurrently executing queries. |
| Type | Int32 |
| Default | 1000 |
| Effective | After restarting system |
- query_thread_count
| Name | query_thread_count |
|---|---|
| Description | How many threads can concurrently execute query statement. When <= 0, use CPU core number. |
| Type | Int32 |
| Default | CPU core number |
| Effective | After restarting system |
- batch_size
| Name | batch_size |
|---|---|
| Description | The amount of data iterate each time in server (the number of data strips, that is, the number of different timestamps.) |
| Type | Int32 |
| Default | 100000 |
| Effective | After restarting system |
TTL Configuration
- ttl_check_interval
| Name | ttl_check_interval |
|---|---|
| Description | The interval of TTL check task in each database. Unit: ms. Default is 2 hours. |
| Type | int |
| Default | 7200000 |
| Effective | After restarting system |
- max_expired_time
| Name | max_expired_time |
|---|---|
| Description | If a file contains device that has expired for more than this duration, then the file will be settled immediately. Unit: ms. Default is 1 month. |
| Type | int |
| Default | 2592000000 |
| Effective | After restarting system |
- expired_data_ratio
| Name | expired_data_ratio |
|---|---|
| Description | The expired device ratio. If the ratio of expired devices in one file exceeds this value, then expired data of this file will be cleaned by compaction. |
| Type | float |
| Default | 0.3 |
| Effective | After restarting system |
Storage Engine Configuration
- timestamp_precision
| Name | timestamp_precision |
|---|---|
| Description | timestamp precision,support ms、us、ns |
| Type | String |
| Default | ms |
| Effective | Only allowed to be modified in first start up |
- tier_ttl_in_ms
| Name | tier_ttl_in_ms |
|---|---|
| Description | Define the maximum age of data for which each tier is responsible |
| Type | long |
| Default | -1 |
| Effective | After restarting system |
- max_waiting_time_when_insert_blocked
| Name | max_waiting_time_when_insert_blocked |
|---|---|
| Description | When the waiting time(in ms) of an inserting exceeds this, throw an exception |
| Type | Int32 |
| Default | 10000 |
| Effective | After restarting system |
- handle_system_error
| Name | handle_system_error |
|---|---|
| Description | What will the system do when unrecoverable error occurs |
| Type | String |
| Default | CHANGE_TO_READ_ONLY |
| Effective | After restarting system |
- write_memory_variation_report_proportion
| Name | write_memory_variation_report_proportion |
|---|---|
| Description | if memory cost of data region increased more than proportion of allocated memory for write, report to system |
| Type | Double |
| Default | 0.001 |
| Effective | After restarting system |
- enable_timed_flush_seq_memtable
| Name | enable_timed_flush_seq_memtable |
|---|---|
| Description | whether to enable timed flush sequence memtable |
| Type | Boolean |
| Default | true |
| Effective | hot-load |
- seq_memtable_flush_interval_in_ms
| Name | seq_memtable_flush_interval_in_ms |
|---|---|
| Description | if a memTable's created time is older than current time minus this, the memtable will be flushed to disk |
| Type | int32 |
| Default | 10800000 |
| Effective | hot-load |
- seq_memtable_flush_check_interval_in_ms
| Name | seq_memtable_flush_check_interval_in_ms |
|---|---|
| Description | the interval to check whether sequence memtables need flushing |
| Type | int32 |
| Default | 600000 |
| Effective | hot-load |
- enable_timed_flush_unseq_memtable
| Name | enable_timed_flush_unseq_memtable |
|---|---|
| Description | whether to enable timed flush unsequence memtable |
| Type | Boolean |
| Default | false |
| Effective | hot-load |
- unseq_memtable_flush_interval_in_ms
| Name | unseq_memtable_flush_interval_in_ms |
|---|---|
| Description | if a memTable's created time is older than current time minus this, the memtable will be flushed to disk |
| Type | int32 |
| Default | 600000 |
| Effective | hot-load |
- unseq_memtable_flush_check_interval_in_ms
| Name | unseq_memtable_flush_check_interval_in_ms |
|---|---|
| Description | the interval to check whether unsequence memtables need flushing |
| Type | int32 |
| Default | 30000 |
| Effective | hot-load |
- tvlist_sort_algorithm
| Name | tvlist_sort_algorithm |
|---|---|
| Description | the sort algorithm used in the memtable's TVList |
| Type | String |
| Default | TIM |
| Effective | After restarting system |
- avg_series_point_number_threshold
| Name | avg_series_point_number_threshold |
|---|---|
| Description | max average number of point of each series in memtable |
| Type | int32 |
| Default | 100000 |
| Effective | After restarting system |
- flush_thread_count
| Name | flush_thread_count |
|---|---|
| Description | The thread number used to perform the operation when IoTDB writes data in memory to disk. If the value is less than or equal to 0, then the number of CPU cores installed on the machine is used. The default is 0. |
| Type | int32 |
| Default | 0 |
| Effective | After restarting system |
- enable_partial_insert
| Name | enable_partial_insert |
|---|---|
| Description | Whether continue to write other measurements if some measurements are failed in one insertion. |
| Type | Boolean |
| Default | true |
| Effective | After restarting system |
- recovery_log_interval_in_ms
| Name | recovery_log_interval_in_ms |
|---|---|
| Description | the interval to log recover progress of each region when starting iotdb |
| Type | Int32 |
| Default | 5000 |
| Effective | After restarting system |
- 0.13_data_insert_adapt
| Name | 0.13_data_insert_adapt |
|---|---|
| Description | if using v0.13 client to insert data, set this configuration to true. |
| Type | Boolean |
| Default | false |
| Effective | After restarting system |
- device_path_cache_size
| Name | device_path_cache_size |
|---|---|
| Description | The max size of the device path cache. This cache is for avoiding initialize duplicated device id object in write process |
| Type | Int32 |
| Default | 500000 |
| Effective | After restarting system |
- insert_multi_tablet_enable_multithreading_column_threshold
| Name | insert_multi_tablet_enable_multithreading_column_threshold |
|---|---|
| Description | When the insert plan column count reaches the specified threshold, multi-threading is enabled. |
| Type | int32 |
| Default | 10 |
| Effective | After restarting system |
Compaction Configurations
- enable_seq_space_compaction
| Name | enable_seq_space_compaction |
|---|---|
| Description | enable the compaction between sequence files |
| Type | Boolean |
| Default | true |
| Effective | hot-load |
- enable_unseq_space_compaction
| Name | enable_unseq_space_compaction |
|---|---|
| Description | enable the compaction between unsequence files |
| Type | Boolean |
| Default | true |
| Effective | hot-load |
- enable_cross_space_compaction
| Name | enable_cross_space_compaction |
|---|---|
| Description | enable the compaction between sequence files and unsequence files |
| Type | Boolean |
| Default | true |
| Effective | hot-load |
- enable_auto_repair_compaction
| Name | enable_auto_repair_compaction |
|---|---|
| Description | enable auto repair unsorted file by compaction |
| Type | Boolean |
| Default | true |
| Effective | hot-load |
- cross_selector
| Name | cross_selector |
|---|---|
| Description | the task selector type of cross space compaction |
| Type | String |
| Default | rewrite |
| Effective | After restart system |
- cross_performer
| Name | cross_performer |
|---|---|
| Description | the task performer type of cross space compaction. The options are read_point and fast, read_point is the default and fast is still under test |
| Type | String |
| Default | read_point |
| Effective | After restart system |
- inner_seq_selector
| Name | inner_seq_selector |
|---|---|
| Description | the task selector type of inner sequence space compaction. Options: size_tiered_single_\target,size_tiered_multi_target |
| Type | String |
| Default | hot-load |
| Effective | hot-load |
- inner_seq_performer
| Name | inner_seq_peformer |
|---|---|
| Description | the task performer type of inner sequence space compaction. The options are read_chunk and fast, read_chunk is the default and fast is still under test |
| Type | String |
| Default | read_chunk |
| Effective | After restart system |
- inner_unseq_selector
| Name | inner_unseq_selector |
|---|---|
| Description | the task selector type of inner unsequence space compactionn. Options: size_tiered_single_\target,size_tiered_multi_target |
| Type | String |
| Default | hot-load |
| Effective | hot-load |
- inner_unseq_performer
| Name | inner_unseq_peformer |
|---|---|
| Description | the task performer type of inner unsequence space compaction. The options are read_point and fast, read_point is the default and fast is still under test |
| Type | String |
| Default | read_point |
| Effective | After restart system |
- compaction_priority
| Name | compaction_priority |
|---|---|
| Description | Priority of compaction task. When it is BALANCE, system executes all types of compaction equally; when it is INNER_CROSS, system takes precedence over executing inner space compaction task; when it is CROSS_INNER, system takes precedence over executing cross space compaction task |
| Type | String |
| Default | INNER_CROSS |
| Effective | After restart system |
- target_compaction_file_size
| Name | target_compaction_file_size |
|---|---|
| Description | The target file size in compaction |
| Type | Int64 |
| Default | 2147483648 |
| Effective | After restart system |
- target_chunk_size
| Name | target_chunk_size |
|---|---|
| Description | The target size of compacted chunk |
| Type | Int64 |
| Default | 1048576 |
| Effective | After restart system |
- target_chunk_point_num
| Name | target_chunk_point_num |
|---|---|
| Description | The target point number of compacted chunk |
| Type | int32 |
| Default | 100000 |
| Effective | After restart system |
- chunk_size_lower_bound_in_compaction
| Name | chunk_size_lower_bound_in_compaction |
|---|---|
| Description | A source chunk will be deserialized in compaction when its size is less than this value |
| Type | Int64 |
| Default | 10240 |
| Effective | After restart system |
- chunk_point_num_lower_bound_in_compaction
| Name | chunk_point_num_lower_bound_in_compaction |
|---|---|
| Description | A source chunk will be deserialized in compaction when its point num is less than this value |
| Type | int32 |
| Default | 1000 |
| Effective | After restart system |
- inner_compaction_total_file_num_threshold
| Name | inner_compaction_total_file_num_threshold |
|---|---|
| Description | The max num of files encounter in inner space compaction |
| Type | int32 |
| Default | 100 |
| Effective | hot-load |
- inner_compaction_total_file_size_threshold
| Name | inner_compaction_total_file_size_threshold |
|---|---|
| Description | The total file size limit in inner space compaction. Unit: byte |
| Type | int64 |
| Default | 10737418240 |
| Effective | hot-load |
- compaction_max_aligned_series_num_in_one_batch
| Name | compaction_max_aligned_series_num_in_one_batch |
|---|---|
| Description | How many value chunk will be compacted in aligned series compaction |
| Type | int32 |
| Default | 10 |
| Effective | hot-load |
- max_level_gap_in_inner_compaction
| Name | max_level_gap_in_inner_compaction |
|---|---|
| Description | The max level gap in inner compaction selection |
| Type | int32 |
| Default | 2 |
| Effective | hot-load |
- inner_compaction_candidate_file_num
| Name | inner_compaction_candidate_file_num |
|---|---|
| Description | The file num requirement when selecting inner space compaction candidate files |
| Type | int32 |
| Default | 30 |
| Effective | hot-load |
- max_cross_compaction_file_num
| Name | max_cross_compaction_candidate_file_num |
|---|---|
| Description | The max num of files encounter in cross space compaction |
| Type | int32 |
| Default | 500 |
| Effective | hot-load |
- max_cross_compaction_file_size
| Name | max_cross_compaction_candidate_file_size |
|---|---|
| Description | The max size of files encounter in cross space compaction |
| Type | Int64 |
| Default | 5368709120 |
| Effective | hot-load |
- compaction_thread_count
| Name | compaction_thread_count |
|---|---|
| Description | thread num to execute compaction |
| Type | int32 |
| Default | 10 |
| Effective | hot-load |
- compaction_schedule_interval_in_ms
| Name | compaction_schedule_interval_in_ms |
|---|---|
| Description | interval of scheduling compaction |
| Type | Int64 |
| Default | 60000 |
| Effective | After restart system |
- compaction_submission_interval_in_ms
| Name | compaction_submission_interval_in_ms |
|---|---|
| Description | interval of submitting compaction task |
| Type | Int64 |
| Default | 60000 |
| Effective | After restart system |
- compaction_write_throughput_mb_per_sec
| Name | compaction_write_throughput_mb_per_sec |
|---|---|
| Description | The write rate of all compaction tasks in MB/s, values less than or equal to 0 means no limit |
| Type | int32 |
| Default | 16 |
| Effective | hot-load |
- compaction_read_throughput_mb_per_sec
| Name | compaction_read_throughput_mb_per_sec |
|---|---|
| Description | The read rate of all compaction tasks in MB/s, values less than or equal to 0 means no limit |
| Type | int32 |
| Default | 0 |
| Effective | hot-load |
- compaction_read_operation_per_sec
| Name | compaction_read_operation_per_sec |
|---|---|
| Description | The read operation of all compaction tasks can reach per second, values less than or equal to 0 means no limit |
| Type | int32 |
| Default | 0 |
| Effective | hot-load |
- sub_compaction_thread_count
| Name | sub_compaction_thread_count |
|---|---|
| Description | the number of sub-compaction threads to accelerate cross space compaction |
| Type | Int32 |
| Default | 4 |
| Effective | hot-load |
- enable_tsfile_validation
| Name | enable_tsfile_validation |
|---|---|
| Description | Verify that TSfiles generated by Flush, Load, and Compaction are correct. |
| Type | boolean |
| Default | false |
| Effective | hot-load |
- candidate_compaction_task_queue_size
| Name | candidate_compaction_task_queue_size |
|---|---|
| Description | The size of candidate compaction task queue |
| Type | Int32 |
| Default | 50 |
| Effective | After restart system |
- compaction_schedule_thread_num
| Name | compaction_schedule_thread_num |
|---|---|
| Description | The number of threads to be set up to select compaction task. |
| Type | Int32 |
| Default | 4 |
| Effective | hot-load |
Write Ahead Log Configuration
- wal_mode
| Name | wal_mode |
|---|---|
| Description | The write mode of wal. For DISABLE mode, the system will disable wal. For SYNC mode, the system will submit wal synchronously, write request will not return until its wal is fsynced to the disk successfully. For ASYNC mode, the system will submit wal asynchronously, write request will return immediately no matter its wal is fsynced to the disk successfully. |
| Type | String |
| Default | ASYNC |
| Effective | After restart system |
- max_wal_nodes_num
| Name | max_wal_nodes_num |
|---|---|
| Description | Max number of wal nodes, each node corresponds to one wal directory. The default value 0 means the number is determined by the system. |
| Type | int32 |
| Default | 0 |
| Effective | After restart system |
- wal_async_mode_fsync_delay_in_ms
| Name | wal_async_mode_fsync_delay_in_ms |
|---|---|
| Description | Duration a wal flush operation will wait before calling fsync in the async mode |
| Type | int32 |
| Default | 1000 |
| Effective | hot-load |
- wal_sync_mode_fsync_delay_in_ms
| Name | wal_sync_mode_fsync_delay_in_ms |
|---|---|
| Description | Duration a wal flush operation will wait before calling fsync in the sync mode |
| Type | int32 |
| Default | 3 |
| Effective | hot-load |
- wal_buffer_size_in_byte
| Name | wal_buffer_size_in_byte |
|---|---|
| Description | Buffer size of each wal node |
| Type | int32 |
| Default | 33554432 |
| Effective | After restart system |
- wal_buffer_queue_capacity
| Name | wal_buffer_queue_capacity |
|---|---|
| Description | Blocking queue capacity of each wal buffer |
| Type | int32 |
| Default | 500 |
| Effective | After restart system |
- wal_file_size_threshold_in_byte
| Name | wal_file_size_threshold_in_byte |
|---|---|
| Description | Size threshold of each wal file |
| Type | int32 |
| Default | 31457280 |
| Effective | hot-load |
- wal_min_effective_info_ratio
| Name | wal_min_effective_info_ratio |
|---|---|
| Description | Minimum ratio of effective information in wal files |
| Type | double |
| Default | 0.1 |
| Effective | hot-load |
- wal_memtable_snapshot_threshold_in_byte
| Name | wal_memtable_snapshot_threshold_in_byte |
|---|---|
| Description | MemTable size threshold for triggering MemTable snapshot in wal |
| Type | int64 |
| Default | 8388608 |
| Effective | hot-load |
- max_wal_memtable_snapshot_num
| Name | max_wal_memtable_snapshot_num |
|---|---|
| Description | MemTable's max snapshot number in wal |
| Type | int32 |
| Default | 1 |
| Effective | hot-load |
- delete_wal_files_period_in_ms
| Name | delete_wal_files_period_in_ms |
|---|---|
| Description | The period when outdated wal files are periodically deleted |
| Type | int64 |
| Default | 20000 |
| Effective | hot-load |
TsFile Configurations
- group_size_in_byte
| Name | group_size_in_byte |
|---|---|
| Description | The data size written to the disk per time |
| Type | int32 |
| Default | 134217728 |
| Effective | hot-load |
- page_size_in_byte
| Name | page_size_in_byte |
|---|---|
| Description | The maximum size of a single page written in memory when each column in memory is written (in bytes) |
| Type | int32 |
| Default | 65536 |
| Effective | hot-load |
- max_number_of_points_in_page
| Name | max_number_of_points_in_page |
|---|---|
| Description | The maximum number of data points (timestamps - valued groups) contained in a page |
| Type | int32 |
| Default | 10000 |
| Effective | hot-load |
- pattern_matching_threshold
| Name | pattern_matching_threshold |
|---|---|
| Description | Max matching time of regex pattern |
| Type | int32 |
| Default | 1000000 |
| Effective | hot-load |
- max_degree_of_index_node
| Name | max_degree_of_index_node |
|---|---|
| Description | The maximum degree of the metadata index tree (that is, the max number of each node's children) |
| Type | int32 |
| Default | 256 |
| Effective | Only allowed to be modified in first start up |
- max_string_length
| Name | max_string_length |
|---|---|
| Description | The maximum length of a single string (number of character) |
| Type | int32 |
| Default | 128 |
| Effective | hot-load |
- value_encoder
| Name | value_encoder |
|---|---|
| Description | Encoding type of value column |
| Type | Enum String: “TS_2DIFF”,“PLAIN”,“RLE” |
| Default | PLAIN |
| Effective | hot-load |
- float_precision
| Name | float_precision |
|---|---|
| Description | The precision of the floating point number.(The number of digits after the decimal point) |
| Type | int32 |
| Default | The default is 2 digits. Note: The 32-bit floating point number has a decimal precision of 7 bits, and the 64-bit floating point number has a decimal precision of 15 bits. If the setting is out of the range, it will have no practical significance. |
| Effective | hot-load |
- compressor
| Name | compressor |
|---|---|
| Description | Data compression method; Time compression method in aligned timeseries |
| Type | Enum String : "UNCOMPRESSED", "SNAPPY", "LZ4", "ZSTD", "LZMA2" |
| Default | SNAPPY |
| Effective | hot-load |
- bloomFilterErrorRate
| Name | bloomFilterErrorRate |
|---|---|
| Description | The false positive rate of bloom filter in each TsFile. Bloom filter checks whether a given time series is in the tsfile before loading metadata. This can improve the performance of loading metadata and skip the tsfile that doesn't contain specified time series. If you want to learn more about its mechanism, you can refer to: wiki page of bloom filter. |
| Type | float, (0, 1) |
| Default | 0.05 |
| Effective | After restarting system |
Authorization Configuration
- authorizer_provider_class
| Name | authorizer_provider_class |
|---|---|
| Description | the class name of the authorization service |
| Type | String |
| Default | org.apache.iotdb.commons.auth.authorizer.LocalFileAuthorizer |
| Effective | After restarting system |
| Other available values | org.apache.iotdb.commons.auth.authorizer.OpenIdAuthorizer |
- openID_url
| Name | openID_url |
|---|---|
| Description | the openID server if OpenIdAuthorizer is enabled |
| Type | String (a http url) |
| Default | no |
| Effective | After restarting system |
- iotdb_server_encrypt_decrypt_provider
| Name | iotdb_server_encrypt_decrypt_provider |
|---|---|
| Description | The Class for user password encryption |
| Type | String |
| Default | org.apache.iotdb.commons.security.encrypt.MessageDigestEncrypt |
| Effective | Only allowed to be modified in first start up |
- iotdb_server_encrypt_decrypt_provider_parameter
| Name | iotdb_server_encrypt_decrypt_provider_parameter |
|---|---|
| Description | Parameters used to initialize the user password encryption class |
| Type | String |
| Default | 空 |
| Effective | After restarting system |
- author_cache_size
| Name | author_cache_size |
|---|---|
| Description | Cache size of user and role |
| Type | int32 |
| Default | 1000 |
| Effective | After restarting system |
- author_cache_expire_time
| Name | author_cache_expire_time |
|---|---|
| Description | Cache expire time of user and role, Unit: minutes |
| Type | int32 |
| Default | 30 |
| Effective | After restarting system |
UDF Configuration
- udf_initial_byte_array_length_for_memory_control
| Name | udf_initial_byte_array_length_for_memory_control |
|---|---|
| Description | Used to estimate the memory usage of text fields in a UDF query. It is recommended to set this value to be slightly larger than the average length of all texts. |
| Type | int32 |
| Default | 48 |
| Effective | After restarting system |
- udf_memory_budget_in_mb
| Name | udf_memory_budget_in_mb |
|---|---|
| Description | How much memory may be used in ONE UDF query (in MB). The upper limit is 20% of allocated memory for read. |
| Type | Float |
| Default | 30.0 |
| Effective | After restarting system |
- udf_reader_transformer_collector_memory_proportion
| Name | udf_reader_transformer_collector_memory_proportion |
|---|---|
| Description | UDF memory allocation ratio for reader, transformer and collector. The parameter form is a : b : c, where a, b, and c are integers. |
| Type | String |
| Default | 1:1:1 |
| Effective | After restarting system |
- udf_root_dir
| Name | udf_root_dir |
|---|---|
| Description | Root directory of UDF |
| Type | String |
| Default | ext/udf(Windows:ext\udf) |
| Effective | After restarting system |
- udf_lib_dir
| Name | udf_lib_dir |
|---|---|
| Description | UDF log and jar file dir |
| Type | String |
| Default | ext/udf(Windows:ext\udf) |
| Effective | After restarting system |
Trigger Configuration
- trigger_lib_dir
| Name | trigger_lib_dir |
|---|---|
| Description | Trigger JAR file dir |
| Type | String |
| Default | ext/trigger |
| Effective | After restarting system |
- stateful_trigger_retry_num_when_not_found
| Name | stateful_trigger_retry_num_when_not_found |
|---|---|
| Description | How many times we will retry to found an instance of stateful trigger on DataNodes |
| Type | Int32 |
| Default | 3 |
| Effective | After restarting system |
SELECT-INTO
- into_operation_buffer_size_in_byte
| Name | into_operation_buffer_size_in_byte |
|---|---|
| Description | When the select-into statement is executed, the maximum memory occupied by the data to be written (unit: Byte) |
| Type | int64 |
| Default | 100MB |
| Effective | hot-load |
- select_into_insert_tablet_plan_row_limit
| Name | select_into_insert_tablet_plan_row_limit |
|---|---|
| Description | The maximum number of rows that can be processed in insert-tablet-plan when executing select-into statements. When <= 0, use 10000. |
| Type | int32 |
| Default | 10000 |
| Effective | hot-load |
- into_operation_execution_thread_count
| Name | into_operation_execution_thread_count |
|---|---|
| Description | The number of threads in the thread pool that execute insert-tablet tasks |
| Type | int32 |
| Default | 2 |
| Effective | After restarting system |
Continuous Query
- continuous_query_execution_thread
| Name | continuous_query_execution_thread |
|---|---|
| Description | How many threads will be set up to perform continuous queries |
| Type | int32 |
| Default | max(1, the / 2) |
| Effective | After restarting system |
- continuous_query_min_every_interval
| Name | continuous_query_min_every_interval |
|---|---|
| Description | Minimum every interval to perform continuous query. |
| Type | duration |
| Default | 1s |
| Effective | After restarting system |
PIPE Configuration
- pipe_lib_dir
| Name | pipe_lib_dir |
|---|---|
| Description | Directory for storing custom Pipe plugins |
| Type | string |
| Default Value | ext/pipe |
| Effective | Not currently supported for modification |
- pipe_subtask_executor_max_thread_num
| Name | pipe_subtask_executor_max_thread_num |
|---|---|
| Description | The maximum number of threads that can be used for processors and sinks in Pipe subtasks. The actual value will be the minimum of pipe_subtask_executor_max_thread_num and the maximum of 1 and half of the CPU core count. |
| Type | int |
| Default Value | 5 |
| Effective | After restarting system |
- pipe_sink_timeout_ms
| Name | pipe_sink_timeout_ms |
|---|---|
| Description | The connection timeout for Thrift clients in milliseconds. |
| Type | int |
| Default Value | 900000 |
| Effective | After restarting system |
- pipe_sink_selector_number
| Name | pipe_sink_selector_number |
|---|---|
| Description | The maximum number of threads for processing execution results in the iotdb-thrift-async-sink plugin. It is recommended to set this value to be less than or equal to pipe_sink_max_client_number. |
| Type | int |
| Default Value | 4 |
| Effective | After restarting system |
- pipe_sink_max_client_number
| Name | pipe_sink_max_client_number |
|---|---|
| Description | The maximum number of clients that can be used in the iotdb-thrift-async-sink plugin. |
| Type | int |
| Default Value | 16 |
| Effective | After restarting system |
- pipe_air_gap_receiver_enabled
| Name | pipe_air_gap_receiver_enabled |
|---|---|
| Description | Whether to enable receiving Pipe data through a gateway. The receiver can only return 0 or 1 in TCP mode to indicate whether the data was successfully received. |
| Type | Boolean |
| Default Value | false |
| Effective | After restarting system |
- pipe_air_gap_receiver_port
| Name | pipe_air_gap_receiver_port |
|---|---|
| Description | The port used by the server to receive Pipe data through a gateway. |
| Type | int |
| Default Value | 9780 |
| Effective | After restarting system |
- pipe_all_sinks_rate_limit_bytes_per_second
| Name | pipe_all_sinks_rate_limit_bytes_per_second |
|---|---|
| Description | The total number of bytes per second that all Pipe sinks can transmit. When the given value is less than or equal to 0, it indicates there is no limit. The default value is -1, which means there is no limit. |
| Type | double |
| Default Value | -1 |
| Effective | Can be hot-loaded |
IOTConsensus Configuration
- data_region_iot_max_log_entries_num_per_batch
| Name | data_region_iot_max_log_entries_num_per_batch |
|---|---|
| Description | The maximum log entries num in IoTConsensus Batch |
| Type | int32 |
| Default | 1024 |
| Effective | After restarting system |
- data_region_iot_max_size_per_batch
| Name | data_region_iot_max_size_per_batch |
|---|---|
| Description | The maximum size in IoTConsensus Batch |
| Type | int32 |
| Default | 16MB |
| Effective | After restarting system |
- data_region_iot_max_pending_batches_num
| Name | data_region_iot_max_pending_batches_num |
|---|---|
| Description | The maximum pending batches num in IoTConsensus |
| Type | int32 |
| Default | 12 |
| Effective | After restarting system |
- data_region_iot_max_memory_ratio_for_queue
| Name | data_region_iot_max_memory_ratio_for_queue |
|---|---|
| Description | The maximum memory ratio for queue in IoTConsensus |
| Type | double |
| Default | 0.6 |
| Effective | After restarting system |
RatisConsensus Configuration
- config_node_ratis_log_appender_buffer_size_max
| Name | config_node_ratis_log_appender_buffer_size_max |
|---|---|
| Description | confignode max payload size for a single log-sync-RPC from leader to follower |
| Type | int32 |
| Default | 4MB |
| Effective | After restarting system |
- schema_region_ratis_log_appender_buffer_size_max
| Name | schema_region_ratis_log_appender_buffer_size_max |
|---|---|
| Description | schema region max payload size for a single log-sync-RPC from leader to follower |
| Type | int32 |
| Default | 4MB |
| Effective | After restarting system |
- data_region_ratis_log_appender_buffer_size_max
| Name | data_region_ratis_log_appender_buffer_size_max |
|---|---|
| Description | data region max payload size for a single log-sync-RPC from leader to follower |
| Type | int32 |
| Default | 4MB |
| Effective | After restarting system |
- config_node_ratis_snapshot_trigger_threshold
| Name | config_node_ratis_snapshot_trigger_threshold |
|---|---|
| Description | confignode trigger a snapshot when snapshot_trigger_threshold logs are written |
| Type | int32 |
| Default | 400,000 |
| Effective | After restarting system |
- schema_region_ratis_snapshot_trigger_threshold
| Name | schema_region_ratis_snapshot_trigger_threshold |
|---|---|
| Description | schema region trigger a snapshot when snapshot_trigger_threshold logs are written |
| Type | int32 |
| Default | 400,000 |
| Effective | After restarting system |
- data_region_ratis_snapshot_trigger_threshold
| Name | data_region_ratis_snapshot_trigger_threshold |
|---|---|
| Description | data region trigger a snapshot when snapshot_trigger_threshold logs are written |
| Type | int32 |
| Default | 400,000 |
| Effective | After restarting system |
- config_node_ratis_log_unsafe_flush_enable
| Name | config_node_ratis_log_unsafe_flush_enable |
|---|---|
| Description | confignode allows flushing Raft Log asynchronously |
| Type | boolean |
| Default | false |
| Effective | After restarting system |
- schema_region_ratis_log_unsafe_flush_enable
| Name | schema_region_ratis_log_unsafe_flush_enable |
|---|---|
| Description | schema region allows flushing Raft Log asynchronously |
| Type | boolean |
| Default | false |
| Effective | After restarting system |
- data_region_ratis_log_unsafe_flush_enable
| Name | data_region_ratis_log_unsafe_flush_enable |
|---|---|
| Description | data region allows flushing Raft Log asynchronously |
| Type | boolean |
| Default | false |
| Effective | After restarting system |
- config_node_ratis_log_segment_size_max_in_byte
| Name | config_node_ratis_log_segment_size_max_in_byte |
|---|---|
| Description | confignode max capacity of a single Log segment file |
| Type | int32 |
| Default | 24MB |
| Effective | After restarting system |
- schema_region_ratis_log_segment_size_max_in_byte
| Name | schema_region_ratis_log_segment_size_max_in_byte |
|---|---|
| Description | schema region max capacity of a single Log segment file |
| Type | int32 |
| Default | 24MB |
| Effective | After restarting system |
- data_region_ratis_log_segment_size_max_in_byte
| Name | data_region_ratis_log_segment_size_max_in_byte |
|---|---|
| Description | data region max capacity of a single Log segment file |
| Type | int32 |
| Default | 24MB |
| Effective | After restarting system |
- config_node_ratis_grpc_flow_control_window
| Name | config_node_ratis_grpc_flow_control_window |
|---|---|
| Description | confignode flow control window for ratis grpc log appender |
| Type | int32 |
| Default | 4MB |
| Effective | After restarting system |
- schema_region_ratis_grpc_flow_control_window
| Name | schema_region_ratis_grpc_flow_control_window |
|---|---|
| Description | schema region flow control window for ratis grpc log appender |
| Type | int32 |
| Default | 4MB |
| Effective | After restarting system |
- data_region_ratis_grpc_flow_control_window
| Name | data_region_ratis_grpc_flow_control_window |
|---|---|
| Description | data region flow control window for ratis grpc log appender |
| Type | int32 |
| Default | 4MB |
| Effective | After restarting system |
- config_node_ratis_grpc_leader_outstanding_appends_max
| Name | config_node_ratis_grpc_leader_outstanding_appends_max |
|---|---|
| Description | config node grpc pipeline concurrency threshold |
| Type | int32 |
| Default | 128 |
| Effective | After restarting system |
- schema_region_ratis_grpc_leader_outstanding_appends_max
| Name | schema_region_ratis_grpc_leader_outstanding_appends_max |
|---|---|
| Description | schema region grpc pipeline concurrency threshold |
| Type | int32 |
| Default | 128 |
| Effective | After restarting system |
- data_region_ratis_grpc_leader_outstanding_appends_max
| Name | data_region_ratis_grpc_leader_outstanding_appends_max |
|---|---|
| Description | data region grpc pipeline concurrency threshold |
| Type | int32 |
| Default | 128 |
| Effective | After restarting system |
- config_node_ratis_log_force_sync_num
| Name | config_node_ratis_log_force_sync_num |
|---|---|
| Description | config node fsync threshold |
| Type | int32 |
| Default | 128 |
| Effective | After restarting system |
- schema_region_ratis_log_force_sync_num
| Name | schema_region_ratis_log_force_sync_num |
|---|---|
| Description | schema region fsync threshold |
| Type | int32 |
| Default | 128 |
| Effective | After restarting system |
- data_region_ratis_log_force_sync_num
| Name | data_region_ratis_log_force_sync_num |
|---|---|
| Description | data region fsync threshold |
| Type | int32 |
| Default | 128 |
| Effective | After restarting system |
- config_node_ratis_rpc_leader_election_timeout_min_ms
| Name | config_node_ratis_rpc_leader_election_timeout_min_ms |
|---|---|
| Description | confignode min election timeout for leader election |
| Type | int32 |
| Default | 2000ms |
| Effective | After restarting system |
- schema_region_ratis_rpc_leader_election_timeout_min_ms
| Name | schema_region_ratis_rpc_leader_election_timeout_min_ms |
|---|---|
| Description | schema region min election timeout for leader election |
| Type | int32 |
| Default | 2000ms |
| Effective | After restarting system |
- data_region_ratis_rpc_leader_election_timeout_min_ms
| Name | data_region_ratis_rpc_leader_election_timeout_min_ms |
|---|---|
| Description | data region min election timeout for leader election |
| Type | int32 |
| Default | 2000ms |
| Effective | After restarting system |
- config_node_ratis_rpc_leader_election_timeout_max_ms
| Name | config_node_ratis_rpc_leader_election_timeout_max_ms |
|---|---|
| Description | confignode max election timeout for leader election |
| Type | int32 |
| Default | 2000ms |
| Effective | After restarting system |
- schema_region_ratis_rpc_leader_election_timeout_max_ms
| Name | schema_region_ratis_rpc_leader_election_timeout_max_ms |
|---|---|
| Description | schema region max election timeout for leader election |
| Type | int32 |
| Default | 2000ms |
| Effective | After restarting system |
- data_region_ratis_rpc_leader_election_timeout_max_ms
| Name | data_region_ratis_rpc_leader_election_timeout_max_ms |
|---|---|
| Description | data region max election timeout for leader election |
| Type | int32 |
| Default | 2000ms |
| Effective | After restarting system |
- config_node_ratis_request_timeout_ms
| Name | config_node_ratis_request_timeout_ms |
|---|---|
| Description | confignode ratis client retry threshold |
| Type | int32 |
| Default | 10s |
| Effective | After restarting system |
- schema_region_ratis_request_timeout_ms
| Name | schema_region_ratis_request_timeout_ms |
|---|---|
| Description | schema region ratis client retry threshold |
| Type | int32 |
| Default | 10s |
| Effective | After restarting system |
- data_region_ratis_request_timeout_ms
| Name | data_region_ratis_request_timeout_ms |
|---|---|
| Description | data region ratis client retry threshold |
| Type | int32 |
| Default | 10s |
| Effective | After restarting system |
- config_node_ratis_max_retry_attempts
| Name | config_node_ratis_max_retry_attempts |
|---|---|
| Description | confignode ratis client max retry attempts |
| Type | int32 |
| Default | 10 |
| Effective | After restarting system |
- config_node_ratis_initial_sleep_time_ms
| Name | config_node_ratis_initial_sleep_time_ms |
|---|---|
| Description | confignode ratis client retry initial sleep time |
| Type | int32 |
| Default | 100ms |
| Effective | After restarting system |
- config_node_ratis_max_sleep_time_ms
| Name | config_node_ratis_max_sleep_time_ms |
|---|---|
| Description | confignode ratis client retry max sleep time |
| Type | int32 |
| Default | 10s |
| Effective | After restarting system |
- schema_region_ratis_max_retry_attempts
| Name | schema_region_ratis_max_retry_attempts |
|---|---|
| Description | schema region ratis client max retry attempts |
| Type | int32 |
| Default | 10 |
| Effective | After restarting system |
- schema_region_ratis_initial_sleep_time_ms
| Name | schema_region_ratis_initial_sleep_time_ms |
|---|---|
| Description | schema region ratis client retry initial sleep time |
| Type | int32 |
| Default | 100ms |
| Effective | After restarting system |
- schema_region_ratis_max_sleep_time_ms
| Name | schema_region_ratis_max_sleep_time_ms |
|---|---|
| Description | schema region ratis client retry max sleep time |
| Type | int32 |
| Default | 10s |
| Effective | After restarting system |
- data_region_ratis_max_retry_attempts
| Name | data_region_ratis_max_retry_attempts |
|---|---|
| Description | data region ratis client max retry attempts |
| Type | int32 |
| Default | 10 |
| Effective | After restarting system |
- data_region_ratis_initial_sleep_time_ms
| Name | data_region_ratis_initial_sleep_time_ms |
|---|---|
| Description | data region ratis client retry initial sleep time |
| Type | int32 |
| Default | 100ms |
| Effective | After restarting system |
- data_region_ratis_max_sleep_time_ms
| Name | data_region_ratis_max_sleep_time_ms |
|---|---|
| Description | data region ratis client retry max sleep time |
| Type | int32 |
| Default | 10s |
| Effective | After restarting system |
- ratis_first_election_timeout_min_ms
| Name | ratis_first_election_timeout_min_ms |
|---|---|
| Description | minimal first election timeout for RatisConsensus |
| Type | int64 |
| Default | 50 (ms) |
| Effective | After restarting system |
- ratis_first_election_timeout_max_ms
| Name | ratis_first_election_timeout_max_ms |
|---|---|
| Description | maximal first election timeout for RatisConsensus |
| Type | int64 |
| Default | 150 (ms) |
| Effective | After restarting system |
- config_node_ratis_preserve_logs_num_when_purge
| Name | config_node_ratis_preserve_logs_num_when_purge |
|---|---|
| Description | confignode preserves certain logs when take snapshot and purge |
| Type | int32 |
| Default | 1000 |
| Effective | After restarting system |
- schema_region_ratis_preserve_logs_num_when_purge
| Name | schema_region_ratis_preserve_logs_num_when_purge |
|---|---|
| Description | schema region preserves certain logs when take snapshot and purge |
| Type | int32 |
| Default | 1000 |
| Effective | After restarting system |
- data_region_ratis_preserve_logs_num_when_purge
| Name | data_region_ratis_preserve_logs_num_when_purge |
|---|---|
| Description | data region preserves certain logs when take snapshot and purge |
| Type | int32 |
| Default | 1000 |
| Effective | After restarting system |
- config_node_ratis_log_max_size
| Name | config_node_ratis_log_max_size |
|---|---|
| Description | Max file size of in-disk Raft Log for config node |
| Type | int64 |
| Default | 2147483648 (2GB) |
| Effective | After restarting system |
- schema_region_ratis_log_max_size
| Name | schema_region_ratis_log_max_size |
|---|---|
| Description | Max file size of in-disk Raft Log for schema region |
| Type | int64 |
| Default | 2147483648 (2GB) |
| Effective | After restarting system |
- data_region_ratis_log_max_size
| Name | data_region_ratis_log_max_size |
|---|---|
| Description | Max file size of in-disk Raft Log for data region |
| Type | int64 |
| Default | 21474836480 (20GB) |
| Effective | After restarting system |
- config_node_ratis_periodic_snapshot_interval
| Name | config_node_ratis_periodic_snapshot_interval |
|---|---|
| Description | duration interval of config-node periodic snapshot |
| Type | int64 |
| Default | 86400 (seconds) |
| Effective | After restarting system |
- schema_region_ratis_periodic_snapshot_interval
| Name | schema_region_ratis_preserve_logs_num_when_purge |
|---|---|
| Description | duration interval of schema-region periodic snapshot |
| Type | int64 |
| Default | 86400 (seconds) |
| Effective | After restarting system |
- data_region_ratis_periodic_snapshot_interval
| Name | data_region_ratis_preserve_logs_num_when_purge |
|---|---|
| Description | duration interval of data-region periodic snapshot |
| Type | int64 |
| Default | 86400 (seconds) |
| Effective | After restarting system |
Procedure Configuration
- procedure_core_worker_thread_count
| Name | procedure_core_worker_thread_count |
|---|---|
| Description | The number of worker thread count |
| Type | int32 |
| Default | 4 |
| Effective | After restarting system |
- procedure_completed_clean_interval
| Name | procedure_completed_clean_interval |
|---|---|
| Description | Time interval of completed procedure cleaner work in |
| Type | int32 |
| Unit | second |
| Default | 30 |
| Effective | After restarting system |
- procedure_completed_evict_ttl
| Name | procedure_completed_evict_ttl |
|---|---|
| Description | The ttl of completed procedure |
| Type | int32 |
| Unit | second |
| Default | 800 |
| Effective | After restarting system |
MQTT Broker Configuration
- enable_mqtt_service
| Name | enable_mqtt_service。 |
|---|---|
| Description | Whether to enable the MQTT service |
| Type | Boolean |
| Default | False |
| Effective | hot-load |
- mqtt_host
| Name | mqtt_host |
|---|---|
| Description | The host to which the MQTT service is bound |
| Type | String |
| Default | 0.0.0.0 |
| Effective | hot-load |
- mqtt_port
| Name | mqtt_port |
|---|---|
| Description | The port to which the MQTT service is bound |
| Type | int32 |
| Default | 1883 |
| Effective | hot-load |
- mqtt_handler_pool_size
| Name | mqtt_handler_pool_size |
|---|---|
| Description | The size of the handler pool used to process MQTT messages |
| Type | int32 |
| Default | 1 |
| Effective | hot-load |
- mqtt_payload_formatter
| Name | mqtt_payload_formatter |
|---|---|
| Description | MQTT message payload formatter |
| Type | String |
| Default | JSON |
| Effective | hot-load |
- mqtt_max_message_size
| Name | mqtt_max_message_size |
|---|---|
| Description | Maximum length of MQTT message in bytes |
| Type | int32 |
| Default | 1048576 |
| Effective | hot-load |
TsFile Active Listening&Loading Function Configuration
- load_active_listening_enable
| Name | load_active_listening_enable |
|---|---|
| Description | Whether to enable the DataNode's active listening and loading of tsfile functionality (default is enabled). |
| Type | Boolean |
| Default | true |
| Effective | hot-load |
- load_active_listening_dirs
| Name | load_active_listening_dirs |
|---|---|
| Description | The directories to be listened to (automatically includes subdirectories of the directory), if there are multiple, separate with “,”. The default directory is ext/load/pending (supports hot loading). |
| Type | String |
| Default | ext/load/pending |
| Effective | hot-load |
- load_active_listening_fail_dir
| Name | load_active_listening_fail_dir |
|---|---|
| Description | The directory to which files are transferred after the execution of loading tsfile files fails, only one directory can be configured. |
| Type | String |
| Default | ext/load/failed |
| Effective | hot-load |
- load_active_listening_max_thread_num
| Name | load_active_listening_max_thread_num |
|---|---|
| Description | The maximum number of threads to perform loading tsfile tasks simultaneously. The default value when the parameter is commented out is max(1, CPU core count / 2). When the user sets a value not in the range [1, CPU core count / 2], it will be set to the default value (1, CPU core count / 2). |
| Type | Long |
| Default | max(1, CPU core count / 2) |
| Effective | Effective after restart |
- load_active_listening_check_interval_seconds
| Name | load_active_listening_check_interval_seconds |
|---|---|
| Description | Active listening polling interval in seconds. The function of actively listening to tsfile is achieved by polling the folder. This configuration specifies the time interval between two checks of load_active_listening_dirs, and the next check will be executed after load_active_listening_check_interval_seconds seconds of each check. When the user sets the polling interval to less than 1, it will be set to the default value of 5 seconds. |
| Type | Long |
| Default | 5 |
| Effective | Effective after restart |
