M(SettingUInt64,min_compress_block_size,DEFAULT_MIN_COMPRESS_BLOCK_SIZE, "The actual size of the block, 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.") \
M(SettingUInt64,max_compress_block_size,DEFAULT_MAX_COMPRESS_BLOCK_SIZE, "The maximum size of blocks of uncompressed data before compressing for writing to a table.") \
M(SettingUInt64,max_insert_block_size,DEFAULT_INSERT_BLOCK_SIZE, "The maximum block size for insertion, if we control the creation of blocks for insertion.") \
/** Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough. */ \
M(SettingUInt64,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.") \
/** Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough. */ \
M(SettingUInt64,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.") \
/** The maximum number of threads to execute the request. By default, it is determined automatically. */ \
M(SettingMaxThreads,max_threads,0) \
M(SettingMaxThreads,max_threads,0, "The maximum number of threads to execute the request. By default, it is determined automatically.") \
/** The maximum size of the buffer to read from the filesystem. */ \
M(SettingUInt64,max_distributed_connections,DEFAULT_MAX_DISTRIBUTED_CONNECTIONS, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).") \
/** Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later) */ \
M(SettingUInt64,max_query_size,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)") \
/** The interval in microseconds to check if the request is cancelled, and to send progress info. */ \
M(SettingUInt64,interactive_delay,DEFAULT_INTERACTIVE_DELAY, "The interval in microseconds to check if the request is cancelled, and to send progress info.") \
M(SettingMilliseconds,queue_max_wait_ms,DEFAULT_QUERIES_QUEUE_WAIT_TIME_MS, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.") \
/** Block at the query wait cycle on the server for the specified number of seconds. */ \
M(SettingUInt64,distributed_connections_pool_size,DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.") \
/** The maximum number of attempts to connect to replicas. */ \
M(SettingUInt64,connections_with_failover_max_tries,DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.") \
/** Calculate minimums and maximums of the result columns. They can be output in JSON-formats. */ \
M(SettingBool,extremes,false) \
M(SettingBool,extremes,false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.") \
/** Whether to use the cache of uncompressed blocks. */ \
M(SettingBool,use_uncompressed_cache,true) \
M(SettingBool,use_uncompressed_cache,true, "Whether to use the cache of uncompressed blocks.") \
/** Whether the running request should be canceled with the same id as the new one. */ \
M(SettingBool,replace_running_query,false) \
M(SettingBool,replace_running_query,false, "Whether the running request should be canceled with the same id as the new one.") \
/** Number of threads performing background work for tables (for example, merging in merge tree). \
M(SettingUInt64,background_pool_size,DBMS_DEFAULT_BACKGROUND_POOL_SIZE, "Number of threads performing background work for tables (for example, merging in merge tree).") \
\
/** Sleep time for StorageDistributed DirectoryMonitors in case there is no work or exception has been thrown */ \
M(SettingMilliseconds,distributed_directory_monitor_sleep_time_ms,DBMS_DISTRIBUTED_DIRECTORY_MONITOR_SLEEP_TIME_MS, "Sleep time for StorageDistributed DirectoryMonitors in case there is no work or exception has been thrown.") \
\
/** Should StorageDistributed DirectoryMonitors try to batch individual inserts into bigger ones. */ \
M(SettingUInt64,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.") \
/** Wait for actions to change the table structure within the specified number of seconds. 0 - wait unlimited time. */ \
M(SettingUInt64,replication_alter_columns_timeout,60, "Wait for actions to change the table structure within the specified number of seconds. 0 - wait unlimited time.") \
M(SettingLoadBalancing,load_balancing,LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.") \
M(SettingTotalsMode,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.") \
M(SettingFloat,totals_auto_threshold,0.5, "The threshold for totals_mode = 'auto'.") \
\
/** Whether query compilation is enabled. */ \
M(SettingBool,compile,false) \
M(SettingBool,compile,false, "Whether query compilation is enabled.") \
/** The number of structurally identical queries before they are compiled. */ \
M(SettingUInt64,min_count_to_compile,3) \
M(SettingUInt64,min_count_to_compile,3, "The number of structurally identical queries before they are compiled.") \
/** From what number of keys, a two-level aggregation starts. 0 - the threshold is not set. */ \
M(SettingUInt64,group_by_two_level_threshold_bytes,100000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set.") \
/** Is the memory-saving mode of distributed aggregation enabled. */ \
M(SettingUInt64,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'.") \
\
/** The maximum number of replicas of each shard used when executing the query */ \
M(SettingUInt64,max_parallel_replicas,1) \
M(SettingUInt64,parallel_replicas_count,0) \
M(SettingUInt64,parallel_replica_offset,0) \
M(SettingUInt64,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.") \
M(SettingBool,distributed_group_by_no_merge,false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.") \
\
/** Advanced settings for reading from MergeTree */ \
\
/** If at least as many lines are read from one file, the reading can be parallelized. */ \
M(SettingUInt64,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.") \
/** You can skip reading more than that number of rows at the price of one seek per file. */ \
M(SettingUInt64,merge_tree_min_rows_for_seek,0) \
M(SettingUInt64,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.") \
/** If the index segment can contain the required keys, divide it into as many parts and recursively check them. */ \
M(SettingUInt64,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. ") \
/** The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. \
M(SettingUInt64,merge_tree_max_rows_to_use_cache,(1024*1024), "The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used.") \
\
/** Distribute read from MergeTree over threads evenly, ensuring stable average execution time of each thread within one read operation. */ \
M(SettingBool,merge_tree_uniform_read_distribution,true, "Distribute read from MergeTree over threads evenly, ensuring stable average execution time of each thread within one read operation.") \
\
/** The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization */ \
M(SettingUInt64,optimize_min_equality_disjunction_chain_length,3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ") \
\
/** The minimum number of bytes for input/output operations is bypassing the page cache. 0 - disabled. */ \
M(SettingUInt64,min_bytes_to_use_direct_io,0) \
M(SettingUInt64,min_bytes_to_use_direct_io,0, "The minimum number of bytes for input/output operations is bypassing the page cache. 0 - disabled.") \
\
/** Throw an exception if there is an index, and it is not used. */ \
M(SettingBool,force_index_by_date,0) \
M(SettingBool,force_primary_key,0) \
M(SettingBool,force_index_by_date,0, "") \
M(SettingBool,force_primary_key,0, "") \
\
/** In the INSERT query with specified columns, fill in the default values only for columns with explicit DEFAULTs. */ \
M(SettingBool,strict_insert_defaults,0) \
M(SettingBool,strict_insert_defaults,0, "In the INSERT query with specified columns, fill in the default values only for columns with explicit DEFAULTs.") \
\
/** If the maximum size of mark_cache is exceeded, delete only records older than mark_cache_min_lifetime seconds. */ \
M(SettingUInt64,mark_cache_min_lifetime,10000) \
M(SettingUInt64,mark_cache_min_lifetime,10000, "If the maximum size of mark_cache is exceeded, delete only records older than mark_cache_min_lifetime seconds.") \
\
/** Allows you to use more sources than the number of threads - to more evenly distribute work across threads. \
M(SettingUInt64,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.") \
\
/** How are distributed subqueries performed inside IN or JOIN sections? */ \
M(SettingDistributedProductMode,distributed_product_mode,DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?") \
\
/** The scheme for executing GLOBAL subqueries. */ \
M(SettingUInt64,max_concurrent_queries_for_user,0, "The maximum number of concurrent requests per user.") \
\
/** For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be preformed */ \
M(SettingBool,insert_deduplicate,true) \
M(SettingBool,insert_deduplicate,true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be preformed") \
\
/** For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled. */ \
M(SettingUInt64,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.") \
M(SettingUInt64,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.") \
/** The maximum number of different shards and the maximum number of replicas of one shard in the `remote` function. */ \
M(SettingUInt64,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.") \
/** Settings to reduce the number of threads in case of slow reads. */ \
/** Pay attention only to readings that took at least that much time. */ \
M(SettingMilliseconds,read_backoff_min_interval_between_events_ms,1000, "Do not pay attention to the event, if the previous one has passed less than a certain amount of time.") \
/** The number of events after which the number of threads will be reduced. */ \
M(SettingUInt64,read_backoff_min_events,2) \
M(SettingUInt64,read_backoff_min_events,2, "The number of events after which the number of threads will be reduced.") \
\
/** For testing of `exception safety` - throw an exception every time you allocate memory with the specified probability. */ \
M(SettingFloat,memory_tracker_fault_probability,0., "For testing of `exception safety` - throw an exception every time you allocate memory with the specified probability.") \
\
/** Compress the result if the client over HTTP said that it understands data compressed by gzip or deflate */ \
M(SettingBool,enable_http_compression,0) \
M(SettingBool,enable_http_compression,0, "Compress the result if the client over HTTP said that it understands data compressed by gzip or deflate.") \
/** Compression level - used if the client on HTTP said that it understands data compressed by gzip or deflate */ \
M(SettingInt64,http_zlib_compression_level,3) \
M(SettingInt64,http_zlib_compression_level,3, "Compression level - used if the client on HTTP said that it understands data compressed by gzip or deflate.") \
\
/** If you uncompress the POST data from the client compressed by the native format, do not check the checksum */ \
M(SettingBool,http_native_compression_disable_checksumming_on_decompress,0, "If you uncompress the POST data from the client compressed by the native format, do not check the checksum.") \
M(SettingBool,input_format_skip_unknown_fields,false, "Skip columns with unknown names from input data (it works for JSONEachRow and TSKV formats).") \
\
/** For Values format: if field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression. */ \
M(SettingBool,input_format_values_interpret_expressions,true, "For Values format: if field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.") \
\
/** Controls quoting of 64-bit integers in JSON output format. */ \
M(SettingUInt64,http_headers_progress_interval_ms,100, "Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval.") \
\
/** Do fsync after changing metadata for tables and databases (.sql files). \
M(SettingUInt64,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).") \
/** Suppose max_replica_delay_for_distributed_queries is set and all replicas for the queried table are stale. \
M(SettingUInt64,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.") \
\
/** If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster. \
*/ \
M(SettingBool,insert_distributed_sync,false) \
M(SettingBool,insert_distributed_sync,false, "If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.") \
/** Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. \
*Zerovaluemeansnotimeout. \
*/ \
M(SettingUInt64,insert_distributed_timeout,0) \
M(SettingUInt64,insert_distributed_timeout,0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled.") \
/* Timeout for DDL query responses from all hosts in cluster. Negative value means infinite. */ \
Blocking a user prevents them from interacting with repositories, such as opening or commenting on pull requests or issues. Learn more about blocking a user.