From b76b23ce31c2ede1260ad9e64605c51d49a229db Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 27 Jul 2020 00:41:27 +0300 Subject: [PATCH 01/20] Better conversion of String and std::string_view to Field. --- src/Columns/ColumnFixedString.h | 4 +- src/Columns/ColumnString.h | 2 +- src/Core/Field.h | 94 ++++++++++++++++++++++++--------- 3 files changed, 73 insertions(+), 27 deletions(-) diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 18b4ec155b3..3bc74d4c2ff 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -69,12 +69,12 @@ public: Field operator[](size_t index) const override { - return String(reinterpret_cast(&chars[n * index]), n); + return Field{&chars[n * index], n}; } void get(size_t index, Field & res) const override { - res.assignString(reinterpret_cast(&chars[n * index]), n); + res = std::string_view{reinterpret_cast(&chars[n * index]), n}; } StringRef getDataAt(size_t index) const override diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 757c5f250ec..d8948afacbc 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -86,7 +86,7 @@ public: void get(size_t n, Field & res) const override { assert(n < size()); - res.assignString(&chars[offsetAt(n)], sizeAt(n) - 1); + res = std::string_view{reinterpret_cast(&chars[offsetAt(n)]), sizeAt(n) - 1}; } StringRef getDataAt(size_t n) const override diff --git a/src/Core/Field.h b/src/Core/Field.h index 152ae29bd1e..63054922fef 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -187,6 +187,7 @@ template <> struct NearestFieldTypeImpl> { using Type = template <> struct NearestFieldTypeImpl { using Type = Float64; }; template <> struct NearestFieldTypeImpl { using Type = Float64; }; template <> struct NearestFieldTypeImpl { using Type = String; }; +template <> struct NearestFieldTypeImpl { using Type = String; }; template <> struct NearestFieldTypeImpl { using Type = String; }; template <> struct NearestFieldTypeImpl { using Type = Array; }; template <> struct NearestFieldTypeImpl { using Type = Tuple; }; @@ -272,7 +273,10 @@ public: template struct TypeToEnum; template struct EnumToType; - static bool IsDecimal(Types::Which which) { return which >= Types::Decimal32 && which <= Types::Decimal128; } + + /// Templates to avoid ambiguity. + template + using enable_if_not_field_or_stringlike_t = std::enable_if_t, Field> && !std::is_same_v>, String>, Z>; Field() : which(Types::Null) @@ -293,23 +297,20 @@ public: } template - Field(T && rhs, std::enable_if_t, Field>, void *> = nullptr); + Field(T && rhs, enable_if_not_field_or_stringlike_t = nullptr); /// Create a string inplace. + Field(const std::string_view & str) { create(str.data(), str.size()); } + Field(const String & str) { create(std::string_view{str}); } + Field(String && str) { create(std::move(str)); } + Field(const char * str) { create(std::string_view{str}); } + template Field(const CharT * data, size_t size) { create(data, size); } - /// NOTE In case when field already has string type, more direct assign is possible. - template - void assignString(const CharT * data, size_t size) - { - destroy(); - create(data, size); - } - Field & operator= (const Field & rhs) { if (this != &rhs) @@ -340,9 +341,19 @@ public: return *this; } + /// Allows expressions like + /// Field f = 1; + /// Things to note: + /// 1. float <--> int needs explicit cast + /// 2. customized types needs explicit cast template - std::enable_if_t, Field>, Field &> - operator= (T && rhs); + enable_if_not_field_or_stringlike_t & + operator=(T && rhs); + + Field & operator =(const std::string_view & str); + Field & operator =(const String & str) { return *this = std::string_view{str}; } + Field & operator =(String && str); + Field & operator =(const char * str) { return *this = std::string_view{str}; } ~Field() { @@ -410,7 +421,6 @@ public: return get(); } - bool operator< (const Field & rhs) const { if (which < rhs.which) @@ -590,6 +600,20 @@ private: *ptr = std::forward(x); } + template + std::enable_if_t assignString(const CharT * data, size_t size) + { + assert(which == Types::String); + String * ptr = reinterpret_cast(&storage); + ptr->assign(reinterpret_cast(data), size); + } + + void assignString(String && str) + { + assert(which == Types::String); + String * ptr = reinterpret_cast(&storage); + ptr->assign(std::move(str)); + } void create(const Field & x) { @@ -618,6 +642,12 @@ private: which = Types::String; } + void create(String && str) + { + new (&storage) String(std::move(str)); + which = Types::String; + } + ALWAYS_INLINE void destroy() { if (which < Types::MIN_NON_POD) @@ -753,23 +783,16 @@ decltype(auto) castToNearestFieldType(T && x) return U(x); } -/// This (rather tricky) code is to avoid ambiguity in expressions like -/// Field f = 1; -/// instead of -/// Field f = Int64(1); -/// Things to note: -/// 1. float <--> int needs explicit cast -/// 2. customized types needs explicit cast template -Field::Field(T && rhs, std::enable_if_t, Field>, void *>) +Field::Field(T && rhs, enable_if_not_field_or_stringlike_t) { auto && val = castToNearestFieldType(std::forward(rhs)); createConcrete(std::forward(val)); } template -std::enable_if_t, Field>, Field &> -Field::operator= (T && rhs) +Field::enable_if_not_field_or_stringlike_t & +Field::operator=(T && rhs) { auto && val = castToNearestFieldType(std::forward(rhs)); using U = decltype(val); @@ -780,11 +803,34 @@ Field::operator= (T && rhs) } else assignConcrete(std::forward(val)); - return *this; } +inline Field & Field::operator=(const std::string_view & str) +{ + if (which != Types::String) + { + destroy(); + create(str.data(), str.size()); + } + else + assignString(str.data(), str.size()); + return *this; +} + +inline Field & Field::operator=(String && str) +{ + if (which != Types::String) + { + destroy(); + create(std::move(str)); + } + else + assignString(std::move(str)); + return *this; +} + class ReadBuffer; class WriteBuffer; From 30b34e6a15c8f72e71ab2e344574073629f84080 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 16 Jul 2020 16:38:58 +0300 Subject: [PATCH 02/20] Use SettingSeconds only in Settings, use std::chrono::seconds in other places. --- src/Core/SettingsCollection.h | 11 ++++++++++- src/Databases/DatabaseMySQL.cpp | 3 +-- src/Storages/IStorage.cpp | 12 ++++++------ src/Storages/IStorage.h | 8 ++++---- src/Storages/System/StorageSystemColumns.cpp | 2 +- 5 files changed, 22 insertions(+), 14 deletions(-) diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h index 194c9ed4ff1..1098733cbe1 100644 --- a/src/Core/SettingsCollection.h +++ b/src/Core/SettingsCollection.h @@ -106,7 +106,13 @@ struct SettingTimespan SettingTimespan(UInt64 x = 0) : value(x * microseconds_per_io_unit) {} operator Poco::Timespan() const { return value; } - SettingTimespan & operator= (const Poco::Timespan & x) { set(x); return *this; } + SettingTimespan & operator=(const Poco::Timespan & x) { set(x); return *this; } + + template > + operator std::chrono::duration() const { return std::chrono::duration_cast>(std::chrono::microseconds(value.totalMicroseconds())); } + + template > + SettingTimespan & operator=(const std::chrono::duration & x) { set(x); return *this; } Poco::Timespan::TimeDiff totalSeconds() const { return value.totalSeconds(); } Poco::Timespan::TimeDiff totalMilliseconds() const { return value.totalMilliseconds(); } @@ -116,6 +122,9 @@ struct SettingTimespan void set(const Poco::Timespan & x); + template > + void set(const std::chrono::duration & duration) { set(static_cast(std::chrono::duration_cast(duration).count())); } + void set(UInt64 x); void set(const Field & x); void set(const String & x); diff --git a/src/Databases/DatabaseMySQL.cpp b/src/Databases/DatabaseMySQL.cpp index 6e5837257f0..56cd5c8088d 100644 --- a/src/Databases/DatabaseMySQL.cpp +++ b/src/Databases/DatabaseMySQL.cpp @@ -4,7 +4,6 @@ #if USE_MYSQL # include -# include # include # include # include @@ -42,7 +41,7 @@ namespace ErrorCodes constexpr static const auto suffix = ".remove_flag"; static constexpr const std::chrono::seconds cleaner_sleep_time{30}; -static const SettingSeconds lock_acquire_timeout{10}; +static const std::chrono::seconds lock_acquire_timeout{10}; static String toQueryStringWithQuote(const std::vector & quote_list) { diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 4513394cada..c43002366c2 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -31,22 +31,22 @@ bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadata } RWLockImpl::LockHolder IStorage::tryLockTimed( - const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const + const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const std::chrono::milliseconds & acquire_timeout) const { - auto lock_holder = rwlock->getLock(type, query_id, std::chrono::milliseconds(acquire_timeout.totalMilliseconds())); + auto lock_holder = rwlock->getLock(type, query_id, acquire_timeout); if (!lock_holder) { const String type_str = type == RWLockImpl::Type::Read ? "READ" : "WRITE"; throw Exception( type_str + " locking attempt on \"" + getStorageID().getFullTableName() + - "\" has timed out! (" + toString(acquire_timeout.totalMilliseconds()) + "ms) " + "\" has timed out! (" + std::to_string(acquire_timeout.count()) + "ms) " "Possible deadlock avoided. Client should retry.", ErrorCodes::DEADLOCK_AVOIDED); } return lock_holder; } -TableLockHolder IStorage::lockForShare(const String & query_id, const SettingSeconds & acquire_timeout) +TableLockHolder IStorage::lockForShare(const String & query_id, const std::chrono::milliseconds & acquire_timeout) { TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout); @@ -56,7 +56,7 @@ TableLockHolder IStorage::lockForShare(const String & query_id, const SettingSec return result; } -TableLockHolder IStorage::lockForAlter(const String & query_id, const SettingSeconds & acquire_timeout) +TableLockHolder IStorage::lockForAlter(const String & query_id, const std::chrono::milliseconds & acquire_timeout) { TableLockHolder result = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout); @@ -67,7 +67,7 @@ TableLockHolder IStorage::lockForAlter(const String & query_id, const SettingSec } -TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout) +TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, const std::chrono::milliseconds & acquire_timeout) { TableExclusiveLockHolder result; result.alter_lock = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 1c75a661339..5e91429de34 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -185,18 +185,18 @@ private: MultiVersionStorageMetadataPtr metadata; private: RWLockImpl::LockHolder tryLockTimed( - const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const; + const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const std::chrono::milliseconds & acquire_timeout) const; public: /// Lock table for share. This lock must be acuqired if you want to be sure, /// that table will be not dropped while you holding this lock. It's used in /// variety of cases starting from SELECT queries to background merges in /// MergeTree. - TableLockHolder lockForShare(const String & query_id, const SettingSeconds & acquire_timeout); + TableLockHolder lockForShare(const String & query_id, const std::chrono::milliseconds & acquire_timeout); /// Lock table for alter. This lock must be acuqired in ALTER queries to be /// sure, that we execute only one simultaneous alter. Doesn't affect share lock. - TableLockHolder lockForAlter(const String & query_id, const SettingSeconds & acquire_timeout); + TableLockHolder lockForAlter(const String & query_id, const std::chrono::milliseconds & acquire_timeout); /// Lock table exclusively. This lock must be acuired if you want to be /// sure, that no other thread (SELECT, merge, ALTER, etc.) doing something @@ -205,7 +205,7 @@ public: /// /// NOTE: You have to be 100% sure that you need this lock. It's extremely /// heavyweight and makes table irresponsive. - TableExclusiveLockHolder lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout); + TableExclusiveLockHolder lockExclusively(const String & query_id, const std::chrono::milliseconds & acquire_timeout); /** Returns stage to which query is going to be processed in read() function. * (Normally, the function only reads the columns from the list, but in other cases, diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index beb01bc6192..57d8d13e79b 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -236,7 +236,7 @@ private: size_t total_tables; std::shared_ptr access; String query_id; - SettingSeconds lock_acquire_timeout; + std::chrono::milliseconds lock_acquire_timeout; }; From cd372de4175ff38dbb52df54b363041aa3001275 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 17 Jul 2020 01:08:44 +0300 Subject: [PATCH 03/20] Use SettingUInt64 only in Settings, use UInt64 in other places. --- src/DataStreams/TTLBlockInputStream.cpp | 3 +-- src/IO/ReadWriteBufferFromHTTP.h | 12 ++++++------ src/Interpreters/InJoinSubqueriesPreprocessor.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 3 +-- 4 files changed, 9 insertions(+), 11 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index bc5270687d8..6d80e784c03 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -91,8 +91,7 @@ TTLBlockInputStream::TTLBlockInputStream( const Settings & settings = storage.global_context.getSettingsRef(); Aggregator::Params params(header, keys, aggregates, - false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, - SettingUInt64(0), SettingUInt64(0), + false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0, settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, storage.global_context.getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); aggregator = std::make_unique(params); diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 40057e1f80e..0eb2136ca6c 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -44,14 +44,14 @@ protected: UInt64 redirects { 0 }; Poco::URI initial_uri; const ConnectionTimeouts & timeouts; - SettingUInt64 max_redirects; + UInt64 max_redirects; public: virtual void buildNewSession(const Poco::URI & uri) = 0; explicit UpdatableSessionBase(const Poco::URI uri, const ConnectionTimeouts & timeouts_, - SettingUInt64 max_redirects_) + UInt64 max_redirects_) : initial_uri { uri } , timeouts { timeouts_ } , max_redirects { max_redirects_ } @@ -228,7 +228,7 @@ class UpdatableSession : public UpdatableSessionBase public: explicit UpdatableSession(const Poco::URI uri, const ConnectionTimeouts & timeouts_, - const SettingUInt64 max_redirects_) + const UInt64 max_redirects_) : Parent(uri, timeouts_, max_redirects_) { session = makeHTTPSession(initial_uri, timeouts); @@ -249,7 +249,7 @@ public: const std::string & method_, OutStreamCallback out_stream_callback_, const ConnectionTimeouts & timeouts, - const SettingUInt64 max_redirects = 0, + const UInt64 max_redirects = 0, const Poco::Net::HTTPBasicCredentials & credentials_ = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, const HTTPHeaderEntries & http_header_entries_ = {}, @@ -269,7 +269,7 @@ private: public: explicit UpdatablePooledSession(const Poco::URI uri, const ConnectionTimeouts & timeouts_, - const SettingUInt64 max_redirects_, + const UInt64 max_redirects_, size_t per_endpoint_pool_size_) : Parent(uri, timeouts_, max_redirects_) , per_endpoint_pool_size { per_endpoint_pool_size_ } @@ -294,7 +294,7 @@ public: const ConnectionTimeouts & timeouts_ = {}, const Poco::Net::HTTPBasicCredentials & credentials_ = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, - const SettingUInt64 max_redirects = 0, + const UInt64 max_redirects = 0, size_t max_connections_per_endpoint = DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT) : Parent(std::make_shared(uri_, timeouts_, max_redirects, max_connections_per_endpoint), uri_, diff --git a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp index 051341964a2..244693396ca 100644 --- a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp +++ b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp @@ -53,7 +53,7 @@ struct NonGlobalTableData private: void renameIfNeeded(ASTPtr & database_and_table) { - const SettingDistributedProductMode distributed_product_mode = context.getSettingsRef().distributed_product_mode; + const DistributedProductMode distributed_product_mode = context.getSettingsRef().distributed_product_mode; StoragePtr storage = tryGetTable(database_and_table, context); if (!storage || !checker.hasAtLeastTwoShards(*storage)) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f5ea9ec965d..19a4e998dc7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1527,8 +1527,7 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific const Settings & settings = context->getSettingsRef(); Aggregator::Params params(header_before_transform, keys, query_analyzer->aggregates(), - false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, - SettingUInt64(0), SettingUInt64(0), + false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0, settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, context->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); From 668653600cd2b8b749c5242e367da94a7ab55b00 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 17 Jul 2020 02:12:47 +0300 Subject: [PATCH 04/20] Use SettingMaxThreads only in Settings, call getNumberOfPhysicalCPUCores() instead of SettingMaxThreads::getAuto(). --- src/Common/ThreadPool.cpp | 8 ++++ src/Common/ThreadPool.h | 3 ++ src/Common/getNumberOfPhysicalCPUCores.cpp | 50 ++++++++++++---------- src/Core/SettingsCollection.cpp | 3 +- src/Databases/DatabaseOnDisk.cpp | 2 +- src/Databases/DatabaseOrdinary.cpp | 2 +- src/Interpreters/DatabaseCatalog.cpp | 2 +- 7 files changed, 42 insertions(+), 28 deletions(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 63351a77544..49516d777fb 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -24,6 +25,13 @@ namespace CurrentMetrics } +template +ThreadPoolImpl::ThreadPoolImpl() + : ThreadPoolImpl(getNumberOfPhysicalCPUCores()) +{ +} + + template ThreadPoolImpl::ThreadPoolImpl(size_t max_threads_) : ThreadPoolImpl(max_threads_, max_threads_, max_threads_) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index c1304051ea7..55796905b73 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -29,6 +29,9 @@ class ThreadPoolImpl public: using Job = std::function; + /// Maximum number of threads is based on the number of physical cores. + ThreadPoolImpl(); + /// Size is constant. Up to num_threads are created on demand and then run until shutdown. explicit ThreadPoolImpl(size_t max_threads_); diff --git a/src/Common/getNumberOfPhysicalCPUCores.cpp b/src/Common/getNumberOfPhysicalCPUCores.cpp index 3808a367f8b..13485c634e8 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.cpp +++ b/src/Common/getNumberOfPhysicalCPUCores.cpp @@ -9,29 +9,33 @@ unsigned getNumberOfPhysicalCPUCores() { -#if USE_CPUID - cpu_raw_data_t raw_data; - cpu_id_t data; + static const unsigned number = [] + { +# if USE_CPUID + cpu_raw_data_t raw_data; + cpu_id_t data; - /// On Xen VMs, libcpuid returns wrong info (zero number of cores). Fallback to alternative method. - /// Also, libcpuid does not support some CPUs like AMD Hygon C86 7151. - if (0 != cpuid_get_raw_data(&raw_data) || 0 != cpu_identify(&raw_data, &data) || data.num_logical_cpus == 0) + /// On Xen VMs, libcpuid returns wrong info (zero number of cores). Fallback to alternative method. + /// Also, libcpuid does not support some CPUs like AMD Hygon C86 7151. + if (0 != cpuid_get_raw_data(&raw_data) || 0 != cpu_identify(&raw_data, &data) || data.num_logical_cpus == 0) + return std::thread::hardware_concurrency(); + + unsigned res = data.num_cores * data.total_logical_cpus / data.num_logical_cpus; + + /// Also, libcpuid gives strange result on Google Compute Engine VMs. + /// Example: + /// num_cores = 12, /// number of physical cores on current CPU socket + /// total_logical_cpus = 1, /// total number of logical cores on all sockets + /// num_logical_cpus = 24. /// number of logical cores on current CPU socket + /// It means two-way hyper-threading (24 / 12), but contradictory, 'total_logical_cpus' == 1. + + if (res != 0) + return res; +# endif + + /// As a fallback (also for non-x86 architectures) assume there are no hyper-threading on the system. + /// (Actually, only Aarch64 is supported). return std::thread::hardware_concurrency(); - - unsigned res = data.num_cores * data.total_logical_cpus / data.num_logical_cpus; - - /// Also, libcpuid gives strange result on Google Compute Engine VMs. - /// Example: - /// num_cores = 12, /// number of physical cores on current CPU socket - /// total_logical_cpus = 1, /// total number of logical cores on all sockets - /// num_logical_cpus = 24. /// number of logical cores on current CPU socket - /// It means two-way hyper-threading (24 / 12), but contradictory, 'total_logical_cpus' == 1. - - if (res != 0) - return res; -#endif - - /// As a fallback (also for non-x86 architectures) assume there are no hyper-threading on the system. - /// (Actually, only Aarch64 is supported). - return std::thread::hardware_concurrency(); + }(); + return number; } diff --git a/src/Core/SettingsCollection.cpp b/src/Core/SettingsCollection.cpp index 32bf1f29c90..ed1a16eab0e 100644 --- a/src/Core/SettingsCollection.cpp +++ b/src/Core/SettingsCollection.cpp @@ -210,8 +210,7 @@ void SettingMaxThreads::setAuto() UInt64 SettingMaxThreads::getAutoValue() { - static auto res = getNumberOfPhysicalCPUCores(); - return res; + return getNumberOfPhysicalCPUCores(); } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 43846ff6d64..799ed041bef 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -426,7 +426,7 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const Iterati } /// Read and parse metadata in parallel - ThreadPool pool(SettingMaxThreads().getAutoValue()); + ThreadPool pool; for (const auto & file : metadata_files) { pool.scheduleOrThrowOnError([&]() diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 9e7d2b52199..1e82420298b 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -152,7 +152,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto std::atomic tables_processed{0}; std::atomic dictionaries_processed{0}; - ThreadPool pool(SettingMaxThreads().getAutoValue()); + ThreadPool pool; /// Attach tables. for (const auto & name_with_query : file_names) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 545d586fa67..49b79ad0314 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -560,7 +560,7 @@ void DatabaseCatalog::loadMarkedAsDroppedTables() dropped_metadata.emplace(std::move(full_path), std::move(dropped_id)); } - ThreadPool pool(SettingMaxThreads().getAutoValue()); + ThreadPool pool; for (const auto & elem : dropped_metadata) { pool.scheduleOrThrowOnError([&]() From 4ad99f147227942d9bec275d7417c654f154b60a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 16 Jul 2020 18:16:50 +0300 Subject: [PATCH 05/20] Make type names of settings shorter: SettingUInt64 => UInt64. --- src/Core/Settings.h | 666 +++++++++--------- src/Core/SettingsCollection.h | 8 +- src/Core/SettingsCollectionImpl.h | 4 +- src/Storages/Kafka/KafkaSettings.h | 28 +- src/Storages/MergeTree/MergeTreeSettings.h | 128 ++-- src/Storages/RabbitMQ/RabbitMQSettings.h | 18 +- .../01221_system_settings.reference | 4 +- .../0_stateless/01221_system_settings.sql | 52 +- 8 files changed, 454 insertions(+), 454 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ac325538923..091979a597a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -47,192 +47,192 @@ struct Settings : public SettingsCollection */ #define COMMON_SETTINGS(M) \ - M(SettingUInt64, 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(SettingUInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.", 0) \ - M(SettingUInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading", 0) \ - 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.", 0) \ - 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.", 0) \ - 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.", 0) \ - M(SettingUInt64, 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(SettingUInt64, 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(SettingUInt64, max_joined_block_size_rows, DEFAULT_BLOCK_SIZE, "Maximum block size for JOIN result (if join algorithm supports it). 0 means unlimited.", 0) \ - M(SettingUInt64, 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(SettingUInt64, max_final_threads, 16, "The maximum number of threads to read from table with FINAL.", 0) \ - M(SettingMaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.", 0) \ - M(SettingMaxThreads, max_alter_threads, 0, "The maximum number of threads to execute the ALTER requests. By default, it is determined automatically.", 0) \ - M(SettingUInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.", 0) \ - M(SettingUInt64, max_distributed_connections, 1024, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).", 0) \ - M(SettingUInt64, max_query_size, 262144, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)", 0) \ - M(SettingUInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \ - M(SettingSeconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \ - M(SettingMilliseconds, connect_timeout_with_failover_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS, "Connection timeout for selecting first healthy replica.", 0) \ - M(SettingMilliseconds, connect_timeout_with_failover_secure_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_SECURE_MS, "Connection timeout for selecting first healthy replica (for secure connections).", 0) \ - M(SettingSeconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \ - M(SettingSeconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \ - M(SettingSeconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \ - M(SettingMilliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.", 0) \ - M(SettingMilliseconds, connection_pool_max_wait_ms, 0, "The wait time when the connection pool is full.", 0) \ - M(SettingMilliseconds, 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(SettingMilliseconds, kafka_max_wait_ms, 5000, "The wait time for reading from Kafka before retry.", 0) \ - M(SettingMilliseconds, rabbitmq_max_wait_ms, 5000, "The wait time for reading from RabbitMQ before retry.", 0) \ - M(SettingUInt64, poll_interval, DBMS_DEFAULT_POLL_INTERVAL, "Block at the query wait loop on the server for the specified number of seconds.", 0) \ - M(SettingUInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.", 0) \ - M(SettingUInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.", 0) \ - M(SettingUInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \ - M(SettingUInt64, s3_min_upload_part_size, 512*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \ - M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ - M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \ - M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \ - M(SettingUInt64, background_buffer_flush_schedule_pool_size, 16, "Number of threads performing background flush for tables with Buffer engine. Only has meaning at server startup.", 0) \ - M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \ - M(SettingUInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \ - M(SettingUInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables, kafka streaming, dns cache updates. Only has meaning at server startup.", 0) \ - M(SettingUInt64, background_distributed_schedule_pool_size, 16, "Number of threads performing background tasks for distributed sends. Only has meaning at server startup.", 0) \ + 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_final_threads, 16, "The maximum number of threads to read from table with FINAL.", 0) \ + M(MaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.", 0) \ + M(MaxThreads, max_alter_threads, 0, "The maximum number of threads to execute the ALTER requests. By default, it is determined automatically.", 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, 262144, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)", 0) \ + M(UInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \ + M(Seconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \ + M(Milliseconds, connect_timeout_with_failover_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS, "Connection timeout for selecting first healthy replica.", 0) \ + M(Milliseconds, connect_timeout_with_failover_secure_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_SECURE_MS, "Connection timeout for selecting first healthy replica (for secure connections).", 0) \ + M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \ + M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \ + M(Seconds, tcp_keep_alive_timeout, 0, "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, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.", 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, 512*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \ + M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ + M(Bool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \ + M(Bool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \ + M(UInt64, background_buffer_flush_schedule_pool_size, 16, "Number of threads performing background flush for tables with Buffer engine. Only has meaning at server startup.", 0) \ + M(UInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \ + M(UInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \ + M(UInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables, kafka streaming, dns cache updates. Only has meaning at server startup.", 0) \ + M(UInt64, background_distributed_schedule_pool_size, 16, "Number of threads performing background tasks for distributed sends. Only has meaning at server startup.", 0) \ \ - M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \ - M(SettingMilliseconds, distributed_directory_monitor_max_sleep_time_ms, 30000, "Maximum sleep time for StorageDistributed DirectoryMonitors, it limits exponential growth too.", 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(SettingBool, distributed_directory_monitor_batch_inserts, false, "Should StorageDistributed DirectoryMonitors try to batch individual inserts into bigger ones.", 0) \ + M(Bool, distributed_directory_monitor_batch_inserts, false, "Should StorageDistributed DirectoryMonitors try to batch individual inserts into bigger ones.", 0) \ \ - M(SettingBool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.", 0) \ + M(Bool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.", 0) \ \ - M(SettingMilliseconds, insert_in_memory_parts_timeout, 600000, "", 0) \ + M(Milliseconds, insert_in_memory_parts_timeout, 600000, "", 0) \ \ - 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.", 0) \ - 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.", 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(UInt64, replication_alter_columns_timeout, 60, "Wait for actions to change the table structure within the specified number of seconds. 0 - wait unlimited time.", 0) \ \ - M(SettingLoadBalancing, load_balancing, LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.", 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(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.", IMPORTANT) \ - M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.", 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(SettingBool, 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(SettingBool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ - M(SettingUInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ - M(SettingUInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \ - 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. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ - M(SettingBool, distributed_aggregation_memory_efficient, false, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ - 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'.", 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(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ + M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ + M(UInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \ + M(UInt64, 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. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ + M(Bool, distributed_aggregation_memory_efficient, false, "Is the memory-saving mode of distributed aggregation enabled.", 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(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.", 0) \ - M(SettingUInt64, parallel_replicas_count, 0, "", 0) \ - M(SettingUInt64, parallel_replica_offset, 0, "", 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(UInt64, parallel_replicas_count, 0, "", 0) \ + M(UInt64, parallel_replica_offset, 0, "", 0) \ \ - M(SettingSpecialSort, special_sort, SpecialSort::NOT_SPECIFIED, "Specifies a sorting algorithm which will be using in ORDER BY query.", 0) \ + M(SpecialSort, special_sort, SpecialSort::NOT_SPECIFIED, "Specifies a sorting algorithm which will be using in ORDER BY query.", 0) \ \ - M(SettingBool, skip_unavailable_shards, false, "If 1, 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(Bool, skip_unavailable_shards, false, "If 1, 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(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.", 0) \ - M(SettingBool, parallel_distributed_insert_select, false, "If true, distributed insert select query in the same cluster will be processed on local tables on every shard", 0) \ - M(SettingBool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avodiing costly aggregation on the initiator server).", 0) \ - M(SettingBool, 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(SettingUInt64, 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(SettingUInt64, optimize_skip_unused_shards_nesting, 0, "Same as optimize_skip_unused_shards, but accept nesting level until which it will work.", 0) \ - M(SettingUInt64, 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, 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.", 0) \ + M(Bool, parallel_distributed_insert_select, false, "If true, distributed insert select query in the same cluster will be processed on local tables on every shard", 0) \ + M(Bool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avodiing costly aggregation on the initiator server).", 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(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(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \ - M(SettingUInt64, min_chunk_bytes_for_parallel_parsing, (10 * 1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 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(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.", 0) \ - M(SettingUInt64, 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(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.", 0) \ - M(SettingUInt64, 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(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.", 0) \ - M(SettingUInt64, 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(SettingUInt64, 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, 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(SettingUInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 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(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ", 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(SettingUInt64, 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(SettingUInt64, 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(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(SettingBool, force_index_by_date, 0, "Throw an exception if there is a partition key in a table, and it is not used.", 0) \ - M(SettingBool, force_primary_key, 0, "Throw an exception if there is primary key in a table, and it is not used.", 0) \ + M(Bool, force_index_by_date, 0, "Throw an exception if there is a partition key in a table, and it is not used.", 0) \ + M(Bool, force_primary_key, 0, "Throw an exception if there is primary key in a table, and it is not used.", 0) \ \ - M(SettingFloat, 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(SettingFloat, 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(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(SettingString, network_compression_method, "LZ4", "Allows you to select the method of data compression when writing.", 0) \ + M(String, network_compression_method, "LZ4", "Allows you to select the method of data compression when writing.", 0) \ \ - M(SettingInt64, network_zstd_compression_level, 1, "Allows you to select the level of ZSTD compression.", 0) \ + M(Int64, network_zstd_compression_level, 1, "Allows you to select the level of ZSTD compression.", 0) \ \ - M(SettingUInt64, priority, 0, "Priority of the query. 1 - the highest, higher value - lower priority; 0 - do not use priorities.", 0) \ - M(SettingInt64, 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(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(SettingBool, log_queries, 1, "Log requests and write the log to the system table.", 0) \ - M(SettingLogQueriesType, 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(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.", 0) \ + M(Bool, log_queries, 1, "Log requests and write the log to the system table.", 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(SettingDistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", IMPORTANT) \ + M(DistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", IMPORTANT) \ \ - M(SettingUInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \ + M(UInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \ \ - M(SettingBool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be preformed", 0) \ + M(Bool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be preformed", 0) \ \ - 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.", 0) \ - M(SettingMilliseconds, insert_quorum_timeout, 600000, "", 0) \ - 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; do not read the parts that have not yet been written with the quorum.", 0) \ - 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.", 0) \ - M(SettingMilliseconds, 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(SettingUInt64, 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(SettingMilliseconds, 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(SettingUInt64, 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(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(Milliseconds, insert_quorum_timeout, 600000, "", 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(SettingFloat, memory_tracker_fault_probability, 0., "For testing of `exception safety` - throw an exception every time you allocate memory with the specified probability.", 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(SettingBool, enable_http_compression, 0, "Compress the result if the client over HTTP said that it understands data compressed by gzip or deflate.", 0) \ - 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.", 0) \ + M(Bool, enable_http_compression, 0, "Compress the result if the client over HTTP said that it understands data compressed by gzip or deflate.", 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(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.", 0) \ + M(Bool, 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.", 0) \ \ - M(SettingString, count_distinct_implementation, "uniqExact", "What aggregate function to use for implementation of count(DISTINCT ...)", 0) \ + M(String, count_distinct_implementation, "uniqExact", "What aggregate function to use for implementation of count(DISTINCT ...)", 0) \ \ - M(SettingBool, add_http_cors_header, false, "Write add http CORS header.", 0) \ + M(Bool, add_http_cors_header, false, "Write add http CORS header.", 0) \ \ - M(SettingUInt64, 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(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(SettingBool, use_client_time_zone, false, "Use client timezone for interpreting DateTime string values, instead of adopting server timezone.", 0) \ + M(Bool, use_client_time_zone, false, "Use client timezone for interpreting DateTime string values, instead of adopting server timezone.", 0) \ \ - M(SettingBool, 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, 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(SettingUInt64, http_headers_progress_interval_ms, 100, "Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval.", 0) \ + M(UInt64, http_headers_progress_interval_ms, 100, "Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval.", 0) \ \ - M(SettingBool, fsync_metadata, 1, "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, fsync_metadata, 1, "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(SettingBool, join_use_nulls, 0, "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(Bool, join_use_nulls, 0, "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(SettingJoinStrictness, 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(SettingBool, 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(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(SettingUInt64, preferred_block_size_bytes, 1000000, "", 0) \ + M(UInt64, preferred_block_size_bytes, 1000000, "", 0) \ \ - 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). Zero means do not take delay into account.", 0) \ - M(SettingBool, fallback_to_stale_replicas_for_distributed_queries, 1, "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(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.", 0) \ + 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, 1, "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(SettingBool, 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(SettingUInt64, 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(SettingInt64, 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.", 0) \ - M(SettingMilliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ - M(SettingMilliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 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.", 0) \ + M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ + M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ \ - M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.", 0) \ - M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \ - M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \ - M(SettingSeconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout", 0) \ - M(SettingBool, 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(SettingBool, 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(SettingBool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \ - M(SettingBool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \ - M(SettingBool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ - M(SettingBool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ - M(SettingUInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.", 0) \ - M(SettingUInt64, query_profiler_real_time_period_ns, 1000000000, "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(SettingUInt64, query_profiler_cpu_time_period_ns, 1000000000, "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(SettingBool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \ - M(SettingString, 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, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.", 0) \ + M(Seconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \ + M(Seconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \ + M(Seconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout", 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_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ + M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ + M(UInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.", 0) \ + M(UInt64, query_profiler_real_time_period_ns, 1000000000, "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, 1000000000, "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) \ \ \ /** Limits during query execution are part of the settings. \ @@ -242,237 +242,237 @@ struct Settings : public SettingsCollection * Almost all limits apply to each stream individually. \ */ \ \ - M(SettingUInt64, 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(SettingUInt64, 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(SettingOverflowMode, read_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 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(SettingUInt64, max_rows_to_group_by, 0, "", 0) \ - M(SettingOverflowModeGroupBy, group_by_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ - M(SettingUInt64, max_bytes_before_external_group_by, 0, "", 0) \ + M(UInt64, max_rows_to_group_by, 0, "", 0) \ + M(OverflowModeGroupBy, group_by_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ + M(UInt64, max_bytes_before_external_group_by, 0, "", 0) \ \ - M(SettingUInt64, max_rows_to_sort, 0, "", 0) \ - M(SettingUInt64, max_bytes_to_sort, 0, "", 0) \ - M(SettingOverflowMode, sort_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ - M(SettingUInt64, max_bytes_before_external_sort, 0, "", 0) \ - M(SettingUInt64, 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_rows_to_sort, 0, "", 0) \ + M(UInt64, max_bytes_to_sort, 0, "", 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(SettingUInt64, max_result_rows, 0, "Limit on result size in rows. Also checked for intermediate data sent from remote servers.", 0) \ - M(SettingUInt64, max_result_bytes, 0, "Limit on result size in bytes (uncompressed). Also checked for intermediate data sent from remote servers.", 0) \ - M(SettingOverflowMode, result_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ + M(UInt64, max_result_rows, 0, "Limit on result size in rows. Also checked for intermediate data sent from remote servers.", 0) \ + M(UInt64, max_result_bytes, 0, "Limit on result size in bytes (uncompressed). Also checked for intermediate data sent from remote servers.", 0) \ + M(OverflowMode, result_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ /* TODO: Check also when merging and finalizing aggregate functions. */ \ - M(SettingSeconds, max_execution_time, 0, "", 0) \ - M(SettingOverflowMode, timeout_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ + M(Seconds, max_execution_time, 0, "", 0) \ + M(OverflowMode, timeout_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ - M(SettingUInt64, min_execution_speed, 0, "Minimum number of execution rows per second.", 0) \ - M(SettingUInt64, max_execution_speed, 0, "Maximum number of execution rows per second.", 0) \ - M(SettingUInt64, min_execution_speed_bytes, 0, "Minimum number of execution bytes per second.", 0) \ - M(SettingUInt64, max_execution_speed_bytes, 0, "Maximum number of execution bytes per second.", 0) \ - M(SettingSeconds, timeout_before_checking_execution_speed, 10, "Check that the speed is not too low after the specified time has elapsed.", 0) \ + M(UInt64, min_execution_speed, 0, "Minimum number of execution rows per second.", 0) \ + M(UInt64, max_execution_speed, 0, "Maximum number of execution rows per second.", 0) \ + M(UInt64, min_execution_speed_bytes, 0, "Minimum number of execution bytes per second.", 0) \ + M(UInt64, max_execution_speed_bytes, 0, "Maximum number of execution bytes per second.", 0) \ + M(Seconds, timeout_before_checking_execution_speed, 10, "Check that the speed is not too low after the specified time has elapsed.", 0) \ \ - M(SettingUInt64, max_columns_to_read, 0, "", 0) \ - M(SettingUInt64, max_temporary_columns, 0, "", 0) \ - M(SettingUInt64, max_temporary_non_const_columns, 0, "", 0) \ + M(UInt64, max_columns_to_read, 0, "", 0) \ + M(UInt64, max_temporary_columns, 0, "", 0) \ + M(UInt64, max_temporary_non_const_columns, 0, "", 0) \ \ - M(SettingUInt64, max_subquery_depth, 100, "", 0) \ - M(SettingUInt64, max_pipeline_depth, 1000, "", 0) \ - M(SettingUInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.", 0) \ - M(SettingUInt64, max_ast_elements, 50000, "Maximum size of query syntax tree in number of nodes. Checked after parsing.", 0) \ - M(SettingUInt64, 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, max_subquery_depth, 100, "", 0) \ + M(UInt64, max_pipeline_depth, 1000, "", 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(SettingUInt64, 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, 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(SettingUInt64, max_rows_in_set, 0, "Maximum size of the set (in number of elements) resulting from the execution of the IN section.", 0) \ - M(SettingUInt64, max_bytes_in_set, 0, "Maximum size of the set (in bytes in memory) resulting from the execution of the IN section.", 0) \ - M(SettingOverflowMode, set_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 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(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).", 0) \ - M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).", 0) \ - M(SettingOverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ - M(SettingBool, 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(SettingJoinAlgorithm, join_algorithm, JoinAlgorithm::HASH, "Specify join algorithm: 'auto', 'hash', 'partial_merge', 'prefer_partial_merge'. 'auto' tries to change HashJoin to MergeJoin on the fly to avoid out of memory.", 0) \ - M(SettingBool, partial_merge_join_optimizations, true, "Enable optimizations in partial merge join", 0) \ - M(SettingUInt64, default_max_bytes_in_join, 1000000000, "Maximum size of right-side table if limit is required but max_bytes_in_join is not set.", 0) \ - M(SettingUInt64, partial_merge_join_left_table_buffer_bytes, 32000000, "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. In current version work only with 'partial_merge_join_optimizations = 1'.", 0) \ - M(SettingUInt64, 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(SettingUInt64, 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(SettingString, temporary_files_codec, "LZ4", "Set compression codec for temporary files (sort and join on disk). I.e. LZ4, NONE.", 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(JoinAlgorithm, join_algorithm, JoinAlgorithm::HASH, "Specify join algorithm: 'auto', 'hash', 'partial_merge', 'prefer_partial_merge'. 'auto' tries to change HashJoin to MergeJoin on the fly to avoid out of memory.", 0) \ + M(Bool, partial_merge_join_optimizations, true, "Enable optimizations in partial merge join", 0) \ + M(UInt64, default_max_bytes_in_join, 1000000000, "Maximum size of right-side table if limit is required but max_bytes_in_join is not set.", 0) \ + M(UInt64, partial_merge_join_left_table_buffer_bytes, 32000000, "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. In current version work only with 'partial_merge_join_optimizations = 1'.", 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(SettingUInt64, 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(SettingUInt64, 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(SettingOverflowMode, transfer_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 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(SettingUInt64, max_rows_in_distinct, 0, "Maximum number of elements during execution of DISTINCT.", 0) \ - M(SettingUInt64, max_bytes_in_distinct, 0, "Maximum total size of state (in uncompressed bytes) in memory for the execution of DISTINCT.", 0) \ - M(SettingOverflowMode, distinct_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(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \ - M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \ - M(SettingUInt64, 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(SettingUInt64, memory_profiler_step, 0, "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(SettingFloat, 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, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \ + M(UInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 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, 0, "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(SettingUInt64, 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(SettingUInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.", 0) \ - M(SettingUInt64, 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(SettingUInt64, 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(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(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.", 0) \ - M(SettingBool, log_query_settings, true, "Log query settings into the query_log.", 0) \ - M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \ - M(SettingLogsLevel, 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(SettingBool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.", 0) \ - M(SettingBool, enable_optimize_predicate_expression_to_final_subquery, 1, "Allow push predicate to final subquery.", 0) \ - M(SettingBool, allow_push_predicate_when_subquery_contains_with, 1, "Allows push predicate when subquery contains WITH clause", 0) \ -\ - M(SettingUInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.", 0) \ - M(SettingBool, 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(SettingBool, decimal_check_overflow, true, "Check overflow of decimal arithmetic/comparison operations", 0) \ + M(Bool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.", 0) \ + M(Bool, log_query_settings, true, "Log query settings into the query_log.", 0) \ + M(Bool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 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(Bool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.", 0) \ + M(Bool, enable_optimize_predicate_expression_to_final_subquery, 1, "Allow push predicate to final subquery.", 0) \ + M(Bool, allow_push_predicate_when_subquery_contains_with, 1, "Allows push predicate when subquery contains WITH clause", 0) \ \ - M(SettingBool, prefer_localhost_replica, 1, "1 - always send query to local replica, if it exists. 0 - choose replica to send query between local and remote ones according to load_balancing", 0) \ - M(SettingUInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.", 0) \ - M(SettingUInt64, 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(SettingBool, calculate_text_stack_trace, 1, "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(SettingBool, allow_ddl, true, "If it is set to true, then a user is allowed to executed DDL queries.", 0) \ - M(SettingBool, parallel_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.", 0) \ - M(SettingBool, enable_debug_queries, false, "Enables debug queries such as AST.", 0) \ - M(SettingBool, 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(SettingBool, optimize_read_in_order, true, "Enable ORDER BY optimization for reading data in corresponding order in MergeTree tables.", 0) \ - M(SettingBool, optimize_aggregation_in_order, false, "Enable GROUP BY optimization for aggregating data in corresponding order in MergeTree tables.", 0) \ - M(SettingUInt64, 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(SettingBool, 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(SettingBool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.", 0) \ - M(SettingBool, 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' and 'odbc' table functions.", 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(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \ - M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \ - M(SettingBool, 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(Bool, prefer_localhost_replica, 1, "1 - always send query to local replica, if it exists. 0 - choose replica to send query 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, 1, "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_debug_queries, false, "Enables debug queries such as AST.", 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_aggregation_in_order, false, "Enable GROUP BY optimization for aggregating data in corresponding order in MergeTree tables.", 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' and 'odbc' table functions.", 0) \ \ - M(SettingUInt64, 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(SettingBool, check_query_single_value_result, true, "Return check query result as single 1/0 value", 0) \ - M(SettingBool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries", 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(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(SettingSeconds, 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(SettingUInt64, 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(SettingUInt64, distributed_replica_max_ignored_errors, 0, "Number of errors that will be ignored while choosing replicas", 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(Bool, check_query_single_value_result, true, "Return check query result as single 1/0 value", 0) \ + M(Bool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries", 0) \ \ - M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \ - M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.", 0) \ - M(SettingUInt64, 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(SettingUInt64, 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(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(SettingDefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \ - M(SettingBool, allow_experimental_database_atomic, false, "Allow to create database with Engine=Atomic.", 0) \ - M(SettingBool, show_table_uuid_in_table_create_query_if_not_nil, true, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \ - M(SettingBool, 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(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ - M(SettingUInt64, 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(SettingBool, optimize_move_functions_out_of_any, true, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ - M(SettingBool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ - M(SettingBool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \ - M(SettingBool, optimize_duplicate_order_by_and_distinct, true, "Remove duplicate ORDER BY and DISTINCT if it's possible", 0) \ - M(SettingBool, optimize_redundant_functions_in_order_by, true, "Remove functions from ORDER BY if its argument is also in ORDER BY", 0) \ - M(SettingBool, 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(SettingBool, 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(SettingBool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 0) \ - M(SettingBool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \ - M(SettingBool, 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, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \ + M(Seconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.", 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(SettingBool, 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(SettingBool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \ - M(SettingUInt64, multiple_joins_rewriter_version, 2, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \ - M(SettingBool, 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(SettingUInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ - M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \ - M(SettingBool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \ - M(SettingBool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \ - M(SettingSeconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \ - M(SettingBool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \ - M(SettingString, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \ + M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \ + M(Bool, allow_experimental_database_atomic, false, "Allow to create database with Engine=Atomic.", 0) \ + M(Bool, show_table_uuid_in_table_create_query_if_not_nil, true, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 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(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_move_functions_out_of_any, true, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ + M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 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, 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(SettingBool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ - M(SettingBool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ - M(SettingBool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ - M(SettingBool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 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, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \ + M(UInt64, multiple_joins_rewriter_version, 2, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 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(Seconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \ + M(Bool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \ + M(Bool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \ + M(Seconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \ + M(Bool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \ + M(String, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \ + \ + M(Bool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ + M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \ + M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ + M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ - M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13", 0) \ - M(SettingBool, compile, false, "Whether query compilation is enabled. Will be removed after 2020-03-13", 0) \ - M(SettingUInt64, min_count_to_compile, 0, "Obsolete setting, does nothing. Will be removed after 2020-03-13", 0) \ - M(SettingBool, allow_experimental_multiple_joins_emulation, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ - M(SettingBool, allow_experimental_cross_to_join_conversion, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ - M(SettingBool, allow_experimental_data_skipping_indices, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ - M(SettingBool, merge_tree_uniform_read_distribution, true, "Obsolete setting, does nothing. Will be removed after 2020-05-20", 0) \ - M(SettingUInt64, mark_cache_min_lifetime, 0, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ - M(SettingBool, partial_merge_join, false, "Obsolete. Use join_algorithm='prefer_partial_merge' instead.", 0) \ - M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \ + M(Bool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13", 0) \ + M(Bool, compile, false, "Whether query compilation is enabled. Will be removed after 2020-03-13", 0) \ + M(UInt64, min_count_to_compile, 0, "Obsolete setting, does nothing. Will be removed after 2020-03-13", 0) \ + M(Bool, allow_experimental_multiple_joins_emulation, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ + M(Bool, allow_experimental_cross_to_join_conversion, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ + M(Bool, allow_experimental_data_skipping_indices, true, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ + M(Bool, merge_tree_uniform_read_distribution, true, "Obsolete setting, does nothing. Will be removed after 2020-05-20", 0) \ + M(UInt64, mark_cache_min_lifetime, 0, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ + M(Bool, partial_merge_join, false, "Obsolete. Use join_algorithm='prefer_partial_merge' instead.", 0) \ + M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \ \ - M(SettingBool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \ - M(SettingBool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) + M(Bool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \ + M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) #define FORMAT_FACTORY_SETTINGS(M) \ - M(SettingChar, 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(SettingBool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.", 0) \ - M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.", 0) \ - M(SettingBool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \ - M(SettingBool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N", 0) \ - M(SettingBool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \ - M(SettingBool, input_format_with_names_use_header, true, "For TSVWithNames and CSVWithNames 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(SettingBool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \ - M(SettingBool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \ - M(SettingBool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", IMPORTANT) \ - M(SettingBool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ - M(SettingBool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 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, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.", 0) \ + M(Bool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.", 0) \ + M(Bool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \ + M(Bool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N", 0) \ + M(Bool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \ + M(Bool, input_format_with_names_use_header, true, "For TSVWithNames and CSVWithNames 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_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \ + M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \ + M(Bool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", IMPORTANT) \ + M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ + M(Bool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \ \ - M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \ + M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \ \ - M(SettingBool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \ - M(SettingBool, 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(SettingBool, 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(SettingBool, 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(SettingBool, 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(SettingURI, format_avro_schema_registry_url, {}, "For AvroConfluent format: Confluent Schema Registry URL.", 0) \ + M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 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(URI, format_avro_schema_registry_url, {}, "For AvroConfluent format: Confluent Schema Registry URL.", 0) \ \ - M(SettingBool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ + M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ \ - M(SettingBool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \ + M(Bool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \ \ - M(SettingBool, 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(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(SettingUInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \ - M(SettingUInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \ - M(SettingUInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \ - M(SettingBool, output_format_pretty_color, true, "Use ANSI escape sequences to paint colors in Pretty formats", 0) \ - M(SettingString, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \ - M(SettingUInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.", 0) \ - M(SettingString, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ - M(SettingUInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ - M(SettingBool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \ + M(UInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \ + M(UInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \ + M(UInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \ + M(Bool, output_format_pretty_color, true, "Use ANSI escape sequences to paint colors in Pretty formats", 0) \ + M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \ + M(UInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.", 0) \ + M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ + M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ + M(Bool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \ \ - M(SettingUInt64, 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(SettingFloat, 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(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(SettingString, format_schema, "", "Schema identifier (used by schema-based formats)", 0) \ - M(SettingString, format_template_resultset, "", "Path to file which contains format string for result set (for Template format)", 0) \ - M(SettingString, format_template_row, "", "Path to file which contains format string for rows (for Template format)", 0) \ - M(SettingString, format_template_rows_between_delimiter, "\n", "Delimiter between rows (for Template format)", 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(SettingString, format_custom_escaping_rule, "Escaped", "Field escaping rule (for CustomSeparated format)", 0) \ - M(SettingString, format_custom_field_delimiter, "\t", "Delimiter between fields (for CustomSeparated format)", 0) \ - M(SettingString, format_custom_row_before_delimiter, "", "Delimiter before field of the first column (for CustomSeparated format)", 0) \ - M(SettingString, format_custom_row_after_delimiter, "\n", "Delimiter after field of the last column (for CustomSeparated format)", 0) \ - M(SettingString, format_custom_row_between_delimiter, "", "Delimiter between rows (for CustomSeparated format)", 0) \ - M(SettingString, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)", 0) \ - M(SettingString, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \ + M(String, format_custom_escaping_rule, "Escaped", "Field escaping rule (for CustomSeparated format)", 0) \ + M(String, format_custom_field_delimiter, "\t", "Delimiter between fields (for CustomSeparated format)", 0) \ + M(String, format_custom_row_before_delimiter, "", "Delimiter before field of the first column (for CustomSeparated format)", 0) \ + M(String, format_custom_row_after_delimiter, "\n", "Delimiter after field of the last column (for CustomSeparated format)", 0) \ + M(String, format_custom_row_between_delimiter, "", "Delimiter between rows (for CustomSeparated format)", 0) \ + M(String, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)", 0) \ + M(String, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \ \ - M(SettingString, format_regexp, "", "Regular expression (for Regexp format)", 0) \ - M(SettingString, format_regexp_escaping_rule, "Escaped", "Field escaping rule (for Regexp format)", 0) \ - M(SettingBool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \ + M(String, format_regexp, "", "Regular expression (for Regexp format)", 0) \ + M(String, format_regexp_escaping_rule, "Escaped", "Field escaping rule (for Regexp format)", 0) \ + M(Bool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \ \ - M(SettingBool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ - M(SettingBool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \ - M(SettingBool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ + M(Bool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ + M(Bool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \ + M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ #define LIST_OF_SETTINGS(M) \ COMMON_SETTINGS(M) \ diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h index 1098733cbe1..08168b79e64 100644 --- a/src/Core/SettingsCollection.h +++ b/src/Core/SettingsCollection.h @@ -353,9 +353,9 @@ enum class SettingsBinaryFormat * struct MySettings : public SettingsCollection * { * # define APPLY_FOR_MYSETTINGS(M) \ - * M(SettingUInt64, a, 100, "Description of a", 0) \ - * M(SettingFloat, f, 3.11, "Description of f", IMPORTANT) // IMPORTANT - means the setting can't be ignored by older versions) \ - * M(SettingString, s, "default", "Description of s", 0) + * M(UInt64, a, 100, "Description of a", 0) \ + * M(Float, f, 3.11, "Description of f", IMPORTANT) // IMPORTANT - means the setting can't be ignored by older versions) \ + * M(String, s, "default", "Description of s", 0) * * DECLARE_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS) * }; @@ -586,5 +586,5 @@ public: LIST_OF_SETTINGS_MACRO(DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_) #define DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ - TYPE NAME {DEFAULT}; + Setting##TYPE NAME {DEFAULT}; } diff --git a/src/Core/SettingsCollectionImpl.h b/src/Core/SettingsCollectionImpl.h index f0854f11b8a..2871816f43f 100644 --- a/src/Core/SettingsCollectionImpl.h +++ b/src/Core/SettingsCollectionImpl.h @@ -342,8 +342,8 @@ void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFo static void NAME##_setField(Derived & collection, const Field & value) { collection.NAME.set(value); } \ static void NAME##_serialize(const Derived & collection, WriteBuffer & buf, SettingsBinaryFormat format) { collection.NAME.serialize(buf, format); } \ static void NAME##_deserialize(Derived & collection, ReadBuffer & buf, SettingsBinaryFormat format) { collection.NAME.deserialize(buf, format); } \ - static String NAME##_valueToString(const Field & value) { TYPE temp{DEFAULT}; temp.set(value); return temp.toString(); } \ - static Field NAME##_valueToCorrespondingType(const Field & value) { TYPE temp{DEFAULT}; temp.set(value); return temp.toField(); } \ + static String NAME##_valueToString(const Field & value) { Setting##TYPE temp{DEFAULT}; temp.set(value); return temp.toString(); } \ + static Field NAME##_valueToCorrespondingType(const Field & value) { Setting##TYPE temp{DEFAULT}; temp.set(value); return temp.toField(); } \ #define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index e65522b3606..7cf9b90e081 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -16,25 +16,25 @@ struct KafkaSettings : public SettingsCollection #define KAFKA_RELATED_SETTINGS(M) \ - M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.", 0) \ - M(SettingString, kafka_topic_list, "", "A list of Kafka topics.", 0) \ - M(SettingString, kafka_group_name, "", "Client group id string. All Kafka consumers sharing the same group.id belong to the same group.", 0) \ - M(SettingString, kafka_client_id, "", "Client identifier.", 0) \ - M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.", 0) \ - M(SettingBool, kafka_commit_every_batch, false, "Commit every consumed and handled batch instead of a single commit after writing a whole block", 0) \ + M(String, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.", 0) \ + M(String, kafka_topic_list, "", "A list of Kafka topics.", 0) \ + M(String, kafka_group_name, "", "Client group id string. All Kafka consumers sharing the same group.id belong to the same group.", 0) \ + M(String, kafka_client_id, "", "Client identifier.", 0) \ + M(UInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.", 0) \ + M(Bool, kafka_commit_every_batch, false, "Commit every consumed and handled batch instead of a single commit after writing a whole block", 0) \ /* default is stream_poll_timeout_ms */ \ - M(SettingMilliseconds, kafka_poll_timeout_ms, 0, "Timeout for single poll from Kafka.", 0) \ + M(Milliseconds, kafka_poll_timeout_ms, 0, "Timeout for single poll from Kafka.", 0) \ /* default is min(max_block_size, kafka_max_block_size)*/ \ - M(SettingUInt64, kafka_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single Kafka poll.", 0) \ + M(UInt64, kafka_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single Kafka poll.", 0) \ /* default is = min_insert_block_size / kafka_num_consumers */ \ - M(SettingUInt64, kafka_max_block_size, 0, "Number of row collected by poll(s) for flushing data from Kafka.", 0) \ + M(UInt64, kafka_max_block_size, 0, "Number of row collected by poll(s) for flushing data from Kafka.", 0) \ /* default is stream_flush_interval_ms */ \ - M(SettingMilliseconds, kafka_flush_interval_ms, 0, "Timeout for flushing data from Kafka.", 0) \ + M(Milliseconds, kafka_flush_interval_ms, 0, "Timeout for flushing data from Kafka.", 0) \ /* those are mapped to format factory settings */ \ - M(SettingString, kafka_format, "", "The message format for Kafka engine.", 0) \ - M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.", 0) \ - M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \ - M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block", 0) + M(String, kafka_format, "", "The message format for Kafka engine.", 0) \ + M(Char, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.", 0) \ + M(String, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \ + M(UInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block", 0) /** TODO: */ /* https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md */ diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 833425ff592..410b8caee62 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -26,84 +26,84 @@ struct MergeTreeSettings : public SettingsCollection { #define LIST_OF_MERGE_TREE_SETTINGS(M) \ - M(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ + M(UInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ \ /** Data storing format settings. */ \ - M(SettingUInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ - M(SettingUInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ - M(SettingUInt64, min_bytes_for_compact_part, 0, "Experimental. Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ - M(SettingUInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ - M(SettingBool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ - M(SettingUInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ + M(UInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ + M(UInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ + M(UInt64, min_bytes_for_compact_part, 0, "Experimental. Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ + M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ + M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ + M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ \ /** Merge settings. */ \ - M(SettingUInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ - M(SettingUInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).", 0) \ - M(SettingUInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \ - M(SettingUInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ - M(SettingUInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ - M(SettingUInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ - M(SettingUInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ - M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ - M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ - M(SettingSeconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ + M(UInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ + M(UInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).", 0) \ + M(UInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \ + M(UInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(UInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ + M(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + M(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ + M(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ + M(Seconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ \ /** Inserts settings. */ \ - M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ - M(SettingUInt64, parts_to_throw_insert, 300, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.", 0) \ - M(SettingUInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.", 0) \ - M(SettingUInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \ + M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ + M(UInt64, parts_to_throw_insert, 300, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.", 0) \ + M(UInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.", 0) \ + M(UInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \ \ /** Replication settings. */ \ - M(SettingUInt64, replicated_deduplication_window, 100, "How many last blocks of hashes should be kept in ZooKeeper (old blocks will be deleted).", 0) \ - M(SettingUInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ - M(SettingUInt64, max_replicated_logs_to_keep, 100, "How many records may be in log, if there is inactive replica.", 0) \ - M(SettingUInt64, min_replicated_logs_to_keep, 10, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \ - M(SettingSeconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ - M(SettingUInt64, prefer_fetch_merged_part_size_threshold, 10ULL * 1024 * 1024 * 1024, "If sum size of parts exceeds this threshold and time passed after replication log entry creation is greater than \"prefer_fetch_merged_part_time_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ - M(SettingBool, always_fetch_merged_part, 0, "If true, replica never merge parts and always download merged parts from other replicas.", 0) \ - M(SettingUInt64, max_suspicious_broken_parts, 10, "Max broken parts, if more - deny automatic deletion.", 0) \ - M(SettingUInt64, max_files_to_modify_in_alter_columns, 75, "Not apply ALTER if number of files for modification(deletion, addition) more than this.", 0) \ - M(SettingUInt64, max_files_to_remove_in_alter_columns, 50, "Not apply ALTER, if number of files for deletion more than this.", 0) \ - M(SettingFloat, replicated_max_ratio_of_wrong_parts, 0.5, "If ratio of wrong parts to total number of parts is less than this - allow to start.", 0) \ - M(SettingUInt64, replicated_max_parallel_fetches, 0, "Limit parallel fetches.", 0) \ - M(SettingUInt64, replicated_max_parallel_fetches_for_table, 0, "Limit parallel fetches for one table.", 0) \ - M(SettingUInt64, replicated_max_parallel_fetches_for_host, DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT, "Limit parallel fetches from endpoint (actually pool size).", 0) \ - M(SettingUInt64, replicated_max_parallel_sends, 0, "Limit parallel sends.", 0) \ - M(SettingUInt64, replicated_max_parallel_sends_for_table, 0, "Limit parallel sends for one table.", 0) \ - M(SettingBool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.", 0) \ - M(SettingSeconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.", 0) \ + M(UInt64, replicated_deduplication_window, 100, "How many last blocks of hashes should be kept in ZooKeeper (old blocks will be deleted).", 0) \ + M(UInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ + M(UInt64, max_replicated_logs_to_keep, 100, "How many records may be in log, if there is inactive replica.", 0) \ + M(UInt64, min_replicated_logs_to_keep, 10, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \ + M(Seconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ + M(UInt64, prefer_fetch_merged_part_size_threshold, 10ULL * 1024 * 1024 * 1024, "If sum size of parts exceeds this threshold and time passed after replication log entry creation is greater than \"prefer_fetch_merged_part_time_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ + M(Bool, always_fetch_merged_part, 0, "If true, replica never merge parts and always download merged parts from other replicas.", 0) \ + M(UInt64, max_suspicious_broken_parts, 10, "Max broken parts, if more - deny automatic deletion.", 0) \ + M(UInt64, max_files_to_modify_in_alter_columns, 75, "Not apply ALTER if number of files for modification(deletion, addition) more than this.", 0) \ + M(UInt64, max_files_to_remove_in_alter_columns, 50, "Not apply ALTER, if number of files for deletion more than this.", 0) \ + M(Float, replicated_max_ratio_of_wrong_parts, 0.5, "If ratio of wrong parts to total number of parts is less than this - allow to start.", 0) \ + M(UInt64, replicated_max_parallel_fetches, 0, "Limit parallel fetches.", 0) \ + M(UInt64, replicated_max_parallel_fetches_for_table, 0, "Limit parallel fetches for one table.", 0) \ + M(UInt64, replicated_max_parallel_fetches_for_host, DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT, "Limit parallel fetches from endpoint (actually pool size).", 0) \ + M(UInt64, replicated_max_parallel_sends, 0, "Limit parallel sends.", 0) \ + M(UInt64, replicated_max_parallel_sends_for_table, 0, "Limit parallel sends for one table.", 0) \ + M(Bool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.", 0) \ + M(Seconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.", 0) \ \ /** Check delay of replicas settings. */ \ - M(SettingUInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \ - M(SettingUInt64, cleanup_delay_period, 30, "Period to clean old queue logs, blocks hashes and parts.", 0) \ - M(SettingUInt64, cleanup_delay_period_random_add, 10, "Add uniformly distributed value from 0 to x seconds to cleanup_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables.", 0) \ - M(SettingUInt64, min_relative_delay_to_close, 300, "Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.", 0) \ - M(SettingUInt64, min_absolute_delay_to_close, 0, "Minimal absolute delay to close, stop serving requests and not return Ok during status check.", 0) \ - M(SettingUInt64, enable_vertical_merge_algorithm, 1, "Enable usage of Vertical merge algorithm.", 0) \ - M(SettingUInt64, vertical_merge_algorithm_min_rows_to_activate, 16 * DEFAULT_MERGE_BLOCK_SIZE, "Minimal (approximate) sum of rows in merging parts to activate Vertical merge algorithm.", 0) \ - M(SettingUInt64, vertical_merge_algorithm_min_columns_to_activate, 11, "Minimal amount of non-PK columns to activate Vertical merge algorithm.", 0) \ + M(UInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \ + M(UInt64, cleanup_delay_period, 30, "Period to clean old queue logs, blocks hashes and parts.", 0) \ + M(UInt64, cleanup_delay_period_random_add, 10, "Add uniformly distributed value from 0 to x seconds to cleanup_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables.", 0) \ + M(UInt64, min_relative_delay_to_close, 300, "Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.", 0) \ + M(UInt64, min_absolute_delay_to_close, 0, "Minimal absolute delay to close, stop serving requests and not return Ok during status check.", 0) \ + M(UInt64, enable_vertical_merge_algorithm, 1, "Enable usage of Vertical merge algorithm.", 0) \ + M(UInt64, vertical_merge_algorithm_min_rows_to_activate, 16 * DEFAULT_MERGE_BLOCK_SIZE, "Minimal (approximate) sum of rows in merging parts to activate Vertical merge algorithm.", 0) \ + M(UInt64, vertical_merge_algorithm_min_columns_to_activate, 11, "Minimal amount of non-PK columns to activate Vertical merge algorithm.", 0) \ \ /** Compatibility settings */ \ - M(SettingBool, compatibility_allow_sampling_expression_not_in_primary_key, false, "Allow to create a table with sampling expression not in primary key. This is needed only to temporarily allow to run the server with wrong tables for backward compatibility.", 0) \ - M(SettingBool, use_minimalistic_checksums_in_zookeeper, true, "Use small format (dozens bytes) for part checksums in ZooKeeper instead of ordinary ones (dozens KB). Before enabling check that all replicas support new format.", 0) \ - M(SettingBool, use_minimalistic_part_header_in_zookeeper, true, "Store part header (checksums and columns) in a compact format and a single part znode instead of separate znodes (/columns and /checksums). This can dramatically reduce snapshot size in ZooKeeper. Before enabling check that all replicas support new format.", 0) \ - M(SettingUInt64, finished_mutations_to_keep, 100, "How many records about mutations that are done to keep. If zero, then keep all of them.", 0) \ - M(SettingUInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \ - M(SettingUInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \ - M(SettingInt64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.", 0) \ - M(SettingBool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ - M(SettingBool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)", 0) \ - M(SettingBool, enable_mixed_granularity_parts, 0, "Enable parts with adaptive and non adaptive granularity", 0) \ - M(SettingMaxThreads, max_part_loading_threads, 0, "The number of threads to load data parts at startup.", 0) \ - M(SettingMaxThreads, max_part_removal_threads, 0, "The number of threads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).", 0) \ - M(SettingUInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ - M(SettingString, storage_policy, "default", "Name of storage disk policy", 0) \ - M(SettingBool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ + M(Bool, compatibility_allow_sampling_expression_not_in_primary_key, false, "Allow to create a table with sampling expression not in primary key. This is needed only to temporarily allow to run the server with wrong tables for backward compatibility.", 0) \ + M(Bool, use_minimalistic_checksums_in_zookeeper, true, "Use small format (dozens bytes) for part checksums in ZooKeeper instead of ordinary ones (dozens KB). Before enabling check that all replicas support new format.", 0) \ + M(Bool, use_minimalistic_part_header_in_zookeeper, true, "Store part header (checksums and columns) in a compact format and a single part znode instead of separate znodes (/columns and /checksums). This can dramatically reduce snapshot size in ZooKeeper. Before enabling check that all replicas support new format.", 0) \ + M(UInt64, finished_mutations_to_keep, 100, "How many records about mutations that are done to keep. If zero, then keep all of them.", 0) \ + M(UInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \ + M(UInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \ + M(Int64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.", 0) \ + M(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ + M(Bool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)", 0) \ + M(Bool, enable_mixed_granularity_parts, 0, "Enable parts with adaptive and non adaptive granularity", 0) \ + M(MaxThreads, max_part_loading_threads, 0, "The number of threads to load data parts at startup.", 0) \ + M(MaxThreads, max_part_removal_threads, 0, "The number of threads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).", 0) \ + M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ + M(String, storage_policy, "default", "Name of storage disk policy", 0) \ + M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ \ /** Obsolete settings. Kept for backward compatibility only. */ \ - M(SettingUInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \ - M(SettingUInt64, check_delay_period, 60, "Obsolete setting, does nothing.", 0) \ + M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \ + M(UInt64, check_delay_period, 60, "Obsolete setting, does nothing.", 0) \ DECLARE_SETTINGS_COLLECTION(LIST_OF_MERGE_TREE_SETTINGS) diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 5cd52ed9ef7..b0ce82b0420 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -10,15 +10,15 @@ namespace DB { #define LIST_OF_RABBITMQ_SETTINGS(M) \ - M(SettingString, rabbitmq_host_port, "", "A host-port to connect to RabbitMQ server.", 0) \ - M(SettingString, rabbitmq_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \ - M(SettingString, rabbitmq_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \ - M(SettingString, rabbitmq_format, "", "The message format.", 0) \ - M(SettingChar, rabbitmq_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \ - M(SettingString, rabbitmq_exchange_type, "default", "The exchange type.", 0) \ - M(SettingUInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \ - M(SettingUInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \ - M(SettingBool, rabbitmq_transactional_channel, false, "Use transactional channel for publishing.", 0) \ + M(String, rabbitmq_host_port, "", "A host-port to connect to RabbitMQ server.", 0) \ + M(String, rabbitmq_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \ + M(String, rabbitmq_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \ + M(String, rabbitmq_format, "", "The message format.", 0) \ + M(Char, rabbitmq_row_delimiter, '\0', "The character to be considered as a delimiter.", 0) \ + M(String, rabbitmq_exchange_type, "default", "The exchange type.", 0) \ + M(UInt64, rabbitmq_num_consumers, 1, "The number of consumer channels per table.", 0) \ + M(UInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \ + M(Bool, rabbitmq_transactional_channel, false, "Use transactional channel for publishing.", 0) \ DECLARE_SETTINGS_COLLECTION(LIST_OF_RABBITMQ_SETTINGS) diff --git a/tests/queries/0_stateless/01221_system_settings.reference b/tests/queries/0_stateless/01221_system_settings.reference index 83c2c258c6e..52e845912cc 100644 --- a/tests/queries/0_stateless/01221_system_settings.reference +++ b/tests/queries/0_stateless/01221_system_settings.reference @@ -1,4 +1,4 @@ -send_timeout 300 0 \N \N 0 SettingSeconds -replicated_max_parallel_sends 0 0 Limit parallel sends. SettingUInt64 +send_timeout 300 0 \N \N 0 Seconds +replicated_max_parallel_sends 0 0 Limit parallel sends. UInt64 1 1 diff --git a/tests/queries/0_stateless/01221_system_settings.sql b/tests/queries/0_stateless/01221_system_settings.sql index 5f4755add61..226be55503d 100644 --- a/tests/queries/0_stateless/01221_system_settings.sql +++ b/tests/queries/0_stateless/01221_system_settings.sql @@ -2,33 +2,33 @@ select * from system.settings where name = 'send_timeout'; select * from system.merge_tree_settings order by length(description) limit 1; with [ - 'SettingSeconds', - 'SettingBool', - 'SettingInt64', - 'SettingString', - 'SettingChar', - 'SettingLogsLevel', - 'SettingURI', - 'SettingFloat', - 'SettingUInt64', - 'SettingMaxThreads', - 'SettingMilliseconds', - 'SettingJoinStrictness', - 'SettingJoinAlgorithm', - 'SettingOverflowMode', - 'SettingTotalsMode', - 'SettingLoadBalancing', - 'SettingOverflowModeGroupBy', - 'SettingDateTimeInputFormat', - 'SettingDistributedProductMode' + 'Seconds', + 'Bool', + 'Int64', + 'String', + 'Char', + 'LogsLevel', + 'URI', + 'Float', + 'UInt64', + 'MaxThreads', + 'Milliseconds', + 'JoinStrictness', + 'JoinAlgorithm', + 'OverflowMode', + 'TotalsMode', + 'LoadBalancing', + 'OverflowModeGroupBy', + 'DateTimeInputFormat', + 'DistributedProductMode' ] as types select hasAll(arrayDistinct(groupArray(type)), types) from system.settings; with [ - 'SettingSeconds', - 'SettingBool', - 'SettingInt64', - 'SettingString', - 'SettingFloat', - 'SettingUInt64', - 'SettingMaxThreads' + 'Seconds', + 'Bool', + 'Int64', + 'String', + 'Float', + 'UInt64', + 'MaxThreads' ] as types select hasAll(arrayDistinct(groupArray(type)), types) from system.merge_tree_settings; From d152e84a2ea947ae94dc53e31f52302fdf22bf12 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 17 Jul 2020 02:14:07 +0300 Subject: [PATCH 06/20] Rename Setting* => SettingField*. --- src/Core/SettingsCollection.cpp | 134 +++++++++++++++--------------- src/Core/SettingsCollection.h | 90 ++++++++++---------- src/Core/SettingsCollectionImpl.h | 4 +- 3 files changed, 114 insertions(+), 114 deletions(-) diff --git a/src/Core/SettingsCollection.cpp b/src/Core/SettingsCollection.cpp index ed1a16eab0e..5f90b8f2635 100644 --- a/src/Core/SettingsCollection.cpp +++ b/src/Core/SettingsCollection.cpp @@ -27,26 +27,26 @@ namespace ErrorCodes template -String SettingNumber::toString() const +String SettingFieldNumber::toString() const { return DB::toString(value); } template -Field SettingNumber::toField() const +Field SettingFieldNumber::toField() const { return value; } template -void SettingNumber::set(Type x) +void SettingFieldNumber::set(Type x) { value = x; changed = true; } template -void SettingNumber::set(const Field & x) +void SettingFieldNumber::set(const Field & x) { if (x.getType() == Field::Types::String) set(get(x)); @@ -55,13 +55,13 @@ void SettingNumber::set(const Field & x) } template -void SettingNumber::set(const String & x) +void SettingFieldNumber::set(const String & x) { set(parseWithSizeSuffix(x)); } template <> -void SettingNumber::set(const String & x) +void SettingFieldNumber::set(const String & x) { if (x.size() == 1) { @@ -85,7 +85,7 @@ void SettingNumber::set(const String & x) } template -void SettingNumber::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const +void SettingFieldNumber::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const { if (format >= SettingsBinaryFormat::STRINGS) { @@ -105,7 +105,7 @@ void SettingNumber::serialize(WriteBuffer & buf, SettingsBinaryFormat form } template -void SettingNumber::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) +void SettingFieldNumber::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) { if (format >= SettingsBinaryFormat::STRINGS) { @@ -136,31 +136,31 @@ void SettingNumber::deserialize(ReadBuffer & buf, SettingsBinaryFormat for } } -template struct SettingNumber; -template struct SettingNumber; -template struct SettingNumber; -template struct SettingNumber; +template struct SettingFieldNumber; +template struct SettingFieldNumber; +template struct SettingFieldNumber; +template struct SettingFieldNumber; -String SettingMaxThreads::toString() const +String SettingFieldMaxThreads::toString() const { /// Instead of the `auto` value, we output the actual value to make it easier to see. return is_auto ? ("auto(" + DB::toString(value) + ")") : DB::toString(value); } -Field SettingMaxThreads::toField() const +Field SettingFieldMaxThreads::toField() const { return is_auto ? 0 : value; } -void SettingMaxThreads::set(UInt64 x) +void SettingFieldMaxThreads::set(UInt64 x) { value = x ? x : getAutoValue(); is_auto = x == 0; changed = true; } -void SettingMaxThreads::set(const Field & x) +void SettingFieldMaxThreads::set(const Field & x) { if (x.getType() == Field::Types::String) set(get(x)); @@ -168,7 +168,7 @@ void SettingMaxThreads::set(const Field & x) set(applyVisitor(FieldVisitorConvertToNumber(), x)); } -void SettingMaxThreads::set(const String & x) +void SettingFieldMaxThreads::set(const String & x) { if (startsWith(x, "auto")) setAuto(); @@ -176,7 +176,7 @@ void SettingMaxThreads::set(const String & x) set(parse(x)); } -void SettingMaxThreads::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const +void SettingFieldMaxThreads::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const { if (format >= SettingsBinaryFormat::STRINGS) { @@ -187,7 +187,7 @@ void SettingMaxThreads::serialize(WriteBuffer & buf, SettingsBinaryFormat format writeVarUInt(is_auto ? 0 : value, buf); } -void SettingMaxThreads::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) +void SettingFieldMaxThreads::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) { if (format >= SettingsBinaryFormat::STRINGS) { @@ -202,45 +202,45 @@ void SettingMaxThreads::deserialize(ReadBuffer & buf, SettingsBinaryFormat forma set(x); } -void SettingMaxThreads::setAuto() +void SettingFieldMaxThreads::setAuto() { value = getAutoValue(); is_auto = true; } -UInt64 SettingMaxThreads::getAutoValue() +UInt64 SettingFieldMaxThreads::getAutoValue() { return getNumberOfPhysicalCPUCores(); } -template -String SettingTimespan::toString() const +template +String SettingFieldTimespan::toString() const { return DB::toString(value.totalMicroseconds() / microseconds_per_io_unit); } -template -Field SettingTimespan::toField() const +template +Field SettingFieldTimespan::toField() const { return value.totalMicroseconds() / microseconds_per_io_unit; } -template -void SettingTimespan::set(const Poco::Timespan & x) +template +void SettingFieldTimespan::set(const Poco::Timespan & x) { value = x; changed = true; } -template -void SettingTimespan::set(UInt64 x) +template +void SettingFieldTimespan::set(UInt64 x) { set(Poco::Timespan(x * microseconds_per_io_unit)); } -template -void SettingTimespan::set(const Field & x) +template +void SettingFieldTimespan::set(const Field & x) { if (x.getType() == Field::Types::String) set(get(x)); @@ -248,14 +248,14 @@ void SettingTimespan::set(const Field & x) set(applyVisitor(FieldVisitorConvertToNumber(), x)); } -template -void SettingTimespan::set(const String & x) +template +void SettingFieldTimespan::set(const String & x) { set(parse(x)); } -template -void SettingTimespan::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const +template +void SettingFieldTimespan::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const { if (format >= SettingsBinaryFormat::STRINGS) { @@ -266,8 +266,8 @@ void SettingTimespan::serialize(WriteBuffer & buf, SettingsBinaryFormat writeVarUInt(value.totalMicroseconds() / microseconds_per_io_unit, buf); } -template -void SettingTimespan::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) +template +void SettingFieldTimespan::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) { if (format >= SettingsBinaryFormat::STRINGS) { @@ -282,37 +282,37 @@ void SettingTimespan::deserialize(ReadBuffer & buf, SettingsBinaryForma set(x); } -template struct SettingTimespan; -template struct SettingTimespan; +template struct SettingFieldTimespan; +template struct SettingFieldTimespan; -String SettingString::toString() const +String SettingFieldString::toString() const { return value; } -Field SettingString::toField() const +Field SettingFieldString::toField() const { return value; } -void SettingString::set(const String & x) +void SettingFieldString::set(const String & x) { value = x; changed = true; } -void SettingString::set(const Field & x) +void SettingFieldString::set(const Field & x) { set(safeGet(x)); } -void SettingString::serialize(WriteBuffer & buf, SettingsBinaryFormat) const +void SettingFieldString::serialize(WriteBuffer & buf, SettingsBinaryFormat) const { writeStringBinary(value, buf); } -void SettingString::deserialize(ReadBuffer & buf, SettingsBinaryFormat) +void SettingFieldString::deserialize(ReadBuffer & buf, SettingsBinaryFormat) { String s; readStringBinary(s, buf); @@ -320,23 +320,23 @@ void SettingString::deserialize(ReadBuffer & buf, SettingsBinaryFormat) } -String SettingChar::toString() const +String SettingFieldChar::toString() const { return String(1, value); } -Field SettingChar::toField() const +Field SettingFieldChar::toField() const { return toString(); } -void SettingChar::set(char x) +void SettingFieldChar::set(char x) { value = x; changed = true; } -void SettingChar::set(const String & x) +void SettingFieldChar::set(const String & x) { if (x.size() > 1) throw Exception("A setting's value string has to be an exactly one character long", ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH); @@ -344,18 +344,18 @@ void SettingChar::set(const String & x) set(c); } -void SettingChar::set(const Field & x) +void SettingFieldChar::set(const Field & x) { const String & s = safeGet(x); set(s); } -void SettingChar::serialize(WriteBuffer & buf, SettingsBinaryFormat) const +void SettingFieldChar::serialize(WriteBuffer & buf, SettingsBinaryFormat) const { writeStringBinary(toString(), buf); } -void SettingChar::deserialize(ReadBuffer & buf, SettingsBinaryFormat) +void SettingFieldChar::deserialize(ReadBuffer & buf, SettingsBinaryFormat) { String s; readStringBinary(s, buf); @@ -364,13 +364,13 @@ void SettingChar::deserialize(ReadBuffer & buf, SettingsBinaryFormat) template -void SettingEnum::serialize(WriteBuffer & buf, SettingsBinaryFormat) const +void SettingFieldEnum::serialize(WriteBuffer & buf, SettingsBinaryFormat) const { writeStringBinary(toString(), buf); } template -void SettingEnum::deserialize(ReadBuffer & buf, SettingsBinaryFormat) +void SettingFieldEnum::deserialize(ReadBuffer & buf, SettingsBinaryFormat) { String s; readStringBinary(s, buf); @@ -378,51 +378,51 @@ void SettingEnum::deserialize(ReadBuffer & buf, SettingsBinaryFor } template -Field SettingEnum::toField() const +Field SettingFieldEnum::toField() const { return toString(); } template -void SettingEnum::set(const Field & x) +void SettingFieldEnum::set(const Field & x) { set(safeGet(x)); } -String SettingURI::toString() const +String SettingFieldURI::toString() const { return value.toString(); } -Field SettingURI::toField() const +Field SettingFieldURI::toField() const { return value.toString(); } -void SettingURI::set(const Poco::URI & x) +void SettingFieldURI::set(const Poco::URI & x) { value = x; changed = true; } -void SettingURI::set(const Field & x) +void SettingFieldURI::set(const Field & x) { const String & s = safeGet(x); set(s); } -void SettingURI::set(const String & x) +void SettingFieldURI::set(const String & x) { set(Poco::URI(x)); } -void SettingURI::serialize(WriteBuffer & buf, SettingsBinaryFormat) const +void SettingFieldURI::serialize(WriteBuffer & buf, SettingsBinaryFormat) const { writeStringBinary(toString(), buf); } -void SettingURI::deserialize(ReadBuffer & buf, SettingsBinaryFormat) +void SettingFieldURI::deserialize(ReadBuffer & buf, SettingsBinaryFormat) { String s; readStringBinary(s, buf); @@ -435,7 +435,7 @@ void SettingURI::deserialize(ReadBuffer & buf, SettingsBinaryFormat) #define IMPLEMENT_SETTING_ENUM_WITH_TAG(ENUM_NAME, TAG, LIST_OF_NAMES_MACRO, ERROR_CODE_FOR_UNEXPECTED_NAME) \ template <> \ - String SettingEnum::toString() const \ + String SettingFieldEnum::toString() const \ { \ using EnumType = ENUM_NAME; \ using UnderlyingType = std::underlying_type::type; \ @@ -447,7 +447,7 @@ void SettingURI::deserialize(ReadBuffer & buf, SettingsBinaryFormat) } \ \ template <> \ - void SettingEnum::set(const String & s) \ + void SettingFieldEnum::set(const String & s) \ { \ using EnumType = ENUM_NAME; \ LIST_OF_NAMES_MACRO(IMPLEMENT_SETTING_ENUM_FROM_STRING_HELPER_) \ @@ -458,7 +458,7 @@ void SettingURI::deserialize(ReadBuffer & buf, SettingsBinaryFormat) ERROR_CODE_FOR_UNEXPECTED_NAME); \ } \ \ - template struct SettingEnum; + template struct SettingFieldEnum; #define IMPLEMENT_SETTING_ENUM_TO_STRING_HELPER_(NAME, IO_NAME) \ case static_cast(EnumType::NAME): return IO_NAME; @@ -522,7 +522,7 @@ IMPLEMENT_SETTING_ENUM(OverflowMode, OVERFLOW_MODE_LIST_OF_NAMES, ErrorCodes::UN M(THROW, "throw") \ M(BREAK, "break") \ M(ANY, "any") -IMPLEMENT_SETTING_ENUM_WITH_TAG(OverflowMode, SettingOverflowModeGroupByTag, OVERFLOW_MODE_LIST_OF_NAMES_WITH_ANY, ErrorCodes::UNKNOWN_OVERFLOW_MODE) +IMPLEMENT_SETTING_ENUM_WITH_TAG(OverflowMode, SettingFieldOverflowModeGroupByTag, OVERFLOW_MODE_LIST_OF_NAMES_WITH_ANY, ErrorCodes::UNKNOWN_OVERFLOW_MODE) #define DISTRIBUTED_PRODUCT_MODE_LIST_OF_NAMES(M) \ diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h index 08168b79e64..b2fc2a479e7 100644 --- a/src/Core/SettingsCollection.h +++ b/src/Core/SettingsCollection.h @@ -28,15 +28,15 @@ enum class SettingsBinaryFormat; */ template -struct SettingNumber +struct SettingFieldNumber { Type value; bool changed = false; - SettingNumber(Type x = 0) : value(x) {} + SettingFieldNumber(Type x = 0) : value(x) {} operator Type() const { return value; } - SettingNumber & operator= (Type x) { set(x); return *this; } + SettingFieldNumber & operator= (Type x) { set(x); return *this; } /// Serialize to a test string. String toString() const; @@ -59,26 +59,26 @@ struct SettingNumber void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); }; -using SettingUInt64 = SettingNumber; -using SettingInt64 = SettingNumber; -using SettingFloat = SettingNumber; -using SettingBool = SettingNumber; +using SettingFieldUInt64 = SettingFieldNumber; +using SettingFieldInt64 = SettingFieldNumber; +using SettingFieldFloat = SettingFieldNumber; +using SettingFieldBool = SettingFieldNumber; /** Unlike SettingUInt64, supports the value of 'auto' - the number of processor cores without taking into account SMT. * A value of 0 is also treated as auto. * When serializing, `auto` is written in the same way as 0. */ -struct SettingMaxThreads +struct SettingFieldMaxThreads { UInt64 value; bool is_auto; bool changed = false; - SettingMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {} + SettingFieldMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {} operator UInt64() const { return value; } - SettingMaxThreads & operator= (UInt64 x) { set(x); return *this; } + SettingFieldMaxThreads & operator= (UInt64 x) { set(x); return *this; } String toString() const; Field toField() const; @@ -95,24 +95,24 @@ struct SettingMaxThreads }; -enum class SettingTimespanIO { MILLISECOND, SECOND }; +enum class SettingFieldTimespanIO { MILLISECOND, SECOND }; -template -struct SettingTimespan +template +struct SettingFieldTimespan { Poco::Timespan value; bool changed = false; - SettingTimespan(UInt64 x = 0) : value(x * microseconds_per_io_unit) {} + SettingFieldTimespan(UInt64 x = 0) : value(x * microseconds_per_io_unit) {} operator Poco::Timespan() const { return value; } - SettingTimespan & operator=(const Poco::Timespan & x) { set(x); return *this; } + SettingFieldTimespan & operator=(const Poco::Timespan & x) { set(x); return *this; } template > operator std::chrono::duration() const { return std::chrono::duration_cast>(std::chrono::microseconds(value.totalMicroseconds())); } template > - SettingTimespan & operator=(const std::chrono::duration & x) { set(x); return *this; } + SettingFieldTimespan & operator=(const std::chrono::duration & x) { set(x); return *this; } Poco::Timespan::TimeDiff totalSeconds() const { return value.totalSeconds(); } Poco::Timespan::TimeDiff totalMilliseconds() const { return value.totalMilliseconds(); } @@ -132,22 +132,22 @@ struct SettingTimespan void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); - static constexpr UInt64 microseconds_per_io_unit = (io_unit == SettingTimespanIO::MILLISECOND) ? 1000 : 1000000; + static constexpr UInt64 microseconds_per_io_unit = (io_unit == SettingFieldTimespanIO::MILLISECOND) ? 1000 : 1000000; }; -using SettingSeconds = SettingTimespan; -using SettingMilliseconds = SettingTimespan; +using SettingFieldSeconds = SettingFieldTimespan; +using SettingFieldMilliseconds = SettingFieldTimespan; -struct SettingString +struct SettingFieldString { String value; bool changed = false; - SettingString(const String & x = String{}) : value(x) {} + SettingFieldString(const String & x = String{}) : value(x) {} operator String() const { return value; } - SettingString & operator= (const String & x) { set(x); return *this; } + SettingFieldString & operator= (const String & x) { set(x); return *this; } String toString() const; Field toField() const; @@ -160,16 +160,16 @@ struct SettingString }; -struct SettingChar +struct SettingFieldChar { public: char value; bool changed = false; - SettingChar(char x = '\0') : value(x) {} + SettingFieldChar(char x = '\0') : value(x) {} operator char() const { return value; } - SettingChar & operator= (char x) { set(x); return *this; } + SettingFieldChar & operator= (char x) { set(x); return *this; } String toString() const; Field toField() const; @@ -185,15 +185,15 @@ public: /// Template class to define enum-based settings. template -struct SettingEnum +struct SettingFieldEnum { EnumType value; bool changed = false; - SettingEnum(EnumType x) : value(x) {} + SettingFieldEnum(EnumType x) : value(x) {} operator EnumType() const { return value; } - SettingEnum & operator= (EnumType x) { set(x); return *this; } + SettingFieldEnum & operator= (EnumType x) { set(x); return *this; } String toString() const; Field toField() const; @@ -206,15 +206,15 @@ struct SettingEnum void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); }; -struct SettingURI +struct SettingFieldURI { Poco::URI value; bool changed = false; - SettingURI(const Poco::URI & x = Poco::URI{}) : value(x) {} + SettingFieldURI(const Poco::URI & x = Poco::URI{}) : value(x) {} operator Poco::URI() const { return value; } - SettingURI & operator= (const Poco::URI & x) { set(x); return *this; } + SettingFieldURI & operator= (const Poco::URI & x) { set(x); return *this; } String toString() const; Field toField() const; @@ -243,7 +243,7 @@ enum class LoadBalancing // round robin across replicas with the same number of errors. ROUND_ROBIN, }; -using SettingLoadBalancing = SettingEnum; +using SettingFieldLoadBalancing = SettingFieldEnum; enum class JoinStrictness @@ -252,7 +252,7 @@ enum class JoinStrictness ALL, /// Query JOIN without strictness -> ALL JOIN ... ANY, /// Query JOIN without strictness -> ANY JOIN ... }; -using SettingJoinStrictness = SettingEnum; +using SettingFieldJoinStrictness = SettingFieldEnum; enum class JoinAlgorithm { @@ -261,7 +261,7 @@ enum class JoinAlgorithm PARTIAL_MERGE, PREFER_PARTIAL_MERGE, }; -using SettingJoinAlgorithm = SettingEnum; +using SettingFieldJoinAlgorithm = SettingFieldEnum; enum class SpecialSort @@ -269,7 +269,7 @@ enum class SpecialSort NOT_SPECIFIED = 0, OPENCL_BITONIC, }; -using SettingSpecialSort = SettingEnum; +using SettingFieldSpecialSort = SettingFieldEnum; /// Which rows should be included in TOTALS. @@ -283,15 +283,15 @@ enum class TotalsMode AFTER_HAVING_EXCLUSIVE = 2, /// Include only the rows that passed and max_rows_to_group_by, and HAVING. AFTER_HAVING_AUTO = 3, /// Automatically select between INCLUSIVE and EXCLUSIVE, }; -using SettingTotalsMode = SettingEnum; +using SettingFieldTotalsMode = SettingFieldEnum; /// The settings keeps OverflowMode which cannot be OverflowMode::ANY. -using SettingOverflowMode = SettingEnum; -struct SettingOverflowModeGroupByTag; +using SettingFieldOverflowMode = SettingFieldEnum; +struct SettingFieldOverflowModeGroupByTag; /// The settings keeps OverflowMode which can be OverflowMode::ANY. -using SettingOverflowModeGroupBy = SettingEnum; +using SettingFieldOverflowModeGroupBy = SettingFieldEnum; /// The setting for executing distributed subqueries inside IN or JOIN sections. @@ -302,10 +302,10 @@ enum class DistributedProductMode GLOBAL, /// Convert to global query ALLOW /// Enable }; -using SettingDistributedProductMode = SettingEnum; +using SettingFieldDistributedProductMode = SettingFieldEnum; -using SettingDateTimeInputFormat = SettingEnum; +using SettingFieldDateTimeInputFormat = SettingFieldEnum; enum class LogsLevel @@ -318,14 +318,14 @@ enum class LogsLevel debug, trace, }; -using SettingLogsLevel = SettingEnum; +using SettingFieldLogsLevel = SettingFieldEnum; enum class DefaultDatabaseEngine { Ordinary, Atomic, }; -using SettingDefaultDatabaseEngine = SettingEnum; +using SettingFieldDefaultDatabaseEngine = SettingFieldEnum; // Make it signed for compatibility with DataTypeEnum8 enum QueryLogElementType : int8_t @@ -335,7 +335,7 @@ enum QueryLogElementType : int8_t EXCEPTION_BEFORE_START = 3, EXCEPTION_WHILE_PROCESSING = 4, }; -using SettingLogQueriesType = SettingEnum; +using SettingFieldLogQueriesType = SettingFieldEnum; enum class SettingsBinaryFormat @@ -586,5 +586,5 @@ public: LIST_OF_SETTINGS_MACRO(DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_) #define DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ - Setting##TYPE NAME {DEFAULT}; + SettingField##TYPE NAME {DEFAULT}; } diff --git a/src/Core/SettingsCollectionImpl.h b/src/Core/SettingsCollectionImpl.h index 2871816f43f..5da5f8ea648 100644 --- a/src/Core/SettingsCollectionImpl.h +++ b/src/Core/SettingsCollectionImpl.h @@ -342,8 +342,8 @@ void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFo static void NAME##_setField(Derived & collection, const Field & value) { collection.NAME.set(value); } \ static void NAME##_serialize(const Derived & collection, WriteBuffer & buf, SettingsBinaryFormat format) { collection.NAME.serialize(buf, format); } \ static void NAME##_deserialize(Derived & collection, ReadBuffer & buf, SettingsBinaryFormat format) { collection.NAME.deserialize(buf, format); } \ - static String NAME##_valueToString(const Field & value) { Setting##TYPE temp{DEFAULT}; temp.set(value); return temp.toString(); } \ - static Field NAME##_valueToCorrespondingType(const Field & value) { Setting##TYPE temp{DEFAULT}; temp.set(value); return temp.toField(); } \ + static String NAME##_valueToString(const Field & value) { SettingField##TYPE temp{DEFAULT}; temp.set(value); return temp.toString(); } \ + static Field NAME##_valueToCorrespondingType(const Field & value) { SettingField##TYPE temp{DEFAULT}; temp.set(value); return temp.toField(); } \ #define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ From 90602b869a37c66ec5ffea4f99522638a9e52704 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 22 Jul 2020 15:02:47 +0300 Subject: [PATCH 07/20] Make SettingsChanges a class. --- src/Access/EnabledSettings.cpp | 1 + src/Access/EnabledSettings.h | 1 - src/Access/SettingsConstraints.cpp | 1 + src/Access/SettingsConstraints.h | 3 +- src/Access/SettingsProfileElement.cpp | 1 + src/Access/SettingsProfileElement.h | 3 +- src/Access/SettingsProfilesCache.cpp | 1 + src/Common/SettingsChanges.cpp | 50 +++++++++++++++++++ src/Common/SettingsChanges.h | 19 ++++--- src/Common/ya.make | 1 + src/Core/SettingsCollection.h | 2 +- .../QueryWithOutputSettingsPushDownVisitor.h | 2 +- src/Storages/IStorage.h | 2 - src/Storages/Kafka/StorageKafka.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 24 ++++----- src/Storages/MergeTree/MergeTreeSettings.h | 1 - 16 files changed, 85 insertions(+), 28 deletions(-) create mode 100644 src/Common/SettingsChanges.cpp diff --git a/src/Access/EnabledSettings.cpp b/src/Access/EnabledSettings.cpp index 65e38e4827f..f913acb0150 100644 --- a/src/Access/EnabledSettings.cpp +++ b/src/Access/EnabledSettings.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB diff --git a/src/Access/EnabledSettings.h b/src/Access/EnabledSettings.h index 8e92298328c..cc30e4481fc 100644 --- a/src/Access/EnabledSettings.h +++ b/src/Access/EnabledSettings.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 11dbd016e64..8ca2262f8cd 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 65537250957..f8267b2b47a 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include @@ -18,6 +17,8 @@ namespace Util namespace DB { struct Settings; +struct SettingChange; +class SettingsChanges; /** Checks if specified changes of settings are allowed or not. * If the changes are not allowed (i.e. violates some constraints) this class throws an exception. diff --git a/src/Access/SettingsProfileElement.cpp b/src/Access/SettingsProfileElement.cpp index d4f6ff5d0f2..4fbe4aec2f8 100644 --- a/src/Access/SettingsProfileElement.cpp +++ b/src/Access/SettingsProfileElement.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include diff --git a/src/Access/SettingsProfileElement.h b/src/Access/SettingsProfileElement.h index f64317344b8..88c8178b426 100644 --- a/src/Access/SettingsProfileElement.h +++ b/src/Access/SettingsProfileElement.h @@ -9,8 +9,7 @@ namespace DB { struct Settings; -struct SettingChange; -using SettingsChanges = std::vector; +class SettingsChanges; class SettingsConstraints; class ASTSettingsProfileElement; class ASTSettingsProfileElements; diff --git a/src/Access/SettingsProfilesCache.cpp b/src/Access/SettingsProfilesCache.cpp index 95074ff7d0b..e663ee564aa 100644 --- a/src/Access/SettingsProfilesCache.cpp +++ b/src/Access/SettingsProfilesCache.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/SettingsChanges.cpp b/src/Common/SettingsChanges.cpp new file mode 100644 index 00000000000..e7c769ad55a --- /dev/null +++ b/src/Common/SettingsChanges.cpp @@ -0,0 +1,50 @@ +#include + + +namespace DB +{ +namespace +{ + SettingChange * find(SettingsChanges & changes, const std::string_view & name) + { + auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); + if (it == changes.end()) + return nullptr; + return &*it; + } + + const SettingChange * find(const SettingsChanges & changes, const std::string_view & name) + { + auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); + if (it == changes.end()) + return nullptr; + return &*it; + } +} + +bool SettingsChanges::tryGet(const std::string_view & name, Field & out_value) const +{ + const auto * change = find(*this, name); + if (!change) + return false; + out_value = change->value; + return true; +} + +const Field * SettingsChanges::tryGet(const std::string_view & name) const +{ + const auto * change = find(*this, name); + if (!change) + return nullptr; + return &change->value; +} + +Field * SettingsChanges::tryGet(const std::string_view & name) +{ + auto * change = find(*this, name); + if (!change) + return nullptr; + return &change->value; +} + +} diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 004a08c3b4b..734d8ecb227 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -5,21 +5,28 @@ namespace DB { - struct SettingChange { String name; Field value; - SettingChange() {} - SettingChange(const String & name_, const Field value_) - : name(name_) - , value(value_) {} + SettingChange() {} + SettingChange(const std::string_view & name_, const Field & value_) : name(name_), value(value_) {} + SettingChange(const std::string_view & name_, Field && value_) : name(name_), value(std::move(value_)) {} friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } }; -using SettingsChanges = std::vector; + +class SettingsChanges : public std::vector +{ +public: + using std::vector::vector; + + bool tryGet(const std::string_view & name, Field & out_value) const; + const Field * tryGet(const std::string_view & name) const; + Field * tryGet(const std::string_view & name); +}; } diff --git a/src/Common/ya.make b/src/Common/ya.make index 32cb55adf4f..f1b2ba0470e 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -75,6 +75,7 @@ SRCS( RWLock.cpp SensitiveDataMasker.cpp setThreadName.cpp + SettingsChanges.cpp SharedLibrary.cpp ShellCommand.cpp StackTrace.cpp diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h index b2fc2a479e7..62a3e2764d5 100644 --- a/src/Core/SettingsCollection.h +++ b/src/Core/SettingsCollection.h @@ -14,7 +14,7 @@ namespace DB class Field; struct SettingChange; -using SettingsChanges = std::vector; +class SettingsChanges; class ReadBuffer; class WriteBuffer; enum class SettingsBinaryFormat; diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h index 3746d1eb0a5..2a7ed0125fa 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h @@ -8,7 +8,7 @@ namespace DB class ASTSelectQuery; struct SettingChange; -using SettingsChanges = std::vector; +class SettingsChanges; /// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query: /// (since settings after FORMAT parsed separatelly not in the ParserSelectQuery but in ParserQueryWithOutput) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 5e91429de34..91c05175f87 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -37,8 +37,6 @@ using StorageActionBlockType = size_t; class ASTCreateQuery; struct Settings; -struct SettingChange; -using SettingsChanges = std::vector; class AlterCommands; class MutationCommands; diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index b7e6ea2a7e0..dd35f8dabbb 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 740d44605ee..67a00d7cd0c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1472,24 +1472,22 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S const auto & new_changes = new_metadata.settings_changes->as().changes; for (const auto & changed_setting : new_changes) { - if (MergeTreeSettings::findIndex(changed_setting.name) == MergeTreeSettings::npos) - throw Exception{"Storage '" + getName() + "' doesn't have setting '" + changed_setting.name + "'", + const auto & setting_name = changed_setting.name; + const auto & new_value = changed_setting.value; + if (MergeTreeSettings::findIndex(setting_name) == MergeTreeSettings::npos) + throw Exception{"Storage '" + getName() + "' doesn't have setting '" + setting_name + "'", ErrorCodes::UNKNOWN_SETTING}; - auto comparator = [&changed_setting](const auto & change) { return change.name == changed_setting.name; }; + const Field * current_value = current_changes.tryGet(setting_name); - auto current_setting_it - = std::find_if(current_changes.begin(), current_changes.end(), comparator); - - if ((current_setting_it == current_changes.end() || *current_setting_it != changed_setting) - && MergeTreeSettings::isReadonlySetting(changed_setting.name)) + if ((!current_value || *current_value != new_value) + && MergeTreeSettings::isReadonlySetting(setting_name)) { - throw Exception{"Setting '" + changed_setting.name + "' is readonly for storage '" + getName() + "'", + throw Exception{"Setting '" + setting_name + "' is readonly for storage '" + getName() + "'", ErrorCodes::READONLY_SETTING}; } - if (current_setting_it == current_changes.end() - && MergeTreeSettings::isPartFormatSetting(changed_setting.name)) + if (!current_value && MergeTreeSettings::isPartFormatSetting(setting_name)) { MergeTreeSettings copy = *getSettings(); copy.applyChange(changed_setting); @@ -1498,8 +1496,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S throw Exception("Can't change settings. Reason: " + reason, ErrorCodes::NOT_IMPLEMENTED); } - if (changed_setting.name == "storage_policy") - checkStoragePolicy(global_context.getStoragePolicy(changed_setting.value.safeGet())); + if (setting_name == "storage_policy") + checkStoragePolicy(global_context.getStoragePolicy(new_value.safeGet())); } } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 410b8caee62..31436121adc 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -2,7 +2,6 @@ #include #include -#include namespace Poco From 18e3f1f60db8c9888431bdc0660bce4a94de0b91 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 17 Jul 2020 01:01:08 +0300 Subject: [PATCH 08/20] Split SettingsCollection.h into 3 files: SettingsFields.h, SettingsEnums.h, SettingsCollection.h --- programs/local/LocalServer.cpp | 1 - src/Access/AccessControlManager.h | 2 +- src/Common/ThreadStatus.h | 2 +- src/Core/Settings.h | 1 + src/Core/SettingsCollection.cpp | 546 ------------------------------ src/Core/SettingsCollection.h | 334 +----------------- src/Core/SettingsCollectionImpl.h | 40 ++- src/Core/SettingsEnums.cpp | 94 +++++ src/Core/SettingsEnums.h | 129 +++++++ src/Core/SettingsFields.cpp | 365 ++++++++++++++++++++ src/Core/SettingsFields.h | 313 +++++++++++++++++ src/Core/SortDescription.h | 2 +- src/Core/ya.make | 2 + src/Interpreters/QueryLog.h | 1 - src/Interpreters/TableJoin.h | 2 +- 15 files changed, 940 insertions(+), 894 deletions(-) create mode 100644 src/Core/SettingsEnums.cpp create mode 100644 src/Core/SettingsEnums.h create mode 100644 src/Core/SettingsFields.cpp create mode 100644 src/Core/SettingsFields.h diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index a09b7239abc..e3a5306a6f1 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -20,7 +20,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Access/AccessControlManager.h b/src/Access/AccessControlManager.h index 467b7471423..5944916e798 100644 --- a/src/Access/AccessControlManager.h +++ b/src/Access/AccessControlManager.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index a3394dc7458..89bf3703d0b 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 091979a597a..208de421b76 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include diff --git a/src/Core/SettingsCollection.cpp b/src/Core/SettingsCollection.cpp index 5f90b8f2635..f0c2b35b4a2 100644 --- a/src/Core/SettingsCollection.cpp +++ b/src/Core/SettingsCollection.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -14,555 +13,10 @@ namespace DB { namespace ErrorCodes { - extern const int UNKNOWN_LOAD_BALANCING; - extern const int UNKNOWN_OVERFLOW_MODE; - extern const int UNKNOWN_TOTALS_MODE; - extern const int UNKNOWN_DISTRIBUTED_PRODUCT_MODE; - extern const int UNKNOWN_JOIN; - extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; extern const int UNKNOWN_SETTING; - extern const int CANNOT_PARSE_BOOL; } -template -String SettingFieldNumber::toString() const -{ - return DB::toString(value); -} - -template -Field SettingFieldNumber::toField() const -{ - return value; -} - -template -void SettingFieldNumber::set(Type x) -{ - value = x; - changed = true; -} - -template -void SettingFieldNumber::set(const Field & x) -{ - if (x.getType() == Field::Types::String) - set(get(x)); - else - set(applyVisitor(FieldVisitorConvertToNumber(), x)); -} - -template -void SettingFieldNumber::set(const String & x) -{ - set(parseWithSizeSuffix(x)); -} - -template <> -void SettingFieldNumber::set(const String & x) -{ - if (x.size() == 1) - { - if (x[0] == '0') - set(false); - else if (x[0] == '1') - set(true); - else - throw Exception("Cannot parse bool from string '" + x + "'", ErrorCodes::CANNOT_PARSE_BOOL); - } - else - { - ReadBufferFromString buf(x); - if (checkStringCaseInsensitive("true", buf)) - set(true); - else if (checkStringCaseInsensitive("false", buf)) - set(false); - else - throw Exception("Cannot parse bool from string '" + x + "'", ErrorCodes::CANNOT_PARSE_BOOL); - } -} - -template -void SettingFieldNumber::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const -{ - if (format >= SettingsBinaryFormat::STRINGS) - { - writeStringBinary(toString(), buf); - return; - } - - if constexpr (is_integral_v && is_unsigned_v) - writeVarUInt(static_cast(value), buf); - else if constexpr (is_integral_v && is_signed_v) - writeVarInt(static_cast(value), buf); - else - { - static_assert(std::is_floating_point_v); - writeStringBinary(toString(), buf); - } -} - -template -void SettingFieldNumber::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) -{ - if (format >= SettingsBinaryFormat::STRINGS) - { - String x; - readStringBinary(x, buf); - set(x); - return; - } - - if constexpr (is_integral_v && is_unsigned_v) - { - UInt64 x; - readVarUInt(x, buf); - set(static_cast(x)); - } - else if constexpr (is_integral_v && is_signed_v) - { - Int64 x; - readVarInt(x, buf); - set(static_cast(x)); - } - else - { - static_assert(std::is_floating_point_v); - String x; - readStringBinary(x, buf); - set(x); - } -} - -template struct SettingFieldNumber; -template struct SettingFieldNumber; -template struct SettingFieldNumber; -template struct SettingFieldNumber; - - -String SettingFieldMaxThreads::toString() const -{ - /// Instead of the `auto` value, we output the actual value to make it easier to see. - return is_auto ? ("auto(" + DB::toString(value) + ")") : DB::toString(value); -} - -Field SettingFieldMaxThreads::toField() const -{ - return is_auto ? 0 : value; -} - -void SettingFieldMaxThreads::set(UInt64 x) -{ - value = x ? x : getAutoValue(); - is_auto = x == 0; - changed = true; -} - -void SettingFieldMaxThreads::set(const Field & x) -{ - if (x.getType() == Field::Types::String) - set(get(x)); - else - set(applyVisitor(FieldVisitorConvertToNumber(), x)); -} - -void SettingFieldMaxThreads::set(const String & x) -{ - if (startsWith(x, "auto")) - setAuto(); - else - set(parse(x)); -} - -void SettingFieldMaxThreads::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const -{ - if (format >= SettingsBinaryFormat::STRINGS) - { - writeStringBinary(is_auto ? "auto" : DB::toString(value), buf); - return; - } - - writeVarUInt(is_auto ? 0 : value, buf); -} - -void SettingFieldMaxThreads::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) -{ - if (format >= SettingsBinaryFormat::STRINGS) - { - String x; - readStringBinary(x, buf); - set(x); - return; - } - - UInt64 x = 0; - readVarUInt(x, buf); - set(x); -} - -void SettingFieldMaxThreads::setAuto() -{ - value = getAutoValue(); - is_auto = true; -} - -UInt64 SettingFieldMaxThreads::getAutoValue() -{ - return getNumberOfPhysicalCPUCores(); -} - - -template -String SettingFieldTimespan::toString() const -{ - return DB::toString(value.totalMicroseconds() / microseconds_per_io_unit); -} - -template -Field SettingFieldTimespan::toField() const -{ - return value.totalMicroseconds() / microseconds_per_io_unit; -} - -template -void SettingFieldTimespan::set(const Poco::Timespan & x) -{ - value = x; - changed = true; -} - -template -void SettingFieldTimespan::set(UInt64 x) -{ - set(Poco::Timespan(x * microseconds_per_io_unit)); -} - -template -void SettingFieldTimespan::set(const Field & x) -{ - if (x.getType() == Field::Types::String) - set(get(x)); - else - set(applyVisitor(FieldVisitorConvertToNumber(), x)); -} - -template -void SettingFieldTimespan::set(const String & x) -{ - set(parse(x)); -} - -template -void SettingFieldTimespan::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const -{ - if (format >= SettingsBinaryFormat::STRINGS) - { - writeStringBinary(toString(), buf); - return; - } - - writeVarUInt(value.totalMicroseconds() / microseconds_per_io_unit, buf); -} - -template -void SettingFieldTimespan::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) -{ - if (format >= SettingsBinaryFormat::STRINGS) - { - String x; - readStringBinary(x, buf); - set(x); - return; - } - - UInt64 x = 0; - readVarUInt(x, buf); - set(x); -} - -template struct SettingFieldTimespan; -template struct SettingFieldTimespan; - - -String SettingFieldString::toString() const -{ - return value; -} - -Field SettingFieldString::toField() const -{ - return value; -} - -void SettingFieldString::set(const String & x) -{ - value = x; - changed = true; -} - -void SettingFieldString::set(const Field & x) -{ - set(safeGet(x)); -} - -void SettingFieldString::serialize(WriteBuffer & buf, SettingsBinaryFormat) const -{ - writeStringBinary(value, buf); -} - -void SettingFieldString::deserialize(ReadBuffer & buf, SettingsBinaryFormat) -{ - String s; - readStringBinary(s, buf); - set(s); -} - - -String SettingFieldChar::toString() const -{ - return String(1, value); -} - -Field SettingFieldChar::toField() const -{ - return toString(); -} - -void SettingFieldChar::set(char x) -{ - value = x; - changed = true; -} - -void SettingFieldChar::set(const String & x) -{ - if (x.size() > 1) - throw Exception("A setting's value string has to be an exactly one character long", ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH); - char c = (x.size() == 1) ? x[0] : '\0'; - set(c); -} - -void SettingFieldChar::set(const Field & x) -{ - const String & s = safeGet(x); - set(s); -} - -void SettingFieldChar::serialize(WriteBuffer & buf, SettingsBinaryFormat) const -{ - writeStringBinary(toString(), buf); -} - -void SettingFieldChar::deserialize(ReadBuffer & buf, SettingsBinaryFormat) -{ - String s; - readStringBinary(s, buf); - set(s); -} - - -template -void SettingFieldEnum::serialize(WriteBuffer & buf, SettingsBinaryFormat) const -{ - writeStringBinary(toString(), buf); -} - -template -void SettingFieldEnum::deserialize(ReadBuffer & buf, SettingsBinaryFormat) -{ - String s; - readStringBinary(s, buf); - set(s); -} - -template -Field SettingFieldEnum::toField() const -{ - return toString(); -} - -template -void SettingFieldEnum::set(const Field & x) -{ - set(safeGet(x)); -} - - -String SettingFieldURI::toString() const -{ - return value.toString(); -} - -Field SettingFieldURI::toField() const -{ - return value.toString(); -} - -void SettingFieldURI::set(const Poco::URI & x) -{ - value = x; - changed = true; -} - -void SettingFieldURI::set(const Field & x) -{ - const String & s = safeGet(x); - set(s); -} - -void SettingFieldURI::set(const String & x) -{ - set(Poco::URI(x)); -} - -void SettingFieldURI::serialize(WriteBuffer & buf, SettingsBinaryFormat) const -{ - writeStringBinary(toString(), buf); -} - -void SettingFieldURI::deserialize(ReadBuffer & buf, SettingsBinaryFormat) -{ - String s; - readStringBinary(s, buf); - set(s); -} - - -#define IMPLEMENT_SETTING_ENUM(ENUM_NAME, LIST_OF_NAMES_MACRO, ERROR_CODE_FOR_UNEXPECTED_NAME) \ - IMPLEMENT_SETTING_ENUM_WITH_TAG(ENUM_NAME, void, LIST_OF_NAMES_MACRO, ERROR_CODE_FOR_UNEXPECTED_NAME) - -#define IMPLEMENT_SETTING_ENUM_WITH_TAG(ENUM_NAME, TAG, LIST_OF_NAMES_MACRO, ERROR_CODE_FOR_UNEXPECTED_NAME) \ - template <> \ - String SettingFieldEnum::toString() const \ - { \ - using EnumType = ENUM_NAME; \ - using UnderlyingType = std::underlying_type::type; \ - switch (static_cast(value)) \ - { \ - LIST_OF_NAMES_MACRO(IMPLEMENT_SETTING_ENUM_TO_STRING_HELPER_) \ - } \ - throw Exception("Unknown " #ENUM_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME); \ - } \ - \ - template <> \ - void SettingFieldEnum::set(const String & s) \ - { \ - using EnumType = ENUM_NAME; \ - LIST_OF_NAMES_MACRO(IMPLEMENT_SETTING_ENUM_FROM_STRING_HELPER_) \ - \ - String all_io_names; \ - LIST_OF_NAMES_MACRO(IMPLEMENT_SETTING_ENUM_CONCAT_NAMES_HELPER_) \ - throw Exception("Unknown " #ENUM_NAME " : '" + s + "', must be one of " + all_io_names, \ - ERROR_CODE_FOR_UNEXPECTED_NAME); \ - } \ - \ - template struct SettingFieldEnum; - -#define IMPLEMENT_SETTING_ENUM_TO_STRING_HELPER_(NAME, IO_NAME) \ - case static_cast(EnumType::NAME): return IO_NAME; - -#define IMPLEMENT_SETTING_ENUM_FROM_STRING_HELPER_(NAME, IO_NAME) \ - if (s == (IO_NAME)) \ - { \ - set(EnumType::NAME); \ - return; \ - } - -#define IMPLEMENT_SETTING_ENUM_CONCAT_NAMES_HELPER_(NAME, IO_NAME) \ - if (!all_io_names.empty()) \ - all_io_names += ", "; \ - all_io_names += String("'") + (IO_NAME) + "'"; - - -#define LOAD_BALANCING_LIST_OF_NAMES(M) \ - M(RANDOM, "random") \ - M(NEAREST_HOSTNAME, "nearest_hostname") \ - M(IN_ORDER, "in_order") \ - M(FIRST_OR_RANDOM, "first_or_random") \ - M(ROUND_ROBIN, "round_robin") -IMPLEMENT_SETTING_ENUM(LoadBalancing, LOAD_BALANCING_LIST_OF_NAMES, ErrorCodes::UNKNOWN_LOAD_BALANCING) - - -#define SPECIAL_SORT_ALGORITHM_NAMES(M) \ - M(NOT_SPECIFIED, "not_specified") \ - M(OPENCL_BITONIC, "opencl_bitonic") -IMPLEMENT_SETTING_ENUM(SpecialSort, SPECIAL_SORT_ALGORITHM_NAMES, ErrorCodes::UNKNOWN_JOIN) - - -#define JOIN_STRICTNESS_LIST_OF_NAMES(M) \ - M(Unspecified, "") \ - M(ALL, "ALL") \ - M(ANY, "ANY") -IMPLEMENT_SETTING_ENUM(JoinStrictness, JOIN_STRICTNESS_LIST_OF_NAMES, ErrorCodes::UNKNOWN_JOIN) // NOLINT - -#define JOIN_ALGORITHM_NAMES(M) \ - M(AUTO, "auto") \ - M(HASH, "hash") \ - M(PARTIAL_MERGE, "partial_merge") \ - M(PREFER_PARTIAL_MERGE, "prefer_partial_merge") -IMPLEMENT_SETTING_ENUM(JoinAlgorithm, JOIN_ALGORITHM_NAMES, ErrorCodes::UNKNOWN_JOIN) - -#define TOTALS_MODE_LIST_OF_NAMES(M) \ - M(BEFORE_HAVING, "before_having") \ - M(AFTER_HAVING_EXCLUSIVE, "after_having_exclusive") \ - M(AFTER_HAVING_INCLUSIVE, "after_having_inclusive") \ - M(AFTER_HAVING_AUTO, "after_having_auto") -IMPLEMENT_SETTING_ENUM(TotalsMode, TOTALS_MODE_LIST_OF_NAMES, ErrorCodes::UNKNOWN_TOTALS_MODE) - - -#define OVERFLOW_MODE_LIST_OF_NAMES(M) \ - M(THROW, "throw") \ - M(BREAK, "break") -IMPLEMENT_SETTING_ENUM(OverflowMode, OVERFLOW_MODE_LIST_OF_NAMES, ErrorCodes::UNKNOWN_OVERFLOW_MODE) - - -#define OVERFLOW_MODE_LIST_OF_NAMES_WITH_ANY(M) \ - M(THROW, "throw") \ - M(BREAK, "break") \ - M(ANY, "any") -IMPLEMENT_SETTING_ENUM_WITH_TAG(OverflowMode, SettingFieldOverflowModeGroupByTag, OVERFLOW_MODE_LIST_OF_NAMES_WITH_ANY, ErrorCodes::UNKNOWN_OVERFLOW_MODE) - - -#define DISTRIBUTED_PRODUCT_MODE_LIST_OF_NAMES(M) \ - M(DENY, "deny") \ - M(LOCAL, "local") \ - M(GLOBAL, "global") \ - M(ALLOW, "allow") -IMPLEMENT_SETTING_ENUM(DistributedProductMode, DISTRIBUTED_PRODUCT_MODE_LIST_OF_NAMES, ErrorCodes::UNKNOWN_DISTRIBUTED_PRODUCT_MODE) - - -#define DATE_TIME_INPUT_FORMAT_LIST_OF_NAMES(M) \ - M(Basic, "basic") \ - M(BestEffort, "best_effort") -IMPLEMENT_SETTING_ENUM(FormatSettings::DateTimeInputFormat, DATE_TIME_INPUT_FORMAT_LIST_OF_NAMES, ErrorCodes::BAD_ARGUMENTS) - - -#define LOGS_LEVEL_LIST_OF_NAMES(M) \ - M(none, "none") \ - M(fatal, "fatal") \ - M(error, "error") \ - M(warning, "warning") \ - M(information, "information") \ - M(debug, "debug") \ - M(trace, "trace") -IMPLEMENT_SETTING_ENUM(LogsLevel, LOGS_LEVEL_LIST_OF_NAMES, ErrorCodes::BAD_ARGUMENTS) - -#define LOG_QUERIES_TYPE_LIST_OF_NAMES(M) \ - M(QUERY_START, "QUERY_START") \ - M(QUERY_FINISH, "QUERY_FINISH") \ - M(EXCEPTION_BEFORE_START, "EXCEPTION_BEFORE_START") \ - M(EXCEPTION_WHILE_PROCESSING, "EXCEPTION_WHILE_PROCESSING") -IMPLEMENT_SETTING_ENUM(QueryLogElementType, LOG_QUERIES_TYPE_LIST_OF_NAMES, ErrorCodes::BAD_ARGUMENTS) - - -#define DEFAULT_DATABASE_ENGINE_LIST_OF_NAMES(M) \ - M(Ordinary, "Ordinary") \ - M(Atomic, "Atomic") -IMPLEMENT_SETTING_ENUM(DefaultDatabaseEngine , DEFAULT_DATABASE_ENGINE_LIST_OF_NAMES, ErrorCodes::BAD_ARGUMENTS) - - namespace details { void SettingsCollectionUtils::serializeName(const StringRef & name, WriteBuffer & buf) diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h index 62a3e2764d5..5a6bef3d1d0 100644 --- a/src/Core/SettingsCollection.h +++ b/src/Core/SettingsCollection.h @@ -1,11 +1,7 @@ #pragma once -#include -#include -#include -#include #include -#include +#include #include @@ -17,326 +13,6 @@ struct SettingChange; class SettingsChanges; class ReadBuffer; class WriteBuffer; -enum class SettingsBinaryFormat; - - -/** One setting for any type. - * Stores a value within itself, as well as a flag - whether the value was changed. - * This is done so that you can send to the remote servers only changed settings (or explicitly specified in the config) values. - * That is, if the configuration was not specified in the config and was not dynamically changed, it is not sent to the remote server, - * and the remote server will use its default value. - */ - -template -struct SettingFieldNumber -{ - Type value; - bool changed = false; - - SettingFieldNumber(Type x = 0) : value(x) {} - - operator Type() const { return value; } - SettingFieldNumber & operator= (Type x) { set(x); return *this; } - - /// Serialize to a test string. - String toString() const; - - /// Converts to a field. - Field toField() const; - - void set(Type x); - - /// Read from SQL literal. - void set(const Field & x); - - /// Read from text string. - void set(const String & x); - - /// Serialize to binary stream suitable for transfer over network. - void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; - - /// Read from binary stream. - void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); -}; - -using SettingFieldUInt64 = SettingFieldNumber; -using SettingFieldInt64 = SettingFieldNumber; -using SettingFieldFloat = SettingFieldNumber; -using SettingFieldBool = SettingFieldNumber; - - -/** Unlike SettingUInt64, supports the value of 'auto' - the number of processor cores without taking into account SMT. - * A value of 0 is also treated as auto. - * When serializing, `auto` is written in the same way as 0. - */ -struct SettingFieldMaxThreads -{ - UInt64 value; - bool is_auto; - bool changed = false; - - SettingFieldMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {} - - operator UInt64() const { return value; } - SettingFieldMaxThreads & operator= (UInt64 x) { set(x); return *this; } - - String toString() const; - Field toField() const; - - void set(UInt64 x); - void set(const Field & x); - void set(const String & x); - - void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; - void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); - - void setAuto(); - static UInt64 getAutoValue(); -}; - - -enum class SettingFieldTimespanIO { MILLISECOND, SECOND }; - -template -struct SettingFieldTimespan -{ - Poco::Timespan value; - bool changed = false; - - SettingFieldTimespan(UInt64 x = 0) : value(x * microseconds_per_io_unit) {} - - operator Poco::Timespan() const { return value; } - SettingFieldTimespan & operator=(const Poco::Timespan & x) { set(x); return *this; } - - template > - operator std::chrono::duration() const { return std::chrono::duration_cast>(std::chrono::microseconds(value.totalMicroseconds())); } - - template > - SettingFieldTimespan & operator=(const std::chrono::duration & x) { set(x); return *this; } - - Poco::Timespan::TimeDiff totalSeconds() const { return value.totalSeconds(); } - Poco::Timespan::TimeDiff totalMilliseconds() const { return value.totalMilliseconds(); } - - String toString() const; - Field toField() const; - - void set(const Poco::Timespan & x); - - template > - void set(const std::chrono::duration & duration) { set(static_cast(std::chrono::duration_cast(duration).count())); } - - void set(UInt64 x); - void set(const Field & x); - void set(const String & x); - - void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; - void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); - - static constexpr UInt64 microseconds_per_io_unit = (io_unit == SettingFieldTimespanIO::MILLISECOND) ? 1000 : 1000000; -}; - -using SettingFieldSeconds = SettingFieldTimespan; -using SettingFieldMilliseconds = SettingFieldTimespan; - - -struct SettingFieldString -{ - String value; - bool changed = false; - - SettingFieldString(const String & x = String{}) : value(x) {} - - operator String() const { return value; } - SettingFieldString & operator= (const String & x) { set(x); return *this; } - - String toString() const; - Field toField() const; - - void set(const String & x); - void set(const Field & x); - - void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; - void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); -}; - - -struct SettingFieldChar -{ -public: - char value; - bool changed = false; - - SettingFieldChar(char x = '\0') : value(x) {} - - operator char() const { return value; } - SettingFieldChar & operator= (char x) { set(x); return *this; } - - String toString() const; - Field toField() const; - - void set(char x); - void set(const String & x); - void set(const Field & x); - - void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; - void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); -}; - - -/// Template class to define enum-based settings. -template -struct SettingFieldEnum -{ - EnumType value; - bool changed = false; - - SettingFieldEnum(EnumType x) : value(x) {} - - operator EnumType() const { return value; } - SettingFieldEnum & operator= (EnumType x) { set(x); return *this; } - - String toString() const; - Field toField() const; - - void set(EnumType x) { value = x; changed = true; } - void set(const Field & x); - void set(const String & x); - - void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; - void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); -}; - -struct SettingFieldURI -{ - Poco::URI value; - bool changed = false; - - SettingFieldURI(const Poco::URI & x = Poco::URI{}) : value(x) {} - - operator Poco::URI() const { return value; } - SettingFieldURI & operator= (const Poco::URI & x) { set(x); return *this; } - - String toString() const; - Field toField() const; - - void set(const Poco::URI & x); - void set(const Field & x); - void set(const String & x); - - void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; - void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); -}; - -enum class LoadBalancing -{ - /// among replicas with a minimum number of errors selected randomly - RANDOM = 0, - /// a replica is selected among the replicas with the minimum number of errors - /// with the minimum number of distinguished characters in the replica name and local hostname - NEAREST_HOSTNAME, - // replicas with the same number of errors are accessed in the same order - // as they are specified in the configuration. - IN_ORDER, - /// if first replica one has higher number of errors, - /// pick a random one from replicas with minimum number of errors - FIRST_OR_RANDOM, - // round robin across replicas with the same number of errors. - ROUND_ROBIN, -}; -using SettingFieldLoadBalancing = SettingFieldEnum; - - -enum class JoinStrictness -{ - Unspecified = 0, /// Query JOIN without strictness will throw Exception. - ALL, /// Query JOIN without strictness -> ALL JOIN ... - ANY, /// Query JOIN without strictness -> ANY JOIN ... -}; -using SettingFieldJoinStrictness = SettingFieldEnum; - -enum class JoinAlgorithm -{ - AUTO = 0, - HASH, - PARTIAL_MERGE, - PREFER_PARTIAL_MERGE, -}; -using SettingFieldJoinAlgorithm = SettingFieldEnum; - - -enum class SpecialSort -{ - NOT_SPECIFIED = 0, - OPENCL_BITONIC, -}; -using SettingFieldSpecialSort = SettingFieldEnum; - - -/// Which rows should be included in TOTALS. -enum class TotalsMode -{ - BEFORE_HAVING = 0, /// Count HAVING for all read rows; - /// including those not in max_rows_to_group_by - /// and have not passed HAVING after grouping. - AFTER_HAVING_INCLUSIVE = 1, /// Count on all rows except those that have not passed HAVING; - /// that is, to include in TOTALS all the rows that did not pass max_rows_to_group_by. - AFTER_HAVING_EXCLUSIVE = 2, /// Include only the rows that passed and max_rows_to_group_by, and HAVING. - AFTER_HAVING_AUTO = 3, /// Automatically select between INCLUSIVE and EXCLUSIVE, -}; -using SettingFieldTotalsMode = SettingFieldEnum; - - -/// The settings keeps OverflowMode which cannot be OverflowMode::ANY. -using SettingFieldOverflowMode = SettingFieldEnum; -struct SettingFieldOverflowModeGroupByTag; - -/// The settings keeps OverflowMode which can be OverflowMode::ANY. -using SettingFieldOverflowModeGroupBy = SettingFieldEnum; - - -/// The setting for executing distributed subqueries inside IN or JOIN sections. -enum class DistributedProductMode -{ - DENY = 0, /// Disable - LOCAL, /// Convert to local query - GLOBAL, /// Convert to global query - ALLOW /// Enable -}; -using SettingFieldDistributedProductMode = SettingFieldEnum; - - -using SettingFieldDateTimeInputFormat = SettingFieldEnum; - - -enum class LogsLevel -{ - none = 0, /// Disable - fatal, - error, - warning, - information, - debug, - trace, -}; -using SettingFieldLogsLevel = SettingFieldEnum; - -enum class DefaultDatabaseEngine -{ - Ordinary, - Atomic, -}; -using SettingFieldDefaultDatabaseEngine = SettingFieldEnum; - -// Make it signed for compatibility with DataTypeEnum8 -enum QueryLogElementType : int8_t -{ - QUERY_START = 1, - QUERY_FINISH = 2, - EXCEPTION_BEFORE_START = 3, - EXCEPTION_WHILE_PROCESSING = 4, -}; -using SettingFieldLogQueriesType = SettingFieldEnum; - enum class SettingsBinaryFormat { @@ -377,8 +53,8 @@ private: using GetFieldFunction = Field (*)(const Derived &); using SetStringFunction = void (*)(Derived &, const String &); using SetFieldFunction = void (*)(Derived &, const Field &); - using SerializeFunction = void (*)(const Derived &, WriteBuffer & buf, SettingsBinaryFormat); - using DeserializeFunction = void (*)(Derived &, ReadBuffer & buf, SettingsBinaryFormat); + using WriteBinaryFunction = void (*)(const Derived &, WriteBuffer & buf); + using ReadBinaryFunction = void (*)(Derived &, ReadBuffer & buf); using ValueToStringFunction = String (*)(const Field &); using ValueToCorrespondingTypeFunction = Field (*)(const Field &); @@ -391,8 +67,8 @@ private: GetFieldFunction get_field; SetStringFunction set_string; SetFieldFunction set_field; - SerializeFunction serialize; - DeserializeFunction deserialize; + WriteBinaryFunction write_binary; + ReadBinaryFunction read_binary; ValueToStringFunction value_to_string; ValueToCorrespondingTypeFunction value_to_corresponding_type; }; diff --git a/src/Core/SettingsCollectionImpl.h b/src/Core/SettingsCollectionImpl.h index 5da5f8ea648..913ba9f5b33 100644 --- a/src/Core/SettingsCollectionImpl.h +++ b/src/Core/SettingsCollectionImpl.h @@ -278,8 +278,12 @@ void SettingsCollection::serialize(WriteBuffer & buf, SettingsBinaryFor { details::SettingsCollectionUtils::serializeName(member.name, buf); if (format >= SettingsBinaryFormat::STRINGS) + { details::SettingsCollectionUtils::serializeFlag(member.is_important, buf); - member.serialize(castToDerived(), buf, format); + details::SettingsCollectionUtils::serializeName(member.get_string(castToDerived()), buf); + } + else + member.write_binary(castToDerived(), buf); } } details::SettingsCollectionUtils::serializeName(StringRef{} /* empty string is a marker of the end of settings */, buf); @@ -296,20 +300,30 @@ void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFo if (name.empty() /* empty string is a marker of the end of settings */) break; auto * member = the_members.find(name); - bool is_important = (format >= SettingsBinaryFormat::STRINGS) ? details::SettingsCollectionUtils::deserializeFlag(buf) : true; - if (member) + bool is_important = true; + if (format >= SettingsBinaryFormat::STRINGS) + is_important = details::SettingsCollectionUtils::deserializeFlag(buf); + if (!member) { - member->deserialize(castToDerived(), buf, format); + if (is_important) + { + details::SettingsCollectionUtils::throwNameNotFound(name); + } + else + { + details::SettingsCollectionUtils::warningNameNotFound(name); + details::SettingsCollectionUtils::skipValue(buf); + continue; + } } - else if (is_important) + + if (format >= SettingsBinaryFormat::STRINGS) { - details::SettingsCollectionUtils::throwNameNotFound(name); + String value = details::SettingsCollectionUtils::deserializeName(buf); + member->set_string(castToDerived(), value); } else - { - details::SettingsCollectionUtils::warningNameNotFound(name); - details::SettingsCollectionUtils::skipValue(buf); - } + member->read_binary(castToDerived(), buf); } } @@ -340,8 +354,8 @@ void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFo static Field NAME##_getField(const Derived & collection) { return collection.NAME.toField(); } \ static void NAME##_setString(Derived & collection, const String & value) { collection.NAME.set(value); } \ static void NAME##_setField(Derived & collection, const Field & value) { collection.NAME.set(value); } \ - static void NAME##_serialize(const Derived & collection, WriteBuffer & buf, SettingsBinaryFormat format) { collection.NAME.serialize(buf, format); } \ - static void NAME##_deserialize(Derived & collection, ReadBuffer & buf, SettingsBinaryFormat format) { collection.NAME.deserialize(buf, format); } \ + static void NAME##_writeBinary(const Derived & collection, WriteBuffer & buf) { collection.NAME.writeBinary(buf); } \ + static void NAME##_readBinary(Derived & collection, ReadBuffer & buf) { collection.NAME.readBinary(buf); } \ static String NAME##_valueToString(const Field & value) { SettingField##TYPE temp{DEFAULT}; temp.set(value); return temp.toString(); } \ static Field NAME##_valueToCorrespondingType(const Field & value) { SettingField##TYPE temp{DEFAULT}; temp.set(value); return temp.toField(); } \ @@ -354,6 +368,6 @@ void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFo [](const Derived & d) { return d.NAME.changed; }, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \ - &Functions::NAME##_serialize, &Functions::NAME##_deserialize, \ + &Functions::NAME##_writeBinary, &Functions::NAME##_readBinary, \ &Functions::NAME##_valueToString, &Functions::NAME##_valueToCorrespondingType}); } diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp new file mode 100644 index 00000000000..1a03f5f4578 --- /dev/null +++ b/src/Core/SettingsEnums.cpp @@ -0,0 +1,94 @@ +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int UNKNOWN_LOAD_BALANCING; + extern const int UNKNOWN_OVERFLOW_MODE; + extern const int UNKNOWN_TOTALS_MODE; + extern const int UNKNOWN_DISTRIBUTED_PRODUCT_MODE; + extern const int UNKNOWN_JOIN; + extern const int BAD_ARGUMENTS; +} + + +IMPLEMENT_SETTING_ENUM(LoadBalancing, ErrorCodes::UNKNOWN_LOAD_BALANCING, + {{"random", LoadBalancing::RANDOM}, + {"nearest_hostname", LoadBalancing::NEAREST_HOSTNAME}, + {"in_order", LoadBalancing::IN_ORDER}, + {"first_or_random", LoadBalancing::FIRST_OR_RANDOM}, + {"round_robin", LoadBalancing::ROUND_ROBIN}}) + + +IMPLEMENT_SETTING_ENUM(SpecialSort, ErrorCodes::UNKNOWN_JOIN, + {{"not_specified", SpecialSort::NOT_SPECIFIED}, + {"opencl_bitonic", SpecialSort::OPENCL_BITONIC}}) + + +IMPLEMENT_SETTING_ENUM(JoinStrictness, ErrorCodes::UNKNOWN_JOIN, + {{"", JoinStrictness::Unspecified}, + {"ALL", JoinStrictness::ALL}, + {"ANY", JoinStrictness::ANY}}) + + +IMPLEMENT_SETTING_ENUM(JoinAlgorithm, ErrorCodes::UNKNOWN_JOIN, + {{"auto", JoinAlgorithm::AUTO}, + {"hash", JoinAlgorithm::HASH}, + {"partial_merge", JoinAlgorithm::PARTIAL_MERGE}, + {"prefer_partial_merge", JoinAlgorithm::PREFER_PARTIAL_MERGE}}) + + +IMPLEMENT_SETTING_ENUM(TotalsMode, ErrorCodes::UNKNOWN_TOTALS_MODE, + {{"before_having", TotalsMode::BEFORE_HAVING}, + {"after_having_exclusive", TotalsMode::AFTER_HAVING_EXCLUSIVE}, + {"after_having_inclusive", TotalsMode::AFTER_HAVING_INCLUSIVE}, + {"after_having_auto", TotalsMode::AFTER_HAVING_AUTO}}) + + +IMPLEMENT_SETTING_ENUM(OverflowMode, ErrorCodes::UNKNOWN_OVERFLOW_MODE, + {{"throw", OverflowMode::THROW}, + {"break", OverflowMode::BREAK}}) + + +IMPLEMENT_SETTING_ENUM_WITH_RENAME(OverflowModeGroupBy, ErrorCodes::UNKNOWN_OVERFLOW_MODE, + {{"throw", OverflowMode::THROW}, + {"break", OverflowMode::BREAK}, + {"any", OverflowMode::ANY}}) + + +IMPLEMENT_SETTING_ENUM(DistributedProductMode, ErrorCodes::UNKNOWN_DISTRIBUTED_PRODUCT_MODE, + {{"deny", DistributedProductMode::DENY}, + {"local", DistributedProductMode::LOCAL}, + {"global", DistributedProductMode::GLOBAL}, + {"allow", DistributedProductMode::ALLOW}}) + + +IMPLEMENT_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, ErrorCodes::BAD_ARGUMENTS, + {{"basic", FormatSettings::DateTimeInputFormat::Basic}, + {"best_effort", FormatSettings::DateTimeInputFormat::BestEffort}}) + + +IMPLEMENT_SETTING_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS, + {{"none", LogsLevel::none}, + {"fatal", LogsLevel::fatal}, + {"error", LogsLevel::error}, + {"warning", LogsLevel::warning}, + {"information", LogsLevel::information}, + {"debug", LogsLevel::debug}, + {"trace", LogsLevel::trace}}) + + +IMPLEMENT_SETTING_ENUM_WITH_RENAME(LogQueriesType, ErrorCodes::BAD_ARGUMENTS, + {{"QUERY_START", QUERY_START}, + {"QUERY_FINISH", QUERY_FINISH}, + {"EXCEPTION_BEFORE_START", EXCEPTION_BEFORE_START}, + {"EXCEPTION_WHILE_PROCESSING", EXCEPTION_WHILE_PROCESSING}}) + + +IMPLEMENT_SETTING_ENUM_WITH_RENAME(DefaultDatabaseEngine, ErrorCodes::BAD_ARGUMENTS, + {{"Ordinary", DefaultDatabaseEngine::Ordinary}, + {"Atomic", DefaultDatabaseEngine::Atomic}}) + +} diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h new file mode 100644 index 00000000000..16ebef87e01 --- /dev/null +++ b/src/Core/SettingsEnums.h @@ -0,0 +1,129 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +enum class LoadBalancing +{ + /// among replicas with a minimum number of errors selected randomly + RANDOM = 0, + /// a replica is selected among the replicas with the minimum number of errors + /// with the minimum number of distinguished characters in the replica name and local hostname + NEAREST_HOSTNAME, + // replicas with the same number of errors are accessed in the same order + // as they are specified in the configuration. + IN_ORDER, + /// if first replica one has higher number of errors, + /// pick a random one from replicas with minimum number of errors + FIRST_OR_RANDOM, + // round robin across replicas with the same number of errors. + ROUND_ROBIN, +}; + +DECLARE_SETTING_ENUM(LoadBalancing) + + +enum class JoinStrictness +{ + Unspecified = 0, /// Query JOIN without strictness will throw Exception. + ALL, /// Query JOIN without strictness -> ALL JOIN ... + ANY, /// Query JOIN without strictness -> ANY JOIN ... +}; + +DECLARE_SETTING_ENUM(JoinStrictness) + +enum class JoinAlgorithm +{ + AUTO = 0, + HASH, + PARTIAL_MERGE, + PREFER_PARTIAL_MERGE, +}; + +DECLARE_SETTING_ENUM(JoinAlgorithm) + + +enum class SpecialSort +{ + NOT_SPECIFIED = 0, + OPENCL_BITONIC, +}; + +DECLARE_SETTING_ENUM(SpecialSort) + + +/// Which rows should be included in TOTALS. +enum class TotalsMode +{ + BEFORE_HAVING = 0, /// Count HAVING for all read rows; + /// including those not in max_rows_to_group_by + /// and have not passed HAVING after grouping. + AFTER_HAVING_INCLUSIVE = 1, /// Count on all rows except those that have not passed HAVING; + /// that is, to include in TOTALS all the rows that did not pass max_rows_to_group_by. + AFTER_HAVING_EXCLUSIVE = 2, /// Include only the rows that passed and max_rows_to_group_by, and HAVING. + AFTER_HAVING_AUTO = 3, /// Automatically select between INCLUSIVE and EXCLUSIVE, +}; + +DECLARE_SETTING_ENUM(TotalsMode) + + +/// The settings keeps OverflowMode which cannot be OverflowMode::ANY. +DECLARE_SETTING_ENUM(OverflowMode) + +/// The settings keeps OverflowMode which can be OverflowMode::ANY. +DECLARE_SETTING_ENUM_WITH_RENAME(OverflowModeGroupBy, OverflowMode) + + +/// The setting for executing distributed subqueries inside IN or JOIN sections. +enum class DistributedProductMode +{ + DENY = 0, /// Disable + LOCAL, /// Convert to local query + GLOBAL, /// Convert to global query + ALLOW /// Enable +}; + +DECLARE_SETTING_ENUM(DistributedProductMode) + + +DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, FormatSettings::DateTimeInputFormat) + + +enum class LogsLevel +{ + none = 0, /// Disable + fatal, + error, + warning, + information, + debug, + trace, +}; + +DECLARE_SETTING_ENUM(LogsLevel) + + +// Make it signed for compatibility with DataTypeEnum8 +enum QueryLogElementType : int8_t +{ + QUERY_START = 1, + QUERY_FINISH = 2, + EXCEPTION_BEFORE_START = 3, + EXCEPTION_WHILE_PROCESSING = 4, +}; + +DECLARE_SETTING_ENUM_WITH_RENAME(LogQueriesType, QueryLogElementType) + + +enum class DefaultDatabaseEngine +{ + Ordinary, + Atomic, +}; + +DECLARE_SETTING_ENUM(DefaultDatabaseEngine) +} diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp new file mode 100644 index 00000000000..b2053a6a8c1 --- /dev/null +++ b/src/Core/SettingsFields.cpp @@ -0,0 +1,365 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH; + extern const int CANNOT_PARSE_BOOL; +} + + +template +String SettingFieldNumber::toString() const +{ + return DB::toString(value); +} + +template +Field SettingFieldNumber::toField() const +{ + return value; +} + +template +void SettingFieldNumber::set(Type x) +{ + value = x; + changed = true; +} + +template +void SettingFieldNumber::set(const Field & x) +{ + if (x.getType() == Field::Types::String) + set(get(x)); + else + set(applyVisitor(FieldVisitorConvertToNumber(), x)); +} + +template +void SettingFieldNumber::set(const String & x) +{ + set(parseWithSizeSuffix(x)); +} + +template <> +void SettingFieldNumber::set(const String & x) +{ + if (x.size() == 1) + { + if (x[0] == '0') + set(false); + else if (x[0] == '1') + set(true); + else + throw Exception("Cannot parse bool from string '" + x + "'", ErrorCodes::CANNOT_PARSE_BOOL); + } + else + { + ReadBufferFromString buf(x); + if (checkStringCaseInsensitive("true", buf)) + set(true); + else if (checkStringCaseInsensitive("false", buf)) + set(false); + else + throw Exception("Cannot parse bool from string '" + x + "'", ErrorCodes::CANNOT_PARSE_BOOL); + } +} + + +template +void SettingFieldNumber::writeBinary(WriteBuffer & out) const +{ + if constexpr (is_integral_v && is_unsigned_v) + writeVarUInt(static_cast(value), out); + else if constexpr (is_integral_v && is_signed_v) + writeVarInt(static_cast(value), out); + else + { + static_assert(std::is_floating_point_v); + writeStringBinary(toString(), out); + } +} + +template +void SettingFieldNumber::readBinary(ReadBuffer & in) +{ + if constexpr (is_integral_v && is_unsigned_v) + { + UInt64 x; + readVarUInt(x, in); + set(static_cast(x)); + } + else if constexpr (is_integral_v && is_signed_v) + { + Int64 x; + readVarInt(x, in); + set(static_cast(x)); + } + else + { + static_assert(std::is_floating_point_v); + String x; + readStringBinary(x, in); + set(x); + } +} + +template struct SettingFieldNumber; +template struct SettingFieldNumber; +template struct SettingFieldNumber; +template struct SettingFieldNumber; + + +String SettingFieldMaxThreads::toString() const +{ + /// Instead of the `auto` value, we output the actual value to make it easier to see. + return is_auto ? ("auto(" + DB::toString(value) + ")") : DB::toString(value); +} + +Field SettingFieldMaxThreads::toField() const +{ + return is_auto ? 0 : value; +} + +void SettingFieldMaxThreads::set(UInt64 x) +{ + value = x ? x : getAutoValue(); + is_auto = x == 0; + changed = true; +} + +void SettingFieldMaxThreads::set(const Field & x) +{ + if (x.getType() == Field::Types::String) + set(get(x)); + else + set(applyVisitor(FieldVisitorConvertToNumber(), x)); +} + +void SettingFieldMaxThreads::set(const String & x) +{ + if (startsWith(x, "auto")) + setAuto(); + else + set(parse(x)); +} + +void SettingFieldMaxThreads::setAuto() +{ + value = getAutoValue(); + is_auto = true; +} + +UInt64 SettingFieldMaxThreads::getAutoValue() +{ + static auto res = getNumberOfPhysicalCPUCores(); + return res; +} + +void SettingFieldMaxThreads::writeBinary(WriteBuffer & out) const +{ + writeVarUInt(is_auto ? 0 : value, out); +} + +void SettingFieldMaxThreads::readBinary(ReadBuffer & in) +{ + UInt64 x = 0; + readVarUInt(x, in); + set(x); +} + + +template +String SettingFieldTimespan::toString() const +{ + return DB::toString(value.totalMicroseconds() / microseconds_per_unit); +} + +template +Field SettingFieldTimespan::toField() const +{ + return value.totalMicroseconds() / microseconds_per_unit; +} + +template +void SettingFieldTimespan::set(const Poco::Timespan & x) +{ + value = x; + changed = true; +} + +template +void SettingFieldTimespan::set(UInt64 x) +{ + set(Poco::Timespan(x * microseconds_per_unit)); +} + +template +void SettingFieldTimespan::set(const Field & x) +{ + if (x.getType() == Field::Types::String) + set(get(x)); + else + set(applyVisitor(FieldVisitorConvertToNumber(), x)); +} + +template +void SettingFieldTimespan::set(const String & x) +{ + set(parse(x)); +} + +template +void SettingFieldTimespan::writeBinary(WriteBuffer & out) const +{ + writeVarUInt(value.totalMicroseconds() / microseconds_per_unit, out); +} + +template +void SettingFieldTimespan::readBinary(ReadBuffer & in) +{ + UInt64 x = 0; + readVarUInt(x, in); + set(x); +} + +template struct SettingFieldTimespan; +template struct SettingFieldTimespan; + + +Field SettingFieldString::toField() const +{ + return value; +} + +void SettingFieldString::set(const String & x) +{ + value = x; + changed = true; +} + +void SettingFieldString::set(const Field & x) +{ + set(safeGet(x)); +} + +void SettingFieldString::writeBinary(WriteBuffer & out) const +{ + writeStringBinary(value, out); +} + +void SettingFieldString::readBinary(ReadBuffer & in) +{ + String s; + readStringBinary(s, in); + set(s); +} + + +String SettingFieldChar::toString() const +{ + return String(1, value); +} + +Field SettingFieldChar::toField() const +{ + return toString(); +} + +void SettingFieldChar::set(char x) +{ + value = x; + changed = true; +} + +void SettingFieldChar::set(const String & x) +{ + if (x.size() > 1) + throw Exception("A setting's value string has to be an exactly one character long", ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH); + char c = (x.size() == 1) ? x[0] : '\0'; + set(c); +} + +void SettingFieldChar::set(const Field & x) +{ + const String & s = safeGet(x); + set(s); +} + +void SettingFieldChar::writeBinary(WriteBuffer & out) const +{ + writeStringBinary(toString(), out); +} + +void SettingFieldChar::readBinary(ReadBuffer & in) +{ + String s; + readStringBinary(s, in); + set(s); +} + + +String SettingFieldURI::toString() const +{ + return value.toString(); +} + +Field SettingFieldURI::toField() const +{ + return value.toString(); +} + +void SettingFieldURI::set(const Poco::URI & x) +{ + value = x; + changed = true; +} + +void SettingFieldURI::set(const Field & x) +{ + const String & s = safeGet(x); + set(s); +} + +void SettingFieldURI::set(const String & x) +{ + set(Poco::URI(x)); +} + +void SettingFieldURI::writeBinary(WriteBuffer & buf) const +{ + writeStringBinary(toString(), buf); +} + +void SettingFieldURI::readBinary(ReadBuffer & buf) +{ + String s; + readStringBinary(s, buf); + set(s); +} + + +void SettingFieldEnumHelpers::writeBinary(const std::string_view & str, WriteBuffer & out) +{ + writeStringBinary(str, out); +} + +String SettingFieldEnumHelpers::readBinary(ReadBuffer & in) +{ + String str; + readStringBinary(str, in); + return str; +} + +} diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h new file mode 100644 index 00000000000..a972180eaee --- /dev/null +++ b/src/Core/SettingsFields.h @@ -0,0 +1,313 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +class ReadBuffer; +class WriteBuffer; + + +/** One setting for any type. + * Stores a value within itself, as well as a flag - whether the value was changed. + * This is done so that you can send to the remote servers only changed settings (or explicitly specified in the config) values. + * That is, if the configuration was not specified in the config and was not dynamically changed, it is not sent to the remote server, + * and the remote server will use its default value. + */ + +template +struct SettingFieldNumber +{ + Type value; + bool changed = false; + + SettingFieldNumber(Type x = 0) : value(x) {} + + operator Type() const { return value; } + SettingFieldNumber & operator=(Type x) { set(x); return *this; } + + /// Serialize to a test string. + String toString() const; + + /// Converts to a field. + Field toField() const; + + void set(Type x); + + /// Read from SQL literal. + void set(const Field & x); + + /// Read from text string. + void set(const String & x); + + /// Serialize to binary stream. + void writeBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); +}; + +using SettingFieldUInt64 = SettingFieldNumber; +using SettingFieldInt64 = SettingFieldNumber; +using SettingFieldFloat = SettingFieldNumber; +using SettingFieldBool = SettingFieldNumber; + + +/** Unlike SettingFieldUInt64, supports the value of 'auto' - the number of processor cores without taking into account SMT. + * A value of 0 is also treated as auto. + * When serializing, `auto` is written in the same way as 0. + */ +struct SettingFieldMaxThreads +{ + UInt64 value; + bool is_auto; + bool changed = false; + + SettingFieldMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {} + + operator UInt64() const { return value; } + SettingFieldMaxThreads & operator=(UInt64 x) { set(x); return *this; } + + String toString() const; + Field toField() const; + + void set(UInt64 x); + void set(const Field & x); + void set(const String & x); + + void setAuto(); + static UInt64 getAutoValue(); + + void writeBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); +}; + + +enum class SettingFieldTimespanUnit { MILLISECOND, SECOND }; + +template +struct SettingFieldTimespan +{ + static constexpr UInt64 microseconds_per_unit = (unit == SettingFieldTimespanUnit::MILLISECOND) ? 1000 : 1000000; + + Poco::Timespan value; + bool changed = false; + + SettingFieldTimespan(UInt64 x = 0) : value(x * microseconds_per_unit) {} + + operator Poco::Timespan() const { return value; } + SettingFieldTimespan & operator=(const Poco::Timespan & x) { set(x); return *this; } + + template > + operator std::chrono::duration() const { return std::chrono::duration_cast>(std::chrono::microseconds(value.totalMicroseconds())); } + + template > + SettingFieldTimespan & operator=(const std::chrono::duration & x) { set(x); return *this; } + + Poco::Timespan::TimeDiff totalSeconds() const { return value.totalSeconds(); } + Poco::Timespan::TimeDiff totalMilliseconds() const { return value.totalMilliseconds(); } + + String toString() const; + Field toField() const; + + void set(const Poco::Timespan & x); + + template > + void set(const std::chrono::duration & duration) { set(static_cast(std::chrono::duration_cast(duration).count())); } + + void set(UInt64 x); + void set(const Field & x); + void set(const String & x); + + void writeBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); +}; + +using SettingFieldSeconds = SettingFieldTimespan; +using SettingFieldMilliseconds = SettingFieldTimespan; + + +struct SettingFieldString +{ + String value; + bool changed = false; + + SettingFieldString(const String & x = String{}) : value(x) {} + + operator String() const { return value; } + SettingFieldString & operator=(const String & x) { set(x); return *this; } + + const String & toString() const { return value; } + Field toField() const; + + void set(const String & x); + void set(const Field & x); + + void writeBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); +}; + + +struct SettingFieldChar +{ +public: + char value; + bool changed = false; + SettingFieldChar(char x = '\0') : value(x) {} + + operator char() const { return value; } + SettingFieldChar & operator=(char x) { set(x); return *this; } + + String toString() const; + Field toField() const; + + void set(char x); + void set(const String & x); + void set(const Field & x); + + void writeBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); +}; + + +struct SettingFieldURI +{ + Poco::URI value; + bool changed = false; + + SettingFieldURI(const Poco::URI & x = Poco::URI{}) : value(x) {} + + operator Poco::URI() const { return value; } + SettingFieldURI & operator=(const Poco::URI & x) { set(x); return *this; } + + String toString() const; + Field toField() const; + + void set(const Poco::URI & x); + void set(const Field & x); + void set(const String & x); + + void writeBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); +}; + + +/** Template class to define enum-based settings. + * Example of usage: + * + * mysettings.h: + * enum Gender { Male, Female }; + * DECLARE_SETTING_ENUM(SettingFieldGender, Gender) + * + * mysettings.cpp: + * IMPLEMENT_SETTING_ENUM(SettingFieldGender, ErrorCodes::BAD_ARGUMENTS, + * {{"Male", Gender::Male}, {"Female", Gender::Female}}) + */ +template +struct SettingFieldEnum +{ + EnumType value; + bool changed = false; + + SettingFieldEnum(EnumType x) : value(x) {} + + operator EnumType() const { return value; } + SettingFieldEnum & operator=(EnumType x) { set(x); return *this; } + + const String & toString() const { return NameValueConverter::toString(value); } + Field toField() const { return toString(); } + + void set(EnumType x) { value = x; changed = true; } + void set(const Field & x) { set(safeGet(x)); } + void set(const String & x) { set(NameValueConverter::fromString(x)); } + + void writeBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); +}; + +struct SettingFieldEnumHelpers +{ + static void writeBinary(const std::string_view & str, WriteBuffer & out); + static String readBinary(ReadBuffer & in); +}; + +template +inline void SettingFieldEnum::writeBinary(WriteBuffer & out) const +{ + SettingFieldEnumHelpers::writeBinary(toString(), out); +} + +template +inline void SettingFieldEnum::readBinary(ReadBuffer & in) +{ + set(SettingFieldEnumHelpers::readBinary(in)); +} + +#define DECLARE_SETTING_ENUM(ENUM_TYPE) \ + DECLARE_SETTING_ENUM_WITH_RENAME(ENUM_TYPE, ENUM_TYPE) + +#define IMPLEMENT_SETTING_ENUM(ENUM_TYPE, ERROR_CODE_FOR_UNEXPECTED_NAME, ...) \ + IMPLEMENT_SETTING_ENUM_WITH_RENAME(ENUM_TYPE, ERROR_CODE_FOR_UNEXPECTED_NAME, __VA_ARGS__) + +#define DECLARE_SETTING_ENUM_WITH_RENAME(NEW_NAME, ENUM_TYPE) \ + struct SettingField##NEW_NAME##NameValueConverter \ + { \ + using EnumType = ENUM_TYPE; \ + static const String & toString(EnumType value); \ + static EnumType fromString(const std::string_view & str); \ + }; \ + \ + using SettingField##NEW_NAME = SettingFieldEnum; + +#define IMPLEMENT_SETTING_ENUM_WITH_RENAME(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, ...) \ + const String & SettingField##NEW_NAME##NameValueConverter::toString(typename SettingField##NEW_NAME##NameValueConverter::EnumType value) \ + { \ + static const std::unordered_map map = [] { \ + std::unordered_map res; \ + constexpr std::pair pairs[] = __VA_ARGS__; \ + for (const auto & [name, val] : pairs) \ + res.emplace(val, name); \ + return res; \ + }(); \ + auto it = map.find(value); \ + if (it != map.end()) \ + return it->second; \ + throw Exception( \ + "Unexpected value of " #NEW_NAME ":" + std::to_string(std::underlying_type::type(value)), \ + ERROR_CODE_FOR_UNEXPECTED_NAME); \ + } \ + \ + typename SettingField##NEW_NAME##NameValueConverter::EnumType SettingField##NEW_NAME##NameValueConverter::fromString(const std::string_view & str) \ + { \ + static const std::unordered_map map = [] { \ + std::unordered_map res; \ + constexpr std::pair pairs[] = __VA_ARGS__; \ + for (const auto & [name, val] : pairs) \ + res.emplace(name, val); \ + return res; \ + }(); \ + auto it = map.find(str); \ + if (it != map.end()) \ + return it->second; \ + String msg = "Unexpected value of " #NEW_NAME ": '" + String{str} + "'. Must be one of ["; \ + bool need_comma = false; \ + for (auto & name : map | boost::adaptors::map_keys) \ + { \ + if (std::exchange(need_comma, true)) \ + msg += ", "; \ + msg += "'" + String{name} + "'"; \ + } \ + msg += "]"; \ + throw Exception(msg, ERROR_CODE_FOR_UNEXPECTED_NAME); \ + } +} diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index d433d369d0b..2198789b0b6 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include class Collator; diff --git a/src/Core/ya.make b/src/Core/ya.make index 14d609dfa96..066c45e17ba 100644 --- a/src/Core/ya.make +++ b/src/Core/ya.make @@ -20,6 +20,8 @@ SRCS( NamesAndTypes.cpp Settings.cpp SettingsCollection.cpp + SettingsEnums.cpp + SettingsFields.cpp SortDescription.cpp ) diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 6fbb8342f30..09880d44eaa 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -2,7 +2,6 @@ #include #include -#include namespace ProfileEvents diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 77e32c1076d..cad4c979004 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include From 300727afa316a01cb1685636090f6a89a9116887 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 27 Jul 2020 17:26:18 +0300 Subject: [PATCH 09/20] Rework the StringField* classes and make conversion String => StringField* explicit. --- src/Access/SettingsConstraints.cpp | 8 +- src/Access/SettingsProfileElement.cpp | 6 +- src/Access/UsersConfigAccessStorage.cpp | 6 +- src/Core/Settings.h | 2 +- src/Core/SettingsCollection.h | 42 +-- src/Core/SettingsCollectionImpl.h | 41 +-- src/Core/SettingsFields.cpp | 369 ++++++++++-------------- src/Core/SettingsFields.h | 195 +++++++------ src/IO/ReadHelpers.h | 4 +- src/Storages/Kafka/StorageKafka.cpp | 6 +- src/Storages/StorageJoin.cpp | 12 +- 11 files changed, 328 insertions(+), 363 deletions(-) diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 8ca2262f8cd..2c16f23583f 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -38,7 +38,7 @@ void SettingsConstraints::setMinValue(const StringRef & setting_name, const Fiel void SettingsConstraints::setMinValue(size_t setting_index, const Field & min_value) { - getConstraintRef(setting_index).min_value = Settings::valueToCorrespondingType(setting_index, min_value); + getConstraintRef(setting_index).min_value = Settings::castValue(setting_index, min_value); } Field SettingsConstraints::getMinValue(const StringRef & setting_name) const @@ -63,7 +63,7 @@ void SettingsConstraints::setMaxValue(const StringRef & name, const Field & max_ void SettingsConstraints::setMaxValue(size_t setting_index, const Field & max_value) { - getConstraintRef(setting_index).max_value = Settings::valueToCorrespondingType(setting_index, max_value); + getConstraintRef(setting_index).max_value = Settings::castValue(setting_index, max_value); } Field SettingsConstraints::getMaxValue(const StringRef & setting_name) const @@ -179,7 +179,7 @@ void SettingsConstraints::check(const Settings & current_settings, const Setting if (setting_index == Settings::npos) return; - Field new_value = Settings::valueToCorrespondingType(setting_index, change.value); + Field new_value = Settings::castValue(setting_index, change.value); Field current_value = current_settings.get(setting_index); /// Setting isn't checked if value wasn't changed. @@ -233,7 +233,7 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingChange if (setting_index == Settings::npos) return; - Field new_value = Settings::valueToCorrespondingType(setting_index, change.value); + Field new_value = Settings::castValue(setting_index, change.value); Field current_value = current_settings.get(setting_index); /// Setting isn't checked if value wasn't changed. diff --git a/src/Access/SettingsProfileElement.cpp b/src/Access/SettingsProfileElement.cpp index 4fbe4aec2f8..cb9b26cce53 100644 --- a/src/Access/SettingsProfileElement.cpp +++ b/src/Access/SettingsProfileElement.cpp @@ -43,11 +43,11 @@ void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const A readonly = ast.readonly; if (!value.isNull()) - value = Settings::valueToCorrespondingType(setting_index, value); + value = Settings::castValue(setting_index, value); if (!min_value.isNull()) - min_value = Settings::valueToCorrespondingType(setting_index, min_value); + min_value = Settings::castValue(setting_index, min_value); if (!max_value.isNull()) - max_value = Settings::valueToCorrespondingType(setting_index, max_value); + max_value = Settings::castValue(setting_index, max_value); } } diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index e3fb9104a66..a4a0acfc664 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -375,9 +375,9 @@ namespace for (const String & constraint_type : constraint_types) { if (constraint_type == "min") - profile_element.min_value = Settings::valueToCorrespondingType(setting_index, config.getString(path_to_name + "." + constraint_type)); + profile_element.min_value = Settings::stringToValue(setting_index, config.getString(path_to_name + "." + constraint_type)); else if (constraint_type == "max") - profile_element.max_value = Settings::valueToCorrespondingType(setting_index, config.getString(path_to_name + "." + constraint_type)); + profile_element.max_value = Settings::stringToValue(setting_index, config.getString(path_to_name + "." + constraint_type)); else if (constraint_type == "readonly") profile_element.readonly = true; else @@ -419,7 +419,7 @@ namespace SettingsProfileElement profile_element; size_t setting_index = Settings::findIndexStrict(key); profile_element.setting_index = setting_index; - profile_element.value = Settings::valueToCorrespondingType(setting_index, config.getString(profile_config + "." + key)); + profile_element.value = Settings::stringToValue(setting_index, config.getString(profile_config + "." + key)); profile->elements.emplace_back(std::move(profile_element)); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 208de421b76..9cc26f9c324 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -433,7 +433,7 @@ struct Settings : public SettingsCollection 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(URI, format_avro_schema_registry_url, {}, "For AvroConfluent format: Confluent Schema Registry URL.", 0) \ + M(URI, format_avro_schema_registry_url, "", "For AvroConfluent format: Confluent Schema Registry URL.", 0) \ \ M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ \ diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h index 5a6bef3d1d0..9b19e7ae04e 100644 --- a/src/Core/SettingsCollection.h +++ b/src/Core/SettingsCollection.h @@ -49,28 +49,30 @@ private: struct MemberInfo { using IsChangedFunction = bool (*)(const Derived &); - using GetStringFunction = String (*)(const Derived &); - using GetFieldFunction = Field (*)(const Derived &); - using SetStringFunction = void (*)(Derived &, const String &); - using SetFieldFunction = void (*)(Derived &, const Field &); + using GetValueFunction = Field (*)(const Derived &); + using SetValueFunction = void (*)(Derived &, const Field &); + using GetValueAsStringFunction = String (*)(const Derived &); + using ParseValueFromStringFunction = void (*)(Derived &, const String &); using WriteBinaryFunction = void (*)(const Derived &, WriteBuffer & buf); using ReadBinaryFunction = void (*)(Derived &, ReadBuffer & buf); + using CastValueFunction = Field (*)(const Field &); + using StringToValueFunction = Field (*)(const String &); using ValueToStringFunction = String (*)(const Field &); - using ValueToCorrespondingTypeFunction = Field (*)(const Field &); StringRef name; StringRef description; StringRef type; bool is_important; IsChangedFunction is_changed; - GetStringFunction get_string; - GetFieldFunction get_field; - SetStringFunction set_string; - SetFieldFunction set_field; + GetValueFunction get_value; + SetValueFunction set_value; + GetValueAsStringFunction get_value_as_string; + ParseValueFromStringFunction parse_value_from_string; WriteBinaryFunction write_binary; ReadBinaryFunction read_binary; + CastValueFunction cast_value; + StringToValueFunction string_to_value; ValueToStringFunction value_to_string; - ValueToCorrespondingTypeFunction value_to_corresponding_type; }; class MemberInfos @@ -110,7 +112,7 @@ public: const StringRef & getType() const { return member->type; } bool isChanged() const { return member->is_changed(*collection); } Field getValue() const; - String getValueAsString() const { return member->get_string(*collection); } + String getValueAsString() const { return member->get_value_as_string(*collection); } protected: friend class SettingsCollection::const_iterator; @@ -126,8 +128,8 @@ public: public: reference(Derived & collection_, const MemberInfo & member_) : const_reference(collection_, member_) {} reference(const const_reference & src) : const_reference(src) {} - void setValue(const Field & value) { this->member->set_field(*const_cast(this->collection), value); } - void setValue(const String & value) { this->member->set_string(*const_cast(this->collection), value); } + void setValue(const Field & value) { this->member->set_value(*const_cast(this->collection), value); } + void parseFromString(const String & value) { this->member->parse_value_from_string(*const_cast(this->collection), value); } }; /// Iterator to iterating through all the settings. @@ -178,14 +180,16 @@ public: /// Searches a setting by its name; throws an exception if not found. static size_t findIndexStrict(const StringRef & name) { return members().findIndexStrict(name); } - /// Casts a value to a string according to a specified setting without actual changing this settings. - static String valueToString(size_t index, const Field & value) { return members()[index].value_to_string(value); } - static String valueToString(const StringRef & name, const Field & value) { return members().findStrict(name).value_to_string(value); } - /// Casts a value to a type according to a specified setting without actual changing this settings. /// E.g. for SettingInt64 it casts Field to Field::Types::Int64. - static Field valueToCorrespondingType(size_t index, const Field & value); - static Field valueToCorrespondingType(const StringRef & name, const Field & value); + static Field castValue(size_t index, const Field & value); + static Field castValue(const StringRef & name, const Field & value); + + /// Casts a value to a string according to a specified setting without actual changing this settings. + static Field stringToValue(size_t index, const String & str) { return members()[index].string_to_value(str); } + static Field stringToValue(const StringRef & name, const String & str) { return members().findStrict(name).string_to_value(str); } + static String valueToString(size_t index, const Field & value) { return members()[index].value_to_string(value); } + static String valueToString(const StringRef & name, const Field & value) { return members().findStrict(name).value_to_string(value); } iterator begin() { return iterator(castToDerived(), members().data()); } const_iterator begin() const { return const_iterator(castToDerived(), members().data()); } diff --git a/src/Core/SettingsCollectionImpl.h b/src/Core/SettingsCollectionImpl.h index 913ba9f5b33..fd365b9587e 100644 --- a/src/Core/SettingsCollectionImpl.h +++ b/src/Core/SettingsCollectionImpl.h @@ -6,8 +6,10 @@ * instantiation of SettingsCollection<>. */ +#include #include #include +#include namespace DB @@ -86,16 +88,16 @@ SettingsCollection::members() template Field SettingsCollection::const_reference::getValue() const { - return member->get_field(*collection); + return member->get_value(*collection); } template -Field SettingsCollection::valueToCorrespondingType(size_t index, const Field & value) +Field SettingsCollection::castValue(size_t index, const Field & value) { try { - return members()[index].value_to_corresponding_type(value); + return members()[index].cast_value(value); } catch (Exception & e) { @@ -107,9 +109,9 @@ Field SettingsCollection::valueToCorrespondingType(size_t index, const template -Field SettingsCollection::valueToCorrespondingType(const StringRef & name, const Field & value) +Field SettingsCollection::castValue(const StringRef & name, const Field & value) { - return members().findStrict(name).value_to_corresponding_type(value); + return members().findStrict(name).cast_value(value); } @@ -196,7 +198,7 @@ bool SettingsCollection::operator ==(const SettingsCollection { if (left_changed != right_changed) return false; - if (member.get_field(castToDerived()) != member.get_field(rhs.castToDerived())) + if (member.get_value(castToDerived()) != member.get_value(rhs.castToDerived())) return false; } } @@ -213,7 +215,7 @@ SettingsChanges SettingsCollection::changes() const { const auto & member = the_members[i]; if (member.is_changed(castToDerived())) - found_changes.push_back({member.name.toString(), member.get_field(castToDerived())}); + found_changes.push_back({member.name.toString(), member.get_value(castToDerived())}); } return found_changes; } @@ -255,7 +257,7 @@ void SettingsCollection::copyChangesFrom(const Derived & src) { const auto & member = the_members[i]; if (member.is_changed(src)) - member.set_field(castToDerived(), member.get_field(src)); + member.set_value(castToDerived(), member.get_value(src)); } } @@ -280,7 +282,7 @@ void SettingsCollection::serialize(WriteBuffer & buf, SettingsBinaryFor if (format >= SettingsBinaryFormat::STRINGS) { details::SettingsCollectionUtils::serializeFlag(member.is_important, buf); - details::SettingsCollectionUtils::serializeName(member.get_string(castToDerived()), buf); + details::SettingsCollectionUtils::serializeName(member.get_value_as_string(castToDerived()), buf); } else member.write_binary(castToDerived(), buf); @@ -320,7 +322,7 @@ void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFo if (format >= SettingsBinaryFormat::STRINGS) { String value = details::SettingsCollectionUtils::deserializeName(buf); - member->set_string(castToDerived(), value); + member->parse_value_from_string(castToDerived(), value); } else member->read_binary(castToDerived(), buf); @@ -350,14 +352,15 @@ void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFo #define IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ - static String NAME##_getString(const Derived & collection) { return collection.NAME.toString(); } \ - static Field NAME##_getField(const Derived & collection) { return collection.NAME.toField(); } \ - static void NAME##_setString(Derived & collection, const String & value) { collection.NAME.set(value); } \ - static void NAME##_setField(Derived & collection, const Field & value) { collection.NAME.set(value); } \ + static Field NAME##_getValue(const Derived & collection) { return static_cast(collection.NAME); } \ + static void NAME##_setValue(Derived & collection, const Field & value) { collection.NAME = value; } \ + static String NAME##_toString(const Derived & collection) { return collection.NAME.toString(); } \ + static void NAME##_parseFromString(Derived & collection, const String & value) { collection.NAME.parseFromString(value); } \ static void NAME##_writeBinary(const Derived & collection, WriteBuffer & buf) { collection.NAME.writeBinary(buf); } \ static void NAME##_readBinary(Derived & collection, ReadBuffer & buf) { collection.NAME.readBinary(buf); } \ - static String NAME##_valueToString(const Field & value) { SettingField##TYPE temp{DEFAULT}; temp.set(value); return temp.toString(); } \ - static Field NAME##_valueToCorrespondingType(const Field & value) { SettingField##TYPE temp{DEFAULT}; temp.set(value); return temp.toField(); } \ + static Field NAME##_castValue(const Field & value) { return static_cast(SettingField##TYPE{value}); } \ + static Field NAME##_stringToValue(const String & str) { SettingField##TYPE temp; temp.parseFromString(str); return static_cast(temp); } \ + static String NAME##_valueToString(const Field & value) { return SettingField##TYPE{value}.toString(); } \ #define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ @@ -366,8 +369,8 @@ void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFo StringRef(#TYPE, strlen(#TYPE)), \ FLAGS & IMPORTANT, \ [](const Derived & d) { return d.NAME.changed; }, \ - &Functions::NAME##_getString, &Functions::NAME##_getField, \ - &Functions::NAME##_setString, &Functions::NAME##_setField, \ + &Functions::NAME##_getValue, &Functions::NAME##_setValue, \ + &Functions::NAME##_toString, &Functions::NAME##_parseFromString, \ &Functions::NAME##_writeBinary, &Functions::NAME##_readBinary, \ - &Functions::NAME##_valueToString, &Functions::NAME##_valueToCorrespondingType}); + &Functions::NAME##_castValue, &Functions::NAME##_stringToValue, &Functions::NAME##_valueToString }); } diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index b2053a6a8c1..628e6ef6d4c 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -18,100 +19,96 @@ namespace ErrorCodes } -template -String SettingFieldNumber::toString() const +namespace { - return DB::toString(value); -} - -template -Field SettingFieldNumber::toField() const -{ - return value; -} - -template -void SettingFieldNumber::set(Type x) -{ - value = x; - changed = true; -} - -template -void SettingFieldNumber::set(const Field & x) -{ - if (x.getType() == Field::Types::String) - set(get(x)); - else - set(applyVisitor(FieldVisitorConvertToNumber(), x)); -} - -template -void SettingFieldNumber::set(const String & x) -{ - set(parseWithSizeSuffix(x)); -} - -template <> -void SettingFieldNumber::set(const String & x) -{ - if (x.size() == 1) + template + T stringToNumber(const String & str) { - if (x[0] == '0') - set(false); - else if (x[0] == '1') - set(true); + if constexpr (std::is_same_v) + { + if (str == "0") + return false; + if (str == "1") + return true; + if (boost::iequals(str, "false")) + return false; + if (boost::iequals(str, "true")) + return true; + throw Exception("Cannot parse bool from string '" + str + "'", ErrorCodes::CANNOT_PARSE_BOOL); + } else - throw Exception("Cannot parse bool from string '" + x + "'", ErrorCodes::CANNOT_PARSE_BOOL); + return parseWithSizeSuffix(str); } - else + + template + T fieldToNumber(const Field & f) { - ReadBufferFromString buf(x); - if (checkStringCaseInsensitive("true", buf)) - set(true); - else if (checkStringCaseInsensitive("false", buf)) - set(false); + if (f.getType() == Field::Types::String) + return stringToNumber(f.get()); else - throw Exception("Cannot parse bool from string '" + x + "'", ErrorCodes::CANNOT_PARSE_BOOL); + return applyVisitor(FieldVisitorConvertToNumber(), f); } } - -template -void SettingFieldNumber::writeBinary(WriteBuffer & out) const +template +SettingFieldNumber::SettingFieldNumber(const Field & f) : SettingFieldNumber(fieldToNumber(f)) { - if constexpr (is_integral_v && is_unsigned_v) +} + +template +SettingFieldNumber & SettingFieldNumber::operator=(const Field & f) +{ + *this = fieldToNumber(f); + return *this; +} + +template +String SettingFieldNumber::toString() const +{ + return ::DB::toString(value); +} + +template +void SettingFieldNumber::parseFromString(const String & str) +{ + *this = stringToNumber(str); +} + +template +void SettingFieldNumber::writeBinary(WriteBuffer & out) const +{ + if constexpr (is_integral_v && is_unsigned_v) writeVarUInt(static_cast(value), out); - else if constexpr (is_integral_v && is_signed_v) + else if constexpr (is_integral_v && is_signed_v) writeVarInt(static_cast(value), out); else { - static_assert(std::is_floating_point_v); - writeStringBinary(toString(), out); + static_assert(std::is_floating_point_v); + writeStringBinary(::DB::toString(value), out); } } -template -void SettingFieldNumber::readBinary(ReadBuffer & in) +template +void SettingFieldNumber::readBinary(ReadBuffer & in) { - if constexpr (is_integral_v && is_unsigned_v) + if constexpr (is_integral_v && is_unsigned_v) { UInt64 x; readVarUInt(x, in); - set(static_cast(x)); + *this = static_cast(x); } - else if constexpr (is_integral_v && is_signed_v) + else if constexpr (is_integral_v && is_signed_v) { Int64 x; readVarInt(x, in); - set(static_cast(x)); + *this = static_cast(value); } else { - static_assert(std::is_floating_point_v); - String x; - readStringBinary(x, in); - set(x); + static_assert(std::is_floating_point_v); + String str; + readStringBinary(str, in); + *this = ::DB::parseFromString(str); } } @@ -121,50 +118,45 @@ template struct SettingFieldNumber; template struct SettingFieldNumber; +namespace +{ + UInt64 stringToMaxThreads(const String & str) + { + if (startsWith(str, "auto")) + return 0; + return parseFromString(str); + } + + UInt64 fieldToMaxThreads(const Field & f) + { + if (f.getType() == Field::Types::String) + return stringToMaxThreads(f.get()); + else + return applyVisitor(FieldVisitorConvertToNumber(), f); + } +} + +SettingFieldMaxThreads::SettingFieldMaxThreads(const Field & f) : SettingFieldMaxThreads(fieldToMaxThreads(f)) +{ +} + +SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) +{ + *this = fieldToMaxThreads(f); + return *this; +} + String SettingFieldMaxThreads::toString() const { - /// Instead of the `auto` value, we output the actual value to make it easier to see. - return is_auto ? ("auto(" + DB::toString(value) + ")") : DB::toString(value); -} - -Field SettingFieldMaxThreads::toField() const -{ - return is_auto ? 0 : value; -} - -void SettingFieldMaxThreads::set(UInt64 x) -{ - value = x ? x : getAutoValue(); - is_auto = x == 0; - changed = true; -} - -void SettingFieldMaxThreads::set(const Field & x) -{ - if (x.getType() == Field::Types::String) - set(get(x)); + if (is_auto) + return "'auto(" + ::DB::toString(value) + ")'"; else - set(applyVisitor(FieldVisitorConvertToNumber(), x)); + return ::DB::toString(value); } -void SettingFieldMaxThreads::set(const String & x) +void SettingFieldMaxThreads::parseFromString(const String & str) { - if (startsWith(x, "auto")) - setAuto(); - else - set(parse(x)); -} - -void SettingFieldMaxThreads::setAuto() -{ - value = getAutoValue(); - is_auto = true; -} - -UInt64 SettingFieldMaxThreads::getAutoValue() -{ - static auto res = getNumberOfPhysicalCPUCores(); - return res; + *this = stringToMaxThreads(str); } void SettingFieldMaxThreads::writeBinary(WriteBuffer & out) const @@ -176,83 +168,57 @@ void SettingFieldMaxThreads::readBinary(ReadBuffer & in) { UInt64 x = 0; readVarUInt(x, in); - set(x); + *this = x; } - -template -String SettingFieldTimespan::toString() const +UInt64 SettingFieldMaxThreads::getAuto() { - return DB::toString(value.totalMicroseconds() / microseconds_per_unit); + return getNumberOfPhysicalCPUCores(); } -template -Field SettingFieldTimespan::toField() const + +template +SettingFieldTimespan::SettingFieldTimespan(const Field & f) : SettingFieldTimespan(fieldToNumber(f)) { - return value.totalMicroseconds() / microseconds_per_unit; } -template -void SettingFieldTimespan::set(const Poco::Timespan & x) +template +SettingFieldTimespan & SettingFieldTimespan::operator=(const Field & f) { - value = x; - changed = true; + *this = fieldToNumber(f); + return *this; } -template -void SettingFieldTimespan::set(UInt64 x) +template +String SettingFieldTimespan::toString() const { - set(Poco::Timespan(x * microseconds_per_unit)); + return ::DB::toString(operator UInt64()); } -template -void SettingFieldTimespan::set(const Field & x) +template +void SettingFieldTimespan::parseFromString(const String & str) { - if (x.getType() == Field::Types::String) - set(get(x)); - else - set(applyVisitor(FieldVisitorConvertToNumber(), x)); + *this = stringToNumber(str); } -template -void SettingFieldTimespan::set(const String & x) +template +void SettingFieldTimespan::writeBinary(WriteBuffer & out) const { - set(parse(x)); + auto num_units = operator UInt64(); + writeVarUInt(num_units, out); } -template -void SettingFieldTimespan::writeBinary(WriteBuffer & out) const +template +void SettingFieldTimespan::readBinary(ReadBuffer & in) { - writeVarUInt(value.totalMicroseconds() / microseconds_per_unit, out); + UInt64 num_units = 0; + readVarUInt(num_units, in); + *this = num_units; } -template -void SettingFieldTimespan::readBinary(ReadBuffer & in) -{ - UInt64 x = 0; - readVarUInt(x, in); - set(x); -} +template struct SettingFieldTimespan; +template struct SettingFieldTimespan; -template struct SettingFieldTimespan; -template struct SettingFieldTimespan; - - -Field SettingFieldString::toField() const -{ - return value; -} - -void SettingFieldString::set(const String & x) -{ - value = x; - changed = true; -} - -void SettingFieldString::set(const Field & x) -{ - set(safeGet(x)); -} void SettingFieldString::writeBinary(WriteBuffer & out) const { @@ -261,40 +227,42 @@ void SettingFieldString::writeBinary(WriteBuffer & out) const void SettingFieldString::readBinary(ReadBuffer & in) { - String s; - readStringBinary(s, in); - set(s); + String str; + readStringBinary(str, in); + *this = std::move(str); } -String SettingFieldChar::toString() const +namespace { - return String(1, value); + char stringToChar(const String & str) + { + if (str.size() > 1) + throw Exception("A setting's value string has to be an exactly one character long", ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH); + if (str.empty()) + return '\0'; + return str[0]; + } + + char fieldToChar(const Field & f) + { + return stringToChar(f.safeGet()); + } } -Field SettingFieldChar::toField() const +SettingFieldChar::SettingFieldChar(const Field & f) : SettingFieldChar(fieldToChar(f)) { - return toString(); } -void SettingFieldChar::set(char x) +SettingFieldChar & SettingFieldChar::operator =(const Field & f) { - value = x; - changed = true; + *this = fieldToChar(f); + return *this; } -void SettingFieldChar::set(const String & x) +void SettingFieldChar::parseFromString(const String & str) { - if (x.size() > 1) - throw Exception("A setting's value string has to be an exactly one character long", ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH); - char c = (x.size() == 1) ? x[0] : '\0'; - set(c); -} - -void SettingFieldChar::set(const Field & x) -{ - const String & s = safeGet(x); - set(s); + *this = stringToChar(str); } void SettingFieldChar::writeBinary(WriteBuffer & out) const @@ -304,49 +272,22 @@ void SettingFieldChar::writeBinary(WriteBuffer & out) const void SettingFieldChar::readBinary(ReadBuffer & in) { - String s; - readStringBinary(s, in); - set(s); + String str; + readStringBinary(str, in); + *this = stringToChar(str); } -String SettingFieldURI::toString() const +void SettingFieldURI::writeBinary(WriteBuffer & out) const { - return value.toString(); + writeStringBinary(value.toString(), out); } -Field SettingFieldURI::toField() const +void SettingFieldURI::readBinary(ReadBuffer & in) { - return value.toString(); -} - -void SettingFieldURI::set(const Poco::URI & x) -{ - value = x; - changed = true; -} - -void SettingFieldURI::set(const Field & x) -{ - const String & s = safeGet(x); - set(s); -} - -void SettingFieldURI::set(const String & x) -{ - set(Poco::URI(x)); -} - -void SettingFieldURI::writeBinary(WriteBuffer & buf) const -{ - writeStringBinary(toString(), buf); -} - -void SettingFieldURI::readBinary(ReadBuffer & buf) -{ - String s; - readStringBinary(s, buf); - set(s); + String str; + readStringBinary(str, in); + *this = Poco::URI{str}; } diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index a972180eaee..f6e03ab9dde 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -27,32 +27,26 @@ class WriteBuffer; * and the remote server will use its default value. */ -template +template struct SettingFieldNumber { + using Type = T; + Type value; bool changed = false; - SettingFieldNumber(Type x = 0) : value(x) {} + explicit SettingFieldNumber(Type x = 0) : value(x) {} + explicit SettingFieldNumber(const Field & f); + + SettingFieldNumber & operator=(Type x) { value = x; changed = true; return *this; } + SettingFieldNumber & operator=(const Field & f); operator Type() const { return value; } - SettingFieldNumber & operator=(Type x) { set(x); return *this; } + explicit operator Field() const { return value; } - /// Serialize to a test string. String toString() const; + void parseFromString(const String & str); - /// Converts to a field. - Field toField() const; - - void set(Type x); - - /// Read from SQL literal. - void set(const Field & x); - - /// Read from text string. - void set(const String & x); - - /// Serialize to binary stream. void writeBinary(WriteBuffer & out) const; void readBinary(ReadBuffer & in); }; @@ -69,72 +63,80 @@ using SettingFieldBool = SettingFieldNumber; */ struct SettingFieldMaxThreads { - UInt64 value; bool is_auto; + UInt64 value; bool changed = false; - SettingFieldMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {} + explicit SettingFieldMaxThreads(UInt64 x = 0) : is_auto(!x), value(is_auto ? getAuto() : x) {} + explicit SettingFieldMaxThreads(const Field & f); + + SettingFieldMaxThreads & operator=(UInt64 x) { is_auto = !x; value = is_auto ? getAuto() : x; changed = true; return *this; } + SettingFieldMaxThreads & operator=(const Field & f); operator UInt64() const { return value; } - SettingFieldMaxThreads & operator=(UInt64 x) { set(x); return *this; } + explicit operator Field() const { return value; } + /// Writes "auto()" instead of simple "" if `is_auto==true`. String toString() const; - Field toField() const; - - void set(UInt64 x); - void set(const Field & x); - void set(const String & x); - - void setAuto(); - static UInt64 getAutoValue(); + void parseFromString(const String & str); void writeBinary(WriteBuffer & out) const; void readBinary(ReadBuffer & in); + +private: + static UInt64 getAuto(); }; -enum class SettingFieldTimespanUnit { MILLISECOND, SECOND }; +enum class SettingFieldTimespanUnit { Millisecond, Second }; -template +template struct SettingFieldTimespan { - static constexpr UInt64 microseconds_per_unit = (unit == SettingFieldTimespanUnit::MILLISECOND) ? 1000 : 1000000; - + using Unit = SettingFieldTimespanUnit; + static constexpr Unit unit = unit_; + static constexpr UInt64 microseconds_per_unit = (unit == SettingFieldTimespanUnit::Millisecond) ? 1000 : 1000000; Poco::Timespan value; bool changed = false; - SettingFieldTimespan(UInt64 x = 0) : value(x * microseconds_per_unit) {} + explicit SettingFieldTimespan(const Poco::Timespan & x = {}) : value(x) {} + + template > + explicit SettingFieldTimespan(const std::chrono::duration & x) + : SettingFieldTimespan(Poco::Timespan{static_cast(std::chrono::duration_cast(x).count())}) {} + + explicit SettingFieldTimespan(UInt64 x) : SettingFieldTimespan(Poco::Timespan{static_cast(x * microseconds_per_unit)}) {} + explicit SettingFieldTimespan(const Field & f); + + SettingFieldTimespan & operator =(const Poco::Timespan & x) { value = x; changed = true; return *this; } + + template > + SettingFieldTimespan & operator =(const std::chrono::duration & x) { *this = Poco::Timespan{static_cast(std::chrono::duration_cast(x).count())}; return *this; } + + SettingFieldTimespan & operator =(UInt64 x) { *this = Poco::Timespan{static_cast(x * microseconds_per_unit)}; return *this; } + SettingFieldTimespan & operator =(const Field & f); operator Poco::Timespan() const { return value; } - SettingFieldTimespan & operator=(const Poco::Timespan & x) { set(x); return *this; } template > operator std::chrono::duration() const { return std::chrono::duration_cast>(std::chrono::microseconds(value.totalMicroseconds())); } - template > - SettingFieldTimespan & operator=(const std::chrono::duration & x) { set(x); return *this; } + explicit operator UInt64() const { return value.totalMicroseconds() / microseconds_per_unit; } + explicit operator Field() const { return operator UInt64(); } - Poco::Timespan::TimeDiff totalSeconds() const { return value.totalSeconds(); } + Poco::Timespan::TimeDiff totalMicroseconds() const { return value.totalMicroseconds(); } Poco::Timespan::TimeDiff totalMilliseconds() const { return value.totalMilliseconds(); } + Poco::Timespan::TimeDiff totalSeconds() const { return value.totalSeconds(); } String toString() const; - Field toField() const; - - void set(const Poco::Timespan & x); - - template > - void set(const std::chrono::duration & duration) { set(static_cast(std::chrono::duration_cast(duration).count())); } - - void set(UInt64 x); - void set(const Field & x); - void set(const String & x); + void parseFromString(const String & str); void writeBinary(WriteBuffer & out) const; void readBinary(ReadBuffer & in); }; -using SettingFieldSeconds = SettingFieldTimespan; -using SettingFieldMilliseconds = SettingFieldTimespan; +using SettingFieldSeconds = SettingFieldTimespan; +using SettingFieldMilliseconds = SettingFieldTimespan; struct SettingFieldString @@ -142,16 +144,23 @@ struct SettingFieldString String value; bool changed = false; - SettingFieldString(const String & x = String{}) : value(x) {} + explicit SettingFieldString(const std::string_view & str = {}) : value(str) {} + explicit SettingFieldString(const String & str) : SettingFieldString(std::string_view{str}) {} + explicit SettingFieldString(String && str) : value(std::move(str)) {} + explicit SettingFieldString(const char * str) : SettingFieldString(std::string_view{str}) {} + explicit SettingFieldString(const Field & f) : SettingFieldString(f.safeGet()) {} - operator String() const { return value; } - SettingFieldString & operator=(const String & x) { set(x); return *this; } + SettingFieldString & operator =(const std::string_view & str) { value = str; changed = true; return *this; } + SettingFieldString & operator =(const String & str) { *this = std::string_view{str}; return *this; } + SettingFieldString & operator =(String && str) { value = std::move(str); changed = true; return *this; } + SettingFieldString & operator =(const char * str) { *this = std::string_view{str}; return *this; } + SettingFieldString & operator =(const Field & f) { *this = f.safeGet(); return *this; } + + operator const String &() const { return value; } + explicit operator Field() const { return value; } const String & toString() const { return value; } - Field toField() const; - - void set(const String & x); - void set(const Field & x); + void parseFromString(const String & str) { *this = str; } void writeBinary(WriteBuffer & out) const; void readBinary(ReadBuffer & in); @@ -163,17 +172,18 @@ struct SettingFieldChar public: char value; bool changed = false; - SettingFieldChar(char x = '\0') : value(x) {} + + explicit SettingFieldChar(char c = '\0') : value(c) {} + explicit SettingFieldChar(const Field & f); + + SettingFieldChar & operator =(char c) { value = c; changed = true; return *this; } + SettingFieldChar & operator =(const Field & f); operator char() const { return value; } - SettingFieldChar & operator=(char x) { set(x); return *this; } + explicit operator Field() const { return toString(); } - String toString() const; - Field toField() const; - - void set(char x); - void set(const String & x); - void set(const Field & x); + String toString() const { return String(&value, 1); } + void parseFromString(const String & str); void writeBinary(WriteBuffer & out) const; void readBinary(ReadBuffer & in); @@ -185,17 +195,22 @@ struct SettingFieldURI Poco::URI value; bool changed = false; - SettingFieldURI(const Poco::URI & x = Poco::URI{}) : value(x) {} + explicit SettingFieldURI(const Poco::URI & uri = {}) : value(uri) {} + explicit SettingFieldURI(const String & str) : SettingFieldURI(Poco::URI{str}) {} + explicit SettingFieldURI(const char * str) : SettingFieldURI(Poco::URI{str}) {} + explicit SettingFieldURI(const Field & f) : SettingFieldURI(f.safeGet()) {} - operator Poco::URI() const { return value; } - SettingFieldURI & operator=(const Poco::URI & x) { set(x); return *this; } + SettingFieldURI & operator =(const Poco::URI & x) { value = x; changed = true; return *this; } + SettingFieldURI & operator =(const String & str) { *this = Poco::URI{str}; return *this; } + SettingFieldURI & operator =(const char * str) { *this = Poco::URI{str}; return *this; } + SettingFieldURI & operator =(const Field & f) { *this = f.safeGet(); return *this; } - String toString() const; - Field toField() const; + operator const Poco::URI &() const { return value; } + explicit operator String() const { return toString(); } + explicit operator Field() const { return toString(); } - void set(const Poco::URI & x); - void set(const Field & x); - void set(const String & x); + String toString() const { return value.toString(); } + void parseFromString(const String & str) { *this = str; } void writeBinary(WriteBuffer & out) const; void readBinary(ReadBuffer & in); @@ -213,23 +228,25 @@ struct SettingFieldURI * IMPLEMENT_SETTING_ENUM(SettingFieldGender, ErrorCodes::BAD_ARGUMENTS, * {{"Male", Gender::Male}, {"Female", Gender::Female}}) */ -template +template struct SettingFieldEnum { + using EnumType = EnumT; + EnumType value; bool changed = false; - SettingFieldEnum(EnumType x) : value(x) {} + explicit SettingFieldEnum(EnumType x = EnumType{0}) : value(x) {} + explicit SettingFieldEnum(const Field & f) : SettingFieldEnum(Traits::fromString(f.safeGet())) {} + + SettingFieldEnum & operator =(EnumType x) { value = x; changed = true; return *this; } + SettingFieldEnum & operator =(const Field & f) { *this = Traits::fromString(f.safeGet()); return *this; } operator EnumType() const { return value; } - SettingFieldEnum & operator=(EnumType x) { set(x); return *this; } + explicit operator Field() const { return toString(); } - const String & toString() const { return NameValueConverter::toString(value); } - Field toField() const { return toString(); } - - void set(EnumType x) { value = x; changed = true; } - void set(const Field & x) { set(safeGet(x)); } - void set(const String & x) { set(NameValueConverter::fromString(x)); } + String toString() const { return Traits::toString(value); } + void parseFromString(const String & str) { *this = Traits::fromString(str); } void writeBinary(WriteBuffer & out) const; void readBinary(ReadBuffer & in); @@ -241,16 +258,16 @@ struct SettingFieldEnumHelpers static String readBinary(ReadBuffer & in); }; -template -inline void SettingFieldEnum::writeBinary(WriteBuffer & out) const +template +void SettingFieldEnum::writeBinary(WriteBuffer & out) const { SettingFieldEnumHelpers::writeBinary(toString(), out); } -template -inline void SettingFieldEnum::readBinary(ReadBuffer & in) +template +void SettingFieldEnum::readBinary(ReadBuffer & in) { - set(SettingFieldEnumHelpers::readBinary(in)); + *this = Traits::fromString(SettingFieldEnumHelpers::readBinary(in)); } #define DECLARE_SETTING_ENUM(ENUM_TYPE) \ @@ -260,17 +277,17 @@ inline void SettingFieldEnum::readBinary(ReadBuffe IMPLEMENT_SETTING_ENUM_WITH_RENAME(ENUM_TYPE, ERROR_CODE_FOR_UNEXPECTED_NAME, __VA_ARGS__) #define DECLARE_SETTING_ENUM_WITH_RENAME(NEW_NAME, ENUM_TYPE) \ - struct SettingField##NEW_NAME##NameValueConverter \ + struct SettingField##NEW_NAME##Traits \ { \ using EnumType = ENUM_TYPE; \ static const String & toString(EnumType value); \ static EnumType fromString(const std::string_view & str); \ }; \ \ - using SettingField##NEW_NAME = SettingFieldEnum; + using SettingField##NEW_NAME = SettingFieldEnum; #define IMPLEMENT_SETTING_ENUM_WITH_RENAME(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, ...) \ - const String & SettingField##NEW_NAME##NameValueConverter::toString(typename SettingField##NEW_NAME##NameValueConverter::EnumType value) \ + const String & SettingField##NEW_NAME##Traits::toString(typename SettingField##NEW_NAME::EnumType value) \ { \ static const std::unordered_map map = [] { \ std::unordered_map res; \ @@ -287,7 +304,7 @@ inline void SettingFieldEnum::readBinary(ReadBuffe ERROR_CODE_FOR_UNEXPECTED_NAME); \ } \ \ - typename SettingField##NEW_NAME##NameValueConverter::EnumType SettingField##NEW_NAME##NameValueConverter::fromString(const std::string_view & str) \ + typename SettingField##NEW_NAME::EnumType SettingField##NEW_NAME##Traits::fromString(const std::string_view & str) \ { \ static const std::unordered_map map = [] { \ std::unordered_map res; \ diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index f299ab2286f..d11bd1d8706 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -638,7 +638,7 @@ template inline T parse(const char * data, size_t size); template -inline T parseFromString(const String & str) +inline T parseFromString(const std::string_view & str) { return parse(str.data(), str.size()); } @@ -1096,7 +1096,7 @@ inline T parseWithSizeSuffix(const char * data, size_t size) } template -inline T parseWithSizeSuffix(const String & s) +inline T parseWithSizeSuffix(const std::string_view & s) { return parseWithSizeSuffix(s.data(), s.size()); } diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index c0f7ef8e44d..9d7bc273cd2 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -173,7 +173,7 @@ SettingsChanges StorageKafka::createSettingsAdjustments() { if (it.isChanged() && it.getName().toString().rfind("kafka_",0) == std::string::npos) { - result.emplace_back(it.getName().toString(), it.getValueAsString()); + result.emplace_back(it.getName().toString(), it.getValue()); } } return result; @@ -632,8 +632,8 @@ void registerStorageKafka(StorageFactory & factory) engine_args[(ARG_NUM)-1], \ args.local_context); \ } \ - kafka_settings->PAR_NAME.set( \ - engine_args[(ARG_NUM)-1]->as().value);\ + kafka_settings->PAR_NAME = \ + engine_args[(ARG_NUM)-1]->as().value; \ } \ } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index af1a8bf10d7..74097278026 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -124,17 +124,17 @@ void registerStorageJoin(StorageFactory & factory) for (const auto & setting : args.storage_def->settings->changes) { if (setting.name == "join_use_nulls") - join_use_nulls.set(setting.value); + join_use_nulls = setting.value; else if (setting.name == "max_rows_in_join") - max_rows_in_join.set(setting.value); + max_rows_in_join = setting.value; else if (setting.name == "max_bytes_in_join") - max_bytes_in_join.set(setting.value); + max_bytes_in_join = setting.value; else if (setting.name == "join_overflow_mode") - join_overflow_mode.set(setting.value); + join_overflow_mode = setting.value; else if (setting.name == "join_any_take_last_row") - join_any_take_last_row.set(setting.value); + join_any_take_last_row = setting.value; else if (setting.name == "any_join_distinct_right_table_keys") - old_any_join.set(setting.value); + old_any_join = setting.value; else throw Exception( "Unknown setting " + setting.name + " for storage " + args.engine_name, From e40a8ac176f89ec4fdc9fee62477c143070bdd0b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 19 Jul 2020 17:59:07 +0300 Subject: [PATCH 10/20] Use names of the settings instead of their indices in SettingsConstraints and settings profiles. --- src/Access/SettingsConstraints.cpp | 367 +++++++----------- src/Access/SettingsConstraints.h | 79 ++-- src/Access/SettingsProfileElement.cpp | 32 +- src/Access/SettingsProfileElement.h | 4 +- src/Access/UsersConfigAccessStorage.cpp | 24 +- src/Core/SettingsCollection.h | 4 + src/Interpreters/Context.cpp | 6 +- src/Interpreters/Context.h | 2 +- src/Storages/System/StorageSystemSettings.cpp | 4 +- .../StorageSystemSettingsProfileElements.cpp | 12 +- 10 files changed, 214 insertions(+), 320 deletions(-) diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 2c16f23583f..4f4ff02a5e1 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -1,19 +1,18 @@ #include #include #include -#include +#include #include #include +#include namespace DB { namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; extern const int READONLY; extern const int QUERY_IS_PROHIBITED; - extern const int NO_ELEMENTS_IN_CONFIG; extern const int SETTING_CONSTRAINT_VIOLATION; } @@ -27,28 +26,18 @@ SettingsConstraints::~SettingsConstraints() = default; void SettingsConstraints::clear() { - constraints_by_index.clear(); + constraints.clear(); } -void SettingsConstraints::setMinValue(const StringRef & setting_name, const Field & min_value) +void SettingsConstraints::setMinValue(const std::string_view & setting_name, const Field & min_value) { - setMinValue(Settings::findIndexStrict(setting_name), min_value); + getConstraintRef(setting_name).min_value = Settings::castValue(setting_name, min_value); } -void SettingsConstraints::setMinValue(size_t setting_index, const Field & min_value) +Field SettingsConstraints::getMinValue(const std::string_view & setting_name) const { - getConstraintRef(setting_index).min_value = Settings::castValue(setting_index, min_value); -} - -Field SettingsConstraints::getMinValue(const StringRef & setting_name) const -{ - return getMinValue(Settings::findIndexStrict(setting_name)); -} - -Field SettingsConstraints::getMinValue(size_t setting_index) const -{ - const auto * ptr = tryGetConstraint(setting_index); + const auto * ptr = tryGetConstraint(setting_name); if (ptr) return ptr->min_value; else @@ -56,24 +45,14 @@ Field SettingsConstraints::getMinValue(size_t setting_index) const } -void SettingsConstraints::setMaxValue(const StringRef & name, const Field & max_value) +void SettingsConstraints::setMaxValue(const std::string_view & setting_name, const Field & max_value) { - setMaxValue(Settings::findIndexStrict(name), max_value); + getConstraintRef(setting_name).max_value = Settings::castValue(setting_name, max_value); } -void SettingsConstraints::setMaxValue(size_t setting_index, const Field & max_value) +Field SettingsConstraints::getMaxValue(const std::string_view & setting_name) const { - getConstraintRef(setting_index).max_value = Settings::castValue(setting_index, max_value); -} - -Field SettingsConstraints::getMaxValue(const StringRef & setting_name) const -{ - return getMaxValue(Settings::findIndexStrict(setting_name)); -} - -Field SettingsConstraints::getMaxValue(size_t setting_index) const -{ - const auto * ptr = tryGetConstraint(setting_index); + const auto * ptr = tryGetConstraint(setting_name); if (ptr) return ptr->max_value; else @@ -81,51 +60,32 @@ Field SettingsConstraints::getMaxValue(size_t setting_index) const } -void SettingsConstraints::setReadOnly(const StringRef & setting_name, bool read_only) +void SettingsConstraints::setReadOnly(const std::string_view & setting_name, bool read_only) { - setReadOnly(Settings::findIndexStrict(setting_name), read_only); + getConstraintRef(setting_name).read_only = read_only; } -void SettingsConstraints::setReadOnly(size_t setting_index, bool read_only) +bool SettingsConstraints::isReadOnly(const std::string_view & setting_name) const { - getConstraintRef(setting_index).read_only = read_only; -} - -bool SettingsConstraints::isReadOnly(const StringRef & setting_name) const -{ - return isReadOnly(Settings::findIndexStrict(setting_name)); -} - -bool SettingsConstraints::isReadOnly(size_t setting_index) const -{ - const auto * ptr = tryGetConstraint(setting_index); + const auto * ptr = tryGetConstraint(setting_name); if (ptr) return ptr->read_only; else return false; } -void SettingsConstraints::set(const StringRef & setting_name, const Field & min_value, const Field & max_value, bool read_only) -{ - set(Settings::findIndexStrict(setting_name), min_value, max_value, read_only); -} -void SettingsConstraints::set(size_t setting_index, const Field & min_value, const Field & max_value, bool read_only) +void SettingsConstraints::set(const std::string_view & setting_name, const Field & min_value, const Field & max_value, bool read_only) { - auto & ref = getConstraintRef(setting_index); - ref.min_value = min_value; - ref.max_value = max_value; + auto & ref = getConstraintRef(setting_name); + ref.min_value = Settings::castValue(setting_name, min_value); + ref.max_value = Settings::castValue(setting_name, max_value); ref.read_only = read_only; } -void SettingsConstraints::get(const StringRef & setting_name, Field & min_value, Field & max_value, bool & read_only) const +void SettingsConstraints::get(const std::string_view & setting_name, Field & min_value, Field & max_value, bool & read_only) const { - get(Settings::findIndexStrict(setting_name), min_value, max_value, read_only); -} - -void SettingsConstraints::get(size_t setting_index, Field & min_value, Field & max_value, bool & read_only) const -{ - const auto * ptr = tryGetConstraint(setting_index); + const auto * ptr = tryGetConstraint(setting_name); if (ptr) { min_value = ptr->min_value; @@ -142,9 +102,9 @@ void SettingsConstraints::get(size_t setting_index, Field & min_value, Field & m void SettingsConstraints::merge(const SettingsConstraints & other) { - for (const auto & [setting_index, other_constraint] : other.constraints_by_index) + for (const auto & [other_name, other_constraint] : other.constraints) { - auto & constraint = constraints_by_index[setting_index]; + auto & constraint = getConstraintRef(other_name); if (!other_constraint.min_value.isNull()) constraint.min_value = other_constraint.min_value; if (!other_constraint.max_value.isNull()) @@ -155,95 +115,57 @@ void SettingsConstraints::merge(const SettingsConstraints & other) } -SettingsConstraints::Infos SettingsConstraints::getInfo() const -{ - Infos result; - result.reserve(constraints_by_index.size()); - for (const auto & [setting_index, constraint] : constraints_by_index) - { - result.emplace_back(); - Info & info = result.back(); - info.name = Settings::getName(setting_index); - info.min = constraint.min_value; - info.max = constraint.max_value; - info.read_only = constraint.read_only; - } - return result; -} - - void SettingsConstraints::check(const Settings & current_settings, const SettingChange & change) const { - const String & name = change.name; - size_t setting_index = Settings::findIndex(name); - if (setting_index == Settings::npos) - return; - - Field new_value = Settings::castValue(setting_index, change.value); - Field current_value = current_settings.get(setting_index); - - /// Setting isn't checked if value wasn't changed. - if (current_value == new_value) - return; - - if (!current_settings.allow_ddl && name == "allow_ddl") - throw Exception("Cannot modify 'allow_ddl' setting when DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED); - - /** The `readonly` value is understood as follows: - * 0 - everything allowed. - * 1 - only read queries can be made; you can not change the settings. - * 2 - You can only do read queries and you can change the settings, except for the `readonly` setting. - */ - if (current_settings.readonly == 1) - throw Exception("Cannot modify '" + name + "' setting in readonly mode", ErrorCodes::READONLY); - - if (current_settings.readonly > 1 && name == "readonly") - throw Exception("Cannot modify 'readonly' setting in readonly mode", ErrorCodes::READONLY); - - const Constraint * constraint = tryGetConstraint(setting_index); - if (constraint) - { - if (constraint->read_only) - throw Exception("Setting " + name + " should not be changed", ErrorCodes::SETTING_CONSTRAINT_VIOLATION); - - if (!constraint->min_value.isNull() && (new_value < constraint->min_value)) - throw Exception( - "Setting " + name + " shouldn't be less than " + applyVisitor(FieldVisitorToString(), constraint->min_value), - ErrorCodes::SETTING_CONSTRAINT_VIOLATION); - - if (!constraint->max_value.isNull() && (new_value > constraint->max_value)) - throw Exception( - "Setting " + name + " shouldn't be greater than " + applyVisitor(FieldVisitorToString(), constraint->max_value), - ErrorCodes::SETTING_CONSTRAINT_VIOLATION); - } + checkImpl(current_settings, const_cast(change), THROW_ON_VIOLATION); } - void SettingsConstraints::check(const Settings & current_settings, const SettingsChanges & changes) const { for (const auto & change : changes) check(current_settings, change); } - -void SettingsConstraints::clamp(const Settings & current_settings, SettingChange & change) const +void SettingsConstraints::check(const Settings & current_settings, SettingsChanges & changes) const { - const String & name = change.name; - size_t setting_index = Settings::findIndex(name); - if (setting_index == Settings::npos) - return; + boost::range::remove_erase_if( + changes, + [&](SettingChange & change) -> bool + { + return !checkImpl(current_settings, const_cast(change), THROW_ON_VIOLATION); + }); +} - Field new_value = Settings::castValue(setting_index, change.value); - Field current_value = current_settings.get(setting_index); +void SettingsConstraints::clamp(const Settings & current_settings, SettingsChanges & changes) const +{ + boost::range::remove_erase_if( + changes, + [&](SettingChange & change) -> bool + { + return !checkImpl(current_settings, change, CLAMP_ON_VIOLATION); + }); +} - /// Setting isn't checked if value wasn't changed. - if (current_value == new_value) - return; - if (!current_settings.allow_ddl && name == "allow_ddl") +bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const +{ + const String & setting_name = change.name; + + /// Setting isn't checked if value has not changed. + Field current_value = current_settings.get(setting_name); + if (change.value == current_value) + return false; + + Field new_value = Settings::castValue(setting_name, change.value); + if (new_value == current_value) + return false; + + if (!current_settings.allow_ddl && setting_name == "allow_ddl") { - change.value = current_value; - return; + if (reaction == THROW_ON_VIOLATION) + throw Exception("Cannot modify 'allow_ddl' setting when DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED); + else + return false; } /** The `readonly` value is understood as follows: @@ -253,125 +175,122 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingChange */ if (current_settings.readonly == 1) { - change.value = current_value; - return; + if (reaction == THROW_ON_VIOLATION) + throw Exception("Cannot modify '" + setting_name + "' setting in readonly mode", ErrorCodes::READONLY); + else + return false; } - if (current_settings.readonly > 1 && name == "readonly") + if (current_settings.readonly > 1 && setting_name == "readonly") { - change.value = current_value; - return; + if (reaction == THROW_ON_VIOLATION) + throw Exception("Cannot modify 'readonly' setting in readonly mode", ErrorCodes::READONLY); + else + return false; } - const Constraint * constraint = tryGetConstraint(setting_index); + const Constraint * constraint = tryGetConstraint(setting_name); if (constraint) { if (constraint->read_only) { - change.value = current_value; - return; - } - - if (!constraint->min_value.isNull() && (new_value < constraint->min_value)) - { - if (!constraint->max_value.isNull() && (constraint->min_value > constraint->max_value)) - change.value = current_value; + if (reaction == THROW_ON_VIOLATION) + throw Exception("Setting " + setting_name + " should not be changed", ErrorCodes::SETTING_CONSTRAINT_VIOLATION); else - change.value = constraint->min_value; - return; + return false; } - if (!constraint->max_value.isNull() && (new_value > constraint->max_value)) + const Field & min_value = constraint->min_value; + const Field & max_value = constraint->max_value; + + bool cannot_compare = false; + auto less = [&](const Field & left, const Field & right) { - change.value = constraint->max_value; - return; + cannot_compare = false; + if (reaction == THROW_ON_VIOLATION) + return applyVisitor(FieldVisitorAccurateLess{}, left, right); + else + { + try + { + return applyVisitor(FieldVisitorAccurateLess{}, left, right); + } + catch (...) + { + cannot_compare = true; + return false; + } + } + }; + + if (!min_value.isNull() && !max_value.isNull() && (less(max_value, min_value) || cannot_compare)) + { + if (reaction == THROW_ON_VIOLATION) + throw Exception("Setting " + setting_name + " should not be changed", ErrorCodes::SETTING_CONSTRAINT_VIOLATION); + else + return false; + } + + if (!min_value.isNull() && (less(new_value, min_value) || cannot_compare)) + { + if (reaction == THROW_ON_VIOLATION) + { + throw Exception( + "Setting " + setting_name + " shouldn't be less than " + applyVisitor(FieldVisitorToString(), constraint->min_value), + ErrorCodes::SETTING_CONSTRAINT_VIOLATION); + } + else + change.value = min_value; + } + + if (!max_value.isNull() && (less(max_value, new_value) || cannot_compare)) + { + if (reaction == THROW_ON_VIOLATION) + { + throw Exception( + "Setting " + setting_name + " shouldn't be greater than " + applyVisitor(FieldVisitorToString(), constraint->max_value), + ErrorCodes::SETTING_CONSTRAINT_VIOLATION); + } + else + change.value = max_value; } } + + return true; } -void SettingsConstraints::clamp(const Settings & current_settings, SettingsChanges & changes) const +SettingsConstraints::Constraint & SettingsConstraints::getConstraintRef(const std::string_view & setting_name) { - for (auto & change : changes) - clamp(current_settings, change); -} - - -SettingsConstraints::Constraint & SettingsConstraints::getConstraintRef(size_t index) -{ - auto it = constraints_by_index.find(index); - if (it == constraints_by_index.end()) - it = constraints_by_index.emplace(index, Constraint{}).first; + auto it = constraints.find(setting_name); + if (it == constraints.end()) + { + auto setting_name_ptr = std::make_shared(setting_name); + Constraint new_constraint; + new_constraint.setting_name = setting_name_ptr; + it = constraints.emplace(*setting_name_ptr, std::move(new_constraint)).first; + } return it->second; } -const SettingsConstraints::Constraint * SettingsConstraints::tryGetConstraint(size_t index) const +const SettingsConstraints::Constraint * SettingsConstraints::tryGetConstraint(const std::string_view & setting_name) const { - auto it = constraints_by_index.find(index); - if (it == constraints_by_index.end()) + auto it = constraints.find(setting_name); + if (it == constraints.end()) return nullptr; return &it->second; } -void SettingsConstraints::setProfile(const String & profile_name, const Poco::Util::AbstractConfiguration & config) +bool SettingsConstraints::Constraint::operator==(const Constraint & other) const { - String elem = "profiles." + profile_name; - - Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys(elem, config_keys); - - for (const std::string & key : config_keys) - { - if (key == "profile" || key.starts_with("profile[")) /// Inheritance of profiles from the current one. - setProfile(config.getString(elem + "." + key), config); - else - continue; - } - - String path_to_constraints = "profiles." + profile_name + ".constraints"; - if (config.has(path_to_constraints)) - loadFromConfig(path_to_constraints, config); + return (read_only == other.read_only) && (min_value == other.min_value) && (max_value == other.max_value) + && (*setting_name == *other.setting_name); } -void SettingsConstraints::loadFromConfig(const String & path_to_constraints, const Poco::Util::AbstractConfiguration & config) +bool operator ==(const SettingsConstraints & left, const SettingsConstraints & right) { - if (!config.has(path_to_constraints)) - throw Exception("There is no path '" + path_to_constraints + "' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); - - Poco::Util::AbstractConfiguration::Keys names; - config.keys(path_to_constraints, names); - - for (const String & name : names) - { - String path_to_name = path_to_constraints + "." + name; - Poco::Util::AbstractConfiguration::Keys constraint_types; - config.keys(path_to_name, constraint_types); - for (const String & constraint_type : constraint_types) - { - auto get_constraint_value = [&]{ return config.getString(path_to_name + "." + constraint_type); }; - if (constraint_type == "min") - setMinValue(name, get_constraint_value()); - else if (constraint_type == "max") - setMaxValue(name, get_constraint_value()); - else if (constraint_type == "readonly") - setReadOnly(name, true); - else - throw Exception("Setting " + constraint_type + " value for " + name + " isn't supported", ErrorCodes::NOT_IMPLEMENTED); - } - } -} - - -bool SettingsConstraints::Constraint::operator==(const Constraint & rhs) const -{ - return (read_only == rhs.read_only) && (min_value == rhs.min_value) && (max_value == rhs.max_value); -} - - -bool operator ==(const SettingsConstraints & lhs, const SettingsConstraints & rhs) -{ - return lhs.constraints_by_index == rhs.constraints_by_index; + return left.constraints == right.constraints; } } diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index f8267b2b47a..39fe3fb810b 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -1,18 +1,13 @@ #pragma once -#include -#include +#include #include -namespace Poco -{ -namespace Util +namespace Poco::Util { class AbstractConfiguration; } -} - namespace DB { @@ -20,6 +15,7 @@ struct Settings; struct SettingChange; class SettingsChanges; + /** Checks if specified changes of settings are allowed or not. * If the changes are not allowed (i.e. violates some constraints) this class throws an exception. * The constraints are set by editing the `users.xml` file. @@ -62,75 +58,56 @@ public: ~SettingsConstraints(); void clear(); - bool empty() const { return constraints_by_index.empty(); } + bool empty() const { return constraints.empty(); } - void setMinValue(const StringRef & setting_name, const Field & min_value); - void setMinValue(size_t setting_index, const Field & min_value); - Field getMinValue(const StringRef & setting_name) const; - Field getMinValue(size_t setting_index) const; + void setMinValue(const std::string_view & setting_name, const Field & min_value); + Field getMinValue(const std::string_view & setting_name) const; - void setMaxValue(const StringRef & setting_name, const Field & max_value); - void setMaxValue(size_t setting_index, const Field & max_value); - Field getMaxValue(const StringRef & setting_name) const; - Field getMaxValue(size_t setting_index) const; + void setMaxValue(const std::string_view & setting_name, const Field & max_value); + Field getMaxValue(const std::string_view & setting_name) const; - void setReadOnly(const StringRef & setting_name, bool read_only); - void setReadOnly(size_t setting_index, bool read_only); - bool isReadOnly(const StringRef & setting_name) const; - bool isReadOnly(size_t setting_index) const; + void setReadOnly(const std::string_view & setting_name, bool read_only); + bool isReadOnly(const std::string_view & setting_name) const; - void set(const StringRef & setting_name, const Field & min_value, const Field & max_value, bool read_only); - void set(size_t setting_index, const Field & min_value, const Field & max_value, bool read_only); - void get(const StringRef & setting_name, Field & min_value, Field & max_value, bool & read_only) const; - void get(size_t setting_index, Field & min_value, Field & max_value, bool & read_only) const; + void set(const std::string_view & setting_name, const Field & min_value, const Field & max_value, bool read_only); + void get(const std::string_view & setting_name, Field & min_value, Field & max_value, bool & read_only) const; void merge(const SettingsConstraints & other); - struct Info - { - StringRef name; - Field min; - Field max; - bool read_only = false; - }; - using Infos = std::vector; - - Infos getInfo() const; - /// Checks whether `change` violates these constraints and throws an exception if so. void check(const Settings & current_settings, const SettingChange & change) const; void check(const Settings & current_settings, const SettingsChanges & changes) const; + void check(const Settings & current_settings, SettingsChanges & changes) const; /// Checks whether `change` violates these and clamps the `change` if so. - void clamp(const Settings & current_settings, SettingChange & change) const; void clamp(const Settings & current_settings, SettingsChanges & changes) const; - /** Set multiple settings from "profile" (in server configuration file (users.xml), profiles contain groups of multiple settings). - * The profile can also be set using the `set` functions, like the profile setting. - */ - void setProfile(const String & profile_name, const Poco::Util::AbstractConfiguration & config); - - /// Loads the constraints from configuration file, at "path" prefix in configuration. - void loadFromConfig(const String & path, const Poco::Util::AbstractConfiguration & config); - - friend bool operator ==(const SettingsConstraints & lhs, const SettingsConstraints & rhs); - friend bool operator !=(const SettingsConstraints & lhs, const SettingsConstraints & rhs) { return !(lhs == rhs); } + friend bool operator ==(const SettingsConstraints & left, const SettingsConstraints & right); + friend bool operator !=(const SettingsConstraints & left, const SettingsConstraints & right) { return !(left == right); } private: struct Constraint { + std::shared_ptr setting_name; bool read_only = false; Field min_value; Field max_value; - bool operator ==(const Constraint & rhs) const; - bool operator !=(const Constraint & rhs) const { return !(*this == rhs); } + bool operator ==(const Constraint & other) const; + bool operator !=(const Constraint & other) const { return !(*this == other); } }; - Constraint & getConstraintRef(size_t index); - const Constraint * tryGetConstraint(size_t) const; + enum ReactionOnViolation + { + THROW_ON_VIOLATION, + CLAMP_ON_VIOLATION, + }; + bool checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const; - std::unordered_map constraints_by_index; + Constraint & getConstraintRef(const std::string_view & setting_name); + const Constraint * tryGetConstraint(const std::string_view & setting_name) const; + + std::unordered_map constraints; }; } diff --git a/src/Access/SettingsProfileElement.cpp b/src/Access/SettingsProfileElement.cpp index cb9b26cce53..3d7ebb76ca0 100644 --- a/src/Access/SettingsProfileElement.cpp +++ b/src/Access/SettingsProfileElement.cpp @@ -36,18 +36,18 @@ void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const A if (!ast.setting_name.empty()) { - setting_index = Settings::findIndexStrict(ast.setting_name); + setting_name = ast.setting_name; value = ast.value; min_value = ast.min_value; max_value = ast.max_value; readonly = ast.readonly; if (!value.isNull()) - value = Settings::castValue(setting_index, value); + value = Settings::castValue(setting_name, value); if (!min_value.isNull()) - min_value = Settings::castValue(setting_index, min_value); + min_value = Settings::castValue(setting_name, min_value); if (!max_value.isNull()) - max_value = Settings::castValue(setting_index, max_value); + max_value = Settings::castValue(setting_name, max_value); } } @@ -60,9 +60,7 @@ std::shared_ptr SettingsProfileElement::toAST() const if (parent_profile) ast->parent_profile = ::DB::toString(*parent_profile); - if (setting_index != static_cast(-1)) - ast->setting_name = Settings::getName(setting_index).toString(); - + ast->setting_name = setting_name; ast->value = value; ast->min_value = min_value; ast->max_value = max_value; @@ -83,9 +81,7 @@ std::shared_ptr SettingsProfileElement::toASTWithName ast->parent_profile = *parent_profile_name; } - if (setting_index != static_cast(-1)) - ast->setting_name = Settings::getName(setting_index).toString(); - + ast->setting_name = setting_name; ast->value = value; ast->min_value = min_value; ast->max_value = max_value; @@ -136,8 +132,8 @@ Settings SettingsProfileElements::toSettings() const Settings res; for (const auto & elem : *this) { - if ((elem.setting_index != static_cast(-1)) && !elem.value.isNull()) - res.set(elem.setting_index, elem.value); + if (!elem.setting_name.empty() && !elem.value.isNull()) + res.set(elem.setting_name, elem.value); } return res; } @@ -147,8 +143,8 @@ SettingsChanges SettingsProfileElements::toSettingsChanges() const SettingsChanges res; for (const auto & elem : *this) { - if ((elem.setting_index != static_cast(-1)) && !elem.value.isNull()) - res.push_back({Settings::getName(elem.setting_index).toString(), elem.value}); + if (!elem.setting_name.empty() && !elem.value.isNull()) + res.push_back({elem.setting_name, elem.value}); } return res; } @@ -158,14 +154,14 @@ SettingsConstraints SettingsProfileElements::toSettingsConstraints() const SettingsConstraints res; for (const auto & elem : *this) { - if (elem.setting_index != static_cast(-1)) + if (!elem.setting_name.empty()) { if (!elem.min_value.isNull()) - res.setMinValue(elem.setting_index, elem.min_value); + res.setMinValue(elem.setting_name, elem.min_value); if (!elem.max_value.isNull()) - res.setMaxValue(elem.setting_index, elem.max_value); + res.setMaxValue(elem.setting_name, elem.max_value); if (elem.readonly) - res.setReadOnly(elem.setting_index, *elem.readonly); + res.setReadOnly(elem.setting_name, *elem.readonly); } } return res; diff --git a/src/Access/SettingsProfileElement.h b/src/Access/SettingsProfileElement.h index 88c8178b426..0de71426d0f 100644 --- a/src/Access/SettingsProfileElement.h +++ b/src/Access/SettingsProfileElement.h @@ -19,13 +19,13 @@ class AccessControlManager; struct SettingsProfileElement { std::optional parent_profile; - size_t setting_index = static_cast(-1); + String setting_name; Field value; Field min_value; Field max_value; std::optional readonly; - auto toTuple() const { return std::tie(parent_profile, setting_index, value, min_value, max_value, readonly); } + auto toTuple() const { return std::tie(parent_profile, setting_name, value, min_value, max_value, readonly); } friend bool operator==(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return lhs.toTuple() == rhs.toTuple(); } friend bool operator!=(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return !(lhs == rhs); } friend bool operator <(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return lhs.toTuple() < rhs.toTuple(); } diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index a4a0acfc664..b378173910c 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -4,9 +4,9 @@ #include #include #include -#include #include #include +#include #include #include #include @@ -362,26 +362,25 @@ namespace const String & path_to_constraints) { SettingsProfileElements profile_elements; - Poco::Util::AbstractConfiguration::Keys names; - config.keys(path_to_constraints, names); - for (const String & name : names) + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(path_to_constraints, keys); + for (const String & setting_name : keys) { SettingsProfileElement profile_element; - size_t setting_index = Settings::findIndexStrict(name); - profile_element.setting_index = setting_index; + profile_element.setting_name = setting_name; Poco::Util::AbstractConfiguration::Keys constraint_types; - String path_to_name = path_to_constraints + "." + name; + String path_to_name = path_to_constraints + "." + setting_name; config.keys(path_to_name, constraint_types); for (const String & constraint_type : constraint_types) { if (constraint_type == "min") - profile_element.min_value = Settings::stringToValue(setting_index, config.getString(path_to_name + "." + constraint_type)); + profile_element.min_value = Settings::stringToValue(setting_name, config.getString(path_to_name + "." + constraint_type)); else if (constraint_type == "max") - profile_element.max_value = Settings::stringToValue(setting_index, config.getString(path_to_name + "." + constraint_type)); + profile_element.max_value = Settings::stringToValue(setting_name, config.getString(path_to_name + "." + constraint_type)); else if (constraint_type == "readonly") profile_element.readonly = true; else - throw Exception("Setting " + constraint_type + " value for " + name + " isn't supported", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("Setting " + constraint_type + " value for " + setting_name + " isn't supported", ErrorCodes::NOT_IMPLEMENTED); } profile_elements.push_back(std::move(profile_element)); } @@ -416,10 +415,11 @@ namespace continue; } + const auto & setting_name = key; SettingsProfileElement profile_element; size_t setting_index = Settings::findIndexStrict(key); - profile_element.setting_index = setting_index; - profile_element.value = Settings::stringToValue(setting_index, config.getString(profile_config + "." + key)); + profile_element.setting_name = setting_name; + profile_element.value = Settings::stringToValue(setting_name, config.getString(profile_config + "." + key)); profile->elements.emplace_back(std::move(profile_element)); } diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h index 9b19e7ae04e..276bde8c695 100644 --- a/src/Core/SettingsCollection.h +++ b/src/Core/SettingsCollection.h @@ -184,12 +184,16 @@ public: /// E.g. for SettingInt64 it casts Field to Field::Types::Int64. static Field castValue(size_t index, const Field & value); static Field castValue(const StringRef & name, const Field & value); + static Field castValue(const std::string_view & name, const Field & value) { return castValue(StringRef{name}, value); } + static Field castValue(const String & name, const Field & value) { return castValue(StringRef{name}, value); } /// Casts a value to a string according to a specified setting without actual changing this settings. static Field stringToValue(size_t index, const String & str) { return members()[index].string_to_value(str); } static Field stringToValue(const StringRef & name, const String & str) { return members().findStrict(name).string_to_value(str); } static String valueToString(size_t index, const Field & value) { return members()[index].value_to_string(value); } static String valueToString(const StringRef & name, const Field & value) { return members().findStrict(name).value_to_string(value); } + static String valueToString(const std::string_view & name, const Field & value) { return valueToString(name, value); } + static String valueToString(const String & name, const Field & value) { return valueToString(name, value); } iterator begin() { return iterator(castToDerived(), members().data()); } const_iterator begin() const { return const_iterator(castToDerived(), members().data()); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a45bac35cd8..9c057d1eb01 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1025,11 +1025,10 @@ void Context::checkSettingsConstraints(const SettingsChanges & changes) const settings_constraints->check(settings, changes); } - -void Context::clampToSettingsConstraints(SettingChange & change) const +void Context::checkSettingsConstraints(SettingsChanges & changes) const { if (auto settings_constraints = getSettingsConstraints()) - settings_constraints->clamp(settings, change); + settings_constraints->check(settings, changes); } void Context::clampToSettingsConstraints(SettingsChanges & changes) const @@ -1038,7 +1037,6 @@ void Context::clampToSettingsConstraints(SettingsChanges & changes) const settings_constraints->clamp(settings, changes); } - std::shared_ptr Context::getSettingsConstraints() const { return getAccess()->getSettingsConstraints(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 23a4be82cb8..dfe2d00873f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -377,7 +377,7 @@ public: /// Checks the constraints. void checkSettingsConstraints(const SettingChange & change) const; void checkSettingsConstraints(const SettingsChanges & changes) const; - void clampToSettingsConstraints(SettingChange & change) const; + void checkSettingsConstraints(SettingsChanges & changes) const; void clampToSettingsConstraints(SettingsChanges & changes) const; /// Returns the current constraints (can return null). diff --git a/src/Storages/System/StorageSystemSettings.cpp b/src/Storages/System/StorageSystemSettings.cpp index f0a3e47defb..5670d63ab71 100644 --- a/src/Storages/System/StorageSystemSettings.cpp +++ b/src/Storages/System/StorageSystemSettings.cpp @@ -32,8 +32,8 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, const Context auto settings_constraints = context.getSettingsConstraints(); for (const auto & setting : settings) { - StringRef setting_name = setting.getName(); - res_columns[0]->insert(setting_name.toString()); + std::string_view setting_name{setting.getName()}; + res_columns[0]->insert(setting_name); res_columns[1]->insert(setting.getValueAsString()); res_columns[2]->insert(setting.isChanged()); res_columns[3]->insert(setting.getDescription().toString()); diff --git a/src/Storages/System/StorageSystemSettingsProfileElements.cpp b/src/Storages/System/StorageSystemSettingsProfileElements.cpp index 2e4d1ad1e05..bfbc765cb9b 100644 --- a/src/Storages/System/StorageSystemSettingsProfileElements.cpp +++ b/src/Storages/System/StorageSystemSettingsProfileElements.cpp @@ -126,12 +126,12 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns } } - if ((element.setting_index != static_cast(-1)) + if (!element.setting_name.empty() && (!element.value.isNull() || !element.min_value.isNull() || !element.max_value.isNull() || element.readonly)) { - auto setting_name = Settings::getName(element.setting_index); + const auto & setting_name = element.setting_name; column_index.push_back(index++); - column_setting_name.insertData(setting_name.data, setting_name.size); + column_setting_name.insertData(setting_name.data(), setting_name.size()); column_setting_name_null_map.push_back(false); if (element.value.isNull()) @@ -141,7 +141,7 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns } else { - String str = Settings::valueToString(element.setting_index, element.value); + String str = Settings::valueToString(setting_name, element.value); column_value.insertData(str.data(), str.length()); column_value_null_map.push_back(false); } @@ -153,7 +153,7 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns } else { - String str = Settings::valueToString(element.setting_index, element.min_value); + String str = Settings::valueToString(setting_name, element.min_value); column_min.insertData(str.data(), str.length()); column_min_null_map.push_back(false); } @@ -165,7 +165,7 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns } else { - String str = Settings::valueToString(element.setting_index, element.max_value); + String str = Settings::valueToString(setting_name, element.max_value); column_max.insertData(str.data(), str.length()); column_max_null_map.push_back(false); } From 56665a15f7fd043ed67ead4d8126f0464267b0fe Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 20 Jul 2020 12:57:17 +0300 Subject: [PATCH 11/20] Rework and rename the template class SettingsCollection => BaseSettings. --- programs/client/Client.cpp | 10 +- programs/server/Server.cpp | 4 +- src/Access/SettingsConstraints.cpp | 79 ++- src/Access/SettingsProfileElement.cpp | 6 +- src/Access/UsersConfigAccessStorage.cpp | 7 +- src/Client/Connection.cpp | 6 +- src/Core/BaseSettings.cpp | 54 ++ src/Core/BaseSettings.h | 652 ++++++++++++++++++ src/Core/Settings.cpp | 35 +- src/Core/Settings.h | 64 +- src/Core/SettingsCollection.cpp | 65 -- src/Core/SettingsCollection.h | 274 -------- src/Core/SettingsCollectionImpl.h | 376 ---------- src/Core/ya.make | 2 +- src/Interpreters/Context.cpp | 4 +- src/Server/TCPHandler.cpp | 12 +- src/Storages/Distributed/DirectoryMonitor.cpp | 4 +- .../DistributedBlockOutputStream.cpp | 2 +- src/Storages/Kafka/KafkaSettings.cpp | 3 +- src/Storages/Kafka/KafkaSettings.h | 20 +- src/Storages/Kafka/StorageKafka.cpp | 9 +- src/Storages/MergeTree/MergeTreeData.cpp | 7 +- src/Storages/MergeTree/MergeTreeSettings.cpp | 3 +- src/Storages/MergeTree/MergeTreeSettings.h | 27 +- src/Storages/RabbitMQ/RabbitMQSettings.cpp | 3 +- src/Storages/RabbitMQ/RabbitMQSettings.h | 14 +- .../System/StorageSystemMergeTreeSettings.cpp | 12 +- src/Storages/System/StorageSystemSettings.cpp | 16 +- .../StorageSystemSettingsProfileElements.cpp | 6 +- 29 files changed, 885 insertions(+), 891 deletions(-) create mode 100644 src/Core/BaseSettings.cpp create mode 100644 src/Core/BaseSettings.h delete mode 100644 src/Core/SettingsCollection.cpp delete mode 100644 src/Core/SettingsCollection.h delete mode 100644 src/Core/SettingsCollectionImpl.h diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 797342a1b44..bb862348e52 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -232,10 +232,10 @@ private: context.setQueryParameters(query_parameters); /// settings and limits could be specified in config file, but passed settings has higher priority - for (const auto & setting : context.getSettingsRef()) + for (auto setting : context.getSettingsRef().allUnchanged()) { - const String & name = setting.getName().toString(); - if (config().has(name) && !setting.isChanged()) + const auto & name = setting.getName(); + if (config().has(name)) context.setSetting(name, config().getString(name)); } @@ -2252,9 +2252,9 @@ public: /// Copy settings-related program options to config. /// TODO: Is this code necessary? - for (const auto & setting : context.getSettingsRef()) + for (auto setting : context.getSettingsRef().all()) { - const String name = setting.getName().toString(); + const auto & name = setting.getName(); if (options.count(name)) config().setString(name, options[name].as()); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c3b17824151..85cd75eaf4f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -223,9 +223,9 @@ void checkForUserSettingsAtTopLevel(const Poco::Util::AbstractConfiguration & co return; Settings settings; - for (const auto & setting : settings) + for (auto setting : settings.all()) { - std::string name = setting.getName().toString(); + const auto & name = setting.getName(); if (config.has(name)) { throw Exception(fmt::format("A setting '{}' appeared at top level in config {}." diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 4f4ff02a5e1..4d39c3c70c3 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -32,7 +32,7 @@ void SettingsConstraints::clear() void SettingsConstraints::setMinValue(const std::string_view & setting_name, const Field & min_value) { - getConstraintRef(setting_name).min_value = Settings::castValue(setting_name, min_value); + getConstraintRef(setting_name).min_value = Settings::castValueUtil(setting_name, min_value); } Field SettingsConstraints::getMinValue(const std::string_view & setting_name) const @@ -47,7 +47,7 @@ Field SettingsConstraints::getMinValue(const std::string_view & setting_name) co void SettingsConstraints::setMaxValue(const std::string_view & setting_name, const Field & max_value) { - getConstraintRef(setting_name).max_value = Settings::castValue(setting_name, max_value); + getConstraintRef(setting_name).max_value = Settings::castValueUtil(setting_name, max_value); } Field SettingsConstraints::getMaxValue(const std::string_view & setting_name) const @@ -78,8 +78,8 @@ bool SettingsConstraints::isReadOnly(const std::string_view & setting_name) cons void SettingsConstraints::set(const std::string_view & setting_name, const Field & min_value, const Field & max_value, bool read_only) { auto & ref = getConstraintRef(setting_name); - ref.min_value = Settings::castValue(setting_name, min_value); - ref.max_value = Settings::castValue(setting_name, max_value); + ref.min_value = Settings::castValueUtil(setting_name, min_value); + ref.max_value = Settings::castValueUtil(setting_name, max_value); ref.read_only = read_only; } @@ -150,14 +150,58 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const { const String & setting_name = change.name; + bool cannot_cast; + auto cast_value = [&](const Field & x) -> Field + { + cannot_cast = false; + if (reaction == THROW_ON_VIOLATION) + return Settings::castValueUtil(setting_name, x); + else + { + try + { + return Settings::castValueUtil(setting_name, x); + } + catch (...) + { + cannot_cast = true; + return {}; + } + } + }; + + bool cannot_compare = false; + auto less = [&](const Field & left, const Field & right) + { + cannot_compare = false; + if (reaction == THROW_ON_VIOLATION) + return applyVisitor(FieldVisitorAccurateLess{}, left, right); + else + { + try + { + return applyVisitor(FieldVisitorAccurateLess{}, left, right); + } + catch (...) + { + cannot_compare = true; + return false; + } + } + }; + + Field current_value; + if (reaction == THROW_ON_VIOLATION) + current_value = current_settings.get(setting_name); + else if (!current_settings.tryGet(setting_name, current_value)) + return false; /// Setting isn't checked if value has not changed. - Field current_value = current_settings.get(setting_name); if (change.value == current_value) return false; - Field new_value = Settings::castValue(setting_name, change.value); - if (new_value == current_value) + Field new_value = cast_value(change.value); + if ((new_value == current_value) || cannot_cast) return false; if (!current_settings.allow_ddl && setting_name == "allow_ddl") @@ -202,27 +246,6 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh const Field & min_value = constraint->min_value; const Field & max_value = constraint->max_value; - - bool cannot_compare = false; - auto less = [&](const Field & left, const Field & right) - { - cannot_compare = false; - if (reaction == THROW_ON_VIOLATION) - return applyVisitor(FieldVisitorAccurateLess{}, left, right); - else - { - try - { - return applyVisitor(FieldVisitorAccurateLess{}, left, right); - } - catch (...) - { - cannot_compare = true; - return false; - } - } - }; - if (!min_value.isNull() && !max_value.isNull() && (less(max_value, min_value) || cannot_compare)) { if (reaction == THROW_ON_VIOLATION) diff --git a/src/Access/SettingsProfileElement.cpp b/src/Access/SettingsProfileElement.cpp index 3d7ebb76ca0..304eb4b742a 100644 --- a/src/Access/SettingsProfileElement.cpp +++ b/src/Access/SettingsProfileElement.cpp @@ -43,11 +43,11 @@ void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const A readonly = ast.readonly; if (!value.isNull()) - value = Settings::castValue(setting_name, value); + value = Settings::castValueUtil(setting_name, value); if (!min_value.isNull()) - min_value = Settings::castValue(setting_name, min_value); + min_value = Settings::castValueUtil(setting_name, min_value); if (!max_value.isNull()) - max_value = Settings::castValue(setting_name, max_value); + max_value = Settings::castValueUtil(setting_name, max_value); } } diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index b378173910c..72d19c7d593 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -374,9 +374,9 @@ namespace for (const String & constraint_type : constraint_types) { if (constraint_type == "min") - profile_element.min_value = Settings::stringToValue(setting_name, config.getString(path_to_name + "." + constraint_type)); + profile_element.min_value = Settings::stringToValueUtil(setting_name, config.getString(path_to_name + "." + constraint_type)); else if (constraint_type == "max") - profile_element.max_value = Settings::stringToValue(setting_name, config.getString(path_to_name + "." + constraint_type)); + profile_element.max_value = Settings::stringToValueUtil(setting_name, config.getString(path_to_name + "." + constraint_type)); else if (constraint_type == "readonly") profile_element.readonly = true; else @@ -417,9 +417,8 @@ namespace const auto & setting_name = key; SettingsProfileElement profile_element; - size_t setting_index = Settings::findIndexStrict(key); profile_element.setting_name = setting_name; - profile_element.value = Settings::stringToValue(setting_name, config.getString(profile_config + "." + key)); + profile_element.value = Settings::stringToValueUtil(setting_name, config.getString(profile_config + "." + key)); profile->elements.emplace_back(std::move(profile_element)); } diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 3bb8af72516..02e107db9a6 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -397,9 +397,9 @@ void Connection::sendQuery( /// Per query settings. if (settings) { - auto settings_format = (server_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsBinaryFormat::STRINGS - : SettingsBinaryFormat::OLD; - settings->serialize(*out, settings_format); + auto settings_format = (server_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsWriteFormat::STRINGS_WITH_FLAGS + : SettingsWriteFormat::BINARY; + settings->write(*out, settings_format); } else writeStringBinary("" /* empty string is a marker of the end of settings */, *out); diff --git a/src/Core/BaseSettings.cpp b/src/Core/BaseSettings.cpp new file mode 100644 index 00000000000..ff2ddf81cff --- /dev/null +++ b/src/Core/BaseSettings.cpp @@ -0,0 +1,54 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +void BaseSettingsHelpers::writeString(const std::string_view & str, WriteBuffer & out) +{ + writeStringBinary(str, out); +} + + +String BaseSettingsHelpers::readString(ReadBuffer & in) +{ + String str; + readStringBinary(str, in); + return str; +} + + +void BaseSettingsHelpers::writeFlags(Flags flags, WriteBuffer & out) +{ + writeVarUInt(flags, out); +} + + +BaseSettingsHelpers::Flags BaseSettingsHelpers::readFlags(ReadBuffer & in) +{ + UInt64 res; + readVarUInt(res, in); + return static_cast(res); +} + + +void BaseSettingsHelpers::throwSettingNotFound(const std::string_view & name) +{ + throw Exception("Unknown setting " + String{name}, ErrorCodes::UNKNOWN_SETTING); +} + + +void BaseSettingsHelpers::warningSettingNotFound(const std::string_view & name) +{ + static auto * log = &Poco::Logger::get("Settings"); + LOG_WARNING(log, "Unknown setting {}, skipping", name); +} + +} diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h new file mode 100644 index 00000000000..d6e751a7ee6 --- /dev/null +++ b/src/Core/BaseSettings.h @@ -0,0 +1,652 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ +class ReadBuffer; +class WriteBuffer; + +enum class SettingsWriteFormat +{ + BINARY, /// Part of the settings are serialized as strings, and other part as variants. This is the old behaviour. + STRINGS_WITH_FLAGS, /// All settings are serialized as strings. Before each value the flag `is_important` is serialized. + DEFAULT = STRINGS_WITH_FLAGS, +}; + + +/** Template class to define collections of settings. + * Example of usage: + * + * mysettings.h: + * #define APPLY_FOR_MYSETTINGS(M) \ + * M(UInt64, a, 100, "Description of a", 0) \ + * M(Float, f, 3.11, "Description of f", IMPORTANT) // IMPORTANT - means the setting can't be ignored by older versions) \ + * M(String, s, "default", "Description of s", 0) + * + * DECLARE_SETTINGS_TRAITS(MySettingsTraits, APPLY_FOR_MYSETTINGS) + + * struct MySettings : public BaseSettings + * { + * }; + * + * mysettings.cpp: + * IMPLEMENT_SETTINGS_TRAITS(MySettingsTraits, APPLY_FOR_MYSETTINGS) + */ +template +class BaseSettings : public Traits_::Data +{ +public: + using Traits = Traits_; + + void set(const std::string_view & name, const Field & value); + Field get(const std::string_view & name) const; + + void setString(const std::string_view & name, const String & value); + String getString(const std::string_view & name) const; + + bool tryGet(const std::string_view & name, Field & value) const; + bool tryGetString(const std::string_view & name, String & value) const; + + bool isChanged(const std::string_view & name) const; + SettingsChanges changes() const; + void applyChange(const SettingChange & change); + void applyChanges(const SettingsChanges & changes); + void applyChanges(const BaseSettings & changes); + + /// Resets all the settings to their default values. + void resetToDefault(); + + static bool has(const std::string_view & name); + static const char * getTypeName(const std::string_view & name); + static const char * getDescription(const std::string_view & name); + + /// Checks if it's possible to assign a field to a specified value and throws an exception if not. + /// This function doesn't change the fields, it performs check only. + static void checkCanSet(const std::string_view & name, const Field & value); + static void checkCanSetString(const std::string_view & name, const String & str); + + /// Conversions without changing the fields. + static Field castValueUtil(const std::string_view & name, const Field & value); + static String valueToStringUtil(const std::string_view & name, const Field & value); + static Field stringToValueUtil(const std::string_view & name, const String & str); + + void write(WriteBuffer & out, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT) const; + void read(ReadBuffer & in, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT); + + // A debugging aid. + std::string toString() const; + + /// Represents a reference to a setting field. + class SettingFieldRef + { + public: + const String & getName() const; + Field getValue() const; + String getValueString() const; + bool isValueChanged() const; + const char * getTypeName() const; + const char * getDescription() const; + + bool operator==(const SettingFieldRef & other) const { return (getName() == other.getName()) && (getValue() == other.getValue()); } + bool operator!=(const SettingFieldRef & other) const { return !(*this == other); } + + private: + friend class BaseSettings; + SettingFieldRef(const typename Traits::Data & data_, const typename Traits::Accessor & accessor_, size_t index_) : data(&data_), accessor(&accessor_), index(index_) {} + + const typename Traits::Data * data = nullptr; + const typename Traits::Accessor * accessor = nullptr; + size_t index = 0; + }; + + enum SkipFlags + { + SKIP_NONE = 0, + SKIP_CHANGED = 0x01, + SKIP_UNCHANGED = 0x02, + SKIP_ALL = SKIP_CHANGED | SKIP_UNCHANGED, + }; + + class Iterator + { + public: + Iterator & operator++(); + Iterator operator++(int); + SettingFieldRef operator *() const; + + bool operator ==(const Iterator & other) const; + bool operator !=(const Iterator & other) const { return !(*this == other); } + + private: + friend class BaseSettings; + Iterator(const BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_); + void doSkip(); + + const BaseSettings * settings = nullptr; + const typename Traits::Accessor * accessor = nullptr; + size_t index; + SkipFlags skip_flags; + }; + + class Range + { + public: + Range(const BaseSettings & settings_, SkipFlags skip_flags_) : settings(settings_), accessor(Traits::Accessor::instance()), skip_flags(skip_flags_) {} + Iterator begin() const { return Iterator(settings, accessor, skip_flags); } + Iterator end() const { return Iterator(settings, accessor, SKIP_ALL); } + + private: + const BaseSettings & settings; + const typename Traits::Accessor & accessor; + SkipFlags skip_flags; + }; + + Range all(SkipFlags skip_flags = SKIP_NONE) const { return Range{*this, skip_flags}; } + Range allChanged() const { return all(SKIP_UNCHANGED); } + Range allUnchanged() const { return all(SKIP_CHANGED); } + + Iterator begin() const { return allChanged().begin(); } + Iterator end() const { return allChanged().end(); } +}; + +struct BaseSettingsHelpers +{ + [[noreturn]] static void throwSettingNotFound(const std::string_view & name); + static void warningSettingNotFound(const std::string_view & name); + + static void writeString(const std::string_view & str, WriteBuffer & out); + static String readString(ReadBuffer & in); + + enum Flags : UInt64 + { + IMPORTANT = 0x01, + }; + static void writeFlags(Flags flags, WriteBuffer & out); + static Flags readFlags(ReadBuffer & in); +}; + +template +void BaseSettings::set(const std::string_view & name, const Field & value) +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + accessor.setValue(*this, index, value); + else + BaseSettingsHelpers::throwSettingNotFound(name); +} + +template +Field BaseSettings::get(const std::string_view & name) const +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + return accessor.getValue(*this, index); + else + BaseSettingsHelpers::throwSettingNotFound(name); +} + +template +void BaseSettings::setString(const std::string_view & name, const String & value) +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + accessor.setValueString(*this, index, value); + else + BaseSettingsHelpers::throwSettingNotFound(name); +} + +template +String BaseSettings::getString(const std::string_view & name) const +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + return accessor.getValueString(*this, index); + else + BaseSettingsHelpers::throwSettingNotFound(name); +} + +template +bool BaseSettings::tryGet(const std::string_view & name, Field & value) const +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + { + value = accessor.getValue(*this, index); + return true; + } + return false; +} + +template +bool BaseSettings::tryGetString(const std::string_view & name, String & value) const +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + { + value = accessor.getValueString(*this, index); + return true; + } + return false; +} + +template +bool BaseSettings::isChanged(const std::string_view & name) const +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + return accessor.isValueChanged(*this, index); + return false; +} + +template +SettingsChanges BaseSettings::changes() const +{ + SettingsChanges res; + for (const auto & field : *this) + res.emplace_back(field.getName(), field.getValue()); + return res; +} + +template +void BaseSettings::applyChange(const SettingChange & change) +{ + set(change.name, change.value); +} + +template +void BaseSettings::applyChanges(const SettingsChanges & changes) +{ + for (const auto & change : changes) + applyChange(change); +} + +template +void BaseSettings::applyChanges(const BaseSettings & other_settings) +{ + for (const auto & field : other_settings) + set(field.getName(), field.getValue()); +} + +template +void BaseSettings::resetToDefault() +{ + const auto & accessor = Traits::Accessor::instance(); + for (size_t i : ext::range(accessor.size())) + { + if (accessor.isValueChanged(*this, i)) + accessor.resetValueToDefault(*this, i); + } +} + +template +bool BaseSettings::has(const std::string_view & name) +{ + const auto & accessor = Traits::Accessor::instance(); + return (accessor.find(name) != static_cast(-1)); +} + + +template +const char * BaseSettings::getTypeName(const std::string_view & name) +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + return accessor.getTypeName(index); + else + BaseSettingsHelpers::throwSettingNotFound(name); +} + +template +const char * BaseSettings::getDescription(const std::string_view & name) +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + return accessor.getDescription(index); + else + BaseSettingsHelpers::throwSettingNotFound(name); +} + +template +void BaseSettings::checkCanSet(const std::string_view & name, const Field & value) +{ + castValueUtil(name, value); +} + +template +void BaseSettings::checkCanSetString(const std::string_view & name, const String & str) +{ + stringToValueUtil(name, str); +} + +template +Field BaseSettings::castValueUtil(const std::string_view & name, const Field & value) +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + return accessor.castValueUtil(index, value); + else + BaseSettingsHelpers::throwSettingNotFound(name); +} + +template +String BaseSettings::valueToStringUtil(const std::string_view & name, const Field & value) +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + return accessor.valueToStringUtil(index, value); + else + BaseSettingsHelpers::throwSettingNotFound(name); +} + +template +Field BaseSettings::stringToValueUtil(const std::string_view & name, const String & str) +{ + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + return accessor.stringToValueUtil(index, str); + else + BaseSettingsHelpers::throwSettingNotFound(name); +} + +template +void BaseSettings::write(WriteBuffer & out, SettingsWriteFormat format) const +{ + const auto & accessor = Traits::Accessor::instance(); + + for (auto field : *this) + { + bool is_important = accessor.isImportant(field.index); + + BaseSettingsHelpers::writeString(field.getName(), out); + + if ((format >= SettingsWriteFormat::STRINGS_WITH_FLAGS)) + { + using Flags = BaseSettingsHelpers::Flags; + Flags flags{0}; + if (is_important) + flags = static_cast(flags | Flags::IMPORTANT); + BaseSettingsHelpers::writeFlags(flags, out); + + BaseSettingsHelpers::writeString(field.getValueString(), out); + } + else + accessor.writeBinary(*this, field.index, out); + } + + /// Empty string is a marker of the end of settings. + BaseSettingsHelpers::writeString(std::string_view{}, out); +} + +template +void BaseSettings::read(ReadBuffer & in, SettingsWriteFormat format) +{ + resetToDefault(); + const auto & accessor = Traits::Accessor::instance(); + while (true) + { + String name = BaseSettingsHelpers::readString(in); + if (name.empty() /* empty string is a marker of the end of settings */) + break; + size_t index = accessor.find(name); + + using Flags = BaseSettingsHelpers::Flags; + Flags flags{0}; + if (format >= SettingsWriteFormat::STRINGS_WITH_FLAGS) + flags = BaseSettingsHelpers::readFlags(in); + bool is_important = (flags & Flags::IMPORTANT); + + if (index != static_cast(-1)) + { + if (format >= SettingsWriteFormat::STRINGS_WITH_FLAGS) + accessor.setValueString(*this, index, BaseSettingsHelpers::readString(in)); + else + accessor.readBinary(*this, index, in); + } + else if (is_important) + { + BaseSettingsHelpers::throwSettingNotFound(name); + } + else + { + BaseSettingsHelpers::warningSettingNotFound(name); + BaseSettingsHelpers::readString(in); + } + } +} + +template +String BaseSettings::toString() const +{ + String res; + for (const auto & field : *this) + { + if (!res.empty()) + res += ", "; + res += field.getName() + " = " + field.getValueString(); + } + return res; +} + +template +bool operator==(const BaseSettings & left, const BaseSettings & right) +{ + auto l = left.begin(); + for (const auto & r : right) + { + if ((l == left.end()) || (*l != r)) + return false; + ++l; + } + return l == left.end(); +} + +template +bool operator!=(const BaseSettings & left, const BaseSettings & right) +{ + return !(left == right); +} + +template +BaseSettings::Iterator::Iterator(const BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_) + : settings(&settings_), accessor(&accessor_), skip_flags(skip_flags_) +{ + if (skip_flags == SKIP_ALL) + index = accessor->size(); + else + index = 0; + + doSkip(); +} + +template +typename BaseSettings::Iterator & BaseSettings::Iterator::operator++() +{ + if (index != accessor->size()) + ++index; + doSkip(); + return *this; +} + +template +typename BaseSettings::Iterator BaseSettings::Iterator::operator++(int) +{ + auto res = *this; + ++*this; + return res; +} + +template +typename BaseSettings::SettingFieldRef BaseSettings::Iterator::operator*() const +{ + return {*settings, *accessor, index}; +} + +template +void BaseSettings::Iterator::doSkip() +{ + if (skip_flags & SKIP_CHANGED) + { + while ((index != accessor->size()) && accessor->isValueChanged(*settings, index)) + ++index; + } + else if (skip_flags & SKIP_UNCHANGED) + { + while ((index != accessor->size()) && !accessor->isValueChanged(*settings, index)) + ++index; + } +} + +template +bool BaseSettings::Iterator::operator ==(const typename BaseSettings::Iterator & other) const +{ + return ((index == other.index) && (settings == other.settings)); +} + +template +const String & BaseSettings::SettingFieldRef::getName() const +{ + return accessor->getName(index); +} + +template +Field BaseSettings::SettingFieldRef::getValue() const +{ + return accessor->getValue(*data, index); +} + +template +String BaseSettings::SettingFieldRef::getValueString() const +{ + return accessor->getValueString(*data, index); +} + +template +bool BaseSettings::SettingFieldRef::isValueChanged() const +{ + return accessor->isValueChanged(*data, index); +} + +template +const char * BaseSettings::SettingFieldRef::getTypeName() const +{ + return accessor->getTypeName(index); +} + +template +const char * BaseSettings::SettingFieldRef::getDescription() const +{ + return accessor->getDescription(index); +} + +#define DECLARE_SETTINGS_TRAITS(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO) \ + struct SETTINGS_TRAITS_NAME \ + { \ + struct Data \ + { \ + LIST_OF_SETTINGS_MACRO(DECLARE_SETTINGS_TRAITS_) \ + }; \ + \ + class Accessor \ + { \ + public: \ + static const Accessor & instance(); \ + size_t size() const { return field_infos.size(); } \ + size_t find(const std::string_view & name) const; \ + const String & getName(size_t index) const { return field_infos[index].name; } \ + const char * getTypeName(size_t index) const { return field_infos[index].type; } \ + const char * getDescription(size_t index) const { return field_infos[index].description; } \ + bool isImportant(size_t index) const { return field_infos[index].is_important; } \ + Field castValueUtil(size_t index, const Field & value) const { return field_infos[index].cast_value_util_function(value); } \ + String valueToStringUtil(size_t index, const Field & value) const { return field_infos[index].value_to_string_util_function(value); } \ + Field stringToValueUtil(size_t index, const String & str) const { return field_infos[index].string_to_value_util_function(str); } \ + void setValue(Data & data, size_t index, const Field & value) const { return field_infos[index].set_value_function(data, value); } \ + Field getValue(const Data & data, size_t index) const { return field_infos[index].get_value_function(data); } \ + void setValueString(Data & data, size_t index, const String & str) const { return field_infos[index].set_value_string_function(data, str); } \ + String getValueString(const Data & data, size_t index) const { return field_infos[index].get_value_string_function(data); } \ + bool isValueChanged(const Data & data, size_t index) const { return field_infos[index].is_value_changed_function(data); } \ + void resetValueToDefault(Data & data, size_t index) const { return field_infos[index].reset_value_to_default_function(data); } \ + void writeBinary(const Data & data, size_t index, WriteBuffer & out) const { return field_infos[index].write_binary_function(data, out); } \ + void readBinary(Data & data, size_t index, ReadBuffer & in) const { return field_infos[index].read_binary_function(data, in); } \ + \ + private: \ + Accessor(); \ + struct FieldInfo \ + { \ + String name; \ + const char * type; \ + const char * description; \ + bool is_important; \ + Field (*cast_value_util_function)(const Field &); \ + String (*value_to_string_util_function)(const Field &); \ + Field (*string_to_value_util_function)(const String &); \ + void (*set_value_function)(Data &, const Field &) ; \ + Field (*get_value_function)(const Data &) ; \ + void (*set_value_string_function)(Data &, const String &) ; \ + String (*get_value_string_function)(const Data &) ; \ + bool (*is_value_changed_function)(const Data &); \ + void (*reset_value_to_default_function)(Data &) ; \ + void (*write_binary_function)(const Data &, WriteBuffer &) ; \ + void (*read_binary_function)(Data &, ReadBuffer &) ; \ + }; \ + std::vector field_infos; \ + std::unordered_map name_to_index_map; \ + }; \ + }; + +#define DECLARE_SETTINGS_TRAITS_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ + SettingField##TYPE NAME {DEFAULT}; + +#define IMPLEMENT_SETTINGS_TRAITS(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO) \ + const SETTINGS_TRAITS_NAME::Accessor & SETTINGS_TRAITS_NAME::Accessor::instance() \ + { \ + static const Accessor the_instance = [] \ + { \ + Accessor res; \ + constexpr int IMPORTANT = 1; \ + UNUSED(IMPORTANT); \ + LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_TRAITS_) \ + for (size_t i : ext::range(res.field_infos.size())) \ + { \ + const auto & info = res.field_infos[i]; \ + res.name_to_index_map.emplace(info.name, i); \ + } \ + return res; \ + }(); \ + return the_instance; \ + } \ + \ + SETTINGS_TRAITS_NAME::Accessor::Accessor() {} \ + \ + size_t SETTINGS_TRAITS_NAME::Accessor::find(const std::string_view & name) const \ + { \ + auto it = name_to_index_map.find(name); \ + if (it != name_to_index_map.end()) \ + return it->second; \ + return static_cast(-1); \ + } \ + \ + template class BaseSettings; + +//-V:IMPLEMENT_SETTINGS:501 +#define IMPLEMENT_SETTINGS_TRAITS_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ + res.field_infos.emplace_back( \ + FieldInfo{#NAME, #TYPE, DESCRIPTION, FLAGS & IMPORTANT, \ + [](const Field & value) -> Field { return static_cast(SettingField##TYPE{value}); }, \ + [](const Field & value) -> String { return SettingField##TYPE{value}.toString(); }, \ + [](const String & str) -> Field { SettingField##TYPE temp; temp.parseFromString(str); return static_cast(temp); }, \ + [](Data & data, const Field & value) { data.NAME = value; }, \ + [](const Data & data) -> Field { return static_cast(data.NAME); }, \ + [](Data & data, const String & str) { data.NAME.parseFromString(str); }, \ + [](const Data & data) -> String { return data.NAME.toString(); }, \ + [](const Data & data) -> bool { return data.NAME.changed; }, \ + [](Data & data) { data.NAME = SettingField##TYPE{DEFAULT}; }, \ + [](const Data & data, WriteBuffer & out) { data.NAME.writeBinary(out); }, \ + [](Data & data, ReadBuffer & in) { data.NAME.readBinary(in); } \ + }); +} diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 7de1b474397..7060ecb4bda 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { @@ -17,7 +16,7 @@ namespace ErrorCodes } -IMPLEMENT_SETTINGS_COLLECTION(Settings, LIST_OF_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS) /** Set the settings from the profile (in the server configuration, many settings can be listed in one profile). @@ -64,27 +63,24 @@ void Settings::dumpToArrayColumns(IColumn * column_names_, IColumn * column_valu auto * column_names = (column_names_) ? &typeid_cast(*column_names_) : nullptr; auto * column_values = (column_values_) ? &typeid_cast(*column_values_) : nullptr; - size_t size = 0; + size_t count = 0; - for (const auto & setting : *this) + for (auto setting : all(changed_only ? SKIP_UNCHANGED : SKIP_NONE)) { - if (!changed_only || setting.isChanged()) + if (column_names) { - if (column_names) - { - StringRef name = setting.getName(); - column_names->getData().insertData(name.data, name.size); - } - if (column_values) - column_values->getData().insert(setting.getValueAsString()); - ++size; + auto name = setting.getName(); + column_names->getData().insertData(name.data(), name.size()); } + if (column_values) + column_values->getData().insert(setting.getValueString()); + ++count; } if (column_names) { auto & offsets = column_names->getOffsets(); - offsets.push_back(offsets.back() + size); + offsets.push_back(offsets.back() + count); } /// Nested columns case @@ -93,20 +89,21 @@ void Settings::dumpToArrayColumns(IColumn * column_names_, IColumn * column_valu if (column_values && !the_same_offsets) { auto & offsets = column_values->getOffsets(); - offsets.push_back(offsets.back() + size); + offsets.push_back(offsets.back() + count); } } void Settings::addProgramOptions(boost::program_options::options_description & options) { - for (size_t index = 0; index != Settings::size(); ++index) + for (auto field : all()) { + const std::string_view name = field.getName(); auto on_program_option - = boost::function1([this, index](const std::string & value) { set(index, value); }); + = boost::function1([this, name](const std::string & value) { set(name, value); }); options.add(boost::shared_ptr(new boost::program_options::option_description( - Settings::getName(index).data, + name.data(), boost::program_options::value()->composing()->notifier(on_program_option), - Settings::getDescription(index).data))); + field.getDescription()))); } } } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9cc26f9c324..6bcd7f8de96 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1,53 +1,37 @@ #pragma once -#include +#include #include #include -namespace Poco +namespace Poco::Util { - namespace Util - { - class AbstractConfiguration; - } + class AbstractConfiguration; } -namespace boost +namespace boost::program_options { - namespace program_options - { - class options_description; - } + class options_description; } namespace DB { - class IColumn; -/** Settings of query execution. - * These settings go to users.xml. +/** List of settings: type, name, default value, description, flags + * + * 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. + * + * `flags` can be either 0 or IMPORTANT. + * A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions. */ -struct Settings : public SettingsCollection -{ - /// For initialization from empty initializer-list to be "value initialization", not "aggregate initialization" in C++14. - /// http://en.cppreference.com/w/cpp/language/aggregate_initialization - Settings() {} - /** List of settings: type, name, default value, description, flags - * - * 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. - * - * `flags` can be either 0 or IMPORTANT. - * A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions. - */ - -#define COMMON_SETTINGS(M) \ +#define COMMON_SETTINGS(M) \ 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) \ @@ -412,7 +396,7 @@ struct Settings : public SettingsCollection M(Bool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \ M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) -#define FORMAT_FACTORY_SETTINGS(M) \ +#define FORMAT_FACTORY_SETTINGS(M) \ 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, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.", 0) \ M(Bool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.", 0) \ @@ -475,11 +459,21 @@ struct Settings : public SettingsCollection M(Bool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \ M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ - #define LIST_OF_SETTINGS(M) \ - COMMON_SETTINGS(M) \ - FORMAT_FACTORY_SETTINGS(M) +#define LIST_OF_SETTINGS(M) \ + COMMON_SETTINGS(M) \ + FORMAT_FACTORY_SETTINGS(M) - DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) +DECLARE_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS) + + +/** Settings of query execution. + * These settings go to users.xml. + */ +struct Settings : public BaseSettings +{ + /// For initialization from empty initializer-list to be "value initialization", not "aggregate initialization" in C++14. + /// http://en.cppreference.com/w/cpp/language/aggregate_initialization + Settings() {} /** Set multiple settings from "profile" (in server configuration file (users.xml), profiles contain groups of multiple settings). * The profile can also be set using the `set` functions, like the profile setting. diff --git a/src/Core/SettingsCollection.cpp b/src/Core/SettingsCollection.cpp deleted file mode 100644 index f0c2b35b4a2..00000000000 --- a/src/Core/SettingsCollection.cpp +++ /dev/null @@ -1,65 +0,0 @@ -#include -#include - -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int UNKNOWN_SETTING; -} - - -namespace details -{ - void SettingsCollectionUtils::serializeName(const StringRef & name, WriteBuffer & buf) - { - writeStringBinary(name, buf); - } - - String SettingsCollectionUtils::deserializeName(ReadBuffer & buf) - { - String name; - readStringBinary(name, buf); - return name; - } - - void SettingsCollectionUtils::serializeFlag(bool flag, WriteBuffer & buf) - { - buf.write(flag); - } - - bool SettingsCollectionUtils::deserializeFlag(ReadBuffer & buf) - { - char c; - buf.readStrict(c); - return c; - } - - void SettingsCollectionUtils::skipValue(ReadBuffer & buf) - { - /// Ignore a string written by the function writeStringBinary(). - UInt64 size; - readVarUInt(size, buf); - buf.ignore(size); - } - - void SettingsCollectionUtils::warningNameNotFound(const StringRef & name) - { - static auto * log = &Poco::Logger::get("Settings"); - LOG_WARNING(log, "Unknown setting {}, skipping", name); - } - - void SettingsCollectionUtils::throwNameNotFound(const StringRef & name) - { - throw Exception("Unknown setting " + name.toString(), ErrorCodes::UNKNOWN_SETTING); - } -} -} diff --git a/src/Core/SettingsCollection.h b/src/Core/SettingsCollection.h deleted file mode 100644 index 276bde8c695..00000000000 --- a/src/Core/SettingsCollection.h +++ /dev/null @@ -1,274 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -class Field; -struct SettingChange; -class SettingsChanges; -class ReadBuffer; -class WriteBuffer; - -enum class SettingsBinaryFormat -{ - OLD, /// Part of the settings are serialized as strings, and other part as variants. This is the old behaviour. - STRINGS, /// All settings are serialized as strings. Before each value the flag `is_ignorable` is serialized. - DEFAULT = STRINGS, -}; - - -/** Template class to define collections of settings. - * Example of usage: - * - * mysettings.h: - * struct MySettings : public SettingsCollection - * { - * # define APPLY_FOR_MYSETTINGS(M) \ - * M(UInt64, a, 100, "Description of a", 0) \ - * M(Float, f, 3.11, "Description of f", IMPORTANT) // IMPORTANT - means the setting can't be ignored by older versions) \ - * M(String, s, "default", "Description of s", 0) - * - * DECLARE_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS) - * }; - * - * mysettings.cpp: - * IMPLEMENT_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS) - */ -template -class SettingsCollection -{ -private: - Derived & castToDerived() { return *static_cast(this); } - const Derived & castToDerived() const { return *static_cast(this); } - - struct MemberInfo - { - using IsChangedFunction = bool (*)(const Derived &); - using GetValueFunction = Field (*)(const Derived &); - using SetValueFunction = void (*)(Derived &, const Field &); - using GetValueAsStringFunction = String (*)(const Derived &); - using ParseValueFromStringFunction = void (*)(Derived &, const String &); - using WriteBinaryFunction = void (*)(const Derived &, WriteBuffer & buf); - using ReadBinaryFunction = void (*)(Derived &, ReadBuffer & buf); - using CastValueFunction = Field (*)(const Field &); - using StringToValueFunction = Field (*)(const String &); - using ValueToStringFunction = String (*)(const Field &); - - StringRef name; - StringRef description; - StringRef type; - bool is_important; - IsChangedFunction is_changed; - GetValueFunction get_value; - SetValueFunction set_value; - GetValueAsStringFunction get_value_as_string; - ParseValueFromStringFunction parse_value_from_string; - WriteBinaryFunction write_binary; - ReadBinaryFunction read_binary; - CastValueFunction cast_value; - StringToValueFunction string_to_value; - ValueToStringFunction value_to_string; - }; - - class MemberInfos - { - public: - MemberInfos(); - - size_t size() const { return infos.size(); } - const MemberInfo * data() const { return infos.data(); } - const MemberInfo & operator[](size_t index) const { return infos[index]; } - - const MemberInfo * find(const StringRef & name) const; - const MemberInfo & findStrict(const StringRef & name) const; - size_t findIndex(const StringRef & name) const; - size_t findIndexStrict(const StringRef & name) const; - - private: - void add(MemberInfo && member); - - std::vector infos; - std::unordered_map by_name_map; - }; - - static const MemberInfos & members(); - -public: - class const_iterator; - - /// Provides read-only access to a setting. - class const_reference - { - public: - const_reference(const Derived & collection_, const MemberInfo & member_) : collection(&collection_), member(&member_) {} - const_reference(const const_reference & src) = default; - const StringRef & getName() const { return member->name; } - const StringRef & getDescription() const { return member->description; } - const StringRef & getType() const { return member->type; } - bool isChanged() const { return member->is_changed(*collection); } - Field getValue() const; - String getValueAsString() const { return member->get_value_as_string(*collection); } - - protected: - friend class SettingsCollection::const_iterator; - const_reference() : collection(nullptr), member(nullptr) {} - const_reference & operator=(const const_reference &) = default; - const Derived * collection; - const MemberInfo * member; - }; - - /// Provides access to a setting. - class reference : public const_reference - { - public: - reference(Derived & collection_, const MemberInfo & member_) : const_reference(collection_, member_) {} - reference(const const_reference & src) : const_reference(src) {} - void setValue(const Field & value) { this->member->set_value(*const_cast(this->collection), value); } - void parseFromString(const String & value) { this->member->parse_value_from_string(*const_cast(this->collection), value); } - }; - - /// Iterator to iterating through all the settings. - class const_iterator - { - public: - const_iterator(const Derived & collection_, const MemberInfo * member_) : ref(const_cast(collection_), *member_) {} - const_iterator() = default; - const_iterator(const const_iterator & src) = default; - const_iterator & operator =(const const_iterator & src) = default; - const const_reference & operator *() const { return ref; } - const const_reference * operator ->() const { return &ref; } - const_iterator & operator ++() { ++ref.member; return *this; } - const_iterator operator ++(int) { const_iterator tmp = *this; ++*this; return tmp; } - bool operator ==(const const_iterator & rhs) const { return ref.member == rhs.ref.member && ref.collection == rhs.ref.collection; } - bool operator !=(const const_iterator & rhs) const { return !(*this == rhs); } - protected: - mutable reference ref; - }; - - class iterator : public const_iterator - { - public: - iterator(Derived & collection_, const MemberInfo * member_) : const_iterator(collection_, member_) {} - iterator() = default; - iterator(const const_iterator & src) : const_iterator(src) {} - iterator & operator =(const const_iterator & src) { const_iterator::operator =(src); return *this; } - reference & operator *() const { return this->ref; } - reference * operator ->() const { return &this->ref; } - iterator & operator ++() { const_iterator::operator ++(); return *this; } - iterator operator ++(int) { iterator tmp = *this; ++*this; return tmp; } - }; - - /// Returns the number of settings. - static size_t size() { return members().size(); } - - /// Returns name of a setting by its index (0..size()-1). - static StringRef getName(size_t index) { return members()[index].name; } - - /// Returns description of a setting. - static StringRef getDescription(size_t index) { return members()[index].description; } - static StringRef getDescription(const String & name) { return members().findStrict(name).description; } - - /// Searches a setting by its name; returns `npos` if not found. - static size_t findIndex(const StringRef & name) { return members().findIndex(name); } - static constexpr size_t npos = static_cast(-1); - - /// Searches a setting by its name; throws an exception if not found. - static size_t findIndexStrict(const StringRef & name) { return members().findIndexStrict(name); } - - /// Casts a value to a type according to a specified setting without actual changing this settings. - /// E.g. for SettingInt64 it casts Field to Field::Types::Int64. - static Field castValue(size_t index, const Field & value); - static Field castValue(const StringRef & name, const Field & value); - static Field castValue(const std::string_view & name, const Field & value) { return castValue(StringRef{name}, value); } - static Field castValue(const String & name, const Field & value) { return castValue(StringRef{name}, value); } - - /// Casts a value to a string according to a specified setting without actual changing this settings. - static Field stringToValue(size_t index, const String & str) { return members()[index].string_to_value(str); } - static Field stringToValue(const StringRef & name, const String & str) { return members().findStrict(name).string_to_value(str); } - static String valueToString(size_t index, const Field & value) { return members()[index].value_to_string(value); } - static String valueToString(const StringRef & name, const Field & value) { return members().findStrict(name).value_to_string(value); } - static String valueToString(const std::string_view & name, const Field & value) { return valueToString(name, value); } - static String valueToString(const String & name, const Field & value) { return valueToString(name, value); } - - iterator begin() { return iterator(castToDerived(), members().data()); } - const_iterator begin() const { return const_iterator(castToDerived(), members().data()); } - iterator end() { const auto & the_members = members(); return iterator(castToDerived(), the_members.data() + the_members.size()); } - const_iterator end() const { const auto & the_members = members(); return const_iterator(castToDerived(), the_members.data() + the_members.size()); } - - /// Returns a proxy object for accessing to a setting. Throws an exception if there is not setting with such name. - reference operator[](size_t index) { return reference(castToDerived(), members()[index]); } - reference operator[](const StringRef & name) { return reference(castToDerived(), members().findStrict(name)); } - const_reference operator[](size_t index) const { return const_reference(castToDerived(), members()[index]); } - const_reference operator[](const StringRef & name) const { return const_reference(castToDerived(), members().findStrict(name)); } - - /// Searches a setting by its name; returns end() if not found. - iterator find(const StringRef & name); - const_iterator find(const StringRef & name) const; - - /// Searches a setting by its name; throws an exception if not found. - iterator findStrict(const StringRef & name); - const_iterator findStrict(const StringRef & name) const; - - /// Sets setting's value. - void set(size_t index, const Field & value) { (*this)[index].setValue(value); } - void set(const StringRef & name, const Field & value) { (*this)[name].setValue(value); } - - /// Sets setting's value. Read value in text form from string (for example, from configuration file or from URL parameter). - void set(size_t index, const String & value) { (*this)[index].setValue(value); } - void set(const StringRef & name, const String & value) { (*this)[name].setValue(value); } - - /// Returns value of a setting. - Field get(size_t index) const; - Field get(const StringRef & name) const; - - /// Returns value of a setting converted to string. - String getAsString(size_t index) const { return (*this)[index].getValueAsString(); } - String getAsString(const StringRef & name) const { return (*this)[name].getValueAsString(); } - - /// Returns value of a setting; returns false if there is no setting with the specified name. - bool tryGet(const StringRef & name, Field & value) const; - - /// Returns value of a setting converted to string; returns false if there is no setting with the specified name. - bool tryGet(const StringRef & name, String & value) const; - - /// Compares two collections of settings. - bool operator ==(const SettingsCollection & rhs) const; - bool operator!=(const SettingsCollection & rhs) const { return !(*this == rhs); } - - /// Gathers all changed values (e.g. for applying them later to another collection of settings). - SettingsChanges changes() const; - - // A debugging aid. - std::string dumpChangesToString() const; - - /// Applies change to concrete setting. - void applyChange(const SettingChange & change); - - /// Applies changes to the settings. - void applyChanges(const SettingsChanges & changes); - - void copyChangesFrom(const Derived & src); - - void copyChangesTo(Derived & dest) const; - - /// Writes the settings to buffer (e.g. to be sent to remote server). - /// Only changed settings are written. They are written as list of contiguous name-value pairs, - /// finished with empty name. - void serialize(WriteBuffer & buf, SettingsBinaryFormat format = SettingsBinaryFormat::DEFAULT) const; - - /// Reads the settings from buffer. - void deserialize(ReadBuffer & buf, SettingsBinaryFormat format = SettingsBinaryFormat::DEFAULT); -}; - - -#define DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS_MACRO) \ - LIST_OF_SETTINGS_MACRO(DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_) - -#define DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ - SettingField##TYPE NAME {DEFAULT}; -} diff --git a/src/Core/SettingsCollectionImpl.h b/src/Core/SettingsCollectionImpl.h deleted file mode 100644 index fd365b9587e..00000000000 --- a/src/Core/SettingsCollectionImpl.h +++ /dev/null @@ -1,376 +0,0 @@ -#pragma once - -/** - * This file implements some functions that are dependent on Field type. - * Unlike SettingsCollection.h, we only have to include it once for each - * instantiation of SettingsCollection<>. - */ - -#include -#include -#include -#include - - -namespace DB -{ -namespace details -{ - struct SettingsCollectionUtils - { - static void serializeName(const StringRef & name, WriteBuffer & buf); - static String deserializeName(ReadBuffer & buf); - static void serializeFlag(bool flag, WriteBuffer & buf); - static bool deserializeFlag(ReadBuffer & buf); - static void skipValue(ReadBuffer & buf); - static void warningNameNotFound(const StringRef & name); - [[noreturn]] static void throwNameNotFound(const StringRef & name); - }; -} - - -template -size_t SettingsCollection::MemberInfos::findIndex(const StringRef & name) const -{ - auto it = by_name_map.find(name); - if (it == by_name_map.end()) - return static_cast(-1); // npos - return it->second; -} - - -template -size_t SettingsCollection::MemberInfos::findIndexStrict(const StringRef & name) const -{ - auto it = by_name_map.find(name); - if (it == by_name_map.end()) - details::SettingsCollectionUtils::throwNameNotFound(name); - return it->second; -} - - -template -const typename SettingsCollection::MemberInfo * SettingsCollection::MemberInfos::find(const StringRef & name) const -{ - auto it = by_name_map.find(name); - if (it == by_name_map.end()) - return nullptr; - else - return &infos[it->second]; -} - - -template -const typename SettingsCollection::MemberInfo & SettingsCollection::MemberInfos::findStrict(const StringRef & name) const -{ - return infos[findIndexStrict(name)]; -} - - -template -void SettingsCollection::MemberInfos::add(MemberInfo && member) -{ - size_t index = infos.size(); - infos.emplace_back(member); - by_name_map.emplace(infos.back().name, index); -} - - -template -const typename SettingsCollection::MemberInfos & -SettingsCollection::members() -{ - static const MemberInfos the_instance; - return the_instance; -} - - -template -Field SettingsCollection::const_reference::getValue() const -{ - return member->get_value(*collection); -} - - -template -Field SettingsCollection::castValue(size_t index, const Field & value) -{ - try - { - return members()[index].cast_value(value); - } - catch (Exception & e) - { - e.addMessage(fmt::format("in attempt to set the value of setting to {}", - applyVisitor(FieldVisitorToString(), value))); - throw; - } -} - - -template -Field SettingsCollection::castValue(const StringRef & name, const Field & value) -{ - return members().findStrict(name).cast_value(value); -} - - -template -typename SettingsCollection::iterator SettingsCollection::find(const StringRef & name) -{ - const auto * member = members().find(name); - if (member) - return iterator(castToDerived(), member); - return end(); -} - - -template -typename SettingsCollection::const_iterator SettingsCollection::find(const StringRef & name) const -{ - const auto * member = members().find(name); - if (member) - return const_iterator(castToDerived(), member); - return end(); -} - - -template -typename SettingsCollection::iterator SettingsCollection::findStrict(const StringRef & name) -{ - return iterator(castToDerived(), &members().findStrict(name)); -} - - -template -typename SettingsCollection::const_iterator SettingsCollection::findStrict(const StringRef & name) const -{ - return const_iterator(castToDerived(), &members().findStrict(name)); -} - - -template -Field SettingsCollection::get(size_t index) const -{ - return (*this)[index].getValue(); -} - - -template -Field SettingsCollection::get(const StringRef & name) const -{ - return (*this)[name].getValue(); -} - - -template -bool SettingsCollection::tryGet(const StringRef & name, Field & value) const -{ - auto it = find(name); - if (it == end()) - return false; - value = it->getValue(); - return true; -} - - -template -bool SettingsCollection::tryGet(const StringRef & name, String & value) const -{ - auto it = find(name); - if (it == end()) - return false; - value = it->getValueAsString(); - return true; -} - - -template -bool SettingsCollection::operator ==(const SettingsCollection & rhs) const -{ - const auto & the_members = members(); - for (size_t i = 0; i != the_members.size(); ++i) - { - const auto & member = the_members[i]; - bool left_changed = member.is_changed(castToDerived()); - bool right_changed = member.is_changed(rhs.castToDerived()); - if (left_changed || right_changed) - { - if (left_changed != right_changed) - return false; - if (member.get_value(castToDerived()) != member.get_value(rhs.castToDerived())) - return false; - } - } - return true; -} - - -template -SettingsChanges SettingsCollection::changes() const -{ - SettingsChanges found_changes; - const auto & the_members = members(); - for (size_t i = 0; i != the_members.size(); ++i) - { - const auto & member = the_members[i]; - if (member.is_changed(castToDerived())) - found_changes.push_back({member.name.toString(), member.get_value(castToDerived())}); - } - return found_changes; -} - - -template -std::string SettingsCollection::dumpChangesToString() const -{ - std::stringstream ss; - for (const auto & c : changes()) - { - ss << c.name << " = " - << applyVisitor(FieldVisitorToString(), c.value) << "\n"; - } - return ss.str(); -} - - -template -void SettingsCollection::applyChange(const SettingChange & change) -{ - set(change.name, change.value); -} - - -template -void SettingsCollection::applyChanges(const SettingsChanges & changes) -{ - for (const SettingChange & change : changes) - applyChange(change); -} - - -template -void SettingsCollection::copyChangesFrom(const Derived & src) -{ - const auto & the_members = members(); - for (size_t i = 0; i != the_members.size(); ++i) - { - const auto & member = the_members[i]; - if (member.is_changed(src)) - member.set_value(castToDerived(), member.get_value(src)); - } -} - - -template -void SettingsCollection::copyChangesTo(Derived & dest) const -{ - dest.copyChangesFrom(castToDerived()); -} - - -template -void SettingsCollection::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const -{ - const auto & the_members = members(); - for (size_t i = 0; i != the_members.size(); ++i) - { - const auto & member = the_members[i]; - if (member.is_changed(castToDerived())) - { - details::SettingsCollectionUtils::serializeName(member.name, buf); - if (format >= SettingsBinaryFormat::STRINGS) - { - details::SettingsCollectionUtils::serializeFlag(member.is_important, buf); - details::SettingsCollectionUtils::serializeName(member.get_value_as_string(castToDerived()), buf); - } - else - member.write_binary(castToDerived(), buf); - } - } - details::SettingsCollectionUtils::serializeName(StringRef{} /* empty string is a marker of the end of settings */, buf); -} - - -template -void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) -{ - const auto & the_members = members(); - while (true) - { - String name = details::SettingsCollectionUtils::deserializeName(buf); - if (name.empty() /* empty string is a marker of the end of settings */) - break; - auto * member = the_members.find(name); - bool is_important = true; - if (format >= SettingsBinaryFormat::STRINGS) - is_important = details::SettingsCollectionUtils::deserializeFlag(buf); - if (!member) - { - if (is_important) - { - details::SettingsCollectionUtils::throwNameNotFound(name); - } - else - { - details::SettingsCollectionUtils::warningNameNotFound(name); - details::SettingsCollectionUtils::skipValue(buf); - continue; - } - } - - if (format >= SettingsBinaryFormat::STRINGS) - { - String value = details::SettingsCollectionUtils::deserializeName(buf); - member->parse_value_from_string(castToDerived(), value); - } - else - member->read_binary(castToDerived(), buf); - } -} - - -//-V:IMPLEMENT_SETTINGS_COLLECTION:501 -#define IMPLEMENT_SETTINGS_COLLECTION(DERIVED_CLASS_NAME, LIST_OF_SETTINGS_MACRO) \ - template<> \ - SettingsCollection::MemberInfos::MemberInfos() \ - { \ - using Derived = DERIVED_CLASS_NAME; \ - struct Functions \ - { \ - LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \ - }; \ - constexpr int IMPORTANT = 1; \ - UNUSED(IMPORTANT); \ - LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_) \ - } \ - /** \ - * Instantiation should happen when all method definitions from SettingsCollectionImpl.h \ - * are accessible, so we instantiate explicitly. \ - */ \ - template class SettingsCollection; - - -#define IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ - static Field NAME##_getValue(const Derived & collection) { return static_cast(collection.NAME); } \ - static void NAME##_setValue(Derived & collection, const Field & value) { collection.NAME = value; } \ - static String NAME##_toString(const Derived & collection) { return collection.NAME.toString(); } \ - static void NAME##_parseFromString(Derived & collection, const String & value) { collection.NAME.parseFromString(value); } \ - static void NAME##_writeBinary(const Derived & collection, WriteBuffer & buf) { collection.NAME.writeBinary(buf); } \ - static void NAME##_readBinary(Derived & collection, ReadBuffer & buf) { collection.NAME.readBinary(buf); } \ - static Field NAME##_castValue(const Field & value) { return static_cast(SettingField##TYPE{value}); } \ - static Field NAME##_stringToValue(const String & str) { SettingField##TYPE temp; temp.parseFromString(str); return static_cast(temp); } \ - static String NAME##_valueToString(const Field & value) { return SettingField##TYPE{value}.toString(); } \ - - -#define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ - add({StringRef(#NAME, strlen(#NAME)), \ - StringRef(DESCRIPTION, strlen(DESCRIPTION)), \ - StringRef(#TYPE, strlen(#TYPE)), \ - FLAGS & IMPORTANT, \ - [](const Derived & d) { return d.NAME.changed; }, \ - &Functions::NAME##_getValue, &Functions::NAME##_setValue, \ - &Functions::NAME##_toString, &Functions::NAME##_parseFromString, \ - &Functions::NAME##_writeBinary, &Functions::NAME##_readBinary, \ - &Functions::NAME##_castValue, &Functions::NAME##_stringToValue, &Functions::NAME##_valueToString }); -} diff --git a/src/Core/ya.make b/src/Core/ya.make index 066c45e17ba..d8bf6782420 100644 --- a/src/Core/ya.make +++ b/src/Core/ya.make @@ -8,6 +8,7 @@ PEERDIR( SRCS( BackgroundSchedulePool.cpp + BaseSettings.cpp Block.cpp BlockInfo.cpp ColumnWithTypeAndName.cpp @@ -19,7 +20,6 @@ SRCS( PostgreSQLProtocol.cpp NamesAndTypes.cpp Settings.cpp - SettingsCollection.cpp SettingsEnums.cpp SettingsFields.cpp SortDescription.cpp diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9c057d1eb01..651b88d370a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -968,7 +968,7 @@ void Context::setSetting(const StringRef & name, const String & value) setProfile(value); return; } - settings.set(name, value); + settings.set(std::string_view{name}, value); if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions") calculateAccessRights(); @@ -983,7 +983,7 @@ void Context::setSetting(const StringRef & name, const Field & value) setProfile(value.safeGet()); return; } - settings.set(name, value); + settings.set(std::string_view{name}, value); if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions") calculateAccessRights(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 5553d5f8f5e..ab4ce820666 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -880,10 +880,10 @@ void TCPHandler::receiveQuery() /// Per query settings are also passed via TCP. /// We need to check them before applying due to they can violate the settings constraints. - auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsBinaryFormat::STRINGS - : SettingsBinaryFormat::OLD; + auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsWriteFormat::STRINGS_WITH_FLAGS + : SettingsWriteFormat::BINARY; Settings passed_settings; - passed_settings.deserialize(*in, settings_format); + passed_settings.read(*in, settings_format); auto settings_changes = passed_settings.changes(); if (client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY) { @@ -925,9 +925,9 @@ void TCPHandler::receiveUnexpectedQuery() skip_client_info.read(*in, client_revision); Settings skip_settings; - auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsBinaryFormat::STRINGS - : SettingsBinaryFormat::OLD; - skip_settings.deserialize(*in, settings_format); + auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsWriteFormat::STRINGS_WITH_FLAGS + : SettingsWriteFormat::BINARY; + skip_settings.read(*in, settings_format); readVarUInt(skip_uint_64, *in); readVarUInt(skip_uint_64, *in); diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 8b45573464f..f45cc7894ea 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -368,7 +368,7 @@ void StorageDistributedDirectoryMonitor::readHeader( } readStringBinary(insert_query, header_buf); - insert_settings.deserialize(header_buf); + insert_settings.read(header_buf); if (header_buf.hasPendingData()) client_info.read(header_buf, initiator_revision); @@ -382,7 +382,7 @@ void StorageDistributedDirectoryMonitor::readHeader( if (query_size == DBMS_DISTRIBUTED_SIGNATURE_HEADER_OLD_FORMAT) { - insert_settings.deserialize(in, SettingsBinaryFormat::OLD); + insert_settings.read(in, SettingsWriteFormat::BINARY); readStringBinary(insert_query, in); return; } diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 5afd000f279..172a398258f 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -591,7 +591,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: WriteBufferFromOwnString header_buf; writeVarUInt(ClickHouseRevision::get(), header_buf); writeStringBinary(query_string, header_buf); - context.getSettingsRef().serialize(header_buf); + context.getSettingsRef().write(header_buf); context.getClientInfo().write(header_buf, ClickHouseRevision::get()); /// Add new fields here, for example: diff --git a/src/Storages/Kafka/KafkaSettings.cpp b/src/Storages/Kafka/KafkaSettings.cpp index 2f572c094b4..4d80419af2d 100644 --- a/src/Storages/Kafka/KafkaSettings.cpp +++ b/src/Storages/Kafka/KafkaSettings.cpp @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB @@ -15,7 +14,7 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -IMPLEMENT_SETTINGS_COLLECTION(KafkaSettings, LIST_OF_KAFKA_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(KafkaSettingsTraits, LIST_OF_KAFKA_SETTINGS) void KafkaSettings::loadFromQuery(ASTStorage & storage_def) { diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 7cf9b90e081..b9b606e4660 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -1,21 +1,15 @@ #pragma once -#include +#include #include + namespace DB { - class ASTStorage; -/** Settings for the Kafka engine. - * Could be loaded from a CREATE TABLE query (SETTINGS clause). - */ -struct KafkaSettings : public SettingsCollection -{ - -#define KAFKA_RELATED_SETTINGS(M) \ +#define KAFKA_RELATED_SETTINGS(M) \ M(String, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.", 0) \ M(String, kafka_topic_list, "", "A list of Kafka topics.", 0) \ M(String, kafka_group_name, "", "Client group id string. All Kafka consumers sharing the same group.id belong to the same group.", 0) \ @@ -44,8 +38,14 @@ struct KafkaSettings : public SettingsCollection KAFKA_RELATED_SETTINGS(M) \ FORMAT_FACTORY_SETTINGS(M) - DECLARE_SETTINGS_COLLECTION(LIST_OF_KAFKA_SETTINGS) +DECLARE_SETTINGS_TRAITS(KafkaSettingsTraits, LIST_OF_KAFKA_SETTINGS) + +/** Settings for the Kafka engine. + * Could be loaded from a CREATE TABLE query (SETTINGS clause). + */ +struct KafkaSettings : public BaseSettings +{ void loadFromQuery(ASTStorage & storage_def); }; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 9d7bc273cd2..ce1f669d70f 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -169,12 +169,11 @@ SettingsChanges StorageKafka::createSettingsAdjustments() if (!schema_name.empty()) result.emplace_back("format_schema", schema_name); - for (auto & it : *kafka_settings) + for (auto setting : *kafka_settings) { - if (it.isChanged() && it.getName().toString().rfind("kafka_",0) == std::string::npos) - { - result.emplace_back(it.getName().toString(), it.getValue()); - } + const auto & name = setting.getName(); + if (name.find("kafka_") == std::string::npos) + result.emplace_back(name, setting.getValue()); } return result; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 67a00d7cd0c..223766e31ac 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -103,7 +103,6 @@ namespace ErrorCodes extern const int BAD_TTL_EXPRESSION; extern const int INCORRECT_FILE_NAME; extern const int BAD_DATA_PART_NAME; - extern const int UNKNOWN_SETTING; extern const int READONLY_SETTING; extern const int ABORTED; extern const int UNKNOWN_PART_TYPE; @@ -1467,17 +1466,13 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S if (old_metadata.hasSettingsChanges()) { - const auto current_changes = old_metadata.getSettingsChanges()->as().changes; const auto & new_changes = new_metadata.settings_changes->as().changes; for (const auto & changed_setting : new_changes) { const auto & setting_name = changed_setting.name; const auto & new_value = changed_setting.value; - if (MergeTreeSettings::findIndex(setting_name) == MergeTreeSettings::npos) - throw Exception{"Storage '" + getName() + "' doesn't have setting '" + setting_name + "'", - ErrorCodes::UNKNOWN_SETTING}; - + MergeTreeSettings::checkCanSet(setting_name, new_value); const Field * current_value = current_changes.tryGet(setting_name); if ((!current_value || *current_value != new_value) diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 5c4113c1565..75a41218efc 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -4,7 +4,6 @@ #include #include #include -#include namespace DB @@ -17,7 +16,7 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -IMPLEMENT_SETTINGS_COLLECTION(MergeTreeSettings, LIST_OF_MERGE_TREE_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(MergeTreeSettingsTraits, LIST_OF_MERGE_TREE_SETTINGS) void MergeTreeSettings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config) { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 31436121adc..b8307ee91d5 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -1,30 +1,21 @@ #pragma once #include -#include +#include -namespace Poco +namespace Poco::Util { - namespace Util - { - class AbstractConfiguration; - } + class AbstractConfiguration; } namespace DB { - class ASTStorage; -/** Settings for the MergeTree family of engines. - * Could be loaded from config or from a CREATE TABLE query (SETTINGS clause). - */ -struct MergeTreeSettings : public SettingsCollection -{ -#define LIST_OF_MERGE_TREE_SETTINGS(M) \ +#define LIST_OF_MERGE_TREE_SETTINGS(M) \ M(UInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ \ /** Data storing format settings. */ \ @@ -104,12 +95,18 @@ struct MergeTreeSettings : public SettingsCollection M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \ M(UInt64, check_delay_period, 60, "Obsolete setting, does nothing.", 0) \ - DECLARE_SETTINGS_COLLECTION(LIST_OF_MERGE_TREE_SETTINGS) - /// Settings that should not change after the creation of a table. #define APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(M) \ M(index_granularity) +DECLARE_SETTINGS_TRAITS(MergeTreeSettingsTraits, LIST_OF_MERGE_TREE_SETTINGS) + + +/** Settings for the MergeTree family of engines. + * Could be loaded from config or from a CREATE TABLE query (SETTINGS clause). + */ +struct MergeTreeSettings : public BaseSettings +{ void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config); /// NOTE: will rewrite the AST to add immutable settings. diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.cpp b/src/Storages/RabbitMQ/RabbitMQSettings.cpp index efb73396515..f956c520749 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSettings.cpp @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB { @@ -14,7 +13,7 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -IMPLEMENT_SETTINGS_COLLECTION(RabbitMQSettings, LIST_OF_RABBITMQ_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) void RabbitMQSettings::loadFromQuery(ASTStorage & storage_def) { diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index b0ce82b0420..e18abb91bb9 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -1,15 +1,14 @@ #pragma once -#include +#include + namespace DB { class ASTStorage; - struct RabbitMQSettings : public SettingsCollection - { -#define LIST_OF_RABBITMQ_SETTINGS(M) \ +#define LIST_OF_RABBITMQ_SETTINGS(M) \ M(String, rabbitmq_host_port, "", "A host-port to connect to RabbitMQ server.", 0) \ M(String, rabbitmq_routing_key_list, "5672", "A string of routing keys, separated by dots.", 0) \ M(String, rabbitmq_exchange_name, "clickhouse-exchange", "The exchange name, to which messages are sent.", 0) \ @@ -20,8 +19,11 @@ namespace DB M(UInt64, rabbitmq_num_queues, 1, "The number of queues per consumer.", 0) \ M(Bool, rabbitmq_transactional_channel, false, "Use transactional channel for publishing.", 0) \ - DECLARE_SETTINGS_COLLECTION(LIST_OF_RABBITMQ_SETTINGS) + DECLARE_SETTINGS_TRAITS(RabbitMQSettingsTraits, LIST_OF_RABBITMQ_SETTINGS) + +struct RabbitMQSettings : public BaseSettings +{ void loadFromQuery(ASTStorage & storage_def); - }; +}; } diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.cpp b/src/Storages/System/StorageSystemMergeTreeSettings.cpp index 9d32ec74d51..5e94a2382f7 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.cpp +++ b/src/Storages/System/StorageSystemMergeTreeSettings.cpp @@ -20,13 +20,13 @@ NamesAndTypesList SystemMergeTreeSettings::getNamesAndTypes() void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - for (const auto & setting : context.getMergeTreeSettings()) + for (auto setting : context.getMergeTreeSettings().all()) { - res_columns[0]->insert(setting.getName().toString()); - res_columns[1]->insert(setting.getValueAsString()); - res_columns[2]->insert(setting.isChanged()); - res_columns[3]->insert(setting.getDescription().toString()); - res_columns[4]->insert(setting.getType().toString()); + res_columns[0]->insert(setting.getName()); + res_columns[1]->insert(setting.getValueString()); + res_columns[2]->insert(setting.isValueChanged()); + res_columns[3]->insert(setting.getDescription()); + res_columns[4]->insert(setting.getTypeName()); } } diff --git a/src/Storages/System/StorageSystemSettings.cpp b/src/Storages/System/StorageSystemSettings.cpp index 5670d63ab71..44af7d0588d 100644 --- a/src/Storages/System/StorageSystemSettings.cpp +++ b/src/Storages/System/StorageSystemSettings.cpp @@ -30,13 +30,13 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, const Context { const Settings & settings = context.getSettingsRef(); auto settings_constraints = context.getSettingsConstraints(); - for (const auto & setting : settings) + for (auto setting : settings.all()) { - std::string_view setting_name{setting.getName()}; + const auto & setting_name = setting.getName(); res_columns[0]->insert(setting_name); - res_columns[1]->insert(setting.getValueAsString()); - res_columns[2]->insert(setting.isChanged()); - res_columns[3]->insert(setting.getDescription().toString()); + res_columns[1]->insert(setting.getValueString()); + res_columns[2]->insert(setting.isValueChanged()); + res_columns[3]->insert(setting.getDescription()); Field min, max; bool read_only = false; @@ -45,9 +45,9 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, const Context /// These two columns can accept strings only. if (!min.isNull()) - min = Settings::valueToString(setting_name, min); + min = Settings::valueToStringUtil(setting_name, min); if (!max.isNull()) - max = Settings::valueToString(setting_name, max); + max = Settings::valueToStringUtil(setting_name, max); if (!read_only) { @@ -60,7 +60,7 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, const Context res_columns[4]->insert(min); res_columns[5]->insert(max); res_columns[6]->insert(read_only); - res_columns[7]->insert(setting.getType().toString()); + res_columns[7]->insert(setting.getTypeName()); } } diff --git a/src/Storages/System/StorageSystemSettingsProfileElements.cpp b/src/Storages/System/StorageSystemSettingsProfileElements.cpp index bfbc765cb9b..6d6df4fe114 100644 --- a/src/Storages/System/StorageSystemSettingsProfileElements.cpp +++ b/src/Storages/System/StorageSystemSettingsProfileElements.cpp @@ -141,7 +141,7 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns } else { - String str = Settings::valueToString(setting_name, element.value); + String str = Settings::valueToStringUtil(setting_name, element.value); column_value.insertData(str.data(), str.length()); column_value_null_map.push_back(false); } @@ -153,7 +153,7 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns } else { - String str = Settings::valueToString(setting_name, element.min_value); + String str = Settings::valueToStringUtil(setting_name, element.min_value); column_min.insertData(str.data(), str.length()); column_min_null_map.push_back(false); } @@ -165,7 +165,7 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns } else { - String str = Settings::valueToString(setting_name, element.max_value); + String str = Settings::valueToStringUtil(setting_name, element.max_value); column_max.insertData(str.data(), str.length()); column_max_null_map.push_back(false); } From 442f3de5a82874fd19218a02a55206437bbb41fa Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 29 Jul 2020 19:30:58 +0300 Subject: [PATCH 12/20] Implement custom settings. --- src/Access/SettingsConstraints.cpp | 29 ++-- src/Core/BaseSettings.h | 231 +++++++++++++++++++++++++++-- src/Core/Settings.h | 2 +- 3 files changed, 233 insertions(+), 29 deletions(-) diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 4d39c3c70c3..5da68402f8b 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -150,6 +150,7 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const { const String & setting_name = change.name; + bool cannot_cast; auto cast_value = [&](const Field & x) -> Field { @@ -190,19 +191,23 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh } }; - Field current_value; - if (reaction == THROW_ON_VIOLATION) - current_value = current_settings.get(setting_name); - else if (!current_settings.tryGet(setting_name, current_value)) - return false; + Field current_value, new_value; + if (current_settings.tryGet(setting_name, current_value)) + { + /// Setting isn't checked if value has not changed. + if (change.value == current_value) + return false; - /// Setting isn't checked if value has not changed. - if (change.value == current_value) - return false; - - Field new_value = cast_value(change.value); - if ((new_value == current_value) || cannot_cast) - return false; + new_value = cast_value(change.value); + if ((new_value == current_value) || cannot_cast) + return false; + } + else + { + new_value = cast_value(change.value); + if (cannot_cast) + return false; + } if (!current_settings.allow_ddl && setting_name == "allow_ddl") { diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index d6e751a7ee6..f0edb0ffbe5 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -19,6 +20,8 @@ enum class SettingsWriteFormat DEFAULT = STRINGS_WITH_FLAGS, }; +using SettingFieldCustom = SettingFieldString; + /** Template class to define collections of settings. * Example of usage: @@ -41,6 +44,7 @@ enum class SettingsWriteFormat template class BaseSettings : public Traits_::Data { + using CustomSettingMap = std::unordered_map, SettingFieldCustom>>; public: using Traits = Traits_; @@ -62,9 +66,12 @@ public: /// Resets all the settings to their default values. void resetToDefault(); - static bool has(const std::string_view & name); - static const char * getTypeName(const std::string_view & name); - static const char * getDescription(const std::string_view & name); + bool has(const std::string_view & name) const { return hasBuiltin(name) || hasCustom(name); } + static bool hasBuiltin(const std::string_view & name); + bool hasCustom(const std::string_view & name) const; + + const char * getTypeName(const std::string_view & name) const; + const char * getDescription(const std::string_view & name) const; /// Checks if it's possible to assign a field to a specified value and throws an exception if not. /// This function doesn't change the fields, it performs check only. @@ -92,6 +99,7 @@ public: bool isValueChanged() const; const char * getTypeName() const; const char * getDescription() const; + bool isCustom() const; bool operator==(const SettingFieldRef & other) const { return (getName() == other.getName()) && (getValue() == other.getValue()); } bool operator!=(const SettingFieldRef & other) const { return !(*this == other); } @@ -99,10 +107,12 @@ public: private: friend class BaseSettings; SettingFieldRef(const typename Traits::Data & data_, const typename Traits::Accessor & accessor_, size_t index_) : data(&data_), accessor(&accessor_), index(index_) {} + SettingFieldRef(const CustomSettingMap::mapped_type & custom_setting_); const typename Traits::Data * data = nullptr; const typename Traits::Accessor * accessor = nullptr; size_t index = 0; + std::conditional_t custom_setting = {}; }; enum SkipFlags @@ -110,7 +120,9 @@ public: SKIP_NONE = 0, SKIP_CHANGED = 0x01, SKIP_UNCHANGED = 0x02, - SKIP_ALL = SKIP_CHANGED | SKIP_UNCHANGED, + SKIP_BUILTIN = 0x04, + SKIP_CUSTOM = 0x08, + SKIP_ALL = SKIP_CHANGED | SKIP_UNCHANGED | SKIP_CUSTOM | SKIP_BUILTIN, }; class Iterator @@ -131,6 +143,7 @@ public: const BaseSettings * settings = nullptr; const typename Traits::Accessor * accessor = nullptr; size_t index; + std::conditional_t custom_settings_iterator; SkipFlags skip_flags; }; @@ -150,9 +163,18 @@ public: Range all(SkipFlags skip_flags = SKIP_NONE) const { return Range{*this, skip_flags}; } Range allChanged() const { return all(SKIP_UNCHANGED); } Range allUnchanged() const { return all(SKIP_CHANGED); } + Range allBuiltin() const { return all(SKIP_CUSTOM); } + Range allCustom() const { return all(SKIP_BUILTIN); } Iterator begin() const { return allChanged().begin(); } Iterator end() const { return allChanged().end(); } + +private: + SettingFieldCustom & getCustomSetting(const std::string_view & name); + const SettingFieldCustom & getCustomSetting(const std::string_view & name) const; + const SettingFieldCustom * tryGetCustomSetting(const std::string_view & name) const; + + std::conditional_t custom_settings_map; }; struct BaseSettingsHelpers @@ -166,6 +188,7 @@ struct BaseSettingsHelpers enum Flags : UInt64 { IMPORTANT = 0x01, + CUSTOM = 0x02, }; static void writeFlags(Flags flags, WriteBuffer & out); static Flags readFlags(ReadBuffer & in); @@ -178,7 +201,7 @@ void BaseSettings::set(const std::string_view & name, const Field & val if (size_t index = accessor.find(name); index != static_cast(-1)) accessor.setValue(*this, index, value); else - BaseSettingsHelpers::throwSettingNotFound(name); + getCustomSetting(name) = value; } template @@ -188,7 +211,7 @@ Field BaseSettings::get(const std::string_view & name) const if (size_t index = accessor.find(name); index != static_cast(-1)) return accessor.getValue(*this, index); else - BaseSettingsHelpers::throwSettingNotFound(name); + return static_cast(getCustomSetting(name)); } template @@ -198,7 +221,7 @@ void BaseSettings::setString(const std::string_view & name, const Strin if (size_t index = accessor.find(name); index != static_cast(-1)) accessor.setValueString(*this, index, value); else - BaseSettingsHelpers::throwSettingNotFound(name); + getCustomSetting(name).parseFromString(value); } template @@ -208,7 +231,7 @@ String BaseSettings::getString(const std::string_view & name) const if (size_t index = accessor.find(name); index != static_cast(-1)) return accessor.getValueString(*this, index); else - BaseSettingsHelpers::throwSettingNotFound(name); + return getCustomSetting(name).toString(); } template @@ -220,6 +243,11 @@ bool BaseSettings::tryGet(const std::string_view & name, Field & value) value = accessor.getValue(*this, index); return true; } + if (const auto * custom_setting = tryGetCustomSetting(name)) + { + value = static_cast(*custom_setting); + return true; + } return false; } @@ -232,6 +260,11 @@ bool BaseSettings::tryGetString(const std::string_view & name, String & value = accessor.getValueString(*this, index); return true; } + if (const auto * custom_setting = tryGetCustomSetting(name)) + { + value = custom_setting->toString(); + return true; + } return false; } @@ -241,7 +274,7 @@ bool BaseSettings::isChanged(const std::string_view & name) const const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) return accessor.isValueChanged(*this, index); - return false; + return tryGetCustomSetting(name) != nullptr; } template @@ -282,32 +315,44 @@ void BaseSettings::resetToDefault() if (accessor.isValueChanged(*this, i)) accessor.resetValueToDefault(*this, i); } + + if constexpr (Traits::allow_custom_settings) + custom_settings_map.clear(); } template -bool BaseSettings::has(const std::string_view & name) +bool BaseSettings::hasBuiltin(const std::string_view & name) { const auto & accessor = Traits::Accessor::instance(); return (accessor.find(name) != static_cast(-1)); } +template +bool BaseSettings::hasCustom(const std::string_view & name) const +{ + return tryGetCustomSetting(name); +} template -const char * BaseSettings::getTypeName(const std::string_view & name) +const char * BaseSettings::getTypeName(const std::string_view & name) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) return accessor.getTypeName(index); + else if (tryGetCustomSetting(name)) + return "String"; else BaseSettingsHelpers::throwSettingNotFound(name); } template -const char * BaseSettings::getDescription(const std::string_view & name) +const char * BaseSettings::getDescription(const std::string_view & name) const { const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) return accessor.getDescription(index); + else if (tryGetCustomSetting(name)) + return "Custom"; else BaseSettingsHelpers::throwSettingNotFound(name); } @@ -330,6 +375,11 @@ Field BaseSettings::castValueUtil(const std::string_view & name, const const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) return accessor.castValueUtil(index, value); + if constexpr (Traits::allow_custom_settings) + { + value.safeGet(); + return value; + } else BaseSettingsHelpers::throwSettingNotFound(name); } @@ -340,6 +390,8 @@ String BaseSettings::valueToStringUtil(const std::string_view & name, c const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) return accessor.valueToStringUtil(index, value); + if constexpr (Traits::allow_custom_settings) + return value.safeGet(); else BaseSettingsHelpers::throwSettingNotFound(name); } @@ -350,6 +402,8 @@ Field BaseSettings::stringToValueUtil(const std::string_view & name, co const auto & accessor = Traits::Accessor::instance(); if (size_t index = accessor.find(name); index != static_cast(-1)) return accessor.stringToValueUtil(index, str); + if constexpr (Traits::allow_custom_settings) + return Field{str}; else BaseSettingsHelpers::throwSettingNotFound(name); } @@ -361,15 +415,18 @@ void BaseSettings::write(WriteBuffer & out, SettingsWriteFormat format) for (auto field : *this) { - bool is_important = accessor.isImportant(field.index); + bool is_custom = field.isCustom(); + bool is_important = !is_custom && accessor.isImportant(field.index); BaseSettingsHelpers::writeString(field.getName(), out); - if ((format >= SettingsWriteFormat::STRINGS_WITH_FLAGS)) + if ((format >= SettingsWriteFormat::STRINGS_WITH_FLAGS) || is_custom) { using Flags = BaseSettingsHelpers::Flags; Flags flags{0}; - if (is_important) + if (is_custom) + flags = static_cast(flags | Flags::CUSTOM); + else if (is_important) flags = static_cast(flags | Flags::IMPORTANT); BaseSettingsHelpers::writeFlags(flags, out); @@ -400,14 +457,25 @@ void BaseSettings::read(ReadBuffer & in, SettingsWriteFormat format) if (format >= SettingsWriteFormat::STRINGS_WITH_FLAGS) flags = BaseSettingsHelpers::readFlags(in); bool is_important = (flags & Flags::IMPORTANT); + bool is_custom = (flags & Flags::CUSTOM); if (index != static_cast(-1)) { - if (format >= SettingsWriteFormat::STRINGS_WITH_FLAGS) + if (is_custom) + { + SettingFieldCustom temp; + temp.parseFromString(BaseSettingsHelpers::readString(in)); + accessor.setValue(*this, index, static_cast(temp)); + } + else if (format >= SettingsWriteFormat::STRINGS_WITH_FLAGS) accessor.setValueString(*this, index, BaseSettingsHelpers::readString(in)); else accessor.readBinary(*this, index, in); } + else if (is_custom && Traits::allow_custom_settings) + { + getCustomSetting(name).parseFromString(BaseSettingsHelpers::readString(in)); + } else if (is_important) { BaseSettingsHelpers::throwSettingNotFound(name); @@ -452,15 +520,78 @@ bool operator!=(const BaseSettings & left, const BaseSettings return !(left == right); } +template +SettingFieldCustom & BaseSettings::getCustomSetting(const std::string_view & name) +{ + if constexpr (Traits::allow_custom_settings) + { + auto it = custom_settings_map.find(name); + if (it == custom_settings_map.end()) + { + auto new_name = std::make_shared(name); + it = custom_settings_map.emplace(*new_name, std::make_pair(new_name, SettingFieldCustom{})).first; + } + return it->second.second; + } + BaseSettingsHelpers::throwSettingNotFound(name); +} + +template +const SettingFieldCustom & BaseSettings::getCustomSetting(const std::string_view & name) const +{ + if constexpr (Traits::allow_custom_settings) + { + auto it = custom_settings_map.find(name); + if (it != custom_settings_map.end()) + return it->second.second; + } + BaseSettingsHelpers::throwSettingNotFound(name); +} + +template +const SettingFieldCustom * BaseSettings::tryGetCustomSetting(const std::string_view & name) const +{ + if constexpr (Traits::allow_custom_settings) + { + auto it = custom_settings_map.find(name); + if (it != custom_settings_map.end()) + return &it->second.second; + } + return nullptr; +} + template BaseSettings::Iterator::Iterator(const BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_) : settings(&settings_), accessor(&accessor_), skip_flags(skip_flags_) { if (skip_flags == SKIP_ALL) + { + index = accessor->size(); + if constexpr (Traits::allow_custom_settings) + custom_settings_iterator = settings->custom_settings_map.end(); + return; + } + + if (skip_flags & SKIP_CHANGED) + { + skip_flags = static_cast(skip_flags | SKIP_CUSTOM); + if (skip_flags & SKIP_UNCHANGED) + skip_flags = static_cast(skip_flags | SKIP_BUILTIN); + } + + if (skip_flags & SKIP_BUILTIN) index = accessor->size(); else index = 0; + if constexpr (Traits::allow_custom_settings) + { + if (skip_flags & SKIP_CUSTOM) + custom_settings_iterator = settings->custom_settings_map.end(); + else + custom_settings_iterator = settings->custom_settings_map.begin(); + } + doSkip(); } @@ -469,6 +600,11 @@ typename BaseSettings::Iterator & BaseSettings::Iterator::oper { if (index != accessor->size()) ++index; + else + { + if constexpr (Traits::allow_custom_settings) + ++custom_settings_iterator; + } doSkip(); return *this; } @@ -484,6 +620,11 @@ typename BaseSettings::Iterator BaseSettings::Iterator::operat template typename BaseSettings::SettingFieldRef BaseSettings::Iterator::operator*() const { + if constexpr (Traits::allow_custom_settings) + { + if (index == accessor->size()) + return {custom_settings_iterator->second}; + } return {*settings, *accessor, index}; } @@ -505,46 +646,103 @@ void BaseSettings::Iterator::doSkip() template bool BaseSettings::Iterator::operator ==(const typename BaseSettings::Iterator & other) const { + if constexpr (Traits_::allow_custom_settings) + { + if (custom_settings_iterator != other.custom_settings_iterator) + return false; + } return ((index == other.index) && (settings == other.settings)); } +template +BaseSettings::SettingFieldRef::SettingFieldRef(const CustomSettingMap::mapped_type & custom_setting_) +{ + if constexpr (Traits_::allow_custom_settings) + custom_setting = &custom_setting_; +} + template const String & BaseSettings::SettingFieldRef::getName() const { + if constexpr (Traits::allow_custom_settings) + { + if (custom_setting) + return *custom_setting->first; + } return accessor->getName(index); } template Field BaseSettings::SettingFieldRef::getValue() const { + if constexpr (Traits::allow_custom_settings) + { + if (custom_setting) + return static_cast(custom_setting->second); + } return accessor->getValue(*data, index); } template String BaseSettings::SettingFieldRef::getValueString() const { + if constexpr (Traits::allow_custom_settings) + { + if (custom_setting) + return custom_setting->second.toString(); + } return accessor->getValueString(*data, index); } template bool BaseSettings::SettingFieldRef::isValueChanged() const { + if constexpr (Traits::allow_custom_settings) + { + if (custom_setting) + return true; + } return accessor->isValueChanged(*data, index); } template const char * BaseSettings::SettingFieldRef::getTypeName() const { + if constexpr (Traits::allow_custom_settings) + { + if (custom_setting) + return "String"; + } return accessor->getTypeName(index); } template const char * BaseSettings::SettingFieldRef::getDescription() const { + if constexpr (Traits::allow_custom_settings) + { + if (custom_setting) + return "Custom"; + } return accessor->getDescription(index); } +template +bool BaseSettings::SettingFieldRef::isCustom() const +{ + if constexpr (Traits::allow_custom_settings) + return custom_setting != nullptr; + else + return false; +} + #define DECLARE_SETTINGS_TRAITS(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO) \ + DECLARE_SETTINGS_TRAITS_COMMON(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO, 0) + +#define DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO) \ + DECLARE_SETTINGS_TRAITS_COMMON(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO, 1) + +#define DECLARE_SETTINGS_TRAITS_COMMON(SETTINGS_TRAITS_NAME, LIST_OF_SETTINGS_MACRO, ALLOW_CUSTOM_SETTINGS) \ struct SETTINGS_TRAITS_NAME \ { \ struct Data \ @@ -597,6 +795,7 @@ const char * BaseSettings::SettingFieldRef::getDescription() const std::vector field_infos; \ std::unordered_map name_to_index_map; \ }; \ + static constexpr bool allow_custom_settings = ALLOW_CUSTOM_SETTINGS; \ }; #define DECLARE_SETTINGS_TRAITS_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6bcd7f8de96..07fdeaabf22 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -463,7 +463,7 @@ class IColumn; COMMON_SETTINGS(M) \ FORMAT_FACTORY_SETTINGS(M) -DECLARE_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS) +DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(SettingsTraits, LIST_OF_SETTINGS) /** Settings of query execution. From 80a3caef1ca737c3a40ab14561ceb612464c223a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 28 Jul 2020 15:37:44 +0300 Subject: [PATCH 13/20] Allow using any types for custom fields. --- src/Common/ErrorCodes.cpp | 1 + src/Common/FieldVisitors.cpp | 4 +- src/Common/StringUtils/StringUtils.h | 18 + src/Core/BaseSettings.h | 13 +- src/Core/Field.cpp | 818 +++++++++++++++++---------- src/Core/Field.h | 7 +- src/Core/SettingsFields.cpp | 23 + src/Core/SettingsFields.h | 19 + src/IO/ReadHelpers.h | 7 + 9 files changed, 585 insertions(+), 325 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index a4abaf91231..d63b9f9ccab 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -500,6 +500,7 @@ namespace ErrorCodes extern const int INCONSISTENT_RESERVATIONS = 533; extern const int NO_RESERVATIONS_PROVIDED = 534; extern const int UNKNOWN_RAID_TYPE = 535; + extern const int CANNOT_RESTORE_FROM_FIELD_DUMP = 536; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/src/Common/FieldVisitors.cpp b/src/Common/FieldVisitors.cpp index 4ac87dfdd24..a353b6ab27f 100644 --- a/src/Common/FieldVisitors.cpp +++ b/src/Common/FieldVisitors.cpp @@ -40,7 +40,6 @@ static inline void writeQuoted(const DecimalField & x, WriteBuffer & buf) writeChar('\'', buf); } - String FieldVisitorDump::operator() (const Null &) const { return "NULL"; } String FieldVisitorDump::operator() (const UInt64 & x) const { return formatQuotedWithPrefix(x, "UInt64_"); } String FieldVisitorDump::operator() (const Int64 & x) const { return formatQuotedWithPrefix(x, "Int64_"); } @@ -93,8 +92,11 @@ String FieldVisitorDump::operator() (const Tuple & x) const String FieldVisitorDump::operator() (const AggregateFunctionStateData & x) const { WriteBufferFromOwnString wb; + wb << "AggregateFunctionState_("; writeQuoted(x.name, wb); + wb << ", "; writeQuoted(x.data, wb); + wb << ')'; return wb.str(); } diff --git a/src/Common/StringUtils/StringUtils.h b/src/Common/StringUtils/StringUtils.h index bac33bdb508..50573694b7a 100644 --- a/src/Common/StringUtils/StringUtils.h +++ b/src/Common/StringUtils/StringUtils.h @@ -235,3 +235,21 @@ std::string trim(const std::string & str, F && predicate) return str.substr(cut_front, size - cut_front - cut_back); } + +inline void trimLeft(std::string_view & str, char c = ' ') +{ + while (str.starts_with(c)) + str.remove_prefix(1); +} + +inline void trimRight(std::string_view & str, char c = ' ') +{ + while (str.ends_with(c)) + str.remove_suffix(1); +} + +inline void trim(std::string_view & str, char c = ' ') +{ + trimLeft(str, c); + trimRight(str, c); +} diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index f0edb0ffbe5..75a76340a4f 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -20,8 +20,6 @@ enum class SettingsWriteFormat DEFAULT = STRINGS_WITH_FLAGS, }; -using SettingFieldCustom = SettingFieldString; - /** Template class to define collections of settings. * Example of usage: @@ -340,7 +338,7 @@ const char * BaseSettings::getTypeName(const std::string_view & name) c if (size_t index = accessor.find(name); index != static_cast(-1)) return accessor.getTypeName(index); else if (tryGetCustomSetting(name)) - return "String"; + return "Custom"; else BaseSettingsHelpers::throwSettingNotFound(name); } @@ -376,10 +374,7 @@ Field BaseSettings::castValueUtil(const std::string_view & name, const if (size_t index = accessor.find(name); index != static_cast(-1)) return accessor.castValueUtil(index, value); if constexpr (Traits::allow_custom_settings) - { - value.safeGet(); return value; - } else BaseSettingsHelpers::throwSettingNotFound(name); } @@ -391,7 +386,7 @@ String BaseSettings::valueToStringUtil(const std::string_view & name, c if (size_t index = accessor.find(name); index != static_cast(-1)) return accessor.valueToStringUtil(index, value); if constexpr (Traits::allow_custom_settings) - return value.safeGet(); + return value.dump(); else BaseSettingsHelpers::throwSettingNotFound(name); } @@ -403,7 +398,7 @@ Field BaseSettings::stringToValueUtil(const std::string_view & name, co if (size_t index = accessor.find(name); index != static_cast(-1)) return accessor.stringToValueUtil(index, str); if constexpr (Traits::allow_custom_settings) - return Field{str}; + return Field::restoreFromDump(str); else BaseSettingsHelpers::throwSettingNotFound(name); } @@ -711,7 +706,7 @@ const char * BaseSettings::SettingFieldRef::getTypeName() const if constexpr (Traits::allow_custom_settings) { if (custom_setting) - return "String"; + return "Custom"; } return accessor->getTypeName(index); } diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index fe835c21b8d..e841ddd60be 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -2,7 +2,8 @@ #include #include #include - +#include +#include #include #include #include @@ -10,326 +11,515 @@ namespace DB { - void readBinary(Array & x, ReadBuffer & buf) +namespace ErrorCodes +{ + extern const int CANNOT_RESTORE_FROM_FIELD_DUMP; + extern const int DECIMAL_OVERFLOW; +} + +void readBinary(Array & x, ReadBuffer & buf) +{ + size_t size; + UInt8 type; + DB::readBinary(type, buf); + DB::readBinary(size, buf); + + for (size_t index = 0; index < size; ++index) + { + switch (type) + { + case Field::Types::Null: + { + x.push_back(DB::Field()); + break; + } + case Field::Types::UInt64: + { + UInt64 value; + DB::readVarUInt(value, buf); + x.push_back(value); + break; + } + case Field::Types::UInt128: + { + UInt128 value; + DB::readBinary(value, buf); + x.push_back(value); + break; + } + case Field::Types::Int64: + { + Int64 value; + DB::readVarInt(value, buf); + x.push_back(value); + break; + } + case Field::Types::Float64: + { + Float64 value; + DB::readFloatBinary(value, buf); + x.push_back(value); + break; + } + case Field::Types::String: + { + std::string value; + DB::readStringBinary(value, buf); + x.push_back(value); + break; + } + case Field::Types::Array: + { + Array value; + DB::readBinary(value, buf); + x.push_back(value); + break; + } + case Field::Types::Tuple: + { + Tuple value; + DB::readBinary(value, buf); + x.push_back(value); + break; + } + case Field::Types::AggregateFunctionState: + { + AggregateFunctionStateData value; + DB::readStringBinary(value.name, buf); + DB::readStringBinary(value.data, buf); + x.push_back(value); + break; + } + } + } +} + +void writeBinary(const Array & x, WriteBuffer & buf) +{ + UInt8 type = Field::Types::Null; + size_t size = x.size(); + if (size) + type = x.front().getType(); + DB::writeBinary(type, buf); + DB::writeBinary(size, buf); + + for (const auto & elem : x) + { + switch (type) + { + case Field::Types::Null: break; + case Field::Types::UInt64: + { + DB::writeVarUInt(get(elem), buf); + break; + } + case Field::Types::UInt128: + { + DB::writeBinary(get(elem), buf); + break; + } + case Field::Types::Int64: + { + DB::writeVarInt(get(elem), buf); + break; + } + case Field::Types::Float64: + { + DB::writeFloatBinary(get(elem), buf); + break; + } + case Field::Types::String: + { + DB::writeStringBinary(get(elem), buf); + break; + } + case Field::Types::Array: + { + DB::writeBinary(get(elem), buf); + break; + } + case Field::Types::Tuple: + { + DB::writeBinary(get(elem), buf); + break; + } + case Field::Types::AggregateFunctionState: + { + DB::writeStringBinary(elem.get().name, buf); + DB::writeStringBinary(elem.get().data, buf); + break; + } + } + } +} + +void writeText(const Array & x, WriteBuffer & buf) +{ + DB::String res = applyVisitor(DB::FieldVisitorToString(), DB::Field(x)); + buf.write(res.data(), res.size()); +} + +void readBinary(Tuple & x, ReadBuffer & buf) +{ + size_t size; + DB::readBinary(size, buf); + + for (size_t index = 0; index < size; ++index) { - size_t size; UInt8 type; DB::readBinary(type, buf); - DB::readBinary(size, buf); - for (size_t index = 0; index < size; ++index) + switch (type) { - switch (type) + case Field::Types::Null: { - case Field::Types::Null: - { - x.push_back(DB::Field()); - break; - } - case Field::Types::UInt64: - { - UInt64 value; - DB::readVarUInt(value, buf); - x.push_back(value); - break; - } - case Field::Types::UInt128: - { - UInt128 value; - DB::readBinary(value, buf); - x.push_back(value); - break; - } - case Field::Types::Int64: - { - Int64 value; - DB::readVarInt(value, buf); - x.push_back(value); - break; - } - case Field::Types::Float64: - { - Float64 value; - DB::readFloatBinary(value, buf); - x.push_back(value); - break; - } - case Field::Types::String: - { - std::string value; - DB::readStringBinary(value, buf); - x.push_back(value); - break; - } - case Field::Types::Array: - { - Array value; - DB::readBinary(value, buf); - x.push_back(value); - break; - } - case Field::Types::Tuple: - { - Tuple value; - DB::readBinary(value, buf); - x.push_back(value); - break; - } - case Field::Types::AggregateFunctionState: - { - AggregateFunctionStateData value; - DB::readStringBinary(value.name, buf); - DB::readStringBinary(value.data, buf); - x.push_back(value); - break; - } + x.push_back(DB::Field()); + break; + } + case Field::Types::UInt64: + { + UInt64 value; + DB::readVarUInt(value, buf); + x.push_back(value); + break; + } + case Field::Types::UInt128: + { + UInt128 value; + DB::readBinary(value, buf); + x.push_back(value); + break; + } + case Field::Types::Int64: + { + Int64 value; + DB::readVarInt(value, buf); + x.push_back(value); + break; + } + case Field::Types::Float64: + { + Float64 value; + DB::readFloatBinary(value, buf); + x.push_back(value); + break; + } + case Field::Types::String: + { + std::string value; + DB::readStringBinary(value, buf); + x.push_back(value); + break; + } + case Field::Types::Array: + { + Array value; + DB::readBinary(value, buf); + x.push_back(value); + break; + } + case Field::Types::Tuple: + { + Tuple value; + DB::readBinary(value, buf); + x.push_back(value); + break; + } + case Field::Types::AggregateFunctionState: + { + AggregateFunctionStateData value; + DB::readStringBinary(value.name, buf); + DB::readStringBinary(value.data, buf); + x.push_back(value); + break; } } } - - void writeBinary(const Array & x, WriteBuffer & buf) - { - UInt8 type = Field::Types::Null; - size_t size = x.size(); - if (size) - type = x.front().getType(); - DB::writeBinary(type, buf); - DB::writeBinary(size, buf); - - for (const auto & elem : x) - { - switch (type) - { - case Field::Types::Null: break; - case Field::Types::UInt64: - { - DB::writeVarUInt(get(elem), buf); - break; - } - case Field::Types::UInt128: - { - DB::writeBinary(get(elem), buf); - break; - } - case Field::Types::Int64: - { - DB::writeVarInt(get(elem), buf); - break; - } - case Field::Types::Float64: - { - DB::writeFloatBinary(get(elem), buf); - break; - } - case Field::Types::String: - { - DB::writeStringBinary(get(elem), buf); - break; - } - case Field::Types::Array: - { - DB::writeBinary(get(elem), buf); - break; - } - case Field::Types::Tuple: - { - DB::writeBinary(get(elem), buf); - break; - } - case Field::Types::AggregateFunctionState: - { - DB::writeStringBinary(elem.get().name, buf); - DB::writeStringBinary(elem.get().data, buf); - break; - } - } - } - } - - void writeText(const Array & x, WriteBuffer & buf) - { - DB::String res = applyVisitor(DB::FieldVisitorToString(), DB::Field(x)); - buf.write(res.data(), res.size()); - } - - void readBinary(Tuple & x, ReadBuffer & buf) - { - size_t size; - DB::readBinary(size, buf); - - for (size_t index = 0; index < size; ++index) - { - UInt8 type; - DB::readBinary(type, buf); - - switch (type) - { - case Field::Types::Null: - { - x.push_back(DB::Field()); - break; - } - case Field::Types::UInt64: - { - UInt64 value; - DB::readVarUInt(value, buf); - x.push_back(value); - break; - } - case Field::Types::UInt128: - { - UInt128 value; - DB::readBinary(value, buf); - x.push_back(value); - break; - } - case Field::Types::Int64: - { - Int64 value; - DB::readVarInt(value, buf); - x.push_back(value); - break; - } - case Field::Types::Float64: - { - Float64 value; - DB::readFloatBinary(value, buf); - x.push_back(value); - break; - } - case Field::Types::String: - { - std::string value; - DB::readStringBinary(value, buf); - x.push_back(value); - break; - } - case Field::Types::Array: - { - Array value; - DB::readBinary(value, buf); - x.push_back(value); - break; - } - case Field::Types::Tuple: - { - Tuple value; - DB::readBinary(value, buf); - x.push_back(value); - break; - } - case Field::Types::AggregateFunctionState: - { - AggregateFunctionStateData value; - DB::readStringBinary(value.name, buf); - DB::readStringBinary(value.data, buf); - x.push_back(value); - break; - } - } - } - } - - void writeBinary(const Tuple & x, WriteBuffer & buf) - { - const size_t size = x.size(); - DB::writeBinary(size, buf); - - for (const auto & elem : x) - { - const UInt8 type = elem.getType(); - DB::writeBinary(type, buf); - - switch (type) - { - case Field::Types::Null: break; - case Field::Types::UInt64: - { - DB::writeVarUInt(get(elem), buf); - break; - } - case Field::Types::UInt128: - { - DB::writeBinary(get(elem), buf); - break; - } - case Field::Types::Int64: - { - DB::writeVarInt(get(elem), buf); - break; - } - case Field::Types::Float64: - { - DB::writeFloatBinary(get(elem), buf); - break; - } - case Field::Types::String: - { - DB::writeStringBinary(get(elem), buf); - break; - } - case Field::Types::Array: - { - DB::writeBinary(get(elem), buf); - break; - } - case Field::Types::Tuple: - { - DB::writeBinary(get(elem), buf); - break; - } - case Field::Types::AggregateFunctionState: - { - DB::writeStringBinary(elem.get().name, buf); - DB::writeStringBinary(elem.get().data, buf); - break; - } - } - } - } - - void writeText(const Tuple & x, WriteBuffer & buf) - { - writeFieldText(DB::Field(x), buf); - } - - void writeFieldText(const Field & x, WriteBuffer & buf) - { - DB::String res = Field::dispatch(DB::FieldVisitorToString(), x); - buf.write(res.data(), res.size()); - } - - - template - static bool decEqual(T x, T y, UInt32 x_scale, UInt32 y_scale) - { - using Comparator = DecimalComparison; - return Comparator::compare(x, y, x_scale, y_scale); - } - - template - static bool decLess(T x, T y, UInt32 x_scale, UInt32 y_scale) - { - using Comparator = DecimalComparison; - return Comparator::compare(x, y, x_scale, y_scale); - } - - template - static bool decLessOrEqual(T x, T y, UInt32 x_scale, UInt32 y_scale) - { - using Comparator = DecimalComparison; - return Comparator::compare(x, y, x_scale, y_scale); - } - - template <> bool decimalEqual(Decimal32 x, Decimal32 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); } - template <> bool decimalLess(Decimal32 x, Decimal32 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); } - template <> bool decimalLessOrEqual(Decimal32 x, Decimal32 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); } - - template <> bool decimalEqual(Decimal64 x, Decimal64 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); } - template <> bool decimalLess(Decimal64 x, Decimal64 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); } - template <> bool decimalLessOrEqual(Decimal64 x, Decimal64 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); } - - template <> bool decimalEqual(Decimal128 x, Decimal128 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); } - template <> bool decimalLess(Decimal128 x, Decimal128 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); } - template <> bool decimalLessOrEqual(Decimal128 x, Decimal128 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); } +} + +void writeBinary(const Tuple & x, WriteBuffer & buf) +{ + const size_t size = x.size(); + DB::writeBinary(size, buf); + + for (const auto & elem : x) + { + const UInt8 type = elem.getType(); + DB::writeBinary(type, buf); + + switch (type) + { + case Field::Types::Null: break; + case Field::Types::UInt64: + { + DB::writeVarUInt(get(elem), buf); + break; + } + case Field::Types::UInt128: + { + DB::writeBinary(get(elem), buf); + break; + } + case Field::Types::Int64: + { + DB::writeVarInt(get(elem), buf); + break; + } + case Field::Types::Float64: + { + DB::writeFloatBinary(get(elem), buf); + break; + } + case Field::Types::String: + { + DB::writeStringBinary(get(elem), buf); + break; + } + case Field::Types::Array: + { + DB::writeBinary(get(elem), buf); + break; + } + case Field::Types::Tuple: + { + DB::writeBinary(get(elem), buf); + break; + } + case Field::Types::AggregateFunctionState: + { + DB::writeStringBinary(elem.get().name, buf); + DB::writeStringBinary(elem.get().data, buf); + break; + } + } + } +} + +void writeText(const Tuple & x, WriteBuffer & buf) +{ + writeFieldText(DB::Field(x), buf); +} + +template +void readQuoted(DecimalField & x, ReadBuffer & buf) +{ + assertChar('\'', buf); + T value; + UInt32 scale; + int32_t exponent; + uint32_t max_digits = static_cast(-1); + readDigits(buf, value, max_digits, exponent, true); + if (exponent > 0) + { + scale = 0; + if (common::mulOverflow(value.value, T::getScaleMultiplier(exponent), value.value)) + throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + } + else + scale = -exponent; + assertChar('\'', buf); + x = DecimalField{value, scale}; +} + +template void readQuoted(DecimalField & x, ReadBuffer & buf); +template void readQuoted(DecimalField & x, ReadBuffer & buf); +template void readQuoted(DecimalField & x, ReadBuffer & buf); + + +void writeFieldText(const Field & x, WriteBuffer & buf) +{ + DB::String res = Field::dispatch(DB::FieldVisitorToString(), x); + buf.write(res.data(), res.size()); +} + + +String Field::dump() const +{ + return applyVisitor(DB::FieldVisitorDump(), *this); +} + +Field Field::restoreFromDump(const std::string_view & dump_) +{ + auto show_error = [&dump_] + { + throw Exception("Couldn't restore Field from dump: " + String{dump_}, ErrorCodes::CANNOT_RESTORE_FROM_FIELD_DUMP); + }; + + std::string_view dump = dump_; + trim(dump); + + if (dump == "NULL") + return {}; + + std::string_view prefix = std::string_view{"Int64_"}; + if (dump.starts_with(prefix)) + { + Int64 value = parseFromString(dump.substr(prefix.length())); + return value; + } + + prefix = std::string_view{"UInt64_"}; + if (dump.starts_with(prefix)) + { + UInt64 value = parseFromString(dump.substr(prefix.length())); + return value; + } + + prefix = std::string_view{"Float64_"}; + if (dump.starts_with(prefix)) + { + Float64 value = parseFromString(dump.substr(prefix.length())); + return value; + } + + prefix = std::string_view{"Decimal32_"}; + if (dump_.starts_with(prefix)) + { + DecimalField decimal; + ReadBufferFromString buf{dump.substr(prefix.length())}; + readQuoted(decimal, buf); + return decimal; + } + + prefix = std::string_view{"Decimal64_"}; + if (dump_.starts_with(prefix)) + { + DecimalField decimal; + ReadBufferFromString buf{dump.substr(prefix.length())}; + readQuoted(decimal, buf); + return decimal; + } + + prefix = std::string_view{"Decimal128_"}; + if (dump_.starts_with(prefix)) + { + DecimalField decimal; + ReadBufferFromString buf{dump.substr(prefix.length())}; + readQuoted(decimal, buf); + return decimal; + } + + prefix = std::string_view{"UUID_"}; + if (dump.starts_with(prefix)) + { + UUID uuid; + ReadBufferFromString buf{dump.substr(prefix.length())}; + readQuoted(uuid, buf); + return uuid; + } + + if (dump.starts_with("\'")) + { + String str; + ReadBufferFromString buf{dump}; + readQuoted(str, buf); + return str; + } + + prefix = std::string_view{"Array_["}; + if (dump.starts_with(prefix)) + { + std::string_view tail = dump.substr(prefix.length()); + trimLeft(tail); + Array array; + while (tail != "]") + { + size_t separator = tail.find_first_of(",]"); + if (separator == std::string_view::npos) + show_error(); + bool comma = (tail[separator] == ','); + std::string_view element = tail.substr(0, separator); + tail.remove_prefix(separator); + if (comma) + tail.remove_prefix(1); + trimLeft(tail); + if (!comma && tail != "]") + show_error(); + array.push_back(Field::restoreFromDump(element)); + } + return array; + } + + prefix = std::string_view{"Tuple_("}; + if (dump.starts_with(prefix)) + { + std::string_view tail = dump.substr(prefix.length()); + trimLeft(tail); + Tuple tuple; + while (tail != ")") + { + size_t separator = tail.find_first_of(",)"); + if (separator == std::string_view::npos) + show_error(); + bool comma = (tail[separator] == ','); + std::string_view element = tail.substr(0, separator); + tail.remove_prefix(separator); + if (comma) + tail.remove_prefix(1); + trimLeft(tail); + if (!comma && tail != ")") + show_error(); + tuple.push_back(Field::restoreFromDump(element)); + } + return tuple; + } + + prefix = std::string_view{"AggregateFunctionState_("}; + if (dump.starts_with(prefix)) + { + std::string_view after_prefix = dump.substr(prefix.length()); + size_t comma = after_prefix.find(','); + size_t end = after_prefix.find(')', comma + 1); + if ((comma == std::string_view::npos) || (end != after_prefix.length() - 1)) + show_error(); + std::string_view name_view = after_prefix.substr(0, comma); + std::string_view data_view = after_prefix.substr(comma + 1, end - comma - 1); + trim(name_view); + trim(data_view); + ReadBufferFromString name_buf{name_view}; + ReadBufferFromString data_buf{data_view}; + AggregateFunctionStateData res; + readQuotedString(res.name, name_buf); + readQuotedString(res.data, data_buf); + return res; + } + + show_error(); + __builtin_unreachable(); +} + + +template +static bool decEqual(T x, T y, UInt32 x_scale, UInt32 y_scale) +{ + using Comparator = DecimalComparison; + return Comparator::compare(x, y, x_scale, y_scale); +} + +template +static bool decLess(T x, T y, UInt32 x_scale, UInt32 y_scale) +{ + using Comparator = DecimalComparison; + return Comparator::compare(x, y, x_scale, y_scale); +} + +template +static bool decLessOrEqual(T x, T y, UInt32 x_scale, UInt32 y_scale) +{ + using Comparator = DecimalComparison; + return Comparator::compare(x, y, x_scale, y_scale); +} + +template <> bool decimalEqual(Decimal32 x, Decimal32 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); } +template <> bool decimalLess(Decimal32 x, Decimal32 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); } +template <> bool decimalLessOrEqual(Decimal32 x, Decimal32 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); } + +template <> bool decimalEqual(Decimal64 x, Decimal64 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); } +template <> bool decimalLess(Decimal64 x, Decimal64 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); } +template <> bool decimalLessOrEqual(Decimal64 x, Decimal64 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); } + +template <> bool decimalEqual(Decimal128 x, Decimal128 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); } +template <> bool decimalLess(Decimal128 x, Decimal128 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); } +template <> bool decimalLessOrEqual(Decimal128 x, Decimal128 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); } + } diff --git a/src/Core/Field.h b/src/Core/Field.h index 63054922fef..1b19426a71e 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -95,7 +95,7 @@ template class DecimalField { public: - DecimalField(T value, UInt32 scale_) + DecimalField(T value = 0, UInt32 scale_ = 0) : dec(value), scale(scale_) {} @@ -564,6 +564,8 @@ public: return f(null); } + String dump() const; + static Field restoreFromDump(const std::string_view & dump_); private: std::aligned_union_t +void readQuoted(DecimalField & x, ReadBuffer & buf); + void writeFieldText(const Field & x, WriteBuffer & buf); [[noreturn]] inline void writeQuoted(const Tuple &, WriteBuffer &) { throw Exception("Cannot write Tuple quoted.", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 628e6ef6d4c..ee2e28dc89d 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -303,4 +303,27 @@ String SettingFieldEnumHelpers::readBinary(ReadBuffer & in) return str; } + +String SettingFieldCustom::toString() const +{ + return value.dump(); +} + +void SettingFieldCustom::parseFromString(const String & str) +{ + *this = Field::restoreFromDump(str); +} + +void SettingFieldCustom::writeBinary(WriteBuffer & out) const +{ + writeStringBinary(toString(), out); +} + +void SettingFieldCustom::readBinary(ReadBuffer & in) +{ + String str; + readStringBinary(str, in); + parseFromString(str); +} + } diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index f6e03ab9dde..ca774336f88 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -327,4 +327,23 @@ void SettingFieldEnum::readBinary(ReadBuffer & in) msg += "]"; \ throw Exception(msg, ERROR_CODE_FOR_UNEXPECTED_NAME); \ } + + +/// Can keep a value of any type. Used for user-defined settings. +struct SettingFieldCustom +{ + Field value; + bool changed = false; + + explicit SettingFieldCustom(const Field & f = {}) : value(f) {} + SettingFieldCustom & operator =(const Field & f) { value = f; changed = true; return *this; } + explicit operator Field() const { return value; } + + String toString() const; + void parseFromString(const String & str); + + void writeBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); +}; + } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index d11bd1d8706..640edc3be5a 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -861,6 +861,13 @@ inline void readQuoted(LocalDateTime & x, ReadBuffer & buf) assertChar('\'', buf); } +inline void readQuoted(UUID & x, ReadBuffer & buf) +{ + assertChar('\'', buf); + readUUIDText(x, buf); + assertChar('\'', buf); +} + /// Same as above, but in double quotes. template From f73a4749cf8459c8790e812cbb12de66115827f0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 28 Jul 2020 23:30:21 +0300 Subject: [PATCH 14/20] Add function getSetting(). --- src/Functions/getSetting.cpp | 67 +++++++++++++++++++ .../registerFunctionsMiscellaneous.cpp | 2 + src/Functions/ya.make | 1 + 3 files changed, 70 insertions(+) create mode 100644 src/Functions/getSetting.cpp diff --git a/src/Functions/getSetting.cpp b/src/Functions/getSetting.cpp new file mode 100644 index 00000000000..7421aca81f3 --- /dev/null +++ b/src/Functions/getSetting.cpp @@ -0,0 +1,67 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; +} + +/// Get the value of a setting. +class FunctionGetSetting : public IFunction +{ +public: + static constexpr auto name = "getSetting"; + + static FunctionPtr create(const Context & context_) { return std::make_shared(context_); } + explicit FunctionGetSetting(const Context & context_) : context(context_) {} + + String getName() const override { return name; } + bool isDeterministic() const override { return false; } + size_t getNumberOfArguments() const override { return 1; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!isString(arguments[0].type)) + throw Exception{"The argument of function " + String{name} + " should be a constant string with the name of a setting", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + const auto * column = arguments[0].column.get(); + if (!column || !checkAndGetColumnConstStringOrFixedString(column)) + throw Exception{"The argument of function " + String{name} + " should be a constant string with the name of a setting", + ErrorCodes::ILLEGAL_COLUMN}; + + std::string_view setting_name{column->getDataAt(0)}; + value = context.getSettingsRef().get(setting_name); + + DataTypePtr type = applyVisitor(FieldToDataType{}, value); + value = convertFieldToType(value, *type); + return type; + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override + { + block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, value); + } + +private: + mutable Field value; + const Context & context; +}; + + +void registerFunctionGetSetting(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 8207fcb8edd..11d3e129c28 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -56,6 +56,7 @@ void registerFunctionBasename(FunctionFactory &); void registerFunctionTransform(FunctionFactory &); void registerFunctionGetMacro(FunctionFactory &); void registerFunctionGetScalar(FunctionFactory &); +void registerFunctionGetSetting(FunctionFactory &); void registerFunctionIsConstant(FunctionFactory &); void registerFunctionGlobalVariable(FunctionFactory &); void registerFunctionHasThreadFuzzer(FunctionFactory &); @@ -116,6 +117,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionTransform(factory); registerFunctionGetMacro(factory); registerFunctionGetScalar(factory); + registerFunctionGetSetting(factory); registerFunctionIsConstant(factory); registerFunctionGlobalVariable(factory); registerFunctionHasThreadFuzzer(factory); diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 54f5ff24990..4df88376064 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -202,6 +202,7 @@ SRCS( geoToH3.cpp getMacro.cpp getScalar.cpp + getSetting.cpp getSizeOfEnumType.cpp globalVariable.cpp greatCircleDistance.cpp From 7c4ae5ee65affcbbc3a6202558aa1c0dd6304245 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 29 Jul 2020 02:39:18 +0300 Subject: [PATCH 15/20] Add the parameter custom_settings_prefixes to the server config. --- programs/server/Server.cpp | 3 + programs/server/config.xml | 3 + src/Access/AccessControlManager.cpp | 80 ++++++++++++++++++++++++++- src/Access/AccessControlManager.h | 12 +++- src/Access/SettingsConstraints.cpp | 15 ++++- src/Access/SettingsConstraints.h | 4 +- src/Access/SettingsProfileElement.cpp | 8 ++- src/Access/SettingsProfileElement.h | 2 +- src/Access/SettingsProfilesCache.cpp | 6 +- 9 files changed, 121 insertions(+), 12 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 85cd75eaf4f..e619511d3f1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -614,6 +614,9 @@ int Server::main(const std::vector & /*args*/) } global_context->setUncompressedCache(uncompressed_cache_size); + if (config().has("custom_settings_prefixes")) + global_context->getAccessControlManager().setCustomSettingsPrefixes(config().getString("custom_settings_prefixes")); + /// Load global settings from default_profile and system_profile. global_context->setDefaultProfiles(config()); const Settings & settings = global_context->getSettingsRef(); diff --git a/programs/server/config.xml b/programs/server/config.xml index 3e01964f0ff..5d29ee1a734 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -262,6 +262,9 @@ default + + + diff --git a/src/Access/AccessControlManager.cpp b/src/Access/AccessControlManager.cpp index 5966c1aff75..93d80701912 100644 --- a/src/Access/AccessControlManager.cpp +++ b/src/Access/AccessControlManager.cpp @@ -11,12 +11,19 @@ #include #include #include +#include #include +#include #include namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + namespace { std::vector> createStorages() @@ -59,6 +66,53 @@ private: }; +class AccessControlManager::CustomSettingsPrefixes +{ +public: + void registerPrefixes(const Strings & prefixes_) + { + std::lock_guard lock{mutex}; + registered_prefixes = prefixes_; + } + + bool isSettingNameAllowed(const std::string_view & setting_name) const + { + if (Settings::hasBuiltin(setting_name)) + return true; + + std::lock_guard lock{mutex}; + for (const auto & prefix : registered_prefixes) + { + if (setting_name.starts_with(prefix)) + return true; + } + + return false; + } + + void checkSettingNameIsAllowed(const std::string_view & setting_name) const + { + if (isSettingNameAllowed(setting_name)) + return; + + std::lock_guard lock{mutex}; + if (!registered_prefixes.empty()) + { + throw Exception( + "Setting " + String{setting_name} + " is neither a builtin setting nor started with the prefix '" + + boost::algorithm::join(registered_prefixes, "' or '") + "' registered for user-defined settings", + ErrorCodes::UNKNOWN_SETTING); + } + else + BaseSettingsHelpers::throwSettingNotFound(setting_name); + } + +private: + Strings registered_prefixes; + mutable std::mutex mutex; +}; + + AccessControlManager::AccessControlManager() : MultipleAccessStorage(createStorages()), context_access_cache(std::make_unique(*this)), @@ -66,7 +120,8 @@ AccessControlManager::AccessControlManager() row_policy_cache(std::make_unique(*this)), quota_cache(std::make_unique(*this)), settings_profiles_cache(std::make_unique(*this)), - external_authenticators(std::make_unique()) + external_authenticators(std::make_unique()), + custom_settings_prefixes(std::make_unique()) { } @@ -100,6 +155,29 @@ void AccessControlManager::setDefaultProfileName(const String & default_profile_ } +void AccessControlManager::setCustomSettingsPrefixes(const Strings & prefixes) +{ + custom_settings_prefixes->registerPrefixes(prefixes); +} + +void AccessControlManager::setCustomSettingsPrefixes(const String & comma_separated_prefixes) +{ + Strings prefixes; + splitInto<','>(prefixes, comma_separated_prefixes); + setCustomSettingsPrefixes(prefixes); +} + +bool AccessControlManager::isSettingNameAllowed(const std::string_view & setting_name) const +{ + return custom_settings_prefixes->isSettingNameAllowed(setting_name); +} + +void AccessControlManager::checkSettingNameIsAllowed(const std::string_view & setting_name) const +{ + custom_settings_prefixes->checkSettingNameIsAllowed(setting_name); +} + + std::shared_ptr AccessControlManager::getContextAccess( const UUID & user_id, const boost::container::flat_set & current_roles, diff --git a/src/Access/AccessControlManager.h b/src/Access/AccessControlManager.h index 5944916e798..443e8fc34b6 100644 --- a/src/Access/AccessControlManager.h +++ b/src/Access/AccessControlManager.h @@ -53,6 +53,13 @@ public: void setUsersConfig(const Poco::Util::AbstractConfiguration & users_config); void setDefaultProfileName(const String & default_profile_name); + /// Sets prefixes which should be used for custom settings. + /// This function also enables custom prefixes to be used. + void setCustomSettingsPrefixes(const Strings & prefixes); + void setCustomSettingsPrefixes(const String & comma_separated_prefixes); + bool isSettingNameAllowed(const std::string_view & name) const; + void checkSettingNameIsAllowed(const std::string_view & name) const; + std::shared_ptr getContextAccess( const UUID & user_id, const boost::container::flat_set & current_roles, @@ -89,14 +96,15 @@ public: const ExternalAuthenticators & getExternalAuthenticators() const; -private: - class ContextAccessCache; +private: class ContextAccessCache; + class CustomSettingsPrefixes; std::unique_ptr context_access_cache; std::unique_ptr role_cache; std::unique_ptr row_policy_cache; std::unique_ptr quota_cache; std::unique_ptr settings_profiles_cache; std::unique_ptr external_authenticators; + std::unique_ptr custom_settings_prefixes; }; } diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 5da68402f8b..c9a6b6f6b7a 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -16,7 +17,11 @@ namespace ErrorCodes extern const int SETTING_CONSTRAINT_VIOLATION; } -SettingsConstraints::SettingsConstraints() = default; + +SettingsConstraints::SettingsConstraints(const AccessControlManager & manager_) : manager(&manager_) +{ +} + SettingsConstraints::SettingsConstraints(const SettingsConstraints & src) = default; SettingsConstraints & SettingsConstraints::operator=(const SettingsConstraints & src) = default; SettingsConstraints::SettingsConstraints(SettingsConstraints && src) = default; @@ -191,6 +196,11 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh } }; + if (reaction == THROW_ON_VIOLATION) + manager->checkSettingNameIsAllowed(setting_name); + else if (!manager->isSettingNameAllowed(setting_name)) + return false; + Field current_value, new_value; if (current_settings.tryGet(setting_name, current_value)) { @@ -316,9 +326,8 @@ bool SettingsConstraints::Constraint::operator==(const Constraint & other) const && (*setting_name == *other.setting_name); } - bool operator ==(const SettingsConstraints & left, const SettingsConstraints & right) { - return left.constraints == right.constraints; + return (left.constraints == right.constraints); } } diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 39fe3fb810b..4259fe15e25 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -14,6 +14,7 @@ namespace DB struct Settings; struct SettingChange; class SettingsChanges; +class AccessControlManager; /** Checks if specified changes of settings are allowed or not. @@ -50,7 +51,7 @@ class SettingsChanges; class SettingsConstraints { public: - SettingsConstraints(); + SettingsConstraints(const AccessControlManager & manager_); SettingsConstraints(const SettingsConstraints & src); SettingsConstraints & operator =(const SettingsConstraints & src); SettingsConstraints(SettingsConstraints && src); @@ -108,6 +109,7 @@ private: const Constraint * tryGetConstraint(const std::string_view & setting_name) const; std::unordered_map constraints; + const AccessControlManager * manager = nullptr; }; } diff --git a/src/Access/SettingsProfileElement.cpp b/src/Access/SettingsProfileElement.cpp index 304eb4b742a..fd2fc9bcb15 100644 --- a/src/Access/SettingsProfileElement.cpp +++ b/src/Access/SettingsProfileElement.cpp @@ -48,6 +48,10 @@ void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const A min_value = Settings::castValueUtil(setting_name, min_value); if (!max_value.isNull()) max_value = Settings::castValueUtil(setting_name, max_value); + + /// Optionally check if a setting with that name is allowed. + if (manager) + manager->checkSettingNameIsAllowed(setting_name); } } @@ -149,9 +153,9 @@ SettingsChanges SettingsProfileElements::toSettingsChanges() const return res; } -SettingsConstraints SettingsProfileElements::toSettingsConstraints() const +SettingsConstraints SettingsProfileElements::toSettingsConstraints(const AccessControlManager & manager) const { - SettingsConstraints res; + SettingsConstraints res{manager}; for (const auto & elem : *this) { if (!elem.setting_name.empty()) diff --git a/src/Access/SettingsProfileElement.h b/src/Access/SettingsProfileElement.h index 0de71426d0f..c9262fecb73 100644 --- a/src/Access/SettingsProfileElement.h +++ b/src/Access/SettingsProfileElement.h @@ -61,7 +61,7 @@ public: Settings toSettings() const; SettingsChanges toSettingsChanges() const; - SettingsConstraints toSettingsConstraints() const; + SettingsConstraints toSettingsConstraints(const AccessControlManager & manager) const; }; } diff --git a/src/Access/SettingsProfilesCache.cpp b/src/Access/SettingsProfilesCache.cpp index e663ee564aa..ef4bffa11f9 100644 --- a/src/Access/SettingsProfilesCache.cpp +++ b/src/Access/SettingsProfilesCache.cpp @@ -103,6 +103,7 @@ void SettingsProfilesCache::setDefaultProfileName(const String & default_profile default_profile_id = it->second; } + void SettingsProfilesCache::mergeSettingsAndConstraints() { /// `mutex` is already locked. @@ -143,9 +144,10 @@ void SettingsProfilesCache::mergeSettingsAndConstraintsFor(EnabledSettings & ena substituteProfiles(merged_settings); + auto settings = merged_settings.toSettings(); + auto constraints = merged_settings.toSettingsConstraints(manager); enabled.setSettingsAndConstraints( - std::make_shared(merged_settings.toSettings()), - std::make_shared(merged_settings.toSettingsConstraints())); + std::make_shared(std::move(settings)), std::make_shared(std::move(constraints))); } From 0fcee343229359d2ba893702337fb178451f5fa0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 29 Jul 2020 18:50:10 +0300 Subject: [PATCH 16/20] Add test. --- docker/test/fasttest/run.sh | 1 + docker/test/stateless/run.sh | 1 + tests/config/custom_settings_prefixes.xml | 4 +++ .../01418_custom_settings.reference | 17 +++++++++++++ .../0_stateless/01418_custom_settings.sql | 25 +++++++++++++++++++ 5 files changed, 48 insertions(+) create mode 100644 tests/config/custom_settings_prefixes.xml create mode 100644 tests/queries/0_stateless/01418_custom_settings.reference create mode 100644 tests/queries/0_stateless/01418_custom_settings.sql diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 7036aa8bf36..1a5ed68a2f4 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -67,6 +67,7 @@ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/conf ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/custom_settings_prefixes.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/ ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/ ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/ diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index a221e76f2f0..2ff15ca9c6a 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -17,6 +17,7 @@ ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/custom_settings_prefixes.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/ ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/ ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/ diff --git a/tests/config/custom_settings_prefixes.xml b/tests/config/custom_settings_prefixes.xml new file mode 100644 index 00000000000..b15b9372bc6 --- /dev/null +++ b/tests/config/custom_settings_prefixes.xml @@ -0,0 +1,4 @@ + + + custom_ + diff --git a/tests/queries/0_stateless/01418_custom_settings.reference b/tests/queries/0_stateless/01418_custom_settings.reference new file mode 100644 index 00000000000..1e5fd30305a --- /dev/null +++ b/tests/queries/0_stateless/01418_custom_settings.reference @@ -0,0 +1,17 @@ +5 UInt8 +-177 Int16 +98.11 Float64 +abc def String +custom_a UInt64_5 +custom_b Int64_-177 +custom_c Float64_98.11 +custom_d \'abc def\' +changed String +\N Nullable(Nothing) +50000 UInt16 +1.11 Float64 +custom_a \'changed\' +custom_b NULL +custom_c UInt64_50000 +custom_d Float64_1.11 +0 UInt8 diff --git a/tests/queries/0_stateless/01418_custom_settings.sql b/tests/queries/0_stateless/01418_custom_settings.sql new file mode 100644 index 00000000000..968ec22b538 --- /dev/null +++ b/tests/queries/0_stateless/01418_custom_settings.sql @@ -0,0 +1,25 @@ +SET custom_a = 5; +SET custom_b = -177; +SET custom_c = 98.11; +SET custom_d = 'abc def'; +SELECT getSetting('custom_a') as v, toTypeName(v); +SELECT getSetting('custom_b') as v, toTypeName(v); +SELECT getSetting('custom_c') as v, toTypeName(v); +SELECT getSetting('custom_d') as v, toTypeName(v); +SELECT name, value FROM system.settings WHERE name LIKE 'custom_%' ORDER BY name; + +SET custom_a = 'changed'; +SET custom_b = NULL; +SET custom_c = 50000; +SET custom_d = 1.11; +SELECT getSetting('custom_a') as v, toTypeName(v); +SELECT getSetting('custom_b') as v, toTypeName(v); +SELECT getSetting('custom_c') as v, toTypeName(v); +SELECT getSetting('custom_d') as v, toTypeName(v); +SELECT name, value FROM system.settings WHERE name LIKE 'custom_%' ORDER BY name; + +SELECT getSetting('custom_e') as v, toTypeName(v); -- { serverError 115 } -- Setting not found. +SET custom_e = 0; +SELECT getSetting('custom_e') as v, toTypeName(v); + +SET invalid_custom = 8; -- { serverError 115 } -- Setting is neither a builtin nor started with one of the registered prefixes for user-defined settings. From 9e97aab3b3b9c49864e3385341e6f0aeee4e9ea6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Aug 2020 05:00:52 +0300 Subject: [PATCH 17/20] Whitespace --- src/IO/parseDateTimeBestEffort.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index e98dbbc0480..a8567c5c529 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -345,8 +345,6 @@ ReturnType parseDateTimeBestEffortImpl( } else return on_error("Cannot read DateTime: unexpected number of decimal digits after day of month: " + toString(num_digits), ErrorCodes::CANNOT_PARSE_DATETIME); - - } if (month > 12) From f633c597370f58d73d58c2bc41f1c9e25719f8a7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Aug 2020 06:09:10 +0300 Subject: [PATCH 18/20] Fix warning in libunwind --- contrib/libunwind-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind-cmake/CMakeLists.txt b/contrib/libunwind-cmake/CMakeLists.txt index e5c83cc33ec..9dbbaf23322 100644 --- a/contrib/libunwind-cmake/CMakeLists.txt +++ b/contrib/libunwind-cmake/CMakeLists.txt @@ -54,7 +54,7 @@ endif () # Example: DwarfInstructions.hpp: register unsigned long long x16 __asm("x16") = cfa; check_cxx_compiler_flag(-Wregister HAVE_WARNING_REGISTER) if (HAVE_WARNING_REGISTER) - target_compile_options(unwind PRIVATE -Wno-register) + target_compile_options(unwind PRIVATE "$<$,CXX>:-Wno-register>") endif () install( From 14174ba38f6de387ccd4f16a2388db8ab5d4b9da Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Aug 2020 06:11:40 +0300 Subject: [PATCH 19/20] Fix warning in libunwind --- contrib/libunwind-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind-cmake/CMakeLists.txt b/contrib/libunwind-cmake/CMakeLists.txt index 9dbbaf23322..cbfc79ed426 100644 --- a/contrib/libunwind-cmake/CMakeLists.txt +++ b/contrib/libunwind-cmake/CMakeLists.txt @@ -54,7 +54,7 @@ endif () # Example: DwarfInstructions.hpp: register unsigned long long x16 __asm("x16") = cfa; check_cxx_compiler_flag(-Wregister HAVE_WARNING_REGISTER) if (HAVE_WARNING_REGISTER) - target_compile_options(unwind PRIVATE "$<$,CXX>:-Wno-register>") + target_compile_options(unwind PRIVATE "$<$,CXX>:-Wno-register>") endif () install( From 139177cb1987a570a98ac34cafdc1c0953dce5aa Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 2 Aug 2020 11:53:22 +0300 Subject: [PATCH 20/20] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index d37ecd85f38..b84ac2e1d03 100644 --- a/README.md +++ b/README.md @@ -16,4 +16,4 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events -* [ClickHouse at ByteDance (in Chinese)](https://mp.weixin.qq.com/s/Em-HjPylO8D7WPui4RREAQ) on July 31, 2020. +* [ClickHouse at ByteDance (in Chinese)](https://mp.weixin.qq.com/s/Em-HjPylO8D7WPui4RREAQ) on August 14, 2020.