M(UInt64,min_compress_block_size,65536,"The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.",0) \
M(UInt64,max_compress_block_size,1048576,"The maximum size of blocks of uncompressed data before compressing for writing to a table.",0) \
M(UInt64,max_block_size,DEFAULT_BLOCK_SIZE,"Maximum block size for reading",0) \
M(UInt64,max_insert_block_size,DEFAULT_INSERT_BLOCK_SIZE,"The maximum block size for insertion, if we control the creation of blocks for insertion.",0) \
M(UInt64,min_insert_block_size_rows,DEFAULT_INSERT_BLOCK_SIZE,"Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough.",0) \
M(UInt64,min_insert_block_size_bytes,(DEFAULT_INSERT_BLOCK_SIZE*256),"Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.",0) \
M(UInt64,min_insert_block_size_rows_for_materialized_views,0,"Like min_insert_block_size_rows, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_rows)",0) \
M(UInt64,min_insert_block_size_bytes_for_materialized_views,0,"Like min_insert_block_size_bytes, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_bytes)",0) \
M(UInt64,max_joined_block_size_rows,DEFAULT_BLOCK_SIZE,"Maximum block size for JOIN result (if join algorithm supports it). 0 means unlimited.",0) \
M(UInt64,max_insert_threads,0,"The maximum number of threads to execute the INSERT SELECT query. Values 0 or 1 means that INSERT SELECT is not run in parallel. Higher values will lead to higher memory usage. Parallel INSERT SELECT has effect only if the SELECT part is run on parallel, see 'max_threads' setting.",0) \
M(UInt64,max_insert_delayed_streams_for_parallel_write,0,"The maximum number of streams (columns) to delay final part flush. Default - auto (1000 in case of underlying storage supports parallel write, for example S3 and disabled otherwise)",0) \
M(UInt64,max_read_buffer_size,DBMS_DEFAULT_BUFFER_SIZE,"The maximum size of the buffer to read from the filesystem.",0) \
M(UInt64,max_distributed_connections,1024,"The maximum number of connections for distributed processing of one query (should be greater than max_threads).",0) \
M(UInt64,max_query_size,DBMS_DEFAULT_MAX_QUERY_SIZE,"Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)",0) \
M(Seconds,tcp_keep_alive_timeout,290/* less than DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC */,"The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes",0) \
M(Milliseconds,queue_max_wait_ms,0,"The wait time in the request queue, if the number of concurrent requests exceeds the maximum.",0) \
M(Milliseconds,connection_pool_max_wait_ms,0,"The wait time when the connection pool is full.",0) \
M(Milliseconds,replace_running_query_max_wait_ms,5000,"The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.",0) \
M(Milliseconds,kafka_max_wait_ms,5000,"The wait time for reading from Kafka before retry.",0) \
M(Milliseconds,rabbitmq_max_wait_ms,5000,"The wait time for reading from RabbitMQ before retry.",0) \
M(UInt64,poll_interval,DBMS_DEFAULT_POLL_INTERVAL,"Block at the query wait loop on the server for the specified number of seconds.",0) \
M(UInt64,idle_connection_timeout,3600,"Close idle TCP connections after specified number of seconds.",0) \
M(UInt64,connections_with_failover_max_tries,DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES,"The maximum number of attempts to connect to replicas.",0) \
M(UInt64,s3_min_upload_part_size,16*1024*1024,"The minimum size of part to upload during multipart upload to S3.",0) \
M(UInt64,s3_upload_part_size_multiply_factor,2,"Multiply s3_min_upload_part_size by this factor each time s3_multiply_parts_count_threshold parts were uploaded from a single write to S3.",0) \
M(UInt64,s3_upload_part_size_multiply_parts_count_threshold,1000,"Each time this number of parts was uploaded to S3 s3_min_upload_part_size multiplied by s3_upload_part_size_multiply_factor.",0) \
M(UInt64,max_replicated_fetches_network_bandwidth_for_server,0,"The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited. Only has meaning at server startup.",0) \
M(UInt64,max_replicated_sends_network_bandwidth_for_server,0,"The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited. Only has meaning at server startup.",0) \
M(UInt64,max_remote_read_network_bandwidth_for_server,0,"The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited. Only has meaning at server startup.",0) \
M(UInt64,max_remote_write_network_bandwidth_for_server,0,"The maximum speed of data exchange over the network in bytes per second for write. Zero means unlimited. Only has meaning at server startup.",0) \
M(Bool,stream_like_engine_allow_direct_select,false,"Allow direct SELECT query for Kafka, RabbitMQ, FileLog, Redis Streams and NATS engines. In case there are attached materialized views, SELECT query is not allowed even if this setting is enabled.",0) \
M(String,stream_like_engine_insert_queue,"","When stream like engine reads from multiple queues, user will need to select one queue to insert into when writing. Used by Redis Streams and NATS.",0) \
M(Milliseconds,distributed_directory_monitor_sleep_time_ms,100,"Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.",0) \
M(Milliseconds,distributed_directory_monitor_max_sleep_time_ms,30000,"Maximum sleep time for StorageDistributed DirectoryMonitors, it limits exponential growth too.",0) \
\
M(Bool,distributed_directory_monitor_batch_inserts,false,"Should StorageDistributed DirectoryMonitors try to batch individual inserts into bigger ones.",0) \
M(Bool,distributed_directory_monitor_split_batch_on_failure,false,"Should StorageDistributed DirectoryMonitors try to split batch into smaller in case of failures.",0) \
M(Bool,optimize_move_to_prewhere_if_final,false,"If query has `FINAL`, the optimization `move_to_prewhere` is not always correct and it is enabled only if both settings `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are turned on",0) \
M(UInt64,replication_alter_partitions_sync,1,"Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.",0) \
M(Int64,replication_wait_for_inactive_replica_timeout,120,"Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.",0) \
M(LoadBalancing,load_balancing,LoadBalancing::RANDOM,"Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.",0) \
M(TotalsMode,totals_mode,TotalsMode::AFTER_HAVING_EXCLUSIVE,"How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = ‘any’ are present.",IMPORTANT) \
M(Float,totals_auto_threshold,0.5,"The threshold for totals_mode = 'auto'.",0) \
M(Bool,allow_suspicious_low_cardinality_types,false,"In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.",0) \
M(UInt64,group_by_two_level_threshold_bytes,50000000,"From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.",0) \
M(UInt64,aggregation_memory_efficient_merge_threads,0,"Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.",0) \
M(UInt64,max_parallel_replicas,1,"The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.",0) \
M(Bool,allow_experimental_parallel_reading_from_replicas,false,"If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.",0) \
M(Bool,skip_unavailable_shards,false,"If true, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.",0) \
M(UInt64,parallel_distributed_insert_select,0,"Process distributed INSERT SELECT query in the same cluster on local tables on every shard; if set to 1 - SELECT is executed on each shard; if set to 2 - SELECT and INSERT are executed on each shard",0) \
M(UInt64,distributed_group_by_no_merge,0,"If 1, Do not merge aggregation states from different servers for distributed queries (shards will process query up to the Complete stage, initiator just proxies the data from the shards). If 2 the initiator will apply ORDER BY and LIMIT stages (it is not in case when shard process query up to the Complete stage)",0) \
M(UInt64,distributed_push_down_limit,1,"If 1, LIMIT will be applied on each shard separatelly. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT.",0) \
M(Bool,optimize_distributed_group_by_sharding_key,true,"Optimize GROUP BY sharding_key queries (by avoiding costly aggregation on the initiator server).",0) \
M(UInt64,optimize_skip_unused_shards_limit,1000,"Limit for number of sharding key values, turns off optimize_skip_unused_shards if the limit is reached",0) \
M(Bool,optimize_skip_unused_shards,false,"Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.",0) \
M(Bool,optimize_skip_unused_shards_rewrite_in,true,"Rewrite IN in query for remote shards to exclude values that does not belong to the shard (requires optimize_skip_unused_shards)",0) \
M(Bool,allow_nondeterministic_optimize_skip_unused_shards,false,"Allow non-deterministic functions (includes dictGet) in sharding_key for optimize_skip_unused_shards",0) \
M(UInt64,force_optimize_skip_unused_shards,0,"Throw an exception if unused shards cannot be skipped (1 - throw only if the table has the sharding key, 2 - always throw.",0) \
M(UInt64,optimize_skip_unused_shards_nesting,0,"Same as optimize_skip_unused_shards, but accept nesting level until which it will work.",0) \
M(UInt64,force_optimize_skip_unused_shards_nesting,0,"Same as force_optimize_skip_unused_shards, but accept nesting level until which it will work.",0) \
\
M(Bool,input_format_parallel_parsing,true,"Enable parallel parsing for some data formats.",0) \
M(UInt64,min_chunk_bytes_for_parallel_parsing,(10*1024*1024),"The minimum chunk size in bytes, which each thread will parse in parallel.",0) \
M(UInt64,merge_tree_min_rows_for_concurrent_read,(20*8192),"If at least as many lines are read from one file, the reading can be parallelized.",0) \
M(UInt64,merge_tree_min_bytes_for_concurrent_read,(24*10*1024*1024),"If at least as many bytes are read from one file, the reading can be parallelized.",0) \
M(UInt64,merge_tree_min_rows_for_seek,0,"You can skip reading more than that number of rows at the price of one seek per file.",0) \
M(UInt64,merge_tree_min_bytes_for_seek,0,"You can skip reading more than that number of bytes at the price of one seek per file.",0) \
M(UInt64,merge_tree_coarse_index_granularity,8,"If the index segment can contain the required keys, divide it into as many parts and recursively check them.",0) \
M(UInt64,merge_tree_max_rows_to_use_cache,(128*8192),"The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)",0) \
M(UInt64,merge_tree_max_bytes_to_use_cache,(192*10*1024*1024),"The maximum number of bytes per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)",0) \
M(UInt64,mysql_max_rows_to_insert,65536,"The maximum number of rows in MySQL batch insertion of the MySQL storage engine",0) \
\
M(UInt64,optimize_min_equality_disjunction_chain_length,3,"The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ",0) \
\
M(UInt64,min_bytes_to_use_direct_io,0,"The minimum number of bytes for reading the data with O_DIRECT option during SELECT queries execution. 0 - disabled.",0) \
M(UInt64,min_bytes_to_use_mmap_io,0,"The minimum number of bytes for reading the data with mmap option during SELECT queries execution. 0 - disabled.",0) \
M(Bool,checksum_on_read,true,"Validate checksums on reading. It is enabled by default and should be always enabled in production. Please do not expect any benefits in disabling this setting. It may only be used for experiments and benchmarks. The setting only applicable for tables of MergeTree family. Checksums are always validated for other table engines and when receiving data over network.",0) \
M(Bool,use_skip_indexes_if_final,false,"If query has FINAL, then skipping data based on indexes may produce incorrect result, hence disabled by default.",0) \
M(String,force_data_skipping_indices,"","Comma separated list of strings or literals with the name of the data skipping indices that should be used during query execution, otherwise an exception will be thrown.",0) \
M(Float,max_streams_to_max_threads_ratio,1,"Allows you to use more sources than the number of threads - to more evenly distribute work across threads. It is assumed that this is a temporary solution, since it will be possible in the future to make the number of sources equal to the number of threads, but for each source to dynamically select available work for itself.",0) \
M(Float,max_streams_multiplier_for_merge_tables,5,"Ask more streams when reading from Merge table. Streams will be spread across tables that Merge table will use. This allows more even distribution of work across threads and especially helpful when merged tables differ in size.",0) \
M(UInt64,priority,0,"Priority of the query. 1 - the highest, higher value - lower priority; 0 - do not use priorities.",0) \
M(Int64,os_thread_priority,0,"If non zero - set corresponding 'nice' value for query processing threads. Can be used to adjust query priority for OS scheduler.",0) \
M(LogQueriesType,log_queries_min_type,QueryLogElementType::QUERY_START,"Minimal type in query_log to log, possible values (from low to high): QUERY_START, QUERY_FINISH, EXCEPTION_BEFORE_START, EXCEPTION_WHILE_PROCESSING.",0) \
M(UInt64,log_queries_cut_to_length,100000,"If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.",0) \
M(DistributedProductMode,distributed_product_mode,DistributedProductMode::DENY,"How are distributed subqueries performed inside IN or JOIN sections?",IMPORTANT) \
M(Bool,insert_deduplicate,true,"For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be performed",0) \
M(UInt64,insert_quorum,0,"For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled.",0) \
M(UInt64,select_sequential_consistency,0,"For SELECT queries from the replicated table, throw an exception if the replica does not have a chunk written with the quorum; do not read the parts that have not yet been written with the quorum.",0) \
M(UInt64,table_function_remote_max_addresses,1000,"The maximum number of different shards and the maximum number of replicas of one shard in the `remote` function.",0) \
M(Milliseconds,read_backoff_min_latency_ms,1000,"Setting to reduce the number of threads in case of slow reads. Pay attention only to reads that took at least that much time.",0) \
M(UInt64,read_backoff_max_throughput,1048576,"Settings to reduce the number of threads in case of slow reads. Count events when the read bandwidth is less than that many bytes per second.",0) \
M(Milliseconds,read_backoff_min_interval_between_events_ms,1000,"Settings to reduce the number of threads in case of slow reads. Do not pay attention to the event, if the previous one has passed less than a certain amount of time.",0) \
M(UInt64,read_backoff_min_events,2,"Settings to reduce the number of threads in case of slow reads. The number of events after which the number of threads will be reduced.",0) \
M(Float,memory_tracker_fault_probability,0.,"For testing of `exception safety` - throw an exception every time you allocate memory with the specified probability.",0) \
M(Int64,http_zlib_compression_level,3,"Compression level - used if the client on HTTP said that it understands data compressed by gzip or deflate.",0) \
M(Bool,http_native_compression_disable_checksumming_on_decompress,false,"If you uncompress the POST data from the client compressed by the native format, do not check the checksum.",0) \
M(UInt64,max_http_get_redirects,0,"Max number of http GET redirects hops allowed. Make sure additional security measures are in place to prevent a malicious server to redirect your requests to unexpected services.",0) \
M(Bool,send_progress_in_http_headers,false,"Send progress notifications using X-ClickHouse-Progress headers. Some clients do not support high amount of HTTP headers (Python requests in particular), so it is disabled by default.",0) \
M(Bool,fsync_metadata,true,"Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.",0) \
M(Bool,join_use_nulls,false,"Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.",IMPORTANT) \
M(JoinStrictness,join_default_strictness,JoinStrictness::ALL,"Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.",0) \
M(Bool,any_join_distinct_right_table_keys,false,"Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.",IMPORTANT) \
M(UInt64,max_replica_delay_for_distributed_queries,300,"If set, distributed queries of Replicated tables will choose servers with replication delay in seconds less than the specified value (not inclusive). Zero means do not take delay into account.",0) \
M(Bool,fallback_to_stale_replicas_for_distributed_queries,true,"Suppose max_replica_delay_for_distributed_queries is set and all replicas for the queried table are stale. If this setting is enabled, the query will be performed anyway, otherwise the error will be reported.",0) \
M(UInt64,preferred_max_column_in_block_size_bytes,0,"Limit on max column size in block while reading. Helps to decrease cache misses count. Should be close to L2 cache size.",0) \
M(UInt64,parts_to_delay_insert,150,"If the destination table contains at least that many active parts in a single partition, artificially slow down insert into table.",0) \
M(UInt64,parts_to_throw_insert,300,"If more than this number active parts in a single partition of the destination table, throw 'Too many parts ...' exception.",0) \
M(Bool,insert_distributed_sync,false,"If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.",0) \
M(UInt64,insert_distributed_timeout,0,"Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.",0) \
M(Int64,distributed_ddl_task_timeout,180,"Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite. Zero means async mode.",0) \
M(Bool,optimize_throw_if_noop,false,"If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown",0) \
M(Bool,use_index_for_in_with_subqueries,true,"Try using an index if there is a subquery or a table expression on the right side of the IN operator.",0) \
M(Bool,joined_subquery_requires_alias,true,"Force joined subqueries and table functions to have aliases for correct name qualification.",0) \
M(Bool,empty_result_for_aggregation_by_empty_set,false,"Return empty result when aggregating without keys on empty set.",0) \
M(Bool,allow_experimental_codecs,false,"If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).",0) \
M(UInt64,query_profiler_real_time_period_ns,QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS,"Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.",0) \
M(UInt64,query_profiler_cpu_time_period_ns,QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS,"Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.",0) \
M(Bool,metrics_perf_events_enabled,false,"If enabled, some of the perf events will be measured throughout queries' execution.",0) \
M(String,metrics_perf_events_list,"","Comma separated list of perf metrics that will be measured throughout queries' execution. Empty means all events. See PerfEventInfo in sources for the available events.",0) \
M(Bool,prefer_global_in_and_join,false,"If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.",0) \
M(UInt64,max_rows_to_read,0,"Limit on read rows from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.",0) \
M(UInt64,max_bytes_to_read,0,"Limit on read bytes (after decompression) from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.",0) \
M(OverflowMode,read_overflow_mode,OverflowMode::THROW,"What to do when the limit is exceeded.",0) \
M(UInt64,max_rows_to_read_leaf,0,"Limit on read rows on the leaf nodes for distributed queries. Limit is applied for local reads only excluding the final merge stage on the root node.",0) \
M(UInt64,max_bytes_to_read_leaf,0,"Limit on read bytes (after decompression) on the leaf nodes for distributed queries. Limit is applied for local reads only excluding the final merge stage on the root node.",0) \
M(OverflowMode,read_overflow_mode_leaf,OverflowMode::THROW,"What to do when the leaf limit is exceeded.",0) \
M(OverflowMode,sort_overflow_mode,OverflowMode::THROW,"What to do when the limit is exceeded.",0) \
M(UInt64,max_bytes_before_external_sort,0,"",0) \
M(UInt64,max_bytes_before_remerge_sort,1000000000,"In case of ORDER BY with LIMIT, when memory usage is higher than specified threshold, perform additional steps of merging blocks before final merge to keep just top LIMIT rows.",0) \
M(UInt64,max_ast_depth,1000,"Maximum depth of query syntax tree. Checked after parsing.",0) \
M(UInt64,max_ast_elements,50000,"Maximum size of query syntax tree in number of nodes. Checked after parsing.",0) \
M(UInt64,max_expanded_ast_elements,500000,"Maximum size of query syntax tree in number of nodes after expansion of aliases and the asterisk.",0) \
\
M(UInt64,readonly,0,"0 - everything is allowed. 1 - only read requests. 2 - only read requests, as well as changing settings, except for the 'readonly' setting.",0) \
\
M(UInt64,max_rows_in_set,0,"Maximum size of the set (in number of elements) resulting from the execution of the IN section.",0) \
M(UInt64,max_bytes_in_set,0,"Maximum size of the set (in bytes in memory) resulting from the execution of the IN section.",0) \
M(OverflowMode,set_overflow_mode,OverflowMode::THROW,"What to do when the limit is exceeded.",0) \
\
M(UInt64,max_rows_in_join,0,"Maximum size of the hash table for JOIN (in number of rows).",0) \
M(UInt64,max_bytes_in_join,0,"Maximum size of the hash table for JOIN (in number of bytes in memory).",0) \
M(OverflowMode,join_overflow_mode,OverflowMode::THROW,"What to do when the limit is exceeded.",0) \
M(Bool,join_any_take_last_row,false,"When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.",IMPORTANT) \
M(UInt64,partial_merge_join_left_table_buffer_bytes,0,"If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.",0) \
M(UInt64,partial_merge_join_rows_in_right_blocks,65536,"Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.",0) \
M(UInt64,join_on_disk_max_files_to_merge,64,"For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.",0) \
M(String,temporary_files_codec,"LZ4","Set compression codec for temporary files (sort and join on disk). I.e. LZ4, NONE.",0) \
\
M(UInt64,max_rows_to_transfer,0,"Maximum size (in rows) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.",0) \
M(UInt64,max_bytes_to_transfer,0,"Maximum size (in uncompressed bytes) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.",0) \
M(OverflowMode,transfer_overflow_mode,OverflowMode::THROW,"What to do when the limit is exceeded.",0) \
\
M(UInt64,max_rows_in_distinct,0,"Maximum number of elements during execution of DISTINCT.",0) \
M(UInt64,max_bytes_in_distinct,0,"Maximum total size of state (in uncompressed bytes) in memory for the execution of DISTINCT.",0) \
M(OverflowMode,distinct_overflow_mode,OverflowMode::THROW,"What to do when the limit is exceeded.",0) \
\
M(UInt64,max_memory_usage,0,"Maximum memory usage for processing of single query. Zero means unlimited.",0) \
M(UInt64,memory_overcommit_ratio_denominator,1_GiB,"It represents soft memory limit on the user level. This value is used to compute query overcommit ratio.",0) \
M(UInt64,memory_overcommit_ratio_denominator_for_user,1_GiB,"It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.",0) \
M(UInt64,max_untracked_memory,(4*1024*1024),"Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.",0) \
M(UInt64,memory_profiler_step,(4*1024*1024),"Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.",0) \
M(Float,memory_profiler_sample_probability,0.,"Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.",0) \
M(UInt64,memory_usage_overcommit_max_wait_microseconds,5'000'000,"Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.",0) \
M(UInt64,max_network_bandwidth,0,"The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.",0) \
M(UInt64,max_network_bytes,0,"The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.",0) \
M(UInt64,max_network_bandwidth_for_user,0,"The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.",0)\
M(UInt64,max_network_bandwidth_for_all_users,0,"The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.",0) \
M(Bool,log_query_views,true,"Log query dependent views into system.query_views_log table. This setting have effect only when 'log_queries' is true.",0) \
M(String,log_comment,"","Log comment into system.query_log table and server log. It can be set to arbitrary string no longer than max_query_size.",0) \
M(LogsLevel,send_logs_level,LogsLevel::fatal,"Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'",0) \
M(UInt64,low_cardinality_max_dictionary_size,8192,"Maximum size (in rows) of shared global dictionary for LowCardinality type.",0) \
M(Bool,low_cardinality_use_single_dictionary_for_part,false,"LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.",0) \
M(Bool,decimal_check_overflow,true,"Check overflow of decimal arithmetic/comparison operations",0) \
M(Bool,prefer_localhost_replica,true,"If it's true then queries will be always sent to local replica (if it exists). If it's false then replica to send a query will be chosen between local and remote ones according to load_balancing",0) \
M(UInt64,max_fetch_partition_retries_count,5,"Amount of retries while fetching partition from another host.",0) \
M(UInt64,http_max_multipart_form_data_size,1024*1024*1024,"Limit on size of multipart/form-data content. This setting cannot be parsed from URL parameters and should be set in user profile. Note that content is parsed and external tables are created in memory before start of query execution. And this is the only limit that has effect on that stage (limits on max memory usage and max execution time have no effect while reading HTTP form data).",0) \
M(Bool,calculate_text_stack_trace,true,"Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when huge amount of wrong queries are executed. In normal cases you should not disable this option.",0) \
M(Bool,allow_ddl,true,"If it is set to true, then a user is allowed to executed DDL queries.",0) \
M(Bool,parallel_view_processing,false,"Enables pushing to attached views concurrently instead of sequentially.",0) \
M(Bool,enable_unaligned_array_join,false,"Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.",0) \
M(Bool,optimize_read_in_order,true,"Enable ORDER BY optimization for reading data in corresponding order in MergeTree tables.",0) \
M(Bool,optimize_read_in_window_order,true,"Enable ORDER BY optimization in window clause for reading data in corresponding order in MergeTree tables.",0) \
M(UInt64,aggregation_in_order_max_block_bytes,50000000,"Maximal size of block in bytes accumulated during aggregation in order of primary key. Lower block size allows to parallelize more final merge stage of aggregation.",0) \
M(UInt64,read_in_order_two_level_merge_threshold,100,"Minimal number of parts to read to run preliminary merge step during multithread reading in order of primary key.",0) \
M(Bool,low_cardinality_allow_in_native_format,true,"Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.",0) \
M(Bool,cancel_http_readonly_queries_on_client_close,false,"Cancel HTTP readonly queries when a client closes the connection without waiting for response.",0) \
M(Bool,external_table_functions_use_nulls,true,"If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql', 'postgresql' and 'odbc' table functions.",0) \
M(Bool,external_table_strict_query,false,"If it is set to true, transforming expression to local filter is forbidden for queries to external tables.",0) \
M(Bool,allow_hyperscan,true,"Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.",0) \
M(UInt64,max_hyperscan_regexp_total_length,0,"Max total length of all regexps than can be used in hyperscan multi-match functions (per every function). Zero means unlimited.",0) \
M(Bool,allow_simdjson,true,"Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.",0) \
M(Bool,allow_introspection_functions,false,"Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.",0) \
\
M(UInt64,max_partitions_per_insert_block,100,"Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.",0) \
M(UInt64,postgresql_connection_pool_wait_timeout,5000,"Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.",0) \
M(Seconds,distributed_replica_error_half_life,DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD,"Time period reduces replica error counter by 2 times.",0) \
M(UInt64,distributed_replica_error_cap,DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT,"Max number of errors per replica, prevents piling up an incredible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.",0) \
M(UInt64,distributed_replica_max_ignored_errors,0,"Number of errors that will be ignored while choosing replicas",0) \
\
M(Bool,allow_experimental_live_view,false,"Enable LIVE VIEW. Not mature enough.",0) \
M(UInt64,max_live_view_insert_blocks_before_refresh,64,"Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.",0) \
M(UInt64,min_free_disk_space_for_temporary_data,0,"The minimum disk space to keep while writing temporary data used in external sorting and aggregation.",0) \
M(Bool,show_table_uuid_in_table_create_query_if_not_nil,false,"For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.",0) \
M(Bool,database_atomic_wait_for_drop_and_detach_synchronously,false,"When executing DROP or DETACH TABLE in Atomic database, wait for table data to be finally dropped or detached.",0) \
M(Bool,enable_scalar_subquery_optimization,true,"If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.",0) \
M(Bool,optimize_trivial_count_query,true,"Process trivial 'SELECT count() FROM table' query from metadata.",0) \
M(Bool,optimize_respect_aliases,true,"If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count",0) \
M(UInt64,mutations_sync,0,"Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.",0) \
M(Bool,optimize_or_like_chain,false,"Optimize multiple OR LIKE into multiMatchAny. This optimization should not be enabled by default, because it defies index analysis in some cases.",0) \
M(Bool,optimize_arithmetic_operations_in_aggregate_functions,true,"Move arithmetic operations out of aggregation functions",0) \
M(Bool,optimize_duplicate_order_by_and_distinct,true,"Remove duplicate ORDER BY and DISTINCT if it's possible",0) \
M(Bool,optimize_redundant_functions_in_order_by,true,"Remove functions from ORDER BY if its argument is also in ORDER BY",0) \
M(Bool,optimize_if_chain_to_multiif,false,"Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.",0) \
M(Bool,optimize_if_transform_strings_to_enum,false,"Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.",0) \
M(Bool,optimize_monotonous_functions_in_order_by,true,"Replace monotonous function with its argument in ORDER BY",0) \
M(Bool,optimize_functions_to_subcolumns,false,"Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ",0) \
M(Bool,allow_experimental_alter_materialized_view_structure,false,"Allow atomic alter on Materialized views. Work in progress.",0) \
M(Bool,enable_early_constant_folding,true,"Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there",0) \
M(Bool,deduplicate_blocks_in_dependent_materialized_views,false,"Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.",0) \
M(Bool,validate_polygons,true,"Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.",0) \
M(UInt64,max_parser_depth,DBMS_DEFAULT_MAX_PARSER_DEPTH,"Maximum parser depth (recursion depth of recursive descend parser).",0) \
M(Bool,allow_settings_after_format_in_insert,false,"Allow SETTINGS after FORMAT, but note, that this is not always safe (note: this is a compatibility setting).",0) \
M(Seconds,periodic_live_view_refresh,DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC,"Interval after which periodically refreshed live view is forced to refresh.",0) \
M(Bool,cast_ipv4_ipv6_default_on_conversion_error,false,"CAST operator into IPv4, CAST operator into IPV6 type, toIPv4, toIPv6 functions will return default value instead of throwing exception on conversion error.",0) \
M(MySQLDataTypesSupport,mysql_datatypes_support_level,0,"Which MySQL types should be converted to corresponding ClickHouse types (rather than being represented as String). Can be empty or any combination of 'decimal', 'datetime64', 'date2Date32' or 'date2String'. When empty MySQL's DECIMAL and DATETIME/TIMESTAMP with non-zero precision are seen as String on ClickHouse's side.",0) \
M(Bool,optimize_syntax_fuse_functions,false,"Not ready for production, do not use. Allow apply syntax optimisation: fuse aggregate functions",0) \
M(Bool,optimize_fuse_sum_count_avg,false,"Not ready for production, do not use. Fuse functions `sum, avg, count` with identical arguments into one `sumCount` (`optimize_syntax_fuse_functions should be enabled)",0) \
M(UInt64,insert_shard_id,0,"If non zero, when insert into a distributed table, the data will be inserted into the shard `insert_shard_id` synchronously. Possible values range from 1 to `shards_number` of corresponding distributed table",0) \
M(Bool,collect_hash_table_stats_during_aggregation,true,"Enable collecting hash table statistics to optimize memory allocation",0) \
M(UInt64,max_entries_for_hash_table_stats,10'000,"How many entries hash table statistics collected during aggregation is allowed to have",0) \
M(UInt64,max_size_to_preallocate_for_aggregation,10'000'000,"For how many elements it is allowed to preallocate space in all hash tables in total before aggregation",0) \
M(Bool,engine_file_allow_create_multiple_files,false,"Enables or disables creating a new file on each insert in file engine tables if format has suffix.",0) \
M(Bool,allow_experimental_database_replicated,false,"Allow to create databases with Replicated engine",0) \
M(UInt64,database_replicated_initial_query_timeout_sec,300,"How long initial DDL query should wait for Replicated database to precess previous DDL queue entries",0) \
M(UInt64,external_storage_max_read_rows,0,"Limit maximum number of rows when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled",0) \
M(UInt64,external_storage_max_read_bytes,0,"Limit maximum number of bytes when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled",0) \
M(UnionMode,union_default_mode,UnionMode::Unspecified,"Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.",0) \
M(Bool,legacy_column_name_of_tuple_literal,false,"List all names of element of large tuple literals in their column names instead of hash. This settings exists only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher.",0) \
M(UInt64,query_plan_max_optimizations_to_apply,10000,"Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception",0) \
M(UInt64,regexp_max_matches_per_row,1000,"Max matches of any single regexp per row, used to safeguard 'extractAllGroupsHorizontal' against consuming too much memory with greedy RE.",0) \
M(UInt64,function_range_max_elements_in_block,500000000,"Maximum number of values generated by function 'range' per block of data (sum of array sizes for every row in a block, see also 'max_block_size' and 'min_insert_block_size_rows'). It is a safety threshold.",0) \
M(ShortCircuitFunctionEvaluation,short_circuit_function_evaluation,ShortCircuitFunctionEvaluation::ENABLE,"Setting for short-circuit function evaluation configuration. Possible values: 'enable' - use short-circuit function evaluation for functions that are suitable for it, 'disable' - disable short-circuit function evaluation, 'force_enable' - use short-circuit function evaluation for all functions.",0) \
M(String,local_filesystem_read_method,"pread_threadpool","Method of reading data from local filesystem, one of: read, pread, mmap, pread_threadpool.",0) \
M(UInt64,merge_tree_min_rows_for_concurrent_read_for_remote_filesystem,(20*8192),"If at least as many lines are read from one file, the reading can be parallelized, when reading from remote filesystem.",0) \
M(UInt64,merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem,(24*10*1024*1024),"If at least as many bytes are read from one file, the reading can be parallelized, when reading from remote filesystem.",0) \
M(UInt64,remote_read_min_bytes_for_seek,4*DBMS_DEFAULT_BUFFER_SIZE,"Min bytes required for remote read (url, s3) to do seek, instead for read with ignore.",0) \
M(UInt64,async_insert_threads,16,"Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled",0) \
M(Bool,async_insert,false,"If true, data from INSERT query is stored in queue and later flushed to table in background. Makes sense only for inserts via HTTP protocol. If wait_for_async_insert is false, INSERT query is processed almost instantly, otherwise client will wait until data will be flushed to table",0) \
M(Bool,wait_for_async_insert,true,"If true wait for processing of asynchronous insertion",0) \
M(Seconds,wait_for_async_insert_timeout,DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC,"Timeout for waiting for processing asynchronous insertion",0) \
M(UInt64,async_insert_max_data_size,100000,"Maximum size in bytes of unparsed data collected per query before being inserted",0) \
M(Milliseconds,async_insert_busy_timeout_ms,200,"Maximum time to wait before dumping collected data per query since the first data appeared",0) \
M(Milliseconds,async_insert_stale_timeout_ms,0,"Maximum time to wait before dumping collected data per query since the last data appeared. Zero means no timeout at all",0) \
M(Bool,enable_filesystem_cache,true,"Use cache for remote filesystem. This setting does not turn on/off cache for disks (must be done via disk config), but allows to bypass cache for some queries if intended",0) \
M(UInt64,filesystem_cache_max_wait_sec,5,"Allow to wait at most this number of seconds for download of current remote_fs_buffer_size bytes, and skip cache if exceeded",0) \
M(Bool,enable_filesystem_cache_on_write_operations,false,"Write into cache on write operations. To actually work this setting requires be added to disk config too",0) \
M(Bool,force_remove_data_recursively_on_drop,false,"Recursively remove data on DROP query. Avoids 'Directory not empty' error, but may silently remove detached data",0) \
M(Bool,allow_unrestricted_reads_from_keeper,false,"Allow unrestricted (without condition on path) reads from system.zookeeper table, can be handy, but is not safe for zookeeper",0) \
M(Bool,schema_inference_cache_require_modification_time_for_url,true,"Use schema from cache for URL with last modification time validation (for urls with Last-Modified header)",0) \
M(String,compatibility,"","Changes other settings according to provided ClickHouse version. If we know that we changed some behaviour in ClickHouse by changing some settings in some version, this compatibility setting will control these settings",0) \
M(Map,additional_table_filters,"","Additional filter expression which would be applied after reading from specified table. Syntax: {'table1': 'expression', 'database.table2': 'expression'}",0) \
M(TransactionsWaitCSNMode,wait_changes_become_visible_after_commit_mode,TransactionsWaitCSNMode::WAIT_UNKNOWN,"Wait for committed changes to become actually visible in the latest snapshot",0) \
M(Bool,implicit_transaction,false,"If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)",0) \
M(Bool,compatibility_ignore_auto_increment_in_create_table,false,"Ignore AUTO_INCREMENT keyword in column declaration if true, otherwise return error. It simplifies migration from MySQL",0) \
M(Bool,optimize_distinct_in_order,true,"Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement",0) \
M(Char,format_csv_delimiter,',',"The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.",0) \
M(Bool,input_format_csv_arrays_as_nested_csv,false,R"(When reading Array from CSV, expect that its elements were serialized in nested CSV and then put into string. Example: "[""Hello"",""world"",""42""""TV""]". Braces around array can be omitted.)",0) \
M(Bool,input_format_skip_unknown_fields,true,"Skip columns with unknown names from input data (it works for JSONEachRow, -WithNames, -WithNamesAndTypes and TSKV formats).",0) \
M(Bool,input_format_with_names_use_header,true,"For -WithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.",0) \
M(Bool,input_format_with_types_use_header,true,"For -WithNamesAndTypes input formats this controls whether format parser should check if data types from the input match data types from the header.",0) \
M(Bool,input_format_defaults_for_omitted_fields,true,"For input data calculate default expressions for omitted fields (it works for JSONEachRow, -WithNames, -WithNamesAndTypes formats).",IMPORTANT) \
M(Bool,input_format_null_as_default,true,"For text input formats initialize null fields with default values if data type of this field is not nullable",0) \
M(UInt64,input_format_msgpack_number_of_columns,0,"The number of columns in inserted MsgPack data. Used for automatic schema inference from data.",0) \
M(MsgPackUUIDRepresentation,output_format_msgpack_uuid_representation,FormatSettings::MsgPackUUIDRepresentation::EXT,"The way how to output UUID in MsgPack format.",0) \
M(Bool,input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference,false,"Skip columns with unsupported types while schema inference for format Parquet",0) \
M(Bool,input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference,false,"Skip fields with unsupported types while schema inference for format Protobuf",0) \
M(Bool,input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference,false,"Skip columns with unsupported types while schema inference for format CapnProto",0) \
M(Bool,input_format_orc_skip_columns_with_unsupported_types_in_schema_inference,false,"Skip columns with unsupported types while schema inference for format ORC",0) \
M(Bool,input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference,false,"Skip columns with unsupported types while schema inference for format Arrow",0) \
M(String,column_names_for_schema_inference,"","The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'",0) \
M(Bool,input_format_protobuf_flatten_google_wrappers,false,"Enable Google wrappers for regular non-nested columns, e.g. google.protobuf.StringValue 'str' for String column 'str'. For Nullable columns empty wrappers are recognized as defaults, and missing as nulls",0) \
M(Bool,output_format_protobuf_nullables_with_google_wrappers,false,"When serializing Nullable columns with Google wrappers, serialize default values as empty wrappers. If turned off, default and null values are not serialized",0) \
M(DateTimeInputFormat,date_time_input_format,FormatSettings::DateTimeInputFormat::Basic,"Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.",0) \
M(DateTimeOutputFormat,date_time_output_format,FormatSettings::DateTimeOutputFormat::Simple,"Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.",0) \
M(Bool,input_format_ipv4_default_on_conversion_error,false,"Deserialization of IPv4 will use default values instead of throwing exception on conversion error.",0) \
M(Bool,input_format_ipv6_default_on_conversion_error,false,"Deserialization of IPV6 will use default values instead of throwing exception on conversion error.",0) \
M(Bool,input_format_values_interpret_expressions,true,"For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.",0) \
M(Bool,input_format_values_deduce_templates_of_expressions,true,"For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.",0) \
M(Bool,input_format_values_accurate_types_of_literals,true,"For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.",0) \
M(Bool,input_format_avro_allow_missing_fields,false,"For Avro/AvroConfluent format: when field is not found in schema use default value instead of error",0) \
M(Bool,output_format_json_escape_forward_slashes,true,"Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.",0) \
M(UInt64,input_format_allow_errors_num,0,"Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.",0) \
M(Float,input_format_allow_errors_ratio,0,"Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.",0) \
\
M(String,format_schema,"","Schema identifier (used by schema-based formats)",0) \
M(String,format_template_resultset,"","Path to file which contains format string for result set (for Template format)",0) \
M(String,format_template_row,"","Path to file which contains format string for rows (for Template format)",0) \
M(String,format_template_rows_between_delimiter,"\n","Delimiter between rows (for Template format)",0) \
M(Bool,output_format_pretty_row_numbers,false,"Add row numbers before each row for pretty output format",0) \
M(Bool,insert_distributed_one_random_shard,false,"If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key",0) \
M(Bool,exact_rows_before_limit,false,"When enabled, ClickHouse will provide exact value for rows_before_limit_at_least statistic, but with the cost that the data before limit will have to be read completely",0) \
M(UInt64,cross_to_inner_join_rewrite,1,"Use inner join instead of comma/cross join if there're joining expressions in the WHERE section. Values: 0 - no rewrite, 1 - apply if possible for comma/cross, 2 - force rewrite all comma joins, cross - if possible",0) \
M(EnumComparingMode,format_capn_proto_enum_comparising_mode,FormatSettings::EnumComparingMode::BY_VALUES,"How to map ClickHouse Enum and CapnProto Enum",0) \
\
M(String,input_format_mysql_dump_table_name,"","Name of the table in MySQL dump from which to read data",0) \
M(Bool,input_format_mysql_dump_map_column_names,true,"Match columns from table in MySQL dump and columns from ClickHouse table by names",0) \