diff --git a/dbms/src/Columns/ReverseIndex.h b/dbms/src/Columns/ReverseIndex.h index 157deef46b8..2e017202741 100644 --- a/dbms/src/Columns/ReverseIndex.h +++ b/dbms/src/Columns/ReverseIndex.h @@ -82,6 +82,8 @@ namespace template static bool isZero(const T &, const State & /*state*/) { + /// Careful: apparently this uses SFINAE to redefine isZero for all types + /// except the IndexType, for which the default ZeroTraits::isZero is used. static_assert(!std::is_same_v::type, typename std::decay::type>); return false; } @@ -122,19 +124,88 @@ namespace }; + /** + * ReverseIndexHashTableBase implements a special hash table interface for + * reverse index. + * + * The following requirements are different compared to a plain hash table: + * + * 1) Provide public access to 'hash table state' that contains + * additional data needed to calculate cell hashes. + * + * 2) Support emplace() and find() with a Key different from the resulting + * hash table key. This means emplace() accepts a different kind of object + * as a key, and then the real key can be read from the returned cell iterator. + * + * These requirements are unique to ReverseIndex and are in conflict with + * supporting hash tables that use alternative key storage, such as FixedHashMap + * or StringHashMap. Therefore, we implement an interface for ReverseIndex + * separately. + */ template - class HashTableWithPublicState : public HashTable, HashTableAllocator> + class ReverseIndexHashTableBase : public HashTable, HashTableAllocator> { using State = typename Cell::State; using Base = HashTable, HashTableAllocator>; public: using Base::Base; + using iterator = typename Base::iterator; State & getState() { return *this; } + + + template + size_t ALWAYS_INLINE reverseIndexFindCell(const ObjectToCompareWith & x, + size_t hash_value, size_t place_value) const + { + while (!this->buf[place_value].isZero(*this) + && !this->buf[place_value].keyEquals(x, hash_value, *this)) + { + place_value = this->grower.next(place_value); + } + + return place_value; + } + + template + void ALWAYS_INLINE reverseIndexEmplaceNonZero(const Key & key, iterator & it, + bool & inserted, size_t hash_value, const ObjectToCompareWith & object) + { + size_t place_value = reverseIndexFindCell(object, hash_value, + this->grower.place(hash_value)); + // emplaceNonZeroImpl() might need to re-find the cell if the table grows, + // but it will find it correctly by the key alone, so we don't have to + // pass it the 'object'. + this->emplaceNonZeroImpl(place_value, key, it, inserted, hash_value); + } + + /// Searches position by object. + template + void ALWAYS_INLINE reverseIndexEmplace(Key key, iterator & it, bool & inserted, + size_t hash_value, const ObjectToCompareWith& object) + { + if (!this->emplaceIfZero(key, it, inserted, hash_value)) + { + reverseIndexEmplaceNonZero(key, it, inserted, hash_value, object); + } + } + + template + iterator ALWAYS_INLINE reverseIndexFind(ObjectToCompareWith x, size_t hash_value) + { + if (Cell::isZero(x, *this)) + return this->hasZero() ? this->iteratorToZero() : this->end(); + + size_t place_value = reverseIndexFindCell(x, hash_value, + this->grower.place(hash_value)); + return !this->buf[place_value].isZero(*this) + ? iterator(this, &this->buf[place_value]) + : this->end(); + } }; template - class ReverseIndexStringHashTable : public HashTableWithPublicState< + class ReverseIndexStringHashTable : public ReverseIndexHashTableBase< IndexType, ReverseIndexHashTableCell< IndexType, @@ -145,7 +216,7 @@ namespace has_base_index>, ReverseIndexHash> { - using Base = HashTableWithPublicState< + using Base = ReverseIndexHashTableBase< IndexType, ReverseIndexHashTableCell< IndexType, @@ -167,7 +238,7 @@ namespace }; template - class ReverseIndexNumberHashTable : public HashTableWithPublicState< + class ReverseIndexNumberHashTable : public ReverseIndexHashTableBase< IndexType, ReverseIndexHashTableCell< IndexType, @@ -178,7 +249,7 @@ namespace has_base_index>, ReverseIndexHash> { - using Base = HashTableWithPublicState< + using Base = ReverseIndexHashTableBase< IndexType, ReverseIndexHashTableCell< IndexType, @@ -357,7 +428,7 @@ void ReverseIndex::buildIndex() else hash = getHash(column->getDataAt(row)); - index->emplace(row + base_index, iterator, inserted, hash, column->getDataAt(row)); + index->reverseIndexEmplace(row + base_index, iterator, inserted, hash, column->getDataAt(row)); if (!inserted) throw Exception("Duplicating keys found in ReverseIndex.", ErrorCodes::LOGICAL_ERROR); @@ -402,7 +473,7 @@ UInt64 ReverseIndex::insert(const StringRef & data) else column->insertData(data.data, data.size); - index->emplace(num_rows + base_index, iterator, inserted, hash, data); + index->reverseIndexEmplace(num_rows + base_index, iterator, inserted, hash, data); if constexpr (use_saved_hash) { @@ -428,7 +499,7 @@ UInt64 ReverseIndex::getInsertionPoint(const StringRef & IteratorType iterator; auto hash = getHash(data); - iterator = index->find(data, hash); + iterator = index->reverseIndexFind(data, hash); return iterator == index->end() ? size() + base_index : iterator->getValue(); } diff --git a/dbms/src/Common/HashTable/FixedHashTable.h b/dbms/src/Common/HashTable/FixedHashTable.h index 663848865da..0c3a2c1237a 100644 --- a/dbms/src/Common/HashTable/FixedHashTable.h +++ b/dbms/src/Common/HashTable/FixedHashTable.h @@ -294,26 +294,22 @@ public: void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted) { emplaceImpl(x, it, inserted); } void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t) { emplaceImpl(x, it, inserted); } - template - iterator ALWAYS_INLINE find(ObjectToCompareWith x) + iterator ALWAYS_INLINE find(Key x) { return !buf[x].isZero(*this) ? iterator(this, &buf[x]) : end(); } - template - const_iterator ALWAYS_INLINE find(ObjectToCompareWith x) const + const_iterator ALWAYS_INLINE find(Key x) const { return !buf[x].isZero(*this) ? const_iterator(this, &buf[x]) : end(); } - template - iterator ALWAYS_INLINE find(ObjectToCompareWith, size_t hash_value) + iterator ALWAYS_INLINE find(Key, size_t hash_value) { return !buf[hash_value].isZero(*this) ? iterator(this, &buf[hash_value]) : end(); } - template - const_iterator ALWAYS_INLINE find(ObjectToCompareWith, size_t hash_value) const + const_iterator ALWAYS_INLINE find(Key, size_t hash_value) const { return !buf[hash_value].isZero(*this) ? const_iterator(this, &buf[hash_value]) : end(); } diff --git a/dbms/src/Common/HashTable/HashTable.h b/dbms/src/Common/HashTable/HashTable.h index f15fe22f8fd..c5a0c812ee2 100644 --- a/dbms/src/Common/HashTable/HashTable.h +++ b/dbms/src/Common/HashTable/HashTable.h @@ -280,8 +280,7 @@ protected: #endif /// Find a cell with the same key or an empty cell, starting from the specified position and further along the collision resolution chain. - template - size_t ALWAYS_INLINE findCell(const ObjectToCompareWith & x, size_t hash_value, size_t place_value) const + size_t ALWAYS_INLINE findCell(const Key & x, size_t hash_value, size_t place_value) const { while (!buf[place_value].isZero(*this) && !buf[place_value].keyEquals(x, hash_value, *this)) { @@ -700,13 +699,6 @@ protected: emplaceNonZeroImpl(place_value, x, it, inserted, hash_value); } - /// Same but find place using object. Hack for ReverseIndex. - template - void ALWAYS_INLINE emplaceNonZero(Key x, iterator & it, bool & inserted, size_t hash_value, const ObjectToCompareWith & object) - { - size_t place_value = findCell(object, hash_value, grower.place(hash_value)); - emplaceNonZeroImpl(place_value, x, it, inserted, hash_value); - } public: @@ -763,14 +755,6 @@ public: emplaceNonZero(x, it, inserted, hash_value); } - /// Same, but search position by object. Hack for ReverseIndex. - template - void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t hash_value, const ObjectToCompareWith & object) - { - if (!emplaceIfZero(x, it, inserted, hash_value)) - emplaceNonZero(x, it, inserted, hash_value, object); - } - /// Copy the cell from another hash table. It is assumed that the cell is not zero, and also that there was no such key in the table yet. void ALWAYS_INLINE insertUniqueNonZero(const Cell * cell, size_t hash_value) { @@ -783,9 +767,7 @@ public: resize(); } - - template - iterator ALWAYS_INLINE find(ObjectToCompareWith x) + iterator ALWAYS_INLINE find(Key x) { if (Cell::isZero(x, *this)) return this->hasZero() ? iteratorToZero() : end(); @@ -796,8 +778,7 @@ public: } - template - const_iterator ALWAYS_INLINE find(ObjectToCompareWith x) const + const_iterator ALWAYS_INLINE find(Key x) const { if (Cell::isZero(x, *this)) return this->hasZero() ? iteratorToZero() : end(); @@ -808,8 +789,7 @@ public: } - template - iterator ALWAYS_INLINE find(ObjectToCompareWith x, size_t hash_value) + iterator ALWAYS_INLINE find(Key x, size_t hash_value) { if (Cell::isZero(x, *this)) return this->hasZero() ? iteratorToZero() : end(); @@ -819,8 +799,7 @@ public: } - template - const_iterator ALWAYS_INLINE find(ObjectToCompareWith x, size_t hash_value) const + const_iterator ALWAYS_INLINE find(Key x, size_t hash_value) const { if (Cell::isZero(x, *this)) return this->hasZero() ? iteratorToZero() : end(); diff --git a/dbms/src/Common/StringSearcher.h b/dbms/src/Common/StringSearcher.h index 25287db11f5..d395e6f254a 100644 --- a/dbms/src/Common/StringSearcher.h +++ b/dbms/src/Common/StringSearcher.h @@ -775,6 +775,7 @@ using ASCIICaseInsensitiveStringSearcher = StringSearcher; using UTF8CaseSensitiveStringSearcher = StringSearcher; using UTF8CaseInsensitiveStringSearcher = StringSearcher; using ASCIICaseSensitiveTokenSearcher = TokenSearcher; +using ASCIICaseInsensitiveTokenSearcher = TokenSearcher; /** Uses functions from libc. diff --git a/dbms/src/Common/Volnitsky.h b/dbms/src/Common/Volnitsky.h index c87bdd79dab..a0273a1d858 100644 --- a/dbms/src/Common/Volnitsky.h +++ b/dbms/src/Common/Volnitsky.h @@ -652,7 +652,8 @@ using VolnitskyUTF8 = VolnitskyBase; /// ignores non-ASCII bytes using VolnitskyCaseInsensitiveUTF8 = VolnitskyBase; -using VolnitskyToken = VolnitskyBase; +using VolnitskyCaseSensitiveToken = VolnitskyBase; +using VolnitskyCaseInsensitiveToken = VolnitskyBase; using MultiVolnitsky = MultiVolnitskyBase; using MultiVolnitskyUTF8 = MultiVolnitskyBase; diff --git a/dbms/src/Compression/CompressionCodecDelta.cpp b/dbms/src/Compression/CompressionCodecDelta.cpp index 9f2397f8e59..abf0b79f9c8 100644 --- a/dbms/src/Compression/CompressionCodecDelta.cpp +++ b/dbms/src/Compression/CompressionCodecDelta.cpp @@ -80,7 +80,7 @@ UInt32 CompressionCodecDelta::doCompressData(const char * source, UInt32 source_ { UInt8 bytes_to_skip = source_size % delta_bytes_size; dest[0] = delta_bytes_size; - dest[1] = bytes_to_skip; + dest[1] = bytes_to_skip; /// unused (backward compatibility) memcpy(&dest[2], source, bytes_to_skip); size_t start_pos = 2 + bytes_to_skip; switch (delta_bytes_size) @@ -101,10 +101,16 @@ UInt32 CompressionCodecDelta::doCompressData(const char * source, UInt32 source_ return 1 + 1 + source_size; } -void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 /* uncompressed_size */) const +void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { + if (source_size < 2) + throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); + UInt8 bytes_size = source[0]; - UInt8 bytes_to_skip = source[1]; + UInt8 bytes_to_skip = uncompressed_size % bytes_size; + + if (UInt32(2 + bytes_to_skip) > source_size) + throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); memcpy(dest, &source[2], bytes_to_skip); UInt32 source_size_no_header = source_size - bytes_to_skip - 2; diff --git a/dbms/src/Compression/CompressionCodecDoubleDelta.cpp b/dbms/src/Compression/CompressionCodecDoubleDelta.cpp index d3830dc9fdb..9a6b551f159 100644 --- a/dbms/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/dbms/src/Compression/CompressionCodecDoubleDelta.cpp @@ -304,7 +304,7 @@ UInt32 CompressionCodecDoubleDelta::doCompressData(const char * source, UInt32 s { UInt8 bytes_to_skip = source_size % data_bytes_size; dest[0] = data_bytes_size; - dest[1] = bytes_to_skip; + dest[1] = bytes_to_skip; /// unused (backward compatibility) memcpy(&dest[2], source, bytes_to_skip); size_t start_pos = 2 + bytes_to_skip; UInt32 compressed_size = 0; @@ -328,10 +328,16 @@ UInt32 CompressionCodecDoubleDelta::doCompressData(const char * source, UInt32 s return 1 + 1 + compressed_size; } -void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 /* uncompressed_size */) const +void CompressionCodecDoubleDelta::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { + if (source_size < 2) + throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); + UInt8 bytes_size = source[0]; - UInt8 bytes_to_skip = source[1]; + UInt8 bytes_to_skip = uncompressed_size % bytes_size; + + if (UInt32(2 + bytes_to_skip) > source_size) + throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); memcpy(dest, &source[2], bytes_to_skip); UInt32 source_size_no_header = source_size - bytes_to_skip - 2; diff --git a/dbms/src/Compression/CompressionCodecGorilla.cpp b/dbms/src/Compression/CompressionCodecGorilla.cpp index 8af6c8bfd39..574e40b06bf 100644 --- a/dbms/src/Compression/CompressionCodecGorilla.cpp +++ b/dbms/src/Compression/CompressionCodecGorilla.cpp @@ -264,7 +264,7 @@ UInt32 CompressionCodecGorilla::doCompressData(const char * source, UInt32 sourc { UInt8 bytes_to_skip = source_size % data_bytes_size; dest[0] = data_bytes_size; - dest[1] = bytes_to_skip; + dest[1] = bytes_to_skip; /// unused (backward compatibility) memcpy(&dest[2], source, bytes_to_skip); size_t start_pos = 2 + bytes_to_skip; UInt32 result_size = 0; @@ -289,10 +289,16 @@ UInt32 CompressionCodecGorilla::doCompressData(const char * source, UInt32 sourc return 1 + 1 + result_size; } -void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 /* uncompressed_size */) const +void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { + if (source_size < 2) + throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); + UInt8 bytes_size = source[0]; - UInt8 bytes_to_skip = source[1]; + UInt8 bytes_to_skip = uncompressed_size % bytes_size; + + if (UInt32(2 + bytes_to_skip) > source_size) + throw Exception("Cannot decompress. File has wrong header", ErrorCodes::CANNOT_DECOMPRESS); memcpy(dest, &source[2], bytes_to_skip); UInt32 source_size_no_header = source_size - bytes_to_skip - 2; diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp index 23c244f4dcb..fad72c932c3 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.cpp +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -88,6 +88,9 @@ void CompressionCodecMultiple::useInfoAboutType(DataTypePtr data_type) void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 decompressed_size) const { + if (source_size < 1 || !source[0]) + throw Exception("Wrong compression methods list", ErrorCodes::CORRUPTED_DATA); + UInt8 compression_methods_size = source[0]; PODArray compressed_buf(&source[compression_methods_size + 1], &source[source_size]); @@ -103,7 +106,8 @@ void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 sour UInt32 uncompressed_size = ICompressionCodec::readDecompressedBlockSize(compressed_buf.data()); if (idx == 0 && uncompressed_size != decompressed_size) - throw Exception("Wrong final decompressed size in codec Multiple, got " + toString(uncompressed_size) + ", expected " + toString(decompressed_size), ErrorCodes::CORRUPTED_DATA); + throw Exception("Wrong final decompressed size in codec Multiple, got " + toString(uncompressed_size) + + ", expected " + toString(decompressed_size), ErrorCodes::CORRUPTED_DATA); uncompressed_buf.resize(uncompressed_size + codec->getAdditionalSizeAtTheEndOfBuffer()); codec->decompress(compressed_buf.data(), source_size, uncompressed_buf.data()); diff --git a/dbms/src/Compression/ICompressionCodec.cpp b/dbms/src/Compression/ICompressionCodec.cpp index aafca2f5eb3..f9d392e5d6d 100644 --- a/dbms/src/Compression/ICompressionCodec.cpp +++ b/dbms/src/Compression/ICompressionCodec.cpp @@ -47,7 +47,7 @@ UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, ch throw Exception("Can't decompress data with codec byte " + toString(method) + " from codec with byte " + toString(method), ErrorCodes::CANNOT_DECOMPRESS); UInt8 header_size = getHeaderSize(); - UInt32 decompressed_size = unalignedLoad(&source[5]); + UInt32 decompressed_size = readDecompressedBlockSize(source); doDecompressData(&source[header_size], source_size - header_size, dest, decompressed_size); return decompressed_size; diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index cb12a969b76..881ea83f30b 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -333,7 +333,7 @@ struct Settings : public SettingsCollection 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 for 'mysql' table function.") \ M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.") \ \ - M(SettingBool, experimental_use_processors, true, "Use processors pipeline.") \ + M(SettingBool, experimental_use_processors, false, "Use processors pipeline.") \ \ M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \ M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \ diff --git a/dbms/src/Functions/FunctionsStringSearch.cpp b/dbms/src/Functions/FunctionsStringSearch.cpp index 5d688232bd4..e44138bb482 100644 --- a/dbms/src/Functions/FunctionsStringSearch.cpp +++ b/dbms/src/Functions/FunctionsStringSearch.cpp @@ -436,7 +436,7 @@ struct MultiSearchFirstIndexImpl /** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation. */ -template +template struct HasTokenImpl { using ResultType = UInt8; @@ -454,7 +454,7 @@ struct HasTokenImpl /// The current index in the array of strings. size_t i = 0; - VolnitskyToken searcher(pattern.data(), pattern.size(), end - pos); + TokenSearcher searcher(pattern.data(), pattern.size(), end - pos); /// We will search for the next occurrence in all rows at once. while (pos < end && end != (pos = searcher.search(pos, end - pos))) @@ -483,7 +483,7 @@ struct HasTokenImpl static void constant_constant(const std::string & data, const std::string & pattern, UInt8 & res) { - VolnitskyToken searcher(pattern.data(), pattern.size(), data.size()); + TokenSearcher searcher(pattern.data(), pattern.size(), data.size()); const auto found = searcher.search(data.c_str(), data.size()) != data.end().base(); res = negate_result ^ found; } @@ -589,6 +589,11 @@ struct NameHasToken static constexpr auto name = "hasToken"; }; +struct NameHasTokenCaseInsensitive +{ + static constexpr auto name = "hasTokenCaseInsensitive"; +}; + using FunctionPosition = FunctionsStringSearch, NamePosition>; using FunctionPositionUTF8 = FunctionsStringSearch, NamePositionUTF8>; @@ -615,7 +620,8 @@ using FunctionMultiSearchFirstPositionUTF8 = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitive>; using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitiveUTF8>; -using FunctionHasToken = FunctionsStringSearch, NameHasToken>; +using FunctionHasToken = FunctionsStringSearch, NameHasToken>; +using FunctionHasTokenCaseInsensitive = FunctionsStringSearch, NameHasTokenCaseInsensitive>; void registerFunctionsStringSearch(FunctionFactory & factory) { @@ -645,6 +651,7 @@ void registerFunctionsStringSearch(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive); } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 39b9fd40c9a..016f1fa0e49 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -142,7 +142,7 @@ struct ContextShared std::unique_ptr ddl_worker; /// Process ddl commands from zk. /// Rules for selecting the compression settings, depending on the size of the part. mutable std::unique_ptr compression_codec_selector; - MergeTreeSettingsPtr merge_tree_settings; /// Settings of MergeTree* engines. + std::optional merge_tree_settings; /// Settings of MergeTree* engines. size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default) size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default) String format_schema_path; /// Path to a directory that contains schema files used by input formats. @@ -1759,9 +1759,9 @@ const MergeTreeSettings & Context::getMergeTreeSettings() const if (!shared->merge_tree_settings) { auto & config = getConfigRef(); - MutableMergeTreeSettingsPtr settings_ptr = MergeTreeSettings::create(); - settings_ptr->loadFromConfig("merge_tree", config); - shared->merge_tree_settings = std::move(settings_ptr); + MergeTreeSettings mt_settings; + mt_settings.loadFromConfig("merge_tree", config); + shared->merge_tree_settings.emplace(mt_settings); } return *shared->merge_tree_settings; diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index a32609bd53d..5798e182b69 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -104,7 +104,7 @@ BlockIO InterpreterAlterQuery::execute() { auto table_lock_holder = table->lockAlterIntention(context.getCurrentQueryId()); alter_commands.validate(*table, context); - table->alter(alter_commands, database_name, table_name, context, table_lock_holder); + table->alter(alter_commands, context, table_lock_holder); } return {}; diff --git a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp index 7c90769b37b..43624bf16b8 100644 --- a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp +++ b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp @@ -30,6 +30,7 @@ namespace DB /// Simplified version of the StorageDistributed class. class StorageDistributedFake : public ext::shared_ptr_helper, public DB::IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "DistributedFake"; } bool isRemote() const override { return true; } diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index b30ed6a3560..58974010c27 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -372,11 +372,12 @@ TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id) void IStorage::alterSettings( const SettingsChanges & new_changes, - const String & current_database_name, - const String & current_table_name, const Context & context, TableStructureWriteLockHolder & /* table_lock_holder */) { + const String current_database_name = getDatabaseName(); + const String current_table_name = getTableName(); + IDatabase::ASTModifier storage_modifier = [&] (IAST & ast) { if (!new_changes.empty()) @@ -404,16 +405,16 @@ void IStorage::alterSettings( void IStorage::alter( const AlterCommands & params, - const String & database_name, - const String & table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { + const String database_name = getDatabaseName(); + const String table_name = getTableName(); if (params.isSettingsAlter()) { SettingsChanges new_changes; params.applyForSettingsOnly(new_changes); - alterSettings(new_changes, database_name, table_name, context, table_lock_holder); + alterSettings(new_changes, context, table_lock_holder); return; } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index b070cb13ee8..5672a1ba3fb 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -281,7 +281,7 @@ public: * This method must fully execute the ALTER query, taking care of the locks itself. * To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata. */ - virtual void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder); + virtual void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder); /** ALTER tables with regard to its partitions. * Should handle locks for each command on its own. @@ -295,8 +295,6 @@ public: */ virtual void alterSettings( const SettingsChanges & new_changes, - const String & current_database_name, - const String & current_table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder); diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 0572636f36c..1694c8e3ade 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -414,8 +414,6 @@ bool StorageKafka::hasSetting(const String & setting_name) const void StorageKafka::alterSettings( const SettingsChanges & /* new_changes */, - const String & /* current_database_name */, - const String & /* current_table_name */, const Context & /* context */, TableStructureWriteLockHolder & /* table_lock_holder */) { diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index a6039eac254..07c95d75e90 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -20,6 +20,7 @@ namespace DB */ class StorageKafka : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Kafka"; } std::string getTableName() const override { return table_name; } @@ -61,8 +62,6 @@ public: void alterSettings( const SettingsChanges & new_changes, - const String & current_database_name, - const String & current_table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; diff --git a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp index 795cc68f1ea..8a6ceec509a 100644 --- a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp @@ -54,7 +54,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED); String part_name = params.get("part"); - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); /// Validation of the input that may come from malicious replica. MergeTreePartInfo::fromPartName(part_name, data.format_version); @@ -175,7 +175,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( { /// Validation of the input that may come from malicious replica. MergeTreePartInfo::fromPartName(part_name, data.format_version); - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); Poco::URI uri; uri.setScheme(interserver_scheme); diff --git a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 11b64fd9788..255fb998446 100644 --- a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -39,7 +39,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( , compute_granularity(index_granularity.empty()) , codec(std::move(codec_)) , skip_indices(indices_to_recalc) - , with_final_mark(storage.getCOWSettings()->write_final_mark && can_use_adaptive_granularity) + , with_final_mark(storage.getSettings()->write_final_mark && can_use_adaptive_granularity) { if (blocks_are_granules_size && !index_granularity.empty()) throw Exception("Can't take information about index granularity from blocks, when non empty index_granularity array specified", ErrorCodes::LOGICAL_ERROR); @@ -139,7 +139,7 @@ void fillIndexGranularityImpl( void IMergedBlockOutputStream::fillIndexGranularity(const Block & block) { - const auto storage_settings = storage.getCOWSettings(); + const auto storage_settings = storage.getSettings(); fillIndexGranularityImpl( block, storage_settings->index_granularity_bytes, diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index da8fea1baf8..0767fb35326 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -107,7 +107,7 @@ MergeTreeData::MergeTreeData( const ASTPtr & sample_by_ast_, const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - MergeTreeSettingsPtr settings_, + std::unique_ptr storage_settings_, bool require_part_metadata_, bool attach, BrokenPartCallback broken_part_callback_) @@ -121,11 +121,11 @@ MergeTreeData::MergeTreeData( full_path(full_path_), broken_part_callback(broken_part_callback_), log_name(database_name + "." + table_name), log(&Logger::get(log_name)), - guarded_settings(settings_), + storage_settings(std::move(storage_settings_)), data_parts_by_info(data_parts_indexes.get()), data_parts_by_state_and_info(data_parts_indexes.get()) { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); setProperties(order_by_ast_, primary_key_ast_, columns_, indices_, constraints_); /// NOTE: using the same columns list as is read when performing actual merges. @@ -732,7 +732,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) { LOG_DEBUG(log, "Loading data parts"); - const auto settings = getCOWSettings(); + const auto settings = getSettings(); Strings part_file_names; Poco::DirectoryIterator end; for (Poco::DirectoryIterator it(full_path); it != end; ++it) @@ -965,7 +965,7 @@ void MergeTreeData::clearOldTemporaryDirectories(ssize_t custom_directories_life if (!lock.try_lock()) return; - const auto settings = getCOWSettings(); + const auto settings = getSettings(); time_t current_time = time(nullptr); ssize_t deadline = (custom_directories_lifetime_seconds >= 0) ? current_time - custom_directories_lifetime_seconds @@ -1020,7 +1020,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts() if (part.unique() && /// Grab only parts that are not used by anyone (SELECTs for example). part_remove_time < now && - now - part_remove_time > getCOWSettings()->old_parts_lifetime.totalSeconds()) + now - part_remove_time > getSettings()->old_parts_lifetime.totalSeconds()) { parts_to_delete.emplace_back(it); } @@ -1104,7 +1104,7 @@ void MergeTreeData::clearOldPartsFromFilesystem() void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_remove) { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); if (parts_to_remove.size() > 1 && settings->max_part_removal_threads > 1 && parts_to_remove.size() > settings->concurrent_part_removal_threshold) { /// Parallel parts removal. @@ -1341,7 +1341,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name const IndicesASTs & old_indices, const IndicesASTs & new_indices, ExpressionActionsPtr & out_expression, NameToNameMap & out_rename_map, bool & out_force_update_metadata) const { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); out_expression = nullptr; out_rename_map = {}; out_force_update_metadata = false; @@ -1507,7 +1507,7 @@ void MergeTreeData::alterDataPart( bool skip_sanity_checks, AlterDataPartTransactionPtr & transaction) { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); ExpressionActionsPtr expression; const auto & part = transaction->getDataPart(); bool force_update_metadata; @@ -1647,16 +1647,16 @@ void MergeTreeData::alterDataPart( void MergeTreeData::alterSettings( const SettingsChanges & new_changes, - const String & current_database_name, - const String & current_table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { - std::lock_guard lock(settings_mutex); - MutableMergeTreeSettingsPtr settings = std::move(*guarded_settings.getPtr()).mutate(); - settings->updateFromChanges(new_changes); - IStorage::alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); - guarded_settings.setPtr(std::move(settings)); + const String current_database_name = getDatabaseName(); + const String current_table_name = getTableName(); + + MergeTreeSettings copy = *getSettings(); + copy.updateFromChanges(new_changes); + IStorage::alterSettings(new_changes, context, table_lock_holder); + storage_settings.set(std::make_unique(copy)); } bool MergeTreeData::hasSetting(const String & setting_name) const @@ -2342,7 +2342,7 @@ std::optional MergeTreeData::getMinPartDataVersion() const void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); const size_t parts_count_in_total = getPartsCount(); if (parts_count_in_total >= settings->max_parts_in_total) { @@ -2380,7 +2380,7 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until) const void MergeTreeData::throwInsertIfNeeded() const { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); const size_t parts_count_in_total = getPartsCount(); if (parts_count_in_total >= settings->max_parts_in_total) { @@ -3075,7 +3075,7 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & bool MergeTreeData::canReplacePartition(const DataPartPtr & src_part) const { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); if (!settings->enable_mixed_granularity_parts || settings->index_granularity_bytes == 0) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 9f3d1127f2c..f759b87f986 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -331,7 +331,7 @@ public: const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - MergeTreeSettingsPtr settings_, + std::unique_ptr settings_, bool require_part_metadata_, bool attach, BrokenPartCallback broken_part_callback_ = [](const String &){}); @@ -541,8 +541,6 @@ public: /// Not atomic, have to be done with alter intention lock. void alterSettings( const SettingsChanges & new_changes, - const String & current_database_name, - const String & current_table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; @@ -620,7 +618,7 @@ public: /// Has additional constraint in replicated version virtual bool canUseAdaptiveGranularity() const { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); return settings->index_granularity_bytes != 0 && (settings->enable_mixed_granularity_parts || !has_non_adaptive_index_granularity_parts); } @@ -683,13 +681,12 @@ public: bool has_non_adaptive_index_granularity_parts = false; - /// Get copy-on-write pointer to storage settings. + /// Get constant pointer to storage settings. /// Copy this pointer into your scope and you will /// get consistent settings. - const MergeTreeSettingsPtr getCOWSettings() const + MergeTreeSettingsPtr getSettings() const { - std::shared_lock lock(settings_mutex); - return guarded_settings.copyPtr(); + return storage_settings.get(); } protected: @@ -721,26 +718,9 @@ protected: String log_name; Logger * log; - /// Just hides settings pointer from direct usage - class MergeTreeSettingsGuard - { - private: - /// Settings COW pointer. Data maybe changed at any point of time. - /// If you need consistent settings, just copy pointer to your scope. - MergeTreeSettingsPtr settings_ptr; - public: - MergeTreeSettingsGuard(MergeTreeSettingsPtr settings_ptr_) - : settings_ptr(settings_ptr_) - {} - - const MergeTreeSettingsPtr copyPtr() const { return settings_ptr; } - MergeTreeSettingsPtr getPtr() { return settings_ptr; } - void setPtr(MergeTreeSettingsPtr ptr) { settings_ptr = ptr; } - }; - - /// Storage settings. Don't use this field directly, if you - /// want readonly settings. Prefer getCOWSettings() method. - MergeTreeSettingsGuard guarded_settings; + /// Storage settings. + /// Use get and set to receive readonly versions. + MultiVersion storage_settings; /// Work with data parts @@ -829,7 +809,6 @@ protected: /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; /// Mutex for settings usage - mutable std::shared_mutex settings_mutex; void setProperties(const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast, const ColumnsDescription & new_columns, diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 312c5a195d8..ad489a91603 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -141,7 +141,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); size_t free_entries = pool_size - pool_used; - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); UInt64 max_size = 0; if (free_entries >= data_settings->number_of_free_entries_in_pool_to_lower_max_size_of_merge) @@ -159,7 +159,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() { - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); size_t total_threads_in_pool = pool.getNumberOfThreads(); size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); @@ -179,7 +179,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( String * out_disable_reason) { MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector(); - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); if (data_parts.empty()) { @@ -556,7 +556,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor Names all_column_names = data.getColumns().getNamesOfPhysical(); NamesAndTypesList all_columns = data.getColumns().getAllPhysical(); - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); NamesAndTypesList gathering_columns, merging_columns; Names gathering_column_names, merging_column_names; @@ -965,7 +965,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor const auto & updated_header = mutations_interpreter.getUpdatedHeader(); NamesAndTypesList all_columns = data.getColumns().getAllPhysical(); - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); Block in_header = in->getHeader(); @@ -1145,7 +1145,7 @@ MergeTreeDataMergerMutator::MergeAlgorithm MergeTreeDataMergerMutator::chooseMer const MergeTreeData::DataPartsVector & parts, size_t sum_rows_upper_bound, const NamesAndTypesList & gathering_columns, bool deduplicate, bool need_remove_expired_values) const { - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); if (deduplicate) return MergeAlgorithm::Horizontal; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 8cbb0819d20..40dc0bf6b52 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -673,7 +673,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( size_t sum_marks = 0; size_t total_rows = 0; - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); size_t adaptive_parts = 0; for (size_t i = 0; i < parts.size(); ++i) { @@ -832,7 +832,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO SortingInfoPtr sorting_info = query_info.sorting_info; size_t adaptive_parts = 0; std::vector sum_marks_in_parts(parts.size()); - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); for (size_t i = 0; i < parts.size(); ++i) { @@ -1035,7 +1035,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal const Names & virt_columns, const Settings & settings) const { - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); size_t sum_marks = 0; size_t adaptive_parts = 0; for (size_t i = 0; i < parts.size(); ++i) diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index 143af37c10d..c25968480c8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -25,7 +25,7 @@ std::optional MergeTreeIndexGranularityInfo::getMrkExtensionFromFS( MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo( const MergeTreeData & storage) { - const auto storage_settings = storage.getCOWSettings(); + const auto storage_settings = storage.getSettings(); fixed_index_granularity = storage_settings->index_granularity; /// Granularity is fixed if (!storage.canUseAdaptiveGranularity()) diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp index 5bc44eee842..9eee33554ab 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -75,9 +75,4 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) #undef ADD_IF_ABSENT } - -MergeTreeSettings::MutablePtr MergeTreeSettings::clone() const -{ - return COW::create(*this); -} } diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index 68d3e1ec66c..6ba08fed5da 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -2,7 +2,6 @@ #include #include -#include namespace Poco @@ -22,11 +21,9 @@ 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, public COW +struct MergeTreeSettings : public SettingsCollection { - friend class COW; - /// M (mutable) for normal settings, IM (immutable) for not updateable settings. #define LIST_OF_MERGE_TREE_SETTINGS(M, IM) \ IM(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.") \ @@ -101,14 +98,8 @@ struct MergeTreeSettings : public SettingsCollection, public /// NOTE: will rewrite the AST to add immutable settings. void loadFromQuery(ASTStorage & storage_def); - - MutablePtr clone() const; -private: - MergeTreeSettings() = default; - MergeTreeSettings(const MergeTreeSettings & o) = default; }; -using MergeTreeSettingsPtr = MergeTreeSettings::Ptr; -using MutableMergeTreeSettingsPtr = MergeTreeSettings::MutablePtr; +using MergeTreeSettingsPtr = std::shared_ptr; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp index 7a09bde0998..69cf173212d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp @@ -31,7 +31,7 @@ MergeTreeThreadSelectBlockInputStream::MergeTreeThreadSelectBlockInputStream( /// Maybe it will make sence to add settings `max_block_size_bytes` if (max_block_size_rows && !storage.canUseAdaptiveGranularity()) { - size_t fixed_index_granularity = storage.getCOWSettings()->index_granularity; + size_t fixed_index_granularity = storage.getSettings()->index_granularity; min_marks_to_read = (min_marks_to_read_ * fixed_index_granularity + max_block_size_rows - 1) / max_block_size_rows * max_block_size_rows / fixed_index_granularity; } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 2b03ed86895..b89fd9629c0 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -27,7 +27,7 @@ ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplic void ReplicatedMergeTreeCleanupThread::run() { - auto storage_settings = storage.getCOWSettings(); + auto storage_settings = storage.getSettings(); const auto CLEANUP_SLEEP_MS = storage_settings->cleanup_delay_period * 1000 + std::uniform_int_distribution(0, storage_settings->cleanup_delay_period_random_add * 1000)(rng); @@ -75,7 +75,7 @@ void ReplicatedMergeTreeCleanupThread::iterate() void ReplicatedMergeTreeCleanupThread::clearOldLogs() { auto zookeeper = storage.getZooKeeper(); - auto storage_settings = storage.getCOWSettings(); + auto storage_settings = storage.getSettings(); Coordination::Stat stat; if (!zookeeper->exists(storage.zookeeper_path + "/log", &stat)) @@ -287,7 +287,7 @@ struct ReplicatedMergeTreeCleanupThread::NodeWithStat void ReplicatedMergeTreeCleanupThread::clearOldBlocks() { auto zookeeper = storage.getZooKeeper(); - auto storage_settings = storage.getCOWSettings(); + auto storage_settings = storage.getSettings(); std::vector timed_blocks; getBlocksSortedByTime(*zookeeper, timed_blocks); @@ -404,7 +404,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper & void ReplicatedMergeTreeCleanupThread::clearOldMutations() { - auto storage_settings = storage.getCOWSettings(); + auto storage_settings = storage.getSettings(); if (!storage_settings->finished_mutations_to_keep) return; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index d8255add55a..e8cfeb78963 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -964,7 +964,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( * Setting max_bytes_to_merge_at_max_space_in_pool still working for regular merges, * because the leader replica does not assign merges of greater size (except OPTIMIZE PARTITION and OPTIMIZE FINAL). */ - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); bool ignore_max_size = (entry.type == LogEntry::MERGE_PARTS) && (max_source_parts_size == data_settings->max_bytes_to_merge_at_max_space_in_pool); if (!ignore_max_size && sum_parts_size_in_bytes > max_source_parts_size) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 6145713492f..c766219a349 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -44,7 +44,7 @@ ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(Storage , log(&Logger::get(log_name)) , active_node_identifier(generateActiveNodeIdentifier()) { - const auto storage_settings = storage.getCOWSettings(); + const auto storage_settings = storage.getSettings(); check_period_ms = storage_settings->zookeeper_session_expiration_check_period.totalSeconds() * 1000; /// Periodicity of checking lag of replica. @@ -122,7 +122,7 @@ void ReplicatedMergeTreeRestartingThread::run() } time_t current_time = time(nullptr); - const auto storage_settings = storage.getCOWSettings(); + const auto storage_settings = storage.getSettings(); if (current_time >= prev_time_of_check_delay + static_cast(storage_settings->check_delay_period)) { /// Find out lag of replicas. @@ -171,7 +171,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() activateReplica(); const auto & zookeeper = storage.getZooKeeper(); - const auto storage_settings = storage.getCOWSettings(); + const auto storage_settings = storage.getSettings(); storage.cloneReplicaIfNeeded(zookeeper); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 6b12df91546..5ada5d50b23 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -27,7 +27,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos]; - const auto data_settings = data.getCOWSettings(); + const auto data_settings = data.getSettings(); sampling_expression = formattedAST(data.sample_by_ast); index_granularity = data_settings->index_granularity; merging_params_mode = static_cast(data.merging_params.mode); diff --git a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 32268499a01..0e15a5660a9 100644 --- a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -14,6 +14,7 @@ namespace DB /// A Storage that allows reading from a single MergeTree data part. class StorageFromMergeTreeDataPart : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: String getName() const override { return "FromMergeTreeDataPart"; } String getTableName() const override { return part->storage.getTableName() + " (part " + part->name + ")"; } diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 4cc9ec1db2a..674116a54bc 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -574,7 +574,9 @@ static StoragePtr create(const StorageFactory::Arguments & args) ASTPtr sample_by_ast; ASTPtr ttl_table_ast; IndicesDescription indices_description; - MutableMergeTreeSettingsPtr storage_settings = MergeTreeSettings::create(args.context.getMergeTreeSettings()); + ConstraintsDescription constraints_description; + + std::unique_ptr storage_settings = std::make_unique(args.context.getMergeTreeSettings()); if (is_extended_storage_def) { diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 49a2dabe380..bb4ccf8720e 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -695,10 +695,13 @@ void StorageBuffer::flushThread() } -void StorageBuffer::alter(const AlterCommands & params, const String & database_name_, const String & table_name_, const Context & context, TableStructureWriteLockHolder & table_lock_holder) +void StorageBuffer::alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); + const String database_name_ = getDatabaseName(); + const String table_name_ = getTableName(); + /// So that no blocks of the old structure remain. optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context); diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index 8060199d20d..b81ca42a0eb 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -39,6 +39,7 @@ class Context; */ class StorageBuffer : public ext::shared_ptr_helper, public IStorage { +friend struct ext::shared_ptr_helper; friend class BufferBlockInputStream; friend class BufferBlockOutputStream; @@ -91,8 +92,9 @@ public: /// The structure of the subordinate table is not checked and does not change. void alter( - const AlterCommands & params, const String & database_name, const String & table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + + ~StorageBuffer() override; private: String table_name; @@ -145,8 +147,6 @@ protected: Context & context_, size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_, const String & destination_database_, const String & destination_table_, bool allow_materialized_); - - ~StorageBuffer() override; }; } diff --git a/dbms/src/Storages/StorageDictionary.h b/dbms/src/Storages/StorageDictionary.h index cf5fd647e74..097e81d15c6 100644 --- a/dbms/src/Storages/StorageDictionary.h +++ b/dbms/src/Storages/StorageDictionary.h @@ -21,6 +21,7 @@ class ExternalDictionaries; class StorageDictionary : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Dictionary"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 90a973ae3b9..b2e632f9414 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -266,11 +266,8 @@ StoragePtr StorageDistributed::createWithOwnCluster( ClusterPtr owned_cluster_, const Context & context_) { - auto res = ext::shared_ptr_helper::create( - String{}, table_name_, columns_, ConstraintsDescription{}, remote_database_, remote_table_, String{}, context_, ASTPtr(), String(), false); - + auto res = create(String{}, table_name_, columns_, ConstraintsDescription{}, remote_database_, remote_table_, String{}, context_, ASTPtr(), String(), false); res->owned_cluster = owned_cluster_; - return res; } @@ -282,11 +279,8 @@ StoragePtr StorageDistributed::createWithOwnCluster( ClusterPtr & owned_cluster_, const Context & context_) { - auto res = ext::shared_ptr_helper::create( - String{}, table_name_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), false); - + auto res = create(String{}, table_name_, columns_, ConstraintsDescription{}, remote_table_function_ptr_, String{}, context_, ASTPtr(), String(), false); res->owned_cluster = owned_cluster_; - return res; } @@ -392,11 +386,13 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const Context & c void StorageDistributed::alter( - const AlterCommands & params, const String & current_database_name, const String & current_table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); + const String current_database_name = getDatabaseName(); + const String current_table_name = getTableName(); + auto new_columns = getColumns(); auto new_indices = getIndices(); auto new_constraints = getConstraints(); diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index e8da6dca4a7..3261f5a0beb 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -29,6 +29,7 @@ class StorageDistributedDirectoryMonitor; */ class StorageDistributed : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; friend class DistributedBlockOutputStream; friend class StorageDistributedDirectoryMonitor; @@ -85,8 +86,7 @@ public: /// in the sub-tables, you need to manually add and delete columns /// the structure of the sub-table is not checked void alter( - const AlterCommands & params, const String & database_name, const String & table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; void startup() override; void shutdown() override; diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index 237e9a6989e..7268c8ddff0 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -20,6 +20,7 @@ class StorageFileBlockOutputStream; class StorageFile : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "File"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/StorageHDFS.h b/dbms/src/Storages/StorageHDFS.h index dae7000dd8a..30a99c9de70 100644 --- a/dbms/src/Storages/StorageHDFS.h +++ b/dbms/src/Storages/StorageHDFS.h @@ -15,6 +15,7 @@ namespace DB */ class StorageHDFS : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: String getName() const override { return "HDFS"; } String getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/StorageJoin.h b/dbms/src/Storages/StorageJoin.h index c9e1283b90f..760e9eb1815 100644 --- a/dbms/src/Storages/StorageJoin.h +++ b/dbms/src/Storages/StorageJoin.h @@ -22,6 +22,7 @@ using JoinPtr = std::shared_ptr; */ class StorageJoin : public ext::shared_ptr_helper, public StorageSetOrJoinBase { + friend struct ext::shared_ptr_helper; public: String getName() const override { return "Join"; } diff --git a/dbms/src/Storages/StorageLog.h b/dbms/src/Storages/StorageLog.h index e646ecd8c46..70de62cb47b 100644 --- a/dbms/src/Storages/StorageLog.h +++ b/dbms/src/Storages/StorageLog.h @@ -21,6 +21,7 @@ class StorageLog : public ext::shared_ptr_helper, public IStorage { friend class LogBlockInputStream; friend class LogBlockOutputStream; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Log"; } diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index 03c93d8d29f..b635634addf 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -11,6 +11,7 @@ namespace DB class StorageMaterializedView : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "MaterializedView"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index 88e9e5dcf04..92dcd4be18f 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -21,6 +21,7 @@ class StorageMemory : public ext::shared_ptr_helper, public IStor { friend class MemoryBlockInputStream; friend class MemoryBlockOutputStream; +friend struct ext::shared_ptr_helper; public: String getName() const override { return "Memory"; } diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 55206d02a0a..6215550e413 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -397,8 +397,7 @@ DatabaseIteratorPtr StorageMerge::getDatabaseIterator(const Context & context) c void StorageMerge::alter( - const AlterCommands & params, const String & database_name_, const String & table_name_, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); @@ -406,7 +405,7 @@ void StorageMerge::alter( auto new_indices = getIndices(); auto new_constraints = getConstraints(); params.applyForColumnsOnly(new_columns); - context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, new_constraints, {}); + context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {}); setColumns(new_columns); } diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index 6d02ad029cc..6708a92c3b0 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -14,6 +14,7 @@ namespace DB */ class StorageMerge : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Merge"; } std::string getTableName() const override { return table_name; } @@ -47,8 +48,7 @@ public: /// you need to add and remove columns in the sub-tables manually /// the structure of sub-tables is not checked void alter( - const AlterCommands & params, const String & database_name, const String & table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index e7be6a35cb9..1504413801c 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -63,14 +63,14 @@ StorageMergeTree::StorageMergeTree( const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - MergeTreeSettingsPtr settings_, + std::unique_ptr storage_settings_, bool has_force_restore_data_flag) : MergeTreeData(database_name_, table_name_, path_ + escapeForFileName(table_name_) + '/', columns_, indices_, constraints_, context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, sample_by_ast_, ttl_table_ast_, merging_params_, - settings_, false, attach), + std::move(storage_settings_), false, attach), path(path_), background_pool(context_.getBackgroundPool()), reader(*this), writer(*this), merger_mutator(*this, global_context.getBackgroundPool()) @@ -245,11 +245,12 @@ std::vector StorageMergeTree::prepar void StorageMergeTree::alter( const AlterCommands & params, - const String & current_database_name, - const String & current_table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { + const String current_database_name = getDatabaseName(); + const String current_table_name = getTableName(); + if (!params.isMutable()) { SettingsChanges new_changes; @@ -257,7 +258,7 @@ void StorageMergeTree::alter( if (params.isSettingsAlter()) { params.applyForSettingsOnly(new_changes); - alterSettings(new_changes, current_database_name, current_table_name, context, table_lock_holder); + alterSettings(new_changes, context, table_lock_holder); return; } @@ -804,7 +805,7 @@ Int64 StorageMergeTree::getCurrentMutationVersion( void StorageMergeTree::clearOldMutations() { - const auto settings = getCOWSettings(); + const auto settings = getSettings(); if (!settings->finished_mutations_to_keep) return; diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 9f51a9357ce..d135ffd6e1a 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -23,6 +23,7 @@ namespace DB */ class StorageMergeTree : public ext::shared_ptr_helper, public MergeTreeData { + friend struct ext::shared_ptr_helper; public: void startup() override; void shutdown() override; @@ -59,9 +60,7 @@ public: void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; - void alter( - const AlterCommands & params, const String & database_name, const String & table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; void checkTableCanBeDropped() const override; @@ -153,7 +152,7 @@ protected: const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - MergeTreeSettingsPtr settings_, + std::unique_ptr settings_, bool has_force_restore_data_flag); }; diff --git a/dbms/src/Storages/StorageMySQL.h b/dbms/src/Storages/StorageMySQL.h index 320eab9d4a9..5f369da4d25 100644 --- a/dbms/src/Storages/StorageMySQL.h +++ b/dbms/src/Storages/StorageMySQL.h @@ -18,6 +18,7 @@ namespace DB */ class StorageMySQL : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: StorageMySQL( const std::string & database_name_, diff --git a/dbms/src/Storages/StorageNull.cpp b/dbms/src/Storages/StorageNull.cpp index 14749000a04..d3b97f9ad46 100644 --- a/dbms/src/Storages/StorageNull.cpp +++ b/dbms/src/Storages/StorageNull.cpp @@ -31,11 +31,13 @@ void registerStorageNull(StorageFactory & factory) } void StorageNull::alter( - const AlterCommands & params, const String & current_database_name, const String & current_table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); + const String current_database_name = getDatabaseName(); + const String current_table_name = getTableName(); + ColumnsDescription new_columns = getColumns(); IndicesDescription new_indices = getIndices(); ConstraintsDescription new_constraints = getConstraints(); diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index d858f738b24..04cd5f25e8f 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -16,6 +16,7 @@ namespace DB */ class StorageNull : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Null"; } std::string getTableName() const override { return table_name; } @@ -44,8 +45,7 @@ public: } void alter( - const AlterCommands & params, const String & database_name, const String & table_name, - const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; private: String table_name; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index ce9448dd0e7..cb3828f5817 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -202,14 +202,14 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const ASTPtr & sample_by_ast_, const ASTPtr & ttl_table_ast_, const MergingParams & merging_params_, - MergeTreeSettingsPtr settings_, + std::unique_ptr settings_, bool has_force_restore_data_flag) : MergeTreeData(database_name_, table_name_, path_ + escapeForFileName(table_name_) + '/', columns_, indices_, constraints_, context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, sample_by_ast_, ttl_table_ast_, merging_params_, - settings_, true, attach, + std::move(settings_), true, attach, [this] (const std::string & name) { enqueuePartForCheck(name); }), zookeeper_path(global_context.getMacros()->expand(zookeeper_path_, database_name_, table_name_)), replica_name(global_context.getMacros()->expand(replica_name_, database_name_, table_name_)), @@ -376,7 +376,7 @@ void StorageReplicatedMergeTree::createTableIfNotExists() } -/** Verify that list of columns and table storage_settings match those specified in ZK (/ metadata). +/** Verify that list of columns and table storage_settings_ptr match those specified in ZK (/ metadata). * If not, throw an exception. */ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bool allow_alter) @@ -637,8 +637,8 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) for (const auto & part : parts) total_rows_on_filesystem += part->rows_count; - const auto storage_settings = getCOWSettings(); - bool insane = unexpected_parts_rows > total_rows_on_filesystem * storage_settings->replicated_max_ratio_of_wrong_parts; + const auto storage_settings_ptr = getSettings(); + bool insane = unexpected_parts_rows > total_rows_on_filesystem * storage_settings_ptr->replicated_max_ratio_of_wrong_parts; if (insane && !skip_sanity_checks) { @@ -781,13 +781,13 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: if (!has_been_already_added) { - const auto storage_settings = getCOWSettings(); + const auto storage_settings_ptr = getSettings(); String part_path = replica_path + "/parts/" + part_name; ops.emplace_back(zkutil::makeCheckRequest( zookeeper_path + "/columns", expected_columns_version)); - if (storage_settings->use_minimalistic_part_header_in_zookeeper) + if (storage_settings_ptr->use_minimalistic_part_header_in_zookeeper) { ops.emplace_back(zkutil::makeCreateRequest( part_path, local_part_header.toString(), zkutil::CreateMode::Persistent)); @@ -864,7 +864,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd String StorageReplicatedMergeTree::getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums) const { return MinimalisticDataPartChecksums::getSerializedString(checksums, - getCOWSettings()->use_minimalistic_checksums_in_zookeeper); + getSettings()->use_minimalistic_checksums_in_zookeeper); } @@ -1035,14 +1035,14 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) parts.push_back(part); } - const auto storage_settings = getCOWSettings(); + const auto storage_settings_ptr = getSettings(); if (!have_all_parts) { /// If you do not have all the necessary parts, try to take some already merged part from someone. LOG_DEBUG(log, "Don't have all parts for merge " << entry.new_part_name << "; will try to fetch it instead"); return false; } - else if (entry.create_time + storage_settings->prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr)) + else if (entry.create_time + storage_settings_ptr->prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr)) { /// If entry is old enough, and have enough size, and part are exists in any replica, /// then prefer fetching of merged part from replica. @@ -1051,7 +1051,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) for (const auto & part : parts) sum_parts_bytes_on_disk += part->bytes_on_disk; - if (sum_parts_bytes_on_disk >= storage_settings->prefer_fetch_merged_part_size_threshold) + if (sum_parts_bytes_on_disk >= storage_settings_ptr->prefer_fetch_merged_part_size_threshold) { String replica = findReplicaHavingPart(entry.new_part_name, true); /// NOTE excessive ZK requests for same data later, may remove. if (!replica.empty()) @@ -1161,7 +1161,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedMergeTree::LogEntry & entry) { const String & source_part_name = entry.source_parts.at(0); - const auto storage_settings = getCOWSettings(); + const auto storage_settings_ptr = getSettings(); LOG_TRACE(log, "Executing log entry to mutate part " << source_part_name << " to " << entry.new_part_name); DataPartPtr source_part = getActiveContainingPart(source_part_name); @@ -1181,8 +1181,8 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM /// TODO - some better heuristic? size_t estimated_space_for_result = MergeTreeDataMergerMutator::estimateNeededDiskSpace({source_part}); - if (entry.create_time + storage_settings->prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr) - && estimated_space_for_result >= storage_settings->prefer_fetch_merged_part_size_threshold) + if (entry.create_time + storage_settings_ptr->prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr) + && estimated_space_for_result >= storage_settings_ptr->prefer_fetch_merged_part_size_threshold) { /// If entry is old enough, and have enough size, and some replica has the desired part, /// then prefer fetching from replica. @@ -1276,21 +1276,21 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) { String replica = findReplicaHavingCoveringPart(entry, true); - const auto storage_settings = getCOWSettings(); + const auto storage_settings_ptr = getSettings(); static std::atomic_uint total_fetches {0}; - if (storage_settings->replicated_max_parallel_fetches && total_fetches >= storage_settings->replicated_max_parallel_fetches) + if (storage_settings_ptr->replicated_max_parallel_fetches && total_fetches >= storage_settings_ptr->replicated_max_parallel_fetches) { - throw Exception("Too many total fetches from replicas, maximum: " + storage_settings->replicated_max_parallel_fetches.toString(), + throw Exception("Too many total fetches from replicas, maximum: " + storage_settings_ptr->replicated_max_parallel_fetches.toString(), ErrorCodes::TOO_MANY_FETCHES); } ++total_fetches; SCOPE_EXIT({--total_fetches;}); - if (storage_settings->replicated_max_parallel_fetches_for_table && current_table_fetches >= storage_settings->replicated_max_parallel_fetches_for_table) + if (storage_settings_ptr->replicated_max_parallel_fetches_for_table && current_table_fetches >= storage_settings_ptr->replicated_max_parallel_fetches_for_table) { - throw Exception("Too many fetches from replicas for table, maximum: " + storage_settings->replicated_max_parallel_fetches_for_table.toString(), + throw Exception("Too many fetches from replicas for table, maximum: " + storage_settings_ptr->replicated_max_parallel_fetches_for_table.toString(), ErrorCodes::TOO_MANY_FETCHES); } @@ -2213,7 +2213,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() if (!is_leader) return; - const auto storage_settings = getCOWSettings(); + const auto storage_settings_ptr = getSettings(); const bool deduplicate = false; /// TODO: read deduplicate option from table config const bool force_ttl = false; @@ -2235,16 +2235,16 @@ void StorageReplicatedMergeTree::mergeSelectingTask() auto merges_and_mutations_queued = queue.countMergesAndPartMutations(); size_t merges_and_mutations_sum = merges_and_mutations_queued.first + merges_and_mutations_queued.second; - if (merges_and_mutations_sum >= storage_settings->max_replicated_merges_in_queue) + if (merges_and_mutations_sum >= storage_settings_ptr->max_replicated_merges_in_queue) { LOG_TRACE(log, "Number of queued merges (" << merges_and_mutations_queued.first << ") and part mutations (" << merges_and_mutations_queued.second << ") is greater than max_replicated_merges_in_queue (" - << storage_settings->max_replicated_merges_in_queue << "), so won't select new parts to merge or mutate."); + << storage_settings_ptr->max_replicated_merges_in_queue << "), so won't select new parts to merge or mutate."); } else { UInt64 max_source_parts_size_for_merge = merger_mutator.getMaxSourcePartsSizeForMerge( - storage_settings->max_replicated_merges_in_queue, merges_and_mutations_sum); + storage_settings_ptr->max_replicated_merges_in_queue, merges_and_mutations_sum); UInt64 max_source_part_size_for_mutation = merger_mutator.getMaxSourcePartSizeForMutation(); FutureMergedMutatedPart future_merged_part; @@ -2256,7 +2256,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() } /// If there are many mutations in queue it may happen, that we cannot enqueue enough merges to merge all new parts else if (max_source_part_size_for_mutation > 0 && queue.countMutations() > 0 - && merges_and_mutations_queued.second < storage_settings->max_replicated_mutations_in_queue) + && merges_and_mutations_queued.second < storage_settings_ptr->max_replicated_mutations_in_queue) { /// Choose a part to mutate. DataPartsVector data_parts = getDataPartsVector(); @@ -3029,11 +3029,11 @@ void StorageReplicatedMergeTree::assertNotReadonly() const BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const Context & context) { - const auto storage_settings = getCOWSettings(); + const auto storage_settings_ptr = getSettings(); assertNotReadonly(); const Settings & query_settings = context.getSettingsRef(); - bool deduplicate = storage_settings->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; + bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; return std::make_shared(*this, query_settings.insert_quorum, query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, deduplicate); @@ -3067,7 +3067,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p }; bool force_ttl = (final && (hasTableTTL() || hasAnyColumnTTL())); - const auto storage_settings = getCOWSettings(); + const auto storage_settings_ptr = getSettings(); if (!partition && final) { @@ -3100,7 +3100,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p if (!partition) { selected = merger_mutator.selectPartsToMerge( - future_merged_part, true, storage_settings->max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason); + future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason); } else { @@ -3142,21 +3142,23 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p void StorageReplicatedMergeTree::alter( - const AlterCommands & params, const String & current_database_name, const String & current_table_name, - const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) + const AlterCommands & params, const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) { assertNotReadonly(); LOG_DEBUG(log, "Doing ALTER"); + const String current_database_name = getDatabaseName(); + const String current_table_name = getTableName(); + if (params.isSettingsAlter()) { - /// We don't replicate storage_settings ALTER. It's local operation. + /// We don't replicate storage_settings_ptr ALTER. It's local operation. /// Also we don't upgrade alter lock to table structure lock. - LOG_DEBUG(log, "ALTER storage_settings only"); + LOG_DEBUG(log, "ALTER storage_settings_ptr only"); SettingsChanges new_changes; params.applyForSettingsOnly(new_changes); - alterSettings(new_changes, current_database_name, current_table_name, query_context, table_lock_holder); + alterSettings(new_changes, query_context, table_lock_holder); return; } @@ -3948,10 +3950,10 @@ void StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(const String & void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) { auto zookeeper = tryGetZooKeeper(); - const auto storage_settings = getCOWSettings(); + const auto storage_settings_ptr = getSettings(); res.is_leader = is_leader; - res.can_become_leader = storage_settings->replicated_can_become_leader; + res.can_become_leader = storage_settings_ptr->replicated_can_become_leader; res.is_readonly = is_readonly; res.is_session_expired = !zookeeper || zookeeper->expired(); @@ -4141,14 +4143,14 @@ void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, t out_absolute_delay = getAbsoluteDelay(); out_relative_delay = 0; - const auto storage_settings = getCOWSettings(); + const auto storage_settings_ptr = getSettings(); /** Relative delay is the maximum difference of absolute delay from any other replica, * (if this replica lags behind any other live replica, or zero, otherwise). * Calculated only if the absolute delay is large enough. */ - if (out_absolute_delay < static_cast(storage_settings->min_relative_delay_to_yield_leadership)) + if (out_absolute_delay < static_cast(storage_settings_ptr->min_relative_delay_to_yield_leadership)) return; auto zookeeper = getZooKeeper(); @@ -4969,7 +4971,7 @@ void StorageReplicatedMergeTree::getCommitPartOps( const String & block_id_path) const { const String & part_name = part->name; - const auto storage_settings = getCOWSettings(); + const auto storage_settings_ptr = getSettings(); if (!block_id_path.empty()) { @@ -4987,7 +4989,7 @@ void StorageReplicatedMergeTree::getCommitPartOps( zookeeper_path + "/columns", columns_version)); - if (storage_settings->use_minimalistic_part_header_in_zookeeper) + if (storage_settings_ptr->use_minimalistic_part_header_in_zookeeper) { ops.emplace_back(zkutil::makeCreateRequest( replica_path + "/parts/" + part->name, @@ -5016,12 +5018,12 @@ void StorageReplicatedMergeTree::updatePartHeaderInZooKeeperAndCommit( AlterDataPartTransaction & transaction) { String part_path = replica_path + "/parts/" + transaction.getPartName(); - const auto storage_settings = getCOWSettings(); + const auto storage_settings_ptr = getSettings(); bool need_delete_columns_and_checksums_nodes = false; try { - if (storage_settings->use_minimalistic_part_header_in_zookeeper) + if (storage_settings_ptr->use_minimalistic_part_header_in_zookeeper) { auto part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums( transaction.getNewColumns(), transaction.getNewChecksums()); @@ -5201,9 +5203,9 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, const C bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const { - const auto storage_settings = getCOWSettings(); - return storage_settings->index_granularity_bytes != 0 && - (storage_settings->enable_mixed_granularity_parts || + const auto storage_settings_ptr = getSettings(); + return storage_settings_ptr->index_granularity_bytes != 0 && + (storage_settings_ptr->enable_mixed_granularity_parts || (!has_non_adaptive_index_granularity_parts && !other_replicas_fixed_granularity)); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 5d8090a8cbc..0fa2e3631e2 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -74,6 +74,7 @@ namespace DB class StorageReplicatedMergeTree : public ext::shared_ptr_helper, public MergeTreeData { + friend struct ext::shared_ptr_helper; public: void startup() override; void shutdown() override; @@ -98,9 +99,7 @@ public: bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & query_context) override; - void alter( - const AlterCommands & params, const String & database_name, const String & table_name, - const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) override; + void alter(const AlterCommands & params, const Context & query_context, TableStructureWriteLockHolder & table_lock_holder) override; void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & query_context) override; @@ -540,7 +539,7 @@ protected: const ASTPtr & sample_by_ast_, const ASTPtr & table_ttl_ast_, const MergingParams & merging_params_, - MergeTreeSettingsPtr settings_, + std::unique_ptr settings_, bool has_force_restore_data_flag); }; diff --git a/dbms/src/Storages/StorageStripeLog.h b/dbms/src/Storages/StorageStripeLog.h index 22be356713b..d53f000f277 100644 --- a/dbms/src/Storages/StorageStripeLog.h +++ b/dbms/src/Storages/StorageStripeLog.h @@ -23,6 +23,7 @@ class StorageStripeLog : public ext::shared_ptr_helper, public { friend class StripeLogBlockInputStream; friend class StripeLogBlockOutputStream; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "StripeLog"; } diff --git a/dbms/src/Storages/StorageTinyLog.h b/dbms/src/Storages/StorageTinyLog.h index 7a37a42ec63..1c148acf957 100644 --- a/dbms/src/Storages/StorageTinyLog.h +++ b/dbms/src/Storages/StorageTinyLog.h @@ -22,6 +22,7 @@ class StorageTinyLog : public ext::shared_ptr_helper, public ISt { friend class TinyLogBlockInputStream; friend class TinyLogBlockOutputStream; +friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "TinyLog"; } diff --git a/dbms/src/Storages/StorageURL.h b/dbms/src/Storages/StorageURL.h index 0c4b4648223..2facca8ce38 100644 --- a/dbms/src/Storages/StorageURL.h +++ b/dbms/src/Storages/StorageURL.h @@ -71,6 +71,7 @@ private: class StorageURL : public ext::shared_ptr_helper, public IStorageURLBase { + friend struct ext::shared_ptr_helper; public: StorageURL( const Poco::URI & uri_, diff --git a/dbms/src/Storages/StorageValues.h b/dbms/src/Storages/StorageValues.h index 36c3bc15301..c07cf7cbb63 100644 --- a/dbms/src/Storages/StorageValues.h +++ b/dbms/src/Storages/StorageValues.h @@ -11,6 +11,7 @@ namespace DB */ class StorageValues : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "Values"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/StorageView.h b/dbms/src/Storages/StorageView.h index cda128027c2..de56f120fa1 100644 --- a/dbms/src/Storages/StorageView.h +++ b/dbms/src/Storages/StorageView.h @@ -12,6 +12,7 @@ namespace DB class StorageView : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "View"; } std::string getTableName() const override { return table_name; } diff --git a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h index 1d7226eda8b..86585f0ea73 100644 --- a/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h +++ b/dbms/src/Storages/System/StorageSystemAggregateFunctionCombinators.h @@ -4,11 +4,13 @@ #include #include #include + namespace DB { class StorageSystemAggregateFunctionCombinators : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h index 853cb97c974..77f247f89eb 100644 --- a/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h +++ b/dbms/src/Storages/System/StorageSystemAsynchronousMetrics.h @@ -12,8 +12,10 @@ class Context; /** Implements system table asynchronous_metrics, which allows to get values of periodically (asynchronously) updated metrics. */ -class StorageSystemAsynchronousMetrics : public ext::shared_ptr_helper, public IStorageSystemOneBlock +class StorageSystemAsynchronousMetrics : public ext::shared_ptr_helper, + public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemAsynchronousMetrics"; } diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.h b/dbms/src/Storages/System/StorageSystemBuildOptions.h index 749ffbddbaf..d81682765a2 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.h +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemBuildOptions : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemClusters.h b/dbms/src/Storages/System/StorageSystemClusters.h index dde9e53b626..8efb148cdb9 100644 --- a/dbms/src/Storages/System/StorageSystemClusters.h +++ b/dbms/src/Storages/System/StorageSystemClusters.h @@ -17,6 +17,7 @@ class Context; */ class StorageSystemClusters : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemClusters"; } diff --git a/dbms/src/Storages/System/StorageSystemCollations.h b/dbms/src/Storages/System/StorageSystemCollations.h index f8b7b6ee3af..eccce154047 100644 --- a/dbms/src/Storages/System/StorageSystemCollations.h +++ b/dbms/src/Storages/System/StorageSystemCollations.h @@ -8,6 +8,7 @@ namespace DB class StorageSystemCollations : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemColumns.h b/dbms/src/Storages/System/StorageSystemColumns.h index b9aa04b0b25..b569c2bafff 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.h +++ b/dbms/src/Storages/System/StorageSystemColumns.h @@ -13,6 +13,7 @@ class Context; */ class StorageSystemColumns : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemColumns"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemContributors.h b/dbms/src/Storages/System/StorageSystemContributors.h index 4e2a47960f3..4fc91ce1fbd 100644 --- a/dbms/src/Storages/System/StorageSystemContributors.h +++ b/dbms/src/Storages/System/StorageSystemContributors.h @@ -14,6 +14,7 @@ class Context; class StorageSystemContributors : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h index 365e2790699..44dd59d471f 100644 --- a/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h +++ b/dbms/src/Storages/System/StorageSystemDataTypeFamilies.h @@ -9,6 +9,7 @@ namespace DB class StorageSystemDataTypeFamilies : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemDatabases.h b/dbms/src/Storages/System/StorageSystemDatabases.h index c83f5a72efc..957fcae2707 100644 --- a/dbms/src/Storages/System/StorageSystemDatabases.h +++ b/dbms/src/Storages/System/StorageSystemDatabases.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemDatabases : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { diff --git a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp index ce1bfc6fc21..b95a299af68 100644 --- a/dbms/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/dbms/src/Storages/System/StorageSystemDetachedParts.cpp @@ -21,6 +21,7 @@ class StorageSystemDetachedParts : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemDetachedParts"; } std::string getTableName() const override { return "detached_parts"; } diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.h b/dbms/src/Storages/System/StorageSystemDictionaries.h index 87df9ceada7..6b28f03d917 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.h +++ b/dbms/src/Storages/System/StorageSystemDictionaries.h @@ -12,6 +12,7 @@ class Context; class StorageSystemDictionaries : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemDictionaries"; } diff --git a/dbms/src/Storages/System/StorageSystemEvents.h b/dbms/src/Storages/System/StorageSystemEvents.h index 5b02b7739f1..88753d3ea4f 100644 --- a/dbms/src/Storages/System/StorageSystemEvents.h +++ b/dbms/src/Storages/System/StorageSystemEvents.h @@ -13,6 +13,7 @@ class Context; */ class StorageSystemEvents : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemEvents"; } diff --git a/dbms/src/Storages/System/StorageSystemFormats.h b/dbms/src/Storages/System/StorageSystemFormats.h index 82f8303b5b0..61bd9ebeb8f 100644 --- a/dbms/src/Storages/System/StorageSystemFormats.h +++ b/dbms/src/Storages/System/StorageSystemFormats.h @@ -7,6 +7,7 @@ namespace DB { class StorageSystemFormats : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemFunctions.h b/dbms/src/Storages/System/StorageSystemFunctions.h index baead3d8186..1ae4483583f 100644 --- a/dbms/src/Storages/System/StorageSystemFunctions.h +++ b/dbms/src/Storages/System/StorageSystemFunctions.h @@ -15,6 +15,7 @@ class Context; */ class StorageSystemFunctions : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemFunctions"; } diff --git a/dbms/src/Storages/System/StorageSystemGraphite.h b/dbms/src/Storages/System/StorageSystemGraphite.h index b874e294782..5ba7b7bda8c 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.h +++ b/dbms/src/Storages/System/StorageSystemGraphite.h @@ -13,6 +13,7 @@ namespace DB /// Provides information about Graphite configuration. class StorageSystemGraphite : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemGraphite"; } diff --git a/dbms/src/Storages/System/StorageSystemMacros.h b/dbms/src/Storages/System/StorageSystemMacros.h index fdc091dfe1b..09cc8e3a2d9 100644 --- a/dbms/src/Storages/System/StorageSystemMacros.h +++ b/dbms/src/Storages/System/StorageSystemMacros.h @@ -15,6 +15,7 @@ class Context; */ class StorageSystemMacros : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemMacros"; } diff --git a/dbms/src/Storages/System/StorageSystemMergeTreeSettings.h b/dbms/src/Storages/System/StorageSystemMergeTreeSettings.h index 780390dd485..cb3668f13ec 100644 --- a/dbms/src/Storages/System/StorageSystemMergeTreeSettings.h +++ b/dbms/src/Storages/System/StorageSystemMergeTreeSettings.h @@ -15,6 +15,7 @@ class Context; */ class SystemMergeTreeSettings : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemMergeTreeSettings"; } diff --git a/dbms/src/Storages/System/StorageSystemMerges.h b/dbms/src/Storages/System/StorageSystemMerges.h index f45f895d661..6d970b8797f 100644 --- a/dbms/src/Storages/System/StorageSystemMerges.h +++ b/dbms/src/Storages/System/StorageSystemMerges.h @@ -15,6 +15,7 @@ class Context; class StorageSystemMerges : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemMerges"; } diff --git a/dbms/src/Storages/System/StorageSystemMetrics.h b/dbms/src/Storages/System/StorageSystemMetrics.h index f74db926126..86d95800a31 100644 --- a/dbms/src/Storages/System/StorageSystemMetrics.h +++ b/dbms/src/Storages/System/StorageSystemMetrics.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemMetrics : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemMetrics"; } diff --git a/dbms/src/Storages/System/StorageSystemModels.h b/dbms/src/Storages/System/StorageSystemModels.h index ef30bd511ea..8bcb70da35e 100644 --- a/dbms/src/Storages/System/StorageSystemModels.h +++ b/dbms/src/Storages/System/StorageSystemModels.h @@ -12,6 +12,7 @@ class Context; class StorageSystemModels : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemModels"; } diff --git a/dbms/src/Storages/System/StorageSystemMutations.h b/dbms/src/Storages/System/StorageSystemMutations.h index d2dcf99aa46..270cef13ef6 100644 --- a/dbms/src/Storages/System/StorageSystemMutations.h +++ b/dbms/src/Storages/System/StorageSystemMutations.h @@ -14,6 +14,7 @@ class Context; /// in the MergeTree tables. class StorageSystemMutations : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: String getName() const override { return "SystemMutations"; } diff --git a/dbms/src/Storages/System/StorageSystemNumbers.h b/dbms/src/Storages/System/StorageSystemNumbers.h index 5efc23a1125..76070839012 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.h +++ b/dbms/src/Storages/System/StorageSystemNumbers.h @@ -25,6 +25,7 @@ class Context; */ class StorageSystemNumbers : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemNumbers"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemOne.h b/dbms/src/Storages/System/StorageSystemOne.h index 974435e99f0..3e35fdb3477 100644 --- a/dbms/src/Storages/System/StorageSystemOne.h +++ b/dbms/src/Storages/System/StorageSystemOne.h @@ -17,6 +17,7 @@ class Context; */ class StorageSystemOne : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemOne"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemParts.h b/dbms/src/Storages/System/StorageSystemParts.h index eb1ded1c5d6..f7f58daad2e 100644 --- a/dbms/src/Storages/System/StorageSystemParts.h +++ b/dbms/src/Storages/System/StorageSystemParts.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemParts : public ext::shared_ptr_helper, public StorageSystemPartsBase { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemParts"; } diff --git a/dbms/src/Storages/System/StorageSystemPartsColumns.h b/dbms/src/Storages/System/StorageSystemPartsColumns.h index bd03e65557c..1ee90c7cde3 100644 --- a/dbms/src/Storages/System/StorageSystemPartsColumns.h +++ b/dbms/src/Storages/System/StorageSystemPartsColumns.h @@ -16,6 +16,7 @@ class Context; class StorageSystemPartsColumns : public ext::shared_ptr_helper, public StorageSystemPartsBase { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemPartsColumns"; } diff --git a/dbms/src/Storages/System/StorageSystemProcesses.h b/dbms/src/Storages/System/StorageSystemProcesses.h index 3cbe0028af3..735315115c4 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.h +++ b/dbms/src/Storages/System/StorageSystemProcesses.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemProcesses : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemProcesses"; } diff --git a/dbms/src/Storages/System/StorageSystemReplicas.h b/dbms/src/Storages/System/StorageSystemReplicas.h index 49865ad869a..fb006f4cbde 100644 --- a/dbms/src/Storages/System/StorageSystemReplicas.h +++ b/dbms/src/Storages/System/StorageSystemReplicas.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemReplicas : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemReplicas"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemReplicationQueue.h b/dbms/src/Storages/System/StorageSystemReplicationQueue.h index 63dc58118cd..79e015250da 100644 --- a/dbms/src/Storages/System/StorageSystemReplicationQueue.h +++ b/dbms/src/Storages/System/StorageSystemReplicationQueue.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemReplicationQueue : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemReplicationQueue"; } diff --git a/dbms/src/Storages/System/StorageSystemSettings.h b/dbms/src/Storages/System/StorageSystemSettings.h index e44e0abbcd4..f745fe32dcf 100644 --- a/dbms/src/Storages/System/StorageSystemSettings.h +++ b/dbms/src/Storages/System/StorageSystemSettings.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemSettings : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemSettings"; } diff --git a/dbms/src/Storages/System/StorageSystemTableEngines.h b/dbms/src/Storages/System/StorageSystemTableEngines.h index f0f6b62d59d..0bbb03d3898 100644 --- a/dbms/src/Storages/System/StorageSystemTableEngines.h +++ b/dbms/src/Storages/System/StorageSystemTableEngines.h @@ -10,6 +10,7 @@ namespace DB class StorageSystemTableEngines : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; diff --git a/dbms/src/Storages/System/StorageSystemTableFunctions.h b/dbms/src/Storages/System/StorageSystemTableFunctions.h index 413af0f5c66..504ecda4afb 100644 --- a/dbms/src/Storages/System/StorageSystemTableFunctions.h +++ b/dbms/src/Storages/System/StorageSystemTableFunctions.h @@ -9,6 +9,7 @@ namespace DB class StorageSystemTableFunctions : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/dbms/src/Storages/System/StorageSystemTables.h b/dbms/src/Storages/System/StorageSystemTables.h index faf419a6139..fc5a82e9a52 100644 --- a/dbms/src/Storages/System/StorageSystemTables.h +++ b/dbms/src/Storages/System/StorageSystemTables.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemTables : public ext::shared_ptr_helper, public IStorage { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemTables"; } std::string getTableName() const override { return name; } diff --git a/dbms/src/Storages/System/StorageSystemZooKeeper.h b/dbms/src/Storages/System/StorageSystemZooKeeper.h index 9644fe96162..ac0ddd29d1f 100644 --- a/dbms/src/Storages/System/StorageSystemZooKeeper.h +++ b/dbms/src/Storages/System/StorageSystemZooKeeper.h @@ -14,6 +14,7 @@ class Context; */ class StorageSystemZooKeeper : public ext::shared_ptr_helper, public IStorageSystemOneBlock { + friend struct ext::shared_ptr_helper; public: std::string getName() const override { return "SystemZooKeeper"; } diff --git a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index 5d1a7338e46..0d0bbae1402 100755 --- a/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -101,7 +101,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE test.src;" $CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" $CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);" $CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2);" -- trash part to be +$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2); -- trash part to be deleted" # Stop replication at the second replica and remove source table to use fetch instead of copying $CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES test.dst_r2;" diff --git a/dbms/tests/queries/0_stateless/00990_hasToken.python b/dbms/tests/queries/0_stateless/00990_hasToken.python index 217d96dfe52..cd2a284655f 100755 --- a/dbms/tests/queries/0_stateless/00990_hasToken.python +++ b/dbms/tests/queries/0_stateless/00990_hasToken.python @@ -2,11 +2,12 @@ # encoding: utf-8 import re +from string import Template HAYSTACKS = [ + "hay hay hay hay hay hay hay hay hay needle hay hay hay hay hay hay hay hay hay", "hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay needle", "needle hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay hay", - "hay hay hay hay hay hay hay hay hay needle hay hay hay hay hay hay hay hay hay", ] NEEDLE = "needle" @@ -48,47 +49,77 @@ def transform_needle(query, string_transformation_func): return NEEDLE_RE.sub(replace_with_transformation, query) - -def create_cases(table_row_template, table_query_template, const_query_template): +def create_cases(case_sensitive_func, case_insensitive_func, table_row_template, table_query_template, const_query_template): const_queries = [] table_rows = [] table_queries = set() - def add_case(haystack, needle, match): + def add_case(func, haystack, needle, match): match = int(match) - const_queries.append(const_query_template.format(haystack=haystack, needle=needle, match=match)) - table_queries.add(table_query_template.format(haystack=haystack, needle=needle, match=match)) - table_rows.append(table_row_template.format(haystack=haystack, needle=needle, match=match)) + args = dict( + func = func, + haystack = haystack, + needle = needle, + match = match + ) + const_queries.append(const_query_template.substitute(args)) + table_queries.add(table_query_template.substitute(args)) + table_rows.append(table_row_template.substitute(args)) + + def add_case_sensitive(haystack, needle, match): + add_case(case_sensitive_func, haystack, needle, match) + if match: + add_case(case_sensitive_func, transform_needle(haystack, str.swapcase), transform_needle(needle, str.swapcase), match) + + def add_case_insensitive(haystack, needle, match): + add_case(case_insensitive_func, haystack, needle, match) + if match: + add_case(case_insensitive_func, transform_needle(haystack, str.swapcase), needle, match) + add_case(case_insensitive_func, haystack, transform_needle(needle, str.swapcase), match) + # Negative cases - add_case(remove_needle(HAYSTACKS[0]), NEEDLE, False) + add_case_sensitive(remove_needle(HAYSTACKS[0]), NEEDLE, False) + add_case_insensitive(remove_needle(HAYSTACKS[0]), NEEDLE, False) + for haystack in HAYSTACKS: - add_case(transform_needle(haystack, str.title), NEEDLE, False) + add_case_sensitive(transform_needle(haystack, str.swapcase), NEEDLE, False) + sep = '' h = replace_separators(haystack, sep) - add_case(h, NEEDLE, False) - add_case(small_needle(h), small_needle(NEEDLE), False) - add_case(enlarge_haystack(h, 10, sep), NEEDLE, False) + + add_case_sensitive(h, NEEDLE, False) + add_case_insensitive(h, NEEDLE, False) + + add_case_sensitive(small_needle(h), small_needle(NEEDLE), False) + add_case_insensitive(small_needle(h), small_needle(NEEDLE), False) + + add_case_sensitive(enlarge_haystack(h, 10, sep), NEEDLE, False) + add_case_insensitive(enlarge_haystack(h, 10, sep), NEEDLE, False) # positive cases for haystack in HAYSTACKS: - add_case(transform_needle(haystack, str.title), transform_needle(NEEDLE, str.title), True) - add_case(transform_needle(haystack, str.upper), transform_needle(NEEDLE, str.upper), True) + add_case_sensitive(haystack, NEEDLE, True) + add_case_insensitive(haystack, NEEDLE, True) - # Not checking all separators since some (like ' and \n) cause issues when coupled with - # re-based replacement and quoting in query - # other are rare in practice and checking all separators makes this test too lengthy. - # r'\\\\' turns into a single '\' in query - #separators = list(''' \t`~!@#$%^&*()-=+|]}[{";:/?.>,<''') + [r'\\\\'] - separators = list(''' \t;:?.,''') + [r'\\\\'] - for sep in separators: + for sep in list(''' ,'''): h = replace_separators(haystack, sep) - add_case(h, NEEDLE, True) - add_case(small_needle(h), small_needle(NEEDLE), True) - add_case(enlarge_haystack(h, 200, sep), NEEDLE, True) - add_case(replace_needle(h, 'иголка'), replace_needle(NEEDLE, 'иголка'), True) - add_case(replace_needle(h, '指针'), replace_needle(NEEDLE, '指针'), True) + add_case_sensitive(h, NEEDLE, True) + add_case_sensitive(small_needle(h), small_needle(NEEDLE), True) + add_case_sensitive(enlarge_haystack(h, 200, sep), NEEDLE, True) + + add_case_insensitive(h, NEEDLE, True) + add_case_insensitive(small_needle(h), small_needle(NEEDLE), True) + add_case_insensitive(enlarge_haystack(h, 200, sep), NEEDLE, True) + + # case insesitivity works only on ASCII strings + add_case_sensitive(replace_needle(h, 'иголка'), replace_needle(NEEDLE, 'иголка'), True) + add_case_sensitive(replace_needle(h, '指针'), replace_needle(NEEDLE, '指针'), True) + + for sep in list('''~!@$%^&*()-=+|]}[{";:/?.><\t''') + [r'\\\\']: + h = replace_separators(HAYSTACKS[0], sep) + add_case(case_sensitive_func, h, NEEDLE, True) return table_rows, table_queries, const_queries @@ -97,12 +128,14 @@ def main(): def query(x): print x - CONST_QUERY = """SELECT hasToken('{haystack}', '{needle}'), ' expecting ', {match};""" - #SELECT hasToken(haystack, '{needle}') FROM ht WHERE needle = '{needle}' AND match = {match};""" - TABLE_QUERY = """WITH '{needle}' as n SELECT haystack, needle, hasToken(haystack, n) as result FROM ht WHERE needle = n AND result != match;""" - TABLE_ROW = """('{haystack}', '{needle}', {match})""" + CONST_QUERY = Template("""SELECT ${func}('${haystack}', '${needle}'), ' expecting ', ${match};""") + TABLE_QUERY = Template("""WITH '${needle}' as n + SELECT haystack, needle, ${func}(haystack, n) as result + FROM ht + WHERE func = '${func}' AND needle = n AND result != match;""") + TABLE_ROW = Template("""('${haystack}', '${needle}', ${match}, '${func}')""") - rows, table_queries, const_queries = create_cases(TABLE_ROW, TABLE_QUERY, CONST_QUERY) + rows, table_queries, const_queries = create_cases('hasToken', 'hasTokenCaseInsensitive', TABLE_ROW, TABLE_QUERY, CONST_QUERY) for q in const_queries: query(q) @@ -112,7 +145,8 @@ def main(): ( haystack String, needle String, - match UInt8 + match UInt8, + func String ) ENGINE MergeTree() ORDER BY haystack; @@ -120,5 +154,7 @@ INSERT INTO ht VALUES {values};""".format(values=", ".join(rows))) for q in sorted(table_queries): query(q) + query("""DROP TABLE ht""") + if __name__ == '__main__': main() diff --git a/dbms/tests/queries/0_stateless/00990_hasToken.reference b/dbms/tests/queries/0_stateless/00990_hasToken.reference index 867c0c1c691..1e8c067362c 100644 --- a/dbms/tests/queries/0_stateless/00990_hasToken.reference +++ b/dbms/tests/queries/0_stateless/00990_hasToken.reference @@ -11,6 +11,47 @@ 0 expecting 0 0 expecting 0 0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +0 expecting 0 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 +1 expecting 1 1 expecting 1 1 expecting 1 1 expecting 1 diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 9f9448f27c8..b6e59c4aa50 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -67,22 +67,22 @@ The command-line client allows passing external data (external temporary tables) ### Queries with Parameters {#cli-queries-with-parameters} -You can create a query with parameters, and pass values for these parameters with the parameters of the client app. For example: +You can create a query with parameters and pass values to them from client application. This allows to avoid formatting query with specific dynamic values on client side. For example: ```bash clickhouse-client --param_parName="[1, 2]" -q "SELECT * FROM table WHERE a = {parName:Array(UInt16)}" ``` -#### Syntax of a Query {#cli-queries-with-parameters-syntax} +#### Query Syntax {#cli-queries-with-parameters-syntax} -Format a query by the standard method. Values that you want to put into the query from the app parameters place in braces and format as follows: +Format a query as usual, then place the values that you want to pass from the app parameters to the query in braces in the following format: ``` {:} ``` -- `name` — Identifier of a placeholder that should be used in app parameter as `--param_name = value`. -- `data type` — A data type of app parameter value. For example, data structure like `(integer, ('string', integer))` can have a data type `Tuple(UInt8, Tuple(String, UInt8))` (also you can use another [integer](../data_types/int_uint.md) types). +- `name` — Placeholder identifier. In the console client it should be used in app parameters as `--param_ = value`. +- `data type` — [Data type](../data_types/index.md) of the app parameter value. For example, a data structure like `(integer, ('string', integer))` can have the `Tuple(UInt8, Tuple(String, UInt8))` data type (you can also use another [integer](../data_types/int_uint.md) types). #### Example @@ -118,6 +118,8 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--stacktrace` – If specified, also print the stack trace if an exception occurs. - `--config-file` – The name of the configuration file. - `--secure` – If specified, will connect to server over secure connection. +- `--param_` — Value for a [query with parameters](#cli-queries-with-parameters). + ### Configuration Files diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index ef32b101b71..80cf72ec0e2 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -246,7 +246,7 @@ Use buffering to avoid situations where a query processing error occurred after ### Queries with Parameters {#cli-queries-with-parameters} -You can create a query with parameters, and pass values for these parameters with the parameters of the HTTP request. For more information, see [CLI Formatted Queries](cli.md#cli-queries-with-parameters). +You can create a query with parameters and pass values for them from the corresponding HTTP request parameters. For more information, see [Queries with Parameters for CLI](cli.md#cli-queries-with-parameters). ### Example diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 0f3ce84345e..59980109240 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -54,8 +54,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA По умолчанию, в качестве формата, используется формат PrettyCompact (красивые таблички). Вы можете изменить формат с помощью секции FORMAT запроса, или с помощью указания `\G` на конце запроса, с помощью аргумента командной строки `--format` или `--vertical`, или с помощью конфигурационного файла клиента. -Чтобы выйти из клиента, нажмите Ctrl+D (или Ctrl+C), или наберите вместо запроса одно из: -"exit", "quit", "logout", "учше", "йгше", "дщпщге", "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", "q", "й", "q", "Q", ":q", "й", "Й", "Жй" +Чтобы выйти из клиента, нажмите Ctrl+D (или Ctrl+C), или наберите вместо запроса одно из: "exit", "quit", "logout", "учше", "йгше", "дщпщге", "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", "q", "й", "q", "Q", ":q", "й", "Й", "Жй" При выполнении запроса, клиент показывает: @@ -68,38 +67,64 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA Клиент командной строки позволяет передать внешние данные (внешние временные таблицы) для использования запроса. Подробнее смотрите раздел "Внешние данные для обработки запроса" +### Запросы с параметрами {#cli-queries-with-parameters} + +Вы можете создать запрос с параметрами и передавать в них значения из приложения. Это позволяет избежать форматирования запросов на стороне клиента, если известно, какие из параметров запроса динамически меняются. Например: + +```bash +clickhouse-client --param_parName="[1, 2]" -q "SELECT * FROM table WHERE a = {parName:Array(UInt16)}" +``` + +#### Cинтаксис запроса {#cli-queries-with-parameters-syntax} + +Отформатируйте запрос обычным способом. Представьте значения, которые вы хотите передать из параметров приложения в запрос в следующем формате: + +``` +{:} +``` + +- `name` — идентификатор подстановки. В консольном клиенте его следует использовать как часть имени параметра `--param_ = value`. +- `data type` — [тип данных](../data_types/index.md) значения. Например, структура данных `(integer, ('string', integer))` может иметь тип данных `Tuple(UInt8, Tuple(String, UInt8))` ([целочисленный](../data_types/int_uint.md) тип может быть и другим). + +#### Пример + +```bash +clickhouse-client --param_tuple_in_tuple="(10, ('dt', 10))" -q "SELECT * FROM table WHERE val = {tuple_in_tuple:Tuple(UInt8, Tuple(String, UInt8))}" +``` + ## Конфигурирование {#interfaces_cli_configuration} В `clickhouse-client` можно передавать различные параметры (все параметры имеют значения по умолчанию) с помощью: - - Командной строки. - - Параметры командной строки переопределяют значения по умолчанию и параметры конфигурационных файлов. - - - Конфигурационных файлов. - +- Командной строки. + + Параметры командной строки переопределяют значения по умолчанию и параметры конфигурационных файлов. + +- Конфигурационных файлов. + Параметры в конфигурационных файлах переопределяют значения по умолчанию. ### Параметры командной строки -- `--host, -h` - имя сервера, по умолчанию - localhost. Вы можете использовать как имя, так и IPv4 или IPv6 адрес. -- `--port` - порт, к которому соединяться, по умолчанию - 9000. Замечу, что для HTTP и родного интерфейса используются разные порты. -- `--user, -u` - имя пользователя, по умолчанию - default. -- `--password` - пароль, по умолчанию - пустая строка. -- `--query, -q` - запрос для выполнения, при использовании в неинтерактивном режиме. -- `--database, -d` - выбрать текущую БД, по умолчанию - текущая БД из настроек сервера (по умолчанию - БД default). -- `--multiline, -m` - если указано - разрешить многострочные запросы, не отправлять запрос по нажатию Enter. -- `--multiquery, -n` - если указано - разрешить выполнять несколько запросов, разделённых точкой с запятой. Работает только в неинтерактивном режиме. -- `--format, -f` - использовать указанный формат по умолчанию для вывода результата. -- `--vertical, -E` - если указано, использовать формат Vertical по умолчанию для вывода результата. То же самое, что --format=Vertical. В этом формате каждое значение выводится на отдельной строке, что удобно для отображения широких таблиц. -- `--time, -t` - если указано, в неинтерактивном режиме вывести время выполнения запроса в stderr. -- `--stacktrace` - если указано, в случае исключения, выводить также его стек трейс. -- `--config-file` - имя конфигурационного файла. -- `--secure` - если указано, будет использован безопасный канал. +- `--host, -h` — имя сервера, по умолчанию — localhost. Вы можете использовать как имя, так и IPv4 или IPv6 адрес. +- `--port` — порт, к которому соединяться, по умолчанию — 9000. Замечу, что для HTTP и родного интерфейса используются разные порты. +- `--user, -u` — имя пользователя, по умолчанию — default. +- `--password` — пароль, по умолчанию — пустая строка. +- `--query, -q` — запрос для выполнения, при использовании в неинтерактивном режиме. +- `--database, -d` — выбрать текущую БД, по умолчанию — текущая БД из настроек сервера (по умолчанию — БД default). +- `--multiline, -m` — если указано — разрешить многострочные запросы, не отправлять запрос по нажатию Enter. +- `--multiquery, -n` — если указано — разрешить выполнять несколько запросов, разделённых точкой с запятой. Работает только в неинтерактивном режиме. +- `--format, -f` — использовать указанный формат по умолчанию для вывода результата. +- `--vertical, -E` — если указано, использовать формат Vertical по умолчанию для вывода результата. То же самое, что --format=Vertical. В этом формате каждое значение выводится на отдельной строке, что удобно для отображения широких таблиц. +- `--time, -t` — если указано, в неинтерактивном режиме вывести время выполнения запроса в stderr. +- `--stacktrace` — если указано, в случае исключения, выводить также его стек трейс. +- `--config-file` — имя конфигурационного файла. +- `--secure` — если указано, будет использован безопасный канал. +- `--param_` — значение параметра для [запроса с параметрами](#cli-queries-with-parameters). ### Конфигурационные файлы -`clickhouse-client` использует первый существующий файл из: +`clickhouse—client` использует первый существующий файл из: - Определенного параметром `--config-file`. - `./clickhouse-client.xml` diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index 20067942dd8..77eb984e8f4 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -245,5 +245,15 @@ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wa Буферизация позволяет избежать ситуации когда код ответа и HTTP-заголовки были отправлены клиенту, после чего возникла ошибка выполнения запроса. В такой ситуации сообщение об ошибке записывается в конце тела ответа, и на стороне клиента ошибка может быть обнаружена только на этапе парсинга. +### Запросы с параметрами {#cli-queries-with-parameters} + +Можно создать запрос с параметрами и передать для них значения из соответствующих параметров HTTP-запроса. Дополнительную информацию смотрите в [Запросы с параметрами для консольного клиента](cli.md#cli-queries-with-parameters). + +### Пример + +```bash +curl -sS "
?param_id=2¶m_phrase=test" -d "SELECT * FROM table WHERE int_column = {id:UInt8} and string_column = {phrase:String}" +``` + [Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/http_interface/) diff --git a/libs/libcommon/include/ext/shared_ptr_helper.h b/libs/libcommon/include/ext/shared_ptr_helper.h index f7fd7c38ace..ca7219e6261 100644 --- a/libs/libcommon/include/ext/shared_ptr_helper.h +++ b/libs/libcommon/include/ext/shared_ptr_helper.h @@ -7,32 +7,16 @@ namespace ext /** Allows to make std::shared_ptr from T with protected constructor. * - * Derive your T class from shared_ptr_helper + * Derive your T class from shared_ptr_helper and add shared_ptr_helper as a friend * and you will have static 'create' method in your class. - * - * Downsides: - * - your class cannot be final; - * - awful compilation error messages; - * - bad code navigation. - * - different dynamic type of created object, you cannot use typeid. */ template struct shared_ptr_helper { template - static auto create(TArgs &&... args) + static std::shared_ptr create(TArgs &&... args) { - /** Local struct makes protected constructor to be accessible by std::make_shared function. - * This trick is suggested by Yurii Diachenko, - * inspired by https://habrahabr.ru/company/mailru/blog/341584/ - * that is translation of http://videocortex.io/2017/Bestiary/#-voldemort-types - */ - struct Local : T - { - Local(TArgs &&... args) : T(std::forward(args)...) {} - }; - - return std::make_shared(std::forward(args)...); + return std::shared_ptr(new T(std::forward(args)...)); } };