2012-03-05 00:09:41 +00:00
# pragma once
2012-07-26 20:16:57 +00:00
# include <Poco/Timespan.h>
2014-02-13 07:17:22 +00:00
# include <Poco/Util/AbstractConfiguration.h>
2017-04-01 09:19:00 +00:00
# include <Core/Defines.h>
# include <Core/Field.h>
2012-03-05 00:09:41 +00:00
2017-04-01 09:19:00 +00:00
# include <Interpreters/Limits.h>
# include <Interpreters/SettingsCommon.h>
2012-12-24 19:50:22 +00:00
2012-03-05 00:09:41 +00:00
namespace DB
{
2017-01-27 04:29:47 +00:00
/** Settings of query execution.
2012-03-05 00:09:41 +00:00
*/
struct Settings
{
2017-06-02 21:37:28 +00:00
/// For initialization from empty initializer-list to be "value initialization", not "aggregate initialization" in C++14.
2017-04-01 07:20:54 +00:00
/// http://en.cppreference.com/w/cpp/language/aggregate_initialization
Settings ( ) { }
2015-11-16 03:28:10 +00:00
2017-04-01 07:20:54 +00:00
/** List of settings: type, name, default value.
*
* This looks rather unconvenient . It is done that way to avoid repeating settings in different places .
* Note : as an alternative , we could implement settings to be completely dynamic in form of map : String - > Field ,
* but we are not going to do it , because settings is used everywhere as static struct fields .
*/
2013-12-10 17:07:06 +00:00
2014-02-17 23:56:45 +00:00
# define APPLY_FOR_SETTINGS(M) \
2017-05-14 11:47:14 +00:00
/** When writing data, a buffer of max_compress_block_size size is allocated for compression. When the buffer overflows or if into the buffer */ \
/** written data is greater than or equal to min_compress_block_size, then with the next mark, the data will also be compressed */ \
/** As a result, for small columns (around 1-8 bytes), with index_granularity = 8192, the block size will be 64 KB. */ \
/** And for large columns (Title - string ~100 bytes), the block size will be ~819 KB. */ \
/** Due to this, the compression ratio almost does not get worse. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , min_compress_block_size , DEFAULT_MIN_COMPRESS_BLOCK_SIZE ) \
M ( SettingUInt64 , max_compress_block_size , DEFAULT_MAX_COMPRESS_BLOCK_SIZE ) \
2017-05-14 11:47:14 +00:00
/** Maximum block size for reading */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , max_block_size , DEFAULT_BLOCK_SIZE ) \
2017-05-14 11:47:14 +00:00
/** The maximum block size for insertion, if we control the creation of blocks for insertion. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , max_insert_block_size , DEFAULT_INSERT_BLOCK_SIZE ) \
/** 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 bytes, if blocks are not big enough. */ \
M ( SettingUInt64 , min_insert_block_size_bytes , ( DEFAULT_INSERT_BLOCK_SIZE * 256 ) ) \
2017-05-14 11:47:14 +00:00
/** The maximum number of threads to execute the request. By default, it is determined automatically. */ \
2017-04-01 07:20:54 +00:00
M ( SettingMaxThreads , max_threads , 0 ) \
2017-06-21 19:07:08 +00:00
/** The maximum size of the buffer to read from the filesystem. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , max_read_buffer_size , DBMS_DEFAULT_BUFFER_SIZE ) \
2017-05-14 11:47:14 +00:00
/** The maximum number of connections for distributed processing of one query (should be greater than max_threads). */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , max_distributed_connections , DEFAULT_MAX_DISTRIBUTED_CONNECTIONS ) \
2017-05-14 11:47:14 +00:00
/** Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later) */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , max_query_size , DEFAULT_MAX_QUERY_SIZE ) \
2017-05-14 11:47:14 +00:00
/** The interval in microseconds to check if the request is cancelled, and to send progress info. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , interactive_delay , DEFAULT_INTERACTIVE_DELAY ) \
M ( SettingSeconds , connect_timeout , DBMS_DEFAULT_CONNECT_TIMEOUT_SEC ) \
2017-05-14 11:47:14 +00:00
/** If you should select one of the working replicas. */ \
2017-04-01 07:20:54 +00:00
M ( SettingMilliseconds , connect_timeout_with_failover_ms , DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS ) \
M ( SettingSeconds , receive_timeout , DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC ) \
M ( SettingSeconds , send_timeout , DBMS_DEFAULT_SEND_TIMEOUT_SEC ) \
2017-05-14 11:47:14 +00:00
/** The wait time in the request queue, if the number of concurrent requests exceeds the maximum. */ \
2017-04-01 07:20:54 +00:00
M ( SettingMilliseconds , queue_max_wait_ms , DEFAULT_QUERIES_QUEUE_WAIT_TIME_MS ) \
2017-05-14 11:47:14 +00:00
/** Block at the query wait cycle on the server for the specified number of seconds. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , poll_interval , DBMS_DEFAULT_POLL_INTERVAL ) \
2017-05-14 11:47:14 +00:00
/** Maximum number of connections with one remote server in the pool. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , distributed_connections_pool_size , DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE ) \
2017-05-14 11:47:14 +00:00
/** The maximum number of attempts to connect to replicas. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , connections_with_failover_max_tries , DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES ) \
2017-05-14 11:47:14 +00:00
/** Calculate minimums and maximums of the result columns. They can be output in JSON-formats. */ \
2017-04-01 07:20:54 +00:00
M ( SettingBool , extremes , false ) \
2017-05-14 11:47:14 +00:00
/** Whether to use the cache of uncompressed blocks. */ \
2017-04-01 07:20:54 +00:00
M ( SettingBool , use_uncompressed_cache , true ) \
2017-05-14 11:47:14 +00:00
/** Whether the running request should be canceled with the same id as the new one. */ \
2017-04-01 07:20:54 +00:00
M ( SettingBool , replace_running_query , false ) \
2017-05-14 11:47:14 +00:00
/** Number of threads performing background work for tables (for example, merging in merge tree). \
* TODO : Now only applies when the server is started . You can make it dynamically variable . */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , background_pool_size , DBMS_DEFAULT_BACKGROUND_POOL_SIZE ) \
\
/** 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 ) \
\
2017-07-10 15:28:04 +00:00
/** Should StorageDistributed DirectoryMonitors try to batch individual inserts into bigger ones. */ \
M ( SettingBool , distributed_directory_monitor_batch_inserts , false ) \
\
2017-04-01 07:20:54 +00:00
/** Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree */ \
M ( SettingBool , optimize_move_to_prewhere , true ) \
\
2017-05-14 11:47:14 +00:00
/** Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , replication_alter_partitions_sync , 1 ) \
2017-05-14 11:47:14 +00:00
/** Wait for actions to change the table structure within the specified number of seconds. 0 - wait unlimited time. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , replication_alter_columns_timeout , 60 ) \
\
M ( SettingLoadBalancing , load_balancing , LoadBalancing : : RANDOM ) \
\
M ( SettingTotalsMode , totals_mode , TotalsMode : : AFTER_HAVING_EXCLUSIVE ) \
M ( SettingFloat , totals_auto_threshold , 0.5 ) \
\
2017-05-14 11:47:14 +00:00
/** Whether query compilation is enabled. */ \
2017-04-01 07:20:54 +00:00
M ( SettingBool , compile , false ) \
2017-05-14 11:47:14 +00:00
/** The number of structurally identical queries before they are compiled. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , min_count_to_compile , 3 ) \
2017-05-14 11:47:14 +00:00
/** From what number of keys, a two-level aggregation starts. 0 - the threshold is not set. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , group_by_two_level_threshold , 100000 ) \
2017-05-14 11:47:14 +00:00
/** 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 . */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , group_by_two_level_threshold_bytes , 100000000 ) \
2017-05-14 11:47:14 +00:00
/** Is the memory-saving mode of distributed aggregation enabled. */ \
2017-04-01 07:20:54 +00:00
M ( SettingBool , distributed_aggregation_memory_efficient , false ) \
/** 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 ' . */ \
M ( SettingUInt64 , aggregation_memory_efficient_merge_threads , 0 ) \
\
2017-05-14 11:47:14 +00:00
/** The maximum number of replicas of each shard used when executing the query */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , max_parallel_replicas , 1 ) \
M ( SettingUInt64 , parallel_replicas_count , 0 ) \
M ( SettingUInt64 , parallel_replica_offset , 0 ) \
\
2017-05-14 11:47:14 +00:00
/** Silently skip unavailable shards. */ \
2017-04-01 07:20:54 +00:00
M ( SettingBool , skip_unavailable_shards , false ) \
\
2017-05-14 11:47:14 +00:00
/** 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 . \
2017-04-01 07:20:54 +00:00
*/ \
M ( SettingBool , distributed_group_by_no_merge , false ) \
\
2017-05-14 11:47:14 +00:00
/** Advanced settings for reading from MergeTree */ \
2017-04-01 07:20:54 +00:00
\
2017-05-14 11:47:14 +00:00
/** If at least as many lines are read from one file, the reading can be parallelized. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , merge_tree_min_rows_for_concurrent_read , ( 20 * 8192 ) ) \
2017-05-14 11:47:14 +00:00
/** You can skip reading more than that number of rows at the price of one seek per file. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , merge_tree_min_rows_for_seek , 0 ) \
2017-05-14 11:47:14 +00:00
/** If the index segment can contain the required keys, divide it into as many parts and recursively check them. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , merge_tree_coarse_index_granularity , 8 ) \
2017-05-14 11:47:14 +00:00
/** 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 . ) */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , merge_tree_max_rows_to_use_cache , ( 1024 * 1024 ) ) \
\
2017-05-14 11:47:14 +00:00
/** Distribute read from MergeTree over threads evenly, ensuring stable average execution time of each thread within one read operation. */ \
2017-04-01 07:20:54 +00:00
M ( SettingBool , merge_tree_uniform_read_distribution , true ) \
\
2017-05-14 11:47:14 +00:00
/** The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , optimize_min_equality_disjunction_chain_length , 3 ) \
\
2017-05-14 11:47:14 +00:00
/** The minimum number of bytes for input/output operations is bypassing the page cache. 0 - disabled. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , min_bytes_to_use_direct_io , 0 ) \
\
2017-05-14 11:47:14 +00:00
/** Throw an exception if there is an index, and it is not used. */ \
2017-04-01 07:20:54 +00:00
M ( SettingBool , force_index_by_date , 0 ) \
M ( SettingBool , force_primary_key , 0 ) \
\
2017-06-02 21:37:28 +00:00
/** In the INSERT query with specified columns, fill in the default values only for columns with explicit DEFAULTs. */ \
2017-04-01 07:20:54 +00:00
M ( SettingBool , strict_insert_defaults , 0 ) \
\
2017-05-14 11:47:14 +00:00
/** If the maximum size of mark_cache is exceeded, delete only records older than mark_cache_min_lifetime seconds. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , mark_cache_min_lifetime , 10000 ) \
\
2017-05-14 11:47:14 +00:00
/** 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 . \
2017-04-01 07:20:54 +00:00
*/ \
M ( SettingFloat , max_streams_to_max_threads_ratio , 1 ) \
\
2017-05-14 11:47:14 +00:00
/** Allows you to select the method of data compression when writing */ \
2017-04-01 07:20:54 +00:00
M ( SettingCompressionMethod , network_compression_method , CompressionMethod : : LZ4 ) \
\
2017-10-12 23:56:28 +00:00
/** Allows you to select the level of ZSTD compression */ \
M ( SettingInt64 , network_zstd_compression_level , 1 ) \
\
2017-05-14 11:47:14 +00:00
/** Priority of the query. 1 - the highest, higher value - lower priority; 0 - do not use priorities. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , priority , 0 ) \
\
2017-05-14 11:47:14 +00:00
/** Log requests and write the log to the system table. */ \
2017-04-01 07:20:54 +00:00
M ( SettingBool , log_queries , 0 ) \
\
/** 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 . \
*/ \
M ( SettingUInt64 , log_queries_cut_to_length , 100000 ) \
\
2017-05-14 11:47:14 +00:00
/** How are distributed subqueries performed inside IN or JOIN sections? */ \
2017-04-01 07:20:54 +00:00
M ( SettingDistributedProductMode , distributed_product_mode , DistributedProductMode : : DENY ) \
\
2017-05-14 11:47:14 +00:00
/** The scheme for executing GLOBAL subqueries. */ \
2017-04-01 07:20:54 +00:00
M ( SettingGlobalSubqueriesMethod , global_subqueries_method , GlobalSubqueriesMethod : : PUSH ) \
\
2017-05-14 11:47:14 +00:00
/** The maximum number of concurrent requests per user. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , max_concurrent_queries_for_user , 0 ) \
\
2017-10-24 19:32:23 +00:00
/** For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be preformed */ \
M ( SettingBool , insert_deduplicate , true ) \
\
2017-05-14 11:47:14 +00:00
/** For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , insert_quorum , 0 ) \
M ( SettingMilliseconds , insert_quorum_timeout , 600000 ) \
2017-05-14 11:47:14 +00:00
/** 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 . */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , select_sequential_consistency , 0 ) \
2017-05-14 11:47:14 +00:00
/** The maximum number of different shards and the maximum number of replicas of one shard in the `remote` function. */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , table_function_remote_max_addresses , 1000 ) \
2017-05-14 11:47:14 +00:00
/** Settings to reduce the number of threads in case of slow reads. */ \
/** Pay attention only to readings that took at least that much time. */ \
2017-10-08 22:53:38 +00:00
M ( SettingMilliseconds , read_backoff_min_latency_ms , 1000 ) \
2017-05-14 11:47:14 +00:00
/** Count events when the bandwidth is less than that many bytes per second. */ \
2017-10-08 22:53:38 +00:00
M ( SettingUInt64 , read_backoff_max_throughput , 1048576 ) \
2017-05-14 11:47:14 +00:00
/** Do not pay attention to the event, if the previous one has passed less than a certain amount of time. */ \
2017-10-08 22:53:38 +00:00
M ( SettingMilliseconds , read_backoff_min_interval_between_events_ms , 1000 ) \
2017-05-14 11:47:14 +00:00
/** The number of events after which the number of threads will be reduced. */ \
2017-10-08 22:53:38 +00:00
M ( SettingUInt64 , read_backoff_min_events , 2 ) \
2017-04-01 07:20:54 +00:00
\
2017-05-14 11:47:14 +00:00
/** For testing of `exception safety` - throw an exception every time you allocate memory with the specified probability. */ \
2017-04-01 07:20:54 +00:00
M ( SettingFloat , memory_tracker_fault_probability , 0. ) \
\
2017-05-14 11:47:14 +00:00
/** Compress the result if the client over HTTP said that it understands data compressed by gzip or deflate */ \
2017-04-01 07:20:54 +00:00
M ( SettingBool , enable_http_compression , 0 ) \
2017-05-14 11:47:14 +00:00
/** Compression level - used if the client on HTTP said that it understands data compressed by gzip or deflate */ \
2017-04-01 07:20:54 +00:00
M ( SettingInt64 , http_zlib_compression_level , 3 ) \
\
2017-05-14 11:47:14 +00:00
/** If you uncompress the POST data from the client compressed by the native format, do not check the checksum */ \
2017-04-01 07:20:54 +00:00
M ( SettingBool , http_native_compression_disable_checksumming_on_decompress , 0 ) \
\
2017-05-14 11:47:14 +00:00
/** Timeout in seconds */ \
2017-04-01 07:20:54 +00:00
M ( SettingUInt64 , resharding_barrier_timeout , 300 ) \
\
/** What aggregate function to use for implementation of count(DISTINCT ...) */ \
M ( SettingString , count_distinct_implementation , " uniqExact " ) \
\
/** Write statistics about read rows, bytes, time elapsed in suitable output formats */ \
M ( SettingBool , output_format_write_statistics , true ) \
\
/** Write add http CORS header */ \
M ( SettingBool , add_http_cors_header , false ) \
\
/** Skip columns with unknown names from input data (it works for JSONEachRow and TSKV formats). */ \
M ( SettingBool , input_format_skip_unknown_fields , false ) \
\
/** 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 ) \
\
/** Controls quoting of 64-bit integers in JSON output format. */ \
M ( SettingBool , output_format_json_quote_64bit_integers , true ) \
\
2017-07-06 14:42:31 +00:00
/** Enables "+nan", "-nan", "+inf", "-inf" outputs in JSON output format. */ \
M ( SettingBool , output_format_json_quote_denormals , false ) \
\
2017-04-01 07:20:54 +00:00
/** Rows limit for Pretty formats. */ \
M ( SettingUInt64 , output_format_pretty_max_rows , 10000 ) \
\
/** Use client timezone for interpreting DateTime string values, instead of adopting server timezone. */ \
M ( SettingBool , use_client_time_zone , 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 . \
*/ \
M ( SettingBool , send_progress_in_http_headers , false ) \
\
/** Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval. */ \
M ( SettingUInt64 , http_headers_progress_interval_ms , 100 ) \
\
/** 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 . \
*/ \
M ( SettingBool , fsync_metadata , 1 ) \
\
/** Maximum amount of errors while reading text formats (like CSV, TSV). \
* In case of error , if both values are non - zero , \
* and at least absolute or relative amount of errors is lower than corresponding value , \
* will skip until next line and continue . \
*/ \
M ( SettingUInt64 , input_format_allow_errors_num , 0 ) \
2017-04-04 06:52:39 +00:00
M ( SettingFloat , input_format_allow_errors_ratio , 0 ) \
\
/** Use NULLs for non-joined rows of outer JOINs. \
* If false , use default value of corresponding columns data type . \
*/ \
M ( SettingBool , join_use_nulls , 0 ) \
2017-03-24 13:52:50 +00:00
/* */ \
2017-04-17 16:16:04 +00:00
M ( SettingUInt64 , preferred_block_size_bytes , 1000000 ) \
/** 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 . \
*/ \
\
2017-08-16 17:21:39 +00:00
M ( SettingUInt64 , max_replica_delay_for_distributed_queries , 300 ) \
2017-04-17 16:16:04 +00:00
/** 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 . \
*/ \
2017-05-30 11:49:17 +00:00
M ( SettingBool , fallback_to_stale_replicas_for_distributed_queries , 1 ) \
/** For development and testing purposes only still */ \
2017-06-30 16:28:27 +00:00
M ( SettingBool , distributed_ddl_allow_replicated_alter , 0 ) \
/** Limit on max column size in block while reading. Helps to decrease cache misses count. \
* Should be close to L2 cache size . */ \
2017-08-08 12:11:49 +00:00
M ( SettingUInt64 , preferred_max_column_in_block_size_bytes , 0 ) \
2017-07-25 19:42:36 +00:00
\
/** If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster. \
*/ \
2017-08-10 03:44:34 +00:00
M ( SettingBool , insert_distributed_sync , false ) \
2017-07-25 19:42:36 +00:00
/** Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. \
* Zero value means no timeout . \
*/ \
2017-08-02 21:37:04 +00:00
M ( SettingUInt64 , insert_distributed_timeout , 0 ) \
/* Timeout for DDL query responses from all hosts in cluster. Negative value means infinite. */ \
2017-09-08 18:11:09 +00:00
M ( SettingInt64 , distributed_ddl_task_timeout , 120 ) \
\
2017-09-18 19:24:27 +00:00
/** If true, allow parameters of storage engines such as partitioning expression, primary key, etc. \
* to be set not in the engine parameters but as separate clauses ( PARTITION BY , ORDER BY . . . ) \
* Enable this setting to allow custom MergeTree partitions . \
2017-09-08 18:11:09 +00:00
*/ \
2017-09-18 19:24:27 +00:00
M ( SettingBool , experimental_allow_extended_storage_definition_syntax , false ) \
2017-10-03 23:39:24 +00:00
/* Timeout for flushing data from streaming storages. */ \
M ( SettingMilliseconds , stream_flush_interval_ms , DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS ) \
/* Schema identifier (used by schema-based formats) */ \
2017-10-19 07:33:24 +00:00
M ( SettingString , format_schema , " " )
2017-04-17 16:16:04 +00:00
2016-09-19 06:02:42 +00:00
2017-04-01 07:20:54 +00:00
/// Possible limits for query execution.
Limits limits ;
2012-12-24 19:50:22 +00:00
2014-02-17 23:56:45 +00:00
# define DECLARE(TYPE, NAME, DEFAULT) \
2017-04-01 07:20:54 +00:00
TYPE NAME { DEFAULT } ;
2014-02-17 23:56:45 +00:00
2017-04-01 07:20:54 +00:00
APPLY_FOR_SETTINGS ( DECLARE )
2014-02-17 23:56:45 +00:00
# undef DECLARE
2017-04-01 07:20:54 +00:00
/// Set setting by name.
void set ( const String & name , const Field & value ) ;
2013-02-01 19:02:04 +00:00
2017-04-01 07:20:54 +00:00
/// Set setting by name. Read value, serialized in binary form from buffer (for inter-server communication).
void set ( const String & name , ReadBuffer & buf ) ;
2014-02-17 23:56:45 +00:00
2017-04-01 07:20:54 +00:00
/// Skip value, serialized in binary form in buffer.
void ignore ( const String & name , ReadBuffer & buf ) ;
2015-02-27 20:35:26 +00:00
2017-04-01 07:20:54 +00:00
/// Set setting by name. Read value in text form from string (for example, from configuration file or from URL parameter).
void set ( const String & name , const String & value ) ;
2013-02-17 19:54:32 +00:00
2017-07-06 14:42:31 +00:00
/// Get setting by name. Converts value to String.
String get ( const String & name ) const ;
bool tryGet ( const String & name , String & value ) const ;
2017-04-01 07:20:54 +00:00
/** Set multiple settings from "profile" (in server configuration file (users.xml), profiles contain groups of multiple settings).
2017-05-14 11:47:14 +00:00
* The profile can also be set using the ` set ` functions , like the profile setting .
2017-04-01 07:20:54 +00:00
*/
void setProfile ( const String & profile_name , Poco : : Util : : AbstractConfiguration & config ) ;
2013-02-17 19:54:32 +00:00
2017-04-01 07:20:54 +00:00
/// Load settings from configuration file, at "path" prefix in configuration.
void loadSettingsFromConfig ( const String & path , const Poco : : Util : : AbstractConfiguration & config ) ;
2015-09-09 11:39:24 +00:00
2017-04-01 07:20:54 +00:00
/// Read settings from buffer. They are serialized as list of contiguous name-value pairs, finished with empty name.
/// If readonly=1 is set, ignore read settings.
void deserialize ( ReadBuffer & buf ) ;
2014-02-17 23:56:45 +00:00
2017-04-01 07:20:54 +00:00
/// Write changed settings to buffer. (For example, to be sent to remote server.)
void serialize ( WriteBuffer & buf ) const ;
2012-03-05 00:09:41 +00:00
} ;
}