diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index a5f953ccc15..0b2044cfe2c 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -151,7 +151,7 @@ Columns CacheDictionary::getColumns( * use default value. */ - if (dictionary_key_type == DictionaryKeyType::complex) + if (dictionary_key_type == DictionaryKeyType::Complex) dict_struct.validateKeyTypes(key_types); DictionaryKeysArenaHolder arena_holder; @@ -268,7 +268,7 @@ ColumnUInt8::Ptr CacheDictionary::hasKeys(const Columns & k * Check that key was fetched during update for that key set true in result array. */ - if (dictionary_key_type == DictionaryKeyType::complex) + if (dictionary_key_type == DictionaryKeyType::Complex) dict_struct.validateKeyTypes(key_types); @@ -364,7 +364,7 @@ ColumnPtr CacheDictionary::getHierarchy( ColumnPtr key_column [[maybe_unused]], const DataTypePtr & key_type [[maybe_unused]]) const { - if (dictionary_key_type == DictionaryKeyType::simple) + if (dictionary_key_type == DictionaryKeyType::Simple) { size_t keys_found; auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found); @@ -382,7 +382,7 @@ ColumnUInt8::Ptr CacheDictionary::isInHierarchy( ColumnPtr in_key_column [[maybe_unused]], const DataTypePtr & key_type [[maybe_unused]]) const { - if (dictionary_key_type == DictionaryKeyType::simple) + if (dictionary_key_type == DictionaryKeyType::Simple) { size_t keys_found; auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found); @@ -492,7 +492,7 @@ Pipe CacheDictionary::read(const Names & column_names, size /// Write lock on storage const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) data.emplace(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names); else { @@ -534,7 +534,7 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtr requested_keys_vector; std::vector requested_complex_key_rows; - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) requested_keys_vector.reserve(requested_keys.size()); else requested_complex_key_rows.reserve(requested_keys.size()); @@ -546,7 +546,7 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtr::update(CacheDictionaryUpdateUnitPtrloadIds(requested_keys_vector)); else pipeline.init(current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows)); @@ -684,7 +684,7 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtr; -template class CacheDictionary; +template class CacheDictionary; +template class CacheDictionary; } diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index 613d73b0f83..0e2a2699c45 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -51,8 +51,7 @@ template class CacheDictionary final : public IDictionary { public: - using KeyType = std::conditional_t; - static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by cache dictionary"); + using KeyType = std::conditional_t; CacheDictionary( const StorageID & dict_id_, @@ -118,7 +117,7 @@ public: DictionaryKeyType getKeyType() const override { - return dictionary_key_type == DictionaryKeyType::simple ? DictionaryKeyType::simple : DictionaryKeyType::complex; + return dictionary_key_type == DictionaryKeyType::Simple ? DictionaryKeyType::Simple : DictionaryKeyType::Complex; } ColumnPtr getColumn( @@ -141,7 +140,7 @@ public: std::exception_ptr getLastException() const override; - bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); } + bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::Simple && dict_struct.hierarchical_attribute_index.has_value(); } ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override; @@ -151,7 +150,7 @@ public: const DataTypePtr & key_type) const override; private: - using FetchResult = std::conditional_t; + using FetchResult = std::conditional_t; static MutableColumns aggregateColumnsInOrderOfKeys( const PaddedPODArray & keys, @@ -219,7 +218,7 @@ private: }; -extern template class CacheDictionary; -extern template class CacheDictionary; +extern template class CacheDictionary; +extern template class CacheDictionary; } diff --git a/src/Dictionaries/CacheDictionaryStorage.h b/src/Dictionaries/CacheDictionaryStorage.h index 8374e649cd1..2c7e9ad7092 100644 --- a/src/Dictionaries/CacheDictionaryStorage.h +++ b/src/Dictionaries/CacheDictionaryStorage.h @@ -41,8 +41,7 @@ class CacheDictionaryStorage final : public ICacheDictionaryStorage static constexpr size_t max_collision_length = 10; public: - using KeyType = std::conditional_t; - static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionaryStorage"); + using KeyType = std::conditional_t; explicit CacheDictionaryStorage( const DictionaryStructure & dictionary_structure, @@ -62,19 +61,19 @@ public: String getName() const override { - if (dictionary_key_type == DictionaryKeyType::simple) + if (dictionary_key_type == DictionaryKeyType::Simple) return "Cache"; else return "ComplexKeyCache"; } - bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::simple; } + bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::Simple; } SimpleKeysStorageFetchResult fetchColumnsForKeys( const PaddedPODArray & keys, const DictionaryStorageFetchRequest & fetch_request) override { - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) return fetchColumnsForKeysImpl(keys, fetch_request); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for complex key storage"); @@ -82,7 +81,7 @@ public: void insertColumnsForKeys(const PaddedPODArray & keys, Columns columns) override { - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) insertColumnsForKeysImpl(keys, columns); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage"); @@ -90,7 +89,7 @@ public: void insertDefaultKeys(const PaddedPODArray & keys) override { - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) insertDefaultKeysImpl(keys); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage"); @@ -98,19 +97,19 @@ public: PaddedPODArray getCachedSimpleKeys() const override { - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) return getCachedKeysImpl(); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage"); } - bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; } + bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::Complex; } ComplexKeysStorageFetchResult fetchColumnsForKeys( const PaddedPODArray & keys, const DictionaryStorageFetchRequest & column_fetch_requests) override { - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) return fetchColumnsForKeysImpl(keys, column_fetch_requests); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage"); @@ -118,7 +117,7 @@ public: void insertColumnsForKeys(const PaddedPODArray & keys, Columns columns) override { - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) insertColumnsForKeysImpl(keys, columns); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage"); @@ -126,7 +125,7 @@ public: void insertDefaultKeys(const PaddedPODArray & keys) override { - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) insertDefaultKeysImpl(keys); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage"); @@ -134,7 +133,7 @@ public: PaddedPODArray getCachedComplexKeys() const override { - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) return getCachedKeysImpl(); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedComplexKeys is not supported for simple key storage"); diff --git a/src/Dictionaries/CacheDictionaryUpdateQueue.cpp b/src/Dictionaries/CacheDictionaryUpdateQueue.cpp index 310abed822f..1d96fcc108b 100644 --- a/src/Dictionaries/CacheDictionaryUpdateQueue.cpp +++ b/src/Dictionaries/CacheDictionaryUpdateQueue.cpp @@ -14,8 +14,8 @@ namespace ErrorCodes extern const int TIMEOUT_EXCEEDED; } -template class CacheDictionaryUpdateUnit; -template class CacheDictionaryUpdateUnit; +template class CacheDictionaryUpdateUnit; +template class CacheDictionaryUpdateUnit; template CacheDictionaryUpdateQueue::CacheDictionaryUpdateQueue( @@ -155,7 +155,7 @@ void CacheDictionaryUpdateQueue::updateThreadFunction() } } -template class CacheDictionaryUpdateQueue; -template class CacheDictionaryUpdateQueue; +template class CacheDictionaryUpdateQueue; +template class CacheDictionaryUpdateQueue; } diff --git a/src/Dictionaries/CacheDictionaryUpdateQueue.h b/src/Dictionaries/CacheDictionaryUpdateQueue.h index 3d27a157752..bcad376bc53 100644 --- a/src/Dictionaries/CacheDictionaryUpdateQueue.h +++ b/src/Dictionaries/CacheDictionaryUpdateQueue.h @@ -39,7 +39,7 @@ template class CacheDictionaryUpdateUnit { public: - using KeyType = std::conditional_t; + using KeyType = std::conditional_t; /// Constructor for complex keys update request explicit CacheDictionaryUpdateUnit( @@ -85,8 +85,8 @@ private: template using CacheDictionaryUpdateUnitPtr = std::shared_ptr>; -extern template class CacheDictionaryUpdateUnit; -extern template class CacheDictionaryUpdateUnit; +extern template class CacheDictionaryUpdateUnit; +extern template class CacheDictionaryUpdateUnit; struct CacheDictionaryUpdateQueueConfiguration { @@ -110,7 +110,6 @@ class CacheDictionaryUpdateQueue public: /// Client of update queue must provide this function in constructor and perform update using update unit. using UpdateFunction = std::function)>; - static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionaryUpdateQueue"); CacheDictionaryUpdateQueue( String dictionary_name_for_logs_, @@ -167,7 +166,7 @@ private: std::atomic finished{false}; }; -extern template class CacheDictionaryUpdateQueue; -extern template class CacheDictionaryUpdateQueue; +extern template class CacheDictionaryUpdateQueue; +extern template class CacheDictionaryUpdateQueue; } diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 3d077414291..0c04d87c959 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -380,14 +380,14 @@ template class DictionaryKeysArenaHolder; template <> -class DictionaryKeysArenaHolder +class DictionaryKeysArenaHolder { public: static Arena * getComplexKeyArena() { return nullptr; } }; template <> -class DictionaryKeysArenaHolder +class DictionaryKeysArenaHolder { public: @@ -402,8 +402,7 @@ template class DictionaryKeysExtractor { public: - using KeyType = std::conditional_t; - static_assert(key_type != DictionaryKeyType::range, "Range key type is not supported by DictionaryKeysExtractor"); + using KeyType = std::conditional_t; explicit DictionaryKeysExtractor(const Columns & key_columns_, Arena * complex_key_arena_) : key_columns(key_columns_) @@ -411,7 +410,7 @@ public: { assert(!key_columns.empty()); - if constexpr (key_type == DictionaryKeyType::simple) + if constexpr (key_type == DictionaryKeyType::Simple) { key_columns[0] = key_columns[0]->convertToFullColumnIfConst(); @@ -437,7 +436,7 @@ public: { assert(current_key_index < keys_size); - if constexpr (key_type == DictionaryKeyType::simple) + if constexpr (key_type == DictionaryKeyType::Simple) { const auto & column_vector = static_cast &>(*key_columns[0]); const auto & data = column_vector.getData(); @@ -465,7 +464,7 @@ public: void rollbackCurrentKey() const { - if constexpr (key_type == DictionaryKeyType::complex) + if constexpr (key_type == DictionaryKeyType::Complex) complex_key_arena->rollback(current_complex_key.size); } @@ -521,8 +520,7 @@ void mergeBlockWithPipe( Block & block_to_update, Pipe pipe) { - using KeyType = std::conditional_t; - static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by updatePreviousyLoadedBlockWithStream"); + using KeyType = std::conditional_t; Columns saved_block_key_columns; saved_block_key_columns.reserve(key_columns_size); diff --git a/src/Dictionaries/DictionarySource.cpp b/src/Dictionaries/DictionarySource.cpp index fbb03cb00fa..a164543e1ff 100644 --- a/src/Dictionaries/DictionarySource.cpp +++ b/src/Dictionaries/DictionarySource.cpp @@ -132,7 +132,7 @@ Block DictionarySourceData::fillBlock( { ColumnPtr column; - if (dictionary_key_type == DictionaryKeyType::simple) + if (dictionary_key_type == DictionaryKeyType::Simple) { column = dictionary->getColumn( attribute.name, diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index eb06701ab7a..e12100a556d 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -40,7 +40,7 @@ Columns DirectDictionary::getColumns( const DataTypes & key_types [[maybe_unused]], const Columns & default_values_columns) const { - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) dict_struct.validateKeyTypes(key_types); DictionaryKeysArenaHolder arena_holder; @@ -161,7 +161,7 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys( const Columns & key_columns, const DataTypes & key_types [[maybe_unused]]) const { - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) dict_struct.validateKeyTypes(key_types); DictionaryKeysArenaHolder arena_holder; @@ -230,7 +230,7 @@ ColumnPtr DirectDictionary::getHierarchy( ColumnPtr key_column, const DataTypePtr & key_type) const { - if (dictionary_key_type == DictionaryKeyType::simple) + if (dictionary_key_type == DictionaryKeyType::Simple) { size_t keys_found; auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found); @@ -248,7 +248,7 @@ ColumnUInt8::Ptr DirectDictionary::isInHierarchy( ColumnPtr in_key_column, const DataTypePtr & key_type) const { - if (dictionary_key_type == DictionaryKeyType::simple) + if (dictionary_key_type == DictionaryKeyType::Simple) { size_t keys_found = 0; auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found); @@ -269,7 +269,7 @@ Pipe DirectDictionary::getSourceBlockInputStream( Pipe pipe; - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) { std::vector ids; ids.reserve(requested_keys_size); @@ -310,9 +310,9 @@ namespace ContextPtr /* context */, bool /* created_from_ddl */) { - const auto * layout_name = dictionary_key_type == DictionaryKeyType::simple ? "direct" : "complex_key_direct"; + const auto * layout_name = dictionary_key_type == DictionaryKeyType::Simple ? "direct" : "complex_key_direct"; - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) { if (dict_struct.key) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, @@ -344,13 +344,13 @@ namespace } } -template class DirectDictionary; -template class DirectDictionary; +template class DirectDictionary; +template class DirectDictionary; void registerDictionaryDirect(DictionaryFactory & factory) { - factory.registerLayout("direct", createDirectDictionary, false); - factory.registerLayout("complex_key_direct", createDirectDictionary, true); + factory.registerLayout("direct", createDirectDictionary, false); + factory.registerLayout("complex_key_direct", createDirectDictionary, true); } diff --git a/src/Dictionaries/DirectDictionary.h b/src/Dictionaries/DirectDictionary.h index 4700e71d94b..ebe5f5fbbc7 100644 --- a/src/Dictionaries/DirectDictionary.h +++ b/src/Dictionaries/DirectDictionary.h @@ -20,8 +20,7 @@ template class DirectDictionary final : public IDictionary { public: - static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by direct dictionary"); - using KeyType = std::conditional_t; + using KeyType = std::conditional_t; DirectDictionary( const StorageID & dict_id_, @@ -30,7 +29,7 @@ public: std::string getTypeName() const override { - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) return "Direct"; else return "ComplexKeyDirect"; @@ -110,7 +109,7 @@ private: mutable std::atomic found_count{0}; }; -extern template class DirectDictionary; -extern template class DirectDictionary; +extern template class DirectDictionary; +extern template class DirectDictionary; } diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 639895ac8ac..26667db1081 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -289,8 +289,8 @@ void FlatDictionary::blockToAttributes(const Block & block) { const auto keys_column = block.safeGetByPosition(0).column; - DictionaryKeysArenaHolder arena_holder; - DictionaryKeysExtractor keys_extractor({ keys_column }, arena_holder.getComplexKeyArena()); + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor keys_extractor({ keys_column }, arena_holder.getComplexKeyArena()); auto keys = keys_extractor.extractAllKeys(); HashSet already_processed_keys; @@ -344,7 +344,7 @@ void FlatDictionary::updateData() else { Pipe pipe(source_ptr->loadUpdatedAll()); - mergeBlockWithPipe( + mergeBlockWithPipe( dict_struct.getKeysSize(), *update_field_loaded_block, std::move(pipe)); diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index e6a07200c05..196194ddb21 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -72,7 +72,7 @@ public: return dict_struct.getAttribute(attribute_name).injective; } - DictionaryKeyType getKeyType() const override { return DictionaryKeyType::simple; } + DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Simple; } ColumnPtr getColumn( const std::string& attribute_name, diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 189994dabf4..d462631fba8 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -61,7 +61,7 @@ ColumnPtr HashedDictionary::getColumn( const DataTypes & key_types [[maybe_unused]], const ColumnPtr & default_values_column) const { - if (dictionary_key_type == DictionaryKeyType::complex) + if (dictionary_key_type == DictionaryKeyType::Complex) dict_struct.validateKeyTypes(key_types); ColumnPtr result; @@ -163,7 +163,7 @@ ColumnPtr HashedDictionary::getColumn( template ColumnUInt8::Ptr HashedDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types) const { - if (dictionary_key_type == DictionaryKeyType::complex) + if (dictionary_key_type == DictionaryKeyType::Complex) dict_struct.validateKeyTypes(key_types); DictionaryKeysArenaHolder arena_holder; @@ -210,7 +210,7 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Co template ColumnPtr HashedDictionary::getHierarchy(ColumnPtr key_column [[maybe_unused]], const DataTypePtr &) const { - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) { PaddedPODArray keys_backup_storage; const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); @@ -258,7 +258,7 @@ ColumnUInt8::Ptr HashedDictionary::isInHierarchy( ColumnPtr in_key_column [[maybe_unused]], const DataTypePtr &) const { - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) { PaddedPODArray keys_backup_storage; const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); @@ -309,7 +309,7 @@ ColumnPtr HashedDictionary::getDescendants( const DataTypePtr &, size_t level [[maybe_unused]]) const { - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) { PaddedPODArray keys_backup; const auto & keys = getColumnVectorData(this, key_column, keys_backup); @@ -665,7 +665,7 @@ Pipe HashedDictionary::read(const Names & column_na }); } - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); else return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), keys, column_names), max_block_size)); @@ -702,10 +702,10 @@ void HashedDictionary::getAttributeContainer(size_t }); } -template class HashedDictionary; -template class HashedDictionary; -template class HashedDictionary; -template class HashedDictionary; +template class HashedDictionary; +template class HashedDictionary; +template class HashedDictionary; +template class HashedDictionary; void registerDictionaryHashed(DictionaryFactory & factory) { @@ -717,9 +717,9 @@ void registerDictionaryHashed(DictionaryFactory & factory) DictionaryKeyType dictionary_key_type, bool sparse) -> DictionaryPtr { - if (dictionary_key_type == DictionaryKeyType::simple && dict_struct.key) + if (dictionary_key_type == DictionaryKeyType::Simple && dict_struct.key) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for simple key hashed dictionary"); - else if (dictionary_key_type == DictionaryKeyType::complex && dict_struct.id) + else if (dictionary_key_type == DictionaryKeyType::Complex && dict_struct.id) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for complex key hashed dictionary"); if (dict_struct.range_min || dict_struct.range_max) @@ -737,32 +737,32 @@ void registerDictionaryHashed(DictionaryFactory & factory) HashedDictionaryStorageConfiguration configuration{preallocate, require_nonempty, dict_lifetime}; - if (dictionary_key_type == DictionaryKeyType::simple) + if (dictionary_key_type == DictionaryKeyType::Simple) { if (sparse) - return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); else - return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); } else { if (sparse) - return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); else - return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); } }; using namespace std::placeholders; factory.registerLayout("hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ false); }, false); factory.registerLayout("sparse_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ true); }, false); factory.registerLayout("complex_key_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ false); }, true); factory.registerLayout("complex_key_sparse_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ true); }, true); } diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index bf58638effc..d1e1f681fa1 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -35,8 +35,7 @@ template class HashedDictionary final : public IDictionary { public: - using KeyType = std::conditional_t; - static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by hashed dictionary"); + using KeyType = std::conditional_t; HashedDictionary( const StorageID & dict_id_, @@ -47,11 +46,11 @@ public: std::string getTypeName() const override { - if constexpr (dictionary_key_type == DictionaryKeyType::simple && sparse) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple && sparse) return "SparseHashed"; - else if constexpr (dictionary_key_type == DictionaryKeyType::simple && !sparse) + else if constexpr (dictionary_key_type == DictionaryKeyType::Simple && !sparse) return "Hashed"; - else if constexpr (dictionary_key_type == DictionaryKeyType::complex && sparse) + else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && sparse) return "ComplexKeySparseHashed"; else return "ComplexKeyHashed"; @@ -102,7 +101,7 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); } + bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::Simple && dict_struct.hierarchical_attribute_index.has_value(); } ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & hierarchy_attribute_type) const override; @@ -121,13 +120,13 @@ public: private: template using CollectionTypeNonSparse = std::conditional_t< - dictionary_key_type == DictionaryKeyType::simple, + dictionary_key_type == DictionaryKeyType::Simple, HashMap, HashMapWithSavedHash>>; template using CollectionTypeSparse = std::conditional_t< - dictionary_key_type == DictionaryKeyType::simple, + dictionary_key_type == DictionaryKeyType::Simple, SparseHashMap, SparseHashMap>; @@ -211,10 +210,10 @@ private: Arena complex_key_arena; }; -extern template class HashedDictionary; -extern template class HashedDictionary; +extern template class HashedDictionary; +extern template class HashedDictionary; -extern template class HashedDictionary; -extern template class HashedDictionary; +extern template class HashedDictionary; +extern template class HashedDictionary; } diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index f9e0223a698..d7778cc0022 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -33,15 +33,20 @@ using DictionaryPtr = std::unique_ptr; * Simple is for dictionaries that support UInt64 key column. * * Complex is for dictionaries that support any combination of key columns. - * - * Range is for dictionary that support combination of UInt64 key column, - * and numeric representable range key column. */ enum class DictionaryKeyType { - simple, - complex, - range + Simple, + Complex +}; + +/** DictionarySpecialKeyType provides IDictionary client information about + * which special key type is supported by dictionary. + */ +enum class DictionarySpecialKeyType +{ + None, + Range }; /** @@ -56,6 +61,7 @@ struct IDictionary : public IExternalLoadable } const std::string & getFullName() const{ return full_name; } + StorageID getDictionaryID() const { std::lock_guard lock{name_mutex}; @@ -109,6 +115,8 @@ struct IDictionary : public IExternalLoadable */ virtual DictionaryKeyType getKeyType() const = 0; + virtual DictionarySpecialKeyType getSpecialKeyType() const { return DictionarySpecialKeyType::None;} + /** Subclass must validate key columns and keys types * and return column representation of dictionary attribute. * diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index af4b77a6ff8..4aba70dd14f 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -67,7 +67,7 @@ public: return dict_struct.getAttribute(attribute_name).injective; } - DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; } + DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Complex; } ColumnPtr getColumn( const std::string& attribute_name, diff --git a/src/Dictionaries/PolygonDictionary.h b/src/Dictionaries/PolygonDictionary.h index 32771be4b16..9ab82890c49 100644 --- a/src/Dictionaries/PolygonDictionary.h +++ b/src/Dictionaries/PolygonDictionary.h @@ -86,7 +86,7 @@ public: bool isInjective(const std::string & attribute_name) const override { return dict_struct.getAttribute(attribute_name).injective; } - DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; } + DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Complex; } ColumnPtr getColumn( const std::string& attribute_name, diff --git a/src/Dictionaries/RangeDictionarySource.h b/src/Dictionaries/RangeDictionarySource.h index 252ab97ac74..dcc5b1ea3b8 100644 --- a/src/Dictionaries/RangeDictionarySource.h +++ b/src/Dictionaries/RangeDictionarySource.h @@ -14,18 +14,12 @@ namespace DB { -enum class RangeDictionaryType -{ - simple, - complex -}; - -template +template class RangeDictionarySourceData { public: - using KeyType = std::conditional_t; + using KeyType = std::conditional_t; RangeDictionarySourceData( std::shared_ptr dictionary, @@ -58,8 +52,8 @@ private: }; -template -RangeDictionarySourceData::RangeDictionarySourceData( +template +RangeDictionarySourceData::RangeDictionarySourceData( std::shared_ptr dictionary_, const Names & column_names_, PaddedPODArray && keys, @@ -73,8 +67,8 @@ RangeDictionarySourceData::RangeDictionarySour { } -template -Block RangeDictionarySourceData::getBlock(size_t start, size_t length) const +template +Block RangeDictionarySourceData::getBlock(size_t start, size_t length) const { PaddedPODArray block_keys; PaddedPODArray block_start_dates; @@ -93,8 +87,8 @@ Block RangeDictionarySourceData::getBlock(size return fillBlock(block_keys, block_start_dates, block_end_dates, start, start + length); } -template -PaddedPODArray RangeDictionarySourceData::makeDateKeys( +template +PaddedPODArray RangeDictionarySourceData::makeDateKeys( const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const { @@ -112,24 +106,14 @@ PaddedPODArray RangeDictionarySourceData -Block RangeDictionarySourceData::fillBlock( +template +Block RangeDictionarySourceData::fillBlock( const PaddedPODArray & keys_to_fill, const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates, size_t start, size_t end) const { - std::cerr << "RangeDictionarySourceData::fillBlock keys_to_fill " << keys_to_fill.size() << std::endl; - - if constexpr (range_dictionary_type == RangeDictionaryType::simple) - { - for (auto & key : keys_to_fill) - { - std::cerr << key << std::endl; - } - } - ColumnsWithTypeAndName columns; const DictionaryStructure & dictionary_structure = dictionary->getStructure(); @@ -137,7 +121,7 @@ Block RangeDictionarySourceData::fillBlock( Columns keys_columns; Strings keys_names = dictionary_structure.getKeysNames(); - if constexpr (range_dictionary_type == RangeDictionaryType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) { keys_columns = {getColumnFromPODArray(keys_to_fill)}; keys_types = {std::make_shared()}; @@ -154,9 +138,6 @@ Block RangeDictionarySourceData::fillBlock( size_t keys_size = keys_names.size(); - std::cerr << "Keys size " << keys_size << " key columns size " << keys_columns.size(); - std::cerr << " keys types size " << keys_types.size() << std::endl; - assert(keys_columns.size() == keys_size); assert(keys_types.size() == keys_size); @@ -204,51 +185,33 @@ Block RangeDictionarySourceData::fillBlock( columns.emplace_back(std::move(column), attribute.type, attribute.name); } - auto result = Block(columns); - - Field value; - std::cerr << "RangeDictionarySourceData::fillBlock result" << std::endl; - for (auto & block_column : result) - { - std::cerr << "Column name " << block_column.name << " type " << block_column.type->getName() << std::endl; - - auto & column = block_column.column; - size_t column_size = column->size(); - - for (size_t i = 0; i < column_size; ++i) - { - column->get(i, value); - std::cerr << "Index " << i << " value " << value.dump() << std::endl; - } - } - return Block(columns); } -template +template class RangeDictionarySource : public DictionarySourceBase { public: - RangeDictionarySource(RangeDictionarySourceData data_, size_t max_block_size); + RangeDictionarySource(RangeDictionarySourceData data_, size_t max_block_size); String getName() const override { return "RangeDictionarySource"; } protected: Block getBlock(size_t start, size_t length) const override; - RangeDictionarySourceData data; + RangeDictionarySourceData data; }; -template -RangeDictionarySource::RangeDictionarySource(RangeDictionarySourceData data_, size_t max_block_size) +template +RangeDictionarySource::RangeDictionarySource(RangeDictionarySourceData data_, size_t max_block_size) : DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size) , data(std::move(data_)) { } -template -Block RangeDictionarySource::getBlock(size_t start, size_t length) const +template +Block RangeDictionarySource::getBlock(size_t start, size_t length) const { return data.getBlock(start, length); } diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index 50935163a96..ea0af493bdf 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -95,6 +95,13 @@ ColumnPtr RangeHashedDictionary::getColumn( const DataTypes & key_types, const ColumnPtr & default_values_column) const { + if (dictionary_key_type == DictionaryKeyType::Complex) + { + auto key_types_copy = key_types; + key_types_copy.pop_back(); + dict_struct.validateKeyTypes(key_types_copy); + } + ColumnPtr result; const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); @@ -206,9 +213,15 @@ ColumnPtr RangeHashedDictionary::getColumn( template ColumnUInt8::Ptr RangeHashedDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types) const { + if (dictionary_key_type == DictionaryKeyType::Complex) + { + auto key_types_copy = key_types; + key_types_copy.pop_back(); + dict_struct.validateKeyTypes(key_types_copy); + } auto range_column_storage_type = std::make_shared(); auto range_storage_column = key_columns.back(); - ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types[1], ""}; + ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types.back(), ""}; auto range_column_updated = castColumnAccurate(column_to_cast, range_column_storage_type); PaddedPODArray range_backup_storage; const PaddedPODArray & dates = getColumnVectorData(this, range_column_updated, range_backup_storage); @@ -383,7 +396,7 @@ void RangeHashedDictionary::calculateBytesAllocated() callOnDictionaryAttributeType(attribute.type, type_call); } - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) bytes_allocated += complex_key_arena.size(); } @@ -607,10 +620,9 @@ Pipe RangeHashedDictionary::readImpl(const Names & column_n PaddedPODArray end_dates; getKeysAndDates(keys, start_dates, end_dates); - static constexpr RangeDictionaryType range_dictionary_type = (dictionary_key_type == DictionaryKeyType::simple) ? RangeDictionaryType::simple : RangeDictionaryType::complex; - using RangeDictionarySourceType = RangeDictionarySource; + using RangeDictionarySourceType = RangeDictionarySource; - auto source_data = RangeDictionarySourceData( + auto source_data = RangeDictionarySourceData( shared_from_this(), column_names, std::move(keys), @@ -690,7 +702,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); }; factory.registerLayout("range_hashed", create_layout_simple, false); @@ -713,7 +725,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); }; factory.registerLayout("complex_key_range_hashed", create_layout_complex, true); } diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index f9b09189265..4cdab66a0e4 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -32,8 +32,7 @@ template class RangeHashedDictionary final : public IDictionary { public: - using KeyType = std::conditional_t; - static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by hashed dictionary"); + using KeyType = std::conditional_t; RangeHashedDictionary( const StorageID & dict_id_, @@ -78,7 +77,9 @@ public: return dict_struct.getAttribute(attribute_name).injective; } - DictionaryKeyType getKeyType() const override { return DictionaryKeyType::range; } + DictionaryKeyType getKeyType() const override { return dictionary_key_type; } + + DictionarySpecialKeyType getSpecialKeyType() const override { return DictionarySpecialKeyType::Range;} ColumnPtr getColumn( const std::string& attribute_name, @@ -104,7 +105,7 @@ private: template using CollectionType = std::conditional_t< - dictionary_key_type == DictionaryKeyType::simple, + dictionary_key_type == DictionaryKeyType::Simple, HashMap>, HashMapWithSavedHash, DefaultHash>>; diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index bdb640c90be..e584b523376 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -823,8 +823,8 @@ template class SSDCacheDictionaryStorage final : public ICacheDictionaryStorage { public: - using SSDCacheKeyType = std::conditional_t; - using KeyType = std::conditional_t; + using SSDCacheKeyType = std::conditional_t; + using KeyType = std::conditional_t; explicit SSDCacheDictionaryStorage(const SSDCacheDictionaryStorageConfiguration & configuration_) : configuration(configuration_) @@ -838,19 +838,19 @@ public: String getName() const override { - if (dictionary_key_type == DictionaryKeyType::simple) + if (dictionary_key_type == DictionaryKeyType::Simple) return "SSDCache"; else return "SSDComplexKeyCache"; } - bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::simple; } + bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::Simple; } SimpleKeysStorageFetchResult fetchColumnsForKeys( const PaddedPODArray & keys, const DictionaryStorageFetchRequest & fetch_request) override { - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) return fetchColumnsForKeysImpl(keys, fetch_request); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage"); @@ -858,7 +858,7 @@ public: void insertColumnsForKeys(const PaddedPODArray & keys, Columns columns) override { - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) insertColumnsForKeysImpl(keys, columns); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage"); @@ -866,7 +866,7 @@ public: void insertDefaultKeys(const PaddedPODArray & keys) override { - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) insertDefaultKeysImpl(keys); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage"); @@ -874,19 +874,19 @@ public: PaddedPODArray getCachedSimpleKeys() const override { - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) return getCachedKeysImpl(); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage"); } - bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; } + bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::Complex; } ComplexKeysStorageFetchResult fetchColumnsForKeys( const PaddedPODArray & keys, const DictionaryStorageFetchRequest & fetch_request) override { - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) return fetchColumnsForKeysImpl(keys, fetch_request); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage"); @@ -894,7 +894,7 @@ public: void insertColumnsForKeys(const PaddedPODArray & keys, Columns columns) override { - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) insertColumnsForKeysImpl(keys, columns); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage"); @@ -902,7 +902,7 @@ public: void insertDefaultKeys(const PaddedPODArray & keys) override { - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) insertDefaultKeysImpl(keys); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage"); @@ -910,7 +910,7 @@ public: PaddedPODArray getCachedComplexKeys() const override { - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) return getCachedKeysImpl(); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for simple key storage"); @@ -1134,7 +1134,7 @@ private: Cell cell; setCellDeadline(cell, now); - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) { /// Copy complex key into arena and put in cache size_t key_size = key.size; @@ -1166,7 +1166,7 @@ private: cell.state = Cell::default_value; - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if constexpr (dictionary_key_type == DictionaryKeyType::Complex) { /// Copy complex key into arena and put in cache size_t key_size = key.size; @@ -1382,7 +1382,7 @@ private: using ComplexKeyHashMap = HashMapWithSavedHash; using CacheMap = std::conditional_t< - dictionary_key_type == DictionaryKeyType::simple, + dictionary_key_type == DictionaryKeyType::Simple, SimpleKeyHashMap, ComplexKeyHashMap>; diff --git a/src/Dictionaries/registerCacheDictionaries.cpp b/src/Dictionaries/registerCacheDictionaries.cpp index d039c5b6630..64c1c55e0ba 100644 --- a/src/Dictionaries/registerCacheDictionaries.cpp +++ b/src/Dictionaries/registerCacheDictionaries.cpp @@ -157,24 +157,23 @@ DictionaryPtr createCacheDictionaryLayout( ContextPtr context [[maybe_unused]], bool created_from_ddl [[maybe_unused]]) { - static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionary"); - String layout_type; - if constexpr (dictionary_key_type == DictionaryKeyType::simple && !ssd) + + if constexpr (dictionary_key_type == DictionaryKeyType::Simple && !ssd) layout_type = "cache"; - else if constexpr (dictionary_key_type == DictionaryKeyType::simple && ssd) + else if constexpr (dictionary_key_type == DictionaryKeyType::Simple && ssd) layout_type = "ssd_cache"; - else if constexpr (dictionary_key_type == DictionaryKeyType::complex && !ssd) + else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && !ssd) layout_type = "complex_key_cache"; - else if constexpr (dictionary_key_type == DictionaryKeyType::complex && ssd) + else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && ssd) layout_type = "complex_key_ssd_cache"; - if constexpr (dictionary_key_type == DictionaryKeyType::simple) + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) { if (dict_struct.key) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: dictionary of layout '{}' 'key' is not supported", full_name, layout_type); } - else if constexpr (dictionary_key_type == DictionaryKeyType::complex) + else if constexpr (dictionary_key_type == DictionaryKeyType::Complex) { if (dict_struct.id) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: dictionary of layout '{}' 'id' is not supported", full_name, layout_type); @@ -243,7 +242,7 @@ void registerDictionaryCache(DictionaryFactory & factory) ContextPtr context, bool created_from_ddl) -> DictionaryPtr { - return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); + return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); }; factory.registerLayout("cache", create_simple_cache_layout, false); @@ -256,7 +255,7 @@ void registerDictionaryCache(DictionaryFactory & factory) ContextPtr context, bool created_from_ddl) -> DictionaryPtr { - return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); + return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); }; factory.registerLayout("complex_key_cache", create_complex_key_cache_layout, true); @@ -271,7 +270,7 @@ void registerDictionaryCache(DictionaryFactory & factory) ContextPtr context, bool created_from_ddl) -> DictionaryPtr { - return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); + return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); }; factory.registerLayout("ssd_cache", create_simple_ssd_cache_layout, false); @@ -283,7 +282,7 @@ void registerDictionaryCache(DictionaryFactory & factory) DictionarySourcePtr source_ptr, ContextPtr context, bool created_from_ddl) -> DictionaryPtr { - return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); + return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); }; factory.registerLayout("complex_key_ssd_cache", create_complex_key_ssd_cache_layout, true); diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 02af528723c..5f94a1e1f4b 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -42,7 +42,6 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_COLUMN; - extern const int BAD_ARGUMENTS; extern const int TYPE_MISMATCH; } @@ -182,15 +181,16 @@ public: auto dictionary = helper.getDictionary(arguments[0].column); auto dictionary_key_type = dictionary->getKeyType(); + auto dictionary_special_key_type = dictionary->getSpecialKeyType(); - const ColumnWithTypeAndName & key_column_with_type = arguments[1]; + const auto & key_column_with_type = arguments[1]; auto key_column = key_column_with_type.column; auto key_column_type = key_column_with_type.type; - ColumnPtr range_col = nullptr; - DataTypePtr range_col_type = nullptr; + ColumnPtr range_col; + DataTypePtr range_col_type; - if (dictionary_key_type == DictionaryKeyType::range) + if (dictionary_special_key_type == DictionarySpecialKeyType::Range) { if (arguments.size() != 3) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, @@ -207,7 +207,10 @@ public: getName()); } - if (dictionary_key_type == DictionaryKeyType::simple) + Columns key_columns; + DataTypes key_types; + + if (dictionary_key_type == DictionaryKeyType::Simple) { if (!WhichDataType(key_column_type).isUInt64()) throw Exception( @@ -216,16 +219,23 @@ public: getName(), key_column_with_type.type->getName()); - return dictionary->hasKeys({key_column}, {std::make_shared()}); + key_columns = {key_column}; + key_types = {std::make_shared()}; } - else if (dictionary_key_type == DictionaryKeyType::complex) + else if (dictionary_key_type == DictionaryKeyType::Complex) { /// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. key_column = key_column->convertToFullColumnIfConst(); - size_t keys_size = dictionary->getStructure().getKeysSize(); - if (!isTuple(key_column_type)) + if (isTuple(key_column_type)) { + key_columns = assert_cast(*key_column).getColumnsCopy(); + key_types = assert_cast(*key_column_type).getElements(); + } + else + { + size_t keys_size = dictionary->getStructure().getKeysSize(); + if (keys_size > 1) { throw Exception( @@ -237,41 +247,19 @@ public: } else { - Columns tuple_columns = {std::move(key_column)}; - key_column = ColumnTuple::create(tuple_columns); - - DataTypes tuple_types = {key_column_type}; - key_column_type = std::make_shared(tuple_types); + key_columns = {key_column}; + key_types = {key_column_type}; } } - - const auto & key_columns = assert_cast(*key_column).getColumnsCopy(); - const auto & key_types = assert_cast(*key_column_type).getElements(); - - return dictionary->hasKeys(key_columns, key_types); } - else + + if (dictionary_special_key_type == DictionarySpecialKeyType::Range) { - /// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. - ColumnPtr key_column = key_column_with_type.column->convertToFullColumnIfConst(); - DataTypePtr key_column_type = key_column_with_type.type; - - Columns key_columns; - DataTypes key_types; - - if (isTuple(key_column_type)) - { - key_columns = assert_cast(*key_column).getColumnsCopy(); - key_types = assert_cast(*key_column_type).getElements(); - } - else - { - key_columns = {key_column, range_col}; - key_types = {std::make_shared(), range_col_type}; - } - - return dictionary->hasKeys({key_column, range_col}, {std::make_shared(), range_col_type}); + key_columns.emplace_back(range_col); + key_types.emplace_back(range_col_type); } + + return dictionary->hasKeys(key_columns, key_types); } private: @@ -369,13 +357,14 @@ public: auto dictionary = helper.getDictionary(dictionary_name); auto dictionary_key_type = dictionary->getKeyType(); + auto dictionary_special_key_type = dictionary->getSpecialKeyType(); size_t current_arguments_index = 3; ColumnPtr range_col = nullptr; DataTypePtr range_col_type = nullptr; - if (dictionary_key_type == DictionaryKeyType::range) + if (dictionary_special_key_type == DictionarySpecialKeyType::Range) { if (current_arguments_index >= arguments.size()) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, @@ -437,12 +426,13 @@ public: default_cols.emplace_back(nullptr); } - ColumnPtr result; + const auto & key_col_with_type = arguments[2]; + auto key_column = key_col_with_type.column; - const ColumnWithTypeAndName & key_col_with_type = arguments[2]; - const auto key_column = key_col_with_type.column; + Columns key_columns; + DataTypes key_types; - if (dictionary_key_type == DictionaryKeyType::simple) + if (dictionary_key_type == DictionaryKeyType::Simple) { if (!WhichDataType(key_col_with_type.type).isUInt64()) throw Exception( @@ -451,24 +441,24 @@ public: getName(), key_col_with_type.type->getName()); - result = executeDictionaryRequest( - dictionary, - attribute_names, - {key_column}, - {std::make_shared()}, - result_type, - default_cols); + key_columns = {key_column}; + key_types = {std::make_shared()}; } - else if (dictionary_key_type == DictionaryKeyType::complex) + else if (dictionary_key_type == DictionaryKeyType::Complex) { /// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. - ColumnPtr key_column = key_col_with_type.column->convertToFullColumnIfConst(); + key_column = key_column->convertToFullColumnIfConst(); DataTypePtr key_column_type = key_col_with_type.type; - size_t keys_size = dictionary->getStructure().getKeysSize(); - - if (!isTuple(key_column_type)) + if (isTuple(key_column_type)) { + key_columns = assert_cast(*key_column).getColumnsCopy(); + key_types = assert_cast(*key_column_type).getElements(); + } + else if (!isTuple(key_column_type)) + { + size_t keys_size = dictionary->getStructure().getKeysSize(); + if (keys_size > 1) { throw Exception( @@ -480,60 +470,19 @@ public: } else { - Columns tuple_columns = {std::move(key_column)}; - key_column = ColumnTuple::create(tuple_columns); - - DataTypes tuple_types = {key_column_type}; - key_column_type = std::make_shared(tuple_types); + key_columns = {std::move(key_column)}; + key_types = {std::move(key_column_type)}; } } - - const auto & key_columns = assert_cast(*key_column).getColumnsCopy(); - const auto & key_types = assert_cast(*key_column_type).getElements(); - - result = executeDictionaryRequest( - dictionary, - attribute_names, - key_columns, - key_types, - result_type, - default_cols); } - else if (dictionary_key_type == DictionaryKeyType::range) + + if (dictionary_special_key_type == DictionarySpecialKeyType::Range) { - /// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. - ColumnPtr key_column = key_col_with_type.column->convertToFullColumnIfConst(); - DataTypePtr key_column_type = key_col_with_type.type; - - Columns key_columns; - DataTypes key_types; - - if (isTuple(key_column_type)) - { - key_columns = assert_cast(*key_column).getColumnsCopy(); - key_types = assert_cast(*key_column_type).getElements(); - } - else - { - key_columns = {key_column}; - key_types = {std::make_shared()}; - } - key_columns.emplace_back(range_col); key_types.emplace_back(range_col_type); - - result = executeDictionaryRequest( - dictionary, - attribute_names, - key_columns, - key_types, - result_type, - default_cols); } - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown dictionary identifier type"); - return result; + return executeDictionaryRequest(dictionary, attribute_names, key_columns, key_types, result_type, default_cols); } private: