#pragma once #include #include #include #include #include #include #include #include #include #include #include #include /** This dictionary stores all attributes in arrays. * Key is stored in hash table and value is index into attribute array. */ namespace DB { struct HashedArrayDictionaryStorageConfiguration { const bool require_nonempty; const DictionaryLifetime lifetime; size_t shards = 1; size_t shard_load_queue_backlog = 10000; bool use_async_executor = false; std::chrono::seconds load_timeout{0}; }; template class HashedArrayDictionary final : public IDictionary { using DictionaryParallelLoaderType = HashedDictionaryImpl::HashedDictionaryParallelLoader>; friend class HashedDictionaryImpl::HashedDictionaryParallelLoader>; public: using KeyType = std::conditional_t; HashedArrayDictionary( const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const HashedArrayDictionaryStorageConfiguration & configuration_, BlockPtr update_field_loaded_block_ = nullptr); std::string getTypeName() const override { if constexpr (dictionary_key_type == DictionaryKeyType::Simple) return "HashedArray"; else return "ComplexHashedArray"; } size_t getBytesAllocated() const override { return bytes_allocated; } size_t getQueryCount() const override { return query_count.load(); } double getFoundRate() const override { size_t queries = query_count.load(); if (!queries) return 0; return std::min(1.0, static_cast(found_count.load()) / queries); } double getHitRate() const override { return 1.0; } size_t getElementCount() const override { return total_element_count; } double getLoadFactor() const override { return static_cast(total_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 & attribute_type, const Columns & key_columns, const DataTypes & key_types, DefaultOrFilter default_or_filter) const override; Columns getColumns( const Strings & attribute_names, const DataTypes & attribute_types, const Columns & key_columns, const DataTypes & key_types, DefaultsOrFilter defaults_or_filter) 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: using KeyContainerType = std::conditional_t< dictionary_key_type == DictionaryKeyType::Simple, HashMap, HashMapWithSavedHash>>; template using AttributeContainerType = std::conditional_t, std::vector, PaddedPODArray>; template using AttributeContainerShardsType = std::vector>; struct Attribute final { AttributeUnderlyingType type; std::variant< AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType, AttributeContainerShardsType> containers; /// One container per shard using RowsMask = std::vector; std::optional> is_index_null; }; struct KeyAttribute final { /// One container per shard std::vector containers; }; void createAttributes(); void blockToAttributes(const Block & block, DictionaryKeysArenaHolder & arena_holder, size_t 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 ColumnPtr getAttributeColumn( const Attribute & attribute, const DictionaryAttribute & dictionary_attribute, size_t keys_size, DefaultOrFilter default_or_filter, KeysProvider && keys_object) const; template void getItemsImpl( const Attribute & attribute, DictionaryKeysExtractor & keys_extractor, ValueSetter && set_value, DefaultValueExtractor & default_value_extractor) const; template void getItemsShortCircuitImpl( const Attribute & attribute, DictionaryKeysExtractor & keys_extractor, ValueSetter && set_value, IColumn::Filter & default_mask) const; using KeyIndexToElementIndex = std::conditional_t>, PaddedPODArray>; template void getItemsImpl( const Attribute & attribute, const KeyIndexToElementIndex & key_index_to_element_index, ValueSetter && set_value, DefaultValueExtractor & default_value_extractor) const; template void getItemsShortCircuitImpl( const Attribute & attribute, const KeyIndexToElementIndex & key_index_to_element_index, ValueSetter && set_value, IColumn::Filter & default_mask [[maybe_unused]]) const; template void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func); template void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func) const; void resize(size_t total_rows); LoggerPtr log; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const HashedArrayDictionaryStorageConfiguration configuration; std::vector attributes; KeyAttribute key_attribute; size_t bytes_allocated = 0; size_t hierarchical_index_bytes_allocated = 0; std::atomic total_element_count = 0; std::vector element_counts; 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; DictionaryHierarchicalParentToChildIndexPtr hierarchical_index; }; extern template class HashedArrayDictionary; extern template class HashedArrayDictionary; extern template class HashedArrayDictionary; extern template class HashedArrayDictionary; }