#pragma once #include #include #include #include #include #include #include #include #include #include #include #include #include /** This dictionary stores all content in a hash table in memory * (a separate Key -> Value map for each attribute) * Two variants of hash table are supported: a fast HashMap and memory efficient sparse_hash_map. */ namespace DB { struct HashedDictionaryConfiguration { const UInt64 shards; const UInt64 shard_load_queue_backlog; const bool require_nonempty; const DictionaryLifetime lifetime; }; template class ParallelDictionaryLoader; template class HashedDictionary final : public IDictionary { friend class ParallelDictionaryLoader; public: using KeyType = std::conditional_t; HashedDictionary( const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const HashedDictionaryConfiguration & configuration_, BlockPtr update_field_loaded_block_ = nullptr); ~HashedDictionary() override; std::string getTypeName() const override { if constexpr (dictionary_key_type == DictionaryKeyType::Simple && sparse) return "SparseHashed"; else if constexpr (dictionary_key_type == DictionaryKeyType::Simple && !sparse) return "Hashed"; else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && sparse) return "ComplexKeySparseHashed"; else return "ComplexKeyHashed"; } size_t getBytesAllocated() const override { return bytes_allocated; } size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); } double getFoundRate() const override { size_t queries = query_count.load(std::memory_order_relaxed); if (!queries) return 0; return static_cast(found_count.load(std::memory_order_relaxed)) / queries; } double getHitRate() const override { return 1.0; } size_t getElementCount() const override { return element_count; } double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } std::shared_ptr clone() const override { return std::make_shared>( getDictionaryID(), dict_struct, source_ptr->clone(), configuration, update_field_loaded_block); } DictionarySourcePtr getSource() const override { return source_ptr; } const DictionaryLifetime & getLifetime() const override { return configuration.lifetime; } const DictionaryStructure & getStructure() const override { return dict_struct; } bool isInjective(const std::string & attribute_name) const override { return dict_struct.getAttribute(attribute_name).injective; } DictionaryKeyType getKeyType() const override { return dictionary_key_type; } ColumnPtr getColumn( const std::string& attribute_name, const DataTypePtr & result_type, const Columns & key_columns, const DataTypes & key_types, const ColumnPtr & default_values_column) const override; 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(); } ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & hierarchy_attribute_type) const override; ColumnUInt8::Ptr isInHierarchy( ColumnPtr key_column, ColumnPtr in_key_column, const DataTypePtr & key_type) const override; DictionaryHierarchicalParentToChildIndexPtr getHierarchicalIndex() const override; size_t getHierarchicalIndexBytesAllocated() const override { return hierarchical_index_bytes_allocated; } ColumnPtr getDescendants( ColumnPtr key_column, const DataTypePtr & key_type, size_t level, DictionaryHierarchicalParentToChildIndexPtr parent_to_child_index) const override; Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const override; private: template using CollectionTypeNonSparse = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, HashMap>, HashMapWithSavedHash>>; using NoAttributesCollectionTypeNonSparse = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, HashSet>, HashSetWithSavedHash>>; /// Here we use sparse_hash_map with DefaultHash<> for the following reasons: /// /// - DefaultHash<> is used for HashMap /// - DefaultHash<> (from HashTable/Hash.h> works better then std::hash<> /// in case of sequential set of keys, but with random access to this set, i.e. /// /// SELECT number FROM numbers(3000000) ORDER BY rand() /// /// And even though std::hash<> works better in some other cases, /// DefaultHash<> is preferred since the difference for this particular /// case is significant, i.e. it can be 10x+. template using CollectionTypeSparse = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, google::sparse_hash_map>, google::sparse_hash_map>>; using NoAttributesCollectionTypeSparse = google::sparse_hash_set>; template using CollectionType = std::conditional_t, CollectionTypeNonSparse>; template using CollectionsHolder = std::vector>; using NoAttributesCollectionType = std::conditional_t; using NullableSet = HashSet>; using NullableSets = std::vector; struct Attribute final { AttributeUnderlyingType type; std::optional is_nullable_sets; std::variant< CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder, CollectionsHolder> containers; }; void createAttributes(); void blockToAttributes(const Block & block, DictionaryKeysArenaHolder & arena_holder, UInt64 shard); void updateData(); void loadData(); void buildHierarchyParentToChildIndexIfNeeded(); void calculateBytesAllocated(); UInt64 getShard(UInt64 key) const { if constexpr (!sharded) return 0; /// NOTE: function here should not match with the DefaultHash<> since /// it used for the HashMap/sparse_hash_map. return intHashCRC32(key) % configuration.shards; } UInt64 getShard(StringRef key) const { if constexpr (!sharded) return 0; return StringRefHash()(key) % configuration.shards; } template void getItemsImpl( const Attribute & attribute, DictionaryKeysExtractor & keys_extractor, ValueSetter && set_value, DefaultValueExtractor & default_value_extractor) const; template void getAttributeContainers(size_t attribute_index, GetContainersFunc && get_containers_func); template void getAttributeContainers(size_t attribute_index, GetContainersFunc && get_containers_func) const; void resize(size_t added_rows); Poco::Logger * log; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const HashedDictionaryConfiguration configuration; std::vector attributes; size_t bytes_allocated = 0; size_t hierarchical_index_bytes_allocated = 0; std::atomic element_count = 0; size_t bucket_count = 0; mutable std::atomic query_count{0}; mutable std::atomic found_count{0}; BlockPtr update_field_loaded_block; std::vector> string_arenas; std::vector no_attributes_containers; DictionaryHierarchicalParentToChildIndexPtr hierarchical_index; }; 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; }