Common Configuration
IoTDB common files for ConfigNode and DataNode are under conf
.
iotdb-common.properties
:IoTDB common 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 configuration
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, SimpleConsensus could only be used in 1 replica,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 |
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, SimpleConsensus could only be used in 1 replica,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_processor
Name | data_region_per_processor |
---|
Description | The maximum number of DataRegion expected to be managed by each processor |
Type | double |
Default | 1.0 |
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
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 |
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
Name | enable_mem_control |
---|
Description | enable memory control to avoid OOM |
Type | Boolean |
Default | true |
Effective | After restarting system |
- storage_query_schema_consensus_free_memory_proportion
Name | storage_query_schema_consensus_free_memory_proportion |
---|
Description | Memory Allocation Ratio: StorageEngine, QueryEngine, SchemaEngine, Consensus and Free Memory |
Type | Ratio |
Default | 3:3:1:1:2 |
Effective | After restarting system |
- schema_memory_allocate_proportion
Name | schema_memory_allocate_proportion |
---|
Description | Schema Memory Allocation Ratio: SchemaRegion, SchemaCache, PartitionCache and LastCache |
Type | Ratio |
Default | 5:3:1: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 |
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 |
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 |
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 |
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 |
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 |
---|
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 |
Name | mlog_buffer_size |
---|
Description | size of log buffer in each metadata operation plan(in byte) |
Type | int32 |
Default | 1048576 |
Effective | After restart system |
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 |
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 | FLOAT |
Effective | After restarting system |
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
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 |
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 |
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 |
Name | max_tsblock_line_numbers |
---|
Description | Maximum number of lines in a single TsBlock |
Type | int32 |
Default | 1000 |
Effective | After restarting system |
Name | slow_query_threshold |
---|
Description | Time cost(ms) threshold for slow query. |
Type | Int32 |
Default | 30000 |
Effective | Trigger |
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 |
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 |
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 |
Storage Engine Configuration
Name | timestamp_precision |
---|
Description | timestamp precision,support ms、us、ns |
Type | String |
Default | ms |
Effective | Only allowed to be modified in first start up |
Name | default_ttl_in_ms |
---|
Description | Default ttl when each database created |
Type | Long |
Default | Infinity |
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 |
- enable_discard_out_of_order_data
Name | enable_discard_out_of_order_data |
---|
Description | whether to discard out of order data |
Type | Boolean |
Default | false |
Effective | After restarting system |
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 |
Name | memtable_size_threshold |
---|
Description | max memtable size |
Type | Long |
Default | 1073741824 |
Effective | when enable_mem_control is false & 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 | 10800000 |
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 | 600000 |
Effective | hot-load |
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 |
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 |
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 |
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 |
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 | false |
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 |
Name | cross_selector |
---|
Description | the task selector type of cross space compaction |
Type | String |
Default | rewrite |
Effective | After restart system |
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 |
Name | inner_seq_selector |
---|
Description | the task selector type of inner sequence space compaction |
Type | String |
Default | size_tiered |
Effective | After restart system |
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 |
Name | inner_unseq_selector |
---|
Description | the task selector type of inner unsequence space compaction |
Type | String |
Default | size_tiered |
Effective | After restart system |
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 |
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 | BALANCE |
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 |
Name | target_chunk_size |
---|
Description | The target size of compacted chunk |
Type | Int64 |
Default | 1048576 |
Effective | After restart system |
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 |
- max_inner_compaction_candidate_file_num
Name | max_inner_compaction_candidate_file_num |
---|
Description | The max num of files encounter in inner space compaction |
Type | int32 |
Default | 30 |
Effective | After restart system |
- 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 | After restart system |
- 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 | After restart system |
- cross_compaction_file_selection_time_budget
Name | cross_compaction_file_selection_time_budget |
---|
Description | Time budget for cross space compaction file selection |
Type | int32 |
Default | 30000 |
Effective | After restart system |
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 |
Type | int32 |
Default | 16 |
Effective | After restart system |
- 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 |
- compaction_validation_level
名字 | compaction_validation_level |
---|
Description | The level of validation after compaction. NONE: the validation after compaction is disabled. RESOURCE_ONLY: the validation after compaction check tsfile resource only. RESOURCE_AND_TSFILE: the validation after compaction check resource and file. |
Type | String |
Default | NONE |
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 |
Write Ahead Log Configuration
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 |
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 |
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
Name | group_size_in_byte |
---|
Description | The data size written to the disk per time |
Type | int32 |
Default | 134217728 |
Effective | hot-load |
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 |
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 |
Name | max_string_length |
---|
Description | The maximum length of a single string (number of character) |
Type | int32 |
Default | 128 |
Effective | hot-load |
Name | value_encoder |
---|
Description | Encoding type of value column |
Type | Enum String: “TS_2DIFF”,“PLAIN”,“RLE” |
Default | PLAIN |
Effective | hot-load |
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 |
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 |
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 filteropen in new window. |
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 |
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 |
Name | author_cache_size |
---|
Description | Cache size of user and role |
Type | int32 |
Default | 1000 |
Effective | After restarting system |
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 |
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 |
Name | udf_root_dir |
---|
Description | Root directory of UDF |
Type | String |
Default | ext/udf(Windows:ext\udf) |
Effective | After restarting system |
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
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
Name | ip_white_list |
---|
Description | Set the white list of IP addresses of the sender of the synchronization, which is expressed in the form of network segments, and multiple network segments are separated by commas. When the sender synchronizes data to the receiver, the receiver allows synchronization only when the IP address of the sender is within the network segment set in the white list. If the whitelist is empty, the receiver does not allow any sender to synchronize data. By default, the receiver rejects the synchronization request of all IP addresses except 127.0.0.1. When configuring this parameter, please ensure that all DataNode addresses on the sender are set. |
Type | String |
Default | 127.0.0.1/32 |
Effective | hot-load |
- max_number_of_sync_file_retry
Name | max_number_of_sync_file_retry |
---|
Description | The maximum number of retries when the sender fails to synchronize files to the receiver. |
Type | int32 |
Default | 5 |
Effective | hot-load |
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 |
- 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 |
- 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 |
- 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 |
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
Name | enable_mqtt_service。 |
---|
Description | Whether to enable the MQTT service |
Type | Boolean |
Default | False |
Effective | hot-load |
Name | mqtt_host |
---|
Description | The host to which the MQTT service is bound |
Type | String |
Default | 0.0.0.0 |
Effective | hot-load |
Name | mqtt_port |
---|
Description | The port to which the MQTT service is bound |
Type | int32 |
Default | 1883 |
Effective | hot-load |
Name | mqtt_handler_pool_size |
---|
Description | The size of the handler pool used to process MQTT messages |
Type | int32 |
Default | 1 |
Effective | hot-load |
Name | mqtt_payload_formatter |
---|
Description | MQTT message payload formatter |
Type | String |
Default | JSON |
Effective | hot-load |
Name | mqtt_max_message_size |
---|
Description | Maximum length of MQTT message in bytes |
Type | int32 |
Default | 1048576 |
Effective | hot-load |
REST Service Configuration
Name | enable_rest_service |
---|
Description | Whether to enable the Rest service |
Type | Boolean |
Default | false |
Effective | After restarting system |
Name | rest_service_port |
---|
Description | The Rest service listens to the port number |
Type | int32 |
Default | 18080 |
Effective | After restarting system |
Name | enable_swagger |
---|
Description | Whether to enable swagger to display rest interface information |
Type | Boolean |
Default | false |
Effective | After restarting system |
- rest_query_default_row_size_limit
Name | rest_query_default_row_size_limit |
---|
Description | The maximum number of rows in a result set that can be returned by a query |
Type | int32 |
Default | 10000 |
Effective | After restarting system |
Name | cache_expire |
---|
Description | Expiration time for caching customer login information |
Type | int32 |
Default | 28800 |
Effective | After restarting system |
Name | cache_max_num |
---|
Description | The maximum number of users stored in the cache |
Type | int32 |
Default | 100 |
Effective | After restarting system |
Name | cache_init_num |
---|
Description | Initial cache capacity |
Type | int32 |
Default | 10 |
Effective | After restarting system |
Name | cache_init_num |
---|
Description | REST Service Specifies whether to enable SSL configuration |
Type | Boolean |
Default | false |
Effective | After restarting system |
Name | key_store_path |
---|
Description | keyStore path (optional) |
Type | String |
Default | “” |
Effective | After restarting system |
Name | key_store_pwd |
---|
Description | keyStore Password (optional) |
Type | String |
Default | “” |
Effective | After restarting system |
Name | trust_store_path |
---|
Description | keyStore Password (optional) |
Type | String |
Default | “” |
Effective | After restarting system |
Name | trust_store_pwd |
---|
Description | trustStore Password (Optional) |
Type | String |
Default | “” |
Effective | After restarting system |
Name | idle_timeout |
---|
Description | SSL timeout duration, expressed in seconds |
Type | int32 |
Default | 5000 |
Effective | After restarting system |