mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #22096 from kitaisreal/hierarchy-dictionaries-updated
Refactored hierarchy dictionaries interface
This commit is contained in:
commit
d06fb1197a
@ -48,7 +48,7 @@ struct HashMapCell
|
||||
|
||||
value_type value;
|
||||
|
||||
HashMapCell() {}
|
||||
HashMapCell() = default;
|
||||
HashMapCell(const Key & key_, const State &) : value(key_, NoInitTag()) {}
|
||||
HashMapCell(const value_type & value_, const State &) : value(value_) {}
|
||||
|
||||
@ -114,8 +114,39 @@ struct HashMapCell
|
||||
|
||||
static void move(HashMapCell * /* old_location */, HashMapCell * /* new_location */) {}
|
||||
|
||||
template <size_t I>
|
||||
auto & get() & {
|
||||
if constexpr (I == 0) return value.first;
|
||||
else if constexpr (I == 1) return value.second;
|
||||
}
|
||||
|
||||
template <size_t I>
|
||||
auto const & get() const & {
|
||||
if constexpr (I == 0) return value.first;
|
||||
else if constexpr (I == 1) return value.second;
|
||||
}
|
||||
|
||||
template <size_t I>
|
||||
auto && get() && {
|
||||
if constexpr (I == 0) return std::move(value.first);
|
||||
else if constexpr (I == 1) return std::move(value.second);
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
namespace std
|
||||
{
|
||||
|
||||
template <typename Key, typename TMapped, typename Hash, typename TState>
|
||||
struct tuple_size<HashMapCell<Key, TMapped, Hash, TState>> : std::integral_constant<size_t, 2> { };
|
||||
|
||||
template <typename Key, typename TMapped, typename Hash, typename TState>
|
||||
struct tuple_element<0, HashMapCell<Key, TMapped, Hash, TState>> { using type = Key; };
|
||||
|
||||
template <typename Key, typename TMapped, typename Hash, typename TState>
|
||||
struct tuple_element<1, HashMapCell<Key, TMapped, Hash, TState>> { using type = TMapped; };
|
||||
}
|
||||
|
||||
template <typename Key, typename TMapped, typename Hash, typename TState = HashTableNoState>
|
||||
struct HashMapCellWithSavedHash : public HashMapCell<Key, TMapped, Hash, TState>
|
||||
{
|
||||
@ -227,6 +258,19 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
namespace std
|
||||
{
|
||||
|
||||
template <typename Key, typename TMapped, typename Hash, typename TState>
|
||||
struct tuple_size<HashMapCellWithSavedHash<Key, TMapped, Hash, TState>> : std::integral_constant<size_t, 2> { };
|
||||
|
||||
template <typename Key, typename TMapped, typename Hash, typename TState>
|
||||
struct tuple_element<0, HashMapCellWithSavedHash<Key, TMapped, Hash, TState>> { using type = Key; };
|
||||
|
||||
template <typename Key, typename TMapped, typename Hash, typename TState>
|
||||
struct tuple_element<1, HashMapCellWithSavedHash<Key, TMapped, Hash, TState>> { using type = TMapped; };
|
||||
}
|
||||
|
||||
|
||||
template <
|
||||
typename Key,
|
||||
|
@ -530,6 +530,31 @@ public:
|
||||
this->c_end += bytes_to_copy;
|
||||
}
|
||||
|
||||
template <typename ... TAllocatorParams>
|
||||
void insertFromItself(iterator from_begin, iterator from_end, TAllocatorParams && ... allocator_params)
|
||||
{
|
||||
static_assert(memcpy_can_be_used_for_assignment<std::decay_t<T>, std::decay_t<decltype(*from_begin)>>);
|
||||
|
||||
/// Convert iterators to indexes because reserve can invalidate iterators
|
||||
size_t start_index = from_begin - begin();
|
||||
size_t end_index = from_end - begin();
|
||||
size_t copy_size = end_index - start_index;
|
||||
|
||||
assert(start_index <= end_index);
|
||||
|
||||
size_t required_capacity = this->size() + copy_size;
|
||||
if (required_capacity > this->capacity())
|
||||
this->reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward<TAllocatorParams>(allocator_params)...);
|
||||
|
||||
size_t bytes_to_copy = this->byte_size(copy_size);
|
||||
if (bytes_to_copy)
|
||||
{
|
||||
auto begin = this->c_start + this->byte_size(start_index);
|
||||
memcpy(this->c_end, reinterpret_cast<const void *>(&*begin), bytes_to_copy);
|
||||
this->c_end += bytes_to_copy;
|
||||
}
|
||||
}
|
||||
|
||||
template <typename It1, typename It2>
|
||||
void insert_assume_reserved(It1 from_begin, It2 from_end)
|
||||
{
|
||||
|
@ -33,6 +33,19 @@ TEST(Common, PODArrayInsert)
|
||||
EXPECT_EQ(str, std::string(chars.data(), chars.size()));
|
||||
}
|
||||
|
||||
TEST(Common, PODArrayInsertFromItself)
|
||||
{
|
||||
{
|
||||
PaddedPODArray<UInt64> array { 1 };
|
||||
|
||||
for (size_t i = 0; i < 3; ++i)
|
||||
array.insertFromItself(array.begin(), array.end());
|
||||
|
||||
PaddedPODArray<UInt64> expected {1,1,1,1,1,1,1,1};
|
||||
ASSERT_EQ(array,expected);
|
||||
}
|
||||
}
|
||||
|
||||
TEST(Common, PODPushBackRawMany)
|
||||
{
|
||||
PODArray<char> chars;
|
||||
|
@ -567,7 +567,7 @@ void DatabaseAtomic::renameDictionaryInMemoryUnlocked(const StorageID & old_name
|
||||
auto result = external_loader.getLoadResult(toString(old_name.uuid));
|
||||
if (!result.object)
|
||||
return;
|
||||
const auto & dict = dynamic_cast<const IDictionaryBase &>(*result.object);
|
||||
const auto & dict = dynamic_cast<const IDictionary &>(*result.object);
|
||||
dict.updateDictionaryName(new_name);
|
||||
}
|
||||
void DatabaseAtomic::waitDetachedTableNotInUse(const UUID & uuid)
|
||||
|
@ -49,7 +49,7 @@ void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name,
|
||||
/// Attach the dictionary as table too.
|
||||
try
|
||||
{
|
||||
/// TODO Make StorageDictionary an owner of IDictionaryBase objects.
|
||||
/// TODO Make StorageDictionary an owner of IDictionary objects.
|
||||
/// All DDL operations with dictionaries will work with StorageDictionary table,
|
||||
/// and StorageDictionary will be responsible for loading of DDL dictionaries.
|
||||
/// ExternalLoaderDatabaseConfigRepository and other hacks related to ExternalLoader
|
||||
|
@ -13,7 +13,9 @@
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/ProfilingScopedRWLock.h>
|
||||
|
||||
#include <Dictionaries/DictionaryBlockInputStream.h>
|
||||
#include <Dictionaries/HierarchyDictionariesUtils.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -39,7 +41,6 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CACHE_DICTIONARY_UPDATE_FAIL;
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
@ -70,8 +71,6 @@ CacheDictionary<dictionary_key_type>::CacheDictionary(
|
||||
{
|
||||
if (!source_ptr->supportsSelectiveLoad())
|
||||
throw Exception{full_name + ": source cannot be used with CacheDictionary", ErrorCodes::UNSUPPORTED_METHOD};
|
||||
|
||||
setupHierarchicalAttribute();
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
@ -120,164 +119,6 @@ const IDictionarySource * CacheDictionary<dictionary_key_type>::getSource() cons
|
||||
return source_ptr.get();
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
void CacheDictionary<dictionary_key_type>::toParent(const PaddedPODArray<UInt64> & ids [[maybe_unused]], PaddedPODArray<UInt64> & out [[maybe_unused]]) const
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
{
|
||||
/// Run update on requested keys before fetch from storage
|
||||
const auto & attribute_name = hierarchical_attribute->name;
|
||||
|
||||
auto result_type = std::make_shared<DataTypeUInt64>();
|
||||
auto input_column = result_type->createColumn();
|
||||
auto & input_column_typed = assert_cast<ColumnVector<UInt64> &>(*input_column);
|
||||
auto & data = input_column_typed.getData();
|
||||
data.insert(ids.begin(), ids.end());
|
||||
|
||||
auto column = getColumn({attribute_name}, result_type, {std::move(input_column)}, {result_type}, {nullptr});
|
||||
const auto & result_column_typed = assert_cast<const ColumnVector<UInt64> &>(*column);
|
||||
const auto & result_data = result_column_typed.getData();
|
||||
|
||||
out.assign(result_data);
|
||||
}
|
||||
else
|
||||
throw Exception("Hierarchy is not supported for complex key CacheDictionary", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
|
||||
/// Allow to use single value in same way as array.
|
||||
static inline UInt64 getAt(const PaddedPODArray<UInt64> & arr, const size_t idx)
|
||||
{
|
||||
return arr[idx];
|
||||
}
|
||||
static inline UInt64 getAt(const UInt64 & value, const size_t)
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
template <typename AncestorType>
|
||||
void CacheDictionary<dictionary_key_type>::isInImpl(const PaddedPODArray<Key> & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
/// Transform all children to parents until ancestor id or null_value will be reached.
|
||||
|
||||
size_t out_size = out.size();
|
||||
memset(out.data(), 0xFF, out_size); /// 0xFF means "not calculated"
|
||||
|
||||
const auto null_value = hierarchical_attribute->null_value.get<UInt64>();
|
||||
|
||||
PaddedPODArray<Key> children(out_size, 0);
|
||||
PaddedPODArray<Key> parents(child_ids.begin(), child_ids.end());
|
||||
|
||||
for (size_t i = 0; i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i)
|
||||
{
|
||||
size_t out_idx = 0;
|
||||
size_t parents_idx = 0;
|
||||
size_t new_children_idx = 0;
|
||||
|
||||
while (out_idx < out_size)
|
||||
{
|
||||
/// Already calculated
|
||||
if (out[out_idx] != 0xFF)
|
||||
{
|
||||
++out_idx;
|
||||
continue;
|
||||
}
|
||||
|
||||
/// No parent
|
||||
if (parents[parents_idx] == null_value)
|
||||
{
|
||||
out[out_idx] = 0;
|
||||
}
|
||||
/// Found ancestor
|
||||
else if (parents[parents_idx] == getAt(ancestor_ids, parents_idx))
|
||||
{
|
||||
out[out_idx] = 1;
|
||||
}
|
||||
/// Loop detected
|
||||
else if (children[new_children_idx] == parents[parents_idx])
|
||||
{
|
||||
out[out_idx] = 1;
|
||||
}
|
||||
/// Found intermediate parent, add this value to search at next loop iteration
|
||||
else
|
||||
{
|
||||
children[new_children_idx] = parents[parents_idx];
|
||||
++new_children_idx;
|
||||
}
|
||||
|
||||
++out_idx;
|
||||
++parents_idx;
|
||||
}
|
||||
|
||||
if (new_children_idx == 0)
|
||||
break;
|
||||
|
||||
/// Transform all children to its parents.
|
||||
children.resize(new_children_idx);
|
||||
parents.resize(new_children_idx);
|
||||
|
||||
toParent(children, parents);
|
||||
}
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
void CacheDictionary<dictionary_key_type>::isInVectorVector(
|
||||
const PaddedPODArray<UInt64> & child_ids, const PaddedPODArray<UInt64> & ancestor_ids, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
isInImpl(child_ids, ancestor_ids, out);
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
void CacheDictionary<dictionary_key_type>::isInVectorConstant(const PaddedPODArray<UInt64> & child_ids, const UInt64 ancestor_id, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
isInImpl(child_ids, ancestor_id, out);
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
void CacheDictionary<dictionary_key_type>::isInConstantVector(const UInt64 child_id, const PaddedPODArray<UInt64> & ancestor_ids, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
/// Special case with single child value.
|
||||
|
||||
const auto null_value = hierarchical_attribute->null_value.get<UInt64>();
|
||||
|
||||
PaddedPODArray<Key> child(1, child_id);
|
||||
PaddedPODArray<Key> parent(1);
|
||||
std::vector<Key> ancestors(1, child_id);
|
||||
|
||||
/// Iteratively find all ancestors for child.
|
||||
for (size_t i = 0; i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i)
|
||||
{
|
||||
toParent(child, parent);
|
||||
|
||||
if (parent[0] == null_value)
|
||||
break;
|
||||
|
||||
child[0] = parent[0];
|
||||
ancestors.push_back(parent[0]);
|
||||
}
|
||||
|
||||
/// Assuming short hierarchy, so linear search is Ok.
|
||||
for (size_t i = 0, out_size = out.size(); i < out_size; ++i)
|
||||
out[i] = std::find(ancestors.begin(), ancestors.end(), ancestor_ids[i]) != ancestors.end();
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
void CacheDictionary<dictionary_key_type>::setupHierarchicalAttribute()
|
||||
{
|
||||
/// TODO: Move this to DictionaryStructure
|
||||
for (const auto & attribute : dict_struct.attributes)
|
||||
{
|
||||
if (attribute.hierarchical)
|
||||
{
|
||||
hierarchical_attribute = &attribute;
|
||||
|
||||
if (attribute.underlying_type != AttributeUnderlyingType::utUInt64)
|
||||
throw Exception{full_name + ": hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
ColumnPtr CacheDictionary<dictionary_key_type>::getColumn(
|
||||
const std::string & attribute_name,
|
||||
@ -296,23 +137,6 @@ Columns CacheDictionary<dictionary_key_type>::getColumns(
|
||||
const Columns & key_columns,
|
||||
const DataTypes & key_types,
|
||||
const Columns & default_values_columns) const
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::complex)
|
||||
dict_struct.validateKeyTypes(key_types);
|
||||
|
||||
Arena complex_keys_arena;
|
||||
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns, complex_keys_arena);
|
||||
auto & keys = extractor.getKeys();
|
||||
|
||||
return getColumnsImpl(attribute_names, key_columns, keys, default_values_columns);
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
Columns CacheDictionary<dictionary_key_type>::getColumnsImpl(
|
||||
const Strings & attribute_names,
|
||||
const Columns & key_columns,
|
||||
const PaddedPODArray<KeyType> & keys,
|
||||
const Columns & default_values_columns) const
|
||||
{
|
||||
/**
|
||||
* Flow of getColumsImpl
|
||||
@ -328,6 +152,13 @@ Columns CacheDictionary<dictionary_key_type>::getColumnsImpl(
|
||||
* use default value.
|
||||
*/
|
||||
|
||||
if (dictionary_key_type == DictionaryKeyType::complex)
|
||||
dict_struct.validateKeyTypes(key_types);
|
||||
|
||||
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
|
||||
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns, arena_holder.getComplexKeyArena());
|
||||
auto keys = extractor.extractAllKeys();
|
||||
|
||||
DictionaryStorageFetchRequest request(dict_struct, attribute_names, default_values_columns);
|
||||
|
||||
FetchResult result_of_fetch_from_storage;
|
||||
@ -440,9 +271,10 @@ ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::hasKeys(const Columns & k
|
||||
if (dictionary_key_type == DictionaryKeyType::complex)
|
||||
dict_struct.validateKeyTypes(key_types);
|
||||
|
||||
Arena complex_keys_arena;
|
||||
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns, complex_keys_arena);
|
||||
const auto & keys = extractor.getKeys();
|
||||
|
||||
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
|
||||
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns, arena_holder.getComplexKeyArena());
|
||||
const auto keys = extractor.extractAllKeys();
|
||||
|
||||
/// We make empty request just to fetch if keys exists
|
||||
DictionaryStorageFetchRequest request(dict_struct, {}, {});
|
||||
@ -526,6 +358,37 @@ ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::hasKeys(const Columns & k
|
||||
return result;
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
ColumnPtr CacheDictionary<dictionary_key_type>::getHierarchy(
|
||||
ColumnPtr key_column [[maybe_unused]],
|
||||
const DataTypePtr & key_type [[maybe_unused]]) const
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
{
|
||||
auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type);
|
||||
query_count.fetch_add(key_column->size(), std::memory_order_relaxed);
|
||||
return result;
|
||||
}
|
||||
else
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::isInHierarchy(
|
||||
ColumnPtr key_column [[maybe_unused]],
|
||||
ColumnPtr in_key_column [[maybe_unused]],
|
||||
const DataTypePtr & key_type [[maybe_unused]]) const
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
{
|
||||
auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type);
|
||||
query_count.fetch_add(key_column->size(), std::memory_order_relaxed);
|
||||
return result;
|
||||
}
|
||||
else
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
MutableColumns CacheDictionary<dictionary_key_type>::aggregateColumnsInOrderOfKeys(
|
||||
const PaddedPODArray<KeyType> & keys,
|
||||
@ -618,19 +481,18 @@ MutableColumns CacheDictionary<dictionary_key_type>::aggregateColumns(
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
BlockInputStreamPtr CacheDictionary<dictionary_key_type>::getBlockInputStream(const Names & column_names, size_t max_block_size) const
|
||||
{
|
||||
using BlockInputStreamType = DictionaryBlockInputStream<Key>;
|
||||
std::shared_ptr<BlockInputStreamType> stream;
|
||||
std::shared_ptr<DictionaryBlockInputStream> stream;
|
||||
|
||||
{
|
||||
/// Write lock on storage
|
||||
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
|
||||
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
stream = std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, cache_storage_ptr->getCachedSimpleKeys(), column_names);
|
||||
stream = std::make_shared<DictionaryBlockInputStream>(shared_from_this(), max_block_size, cache_storage_ptr->getCachedSimpleKeys(), column_names);
|
||||
else
|
||||
{
|
||||
auto keys = cache_storage_ptr->getCachedComplexKeys();
|
||||
stream = std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, keys, column_names);
|
||||
stream = std::make_shared<DictionaryBlockInputStream>(shared_from_this(), max_block_size, keys, column_names);
|
||||
}
|
||||
}
|
||||
|
||||
@ -660,14 +522,20 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
|
||||
|
||||
size_t found_keys_size = 0;
|
||||
|
||||
DictionaryKeysExtractor<dictionary_key_type> requested_keys_extractor(update_unit_ptr->key_columns, update_unit_ptr->complex_key_arena);
|
||||
const auto & requested_keys = requested_keys_extractor.getKeys();
|
||||
Arena * complex_key_arena = update_unit_ptr->complex_keys_arena_holder.getComplexKeyArena();
|
||||
DictionaryKeysExtractor<dictionary_key_type> requested_keys_extractor(update_unit_ptr->key_columns, complex_key_arena);
|
||||
auto requested_keys = requested_keys_extractor.extractAllKeys();
|
||||
|
||||
HashSet<KeyType> not_found_keys;
|
||||
|
||||
std::vector<UInt64> requested_keys_vector;
|
||||
std::vector<size_t> requested_complex_key_rows;
|
||||
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
requested_keys_vector.reserve(requested_keys.size());
|
||||
else
|
||||
requested_complex_key_rows.reserve(requested_keys.size());
|
||||
|
||||
auto & key_index_to_state_from_storage = update_unit_ptr->key_index_to_state;
|
||||
|
||||
for (size_t i = 0; i < key_index_to_state_from_storage.size(); ++i)
|
||||
@ -727,8 +595,8 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
|
||||
block_columns.erase(block_columns.begin());
|
||||
}
|
||||
|
||||
DictionaryKeysExtractor<dictionary_key_type> keys_extractor(key_columns, update_unit_ptr->complex_key_arena);
|
||||
const auto & keys_extracted_from_block = keys_extractor.getKeys();
|
||||
DictionaryKeysExtractor<dictionary_key_type> keys_extractor(key_columns, complex_key_arena);
|
||||
auto keys_extracted_from_block = keys_extractor.extractAllKeys();
|
||||
|
||||
for (size_t index_of_attribute = 0; index_of_attribute < fetched_columns_during_update.size(); ++index_of_attribute)
|
||||
{
|
||||
@ -740,6 +608,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
|
||||
for (size_t i = 0; i < keys_extracted_from_block.size(); ++i)
|
||||
{
|
||||
auto fetched_key_from_source = keys_extracted_from_block[i];
|
||||
|
||||
not_found_keys.erase(fetched_key_from_source);
|
||||
update_unit_ptr->requested_keys_to_fetched_columns_during_update_index[fetched_key_from_source] = found_keys_size;
|
||||
found_keys_in_source.emplace_back(fetched_key_from_source);
|
||||
|
@ -130,33 +130,18 @@ public:
|
||||
|
||||
std::exception_ptr getLastException() const override;
|
||||
|
||||
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && hierarchical_attribute; }
|
||||
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); }
|
||||
|
||||
void toParent(const PaddedPODArray<UInt64> & ids, PaddedPODArray<UInt64> & out) const override;
|
||||
ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override;
|
||||
|
||||
void isInVectorVector(
|
||||
const PaddedPODArray<UInt64> & child_ids,
|
||||
const PaddedPODArray<UInt64> & ancestor_ids,
|
||||
PaddedPODArray<UInt8> & out) const override;
|
||||
|
||||
void isInVectorConstant(
|
||||
const PaddedPODArray<UInt64> & child_ids,
|
||||
const UInt64 ancestor_id, PaddedPODArray<UInt8> & out) const override;
|
||||
|
||||
void isInConstantVector(
|
||||
const UInt64 child_id,
|
||||
const PaddedPODArray<UInt64> & ancestor_ids,
|
||||
PaddedPODArray<UInt8> & out) const override;
|
||||
ColumnUInt8::Ptr isInHierarchy(
|
||||
ColumnPtr key_column,
|
||||
ColumnPtr in_key_column,
|
||||
const DataTypePtr & key_type) const override;
|
||||
|
||||
private:
|
||||
using FetchResult = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, SimpleKeysStorageFetchResult, ComplexKeysStorageFetchResult>;
|
||||
|
||||
Columns getColumnsImpl(
|
||||
const Strings & attribute_names,
|
||||
const Columns & key_columns,
|
||||
const PaddedPODArray<KeyType> & keys,
|
||||
const Columns & default_values_columns) const;
|
||||
|
||||
static MutableColumns aggregateColumnsInOrderOfKeys(
|
||||
const PaddedPODArray<KeyType> & keys,
|
||||
const DictionaryStorageFetchRequest & request,
|
||||
@ -171,8 +156,6 @@ private:
|
||||
const MutableColumns & fetched_columns_during_update,
|
||||
const HashMap<KeyType, size_t> & found_keys_to_fetched_columns_during_update_index);
|
||||
|
||||
void setupHierarchicalAttribute();
|
||||
|
||||
void update(CacheDictionaryUpdateUnitPtr<dictionary_key_type> update_unit_ptr);
|
||||
|
||||
/// Update dictionary source pointer if required and return it. Thread safe.
|
||||
@ -193,9 +176,6 @@ private:
|
||||
return source_ptr;
|
||||
}
|
||||
|
||||
template <typename AncestorType>
|
||||
void isInImpl(const PaddedPODArray<Key> & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
|
||||
|
||||
const DictionaryStructure dict_struct;
|
||||
|
||||
/// Dictionary source should be used with mutex
|
||||
@ -218,8 +198,6 @@ private:
|
||||
/// readers. Surprisingly this lock is also used for last_exception pointer.
|
||||
mutable std::shared_mutex rw_lock;
|
||||
|
||||
const DictionaryAttribute * hierarchical_attribute = nullptr;
|
||||
|
||||
mutable std::exception_ptr last_exception;
|
||||
mutable std::atomic<size_t> error_count {0};
|
||||
mutable std::atomic<std::chrono::system_clock::time_point> backoff_end_time{std::chrono::system_clock::time_point{}};
|
||||
|
@ -66,8 +66,9 @@ public:
|
||||
|
||||
HashMap<KeyType, size_t> requested_keys_to_fetched_columns_during_update_index;
|
||||
MutableColumns fetched_columns_during_update;
|
||||
|
||||
/// Complex keys are serialized in this arena
|
||||
Arena complex_key_arena;
|
||||
DictionaryKeysArenaHolder<dictionary_key_type> complex_keys_arena_holder;
|
||||
|
||||
private:
|
||||
template <DictionaryKeyType>
|
||||
|
@ -1,594 +0,0 @@
|
||||
#include "ComplexKeyHashedDictionary.h"
|
||||
#include <ext/map.h>
|
||||
#include <ext/range.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include "DictionaryBlockInputStream.h"
|
||||
#include "DictionaryFactory.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int DICTIONARY_IS_EMPTY;
|
||||
}
|
||||
|
||||
ComplexKeyHashedDictionary::ComplexKeyHashedDictionary(
|
||||
const StorageID & dict_id_,
|
||||
const DictionaryStructure & dict_struct_,
|
||||
DictionarySourcePtr source_ptr_,
|
||||
const DictionaryLifetime dict_lifetime_,
|
||||
bool require_nonempty_,
|
||||
BlockPtr saved_block_)
|
||||
: IDictionaryBase(dict_id_)
|
||||
, dict_struct(dict_struct_)
|
||||
, source_ptr{std::move(source_ptr_)}
|
||||
, dict_lifetime(dict_lifetime_)
|
||||
, require_nonempty(require_nonempty_)
|
||||
, saved_block{std::move(saved_block_)}
|
||||
{
|
||||
createAttributes();
|
||||
loadData();
|
||||
calculateBytesAllocated();
|
||||
}
|
||||
|
||||
ColumnPtr ComplexKeyHashedDictionary::getColumn(
|
||||
const std::string & attribute_name,
|
||||
const DataTypePtr & result_type,
|
||||
const Columns & key_columns,
|
||||
const DataTypes & key_types,
|
||||
const ColumnPtr & default_values_column) const
|
||||
{
|
||||
dict_struct.validateKeyTypes(key_types);
|
||||
|
||||
ColumnPtr result;
|
||||
|
||||
const auto & attribute = getAttribute(attribute_name);
|
||||
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
|
||||
|
||||
auto keys_size = key_columns.front()->size();
|
||||
|
||||
ColumnUInt8::MutablePtr col_null_map_to;
|
||||
ColumnUInt8::Container * vec_null_map_to = nullptr;
|
||||
if (attribute.is_nullable)
|
||||
{
|
||||
col_null_map_to = ColumnUInt8::create(keys_size, false);
|
||||
vec_null_map_to = &col_null_map_to->getData();
|
||||
}
|
||||
|
||||
auto type_call = [&](const auto &dictionary_attribute_type)
|
||||
{
|
||||
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
|
||||
using AttributeType = typename Type::AttributeType;
|
||||
using ValueType = DictionaryValueType<AttributeType>;
|
||||
using ColumnProvider = DictionaryAttributeColumnProvider<AttributeType>;
|
||||
|
||||
const auto attribute_null_value = std::get<ValueType>(attribute.null_values);
|
||||
AttributeType null_value = static_cast<AttributeType>(attribute_null_value);
|
||||
DictionaryDefaultValueExtractor<AttributeType> default_value_extractor(std::move(null_value), default_values_column);
|
||||
|
||||
auto column = ColumnProvider::getColumn(dictionary_attribute, keys_size);
|
||||
|
||||
if constexpr (std::is_same_v<AttributeType, String>)
|
||||
{
|
||||
auto * out = column.get();
|
||||
|
||||
getItemsImpl<StringRef, StringRef>(
|
||||
attribute,
|
||||
key_columns,
|
||||
[&](const size_t row, const StringRef value, bool is_null)
|
||||
{
|
||||
if (attribute.is_nullable)
|
||||
(*vec_null_map_to)[row] = is_null;
|
||||
|
||||
out->insertData(value.data, value.size);
|
||||
},
|
||||
default_value_extractor);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto & out = column->getData();
|
||||
|
||||
getItemsImpl<AttributeType, AttributeType>(
|
||||
attribute,
|
||||
key_columns,
|
||||
[&](const size_t row, const auto value, bool is_null)
|
||||
{
|
||||
if (attribute.is_nullable)
|
||||
(*vec_null_map_to)[row] = is_null;
|
||||
|
||||
out[row] = value;
|
||||
},
|
||||
default_value_extractor);
|
||||
}
|
||||
|
||||
result = std::move(column);
|
||||
};
|
||||
|
||||
callOnDictionaryAttributeType(attribute.type, type_call);
|
||||
|
||||
if (attribute.is_nullable)
|
||||
{
|
||||
result = ColumnNullable::create(result, std::move(col_null_map_to));
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
ColumnUInt8::Ptr ComplexKeyHashedDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types) const
|
||||
{
|
||||
dict_struct.validateKeyTypes(key_types);
|
||||
|
||||
auto size = key_columns.front()->size();
|
||||
auto result = ColumnUInt8::create(size);
|
||||
auto& out = result->getData();
|
||||
|
||||
const auto & attribute = attributes.front();
|
||||
|
||||
auto type_call = [&](const auto & dictionary_attribute_type)
|
||||
{
|
||||
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
|
||||
using AttributeType = typename Type::AttributeType;
|
||||
using ValueType = DictionaryValueType<AttributeType>;
|
||||
|
||||
has<ValueType>(attribute, key_columns, out);
|
||||
};
|
||||
|
||||
callOnDictionaryAttributeType(attribute.type, type_call);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
void ComplexKeyHashedDictionary::createAttributes()
|
||||
{
|
||||
const auto size = dict_struct.attributes.size();
|
||||
attributes.reserve(size);
|
||||
|
||||
for (const auto & attribute : dict_struct.attributes)
|
||||
{
|
||||
attribute_index_by_name.emplace(attribute.name, attributes.size());
|
||||
attributes.push_back(createAttribute(attribute, attribute.null_value));
|
||||
|
||||
if (attribute.hierarchical)
|
||||
throw Exception{full_name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(),
|
||||
ErrorCodes::TYPE_MISMATCH};
|
||||
}
|
||||
}
|
||||
|
||||
void ComplexKeyHashedDictionary::blockToAttributes(const Block & block)
|
||||
{
|
||||
/// created upfront to avoid excess allocations
|
||||
const auto keys_size = dict_struct.key->size();
|
||||
StringRefs keys(keys_size);
|
||||
|
||||
const auto attributes_size = attributes.size();
|
||||
const auto rows = block.rows();
|
||||
element_count += rows;
|
||||
|
||||
const auto key_column_ptrs = ext::map<Columns>(
|
||||
ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; });
|
||||
|
||||
const auto attribute_column_ptrs = ext::map<Columns>(ext::range(0, attributes_size), [&](const size_t attribute_idx)
|
||||
{
|
||||
return block.safeGetByPosition(keys_size + attribute_idx).column;
|
||||
});
|
||||
|
||||
for (const auto row_idx : ext::range(0, rows))
|
||||
{
|
||||
/// calculate key once per row
|
||||
const auto key = placeKeysInPool(row_idx, key_column_ptrs, keys, keys_pool);
|
||||
|
||||
auto should_rollback = false;
|
||||
|
||||
for (const auto attribute_idx : ext::range(0, attributes_size))
|
||||
{
|
||||
const auto & attribute_column = *attribute_column_ptrs[attribute_idx];
|
||||
auto & attribute = attributes[attribute_idx];
|
||||
const auto inserted = setAttributeValue(attribute, key, attribute_column[row_idx]);
|
||||
if (!inserted)
|
||||
should_rollback = true;
|
||||
}
|
||||
|
||||
/// @note on multiple equal keys the mapped value for the first one is stored
|
||||
if (should_rollback)
|
||||
keys_pool.rollback(key.size);
|
||||
}
|
||||
}
|
||||
|
||||
void ComplexKeyHashedDictionary::updateData()
|
||||
{
|
||||
/// created upfront to avoid excess allocations
|
||||
const auto keys_size = dict_struct.key->size();
|
||||
StringRefs keys(keys_size);
|
||||
|
||||
const auto attributes_size = attributes.size();
|
||||
|
||||
if (!saved_block || saved_block->rows() == 0)
|
||||
{
|
||||
auto stream = source_ptr->loadUpdatedAll();
|
||||
stream->readPrefix();
|
||||
|
||||
while (const auto block = stream->read())
|
||||
{
|
||||
/// We are using this method to keep saved data if input stream consists of multiple blocks
|
||||
if (!saved_block)
|
||||
saved_block = std::make_shared<DB::Block>(block.cloneEmpty());
|
||||
for (const auto attribute_idx : ext::range(0, keys_size + attributes_size))
|
||||
{
|
||||
const IColumn & update_column = *block.getByPosition(attribute_idx).column.get();
|
||||
MutableColumnPtr saved_column = saved_block->getByPosition(attribute_idx).column->assumeMutable();
|
||||
saved_column->insertRangeFrom(update_column, 0, update_column.size());
|
||||
}
|
||||
}
|
||||
stream->readSuffix();
|
||||
}
|
||||
else
|
||||
{
|
||||
auto stream = source_ptr->loadUpdatedAll();
|
||||
|
||||
stream->readPrefix();
|
||||
while (Block block = stream->read())
|
||||
{
|
||||
const auto saved_key_column_ptrs = ext::map<Columns>(
|
||||
ext::range(0, keys_size), [&](const size_t key_idx) { return saved_block->safeGetByPosition(key_idx).column; });
|
||||
|
||||
const auto update_key_column_ptrs = ext::map<Columns>(
|
||||
ext::range(0, keys_size), [&](const size_t key_idx) { return block.safeGetByPosition(key_idx).column; });
|
||||
|
||||
Arena temp_key_pool;
|
||||
ContainerType<std::vector<size_t>> update_key_hash;
|
||||
|
||||
for (size_t i = 0; i < block.rows(); ++i)
|
||||
{
|
||||
const auto u_key = placeKeysInPool(i, update_key_column_ptrs, keys, temp_key_pool);
|
||||
update_key_hash[u_key].push_back(i);
|
||||
}
|
||||
|
||||
const size_t rows = saved_block->rows();
|
||||
IColumn::Filter filter(rows);
|
||||
|
||||
for (size_t i = 0; i < saved_block->rows(); ++i)
|
||||
{
|
||||
const auto s_key = placeKeysInPool(i, saved_key_column_ptrs, keys, temp_key_pool);
|
||||
auto * it = update_key_hash.find(s_key);
|
||||
if (it)
|
||||
filter[i] = 0;
|
||||
else
|
||||
filter[i] = 1;
|
||||
}
|
||||
|
||||
auto block_columns = block.mutateColumns();
|
||||
for (const auto attribute_idx : ext::range(0, keys_size + attributes_size))
|
||||
{
|
||||
auto & column = saved_block->safeGetByPosition(attribute_idx).column;
|
||||
const auto & filtered_column = column->filter(filter, -1);
|
||||
|
||||
block_columns[attribute_idx]->insertRangeFrom(*filtered_column.get(), 0, filtered_column->size());
|
||||
}
|
||||
|
||||
saved_block->setColumns(std::move(block_columns));
|
||||
}
|
||||
stream->readSuffix();
|
||||
}
|
||||
|
||||
if (saved_block)
|
||||
blockToAttributes(*saved_block.get());
|
||||
}
|
||||
|
||||
void ComplexKeyHashedDictionary::loadData()
|
||||
{
|
||||
if (!source_ptr->hasUpdateField())
|
||||
{
|
||||
auto stream = source_ptr->loadAll();
|
||||
stream->readPrefix();
|
||||
|
||||
while (const auto block = stream->read())
|
||||
blockToAttributes(block);
|
||||
|
||||
stream->readSuffix();
|
||||
}
|
||||
else
|
||||
updateData();
|
||||
|
||||
if (require_nonempty && 0 == element_count)
|
||||
throw Exception{full_name + ": dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY};
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void ComplexKeyHashedDictionary::addAttributeSize(const Attribute & attribute)
|
||||
{
|
||||
const auto & map_ref = std::get<ContainerType<T>>(attribute.maps);
|
||||
bytes_allocated += sizeof(ContainerType<T>) + map_ref.getBufferSizeInBytes();
|
||||
bucket_count = map_ref.getBufferSizeInCells();
|
||||
}
|
||||
|
||||
template <>
|
||||
void ComplexKeyHashedDictionary::addAttributeSize<String>(const Attribute & attribute)
|
||||
{
|
||||
const auto & map_ref = std::get<ContainerType<StringRef>>(attribute.maps);
|
||||
bytes_allocated += sizeof(ContainerType<StringRef>) + map_ref.getBufferSizeInBytes();
|
||||
bucket_count = map_ref.getBufferSizeInCells();
|
||||
bytes_allocated += sizeof(Arena) + attribute.string_arena->size();
|
||||
}
|
||||
|
||||
void ComplexKeyHashedDictionary::calculateBytesAllocated()
|
||||
{
|
||||
bytes_allocated += attributes.size() * sizeof(attributes.front());
|
||||
|
||||
for (const auto & attribute : attributes)
|
||||
{
|
||||
auto type_call = [&](const auto & dictionary_attribute_type)
|
||||
{
|
||||
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
|
||||
using AttributeType = typename Type::AttributeType;
|
||||
|
||||
addAttributeSize<AttributeType>(attribute);
|
||||
};
|
||||
|
||||
callOnDictionaryAttributeType(attribute.type, type_call);
|
||||
}
|
||||
|
||||
bytes_allocated += keys_pool.size();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void ComplexKeyHashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value)
|
||||
{
|
||||
attribute.null_values = T(null_value.get<T>());
|
||||
attribute.maps.emplace<ContainerType<T>>();
|
||||
}
|
||||
|
||||
template <>
|
||||
void ComplexKeyHashedDictionary::createAttributeImpl<String>(Attribute & attribute, const Field & null_value)
|
||||
{
|
||||
attribute.string_arena = std::make_unique<Arena>();
|
||||
const String & string = null_value.get<String>();
|
||||
const char * string_in_arena = attribute.string_arena->insert(string.data(), string.size());
|
||||
attribute.null_values.emplace<StringRef>(string_in_arena, string.size());
|
||||
attribute.maps.emplace<ContainerType<StringRef>>();
|
||||
}
|
||||
|
||||
ComplexKeyHashedDictionary::Attribute
|
||||
ComplexKeyHashedDictionary::createAttribute(const DictionaryAttribute & attribute, const Field & null_value)
|
||||
{
|
||||
auto nullable_set = attribute.is_nullable ? std::make_unique<NullableSet>() : nullptr;
|
||||
Attribute attr{attribute.underlying_type, attribute.is_nullable, std::move(nullable_set), {}, {}, {}};
|
||||
|
||||
auto type_call = [&](const auto &dictionary_attribute_type)
|
||||
{
|
||||
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
|
||||
using AttributeType = typename Type::AttributeType;
|
||||
createAttributeImpl<AttributeType>(attr, null_value);
|
||||
};
|
||||
|
||||
callOnDictionaryAttributeType(attribute.underlying_type, type_call);
|
||||
|
||||
return attr;
|
||||
}
|
||||
|
||||
|
||||
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
|
||||
void ComplexKeyHashedDictionary::getItemsImpl(
|
||||
const Attribute & attribute,
|
||||
const Columns & key_columns,
|
||||
ValueSetter && set_value,
|
||||
DefaultValueExtractor & default_value_extractor) const
|
||||
{
|
||||
const auto & attr = std::get<ContainerType<AttributeType>>(attribute.maps);
|
||||
|
||||
const auto keys_size = key_columns.size();
|
||||
StringRefs keys(keys_size);
|
||||
Arena temporary_keys_pool;
|
||||
|
||||
const auto rows = key_columns.front()->size();
|
||||
for (const auto i : ext::range(0, rows))
|
||||
{
|
||||
/// copy key data to arena so it is contiguous and return StringRef to it
|
||||
const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool);
|
||||
|
||||
const auto it = attr.find(key);
|
||||
|
||||
if (it)
|
||||
{
|
||||
set_value(i, static_cast<OutputType>(it->getMapped()), false);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (attribute.is_nullable && attribute.nullable_set->find(key) != nullptr)
|
||||
set_value(i, default_value_extractor[i], true);
|
||||
else
|
||||
set_value(i, default_value_extractor[i], false);
|
||||
}
|
||||
|
||||
/// free memory allocated for the key
|
||||
temporary_keys_pool.rollback(key.size);
|
||||
}
|
||||
|
||||
query_count.fetch_add(rows, std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
bool ComplexKeyHashedDictionary::setAttributeValueImpl(Attribute & attribute, const StringRef key, const T value)
|
||||
{
|
||||
auto & map = std::get<ContainerType<T>>(attribute.maps);
|
||||
const auto pair = map.insert({key, value});
|
||||
return pair.second;
|
||||
}
|
||||
|
||||
template <>
|
||||
bool ComplexKeyHashedDictionary::setAttributeValueImpl<String>(Attribute & attribute, const StringRef key, const String value)
|
||||
{
|
||||
const auto * string_in_arena = attribute.string_arena->insert(value.data(), value.size());
|
||||
return setAttributeValueImpl<StringRef>(attribute, key, StringRef{string_in_arena, value.size()});
|
||||
}
|
||||
|
||||
bool ComplexKeyHashedDictionary::setAttributeValue(Attribute & attribute, const StringRef key, const Field & value)
|
||||
{
|
||||
bool result = false;
|
||||
|
||||
auto type_call = [&](const auto &dictionary_attribute_type)
|
||||
{
|
||||
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
|
||||
using AttributeType = typename Type::AttributeType;
|
||||
|
||||
if (attribute.is_nullable)
|
||||
{
|
||||
if (value.isNull())
|
||||
{
|
||||
attribute.nullable_set->insert(key);
|
||||
result = true;
|
||||
return;
|
||||
}
|
||||
else
|
||||
{
|
||||
attribute.nullable_set->erase(key);
|
||||
}
|
||||
}
|
||||
|
||||
result = setAttributeValueImpl<AttributeType>(attribute, key, value.get<AttributeType>());
|
||||
};
|
||||
|
||||
callOnDictionaryAttributeType(attribute.type, type_call);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
const ComplexKeyHashedDictionary::Attribute & ComplexKeyHashedDictionary::getAttribute(const std::string & attribute_name) const
|
||||
{
|
||||
const auto it = attribute_index_by_name.find(attribute_name);
|
||||
if (it == std::end(attribute_index_by_name))
|
||||
throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
|
||||
|
||||
return attributes[it->second];
|
||||
}
|
||||
|
||||
StringRef ComplexKeyHashedDictionary::placeKeysInPool(const size_t row, const Columns & key_columns, StringRefs & keys, Arena & pool)
|
||||
{
|
||||
const auto keys_size = key_columns.size();
|
||||
size_t sum_keys_size{};
|
||||
|
||||
const char * block_start = nullptr;
|
||||
for (size_t j = 0; j < keys_size; ++j)
|
||||
{
|
||||
keys[j] = key_columns[j]->serializeValueIntoArena(row, pool, block_start);
|
||||
sum_keys_size += keys[j].size;
|
||||
}
|
||||
|
||||
const auto * key_start = block_start;
|
||||
for (size_t j = 0; j < keys_size; ++j)
|
||||
{
|
||||
keys[j].data = key_start;
|
||||
key_start += keys[j].size;
|
||||
}
|
||||
|
||||
return {block_start, sum_keys_size};
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void ComplexKeyHashedDictionary::has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
const auto & attr = std::get<ContainerType<T>>(attribute.maps);
|
||||
const auto keys_size = key_columns.size();
|
||||
StringRefs keys(keys_size);
|
||||
Arena temporary_keys_pool;
|
||||
const auto rows = key_columns.front()->size();
|
||||
|
||||
for (const auto i : ext::range(0, rows))
|
||||
{
|
||||
/// copy key data to arena so it is contiguous and return StringRef to it
|
||||
const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool);
|
||||
|
||||
const auto it = attr.find(key);
|
||||
out[i] = static_cast<bool>(it);
|
||||
|
||||
if (attribute.is_nullable && !out[i])
|
||||
out[i] = attribute.nullable_set->find(key) != nullptr;
|
||||
|
||||
/// free memory allocated for the key
|
||||
temporary_keys_pool.rollback(key.size);
|
||||
}
|
||||
|
||||
query_count.fetch_add(rows, std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
std::vector<StringRef> ComplexKeyHashedDictionary::getKeys() const
|
||||
{
|
||||
const Attribute & attribute = attributes.front();
|
||||
|
||||
std::vector<StringRef> result;
|
||||
|
||||
auto type_call = [&](const auto & dictionary_attribute_type)
|
||||
{
|
||||
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
|
||||
using AttributeType = typename Type::AttributeType;
|
||||
|
||||
if constexpr (std::is_same_v<AttributeType, String>)
|
||||
{
|
||||
result = getKeys<StringRef>(attribute);
|
||||
}
|
||||
else
|
||||
{
|
||||
result = getKeys<AttributeType>(attribute);
|
||||
}
|
||||
};
|
||||
|
||||
callOnDictionaryAttributeType(attribute.type, type_call);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
std::vector<StringRef> ComplexKeyHashedDictionary::getKeys(const Attribute & attribute) const
|
||||
{
|
||||
const ContainerType<T> & attr = std::get<ContainerType<T>>(attribute.maps);
|
||||
std::vector<StringRef> keys;
|
||||
keys.reserve(attr.size());
|
||||
for (const auto & key : attr)
|
||||
keys.push_back(key.getKey());
|
||||
|
||||
if (attribute.is_nullable)
|
||||
{
|
||||
for (const auto & key: *attribute.nullable_set)
|
||||
keys.push_back(key.getKey());
|
||||
}
|
||||
|
||||
return keys;
|
||||
}
|
||||
|
||||
BlockInputStreamPtr ComplexKeyHashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const
|
||||
{
|
||||
using BlockInputStreamType = DictionaryBlockInputStream<UInt64>;
|
||||
auto vector_keys = getKeys();
|
||||
|
||||
PaddedPODArray<StringRef> keys;
|
||||
keys.reserve(vector_keys.size());
|
||||
keys.assign(vector_keys.begin(), vector_keys.end());
|
||||
|
||||
return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, keys, column_names);
|
||||
}
|
||||
|
||||
void registerDictionaryComplexKeyHashed(DictionaryFactory & factory)
|
||||
{
|
||||
auto create_layout = [=](const std::string &,
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr) -> DictionaryPtr
|
||||
{
|
||||
if (!dict_struct.key)
|
||||
throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS};
|
||||
|
||||
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<ComplexKeyHashedDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
|
||||
};
|
||||
factory.registerLayout("complex_key_hashed", create_layout, true);
|
||||
}
|
||||
|
||||
}
|
@ -1,185 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
#include <variant>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
#include <Core/Block.h>
|
||||
#include <common/StringRef.h>
|
||||
#include <ext/range.h>
|
||||
#include "IDictionary.h"
|
||||
#include "IDictionarySource.h"
|
||||
#include "DictionaryStructure.h"
|
||||
#include "DictionaryHelpers.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ComplexKeyHashedDictionary final : public IDictionaryBase
|
||||
{
|
||||
public:
|
||||
ComplexKeyHashedDictionary(
|
||||
const StorageID & dict_id_,
|
||||
const DictionaryStructure & dict_struct_,
|
||||
DictionarySourcePtr source_ptr_,
|
||||
const DictionaryLifetime dict_lifetime_,
|
||||
bool require_nonempty_,
|
||||
BlockPtr saved_block_ = nullptr);
|
||||
|
||||
std::string getKeyDescription() const { return key_description; }
|
||||
|
||||
std::string getTypeName() const override { return "ComplexKeyHashed"; }
|
||||
|
||||
size_t getBytesAllocated() const override { return bytes_allocated; }
|
||||
|
||||
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
|
||||
|
||||
double getHitRate() const override { return 1.0; }
|
||||
|
||||
size_t getElementCount() const override { return element_count; }
|
||||
|
||||
double getLoadFactor() const override { return static_cast<double>(element_count) / bucket_count; }
|
||||
|
||||
std::shared_ptr<const IExternalLoadable> clone() const override
|
||||
{
|
||||
return std::make_shared<ComplexKeyHashedDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
|
||||
}
|
||||
|
||||
const IDictionarySource * getSource() const override { return source_ptr.get(); }
|
||||
|
||||
const DictionaryLifetime & getLifetime() const override { return dict_lifetime; }
|
||||
|
||||
const DictionaryStructure & getStructure() const override { return dict_struct; }
|
||||
|
||||
bool isInjective(const std::string & attribute_name) const override
|
||||
{
|
||||
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
|
||||
}
|
||||
|
||||
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; }
|
||||
|
||||
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;
|
||||
|
||||
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
|
||||
|
||||
private:
|
||||
template <typename Value>
|
||||
using ContainerType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
|
||||
|
||||
using NullableSet = HashSetWithSavedHash<StringRef, StringRefHash>;
|
||||
|
||||
struct Attribute final
|
||||
{
|
||||
AttributeUnderlyingType type;
|
||||
bool is_nullable;
|
||||
std::unique_ptr<NullableSet> nullable_set;
|
||||
|
||||
std::variant<
|
||||
UInt8,
|
||||
UInt16,
|
||||
UInt32,
|
||||
UInt64,
|
||||
UInt128,
|
||||
Int8,
|
||||
Int16,
|
||||
Int32,
|
||||
Int64,
|
||||
Decimal32,
|
||||
Decimal64,
|
||||
Decimal128,
|
||||
Float32,
|
||||
Float64,
|
||||
StringRef>
|
||||
null_values;
|
||||
std::variant<
|
||||
ContainerType<UInt8>,
|
||||
ContainerType<UInt16>,
|
||||
ContainerType<UInt32>,
|
||||
ContainerType<UInt64>,
|
||||
ContainerType<UInt128>,
|
||||
ContainerType<Int8>,
|
||||
ContainerType<Int16>,
|
||||
ContainerType<Int32>,
|
||||
ContainerType<Int64>,
|
||||
ContainerType<Decimal32>,
|
||||
ContainerType<Decimal64>,
|
||||
ContainerType<Decimal128>,
|
||||
ContainerType<Float32>,
|
||||
ContainerType<Float64>,
|
||||
ContainerType<StringRef>>
|
||||
maps;
|
||||
std::unique_ptr<Arena> string_arena;
|
||||
};
|
||||
|
||||
void createAttributes();
|
||||
|
||||
void blockToAttributes(const Block & block);
|
||||
|
||||
void updateData();
|
||||
|
||||
void loadData();
|
||||
|
||||
template <typename T>
|
||||
void addAttributeSize(const Attribute & attribute);
|
||||
|
||||
void calculateBytesAllocated();
|
||||
|
||||
template <typename T>
|
||||
static void createAttributeImpl(Attribute & attribute, const Field & null_value);
|
||||
|
||||
static Attribute createAttribute(const DictionaryAttribute & attribute, const Field & null_value);
|
||||
|
||||
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
|
||||
void getItemsImpl(
|
||||
const Attribute & attribute,
|
||||
const Columns & key_columns,
|
||||
ValueSetter && set_value,
|
||||
DefaultValueExtractor & default_value_extractor) const;
|
||||
|
||||
template <typename T>
|
||||
static bool setAttributeValueImpl(Attribute & attribute, const StringRef key, const T value);
|
||||
|
||||
static bool setAttributeValue(Attribute & attribute, const StringRef key, const Field & value);
|
||||
|
||||
const Attribute & getAttribute(const std::string & attribute_name) const;
|
||||
|
||||
static StringRef placeKeysInPool(const size_t row, const Columns & key_columns, StringRefs & keys, Arena & pool);
|
||||
|
||||
template <typename T>
|
||||
void has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray<UInt8> & out) const;
|
||||
|
||||
std::vector<StringRef> getKeys() const;
|
||||
|
||||
template <typename T>
|
||||
std::vector<StringRef> getKeys(const Attribute & attribute) const;
|
||||
|
||||
const DictionaryStructure dict_struct;
|
||||
const DictionarySourcePtr source_ptr;
|
||||
const DictionaryLifetime dict_lifetime;
|
||||
const bool require_nonempty;
|
||||
const std::string key_description{dict_struct.getKeyDescription()};
|
||||
|
||||
std::map<std::string, size_t> attribute_index_by_name;
|
||||
std::vector<Attribute> attributes;
|
||||
Arena keys_pool;
|
||||
|
||||
size_t bytes_allocated = 0;
|
||||
size_t element_count = 0;
|
||||
size_t bucket_count = 0;
|
||||
mutable std::atomic<size_t> query_count{0};
|
||||
|
||||
BlockPtr saved_block;
|
||||
};
|
||||
|
||||
}
|
200
src/Dictionaries/DictionaryBlockInputStream.cpp
Normal file
200
src/Dictionaries/DictionaryBlockInputStream.cpp
Normal file
@ -0,0 +1,200 @@
|
||||
#include "DictionaryBlockInputStream.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
DictionaryBlockInputStream::DictionaryBlockInputStream(
|
||||
std::shared_ptr<const IDictionary> dictionary_, UInt64 max_block_size_, PaddedPODArray<UInt64> && ids_, const Names & column_names_)
|
||||
: DictionaryBlockInputStreamBase(ids_.size(), max_block_size_)
|
||||
, dictionary(dictionary_)
|
||||
, column_names(column_names_)
|
||||
, ids(std::move(ids_))
|
||||
, key_type(DictionaryInputStreamKeyType::Id)
|
||||
{
|
||||
}
|
||||
|
||||
DictionaryBlockInputStream::DictionaryBlockInputStream(
|
||||
std::shared_ptr<const IDictionary> dictionary_,
|
||||
UInt64 max_block_size_,
|
||||
const PaddedPODArray<StringRef> & keys,
|
||||
const Names & column_names_)
|
||||
: DictionaryBlockInputStreamBase(keys.size(), max_block_size_)
|
||||
, dictionary(dictionary_)
|
||||
, column_names(column_names_)
|
||||
, key_type(DictionaryInputStreamKeyType::ComplexKey)
|
||||
{
|
||||
const DictionaryStructure & dictionary_structure = dictionary->getStructure();
|
||||
fillKeyColumns(keys, 0, keys.size(), dictionary_structure, key_columns);
|
||||
}
|
||||
|
||||
DictionaryBlockInputStream::DictionaryBlockInputStream(
|
||||
std::shared_ptr<const IDictionary> dictionary_,
|
||||
UInt64 max_block_size_,
|
||||
const Columns & data_columns_,
|
||||
const Names & column_names_,
|
||||
GetColumnsFunction && get_key_columns_function_,
|
||||
GetColumnsFunction && get_view_columns_function_)
|
||||
: DictionaryBlockInputStreamBase(data_columns_.front()->size(), max_block_size_)
|
||||
, dictionary(dictionary_)
|
||||
, column_names(column_names_)
|
||||
, data_columns(data_columns_)
|
||||
, get_key_columns_function(std::move(get_key_columns_function_))
|
||||
, get_view_columns_function(std::move(get_view_columns_function_))
|
||||
, key_type(DictionaryInputStreamKeyType::Callback)
|
||||
{
|
||||
}
|
||||
|
||||
Block DictionaryBlockInputStream::getBlock(size_t start, size_t length) const
|
||||
{
|
||||
/// TODO: Rewrite
|
||||
switch (key_type)
|
||||
{
|
||||
case DictionaryInputStreamKeyType::ComplexKey:
|
||||
{
|
||||
Columns columns;
|
||||
ColumnsWithTypeAndName view_columns;
|
||||
columns.reserve(key_columns.size());
|
||||
for (const auto & key_column : key_columns)
|
||||
{
|
||||
ColumnPtr column = key_column.column->cut(start, length);
|
||||
columns.emplace_back(column);
|
||||
view_columns.emplace_back(column, key_column.type, key_column.name);
|
||||
}
|
||||
return fillBlock({}, columns, {}, std::move(view_columns));
|
||||
}
|
||||
|
||||
case DictionaryInputStreamKeyType::Id:
|
||||
{
|
||||
PaddedPODArray<UInt64> ids_to_fill(ids.begin() + start, ids.begin() + start + length);
|
||||
return fillBlock(ids_to_fill, {}, {}, {});
|
||||
}
|
||||
|
||||
case DictionaryInputStreamKeyType::Callback:
|
||||
{
|
||||
Columns columns;
|
||||
columns.reserve(data_columns.size());
|
||||
for (const auto & data_column : data_columns)
|
||||
columns.push_back(data_column->cut(start, length));
|
||||
const DictionaryStructure & dictionaty_structure = dictionary->getStructure();
|
||||
const auto & attributes = *dictionaty_structure.key;
|
||||
ColumnsWithTypeAndName keys_with_type_and_name = get_key_columns_function(columns, attributes);
|
||||
ColumnsWithTypeAndName view_with_type_and_name = get_view_columns_function(columns, attributes);
|
||||
DataTypes types;
|
||||
columns.clear();
|
||||
for (const auto & key_column : keys_with_type_and_name)
|
||||
{
|
||||
columns.push_back(key_column.column);
|
||||
types.push_back(key_column.type);
|
||||
}
|
||||
return fillBlock({}, columns, types, std::move(view_with_type_and_name));
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected DictionaryInputStreamKeyType.");
|
||||
}
|
||||
|
||||
Block DictionaryBlockInputStream::fillBlock(
|
||||
const PaddedPODArray<UInt64> & ids_to_fill,
|
||||
const Columns & keys,
|
||||
const DataTypes & types,
|
||||
ColumnsWithTypeAndName && view) const
|
||||
{
|
||||
std::unordered_set<std::string> names(column_names.begin(), column_names.end());
|
||||
|
||||
DataTypes data_types = types;
|
||||
ColumnsWithTypeAndName block_columns;
|
||||
|
||||
data_types.reserve(keys.size());
|
||||
const DictionaryStructure & dictionary_structure = dictionary->getStructure();
|
||||
if (data_types.empty() && dictionary_structure.key)
|
||||
for (const auto & key : *dictionary_structure.key)
|
||||
data_types.push_back(key.type);
|
||||
|
||||
for (const auto & column : view)
|
||||
if (names.find(column.name) != names.end())
|
||||
block_columns.push_back(column);
|
||||
|
||||
const DictionaryStructure & structure = dictionary->getStructure();
|
||||
ColumnPtr ids_column = getColumnFromIds(ids_to_fill);
|
||||
|
||||
if (structure.id && names.find(structure.id->name) != names.end())
|
||||
{
|
||||
block_columns.emplace_back(ids_column, std::make_shared<DataTypeUInt64>(), structure.id->name);
|
||||
}
|
||||
|
||||
auto dictionary_key_type = dictionary->getKeyType();
|
||||
|
||||
for (const auto idx : ext::range(0, structure.attributes.size()))
|
||||
{
|
||||
const DictionaryAttribute & attribute = structure.attributes[idx];
|
||||
if (names.find(attribute.name) != names.end())
|
||||
{
|
||||
ColumnPtr column;
|
||||
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
{
|
||||
column = dictionary->getColumn(
|
||||
attribute.name,
|
||||
attribute.type,
|
||||
{ids_column},
|
||||
{std::make_shared<DataTypeUInt64>()},
|
||||
nullptr /* default_values_column */);
|
||||
}
|
||||
else
|
||||
{
|
||||
column = dictionary->getColumn(
|
||||
attribute.name,
|
||||
attribute.type,
|
||||
keys,
|
||||
data_types,
|
||||
nullptr /* default_values_column*/);
|
||||
}
|
||||
|
||||
block_columns.emplace_back(column, attribute.type, attribute.name);
|
||||
}
|
||||
}
|
||||
|
||||
return Block(block_columns);
|
||||
}
|
||||
|
||||
ColumnPtr DictionaryBlockInputStream::getColumnFromIds(const PaddedPODArray<UInt64> & ids_to_fill)
|
||||
{
|
||||
auto column_vector = ColumnVector<UInt64>::create();
|
||||
column_vector->getData().assign(ids_to_fill);
|
||||
return column_vector;
|
||||
}
|
||||
|
||||
void DictionaryBlockInputStream::fillKeyColumns(
|
||||
const PaddedPODArray<StringRef> & keys,
|
||||
size_t start,
|
||||
size_t size,
|
||||
const DictionaryStructure & dictionary_structure,
|
||||
ColumnsWithTypeAndName & result)
|
||||
{
|
||||
MutableColumns columns;
|
||||
columns.reserve(dictionary_structure.key->size());
|
||||
|
||||
for (const DictionaryAttribute & attribute : *dictionary_structure.key)
|
||||
columns.emplace_back(attribute.type->createColumn());
|
||||
|
||||
for (auto idx : ext::range(start, size))
|
||||
{
|
||||
const auto & key = keys[idx];
|
||||
const auto *ptr = key.data;
|
||||
for (auto & column : columns)
|
||||
ptr = column->deserializeAndInsertFromArena(ptr);
|
||||
}
|
||||
|
||||
for (size_t i = 0, num_columns = columns.size(); i < num_columns; ++i)
|
||||
{
|
||||
const auto & dictionary_attribute = (*dictionary_structure.key)[i];
|
||||
result.emplace_back(ColumnWithTypeAndName{std::move(columns[i]), dictionary_attribute.type, dictionary_attribute.name});
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -16,27 +16,22 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
/// TODO: Remove this class
|
||||
/* BlockInputStream implementation for external dictionaries
|
||||
* read() returns blocks consisting of the in-memory contents of the dictionaries
|
||||
*/
|
||||
template <typename Key>
|
||||
class DictionaryBlockInputStream : public DictionaryBlockInputStreamBase
|
||||
{
|
||||
public:
|
||||
DictionaryBlockInputStream(
|
||||
std::shared_ptr<const IDictionaryBase> dictionary,
|
||||
std::shared_ptr<const IDictionary> dictionary,
|
||||
UInt64 max_block_size,
|
||||
PaddedPODArray<Key> && ids,
|
||||
PaddedPODArray<UInt64> && ids,
|
||||
const Names & column_names);
|
||||
|
||||
DictionaryBlockInputStream(
|
||||
std::shared_ptr<const IDictionaryBase> dictionary,
|
||||
std::shared_ptr<const IDictionary> dictionary,
|
||||
UInt64 max_block_size,
|
||||
const PaddedPODArray<StringRef> & keys,
|
||||
const Names & column_names);
|
||||
@ -48,7 +43,7 @@ public:
|
||||
// and get_view_columns_function to get key representation.
|
||||
// Now used in trie dictionary, where columns are stored as ip and mask, and are showed as string
|
||||
DictionaryBlockInputStream(
|
||||
std::shared_ptr<const IDictionaryBase> dictionary,
|
||||
std::shared_ptr<const IDictionary> dictionary,
|
||||
UInt64 max_block_size,
|
||||
const Columns & data_columns,
|
||||
const Names & column_names,
|
||||
@ -61,21 +56,24 @@ protected:
|
||||
Block getBlock(size_t start, size_t length) const override;
|
||||
|
||||
private:
|
||||
Block
|
||||
fillBlock(const PaddedPODArray<Key> & ids_to_fill, const Columns & keys, const DataTypes & types, ColumnsWithTypeAndName && view) const;
|
||||
Block fillBlock(
|
||||
const PaddedPODArray<UInt64> & ids_to_fill,
|
||||
const Columns & keys,
|
||||
const DataTypes & types,
|
||||
ColumnsWithTypeAndName && view) const;
|
||||
|
||||
ColumnPtr getColumnFromIds(const PaddedPODArray<Key> & ids_to_fill) const;
|
||||
static ColumnPtr getColumnFromIds(const PaddedPODArray<UInt64> & ids_to_fill);
|
||||
|
||||
void fillKeyColumns(
|
||||
static void fillKeyColumns(
|
||||
const PaddedPODArray<StringRef> & keys,
|
||||
size_t start,
|
||||
size_t size,
|
||||
const DictionaryStructure & dictionary_structure,
|
||||
ColumnsWithTypeAndName & columns) const;
|
||||
ColumnsWithTypeAndName & result);
|
||||
|
||||
std::shared_ptr<const IDictionaryBase> dictionary;
|
||||
std::shared_ptr<const IDictionary> dictionary;
|
||||
Names column_names;
|
||||
PaddedPODArray<Key> ids;
|
||||
PaddedPODArray<UInt64> ids;
|
||||
ColumnsWithTypeAndName key_columns;
|
||||
|
||||
Columns data_columns;
|
||||
@ -92,200 +90,4 @@ private:
|
||||
DictionaryInputStreamKeyType key_type;
|
||||
};
|
||||
|
||||
|
||||
template <typename Key>
|
||||
DictionaryBlockInputStream<Key>::DictionaryBlockInputStream(
|
||||
std::shared_ptr<const IDictionaryBase> dictionary_, UInt64 max_block_size_, PaddedPODArray<Key> && ids_, const Names & column_names_)
|
||||
: DictionaryBlockInputStreamBase(ids_.size(), max_block_size_)
|
||||
, dictionary(dictionary_)
|
||||
, column_names(column_names_)
|
||||
, ids(std::move(ids_))
|
||||
, key_type(DictionaryInputStreamKeyType::Id)
|
||||
{
|
||||
}
|
||||
|
||||
template <typename Key>
|
||||
DictionaryBlockInputStream<Key>::DictionaryBlockInputStream(
|
||||
std::shared_ptr<const IDictionaryBase> dictionary_,
|
||||
UInt64 max_block_size_,
|
||||
const PaddedPODArray<StringRef> & keys,
|
||||
const Names & column_names_)
|
||||
: DictionaryBlockInputStreamBase(keys.size(), max_block_size_)
|
||||
, dictionary(dictionary_)
|
||||
, column_names(column_names_)
|
||||
, key_type(DictionaryInputStreamKeyType::ComplexKey)
|
||||
{
|
||||
const DictionaryStructure & dictionary_structure = dictionary->getStructure();
|
||||
fillKeyColumns(keys, 0, keys.size(), dictionary_structure, key_columns);
|
||||
}
|
||||
|
||||
template <typename Key>
|
||||
DictionaryBlockInputStream<Key>::DictionaryBlockInputStream(
|
||||
std::shared_ptr<const IDictionaryBase> dictionary_,
|
||||
UInt64 max_block_size_,
|
||||
const Columns & data_columns_,
|
||||
const Names & column_names_,
|
||||
GetColumnsFunction && get_key_columns_function_,
|
||||
GetColumnsFunction && get_view_columns_function_)
|
||||
: DictionaryBlockInputStreamBase(data_columns_.front()->size(), max_block_size_)
|
||||
, dictionary(dictionary_)
|
||||
, column_names(column_names_)
|
||||
, data_columns(data_columns_)
|
||||
, get_key_columns_function(std::move(get_key_columns_function_))
|
||||
, get_view_columns_function(std::move(get_view_columns_function_))
|
||||
, key_type(DictionaryInputStreamKeyType::Callback)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
template <typename Key>
|
||||
Block DictionaryBlockInputStream<Key>::getBlock(size_t start, size_t length) const
|
||||
{
|
||||
/// TODO: Rewrite
|
||||
switch (key_type)
|
||||
{
|
||||
case DictionaryInputStreamKeyType::ComplexKey:
|
||||
{
|
||||
Columns columns;
|
||||
ColumnsWithTypeAndName view_columns;
|
||||
columns.reserve(key_columns.size());
|
||||
for (const auto & key_column : key_columns)
|
||||
{
|
||||
ColumnPtr column = key_column.column->cut(start, length);
|
||||
columns.emplace_back(column);
|
||||
view_columns.emplace_back(column, key_column.type, key_column.name);
|
||||
}
|
||||
return fillBlock({}, columns, {}, std::move(view_columns));
|
||||
}
|
||||
|
||||
case DictionaryInputStreamKeyType::Id:
|
||||
{
|
||||
PaddedPODArray<Key> ids_to_fill(ids.begin() + start, ids.begin() + start + length);
|
||||
return fillBlock(ids_to_fill, {}, {}, {});
|
||||
}
|
||||
|
||||
case DictionaryInputStreamKeyType::Callback:
|
||||
{
|
||||
Columns columns;
|
||||
columns.reserve(data_columns.size());
|
||||
for (const auto & data_column : data_columns)
|
||||
columns.push_back(data_column->cut(start, length));
|
||||
const DictionaryStructure & dictionaty_structure = dictionary->getStructure();
|
||||
const auto & attributes = *dictionaty_structure.key;
|
||||
ColumnsWithTypeAndName keys_with_type_and_name = get_key_columns_function(columns, attributes);
|
||||
ColumnsWithTypeAndName view_with_type_and_name = get_view_columns_function(columns, attributes);
|
||||
DataTypes types;
|
||||
columns.clear();
|
||||
for (const auto & key_column : keys_with_type_and_name)
|
||||
{
|
||||
columns.push_back(key_column.column);
|
||||
types.push_back(key_column.type);
|
||||
}
|
||||
return fillBlock({}, columns, types, std::move(view_with_type_and_name));
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception("Unexpected DictionaryInputStreamKeyType.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
template <typename Key>
|
||||
Block DictionaryBlockInputStream<Key>::fillBlock(
|
||||
const PaddedPODArray<Key> & ids_to_fill, const Columns & keys, const DataTypes & types, ColumnsWithTypeAndName && view) const
|
||||
{
|
||||
std::unordered_set<std::string> names(column_names.begin(), column_names.end());
|
||||
|
||||
DataTypes data_types = types;
|
||||
ColumnsWithTypeAndName block_columns;
|
||||
|
||||
data_types.reserve(keys.size());
|
||||
const DictionaryStructure & dictionaty_structure = dictionary->getStructure();
|
||||
if (data_types.empty() && dictionaty_structure.key)
|
||||
for (const auto & key : *dictionaty_structure.key)
|
||||
data_types.push_back(key.type);
|
||||
|
||||
for (const auto & column : view)
|
||||
if (names.find(column.name) != names.end())
|
||||
block_columns.push_back(column);
|
||||
|
||||
const DictionaryStructure & structure = dictionary->getStructure();
|
||||
ColumnPtr ids_column = getColumnFromIds(ids_to_fill);
|
||||
|
||||
if (structure.id && names.find(structure.id->name) != names.end())
|
||||
{
|
||||
block_columns.emplace_back(ids_column, std::make_shared<DataTypeUInt64>(), structure.id->name);
|
||||
}
|
||||
|
||||
auto dictionary_key_type = dictionary->getKeyType();
|
||||
|
||||
for (const auto idx : ext::range(0, structure.attributes.size()))
|
||||
{
|
||||
const DictionaryAttribute & attribute = structure.attributes[idx];
|
||||
if (names.find(attribute.name) != names.end())
|
||||
{
|
||||
ColumnPtr column;
|
||||
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
{
|
||||
column = dictionary->getColumn(
|
||||
attribute.name,
|
||||
attribute.type,
|
||||
{ids_column},
|
||||
{std::make_shared<DataTypeUInt64>()},
|
||||
nullptr /* default_values_column */);
|
||||
}
|
||||
else
|
||||
{
|
||||
column = dictionary->getColumn(
|
||||
attribute.name,
|
||||
attribute.type,
|
||||
keys,
|
||||
data_types,
|
||||
nullptr /* default_values_column*/);
|
||||
}
|
||||
|
||||
block_columns.emplace_back(column, attribute.type, attribute.name);
|
||||
}
|
||||
}
|
||||
|
||||
return Block(block_columns);
|
||||
}
|
||||
|
||||
template <typename Key>
|
||||
ColumnPtr DictionaryBlockInputStream<Key>::getColumnFromIds(const PaddedPODArray<Key> & ids_to_fill) const
|
||||
{
|
||||
auto column_vector = ColumnVector<UInt64>::create();
|
||||
column_vector->getData().reserve(ids_to_fill.size());
|
||||
for (UInt64 id : ids_to_fill)
|
||||
column_vector->insertValue(id);
|
||||
return column_vector;
|
||||
}
|
||||
|
||||
|
||||
template <typename Key>
|
||||
void DictionaryBlockInputStream<Key>::fillKeyColumns(
|
||||
const PaddedPODArray<StringRef> & keys,
|
||||
size_t start,
|
||||
size_t size,
|
||||
const DictionaryStructure & dictionary_structure,
|
||||
ColumnsWithTypeAndName & res) const
|
||||
{
|
||||
MutableColumns columns;
|
||||
columns.reserve(dictionary_structure.key->size());
|
||||
|
||||
for (const DictionaryAttribute & attribute : *dictionary_structure.key)
|
||||
columns.emplace_back(attribute.type->createColumn());
|
||||
|
||||
for (auto idx : ext::range(start, size))
|
||||
{
|
||||
const auto & key = keys[idx];
|
||||
const auto *ptr = key.data;
|
||||
for (auto & column : columns)
|
||||
ptr = column->deserializeAndInsertFromArena(ptr);
|
||||
}
|
||||
|
||||
for (size_t i = 0, num_columns = columns.size(); i < num_columns; ++i)
|
||||
res.emplace_back(
|
||||
ColumnWithTypeAndName{std::move(columns[i]), (*dictionary_structure.key)[i].type, (*dictionary_structure.key)[i].name});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -295,6 +295,28 @@ private:
|
||||
bool use_default_value_from_column = false;
|
||||
};
|
||||
|
||||
template <DictionaryKeyType key_type>
|
||||
class DictionaryKeysArenaHolder;
|
||||
|
||||
template <>
|
||||
class DictionaryKeysArenaHolder<DictionaryKeyType::simple>
|
||||
{
|
||||
public:
|
||||
static Arena * getComplexKeyArena() { return nullptr; }
|
||||
};
|
||||
|
||||
template <>
|
||||
class DictionaryKeysArenaHolder<DictionaryKeyType::complex>
|
||||
{
|
||||
public:
|
||||
|
||||
Arena * getComplexKeyArena() { return &complex_key_arena; }
|
||||
|
||||
private:
|
||||
Arena complex_key_arena;
|
||||
};
|
||||
|
||||
|
||||
template <DictionaryKeyType key_type>
|
||||
class DictionaryKeysExtractor
|
||||
{
|
||||
@ -302,67 +324,96 @@ public:
|
||||
using KeyType = std::conditional_t<key_type == DictionaryKeyType::simple, UInt64, StringRef>;
|
||||
static_assert(key_type != DictionaryKeyType::range, "Range key type is not supported by DictionaryKeysExtractor");
|
||||
|
||||
explicit DictionaryKeysExtractor(const Columns & key_columns, Arena & existing_arena)
|
||||
explicit DictionaryKeysExtractor(const Columns & key_columns_, Arena * complex_key_arena_)
|
||||
: key_columns(key_columns_)
|
||||
, complex_key_arena(complex_key_arena_)
|
||||
{
|
||||
assert(!key_columns.empty());
|
||||
|
||||
if constexpr (key_type == DictionaryKeyType::simple)
|
||||
keys = getColumnVectorData(key_columns.front());
|
||||
{
|
||||
key_columns[0] = key_columns[0]->convertToFullColumnIfConst();
|
||||
|
||||
const auto * vector_col = checkAndGetColumn<ColumnVector<UInt64>>(key_columns[0].get());
|
||||
if (!vector_col)
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Column type mismatch for simple key expected UInt64");
|
||||
}
|
||||
|
||||
keys_size = key_columns.front()->size();
|
||||
}
|
||||
|
||||
inline size_t getKeysSize() const
|
||||
{
|
||||
return keys_size;
|
||||
}
|
||||
|
||||
inline size_t getCurrentKeyIndex() const
|
||||
{
|
||||
return current_key_index;
|
||||
}
|
||||
|
||||
inline KeyType extractCurrentKey()
|
||||
{
|
||||
assert(current_key_index < keys_size);
|
||||
|
||||
if constexpr (key_type == DictionaryKeyType::simple)
|
||||
{
|
||||
const auto & column_vector = static_cast<const ColumnVector<UInt64> &>(*key_columns[0]);
|
||||
const auto & data = column_vector.getData();
|
||||
|
||||
auto key = data[current_key_index];
|
||||
++current_key_index;
|
||||
return key;
|
||||
}
|
||||
else
|
||||
keys = deserializeKeyColumnsInArena(key_columns, existing_arena);
|
||||
}
|
||||
|
||||
|
||||
const PaddedPODArray<KeyType> & getKeys() const
|
||||
{
|
||||
return keys;
|
||||
}
|
||||
|
||||
private:
|
||||
static PaddedPODArray<UInt64> getColumnVectorData(const ColumnPtr column)
|
||||
{
|
||||
PaddedPODArray<UInt64> result;
|
||||
|
||||
auto full_column = column->convertToFullColumnIfConst();
|
||||
const auto *vector_col = checkAndGetColumn<ColumnVector<UInt64>>(full_column.get());
|
||||
|
||||
if (!vector_col)
|
||||
throw Exception{ErrorCodes::TYPE_MISMATCH, "Column type mismatch for simple key expected UInt64"};
|
||||
|
||||
result.assign(vector_col->getData());
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
static PaddedPODArray<StringRef> deserializeKeyColumnsInArena(const Columns & key_columns, Arena & temporary_arena)
|
||||
{
|
||||
size_t keys_size = key_columns.front()->size();
|
||||
|
||||
PaddedPODArray<StringRef> result;
|
||||
result.reserve(keys_size);
|
||||
|
||||
PaddedPODArray<StringRef> temporary_column_data(key_columns.size());
|
||||
|
||||
for (size_t key_index = 0; key_index < keys_size; ++key_index)
|
||||
{
|
||||
size_t allocated_size_for_columns = 0;
|
||||
const char * block_start = nullptr;
|
||||
|
||||
for (size_t column_index = 0; column_index < key_columns.size(); ++column_index)
|
||||
for (const auto & column : key_columns)
|
||||
{
|
||||
const auto & column = key_columns[column_index];
|
||||
temporary_column_data[column_index] = column->serializeValueIntoArena(key_index, temporary_arena, block_start);
|
||||
allocated_size_for_columns += temporary_column_data[column_index].size;
|
||||
StringRef serialized_data = column->serializeValueIntoArena(current_key_index, *complex_key_arena, block_start);
|
||||
allocated_size_for_columns += serialized_data.size;
|
||||
}
|
||||
|
||||
result.push_back(StringRef{block_start, allocated_size_for_columns});
|
||||
++current_key_index;
|
||||
current_complex_key = StringRef{block_start, allocated_size_for_columns};
|
||||
return current_complex_key;
|
||||
}
|
||||
}
|
||||
|
||||
void rollbackCurrentKey() const
|
||||
{
|
||||
if constexpr (key_type == DictionaryKeyType::complex)
|
||||
complex_key_arena->rollback(current_complex_key.size);
|
||||
}
|
||||
|
||||
PaddedPODArray<KeyType> extractAllKeys()
|
||||
{
|
||||
PaddedPODArray<KeyType> result;
|
||||
result.reserve(keys_size - current_key_index);
|
||||
|
||||
for (; current_key_index < keys_size;)
|
||||
{
|
||||
auto value = extractCurrentKey();
|
||||
result.emplace_back(value);
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
PaddedPODArray<KeyType> keys;
|
||||
void reset()
|
||||
{
|
||||
current_key_index = 0;
|
||||
}
|
||||
private:
|
||||
Columns key_columns;
|
||||
|
||||
size_t keys_size = 0;
|
||||
size_t current_key_index = 0;
|
||||
|
||||
KeyType current_complex_key {};
|
||||
Arena * complex_key_arena;
|
||||
};
|
||||
|
||||
/**
|
||||
@ -370,9 +421,10 @@ private:
|
||||
|
||||
* If column is constant parameter backup_storage is used to store values.
|
||||
*/
|
||||
/// TODO: Remove
|
||||
template <typename T>
|
||||
static const PaddedPODArray<T> & getColumnVectorData(
|
||||
const IDictionaryBase * dictionary,
|
||||
const IDictionary * dictionary,
|
||||
const ColumnPtr column,
|
||||
PaddedPODArray<T> & backup_storage)
|
||||
{
|
||||
|
@ -200,8 +200,21 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
|
||||
|
||||
for (size_t i = 0; i < attributes.size(); ++i)
|
||||
{
|
||||
const auto & attribute_name = attributes[i].name;
|
||||
const auto & attribute = attributes[i];
|
||||
const auto & attribute_name = attribute.name;
|
||||
attribute_name_to_index[attribute_name] = i;
|
||||
|
||||
if (attribute.hierarchical)
|
||||
{
|
||||
if (id && attribute.underlying_type != AttributeUnderlyingType::utUInt64)
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"Hierarchical attribute type for dictionary with simple key must be UInt64. Actual ({})",
|
||||
toString(attribute.underlying_type));
|
||||
else if (key)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary with complex key does not support hierarchy");
|
||||
|
||||
hierarchical_attribute_index = i;
|
||||
}
|
||||
}
|
||||
|
||||
if (attributes.empty())
|
||||
|
@ -153,6 +153,8 @@ struct DictionaryStructure final
|
||||
std::unordered_map<std::string, size_t> attribute_name_to_index;
|
||||
std::optional<DictionaryTypedSpecialAttribute> range_min;
|
||||
std::optional<DictionaryTypedSpecialAttribute> range_max;
|
||||
std::optional<size_t> hierarchical_attribute_index;
|
||||
|
||||
bool has_expressions = false;
|
||||
bool access_to_key_from_attributes = false;
|
||||
|
||||
|
@ -1,158 +1,33 @@
|
||||
#include "DirectDictionary.h"
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include "DictionaryBlockInputStream.h"
|
||||
#include "DictionaryFactory.h"
|
||||
#include <Core/Defines.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Interpreters/AggregationCommon.h>
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
#include <Dictionaries/DictionaryFactory.h>
|
||||
#include <Dictionaries/HierarchyDictionariesUtils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
inline UInt64 getAt(const PaddedPODArray<UInt64> & arr, const size_t idx)
|
||||
{
|
||||
return arr[idx];
|
||||
}
|
||||
|
||||
inline UInt64 getAt(const UInt64 & value, const size_t)
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
DirectDictionary<dictionary_key_type>::DirectDictionary(
|
||||
const StorageID & dict_id_,
|
||||
const DictionaryStructure & dict_struct_,
|
||||
DictionarySourcePtr source_ptr_,
|
||||
BlockPtr saved_block_)
|
||||
DictionarySourcePtr source_ptr_)
|
||||
: IDictionary(dict_id_)
|
||||
, dict_struct(dict_struct_)
|
||||
, source_ptr{std::move(source_ptr_)}
|
||||
, saved_block{std::move(saved_block_)}
|
||||
{
|
||||
if (!source_ptr->supportsSelectiveLoad())
|
||||
throw Exception{full_name + ": source cannot be used with DirectDictionary", ErrorCodes::UNSUPPORTED_METHOD};
|
||||
|
||||
setup();
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
void DirectDictionary<dictionary_key_type>::toParent(const PaddedPODArray<Key> & ids [[maybe_unused]], PaddedPODArray<Key> & out [[maybe_unused]]) const
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
{
|
||||
const auto & attribute_name = hierarchical_attribute->name;
|
||||
|
||||
auto result_type = std::make_shared<DataTypeUInt64>();
|
||||
auto input_column = result_type->createColumn();
|
||||
auto & input_column_typed = assert_cast<ColumnVector<UInt64> &>(*input_column);
|
||||
auto & data = input_column_typed.getData();
|
||||
data.insert(ids.begin(), ids.end());
|
||||
|
||||
auto column = getColumn({attribute_name}, result_type, {std::move(input_column)}, {result_type}, {nullptr});
|
||||
const auto & result_column_typed = assert_cast<const ColumnVector<UInt64> &>(*column);
|
||||
const auto & result_data = result_column_typed.getData();
|
||||
|
||||
out.assign(result_data);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Hierarchy is not supported for complex key DirectDictionary");
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
UInt64 DirectDictionary<dictionary_key_type>::getValueOrNullByKey(const Key & to_find) const
|
||||
{
|
||||
std::vector<Key> required_key = {to_find};
|
||||
|
||||
auto stream = source_ptr->loadIds(required_key);
|
||||
stream->readPrefix();
|
||||
|
||||
bool is_found = false;
|
||||
UInt64 result = hierarchical_attribute->null_value.template get<UInt64>();
|
||||
|
||||
while (const auto block = stream->read())
|
||||
{
|
||||
const IColumn & id_column = *block.safeGetByPosition(0).column;
|
||||
|
||||
for (const size_t attribute_idx : ext::range(0, dict_struct.attributes.size()))
|
||||
{
|
||||
if (is_found)
|
||||
break;
|
||||
|
||||
const IColumn & attribute_column = *block.safeGetByPosition(attribute_idx + 1).column;
|
||||
|
||||
for (const auto row_idx : ext::range(0, id_column.size()))
|
||||
{
|
||||
const auto key = id_column[row_idx].get<UInt64>();
|
||||
|
||||
if (key == to_find && hierarchical_attribute->name == attribute_name_by_index.at(attribute_idx))
|
||||
{
|
||||
result = attribute_column[row_idx].get<Key>();
|
||||
is_found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
stream->readSuffix();
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
template <typename ChildType, typename AncestorType>
|
||||
void DirectDictionary<dictionary_key_type>::isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
const auto null_value = hierarchical_attribute->null_value.template get<UInt64>();
|
||||
const auto rows = out.size();
|
||||
|
||||
for (const auto row : ext::range(0, rows))
|
||||
{
|
||||
auto id = getAt(child_ids, row);
|
||||
const auto ancestor_id = getAt(ancestor_ids, row);
|
||||
|
||||
for (size_t i = 0; id != null_value && id != ancestor_id && i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i)
|
||||
id = getValueOrNullByKey(id);
|
||||
|
||||
out[row] = id != null_value && id == ancestor_id;
|
||||
}
|
||||
|
||||
query_count.fetch_add(rows, std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
void DirectDictionary<dictionary_key_type>::isInVectorVector(
|
||||
const PaddedPODArray<UInt64> & child_ids, const PaddedPODArray<UInt64> & ancestor_ids, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
isInImpl(child_ids, ancestor_ids, out);
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
void DirectDictionary<dictionary_key_type>::isInVectorConstant(const PaddedPODArray<UInt64> & child_ids, const UInt64 ancestor_id, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
isInImpl(child_ids, ancestor_id, out);
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
void DirectDictionary<dictionary_key_type>::isInConstantVector(const UInt64 child_id, const PaddedPODArray<UInt64> & ancestor_ids, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
isInImpl(child_id, ancestor_ids, out);
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
@ -166,20 +41,20 @@ ColumnPtr DirectDictionary<dictionary_key_type>::getColumn(
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
|
||||
dict_struct.validateKeyTypes(key_types);
|
||||
|
||||
Arena complex_key_arena;
|
||||
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
|
||||
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns, arena_holder.getComplexKeyArena());
|
||||
const auto requested_keys = extractor.extractAllKeys();
|
||||
|
||||
const DictionaryAttribute & attribute = dict_struct.getAttribute(attribute_name, result_type);
|
||||
DefaultValueProvider default_value_provider(attribute.null_value, default_values_column);
|
||||
|
||||
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns, complex_key_arena);
|
||||
const auto & requested_keys = extractor.getKeys();
|
||||
|
||||
HashMap<KeyType, size_t> key_to_fetched_index;
|
||||
key_to_fetched_index.reserve(requested_keys.size());
|
||||
|
||||
auto fetched_from_storage = attribute.type->createColumn();
|
||||
|
||||
size_t fetched_key_index = 0;
|
||||
size_t requested_attribute_index = attribute_index_by_name.find(attribute_name)->second;
|
||||
size_t requested_attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second;
|
||||
|
||||
Columns block_key_columns;
|
||||
size_t dictionary_keys_size = dict_struct.getKeysNames().size();
|
||||
@ -191,26 +66,19 @@ ColumnPtr DirectDictionary<dictionary_key_type>::getColumn(
|
||||
|
||||
while (const auto block = stream->read())
|
||||
{
|
||||
auto block_columns = block.getColumns();
|
||||
|
||||
/// Split into keys columns and attribute columns
|
||||
for (size_t i = 0; i < dictionary_keys_size; ++i)
|
||||
{
|
||||
block_key_columns.emplace_back(*block_columns.begin());
|
||||
block_columns.erase(block_columns.begin());
|
||||
}
|
||||
block_key_columns.emplace_back(block.safeGetByPosition(i).column);
|
||||
|
||||
DictionaryKeysExtractor<dictionary_key_type> block_keys_extractor(block_key_columns, complex_key_arena);
|
||||
const auto & block_keys = block_keys_extractor.getKeys();
|
||||
size_t block_keys_size = block_keys.size();
|
||||
DictionaryKeysExtractor<dictionary_key_type> block_keys_extractor(block_key_columns, arena_holder.getComplexKeyArena());
|
||||
auto block_keys = block_keys_extractor.extractAllKeys();
|
||||
|
||||
const auto & block_column = block.safeGetByPosition(dictionary_keys_size + requested_attribute_index).column;
|
||||
fetched_from_storage->insertRangeFrom(*block_column, 0, block_keys_size);
|
||||
fetched_from_storage->insertRangeFrom(*block_column, 0, block_keys.size());
|
||||
|
||||
for (size_t block_key_index = 0; block_key_index < block_keys_size; ++block_key_index)
|
||||
for (size_t block_key_index = 0; block_key_index < block_keys.size(); ++block_key_index)
|
||||
{
|
||||
const auto & block_key = block_keys[block_key_index];
|
||||
|
||||
auto block_key = block_keys[block_key_index];
|
||||
key_to_fetched_index[block_key] = fetched_key_index;
|
||||
++fetched_key_index;
|
||||
}
|
||||
@ -223,10 +91,10 @@ ColumnPtr DirectDictionary<dictionary_key_type>::getColumn(
|
||||
Field value_to_insert;
|
||||
|
||||
size_t requested_keys_size = requested_keys.size();
|
||||
|
||||
auto result = fetched_from_storage->cloneEmpty();
|
||||
result->reserve(requested_keys_size);
|
||||
|
||||
|
||||
for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index)
|
||||
{
|
||||
const auto requested_key = requested_keys[requested_key_index];
|
||||
@ -251,10 +119,9 @@ ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::hasKeys(const Columns &
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
|
||||
dict_struct.validateKeyTypes(key_types);
|
||||
|
||||
Arena complex_key_arena;
|
||||
|
||||
DictionaryKeysExtractor<dictionary_key_type> requested_keys_extractor(key_columns, complex_key_arena);
|
||||
const auto & requested_keys = requested_keys_extractor.getKeys();
|
||||
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
|
||||
DictionaryKeysExtractor<dictionary_key_type> requested_keys_extractor(key_columns, arena_holder.getComplexKeyArena());
|
||||
auto requested_keys = requested_keys_extractor.extractAllKeys();
|
||||
size_t requested_keys_size = requested_keys.size();
|
||||
|
||||
HashMap<KeyType, size_t> requested_key_to_index;
|
||||
@ -279,25 +146,24 @@ ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::hasKeys(const Columns &
|
||||
|
||||
while (const auto block = stream->read())
|
||||
{
|
||||
auto block_columns = block.getColumns();
|
||||
|
||||
/// Split into keys columns and attribute columns
|
||||
for (size_t i = 0; i < dictionary_keys_size; ++i)
|
||||
{
|
||||
block_key_columns.emplace_back(*block_columns.begin());
|
||||
block_columns.erase(block_columns.begin());
|
||||
}
|
||||
block_key_columns.emplace_back(block.safeGetByPosition(i).column);
|
||||
|
||||
DictionaryKeysExtractor<dictionary_key_type> block_keys_extractor(block_key_columns, complex_key_arena);
|
||||
const auto & block_keys = block_keys_extractor.getKeys();
|
||||
DictionaryKeysExtractor<dictionary_key_type> block_keys_extractor(block_key_columns, arena_holder.getComplexKeyArena());
|
||||
size_t block_keys_size = block_keys_extractor.getKeysSize();
|
||||
|
||||
for (const auto & block_key : block_keys)
|
||||
for (size_t i = 0; i < block_keys_size; ++i)
|
||||
{
|
||||
auto block_key = block_keys_extractor.extractCurrentKey();
|
||||
|
||||
const auto * it = requested_key_to_index.find(block_key);
|
||||
assert(it);
|
||||
|
||||
size_t result_data_found_index = it->getMapped();
|
||||
result_data[result_data_found_index] = true;
|
||||
|
||||
block_keys_extractor.rollbackCurrentKey();
|
||||
}
|
||||
|
||||
block_key_columns.clear();
|
||||
@ -310,6 +176,37 @@ ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::hasKeys(const Columns &
|
||||
return result;
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
ColumnPtr DirectDictionary<dictionary_key_type>::getHierarchy(
|
||||
ColumnPtr key_column,
|
||||
const DataTypePtr & key_type) const
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
{
|
||||
auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type);
|
||||
query_count.fetch_add(key_column->size(), std::memory_order_relaxed);
|
||||
return result;
|
||||
}
|
||||
else
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::isInHierarchy(
|
||||
ColumnPtr key_column,
|
||||
ColumnPtr in_key_column,
|
||||
const DataTypePtr & key_type) const
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
{
|
||||
auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type);
|
||||
query_count.fetch_add(key_column->size(), std::memory_order_relaxed);
|
||||
return result;
|
||||
}
|
||||
else
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
BlockInputStreamPtr DirectDictionary<dictionary_key_type>::getSourceBlockInputStream(
|
||||
const Columns & key_columns [[maybe_unused]],
|
||||
@ -342,32 +239,6 @@ BlockInputStreamPtr DirectDictionary<dictionary_key_type>::getSourceBlockInputSt
|
||||
return stream;
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
void DirectDictionary<dictionary_key_type>::setup()
|
||||
{
|
||||
/// TODO: Move this to DictionaryStructure
|
||||
size_t dictionary_attributes_size = dict_struct.attributes.size();
|
||||
for (size_t i = 0; i < dictionary_attributes_size; ++i)
|
||||
{
|
||||
const auto & attribute = dict_struct.attributes[i];
|
||||
attribute_index_by_name[attribute.name] = i;
|
||||
attribute_name_by_index[i] = attribute.name;
|
||||
|
||||
if (attribute.hierarchical)
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"({}): hierarchical attributes are not supported for complex key direct dictionary",
|
||||
full_name);
|
||||
|
||||
hierarchical_attribute = &attribute;
|
||||
|
||||
if (attribute.underlying_type != AttributeUnderlyingType::utUInt64)
|
||||
throw Exception{full_name + ": hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
BlockInputStreamPtr DirectDictionary<dictionary_key_type>::getBlockInputStream(const Names & /* column_names */, size_t /* max_block_size */) const
|
||||
{
|
||||
|
@ -18,11 +18,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
class DirectDictionary final : public IDictionary
|
||||
{
|
||||
@ -33,8 +28,7 @@ public:
|
||||
DirectDictionary(
|
||||
const StorageID & dict_id_,
|
||||
const DictionaryStructure & dict_struct_,
|
||||
DictionarySourcePtr source_ptr_,
|
||||
BlockPtr saved_block_ = nullptr);
|
||||
DictionarySourcePtr source_ptr_);
|
||||
|
||||
std::string getTypeName() const override
|
||||
{
|
||||
@ -56,7 +50,7 @@ public:
|
||||
|
||||
std::shared_ptr<const IExternalLoadable> clone() const override
|
||||
{
|
||||
return std::make_shared<DirectDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), saved_block);
|
||||
return std::make_shared<DirectDictionary>(getDictionaryID(), dict_struct, source_ptr->clone());
|
||||
}
|
||||
|
||||
const IDictionarySource * getSource() const override { return source_ptr.get(); }
|
||||
@ -67,26 +61,9 @@ public:
|
||||
|
||||
bool isInjective(const std::string & attribute_name) const override
|
||||
{
|
||||
auto it = attribute_index_by_name.find(attribute_name);
|
||||
|
||||
if (it == attribute_index_by_name.end())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"({}): no attribute with name ({}) in dictionary",
|
||||
full_name,
|
||||
attribute_name);
|
||||
|
||||
return dict_struct.attributes[it->second].injective;
|
||||
return dict_struct.getAttribute(attribute_name).injective;
|
||||
}
|
||||
|
||||
bool hasHierarchy() const override { return hierarchical_attribute; }
|
||||
|
||||
void toParent(const PaddedPODArray<UInt64> & ids, PaddedPODArray<UInt64> & out) const override;
|
||||
|
||||
void isInVectorVector(
|
||||
const PaddedPODArray<UInt64> & child_ids, const PaddedPODArray<UInt64> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
|
||||
void isInVectorConstant(const PaddedPODArray<UInt64> & child_ids, const UInt64 ancestor_id, PaddedPODArray<UInt8> & out) const override;
|
||||
void isInConstantVector(const UInt64 child_id, const PaddedPODArray<UInt64> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
|
||||
|
||||
DictionaryKeyType getKeyType() const override { return dictionary_key_type; }
|
||||
|
||||
ColumnPtr getColumn(
|
||||
@ -98,30 +75,25 @@ public:
|
||||
|
||||
ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override;
|
||||
|
||||
bool hasHierarchy() const override { return dict_struct.hierarchical_attribute_index.has_value(); }
|
||||
|
||||
ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override;
|
||||
|
||||
ColumnUInt8::Ptr isInHierarchy(
|
||||
ColumnPtr key_column,
|
||||
ColumnPtr in_key_column,
|
||||
const DataTypePtr & key_type) const override;
|
||||
|
||||
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
|
||||
|
||||
private:
|
||||
void setup();
|
||||
|
||||
BlockInputStreamPtr getSourceBlockInputStream(const Columns & key_columns, const PaddedPODArray<KeyType> & requested_keys) const;
|
||||
|
||||
UInt64 getValueOrNullByKey(const UInt64 & to_find) const;
|
||||
|
||||
template <typename ChildType, typename AncestorType>
|
||||
void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
|
||||
|
||||
const DictionaryStructure dict_struct;
|
||||
const DictionarySourcePtr source_ptr;
|
||||
const DictionaryLifetime dict_lifetime;
|
||||
|
||||
std::unordered_map<std::string, size_t> attribute_index_by_name;
|
||||
std::unordered_map<size_t, std::string> attribute_name_by_index;
|
||||
|
||||
const DictionaryAttribute * hierarchical_attribute = nullptr;
|
||||
|
||||
mutable std::atomic<size_t> query_count{0};
|
||||
|
||||
BlockPtr saved_block;
|
||||
};
|
||||
|
||||
extern template class DirectDictionary<DictionaryKeyType::simple>;
|
||||
|
@ -1,20 +1,22 @@
|
||||
#include "FlatDictionary.h"
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
#include "DictionaryBlockInputStream.h"
|
||||
#include "DictionaryFactory.h"
|
||||
#include <Dictionaries/DictionaryBlockInputStream.h>
|
||||
#include <Dictionaries/DictionaryFactory.h>
|
||||
#include <Dictionaries/HierarchyDictionariesUtils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int DICTIONARY_IS_EMPTY;
|
||||
@ -24,7 +26,6 @@ namespace ErrorCodes
|
||||
static const auto initial_array_size = 1024;
|
||||
static const auto max_array_size = 500000;
|
||||
|
||||
|
||||
FlatDictionary::FlatDictionary(
|
||||
const StorageID & dict_id_,
|
||||
const DictionaryStructure & dict_struct_,
|
||||
@ -45,69 +46,6 @@ FlatDictionary::FlatDictionary(
|
||||
calculateBytesAllocated();
|
||||
}
|
||||
|
||||
|
||||
void FlatDictionary::toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<Key> & out) const
|
||||
{
|
||||
const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values);
|
||||
DictionaryDefaultValueExtractor<UInt64> extractor(null_value);
|
||||
|
||||
getItemsImpl<UInt64, UInt64>(
|
||||
*hierarchical_attribute,
|
||||
ids,
|
||||
[&](const size_t row, const UInt64 value) { out[row] = value; },
|
||||
extractor);
|
||||
}
|
||||
|
||||
|
||||
/// Allow to use single value in same way as array.
|
||||
static inline FlatDictionary::Key getAt(const PaddedPODArray<FlatDictionary::Key> & arr, const size_t idx)
|
||||
{
|
||||
return arr[idx];
|
||||
}
|
||||
static inline FlatDictionary::Key getAt(const FlatDictionary::Key & value, const size_t)
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
template <typename ChildType, typename AncestorType>
|
||||
void FlatDictionary::isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values);
|
||||
const auto & attr = std::get<ContainerType<Key>>(hierarchical_attribute->arrays);
|
||||
const auto rows = out.size();
|
||||
|
||||
size_t loaded_size = attr.size();
|
||||
for (const auto row : ext::range(0, rows))
|
||||
{
|
||||
auto id = getAt(child_ids, row);
|
||||
const auto ancestor_id = getAt(ancestor_ids, row);
|
||||
|
||||
for (size_t i = 0; id < loaded_size && id != null_value && id != ancestor_id && i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i)
|
||||
id = attr[id];
|
||||
|
||||
out[row] = id != null_value && id == ancestor_id;
|
||||
}
|
||||
|
||||
query_count.fetch_add(rows, std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
|
||||
void FlatDictionary::isInVectorVector(
|
||||
const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
isInImpl(child_ids, ancestor_ids, out);
|
||||
}
|
||||
|
||||
void FlatDictionary::isInVectorConstant(const PaddedPODArray<Key> & child_ids, const Key ancestor_id, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
isInImpl(child_ids, ancestor_id, out);
|
||||
}
|
||||
|
||||
void FlatDictionary::isInConstantVector(const Key child_id, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const
|
||||
{
|
||||
isInImpl(child_id, ancestor_ids, out);
|
||||
}
|
||||
|
||||
ColumnPtr FlatDictionary::getColumn(
|
||||
const std::string & attribute_name,
|
||||
const DataTypePtr & result_type,
|
||||
@ -117,14 +55,16 @@ ColumnPtr FlatDictionary::getColumn(
|
||||
{
|
||||
ColumnPtr result;
|
||||
|
||||
PaddedPODArray<Key> backup_storage;
|
||||
PaddedPODArray<UInt64> backup_storage;
|
||||
const auto & ids = getColumnVectorData(this, key_columns.front(), backup_storage);
|
||||
|
||||
auto size = ids.size();
|
||||
|
||||
const auto & attribute = getAttribute(attribute_name);
|
||||
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
|
||||
|
||||
size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second;
|
||||
const auto & attribute = attributes[attribute_index];
|
||||
|
||||
auto type_call = [&](const auto & dictionary_attribute_type)
|
||||
{
|
||||
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
|
||||
@ -183,10 +123,9 @@ ColumnPtr FlatDictionary::getColumn(
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
ColumnUInt8::Ptr FlatDictionary::hasKeys(const Columns & key_columns, const DataTypes &) const
|
||||
{
|
||||
PaddedPODArray<Key> backup_storage;
|
||||
PaddedPODArray<UInt64> backup_storage;
|
||||
const auto& ids = getColumnVectorData(this, key_columns.front(), backup_storage);
|
||||
|
||||
auto result = ColumnUInt8::create(ext::size(ids));
|
||||
@ -205,24 +144,118 @@ ColumnUInt8::Ptr FlatDictionary::hasKeys(const Columns & key_columns, const Data
|
||||
return result;
|
||||
}
|
||||
|
||||
ColumnPtr FlatDictionary::getHierarchy(ColumnPtr key_column, const DataTypePtr &) const
|
||||
{
|
||||
PaddedPODArray<UInt64> keys_backup_storage;
|
||||
const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage);
|
||||
|
||||
size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index;
|
||||
const auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
|
||||
|
||||
const UInt64 null_value = std::get<UInt64>(hierarchical_attribute.null_values);
|
||||
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.arrays);
|
||||
|
||||
auto is_key_valid_func = [&, this](auto & key)
|
||||
{
|
||||
return key < loaded_ids.size() && loaded_ids[key];
|
||||
};
|
||||
|
||||
auto get_parent_key_func = [&, this](auto & hierarchy_key)
|
||||
{
|
||||
std::optional<UInt64> result;
|
||||
|
||||
if (hierarchy_key >= loaded_ids.size() || !loaded_ids[hierarchy_key])
|
||||
return result;
|
||||
|
||||
result = parent_keys[hierarchy_key];
|
||||
|
||||
return result;
|
||||
};
|
||||
|
||||
auto dictionary_hierarchy_array = getKeysHierarchyArray(keys, null_value, is_key_valid_func, get_parent_key_func);
|
||||
|
||||
query_count.fetch_add(keys.size(), std::memory_order_relaxed);
|
||||
|
||||
return dictionary_hierarchy_array;
|
||||
}
|
||||
|
||||
ColumnUInt8::Ptr FlatDictionary::isInHierarchy(
|
||||
ColumnPtr key_column,
|
||||
ColumnPtr in_key_column,
|
||||
const DataTypePtr &) const
|
||||
{
|
||||
PaddedPODArray<UInt64> keys_backup_storage;
|
||||
const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage);
|
||||
|
||||
PaddedPODArray<UInt64> keys_in_backup_storage;
|
||||
const auto & keys_in = getColumnVectorData(this, in_key_column, keys_in_backup_storage);
|
||||
|
||||
size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index;
|
||||
const auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
|
||||
|
||||
const UInt64 null_value = std::get<UInt64>(hierarchical_attribute.null_values);
|
||||
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.arrays);
|
||||
|
||||
auto is_key_valid_func = [&, this](auto & key)
|
||||
{
|
||||
return key < loaded_ids.size() && loaded_ids[key];
|
||||
};
|
||||
|
||||
auto get_parent_key_func = [&, this](auto & hierarchy_key)
|
||||
{
|
||||
std::optional<UInt64> result;
|
||||
|
||||
if (hierarchy_key >= loaded_ids.size() || !loaded_ids[hierarchy_key])
|
||||
return result;
|
||||
|
||||
result = parent_keys[hierarchy_key];
|
||||
|
||||
return result;
|
||||
};
|
||||
|
||||
auto result = getKeysIsInHierarchyColumn(keys, keys_in, null_value, is_key_valid_func, get_parent_key_func);
|
||||
|
||||
query_count.fetch_add(keys.size(), std::memory_order_relaxed);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
ColumnPtr FlatDictionary::getDescendants(
|
||||
ColumnPtr key_column,
|
||||
const DataTypePtr &,
|
||||
size_t level) const
|
||||
{
|
||||
PaddedPODArray<UInt64> keys_backup;
|
||||
const auto & keys = getColumnVectorData(this, key_column, keys_backup);
|
||||
|
||||
size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index;
|
||||
const auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
|
||||
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.arrays);
|
||||
|
||||
HashMap<UInt64, PaddedPODArray<UInt64>> parent_to_child;
|
||||
|
||||
for (size_t i = 0; i < parent_keys.size(); ++i)
|
||||
{
|
||||
auto parent_key = parent_keys[i];
|
||||
|
||||
if (loaded_ids[i])
|
||||
parent_to_child[parent_key].emplace_back(static_cast<UInt64>(i));
|
||||
}
|
||||
|
||||
auto result = getKeysDescendantsArray(keys, parent_to_child, level);
|
||||
|
||||
query_count.fetch_add(keys.size(), std::memory_order_relaxed);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
void FlatDictionary::createAttributes()
|
||||
{
|
||||
const auto size = dict_struct.attributes.size();
|
||||
attributes.reserve(size);
|
||||
|
||||
for (const auto & attribute : dict_struct.attributes)
|
||||
{
|
||||
attribute_index_by_name.emplace(attribute.name, attributes.size());
|
||||
attributes.push_back(createAttribute(attribute, attribute.null_value));
|
||||
|
||||
if (attribute.hierarchical)
|
||||
{
|
||||
hierarchical_attribute = &attributes.back();
|
||||
|
||||
if (hierarchical_attribute->type != AttributeUnderlyingType::utUInt64)
|
||||
throw Exception{full_name + ": hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void FlatDictionary::blockToAttributes(const Block & block)
|
||||
@ -271,7 +304,7 @@ void FlatDictionary::updateData()
|
||||
const auto & saved_id_column = *saved_block->safeGetByPosition(0).column;
|
||||
const auto & update_id_column = *block.safeGetByPosition(0).column;
|
||||
|
||||
std::unordered_map<Key, std::vector<size_t>> update_ids;
|
||||
std::unordered_map<UInt64, std::vector<size_t>> update_ids;
|
||||
for (size_t row = 0; row < update_id_column.size(); ++row)
|
||||
{
|
||||
const auto id = update_id_column.get64(row);
|
||||
@ -280,7 +313,7 @@ void FlatDictionary::updateData()
|
||||
|
||||
const size_t saved_rows = saved_id_column.size();
|
||||
IColumn::Filter filter(saved_rows);
|
||||
std::unordered_map<Key, std::vector<size_t>>::iterator it;
|
||||
std::unordered_map<UInt64, std::vector<size_t>>::iterator it;
|
||||
|
||||
for (size_t row = 0; row < saved_id_column.size(); ++row)
|
||||
{
|
||||
@ -385,7 +418,6 @@ void FlatDictionary::createAttributeImpl<String>(Attribute & attribute, const Fi
|
||||
attribute.arrays.emplace<ContainerType<StringRef>>(initial_array_size, StringRef(string_in_arena, string.size()));
|
||||
}
|
||||
|
||||
|
||||
FlatDictionary::Attribute FlatDictionary::createAttribute(const DictionaryAttribute& attribute, const Field & null_value)
|
||||
{
|
||||
auto nullable_set = attribute.is_nullable ? std::make_optional<NullableSet>() : std::optional<NullableSet>{};
|
||||
@ -408,7 +440,7 @@ FlatDictionary::Attribute FlatDictionary::createAttribute(const DictionaryAttrib
|
||||
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
|
||||
void FlatDictionary::getItemsImpl(
|
||||
const Attribute & attribute,
|
||||
const PaddedPODArray<Key> & ids,
|
||||
const PaddedPODArray<UInt64> & ids,
|
||||
ValueSetter && set_value,
|
||||
DefaultValueExtractor & default_value_extractor) const
|
||||
{
|
||||
@ -425,7 +457,7 @@ void FlatDictionary::getItemsImpl(
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void FlatDictionary::resize(Attribute & attribute, const Key id)
|
||||
void FlatDictionary::resize(Attribute & attribute, const UInt64 id)
|
||||
{
|
||||
if (id >= max_array_size)
|
||||
throw Exception{full_name + ": identifier should be less than " + toString(max_array_size), ErrorCodes::ARGUMENT_OUT_OF_BOUND};
|
||||
@ -440,7 +472,7 @@ void FlatDictionary::resize(Attribute & attribute, const Key id)
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const Key id, const T & value)
|
||||
void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const UInt64 id, const T & value)
|
||||
{
|
||||
auto & array = std::get<ContainerType<T>>(attribute.arrays);
|
||||
array[id] = value;
|
||||
@ -448,13 +480,13 @@ void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const Key id,
|
||||
}
|
||||
|
||||
template <>
|
||||
void FlatDictionary::setAttributeValueImpl<String>(Attribute & attribute, const Key id, const String & value)
|
||||
void FlatDictionary::setAttributeValueImpl<String>(Attribute & attribute, const UInt64 id, const String & value)
|
||||
{
|
||||
const auto * string_in_arena = attribute.string_arena->insert(value.data(), value.size());
|
||||
setAttributeValueImpl(attribute, id, StringRef{string_in_arena, value.size()});
|
||||
}
|
||||
|
||||
void FlatDictionary::setAttributeValue(Attribute & attribute, const Key id, const Field & value)
|
||||
void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 id, const Field & value)
|
||||
{
|
||||
auto type_call = [&](const auto &dictionary_attribute_type)
|
||||
{
|
||||
@ -484,21 +516,11 @@ void FlatDictionary::setAttributeValue(Attribute & attribute, const Key id, cons
|
||||
callOnDictionaryAttributeType(attribute.type, type_call);
|
||||
}
|
||||
|
||||
|
||||
const FlatDictionary::Attribute & FlatDictionary::getAttribute(const std::string & attribute_name) const
|
||||
{
|
||||
const auto it = attribute_index_by_name.find(attribute_name);
|
||||
if (it == std::end(attribute_index_by_name))
|
||||
throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
|
||||
|
||||
return attributes[it->second];
|
||||
}
|
||||
|
||||
PaddedPODArray<FlatDictionary::Key> FlatDictionary::getIds() const
|
||||
PaddedPODArray<UInt64> FlatDictionary::getIds() const
|
||||
{
|
||||
const auto ids_count = ext::size(loaded_ids);
|
||||
|
||||
PaddedPODArray<Key> ids;
|
||||
PaddedPODArray<UInt64> ids;
|
||||
ids.reserve(ids_count);
|
||||
|
||||
for (auto idx : ext::range(0, ids_count))
|
||||
@ -509,8 +531,7 @@ PaddedPODArray<FlatDictionary::Key> FlatDictionary::getIds() const
|
||||
|
||||
BlockInputStreamPtr FlatDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const
|
||||
{
|
||||
using BlockInputStreamType = DictionaryBlockInputStream<Key>;
|
||||
return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, getIds(), column_names);
|
||||
return std::make_shared<DictionaryBlockInputStream>(shared_from_this(), max_block_size, getIds(), column_names);
|
||||
}
|
||||
|
||||
void registerDictionaryFlat(DictionaryFactory & factory)
|
||||
|
@ -59,18 +59,9 @@ public:
|
||||
|
||||
bool isInjective(const std::string & attribute_name) const override
|
||||
{
|
||||
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
|
||||
return dict_struct.getAttribute(attribute_name).injective;
|
||||
}
|
||||
|
||||
bool hasHierarchy() const override { return hierarchical_attribute; }
|
||||
|
||||
void toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<Key> & out) const override;
|
||||
|
||||
void isInVectorVector(
|
||||
const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
|
||||
void isInVectorConstant(const PaddedPODArray<Key> & child_ids, const Key ancestor_id, PaddedPODArray<UInt8> & out) const override;
|
||||
void isInConstantVector(const Key child_id, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
|
||||
|
||||
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::simple; }
|
||||
|
||||
ColumnPtr getColumn(
|
||||
@ -82,13 +73,27 @@ public:
|
||||
|
||||
ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override;
|
||||
|
||||
bool hasHierarchy() const override { return dict_struct.hierarchical_attribute_index.has_value(); }
|
||||
|
||||
ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override;
|
||||
|
||||
ColumnUInt8::Ptr isInHierarchy(
|
||||
ColumnPtr key_column,
|
||||
ColumnPtr in_key_column,
|
||||
const DataTypePtr & key_type) const override;
|
||||
|
||||
ColumnPtr getDescendants(
|
||||
ColumnPtr key_column,
|
||||
const DataTypePtr & key_type,
|
||||
size_t level) const override;
|
||||
|
||||
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
|
||||
|
||||
private:
|
||||
template <typename Value>
|
||||
using ContainerType = PaddedPODArray<Value>;
|
||||
|
||||
using NullableSet = HashSet<Key, DefaultHash<Key>>;
|
||||
using NullableSet = HashSet<UInt64, DefaultHash<UInt64>>;
|
||||
|
||||
struct Attribute final
|
||||
{
|
||||
@ -151,24 +156,24 @@ private:
|
||||
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
|
||||
void getItemsImpl(
|
||||
const Attribute & attribute,
|
||||
const PaddedPODArray<Key> & ids,
|
||||
const PaddedPODArray<UInt64> & ids,
|
||||
ValueSetter && set_value,
|
||||
DefaultValueExtractor & default_value_extractor) const;
|
||||
|
||||
template <typename T>
|
||||
void resize(Attribute & attribute, const Key id);
|
||||
void resize(Attribute & attribute, const UInt64 id);
|
||||
|
||||
template <typename T>
|
||||
void setAttributeValueImpl(Attribute & attribute, const Key id, const T & value);
|
||||
void setAttributeValueImpl(Attribute & attribute, const UInt64 id, const T & value);
|
||||
|
||||
void setAttributeValue(Attribute & attribute, const Key id, const Field & value);
|
||||
void setAttributeValue(Attribute & attribute, const UInt64 id, const Field & value);
|
||||
|
||||
const Attribute & getAttribute(const std::string & attribute_name) const;
|
||||
|
||||
template <typename ChildType, typename AncestorType>
|
||||
void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
|
||||
|
||||
PaddedPODArray<Key> getIds() const;
|
||||
PaddedPODArray<UInt64> getIds() const;
|
||||
|
||||
const DictionaryStructure dict_struct;
|
||||
const DictionarySourcePtr source_ptr;
|
||||
@ -177,7 +182,6 @@ private:
|
||||
|
||||
std::map<std::string, size_t> attribute_index_by_name;
|
||||
std::vector<Attribute> attributes;
|
||||
const Attribute * hierarchical_attribute = nullptr;
|
||||
std::vector<bool> loaded_ids;
|
||||
|
||||
size_t bytes_allocated = 0;
|
||||
@ -185,6 +189,7 @@ private:
|
||||
size_t bucket_count = 0;
|
||||
mutable std::atomic<size_t> query_count{0};
|
||||
|
||||
/// TODO: Remove
|
||||
BlockPtr saved_block;
|
||||
};
|
||||
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -4,17 +4,21 @@
|
||||
#include <memory>
|
||||
#include <variant>
|
||||
#include <optional>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
|
||||
#include <sparsehash/sparse_hash_map>
|
||||
#include <ext/range.h>
|
||||
#include "DictionaryStructure.h"
|
||||
#include "IDictionary.h"
|
||||
#include "IDictionarySource.h"
|
||||
#include "DictionaryHelpers.h"
|
||||
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
#include <Core/Block.h>
|
||||
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <Dictionaries/IDictionary.h>
|
||||
#include <Dictionaries/IDictionarySource.h>
|
||||
#include <Dictionaries/DictionaryHelpers.h>
|
||||
|
||||
/** This dictionary stores all content in a hash table in memory
|
||||
* (a separate Key -> Value map for each attribute)
|
||||
@ -24,19 +28,32 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type, bool sparse>
|
||||
class HashedDictionary final : public IDictionary
|
||||
{
|
||||
public:
|
||||
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
|
||||
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by hashed dictionary");
|
||||
|
||||
HashedDictionary(
|
||||
const StorageID & dict_id_,
|
||||
const DictionaryStructure & dict_struct_,
|
||||
DictionarySourcePtr source_ptr_,
|
||||
const DictionaryLifetime dict_lifetime_,
|
||||
bool require_nonempty_,
|
||||
bool sparse_,
|
||||
BlockPtr saved_block_ = nullptr);
|
||||
|
||||
std::string getTypeName() const override { return sparse ? "SparseHashed" : "Hashed"; }
|
||||
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 "ComplexKeySpareseHashed";
|
||||
else
|
||||
return "ComplexKeyHashed";
|
||||
}
|
||||
|
||||
size_t getBytesAllocated() const override { return bytes_allocated; }
|
||||
|
||||
@ -50,7 +67,7 @@ public:
|
||||
|
||||
std::shared_ptr<const IExternalLoadable> clone() const override
|
||||
{
|
||||
return std::make_shared<HashedDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, sparse, saved_block);
|
||||
return std::make_shared<HashedDictionary<dictionary_key_type, sparse>>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
|
||||
}
|
||||
|
||||
const IDictionarySource * getSource() const override { return source_ptr.get(); }
|
||||
@ -61,14 +78,10 @@ public:
|
||||
|
||||
bool isInjective(const std::string & attribute_name) const override
|
||||
{
|
||||
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
|
||||
return dict_struct.getAttribute(attribute_name).injective;
|
||||
}
|
||||
|
||||
bool hasHierarchy() const override { return hierarchical_attribute; }
|
||||
|
||||
void toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<Key> & out) const override;
|
||||
|
||||
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::simple; }
|
||||
DictionaryKeyType getKeyType() const override { return dictionary_key_type; }
|
||||
|
||||
ColumnPtr getColumn(
|
||||
const std::string& attribute_name,
|
||||
@ -79,36 +92,52 @@ public:
|
||||
|
||||
ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override;
|
||||
|
||||
void isInVectorVector(
|
||||
const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
|
||||
void isInVectorConstant(const PaddedPODArray<Key> & child_ids, const Key ancestor_id, PaddedPODArray<UInt8> & out) const override;
|
||||
void isInConstantVector(const Key child_id, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) 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;
|
||||
|
||||
ColumnPtr getDescendants(
|
||||
ColumnPtr key_column,
|
||||
const DataTypePtr & key_type,
|
||||
size_t level) const override;
|
||||
|
||||
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
|
||||
|
||||
private:
|
||||
template <typename Value>
|
||||
using CollectionType = HashMap<UInt64, Value>;
|
||||
template <typename Value>
|
||||
using CollectionPtrType = std::unique_ptr<CollectionType<Value>>;
|
||||
using CollectionTypeNonSparse = std::conditional_t<
|
||||
dictionary_key_type == DictionaryKeyType::simple,
|
||||
HashMap<UInt64, Value>,
|
||||
HashMapWithSavedHash<StringRef, Value, DefaultHash<StringRef>>>;
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
template <typename Value>
|
||||
using SparseCollectionType = google::sparse_hash_map<UInt64, Value, DefaultHash<UInt64>>;
|
||||
template <typename Key, typename Value>
|
||||
using SparseHashMap = google::sparse_hash_map<Key, Value, DefaultHash<Key>>;
|
||||
#else
|
||||
template <typename Value>
|
||||
using SparseCollectionType = google::sparsehash::sparse_hash_map<UInt64, Value, DefaultHash<UInt64>>;
|
||||
template <typename Key, typename Value>
|
||||
using SparseHashMap = google::sparsehash::sparse_hash_map<Key, Value, DefaultHash<Key>>;
|
||||
#endif
|
||||
|
||||
template <typename Value>
|
||||
using SparseCollectionPtrType = std::unique_ptr<SparseCollectionType<Value>>;
|
||||
using CollectionTypeSparse = std::conditional_t<
|
||||
dictionary_key_type == DictionaryKeyType::simple,
|
||||
SparseHashMap<UInt64, Value>,
|
||||
SparseHashMap<StringRef, Value>>;
|
||||
|
||||
using NullableSet = HashSet<Key, DefaultHash<Key>>;
|
||||
template <typename Value>
|
||||
using CollectionType = std::conditional_t<sparse, CollectionTypeSparse<Value>, CollectionTypeNonSparse<Value>>;
|
||||
|
||||
using NullableSet = HashSet<KeyType, DefaultHash<KeyType>>;
|
||||
|
||||
struct Attribute final
|
||||
{
|
||||
AttributeUnderlyingType type;
|
||||
std::optional<NullableSet> nullable_set;
|
||||
std::optional<NullableSet> is_nullable_set;
|
||||
|
||||
std::variant<
|
||||
UInt8,
|
||||
@ -127,41 +156,27 @@ private:
|
||||
Float64,
|
||||
StringRef>
|
||||
null_values;
|
||||
|
||||
std::variant<
|
||||
CollectionPtrType<UInt8>,
|
||||
CollectionPtrType<UInt16>,
|
||||
CollectionPtrType<UInt32>,
|
||||
CollectionPtrType<UInt64>,
|
||||
CollectionPtrType<UInt128>,
|
||||
CollectionPtrType<Int8>,
|
||||
CollectionPtrType<Int16>,
|
||||
CollectionPtrType<Int32>,
|
||||
CollectionPtrType<Int64>,
|
||||
CollectionPtrType<Decimal32>,
|
||||
CollectionPtrType<Decimal64>,
|
||||
CollectionPtrType<Decimal128>,
|
||||
CollectionPtrType<Float32>,
|
||||
CollectionPtrType<Float64>,
|
||||
CollectionPtrType<StringRef>>
|
||||
maps;
|
||||
std::variant<
|
||||
SparseCollectionPtrType<UInt8>,
|
||||
SparseCollectionPtrType<UInt16>,
|
||||
SparseCollectionPtrType<UInt32>,
|
||||
SparseCollectionPtrType<UInt64>,
|
||||
SparseCollectionPtrType<UInt128>,
|
||||
SparseCollectionPtrType<Int8>,
|
||||
SparseCollectionPtrType<Int16>,
|
||||
SparseCollectionPtrType<Int32>,
|
||||
SparseCollectionPtrType<Int64>,
|
||||
SparseCollectionPtrType<Decimal32>,
|
||||
SparseCollectionPtrType<Decimal64>,
|
||||
SparseCollectionPtrType<Decimal128>,
|
||||
SparseCollectionPtrType<Float32>,
|
||||
SparseCollectionPtrType<Float64>,
|
||||
SparseCollectionPtrType<StringRef>>
|
||||
sparse_maps;
|
||||
CollectionType<UInt8>,
|
||||
CollectionType<UInt16>,
|
||||
CollectionType<UInt32>,
|
||||
CollectionType<UInt64>,
|
||||
CollectionType<UInt128>,
|
||||
CollectionType<Int8>,
|
||||
CollectionType<Int16>,
|
||||
CollectionType<Int32>,
|
||||
CollectionType<Int64>,
|
||||
CollectionType<Decimal32>,
|
||||
CollectionType<Decimal64>,
|
||||
CollectionType<Decimal128>,
|
||||
CollectionType<Float32>,
|
||||
CollectionType<Float64>,
|
||||
CollectionType<StringRef>>
|
||||
container;
|
||||
|
||||
std::unique_ptr<Arena> string_arena;
|
||||
|
||||
};
|
||||
|
||||
void createAttributes();
|
||||
@ -172,76 +187,47 @@ private:
|
||||
|
||||
void loadData();
|
||||
|
||||
template <typename T>
|
||||
void addAttributeSize(const Attribute & attribute);
|
||||
|
||||
void calculateBytesAllocated();
|
||||
|
||||
template <typename T>
|
||||
void createAttributeImpl(Attribute & attribute, const Field & null_value);
|
||||
|
||||
Attribute createAttribute(const DictionaryAttribute& attribute, const Field & null_value);
|
||||
|
||||
template <typename AttributeType, typename OutputType, typename MapType, typename ValueSetter, typename DefaultValueExtractor>
|
||||
void getItemsAttrImpl(
|
||||
const MapType & attr,
|
||||
const PaddedPODArray<Key> & ids,
|
||||
ValueSetter && set_value,
|
||||
DefaultValueExtractor & default_value_extractor) const;
|
||||
|
||||
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
|
||||
template <typename AttributeType, typename ValueSetter, typename NullableValueSetter, typename DefaultValueExtractor>
|
||||
void getItemsImpl(
|
||||
const Attribute & attribute,
|
||||
const PaddedPODArray<Key> & ids,
|
||||
DictionaryKeysExtractor<dictionary_key_type> & keys_extractor,
|
||||
ValueSetter && set_value,
|
||||
NullableValueSetter && set_nullable_value,
|
||||
DefaultValueExtractor & default_value_extractor) const;
|
||||
|
||||
template <typename T>
|
||||
bool setAttributeValueImpl(Attribute & attribute, const Key id, const T value);
|
||||
template <typename GetContainerFunc>
|
||||
void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func);
|
||||
|
||||
bool setAttributeValue(Attribute & attribute, const Key id, const Field & value);
|
||||
template <typename GetContainerFunc>
|
||||
void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func) const;
|
||||
|
||||
const Attribute & getAttribute(const std::string & attribute_name) const;
|
||||
|
||||
template <typename T>
|
||||
void has(const Attribute & attribute, const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8> & out) const;
|
||||
|
||||
template <typename T, typename AttrType>
|
||||
PaddedPODArray<Key> getIdsAttrImpl(const AttrType & attr) const;
|
||||
template <typename T>
|
||||
PaddedPODArray<Key> getIds(const Attribute & attribute) const;
|
||||
|
||||
PaddedPODArray<Key> getIds() const;
|
||||
|
||||
/// Preallocates the hashtable based on query progress
|
||||
/// (Only while loading all data).
|
||||
///
|
||||
/// @see preallocate
|
||||
template <typename T>
|
||||
void resize(Attribute & attribute, size_t added_rows);
|
||||
void resize(size_t added_rows);
|
||||
|
||||
template <typename AttrType, typename ChildType, typename AncestorType>
|
||||
void isInAttrImpl(const AttrType & attr, const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
|
||||
template <typename ChildType, typename AncestorType>
|
||||
void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
|
||||
StringRef copyKeyInArena(StringRef key);
|
||||
|
||||
const DictionaryStructure dict_struct;
|
||||
const DictionarySourcePtr source_ptr;
|
||||
const DictionaryLifetime dict_lifetime;
|
||||
const bool require_nonempty;
|
||||
const bool sparse;
|
||||
|
||||
std::map<std::string, size_t> attribute_index_by_name;
|
||||
std::vector<Attribute> attributes;
|
||||
const Attribute * hierarchical_attribute = nullptr;
|
||||
|
||||
size_t bytes_allocated = 0;
|
||||
size_t element_count = 0;
|
||||
size_t bucket_count = 0;
|
||||
mutable std::atomic<size_t> query_count{0};
|
||||
|
||||
/// TODO: Remove
|
||||
BlockPtr saved_block;
|
||||
Arena complex_key_arena;
|
||||
};
|
||||
|
||||
extern template class HashedDictionary<DictionaryKeyType::simple, false>;
|
||||
extern template class HashedDictionary<DictionaryKeyType::simple, true>;
|
||||
|
||||
extern template class HashedDictionary<DictionaryKeyType::complex, false>;
|
||||
extern template class HashedDictionary<DictionaryKeyType::complex, true>;
|
||||
|
||||
}
|
||||
|
156
src/Dictionaries/HierarchyDictionariesUtils.cpp
Normal file
156
src/Dictionaries/HierarchyDictionariesUtils.cpp
Normal file
@ -0,0 +1,156 @@
|
||||
#include "HierarchyDictionariesUtils.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
/** In case of cache or direct dictionary we does not have structure with child to parent representation.
|
||||
* This function build such structure calling getColumn for initial keys to request and for next keys in hierarchy,
|
||||
* until all keys are requested or result key is null value.
|
||||
* To distinguish null value key and key that is not present in dictionary, we use special default value column
|
||||
* with max UInt64 value, if result column key has such value we assume that current key is not presented in dictionary storage.
|
||||
*/
|
||||
HashMap<UInt64, UInt64> getChildToParentHierarchyMapImpl(
|
||||
const IDictionary * dictionary,
|
||||
const DictionaryAttribute & hierarchical_attribute,
|
||||
const PaddedPODArray<UInt64> & initial_keys_to_request,
|
||||
const DataTypePtr & key_type)
|
||||
{
|
||||
UInt64 null_value = hierarchical_attribute.null_value.get<UInt64>();
|
||||
|
||||
ColumnPtr key_to_request_column = ColumnVector<UInt64>::create();
|
||||
auto * key_to_request_column_typed = static_cast<ColumnVector<UInt64> *>(key_to_request_column->assumeMutable().get());
|
||||
|
||||
UInt64 key_not_in_storage_value = std::numeric_limits<UInt64>::max();
|
||||
ColumnPtr key_not_in_storage_default_value_column = ColumnVector<UInt64>::create(initial_keys_to_request.size(), key_not_in_storage_value);
|
||||
|
||||
PaddedPODArray<UInt64> & keys_to_request = key_to_request_column_typed->getData();
|
||||
keys_to_request.assign(initial_keys_to_request);
|
||||
|
||||
PaddedPODArray<UInt64> next_keys_to_request;
|
||||
HashSet<UInt64> already_requested_keys;
|
||||
|
||||
HashMap<UInt64, UInt64> child_to_parent_key;
|
||||
|
||||
while (!keys_to_request.empty())
|
||||
{
|
||||
child_to_parent_key.reserve(child_to_parent_key.size() + keys_to_request.size());
|
||||
|
||||
auto parent_key_column = dictionary->getColumn(
|
||||
hierarchical_attribute.name,
|
||||
hierarchical_attribute.type,
|
||||
{key_to_request_column},
|
||||
{key_type},
|
||||
key_not_in_storage_default_value_column);
|
||||
|
||||
const auto * parent_key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(*parent_key_column);
|
||||
if (!parent_key_column_typed)
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
|
||||
"Parent key column should be UInt64. Actual ({})",
|
||||
hierarchical_attribute.type->getName());
|
||||
|
||||
const auto & parent_keys = parent_key_column_typed->getData();
|
||||
next_keys_to_request.clear();
|
||||
|
||||
for (size_t i = 0; i < keys_to_request.size(); ++i)
|
||||
{
|
||||
auto key = keys_to_request[i];
|
||||
auto parent_key = parent_keys[i];
|
||||
|
||||
if (parent_key == key_not_in_storage_value)
|
||||
continue;
|
||||
|
||||
child_to_parent_key[key] = parent_key;
|
||||
|
||||
if (parent_key == null_value ||
|
||||
already_requested_keys.find(parent_key) != nullptr)
|
||||
continue;
|
||||
|
||||
already_requested_keys.insert(parent_key);
|
||||
next_keys_to_request.emplace_back(parent_key);
|
||||
}
|
||||
|
||||
keys_to_request.clear();
|
||||
keys_to_request.assign(next_keys_to_request);
|
||||
}
|
||||
|
||||
return child_to_parent_key;
|
||||
}
|
||||
}
|
||||
|
||||
ColumnPtr getKeysHierarchyDefaultImplementation(const IDictionary * dictionary, ColumnPtr key_column, const DataTypePtr & key_type)
|
||||
{
|
||||
key_column = key_column->convertToFullColumnIfConst();
|
||||
const auto * key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(*key_column);
|
||||
if (!key_column_typed)
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64");
|
||||
|
||||
const auto & dictionary_structure = dictionary->getStructure();
|
||||
size_t hierarchical_attribute_index = *dictionary_structure.hierarchical_attribute_index;
|
||||
const auto & hierarchical_attribute = dictionary_structure.attributes[hierarchical_attribute_index];
|
||||
|
||||
const PaddedPODArray<UInt64> & requested_keys = key_column_typed->getData();
|
||||
HashMap<UInt64, UInt64> key_to_parent_key = getChildToParentHierarchyMapImpl(dictionary, hierarchical_attribute, requested_keys, key_type);
|
||||
|
||||
auto is_key_valid_func = [&](auto & key) { return key_to_parent_key.find(key) != nullptr; };
|
||||
|
||||
auto get_parent_key_func = [&](auto & key)
|
||||
{
|
||||
auto it = key_to_parent_key.find(key);
|
||||
std::optional<UInt64> result = (it != nullptr ? std::make_optional(it->getMapped()) : std::nullopt);
|
||||
return result;
|
||||
};
|
||||
|
||||
UInt64 null_value = hierarchical_attribute.null_value.get<UInt64>();
|
||||
|
||||
auto dictionary_hierarchy_array = getKeysHierarchyArray(requested_keys, null_value, is_key_valid_func, get_parent_key_func);
|
||||
return dictionary_hierarchy_array;
|
||||
}
|
||||
|
||||
ColumnUInt8::Ptr getKeysIsInHierarchyDefaultImplementation(
|
||||
const IDictionary * dictionary,
|
||||
ColumnPtr key_column,
|
||||
ColumnPtr in_key_column,
|
||||
const DataTypePtr & key_type)
|
||||
{
|
||||
key_column = key_column->convertToFullColumnIfConst();
|
||||
in_key_column = in_key_column->convertToFullColumnIfConst();
|
||||
|
||||
const auto * key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(*key_column);
|
||||
if (!key_column_typed)
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64");
|
||||
|
||||
const auto * in_key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(*in_key_column);
|
||||
if (!in_key_column_typed)
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64");
|
||||
|
||||
const auto & dictionary_structure = dictionary->getStructure();
|
||||
size_t hierarchical_attribute_index = *dictionary_structure.hierarchical_attribute_index;
|
||||
const auto & hierarchical_attribute = dictionary_structure.attributes[hierarchical_attribute_index];
|
||||
|
||||
const PaddedPODArray<UInt64> & requested_keys = key_column_typed->getData();
|
||||
HashMap<UInt64, UInt64> key_to_parent_key = getChildToParentHierarchyMapImpl(dictionary, hierarchical_attribute, requested_keys, key_type);
|
||||
|
||||
auto is_key_valid_func = [&](auto & key) { return key_to_parent_key.find(key) != nullptr; };
|
||||
|
||||
auto get_parent_key_func = [&](auto & key)
|
||||
{
|
||||
auto it = key_to_parent_key.find(key);
|
||||
std::optional<UInt64> result = (it != nullptr ? std::make_optional(it->getMapped()) : std::nullopt);
|
||||
return result;
|
||||
};
|
||||
|
||||
UInt64 null_value = hierarchical_attribute.null_value.get<UInt64>();
|
||||
const auto & in_keys = in_key_column_typed->getData();
|
||||
|
||||
auto result = getKeysIsInHierarchyColumn(requested_keys, in_keys, null_value, is_key_valid_func, get_parent_key_func);
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
467
src/Dictionaries/HierarchyDictionariesUtils.h
Normal file
467
src/Dictionaries/HierarchyDictionariesUtils.h
Normal file
@ -0,0 +1,467 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/types.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
|
||||
#include <Dictionaries/IDictionary.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace detail
|
||||
{
|
||||
template <typename KeyType>
|
||||
struct ElementsAndOffsets
|
||||
{
|
||||
PaddedPODArray<KeyType> elements;
|
||||
PaddedPODArray<IColumn::Offset> offsets;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct IsKeyValidFuncInterface
|
||||
{
|
||||
bool operator()(T key [[maybe_unused]]) { return false; }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct GetParentKeyFuncInterface
|
||||
{
|
||||
std::optional<T> operator()(T key [[maybe_unused]]) { return {}; }
|
||||
};
|
||||
|
||||
/** Calculate hierarchy for keys iterating the hierarchy from child to parent using get_parent_key_func provided by client.
|
||||
* Hierarchy iteration is stopped if key equals null value, get_parent_key_func returns null optional, or hierarchy depth
|
||||
* greater or equal than DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH.
|
||||
* IsKeyValidFunc used for each input hierarchy key, if it returns false result hierarchy for that key will have size 0.
|
||||
* Hierarchy result is ElementsAndOffsets structure, for each element there is hierarchy array,
|
||||
* with size offset[element_index] - (element_index > 0 ? offset[element_index - 1] : 0).
|
||||
*
|
||||
* Example:
|
||||
* id parent_id
|
||||
* 1 0
|
||||
* 2 1
|
||||
* 3 1
|
||||
* 4 2
|
||||
*
|
||||
* If hierarchy_null_value will be 0. Requested keys [1, 2, 3, 4, 5].
|
||||
* Result: [1], [2, 1], [3, 1], [4, 2, 1], []
|
||||
* Elements: [1, 2, 1, 3, 1, 4, 2, 1]
|
||||
* Offsets: [1, 3, 5, 8, 8]
|
||||
*/
|
||||
template <typename KeyType, typename IsKeyValidFunc, typename GetParentKeyFunc>
|
||||
ElementsAndOffsets<KeyType> getHierarchy(
|
||||
const PaddedPODArray<KeyType> & keys,
|
||||
const KeyType & hierarchy_null_value,
|
||||
IsKeyValidFunc && is_key_valid_func,
|
||||
GetParentKeyFunc && get_parent_key_func)
|
||||
{
|
||||
size_t hierarchy_keys_size = keys.size();
|
||||
|
||||
PaddedPODArray<KeyType> elements;
|
||||
elements.reserve(hierarchy_keys_size);
|
||||
|
||||
PaddedPODArray<IColumn::Offset> offsets;
|
||||
offsets.reserve(hierarchy_keys_size);
|
||||
|
||||
struct OffsetInArray
|
||||
{
|
||||
size_t offset_index;
|
||||
size_t array_element_offset;
|
||||
};
|
||||
|
||||
HashMap<KeyType, OffsetInArray> already_processes_keys_to_offset;
|
||||
already_processes_keys_to_offset.reserve(hierarchy_keys_size);
|
||||
|
||||
for (size_t i = 0; i < hierarchy_keys_size; ++i)
|
||||
{
|
||||
auto hierarchy_key = keys[i];
|
||||
size_t current_hierarchy_depth = 0;
|
||||
|
||||
bool is_key_valid = std::forward<IsKeyValidFunc>(is_key_valid_func)(hierarchy_key);
|
||||
|
||||
if (!is_key_valid)
|
||||
{
|
||||
offsets.emplace_back(elements.size());
|
||||
continue;
|
||||
}
|
||||
|
||||
while (true)
|
||||
{
|
||||
const auto * it = already_processes_keys_to_offset.find(hierarchy_key);
|
||||
|
||||
if (it)
|
||||
{
|
||||
const auto & index = it->getMapped();
|
||||
|
||||
size_t offset = index.offset_index;
|
||||
|
||||
bool is_loop = (offset == offsets.size());
|
||||
|
||||
if (unlikely(is_loop))
|
||||
break;
|
||||
|
||||
size_t array_element_offset = index.array_element_offset;
|
||||
|
||||
size_t previous_offset_size = offset > 0 ? offsets[offset - 1] : 0;
|
||||
size_t start_index = previous_offset_size + array_element_offset;
|
||||
size_t end_index = offsets[offset];
|
||||
|
||||
elements.insertFromItself(elements.begin() + start_index, elements.begin() + end_index);
|
||||
break;
|
||||
}
|
||||
|
||||
if (hierarchy_key == hierarchy_null_value || current_hierarchy_depth >= DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH)
|
||||
break;
|
||||
|
||||
already_processes_keys_to_offset[hierarchy_key] = {offsets.size(), current_hierarchy_depth};
|
||||
elements.emplace_back(hierarchy_key);
|
||||
++current_hierarchy_depth;
|
||||
|
||||
std::optional<KeyType> parent_key = std::forward<GetParentKeyFunc>(get_parent_key_func)(hierarchy_key);
|
||||
|
||||
if (!parent_key.has_value())
|
||||
break;
|
||||
|
||||
hierarchy_key = *parent_key;
|
||||
}
|
||||
|
||||
offsets.emplace_back(elements.size());
|
||||
}
|
||||
|
||||
ElementsAndOffsets<KeyType> result = {std::move(elements), std::move(offsets)};
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
/** Returns array with UInt8 represent if key from in_keys array is in hierarchy of key from keys column.
|
||||
* If value in result array is 1 that means key from in_keys array is in hierarchy of key from
|
||||
* keys array with same index, 0 therwise.
|
||||
* For getting hierarchy implementation uses getKeysHierarchy function.
|
||||
*
|
||||
* Not: keys size must be equal to in_keys_size.
|
||||
*/
|
||||
template <typename KeyType, typename IsKeyValidFunc, typename GetParentKeyFunc>
|
||||
PaddedPODArray<UInt8> getIsInHierarchy(
|
||||
const PaddedPODArray<KeyType> & keys,
|
||||
const PaddedPODArray<KeyType> & in_keys,
|
||||
const KeyType & hierarchy_null_value,
|
||||
IsKeyValidFunc && is_key_valid_func,
|
||||
GetParentKeyFunc && get_parent_func)
|
||||
{
|
||||
assert(keys.size() == in_keys.size());
|
||||
|
||||
PaddedPODArray<UInt8> result;
|
||||
result.resize_fill(keys.size());
|
||||
|
||||
detail::ElementsAndOffsets<KeyType> hierarchy = detail::getHierarchy(
|
||||
keys,
|
||||
hierarchy_null_value,
|
||||
std::forward<IsKeyValidFunc>(is_key_valid_func),
|
||||
std::forward<GetParentKeyFunc>(get_parent_func));
|
||||
|
||||
auto & offsets = hierarchy.offsets;
|
||||
auto & elements = hierarchy.elements;
|
||||
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
size_t i_elements_start = i > 0 ? offsets[i - 1] : 0;
|
||||
size_t i_elements_end = offsets[i];
|
||||
|
||||
auto & key_to_find = in_keys[i];
|
||||
|
||||
const auto * begin = elements.begin() + i_elements_start;
|
||||
const auto * end = elements.begin() + i_elements_end;
|
||||
|
||||
const auto * it = std::find(begin, end, key_to_find);
|
||||
|
||||
bool contains_key = (it != end);
|
||||
result[i] = contains_key;
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
struct GetAllDescendantsStrategy { size_t level = 0; };
|
||||
struct GetDescendantsAtSpecificLevelStrategy { size_t level = 0; };
|
||||
|
||||
/** Get descendants for keys iterating the hierarchy from parent to child using parent_to_child hash map provided by client.
|
||||
* GetAllDescendantsStrategy get all descendants for key
|
||||
* GetDescendantsAtSpecificLevelStrategy get descendants only for specific hierarchy level.
|
||||
* Hierarchy result is ElementsAndOffsets structure, for each element there is descendants array,
|
||||
* with size offset[element_index] - (element_index > 0 ? offset[element_index - 1] : 0).
|
||||
*
|
||||
* Example:
|
||||
* id parent_id
|
||||
* 1 0
|
||||
* 2 1
|
||||
* 3 1
|
||||
* 4 2
|
||||
*
|
||||
* Example. Strategy GetAllDescendantsStrategy.
|
||||
* Requested keys [0, 1, 2, 3, 4].
|
||||
* Result: [1, 2, 3, 4], [2, 2, 4], [4], [], []
|
||||
* Elements: [1, 2, 3, 4, 2, 3, 4, 4]
|
||||
* Offsets: [4, 7, 8, 8, 8]
|
||||
*
|
||||
* Example. Strategy GetDescendantsAtSpecificLevelStrategy with level 1.
|
||||
* Requested keys [0, 1, 2, 3, 4].
|
||||
* Result: [1], [2, 3], [4], [], [];
|
||||
* Offsets: [1, 3, 4, 4, 4];
|
||||
*/
|
||||
template <typename KeyType, typename Strategy>
|
||||
ElementsAndOffsets<KeyType> getDescendants(
|
||||
const PaddedPODArray<KeyType> & keys,
|
||||
const HashMap<KeyType, PaddedPODArray<KeyType>> & parent_to_child,
|
||||
Strategy strategy)
|
||||
{
|
||||
/// If strategy is GetAllDescendantsStrategy we try to cache and later reuse previously calculated descendants.
|
||||
/// If strategy is GetDescendantsAtSpecificLevelStrategy we does not use cache strategy.
|
||||
size_t keys_size = keys.size();
|
||||
|
||||
PaddedPODArray<KeyType> descendants;
|
||||
descendants.reserve(keys_size);
|
||||
|
||||
PaddedPODArray<IColumn::Offset> descendants_offsets;
|
||||
descendants_offsets.reserve(keys_size);
|
||||
|
||||
struct Range
|
||||
{
|
||||
size_t start_index;
|
||||
size_t end_index;
|
||||
};
|
||||
|
||||
static constexpr Int64 key_range_requires_update = -1;
|
||||
HashMap<KeyType, Range> already_processed_keys_to_range [[maybe_unused]];
|
||||
|
||||
if constexpr (std::is_same_v<Strategy, GetAllDescendantsStrategy>)
|
||||
already_processed_keys_to_range.reserve(keys_size);
|
||||
|
||||
struct KeyAndDepth
|
||||
{
|
||||
KeyType key;
|
||||
Int64 depth;
|
||||
};
|
||||
|
||||
HashSet<KeyType> already_processed_keys_during_loop;
|
||||
already_processed_keys_during_loop.reserve(keys_size);
|
||||
|
||||
PaddedPODArray<KeyAndDepth> next_keys_to_process_stack;
|
||||
next_keys_to_process_stack.reserve(keys_size);
|
||||
|
||||
Int64 level = static_cast<Int64>(strategy.level);
|
||||
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
{
|
||||
const KeyType & requested_key = keys[i];
|
||||
|
||||
if (parent_to_child.find(requested_key) == nullptr)
|
||||
{
|
||||
descendants_offsets.emplace_back(descendants.size());
|
||||
continue;
|
||||
}
|
||||
|
||||
next_keys_to_process_stack.emplace_back(KeyAndDepth{requested_key, 0});
|
||||
|
||||
/** To cache range for key without recursive function calls and custom stack we put special
|
||||
* signaling value on stack key_range_requires_update.
|
||||
* When we pop such value from stack that means processing descendants for key is finished
|
||||
* and we can update range with end_index.
|
||||
*/
|
||||
while (!next_keys_to_process_stack.empty())
|
||||
{
|
||||
KeyAndDepth key_to_process = next_keys_to_process_stack.back();
|
||||
|
||||
KeyType key = key_to_process.key;
|
||||
Int64 depth = key_to_process.depth;
|
||||
next_keys_to_process_stack.pop_back();
|
||||
|
||||
if constexpr (std::is_same_v<Strategy, GetAllDescendantsStrategy>)
|
||||
{
|
||||
/// Update end_index for key
|
||||
if (depth == key_range_requires_update)
|
||||
{
|
||||
auto * it = already_processed_keys_to_range.find(key);
|
||||
assert(it);
|
||||
|
||||
auto & range_to_update = it->getMapped();
|
||||
range_to_update.end_index = descendants.size();
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
if (unlikely(already_processed_keys_during_loop.find(key) != nullptr))
|
||||
{
|
||||
next_keys_to_process_stack.clear();
|
||||
break;
|
||||
}
|
||||
|
||||
if constexpr (std::is_same_v<Strategy, GetAllDescendantsStrategy>)
|
||||
{
|
||||
const auto * already_processed_it = already_processed_keys_to_range.find(key);
|
||||
|
||||
if (already_processed_it)
|
||||
{
|
||||
Range range = already_processed_it->getMapped();
|
||||
|
||||
if (unlikely(range.start_index > range.end_index))
|
||||
{
|
||||
/// Broken range because there was loop
|
||||
already_processed_keys_to_range.erase(key);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto insert_start_iterator = descendants.begin() + range.start_index;
|
||||
auto insert_end_iterator = descendants.begin() + range.end_index;
|
||||
descendants.insertFromItself(insert_start_iterator, insert_end_iterator);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
const auto * it = parent_to_child.find(key);
|
||||
|
||||
if (!it || depth >= DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH)
|
||||
continue;
|
||||
|
||||
if constexpr (std::is_same_v<Strategy, GetDescendantsAtSpecificLevelStrategy>)
|
||||
{
|
||||
if (depth > level)
|
||||
continue;
|
||||
}
|
||||
|
||||
if constexpr (std::is_same_v<Strategy, GetAllDescendantsStrategy>)
|
||||
{
|
||||
/// Put special signaling value on stack and update cache with range start
|
||||
size_t range_start_index = descendants.size();
|
||||
already_processed_keys_to_range[key].start_index = range_start_index;
|
||||
next_keys_to_process_stack.emplace_back(KeyAndDepth{key, key_range_requires_update});
|
||||
}
|
||||
|
||||
already_processed_keys_during_loop.insert(key);
|
||||
|
||||
++depth;
|
||||
|
||||
const auto & children = it->getMapped();
|
||||
|
||||
for (auto child_key : children)
|
||||
{
|
||||
/// In case of GetAllDescendantsStrategy we add any descendant to result array
|
||||
/// If strategy is GetDescendantsAtSpecificLevelStrategy we require depth == level
|
||||
if (std::is_same_v<Strategy, GetAllDescendantsStrategy> || depth == level)
|
||||
descendants.emplace_back(child_key);
|
||||
|
||||
next_keys_to_process_stack.emplace_back(KeyAndDepth{child_key, depth});
|
||||
}
|
||||
}
|
||||
|
||||
already_processed_keys_during_loop.clear();
|
||||
|
||||
descendants_offsets.emplace_back(descendants.size());
|
||||
}
|
||||
|
||||
ElementsAndOffsets<KeyType> result = {std::move(descendants), std::move(descendants_offsets)};
|
||||
return result;
|
||||
}
|
||||
|
||||
/// Converts ElementAndOffsets structure into ArrayColumn
|
||||
template<typename KeyType>
|
||||
ColumnPtr convertElementsAndOffsetsIntoArray(ElementsAndOffsets<KeyType> && elements_and_offsets)
|
||||
{
|
||||
auto elements_column = ColumnVector<KeyType>::create();
|
||||
elements_column->getData() = std::move(elements_and_offsets.elements);
|
||||
|
||||
auto offsets_column = ColumnVector<IColumn::Offset>::create();
|
||||
offsets_column->getData() = std::move(elements_and_offsets.offsets);
|
||||
|
||||
auto column_array = ColumnArray::create(std::move(elements_column), std::move(offsets_column));
|
||||
|
||||
return column_array;
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns hierarchy array column for keys
|
||||
template <typename KeyType, typename IsKeyValidFunc, typename GetParentKeyFunc>
|
||||
ColumnPtr getKeysHierarchyArray(
|
||||
const PaddedPODArray<KeyType> & keys,
|
||||
const KeyType & hierarchy_null_value,
|
||||
IsKeyValidFunc && is_key_valid_func,
|
||||
GetParentKeyFunc && get_parent_func)
|
||||
{
|
||||
auto elements_and_offsets = detail::getHierarchy(
|
||||
keys,
|
||||
hierarchy_null_value,
|
||||
std::forward<IsKeyValidFunc>(is_key_valid_func),
|
||||
std::forward<GetParentKeyFunc>(get_parent_func));
|
||||
|
||||
return detail::convertElementsAndOffsetsIntoArray(std::move(elements_and_offsets));
|
||||
}
|
||||
|
||||
/// Returns is in hierarchy column for keys
|
||||
template <typename KeyType, typename IsKeyValidFunc, typename GetParentKeyFunc>
|
||||
ColumnUInt8::Ptr getKeysIsInHierarchyColumn(
|
||||
const PaddedPODArray<KeyType> & hierarchy_keys,
|
||||
const PaddedPODArray<KeyType> & hierarchy_in_keys,
|
||||
const KeyType & hierarchy_null_value,
|
||||
IsKeyValidFunc && is_key_valid_func,
|
||||
GetParentKeyFunc && get_parent_func)
|
||||
{
|
||||
auto is_in_hierarchy_data = detail::getIsInHierarchy(
|
||||
hierarchy_keys,
|
||||
hierarchy_in_keys,
|
||||
hierarchy_null_value,
|
||||
std::forward<IsKeyValidFunc>(is_key_valid_func),
|
||||
std::forward<GetParentKeyFunc>(get_parent_func));
|
||||
|
||||
auto result = ColumnUInt8::create();
|
||||
result->getData() = std::move(is_in_hierarchy_data);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
/// Returns descendants array column for keys
|
||||
template <typename KeyType>
|
||||
ColumnPtr getKeysDescendantsArray(
|
||||
const PaddedPODArray<KeyType> & requested_keys,
|
||||
const HashMap<KeyType, PaddedPODArray<KeyType>> & parent_to_child,
|
||||
size_t level)
|
||||
{
|
||||
if (level == 0)
|
||||
{
|
||||
detail::GetAllDescendantsStrategy strategy { .level = level };
|
||||
auto elements_and_offsets = detail::getDescendants(requested_keys, parent_to_child, strategy);
|
||||
return detail::convertElementsAndOffsetsIntoArray(std::move(elements_and_offsets));
|
||||
}
|
||||
else
|
||||
{
|
||||
detail::GetDescendantsAtSpecificLevelStrategy strategy { .level = level };
|
||||
auto elements_and_offsets = detail::getDescendants(requested_keys, parent_to_child, strategy);
|
||||
return detail::convertElementsAndOffsetsIntoArray(std::move(elements_and_offsets));
|
||||
}
|
||||
}
|
||||
|
||||
/** Default getHierarchy implementation for dictionaries that does not have structure with child to parent representation.
|
||||
* Implementation will build such structure with getColumn calls, and then getHierarchy for such structure.
|
||||
* Returns ColumnArray with hierarchy arrays for keys from key_column.
|
||||
*/
|
||||
ColumnPtr getKeysHierarchyDefaultImplementation(
|
||||
const IDictionary * dictionary,
|
||||
ColumnPtr key_column,
|
||||
const DataTypePtr & key_type);
|
||||
|
||||
/** Default isInHierarchy implementation for dictionaries that does not have structure with child to parent representation.
|
||||
* Implementation will build such structure with getColumn calls, and then getHierarchy for such structure.
|
||||
* Returns UInt8 column if key from in_key_column is in key hierarchy from key_column.
|
||||
*/
|
||||
ColumnUInt8::Ptr getKeysIsInHierarchyDefaultImplementation(
|
||||
const IDictionary * dictionary,
|
||||
ColumnPtr key_column,
|
||||
ColumnPtr in_key_column,
|
||||
const DataTypePtr & key_type);
|
||||
|
||||
}
|
@ -24,8 +24,8 @@ namespace ErrorCodes
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
struct IDictionaryBase;
|
||||
using DictionaryPtr = std::unique_ptr<IDictionaryBase>;
|
||||
struct IDictionary;
|
||||
using DictionaryPtr = std::unique_ptr<IDictionary>;
|
||||
|
||||
/** DictionaryKeyType provides IDictionary client information about
|
||||
* which key type is supported by dictionary.
|
||||
@ -47,13 +47,11 @@ enum class DictionaryKeyType
|
||||
/**
|
||||
* Base class for Dictionaries implementation.
|
||||
*/
|
||||
struct IDictionaryBase : public IExternalLoadable
|
||||
struct IDictionary : public IExternalLoadable
|
||||
{
|
||||
using Key = UInt64;
|
||||
|
||||
IDictionaryBase(const StorageID & dict_id_)
|
||||
: dict_id(dict_id_)
|
||||
, full_name(dict_id.getInternalDictionaryName())
|
||||
explicit IDictionary(const StorageID & dictionary_id_)
|
||||
: dictionary_id(dictionary_id_)
|
||||
, full_name(dictionary_id.getInternalDictionaryName())
|
||||
{
|
||||
}
|
||||
|
||||
@ -61,14 +59,14 @@ struct IDictionaryBase : public IExternalLoadable
|
||||
StorageID getDictionaryID() const
|
||||
{
|
||||
std::lock_guard lock{name_mutex};
|
||||
return dict_id;
|
||||
return dictionary_id;
|
||||
}
|
||||
|
||||
void updateDictionaryName(const StorageID & new_name) const
|
||||
{
|
||||
std::lock_guard lock{name_mutex};
|
||||
assert(new_name.uuid == dict_id.uuid && dict_id.uuid != UUIDHelpers::Nil);
|
||||
dict_id = new_name;
|
||||
assert(new_name.uuid == dictionary_id.uuid && dictionary_id.uuid != UUIDHelpers::Nil);
|
||||
dictionary_id = new_name;
|
||||
}
|
||||
|
||||
const std::string & getLoadableName() const override final { return getFullName(); }
|
||||
@ -80,8 +78,9 @@ struct IDictionaryBase : public IExternalLoadable
|
||||
|
||||
std::string getDatabaseOrNoDatabaseTag() const
|
||||
{
|
||||
if (!dict_id.database_name.empty())
|
||||
return dict_id.database_name;
|
||||
if (!dictionary_id.database_name.empty())
|
||||
return dictionary_id.database_name;
|
||||
|
||||
return NO_DATABASE_TAG;
|
||||
}
|
||||
|
||||
@ -159,74 +158,65 @@ struct IDictionaryBase : public IExternalLoadable
|
||||
const Columns & key_columns,
|
||||
const DataTypes & key_types) const = 0;
|
||||
|
||||
virtual bool hasHierarchy() const { return false; }
|
||||
|
||||
virtual ColumnPtr getHierarchy(
|
||||
ColumnPtr key_column [[maybe_unused]],
|
||||
const DataTypePtr & key_type [[maybe_unused]]) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Method getHierarchy is not supported for {} dictionary.",
|
||||
getDictionaryID().getNameForLogs());
|
||||
}
|
||||
|
||||
virtual ColumnUInt8::Ptr isInHierarchy(
|
||||
ColumnPtr key_column [[maybe_unused]],
|
||||
ColumnPtr in_key_column [[maybe_unused]],
|
||||
const DataTypePtr & key_type [[maybe_unused]]) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Method isInHierarchy is not supported for {} dictionary.",
|
||||
getDictionaryID().getNameForLogs());
|
||||
}
|
||||
|
||||
virtual ColumnPtr getDescendants(
|
||||
ColumnPtr key_column [[maybe_unused]],
|
||||
const DataTypePtr & key_type [[maybe_unused]],
|
||||
size_t level [[maybe_unused]]) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Method getDescendants is not supported for {} dictionary.",
|
||||
getDictionaryID().getNameForLogs());
|
||||
}
|
||||
|
||||
virtual BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const = 0;
|
||||
|
||||
bool supportUpdates() const override { return true; }
|
||||
|
||||
bool isModified() const override
|
||||
{
|
||||
auto source = getSource();
|
||||
const auto * source = getSource();
|
||||
return source && source->isModified();
|
||||
}
|
||||
|
||||
virtual std::exception_ptr getLastException() const { return {}; }
|
||||
|
||||
std::shared_ptr<IDictionaryBase> shared_from_this()
|
||||
std::shared_ptr<IDictionary> shared_from_this()
|
||||
{
|
||||
return std::static_pointer_cast<IDictionaryBase>(IExternalLoadable::shared_from_this());
|
||||
return std::static_pointer_cast<IDictionary>(IExternalLoadable::shared_from_this());
|
||||
}
|
||||
|
||||
std::shared_ptr<const IDictionaryBase> shared_from_this() const
|
||||
std::shared_ptr<const IDictionary> shared_from_this() const
|
||||
{
|
||||
return std::static_pointer_cast<const IDictionaryBase>(IExternalLoadable::shared_from_this());
|
||||
return std::static_pointer_cast<const IDictionary>(IExternalLoadable::shared_from_this());
|
||||
}
|
||||
|
||||
private:
|
||||
mutable std::mutex name_mutex;
|
||||
mutable StorageID dict_id;
|
||||
mutable StorageID dictionary_id;
|
||||
|
||||
protected:
|
||||
const String full_name;
|
||||
};
|
||||
|
||||
struct IDictionary : IDictionaryBase
|
||||
{
|
||||
IDictionary(const StorageID & dict_id_) : IDictionaryBase(dict_id_) {}
|
||||
|
||||
virtual bool hasHierarchy() const = 0;
|
||||
|
||||
virtual void toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<Key> & out) const = 0;
|
||||
|
||||
/// TODO: Rewrite
|
||||
/// Methods for hierarchy.
|
||||
|
||||
virtual void isInVectorVector(
|
||||
const PaddedPODArray<Key> & /*child_ids*/, const PaddedPODArray<Key> & /*ancestor_ids*/, PaddedPODArray<UInt8> & /*out*/) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Hierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs());
|
||||
}
|
||||
|
||||
virtual void
|
||||
isInVectorConstant(const PaddedPODArray<Key> & /*child_ids*/, const Key /*ancestor_id*/, PaddedPODArray<UInt8> & /*out*/) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Hierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs());
|
||||
}
|
||||
|
||||
virtual void
|
||||
isInConstantVector(const Key /*child_id*/, const PaddedPODArray<Key> & /*ancestor_ids*/, PaddedPODArray<UInt8> & /*out*/) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Hierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs());
|
||||
}
|
||||
|
||||
void isInConstantConstant(const Key child_id, const Key ancestor_id, UInt8 & out) const
|
||||
{
|
||||
PaddedPODArray<UInt8> out_arr(1);
|
||||
isInVectorConstant(PaddedPODArray<Key>(1, child_id), ancestor_id, out_arr);
|
||||
out = out_arr[0];
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -195,7 +195,7 @@ IPAddressDictionary::IPAddressDictionary(
|
||||
DictionarySourcePtr source_ptr_,
|
||||
const DictionaryLifetime dict_lifetime_,
|
||||
bool require_nonempty_)
|
||||
: IDictionaryBase(dict_id_)
|
||||
: IDictionary(dict_id_)
|
||||
, dict_struct(dict_struct_)
|
||||
, source_ptr{std::move(source_ptr_)}
|
||||
, dict_lifetime(dict_lifetime_)
|
||||
@ -804,9 +804,6 @@ static auto keyViewGetter()
|
||||
|
||||
BlockInputStreamPtr IPAddressDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const
|
||||
{
|
||||
using BlockInputStreamType = DictionaryBlockInputStream<UInt64>;
|
||||
|
||||
|
||||
const bool is_ipv4 = std::get_if<IPv4Container>(&ip_column) != nullptr;
|
||||
|
||||
auto get_keys = [is_ipv4](const Columns & columns, const std::vector<DictionaryAttribute> & dict_attributes)
|
||||
@ -827,12 +824,12 @@ BlockInputStreamPtr IPAddressDictionary::getBlockInputStream(const Names & colum
|
||||
if (is_ipv4)
|
||||
{
|
||||
auto get_view = keyViewGetter<ColumnVector<UInt32>, true>();
|
||||
return std::make_shared<BlockInputStreamType>(
|
||||
return std::make_shared<DictionaryBlockInputStream>(
|
||||
shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view));
|
||||
}
|
||||
|
||||
auto get_view = keyViewGetter<ColumnFixedString, false>();
|
||||
return std::make_shared<BlockInputStreamType>(
|
||||
return std::make_shared<DictionaryBlockInputStream>(
|
||||
shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view));
|
||||
}
|
||||
|
||||
|
@ -20,7 +20,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class IPAddressDictionary final : public IDictionaryBase
|
||||
class IPAddressDictionary final : public IDictionary
|
||||
{
|
||||
public:
|
||||
IPAddressDictionary(
|
||||
|
@ -30,7 +30,7 @@ IPolygonDictionary::IPolygonDictionary(
|
||||
const DictionaryLifetime dict_lifetime_,
|
||||
InputType input_type_,
|
||||
PointType point_type_)
|
||||
: IDictionaryBase(dict_id_)
|
||||
: IDictionary(dict_id_)
|
||||
, dict_struct(dict_struct_)
|
||||
, source_ptr(std::move(source_ptr_))
|
||||
, dict_lifetime(dict_lifetime_)
|
||||
@ -142,7 +142,6 @@ ColumnPtr IPolygonDictionary::getColumn(
|
||||
callOnDictionaryAttributeType(attribute.underlying_type, type_call);
|
||||
}
|
||||
|
||||
|
||||
query_count.fetch_add(requested_key_points.size(), std::memory_order_relaxed);
|
||||
|
||||
return result;
|
||||
|
@ -24,7 +24,7 @@ namespace bg = boost::geometry;
|
||||
* An implementation should inherit from this base class and preprocess the data upon construction if needed.
|
||||
* It must override the find method of this class which retrieves the polygon containing a single point.
|
||||
*/
|
||||
class IPolygonDictionary : public IDictionaryBase
|
||||
class IPolygonDictionary : public IDictionary
|
||||
{
|
||||
public:
|
||||
/** Controls the different types of polygons allowed as input.
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
using Key = UInt64;
|
||||
|
||||
RangeDictionaryBlockInputStream(
|
||||
std::shared_ptr<const IDictionaryBase> dictionary,
|
||||
std::shared_ptr<const IDictionary> dictionary,
|
||||
size_t max_block_size,
|
||||
const Names & column_names,
|
||||
PaddedPODArray<Key> && ids_to_fill,
|
||||
@ -49,7 +49,7 @@ private:
|
||||
const PaddedPODArray<RangeType> & block_start_dates,
|
||||
const PaddedPODArray<RangeType> & block_end_dates) const;
|
||||
|
||||
std::shared_ptr<const IDictionaryBase> dictionary;
|
||||
std::shared_ptr<const IDictionary> dictionary;
|
||||
NameSet column_names;
|
||||
PaddedPODArray<Key> ids;
|
||||
PaddedPODArray<RangeType> start_dates;
|
||||
@ -59,7 +59,7 @@ private:
|
||||
|
||||
template <typename RangeType>
|
||||
RangeDictionaryBlockInputStream<RangeType>::RangeDictionaryBlockInputStream(
|
||||
std::shared_ptr<const IDictionaryBase> dictionary_,
|
||||
std::shared_ptr<const IDictionary> dictionary_,
|
||||
size_t max_block_size_,
|
||||
const Names & column_names_,
|
||||
PaddedPODArray<Key> && ids_,
|
||||
|
@ -76,7 +76,7 @@ RangeHashedDictionary::RangeHashedDictionary(
|
||||
DictionarySourcePtr source_ptr_,
|
||||
const DictionaryLifetime dict_lifetime_,
|
||||
bool require_nonempty_)
|
||||
: IDictionaryBase(dict_id_)
|
||||
: IDictionary(dict_id_)
|
||||
, dict_struct(dict_struct_)
|
||||
, source_ptr{std::move(source_ptr_)}
|
||||
, dict_lifetime(dict_lifetime_)
|
||||
@ -185,10 +185,10 @@ ColumnUInt8::Ptr RangeHashedDictionary::hasKeys(const Columns & key_columns, con
|
||||
auto range_column_storage_type = std::make_shared<DataTypeInt64>();
|
||||
auto range_column_updated = castColumnAccurate(column_to_cast, range_column_storage_type);
|
||||
|
||||
PaddedPODArray<Key> key_backup_storage;
|
||||
PaddedPODArray<UInt64> key_backup_storage;
|
||||
PaddedPODArray<RangeStorageType> range_backup_storage;
|
||||
|
||||
const PaddedPODArray<Key> & ids = getColumnVectorData(this, key_columns[0], key_backup_storage);
|
||||
const PaddedPODArray<UInt64> & ids = getColumnVectorData(this, key_columns[0], key_backup_storage);
|
||||
const PaddedPODArray<RangeStorageType> & dates = getColumnVectorData(this, range_column_updated, range_backup_storage);
|
||||
|
||||
const auto & attribute = attributes.front();
|
||||
@ -213,7 +213,7 @@ ColumnUInt8::Ptr RangeHashedDictionary::hasKeys(const Columns & key_columns, con
|
||||
template <typename AttributeType>
|
||||
ColumnUInt8::Ptr RangeHashedDictionary::hasKeysImpl(
|
||||
const Attribute & attribute,
|
||||
const PaddedPODArray<Key> & ids,
|
||||
const PaddedPODArray<UInt64> & ids,
|
||||
const PaddedPODArray<RangeStorageType> & dates) const
|
||||
{
|
||||
auto result = ColumnUInt8::create(ids.size());
|
||||
@ -388,10 +388,10 @@ void RangeHashedDictionary::getItemsImpl(
|
||||
ValueSetter && set_value,
|
||||
DefaultValueExtractor & default_value_extractor) const
|
||||
{
|
||||
PaddedPODArray<Key> key_backup_storage;
|
||||
PaddedPODArray<UInt64> key_backup_storage;
|
||||
PaddedPODArray<RangeStorageType> range_backup_storage;
|
||||
|
||||
const PaddedPODArray<Key> & ids = getColumnVectorData(this, key_columns[0], key_backup_storage);
|
||||
const PaddedPODArray<UInt64> & ids = getColumnVectorData(this, key_columns[0], key_backup_storage);
|
||||
const PaddedPODArray<RangeStorageType> & dates = getColumnVectorData(this, key_columns[1], range_backup_storage);
|
||||
|
||||
const auto & attr = *std::get<Ptr<AttributeType>>(attribute.maps);
|
||||
@ -436,7 +436,7 @@ void RangeHashedDictionary::getItemsImpl(
|
||||
|
||||
|
||||
template <typename T>
|
||||
void RangeHashedDictionary::setAttributeValueImpl(Attribute & attribute, const Key id, const Range & range, const Field & value)
|
||||
void RangeHashedDictionary::setAttributeValueImpl(Attribute & attribute, const UInt64 id, const Range & range, const Field & value)
|
||||
{
|
||||
using ValueType = std::conditional_t<std::is_same_v<T, String>, StringRef, T>;
|
||||
auto & map = *std::get<Ptr<ValueType>>(attribute.maps);
|
||||
@ -480,7 +480,7 @@ void RangeHashedDictionary::setAttributeValueImpl(Attribute & attribute, const K
|
||||
map.insert({id, Values<ValueType>{std::move(value_to_insert)}});
|
||||
}
|
||||
|
||||
void RangeHashedDictionary::setAttributeValue(Attribute & attribute, const Key id, const Range & range, const Field & value)
|
||||
void RangeHashedDictionary::setAttributeValue(Attribute & attribute, const UInt64 id, const Range & range, const Field & value)
|
||||
{
|
||||
auto type_call = [&](const auto &dictionary_attribute_type)
|
||||
{
|
||||
@ -515,7 +515,7 @@ RangeHashedDictionary::getAttributeWithType(const std::string & attribute_name,
|
||||
|
||||
template <typename RangeType>
|
||||
void RangeHashedDictionary::getIdsAndDates(
|
||||
PaddedPODArray<Key> & ids,
|
||||
PaddedPODArray<UInt64> & ids,
|
||||
PaddedPODArray<RangeType> & start_dates,
|
||||
PaddedPODArray<RangeType> & end_dates) const
|
||||
{
|
||||
@ -536,7 +536,7 @@ void RangeHashedDictionary::getIdsAndDates(
|
||||
template <typename T, typename RangeType>
|
||||
void RangeHashedDictionary::getIdsAndDates(
|
||||
const Attribute & attribute,
|
||||
PaddedPODArray<Key> & ids,
|
||||
PaddedPODArray<UInt64> & ids,
|
||||
PaddedPODArray<RangeType> & start_dates,
|
||||
PaddedPODArray<RangeType> & end_dates) const
|
||||
{
|
||||
@ -567,7 +567,7 @@ void RangeHashedDictionary::getIdsAndDates(
|
||||
template <typename RangeType>
|
||||
BlockInputStreamPtr RangeHashedDictionary::getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const
|
||||
{
|
||||
PaddedPODArray<Key> ids;
|
||||
PaddedPODArray<UInt64> ids;
|
||||
PaddedPODArray<RangeType> start_dates;
|
||||
PaddedPODArray<RangeType> end_dates;
|
||||
getIdsAndDates(ids, start_dates, end_dates);
|
||||
|
@ -16,7 +16,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class RangeHashedDictionary final : public IDictionaryBase
|
||||
class RangeHashedDictionary final : public IDictionary
|
||||
{
|
||||
public:
|
||||
RangeHashedDictionary(
|
||||
@ -160,25 +160,25 @@ private:
|
||||
template <typename AttributeType>
|
||||
ColumnUInt8::Ptr hasKeysImpl(
|
||||
const Attribute & attribute,
|
||||
const PaddedPODArray<Key> & ids,
|
||||
const PaddedPODArray<UInt64> & ids,
|
||||
const PaddedPODArray<RangeStorageType> & dates) const;
|
||||
|
||||
template <typename T>
|
||||
static void setAttributeValueImpl(Attribute & attribute, const Key id, const Range & range, const Field & value);
|
||||
static void setAttributeValueImpl(Attribute & attribute, const UInt64 id, const Range & range, const Field & value);
|
||||
|
||||
static void setAttributeValue(Attribute & attribute, const Key id, const Range & range, const Field & value);
|
||||
static void setAttributeValue(Attribute & attribute, const UInt64 id, const Range & range, const Field & value);
|
||||
|
||||
const Attribute & getAttribute(const std::string & attribute_name) const;
|
||||
|
||||
const Attribute & getAttributeWithType(const std::string & name, const AttributeUnderlyingType type) const;
|
||||
|
||||
template <typename RangeType>
|
||||
void getIdsAndDates(PaddedPODArray<Key> & ids, PaddedPODArray<RangeType> & start_dates, PaddedPODArray<RangeType> & end_dates) const;
|
||||
void getIdsAndDates(PaddedPODArray<UInt64> & ids, PaddedPODArray<RangeType> & start_dates, PaddedPODArray<RangeType> & end_dates) const;
|
||||
|
||||
template <typename T, typename RangeType>
|
||||
void getIdsAndDates(
|
||||
const Attribute & attribute,
|
||||
PaddedPODArray<Key> & ids,
|
||||
PaddedPODArray<UInt64> & ids,
|
||||
PaddedPODArray<RangeType> & start_dates,
|
||||
PaddedPODArray<RangeType> & end_dates) const;
|
||||
|
||||
|
@ -57,7 +57,6 @@ void registerDictionaries()
|
||||
{
|
||||
auto & factory = DictionaryFactory::instance();
|
||||
registerDictionaryRangeHashed(factory);
|
||||
registerDictionaryComplexKeyHashed(factory);
|
||||
registerDictionaryTrie(factory);
|
||||
registerDictionaryFlat(factory);
|
||||
registerDictionaryHashed(factory);
|
||||
|
@ -1,7 +1,5 @@
|
||||
#if defined(__linux__) || defined(__FreeBSD__)
|
||||
|
||||
#include <iostream>
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <Dictionaries/SSDCacheDictionaryStorage.h>
|
||||
|
225
src/Dictionaries/tests/gtest_hierarchy_dictionaries_utils.cpp
Normal file
225
src/Dictionaries/tests/gtest_hierarchy_dictionaries_utils.cpp
Normal file
@ -0,0 +1,225 @@
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
|
||||
#include <Dictionaries/HierarchyDictionariesUtils.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
TEST(HierarchyDictionariesUtils, getHierarchy)
|
||||
{
|
||||
{
|
||||
HashMap<UInt64, UInt64> child_to_parent;
|
||||
child_to_parent[1] = 0;
|
||||
child_to_parent[2] = 1;
|
||||
child_to_parent[3] = 1;
|
||||
child_to_parent[4] = 2;
|
||||
|
||||
auto is_key_valid_func = [&](auto key) { return child_to_parent.find(key) != nullptr; };
|
||||
|
||||
auto get_parent_key_func = [&](auto key)
|
||||
{
|
||||
auto it = child_to_parent.find(key);
|
||||
std::optional<UInt64> value = (it != nullptr ? std::make_optional(it->getMapped()) : std::nullopt);
|
||||
return value;
|
||||
};
|
||||
|
||||
UInt64 hierarchy_null_value_key = 0;
|
||||
PaddedPODArray<UInt64> keys = {1, 2, 3, 4, 5};
|
||||
|
||||
auto result = DB::detail::getHierarchy(
|
||||
keys,
|
||||
hierarchy_null_value_key,
|
||||
is_key_valid_func,
|
||||
get_parent_key_func);
|
||||
|
||||
const auto & actual_elements = result.elements;
|
||||
const auto & actual_offsets = result.offsets;
|
||||
|
||||
PaddedPODArray<UInt64> expected_elements = {1, 2, 1, 3, 1, 4, 2, 1};
|
||||
PaddedPODArray<IColumn::Offset> expected_offsets = {1, 3, 5, 8, 8};
|
||||
|
||||
ASSERT_EQ(actual_elements, expected_elements);
|
||||
ASSERT_EQ(actual_offsets, expected_offsets);
|
||||
}
|
||||
{
|
||||
HashMap<UInt64, UInt64> child_to_parent;
|
||||
child_to_parent[1] = 2;
|
||||
child_to_parent[2] = 1;
|
||||
|
||||
auto is_key_valid_func = [&](auto key) { return child_to_parent.find(key) != nullptr; };
|
||||
|
||||
auto get_parent_key_func = [&](auto key)
|
||||
{
|
||||
auto it = child_to_parent.find(key);
|
||||
std::optional<UInt64> value = (it != nullptr ? std::make_optional(it->getMapped()) : std::nullopt);
|
||||
return value;
|
||||
};
|
||||
|
||||
UInt64 hierarchy_null_value_key = 0;
|
||||
PaddedPODArray<UInt64> keys = {1, 2, 3};
|
||||
|
||||
auto result = DB::detail::getHierarchy(
|
||||
keys,
|
||||
hierarchy_null_value_key,
|
||||
is_key_valid_func,
|
||||
get_parent_key_func);
|
||||
|
||||
const auto & actual_elements = result.elements;
|
||||
const auto & actual_offsets = result.offsets;
|
||||
|
||||
PaddedPODArray<UInt64> expected_elements = {1, 2, 2};
|
||||
PaddedPODArray<IColumn::Offset> expected_offsets = {2, 3, 3};
|
||||
|
||||
ASSERT_EQ(actual_elements, expected_elements);
|
||||
ASSERT_EQ(actual_offsets, expected_offsets);
|
||||
}
|
||||
}
|
||||
|
||||
TEST(HierarchyDictionariesUtils, getIsInHierarchy)
|
||||
{
|
||||
{
|
||||
HashMap<UInt64, UInt64> child_to_parent;
|
||||
child_to_parent[1] = 0;
|
||||
child_to_parent[2] = 1;
|
||||
child_to_parent[3] = 1;
|
||||
child_to_parent[4] = 2;
|
||||
|
||||
auto is_key_valid_func = [&](auto key) { return child_to_parent.find(key) != nullptr; };
|
||||
|
||||
auto get_parent_key_func = [&](auto key)
|
||||
{
|
||||
auto it = child_to_parent.find(key);
|
||||
std::optional<UInt64> value = (it != nullptr ? std::make_optional(it->getMapped()) : std::nullopt);
|
||||
return value;
|
||||
};
|
||||
|
||||
UInt64 hierarchy_null_value_key = 0;
|
||||
PaddedPODArray<UInt64> keys = {1, 2, 3, 4, 5};
|
||||
PaddedPODArray<UInt64> keys_in = {1, 1, 1, 2, 5};
|
||||
|
||||
PaddedPODArray<UInt8> actual = DB::detail::getIsInHierarchy(
|
||||
keys,
|
||||
keys_in,
|
||||
hierarchy_null_value_key,
|
||||
is_key_valid_func,
|
||||
get_parent_key_func);
|
||||
|
||||
PaddedPODArray<UInt8> expected = {1,1,1,1,0};
|
||||
|
||||
ASSERT_EQ(actual, expected);
|
||||
}
|
||||
{
|
||||
HashMap<UInt64, UInt64> child_to_parent;
|
||||
child_to_parent[1] = 2;
|
||||
child_to_parent[2] = 1;
|
||||
|
||||
auto is_key_valid_func = [&](auto key)
|
||||
{
|
||||
return child_to_parent.find(key) != nullptr;
|
||||
};
|
||||
|
||||
auto get_parent_key_func = [&](auto key)
|
||||
{
|
||||
auto it = child_to_parent.find(key);
|
||||
std::optional<UInt64> value = (it != nullptr ? std::make_optional(it->getMapped()) : std::nullopt);
|
||||
return value;
|
||||
};
|
||||
|
||||
UInt64 hierarchy_null_value_key = 0;
|
||||
PaddedPODArray<UInt64> keys = {1, 2, 3};
|
||||
PaddedPODArray<UInt64> keys_in = {1, 2, 3};
|
||||
|
||||
PaddedPODArray<UInt8> actual = DB::detail::getIsInHierarchy(
|
||||
keys,
|
||||
keys_in,
|
||||
hierarchy_null_value_key,
|
||||
is_key_valid_func,
|
||||
get_parent_key_func);
|
||||
|
||||
PaddedPODArray<UInt8> expected = {1, 1, 0};
|
||||
ASSERT_EQ(actual, expected);
|
||||
}
|
||||
}
|
||||
|
||||
TEST(HierarchyDictionariesUtils, getDescendants)
|
||||
{
|
||||
{
|
||||
HashMap<UInt64, PaddedPODArray<UInt64>> parent_to_child;
|
||||
parent_to_child[0].emplace_back(1);
|
||||
parent_to_child[1].emplace_back(2);
|
||||
parent_to_child[1].emplace_back(3);
|
||||
parent_to_child[2].emplace_back(4);
|
||||
|
||||
PaddedPODArray<UInt64> keys = {0, 1, 2, 3, 4};
|
||||
|
||||
{
|
||||
auto result = DB::detail::getDescendants(
|
||||
keys,
|
||||
parent_to_child,
|
||||
DB::detail::GetAllDescendantsStrategy());
|
||||
|
||||
const auto & actual_elements = result.elements;
|
||||
const auto & actual_offsets = result.offsets;
|
||||
|
||||
PaddedPODArray<UInt64> expected_elements = {1, 2, 3, 4, 2, 3, 4, 4};
|
||||
PaddedPODArray<IColumn::Offset> expected_offsets = {4, 7, 8, 8, 8};
|
||||
|
||||
ASSERT_EQ(actual_elements, expected_elements);
|
||||
ASSERT_EQ(actual_offsets, expected_offsets);
|
||||
}
|
||||
{
|
||||
auto result = DB::detail::getDescendants(
|
||||
keys,
|
||||
parent_to_child,
|
||||
DB::detail::GetDescendantsAtSpecificLevelStrategy{1});
|
||||
|
||||
const auto & actual_elements = result.elements;
|
||||
const auto & actual_offsets = result.offsets;
|
||||
|
||||
PaddedPODArray<UInt64> expected_elements = {1, 2, 3, 4};
|
||||
PaddedPODArray<IColumn::Offset> expected_offsets = {1, 3, 4, 4, 4};
|
||||
|
||||
ASSERT_EQ(actual_elements, expected_elements);
|
||||
ASSERT_EQ(actual_offsets, expected_offsets);
|
||||
}
|
||||
}
|
||||
{
|
||||
HashMap<UInt64, PaddedPODArray<UInt64>> parent_to_child;
|
||||
parent_to_child[1].emplace_back(2);
|
||||
parent_to_child[2].emplace_back(1);
|
||||
|
||||
PaddedPODArray<UInt64> keys = {1, 2, 3};
|
||||
|
||||
{
|
||||
auto result = DB::detail::getDescendants(
|
||||
keys,
|
||||
parent_to_child,
|
||||
DB::detail::GetAllDescendantsStrategy());
|
||||
|
||||
const auto & actual_elements = result.elements;
|
||||
const auto & actual_offsets = result.offsets;
|
||||
|
||||
PaddedPODArray<UInt64> expected_elements = {2, 1, 1};
|
||||
PaddedPODArray<IColumn::Offset> expected_offsets = {2, 3, 3};
|
||||
|
||||
ASSERT_EQ(actual_elements, expected_elements);
|
||||
ASSERT_EQ(actual_offsets, expected_offsets);
|
||||
}
|
||||
{
|
||||
auto result = DB::detail::getDescendants(
|
||||
keys,
|
||||
parent_to_child,
|
||||
DB::detail::GetDescendantsAtSpecificLevelStrategy{1});
|
||||
|
||||
const auto & actual_elements = result.elements;
|
||||
const auto & actual_offsets = result.offsets;
|
||||
|
||||
PaddedPODArray<UInt64> expected_elements = {2, 1};
|
||||
PaddedPODArray<IColumn::Offset> expected_offsets = {1, 2, 2};
|
||||
|
||||
ASSERT_EQ(actual_elements, expected_elements);
|
||||
ASSERT_EQ(actual_offsets, expected_offsets);
|
||||
}
|
||||
}
|
||||
}
|
@ -26,7 +26,7 @@ SRCS(
|
||||
CassandraDictionarySource.cpp
|
||||
CassandraHelpers.cpp
|
||||
ClickHouseDictionarySource.cpp
|
||||
ComplexKeyHashedDictionary.cpp
|
||||
DictionaryBlockInputStream.cpp
|
||||
DictionaryBlockInputStreamBase.cpp
|
||||
DictionaryFactory.cpp
|
||||
DictionarySourceFactory.cpp
|
||||
@ -48,6 +48,7 @@ SRCS(
|
||||
FlatDictionary.cpp
|
||||
HTTPDictionarySource.cpp
|
||||
HashedDictionary.cpp
|
||||
HierarchyDictionariesUtils.cpp
|
||||
IPAddressDictionary.cpp
|
||||
LibraryDictionarySource.cpp
|
||||
LibraryDictionarySourceExternal.cpp
|
||||
|
@ -24,6 +24,8 @@ void registerFunctionsExternalDictionaries(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionDictGetString>();
|
||||
factory.registerFunction<FunctionDictGetHierarchy>();
|
||||
factory.registerFunction<FunctionDictIsIn>();
|
||||
factory.registerFunction<FunctionDictGetChildren>();
|
||||
factory.registerFunction<FunctionDictGetDescendants>();
|
||||
factory.registerFunction<FunctionDictGetUInt8OrDefault>();
|
||||
factory.registerFunction<FunctionDictGetUInt16OrDefault>();
|
||||
factory.registerFunction<FunctionDictGetUInt32OrDefault>();
|
||||
|
@ -28,16 +28,6 @@
|
||||
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
#include <Dictionaries/FlatDictionary.h>
|
||||
#include <Dictionaries/HashedDictionary.h>
|
||||
#include <Dictionaries/CacheDictionary.h>
|
||||
#include <Dictionaries/ComplexKeyHashedDictionary.h>
|
||||
#include <Dictionaries/RangeHashedDictionary.h>
|
||||
#include <Dictionaries/IPAddressDictionary.h>
|
||||
#include <Dictionaries/PolygonDictionaryImplementations.h>
|
||||
#include <Dictionaries/DirectDictionary.h>
|
||||
|
||||
#include <ext/range.h>
|
||||
|
||||
#include <type_traits>
|
||||
@ -49,7 +39,6 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
extern const int UNKNOWN_TYPE;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -77,7 +66,7 @@ class FunctionDictHelper
|
||||
public:
|
||||
explicit FunctionDictHelper(const Context & context_) : context(context_) {}
|
||||
|
||||
std::shared_ptr<const IDictionaryBase> getDictionary(const String & dictionary_name)
|
||||
std::shared_ptr<const IDictionary> getDictionary(const String & dictionary_name)
|
||||
{
|
||||
auto dict = context.getExternalDictionariesLoader().getDictionary(dictionary_name, context);
|
||||
|
||||
@ -90,9 +79,13 @@ public:
|
||||
return dict;
|
||||
}
|
||||
|
||||
std::shared_ptr<const IDictionaryBase> getDictionary(const ColumnWithTypeAndName & column)
|
||||
std::shared_ptr<const IDictionary> getDictionary(const ColumnPtr & column)
|
||||
{
|
||||
const auto * dict_name_col = checkAndGetColumnConst<ColumnString>(column.column.get());
|
||||
const auto * dict_name_col = checkAndGetColumnConst<ColumnString>(column.get());
|
||||
|
||||
if (!dict_name_col)
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected const String column");
|
||||
|
||||
return getDictionary(dict_name_col->getValue<String>());
|
||||
}
|
||||
|
||||
@ -187,7 +180,7 @@ private:
|
||||
if (input_rows_count == 0)
|
||||
return result_type->createColumn();
|
||||
|
||||
auto dictionary = helper.getDictionary(arguments[0]);
|
||||
auto dictionary = helper.getDictionary(arguments[0].column);
|
||||
auto dictionary_key_type = dictionary->getKeyType();
|
||||
|
||||
const ColumnWithTypeAndName & key_column_with_type = arguments[1];
|
||||
@ -727,12 +720,16 @@ private:
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
|
||||
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of first argument of function ({}). Expected String. Actual type ({})",
|
||||
getName(),
|
||||
arguments[0]->getName());
|
||||
|
||||
if (!WhichDataType(arguments[1]).isUInt64())
|
||||
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
|
||||
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of second argument of function ({}). Expected UInt64. Actual type ({})",
|
||||
getName(),
|
||||
arguments[1]->getName());
|
||||
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
|
||||
}
|
||||
@ -744,109 +741,15 @@ private:
|
||||
if (input_rows_count == 0)
|
||||
return result_type->createColumn();
|
||||
|
||||
auto dict = helper.getDictionary(arguments[0]);
|
||||
ColumnPtr res;
|
||||
auto dictionary = helper.getDictionary(arguments[0].column);
|
||||
|
||||
/// TODO: Rewrite this
|
||||
if (!((res = executeDispatch<FlatDictionary>(arguments, result_type, dict))
|
||||
|| (res = executeDispatch<DirectDictionary<DictionaryKeyType::simple>>(arguments, result_type, dict))
|
||||
|| (res = executeDispatch<HashedDictionary>(arguments, result_type, dict))
|
||||
|| (res = executeDispatch<CacheDictionary<DictionaryKeyType::simple>>(arguments, result_type, dict))))
|
||||
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
||||
if (!dictionary->hasHierarchy())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
|
||||
"Dictionary ({}) does not support hierarchy",
|
||||
dictionary->getFullName());
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
template <typename DictionaryType>
|
||||
ColumnPtr executeDispatch(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const std::shared_ptr<const IDictionaryBase> & dict_ptr) const
|
||||
{
|
||||
const auto * dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
|
||||
if (!dict)
|
||||
return nullptr;
|
||||
|
||||
if (!dict->hasHierarchy())
|
||||
throw Exception{"Dictionary does not have a hierarchy", ErrorCodes::UNSUPPORTED_METHOD};
|
||||
|
||||
const auto get_hierarchies = [&] (const PaddedPODArray<UInt64> & in, PaddedPODArray<UInt64> & out, PaddedPODArray<UInt64> & offsets)
|
||||
{
|
||||
const auto size = in.size();
|
||||
|
||||
/// copy of `in` array
|
||||
auto in_array = std::make_unique<PaddedPODArray<UInt64>>(std::begin(in), std::end(in));
|
||||
/// used for storing and handling result of ::toParent call
|
||||
auto out_array = std::make_unique<PaddedPODArray<UInt64>>(size);
|
||||
/// resulting hierarchies
|
||||
std::vector<std::vector<IDictionary::Key>> hierarchies(size); /// TODO Bad code, poor performance.
|
||||
|
||||
/// total number of non-zero elements, used for allocating all the required memory upfront
|
||||
size_t total_count = 0;
|
||||
|
||||
while (true)
|
||||
{
|
||||
auto all_zeroes = true;
|
||||
|
||||
/// erase zeroed identifiers, store non-zeroed ones
|
||||
for (const auto i : ext::range(0, size))
|
||||
{
|
||||
const auto id = (*in_array)[i];
|
||||
if (0 == id)
|
||||
continue;
|
||||
|
||||
|
||||
auto & hierarchy = hierarchies[i];
|
||||
|
||||
/// Checking for loop
|
||||
if (std::find(std::begin(hierarchy), std::end(hierarchy), id) != std::end(hierarchy))
|
||||
continue;
|
||||
|
||||
all_zeroes = false;
|
||||
/// place id at it's corresponding place
|
||||
hierarchy.push_back(id);
|
||||
|
||||
++total_count;
|
||||
}
|
||||
|
||||
if (all_zeroes)
|
||||
break;
|
||||
|
||||
/// translate all non-zero identifiers at once
|
||||
dict->toParent(*in_array, *out_array);
|
||||
|
||||
/// we're going to use the `in_array` from this iteration as `out_array` on the next one
|
||||
std::swap(in_array, out_array);
|
||||
}
|
||||
|
||||
out.reserve(total_count);
|
||||
offsets.resize(size);
|
||||
|
||||
for (const auto i : ext::range(0, size))
|
||||
{
|
||||
const auto & ids = hierarchies[i];
|
||||
out.insert_assume_reserved(std::begin(ids), std::end(ids));
|
||||
offsets[i] = out.size();
|
||||
}
|
||||
};
|
||||
|
||||
const auto * id_col_untyped = arguments[1].column.get();
|
||||
if (const auto * id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
|
||||
{
|
||||
const auto & in = id_col->getData();
|
||||
auto backend = ColumnUInt64::create();
|
||||
auto offsets = ColumnArray::ColumnOffsets::create();
|
||||
get_hierarchies(in, backend->getData(), offsets->getData());
|
||||
return ColumnArray::create(std::move(backend), std::move(offsets));
|
||||
}
|
||||
else if (const auto * id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
|
||||
{
|
||||
const PaddedPODArray<UInt64> in(1, id_col_const->getValue<UInt64>());
|
||||
auto backend = ColumnUInt64::create();
|
||||
auto offsets = ColumnArray::ColumnOffsets::create();
|
||||
get_hierarchies(in, backend->getData(), offsets->getData());
|
||||
auto array = ColumnArray::create(std::move(backend), std::move(offsets));
|
||||
return result_type->createColumnConst(id_col_const->size(), (*array)[0].get<Array>());
|
||||
}
|
||||
else
|
||||
throw Exception{"Second argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
|
||||
ColumnPtr result = dictionary->getHierarchy(arguments[1].column, std::make_shared<DataTypeUInt64>());
|
||||
return result;
|
||||
}
|
||||
|
||||
mutable FunctionDictHelper helper;
|
||||
@ -877,16 +780,22 @@ private:
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
|
||||
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of first argument of function ({}). Expected String. Actual type ({})",
|
||||
getName(),
|
||||
arguments[0]->getName());
|
||||
|
||||
if (!WhichDataType(arguments[1]).isUInt64())
|
||||
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
|
||||
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of second argument of function ({}). Expected UInt64. Actual type ({})",
|
||||
getName(),
|
||||
arguments[1]->getName());
|
||||
|
||||
if (!WhichDataType(arguments[2]).isUInt64())
|
||||
throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
|
||||
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of third argument of function ({}). Expected UInt64. Actual type ({})",
|
||||
getName(),
|
||||
arguments[2]->getName());
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
@ -898,105 +807,163 @@ private:
|
||||
if (input_rows_count == 0)
|
||||
return result_type->createColumn();
|
||||
|
||||
auto dict = helper.getDictionary(arguments[0]);
|
||||
auto dict = helper.getDictionary(arguments[0].column);
|
||||
|
||||
ColumnPtr res;
|
||||
if (!((res = executeDispatch<FlatDictionary>(arguments, dict))
|
||||
|| (res = executeDispatch<DirectDictionary<DictionaryKeyType::simple>>(arguments, dict))
|
||||
|| (res = executeDispatch<HashedDictionary>(arguments, dict))
|
||||
|| (res = executeDispatch<CacheDictionary<DictionaryKeyType::simple>>(arguments, dict))))
|
||||
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
||||
if (!dict->hasHierarchy())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName());
|
||||
|
||||
ColumnPtr res = dict->isInHierarchy(arguments[1].column, arguments[2].column, std::make_shared<DataTypeUInt64>());
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
template <typename DictionaryType>
|
||||
ColumnPtr executeDispatch(const ColumnsWithTypeAndName & arguments, const std::shared_ptr<const IDictionaryBase> & dict_ptr) const
|
||||
mutable FunctionDictHelper helper;
|
||||
};
|
||||
|
||||
class FunctionDictGetChildren final : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "dictGetChildren";
|
||||
|
||||
static FunctionPtr create(const Context & context)
|
||||
{
|
||||
const auto * dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
|
||||
if (!dict)
|
||||
return nullptr;
|
||||
|
||||
if (!dict->hasHierarchy())
|
||||
throw Exception{"Dictionary does not have a hierarchy", ErrorCodes::UNSUPPORTED_METHOD};
|
||||
|
||||
const auto * child_id_col_untyped = arguments[1].column.get();
|
||||
const auto * ancestor_id_col_untyped = arguments[2].column.get();
|
||||
|
||||
if (const auto * child_id_col = checkAndGetColumn<ColumnUInt64>(child_id_col_untyped))
|
||||
return execute(dict, child_id_col, ancestor_id_col_untyped);
|
||||
else if (const auto * child_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(child_id_col_untyped))
|
||||
return execute(dict, child_id_col_const, ancestor_id_col_untyped);
|
||||
else
|
||||
throw Exception{"Illegal column " + child_id_col_untyped->getName()
|
||||
+ " of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
return std::make_shared<FunctionDictGetChildren>(context);
|
||||
}
|
||||
|
||||
template <typename DictionaryType>
|
||||
ColumnPtr execute(const DictionaryType * dict,
|
||||
const ColumnUInt64 * child_id_col, const IColumn * ancestor_id_col_untyped) const
|
||||
explicit FunctionDictGetChildren(const Context & context_)
|
||||
: helper(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
private:
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const final { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (const auto * ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped))
|
||||
{
|
||||
auto out = ColumnUInt8::create();
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of first argument of function ({}). Expected String. Actual type ({})",
|
||||
getName(),
|
||||
arguments[0]->getName());
|
||||
|
||||
const auto & child_ids = child_id_col->getData();
|
||||
const auto & ancestor_ids = ancestor_id_col->getData();
|
||||
auto & data = out->getData();
|
||||
const auto size = child_id_col->size();
|
||||
data.resize(size);
|
||||
if (!WhichDataType(arguments[1]).isUInt64())
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of second argument of function ({}). Expected UInt64. Actual type ({})",
|
||||
getName(),
|
||||
arguments[1]->getName());
|
||||
|
||||
dict->isInVectorVector(child_ids, ancestor_ids, data);
|
||||
return out;
|
||||
}
|
||||
else if (const auto * ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
|
||||
{
|
||||
auto out = ColumnUInt8::create();
|
||||
|
||||
const auto & child_ids = child_id_col->getData();
|
||||
const auto ancestor_id = ancestor_id_col_const->getValue<UInt64>();
|
||||
auto & data = out->getData();
|
||||
const auto size = child_id_col->size();
|
||||
data.resize(size);
|
||||
|
||||
dict->isInVectorConstant(child_ids, ancestor_id, data);
|
||||
return out;
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception{"Illegal column " + ancestor_id_col_untyped->getName()
|
||||
+ " of third argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
|
||||
}
|
||||
|
||||
template <typename DictionaryType>
|
||||
ColumnPtr execute(const DictionaryType * dict, const ColumnConst * child_id_col, const IColumn * ancestor_id_col_untyped) const
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
if (const auto * ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped))
|
||||
if (input_rows_count == 0)
|
||||
return result_type->createColumn();
|
||||
|
||||
auto dictionary = helper.getDictionary(arguments[0].column);
|
||||
|
||||
if (!dictionary->hasHierarchy())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
|
||||
"Dictionary ({}) does not support hierarchy",
|
||||
dictionary->getFullName());
|
||||
|
||||
ColumnPtr result = dictionary->getDescendants(arguments[1].column, std::make_shared<DataTypeUInt64>(), 1);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
mutable FunctionDictHelper helper;
|
||||
};
|
||||
|
||||
class FunctionDictGetDescendants final : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "dictGetDescendants";
|
||||
|
||||
static FunctionPtr create(const Context & context)
|
||||
{
|
||||
return std::make_shared<FunctionDictGetDescendants>(context);
|
||||
}
|
||||
|
||||
explicit FunctionDictGetDescendants(const Context & context_)
|
||||
: helper(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
private:
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool isVariadic() const override { return true; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const final { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
size_t arguments_size = arguments.size();
|
||||
if (arguments_size < 2 || arguments_size > 3)
|
||||
{
|
||||
auto out = ColumnUInt8::create();
|
||||
|
||||
const auto child_id = child_id_col->getValue<UInt64>();
|
||||
const auto & ancestor_ids = ancestor_id_col->getData();
|
||||
auto & data = out->getData();
|
||||
const auto size = child_id_col->size();
|
||||
data.resize(size);
|
||||
|
||||
dict->isInConstantVector(child_id, ancestor_ids, data);
|
||||
return out;
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Illegal arguments size of function ({}). Expects 2 or 3 arguments size. Actual size ({})",
|
||||
getName(),
|
||||
arguments_size);
|
||||
}
|
||||
else if (const auto * ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
|
||||
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of first argument of function ({}). Expected const String. Actual type ({})",
|
||||
getName(),
|
||||
arguments[0]->getName());
|
||||
|
||||
if (!WhichDataType(arguments[1]).isUInt64())
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of second argument of function ({}). Expected UInt64. Actual type ({})",
|
||||
getName(),
|
||||
arguments[1]->getName());
|
||||
|
||||
if (arguments.size() == 3 && !isUnsignedInteger(arguments[2]))
|
||||
{
|
||||
const auto child_id = child_id_col->getValue<UInt64>();
|
||||
const auto ancestor_id = ancestor_id_col_const->getValue<UInt64>();
|
||||
UInt8 res = 0;
|
||||
|
||||
dict->isInConstantConstant(child_id, ancestor_id, res);
|
||||
return DataTypeUInt8().createColumnConst(child_id_col->size(), res);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of third argument of function ({}). Expected const unsigned integer. Actual type ({})",
|
||||
getName(),
|
||||
arguments[2]->getName());
|
||||
}
|
||||
else
|
||||
throw Exception{"Illegal column " + ancestor_id_col_untyped->getName()
|
||||
+ " of third argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
if (input_rows_count == 0)
|
||||
return result_type->createColumn();
|
||||
|
||||
auto dictionary = helper.getDictionary(arguments[0].column);
|
||||
|
||||
size_t level = 0;
|
||||
|
||||
if (arguments.size() == 3)
|
||||
{
|
||||
if (!isColumnConst(*arguments[2].column))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type of third argument of function ({}). Expected const unsigned integer.",
|
||||
getName());
|
||||
|
||||
level = static_cast<size_t>(arguments[2].column->get64(0));
|
||||
}
|
||||
|
||||
if (!dictionary->hasHierarchy())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
|
||||
"Dictionary ({}) does not support hierarchy",
|
||||
dictionary->getFullName());
|
||||
|
||||
ColumnPtr res = dictionary->getDescendants(arguments[1].column, std::make_shared<DataTypeUInt64>(), level);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
mutable FunctionDictHelper helper;
|
||||
|
@ -46,13 +46,13 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create(
|
||||
ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, const Context & context) const
|
||||
{
|
||||
std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, context.getCurrentDatabase());
|
||||
return std::static_pointer_cast<const IDictionaryBase>(load(resolved_dictionary_name));
|
||||
return std::static_pointer_cast<const IDictionary>(load(resolved_dictionary_name));
|
||||
}
|
||||
|
||||
ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::tryGetDictionary(const std::string & dictionary_name, const Context & context) const
|
||||
{
|
||||
std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, context.getCurrentDatabase());
|
||||
return std::static_pointer_cast<const IDictionaryBase>(tryLoad(resolved_dictionary_name));
|
||||
return std::static_pointer_cast<const IDictionary>(tryLoad(resolved_dictionary_name));
|
||||
}
|
||||
|
||||
|
||||
|
@ -15,7 +15,7 @@ class IExternalLoaderConfigRepository;
|
||||
class ExternalDictionariesLoader : public ExternalLoader
|
||||
{
|
||||
public:
|
||||
using DictPtr = std::shared_ptr<const IDictionaryBase>;
|
||||
using DictPtr = std::shared_ptr<const IDictionary>;
|
||||
|
||||
/// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds.
|
||||
explicit ExternalDictionariesLoader(Context & global_context_);
|
||||
|
@ -58,7 +58,7 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con
|
||||
const auto & external_dictionaries = context.getExternalDictionariesLoader();
|
||||
for (const auto & load_result : external_dictionaries.getLoadResults())
|
||||
{
|
||||
const auto dict_ptr = std::dynamic_pointer_cast<const IDictionaryBase>(load_result.object);
|
||||
const auto dict_ptr = std::dynamic_pointer_cast<const IDictionary>(load_result.object);
|
||||
DictionaryStructure dictionary_structure = ExternalDictionariesLoader::getDictionaryStructure(*load_result.config);
|
||||
|
||||
StorageID dict_id = StorageID::createEmpty();
|
||||
|
@ -1,38 +1,17 @@
|
||||
<test max_ignored_relative_change="0.3">
|
||||
<create_query>
|
||||
CREATE TABLE simple_direct_dictionary_test_table
|
||||
CREATE TABLE simple_key_direct_dictionary_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_int UInt64,
|
||||
value_string String,
|
||||
value_decimal Decimal64(8),
|
||||
value_string_nullable Nullable(String)
|
||||
) ENGINE = TinyLog;
|
||||
) ENGINE = Memory;
|
||||
</create_query>
|
||||
|
||||
<create_query>
|
||||
INSERT INTO simple_direct_dictionary_test_table
|
||||
SELECT number, number, toString(number), toDecimal64(number, 8), toString(number)
|
||||
FROM system.numbers
|
||||
LIMIT 100000;
|
||||
</create_query>
|
||||
|
||||
<create_query>
|
||||
CREATE DICTIONARY simple_direct_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value_int UInt64,
|
||||
value_string String,
|
||||
value_decimal Decimal64(8),
|
||||
value_string_nullable Nullable(String)
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_direct_dictionary_test_table'))
|
||||
LAYOUT(DIRECT())
|
||||
</create_query>
|
||||
|
||||
<create_query>
|
||||
CREATE TABLE complex_direct_dictionary_test_table
|
||||
CREATE TABLE complex_key_direct_dictionary_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -44,14 +23,21 @@
|
||||
</create_query>
|
||||
|
||||
<create_query>
|
||||
INSERT INTO complex_direct_dictionary_test_table
|
||||
SELECT number, toString(number), number, toString(number), toDecimal64(number, 8), toString(number)
|
||||
FROM system.numbers
|
||||
LIMIT 100000;
|
||||
CREATE DICTIONARY simple_key_direct_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value_int UInt64,
|
||||
value_string String,
|
||||
value_decimal Decimal64(8),
|
||||
value_string_nullable Nullable(String)
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_direct_dictionary_source_table'))
|
||||
LAYOUT(DIRECT())
|
||||
</create_query>
|
||||
|
||||
<create_query>
|
||||
CREATE DICTIONARY complex_direct_dictionary
|
||||
CREATE DICTIONARY complex_key_direct_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -61,20 +47,76 @@
|
||||
value_string_nullable Nullable(String)
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(DB 'default' TABLE 'complex_direct_dictionary_test_table'))
|
||||
SOURCE(CLICKHOUSE(DB 'default' TABLE 'complex_key_direct_dictionary_source_table'))
|
||||
LAYOUT(COMPLEX_KEY_DIRECT())
|
||||
</create_query>
|
||||
|
||||
<query>SELECT dictGet('default.simple_direct_dictionary', 'value_int', number) FROM system.numbers LIMIT 150000;</query>
|
||||
<query>SELECT dictGet('default.simple_direct_dictionary', 'value_string', number) FROM system.numbers LIMIT 150000;</query>
|
||||
<query>SELECT dictGet('default.simple_direct_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 150000;</query>
|
||||
<query>SELECT dictGet('default.simple_direct_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 150000;</query>
|
||||
<query>SELECT dictHas('default.simple_direct_dictionary', number) FROM system.numbers LIMIT 150000;</query>
|
||||
<fill_query>
|
||||
INSERT INTO simple_key_direct_dictionary_source_table
|
||||
SELECT number, number, toString(number), toDecimal64(number, 8), toString(number)
|
||||
FROM system.numbers
|
||||
LIMIT 100000;
|
||||
</fill_query>
|
||||
|
||||
<query>SELECT dictGet('default.complex_direct_dictionary', 'value_int', (number, toString(number))) FROM system.numbers LIMIT 150000;</query>
|
||||
<query>SELECT dictGet('default.complex_direct_dictionary', 'value_string', (number, toString(number))) FROM system.numbers LIMIT 150000;</query>
|
||||
<query>SELECT dictGet('default.complex_direct_dictionary', 'value_decimal', (number, toString(number))) FROM system.numbers LIMIT 150000;</query>
|
||||
<query>SELECT dictGet('default.complex_direct_dictionary', 'value_string_nullable', (number, toString(number))) FROM system.numbers LIMIT 150000;</query>
|
||||
<query>SELECT dictHas('default.complex_direct_dictionary', (number, toString(number))) FROM system.numbers LIMIT 150000;</query>
|
||||
<fill_query>
|
||||
INSERT INTO complex_key_direct_dictionary_source_table
|
||||
SELECT number, toString(number), number, toString(number), toDecimal64(number, 8), toString(number)
|
||||
FROM system.numbers
|
||||
LIMIT 100000;
|
||||
</fill_query>
|
||||
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>column_name</name>
|
||||
<values>
|
||||
<value>'value_int'</value>
|
||||
<value>'value_string'</value>
|
||||
<value>'value_decimal'</value>
|
||||
<value>'value_string_nullable'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
|
||||
<substitution>
|
||||
<name>elements_count</name>
|
||||
<values>
|
||||
<value>25000</value>
|
||||
<value>50000</value>
|
||||
<value>75000</value>
|
||||
<value>100000</value>
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<query>
|
||||
SELECT dictGet('default.simple_key_direct_dictionary', {column_name}, number)
|
||||
FROM system.numbers
|
||||
LIMIT {elements_count}
|
||||
FORMAT Null;
|
||||
</query>
|
||||
<query>
|
||||
SELECT dictHas('default.simple_key_direct_dictionary', number)
|
||||
FROM system.numbers
|
||||
LIMIT {elements_count}
|
||||
FORMAT Null;
|
||||
</query>
|
||||
|
||||
<query>
|
||||
SELECT dictGet('default.complex_key_direct_dictionary', {column_name}, (number, toString(number)))
|
||||
FROM system.numbers
|
||||
LIMIT {elements_count}
|
||||
FORMAT Null;
|
||||
</query>
|
||||
<query>
|
||||
SELECT dictHas('default.complex_key_direct_dictionary', (number, toString(number)))
|
||||
FROM system.numbers
|
||||
LIMIT {elements_count}
|
||||
FORMAT Null;
|
||||
</query>
|
||||
|
||||
<drop_query>DROP TABLE IF EXISTS simple_key_direct_dictionary_source_table;</drop_query>
|
||||
<drop_query>DROP TABLE IF EXISTS complex_key_direct_dictionary_source_table;</drop_query>
|
||||
|
||||
<drop_query>DROP DICTIONARY IF EXISTS simple_key_direct_dictionary;</drop_query>
|
||||
<drop_query>DROP DICTIONARY IF EXISTS complex_key_direct_dictionary;</drop_query>
|
||||
|
||||
</test>
|
||||
|
75
tests/performance/flat_dictionary.xml
Normal file
75
tests/performance/flat_dictionary.xml
Normal file
@ -0,0 +1,75 @@
|
||||
<test max_ignored_relative_change="0.3">
|
||||
<create_query>
|
||||
CREATE TABLE simple_key_flat_dictionary_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_int UInt64,
|
||||
value_string String,
|
||||
value_decimal Decimal64(8),
|
||||
value_string_nullable Nullable(String)
|
||||
) ENGINE = Memory;
|
||||
</create_query>
|
||||
|
||||
<create_query>
|
||||
CREATE DICTIONARY simple_key_flat_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value_int UInt64,
|
||||
value_string String,
|
||||
value_decimal Decimal64(8),
|
||||
value_string_nullable Nullable(String)
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_flat_dictionary_source_table'))
|
||||
LAYOUT(FLAT())
|
||||
LIFETIME(MIN 0 MAX 1000)
|
||||
</create_query>
|
||||
|
||||
<fill_query>
|
||||
INSERT INTO simple_key_flat_dictionary_source_table
|
||||
SELECT number, number, toString(number), toDecimal64(number, 8), toString(number)
|
||||
FROM system.numbers
|
||||
LIMIT 500000;
|
||||
</fill_query>
|
||||
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>column_name</name>
|
||||
<values>
|
||||
<value>'value_int'</value>
|
||||
<value>'value_string'</value>
|
||||
<value>'value_decimal'</value>
|
||||
<value>'value_string_nullable'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
|
||||
<substitution>
|
||||
<name>elements_count</name>
|
||||
<values>
|
||||
<value>250000</value>
|
||||
<value>500000</value>
|
||||
<value>750000</value>
|
||||
<value>1000000</value>
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<query>
|
||||
SELECT dictGet('default.simple_key_flat_dictionary', {column_name}, number)
|
||||
FROM system.numbers
|
||||
LIMIT {elements_count}
|
||||
FORMAR Null;
|
||||
</query>
|
||||
|
||||
<query>
|
||||
SELECT dictHas('default.simple_key_flat_dictionary', number)
|
||||
FROM system.numbers
|
||||
LIMIT {elements_count}
|
||||
FORMAT Null;
|
||||
</query>
|
||||
|
||||
<drop_query>DROP TABLE IF EXISTS simple_key_flat_dictionary_source_table</drop_query>
|
||||
|
||||
<drop_query>DROP DICTIONARY IF EXISTS simple_key_flat_dictionary</drop_query>
|
||||
|
||||
</test>
|
124
tests/performance/hashed_dictionary.xml
Normal file
124
tests/performance/hashed_dictionary.xml
Normal file
@ -0,0 +1,124 @@
|
||||
<test max_ignored_relative_change="0.3">
|
||||
<create_query>
|
||||
CREATE TABLE simple_key_hashed_dictionary_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_int UInt64,
|
||||
value_string String,
|
||||
value_decimal Decimal64(8),
|
||||
value_string_nullable Nullable(String)
|
||||
) ENGINE = Memory;
|
||||
</create_query>
|
||||
|
||||
<create_query>
|
||||
CREATE TABLE complex_key_hashed_dictionary_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
value_int UInt64,
|
||||
value_string String,
|
||||
value_decimal Decimal64(8),
|
||||
value_string_nullable Nullable(String)
|
||||
) ENGINE = Memory;
|
||||
</create_query>
|
||||
|
||||
<create_query>
|
||||
CREATE DICTIONARY simple_key_hashed_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value_int UInt64,
|
||||
value_string String,
|
||||
value_decimal Decimal64(8),
|
||||
value_string_nullable Nullable(String)
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_hashed_dictionary_source_table'))
|
||||
LAYOUT(HASHED())
|
||||
LIFETIME(MIN 0 MAX 1000);
|
||||
</create_query>
|
||||
|
||||
<create_query>
|
||||
CREATE DICTIONARY complex_key_hashed_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
value_int UInt64,
|
||||
value_string String,
|
||||
value_decimal Decimal64(8),
|
||||
value_string_nullable Nullable(String)
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(DB 'default' TABLE 'complex_key_hashed_dictionary_source_table'))
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
LIFETIME(MIN 0 MAX 1000);
|
||||
</create_query>
|
||||
|
||||
<fill_query>
|
||||
INSERT INTO simple_key_hashed_dictionary_source_table
|
||||
SELECT number, number, toString(number), toDecimal64(number, 8), toString(number)
|
||||
FROM system.numbers
|
||||
LIMIT 5000000;
|
||||
</fill_query>
|
||||
|
||||
<fill_query>
|
||||
INSERT INTO complex_key_hashed_dictionary_source_table
|
||||
SELECT number, toString(number), number, toString(number), toDecimal64(number, 8), toString(number)
|
||||
FROM system.numbers
|
||||
LIMIT 5000000;
|
||||
</fill_query>
|
||||
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>column_name</name>
|
||||
<values>
|
||||
<value>'value_int'</value>
|
||||
<value>'value_string'</value>
|
||||
<value>'value_decimal'</value>
|
||||
<value>'value_string_nullable'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
|
||||
<substitution>
|
||||
<name>elements_count</name>
|
||||
<values>
|
||||
<value>2500000</value>
|
||||
<value>5000000</value>
|
||||
<value>7500000</value>
|
||||
<value>10000000</value>
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<query>
|
||||
SELECT dictGet('default.simple_key_hashed_dictionary', {column_name}, number)
|
||||
FROM system.numbers
|
||||
LIMIT {elements_count}
|
||||
FORMAT Null;
|
||||
</query>
|
||||
<query>
|
||||
SELECT dictHas('default.simple_key_hashed_dictionary', number)
|
||||
FROM system.numbers
|
||||
LIMIT {elements_count}
|
||||
FORMAT Null;
|
||||
</query>
|
||||
|
||||
<query>
|
||||
SELECT dictGet('default.complex_key_hashed_dictionary', {column_name}, (number, toString(number)))
|
||||
FROM system.numbers
|
||||
LIMIT {elements_count}
|
||||
FORMAT Null;
|
||||
</query>
|
||||
<query>
|
||||
SELECT dictHas('default.complex_key_hashed_dictionary', (number, toString(number)))
|
||||
FROM system.numbers
|
||||
LIMIT {elements_count}
|
||||
FORMAT Null;
|
||||
</query>
|
||||
|
||||
<drop_query>DROP TABLE IF EXISTS simple_key_hashed_dictionary_source_table;</drop_query>
|
||||
<drop_query>DROP TABLE IF EXISTS complex_key_hashed_dictionary_source_table;</drop_query>
|
||||
|
||||
<drop_query>DROP DICTIONARY IF EXISTS simple_key_hashed_dictionary;</drop_query>
|
||||
<drop_query>DROP DICTIONARY IF EXISTS complex_key_hashed_dictionary;</drop_query>
|
||||
|
||||
</test>
|
@ -29,10 +29,10 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
255
|
||||
255
|
||||
0
|
||||
255
|
||||
0
|
||||
0
|
||||
0
|
||||
[11,22]
|
||||
[22,11]
|
||||
[11,22]
|
||||
|
@ -0,0 +1,132 @@
|
||||
Dictionary hashed_dictionary_simple_key_simple_attributes
|
||||
dictGet existing value
|
||||
value_0 value_second_0
|
||||
value_1 value_second_1
|
||||
value_2 value_second_2
|
||||
dictGet with non existing value
|
||||
value_0 value_second_0
|
||||
value_1 value_second_1
|
||||
value_2 value_second_2
|
||||
value_first_default value_second_default
|
||||
dictGetOrDefault existing value
|
||||
value_0 value_second_0
|
||||
value_1 value_second_1
|
||||
value_2 value_second_2
|
||||
dictGetOrDefault non existing value
|
||||
value_0 value_second_0
|
||||
value_1 value_second_1
|
||||
value_2 value_second_2
|
||||
default default
|
||||
dictHas
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
select all values as input stream
|
||||
0 value_0 value_second_0
|
||||
1 value_1 value_second_1
|
||||
2 value_2 value_second_2
|
||||
Dictionary sparse_hashed_dictionary_simple_key_simple_attributes
|
||||
dictGet existing value
|
||||
value_0 value_second_0
|
||||
value_1 value_second_1
|
||||
value_2 value_second_2
|
||||
dictGet with non existing value
|
||||
value_0 value_second_0
|
||||
value_1 value_second_1
|
||||
value_2 value_second_2
|
||||
value_first_default value_second_default
|
||||
dictGetOrDefault existing value
|
||||
value_0 value_second_0
|
||||
value_1 value_second_1
|
||||
value_2 value_second_2
|
||||
dictGetOrDefault non existing value
|
||||
value_0 value_second_0
|
||||
value_1 value_second_1
|
||||
value_2 value_second_2
|
||||
default default
|
||||
dictHas
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
select all values as input stream
|
||||
0 value_0 value_second_0
|
||||
1 value_1 value_second_1
|
||||
2 value_2 value_second_2
|
||||
Dictionary hashed_dictionary_simple_key_complex_attributes
|
||||
dictGet existing value
|
||||
value_0 value_second_0
|
||||
value_1 \N
|
||||
value_2 value_second_2
|
||||
dictGet with non existing value
|
||||
value_0 value_second_0
|
||||
value_1 \N
|
||||
value_2 value_second_2
|
||||
value_first_default value_second_default
|
||||
dictGetOrDefault existing value
|
||||
value_0 value_second_0
|
||||
value_1 \N
|
||||
value_2 value_second_2
|
||||
dictGetOrDefault non existing value
|
||||
value_0 value_second_0
|
||||
value_1 \N
|
||||
value_2 value_second_2
|
||||
default default
|
||||
dictHas
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
select all values as input stream
|
||||
0 value_0 value_second_0
|
||||
1 value_1 \N
|
||||
2 value_2 value_second_2
|
||||
Dictionary sparse_hashed_dictionary_simple_key_complex_attributes
|
||||
dictGet existing value
|
||||
value_0 value_second_0
|
||||
value_1 \N
|
||||
value_2 value_second_2
|
||||
dictGet with non existing value
|
||||
value_0 value_second_0
|
||||
value_1 \N
|
||||
value_2 value_second_2
|
||||
value_first_default value_second_default
|
||||
dictGetOrDefault existing value
|
||||
value_0 value_second_0
|
||||
value_1 \N
|
||||
value_2 value_second_2
|
||||
dictGetOrDefault non existing value
|
||||
value_0 value_second_0
|
||||
value_1 \N
|
||||
value_2 value_second_2
|
||||
default default
|
||||
dictHas
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
select all values as input stream
|
||||
0 value_0 value_second_0
|
||||
1 value_1 \N
|
||||
2 value_2 value_second_2
|
||||
Dictionary hashed_dictionary_simple_key_hierarchy
|
||||
dictGet
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
2
|
||||
dictGetHierarchy
|
||||
[1]
|
||||
[4,2,1]
|
||||
Dictionary sparse_hashed_dictionary_simple_key_hierarchy
|
||||
dictGet
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
2
|
||||
dictGetHierarchy
|
||||
[1]
|
||||
[4,2,1]
|
207
tests/queries/0_stateless/01765_hashed_dictionary_simple_key.sql
Normal file
207
tests/queries/0_stateless/01765_hashed_dictionary_simple_key.sql
Normal file
@ -0,0 +1,207 @@
|
||||
DROP DATABASE IF EXISTS 01765_db;
|
||||
CREATE DATABASE 01765_db;
|
||||
|
||||
CREATE TABLE 01765_db.simple_key_simple_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_first String,
|
||||
value_second String
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01765_db.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO 01765_db.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
|
||||
INSERT INTO 01765_db.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01765_db.hashed_dictionary_simple_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
value_second String DEFAULT 'value_second_default'
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_simple_attributes_source_table'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(HASHED());
|
||||
|
||||
SELECT 'Dictionary hashed_dictionary_simple_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01765_db.hashed_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01765_db.hashed_dictionary_simple_key_simple_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01765_db.hashed_dictionary_simple_key_simple_attributes;
|
||||
|
||||
CREATE DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
value_second String DEFAULT 'value_second_default'
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_simple_attributes_source_table'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(SPARSE_HASHED());
|
||||
|
||||
SELECT 'Dictionary sparse_hashed_dictionary_simple_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01765_db.sparse_hashed_dictionary_simple_key_simple_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_simple_attributes;
|
||||
|
||||
DROP TABLE 01765_db.simple_key_simple_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01765_db.simple_key_complex_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_first String,
|
||||
value_second Nullable(String)
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01765_db.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO 01765_db.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
|
||||
INSERT INTO 01765_db.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01765_db.hashed_dictionary_simple_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
value_second Nullable(String) DEFAULT 'value_second_default'
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_complex_attributes_source_table'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(HASHED());
|
||||
|
||||
SELECT 'Dictionary hashed_dictionary_simple_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01765_db.hashed_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01765_db.hashed_dictionary_simple_key_complex_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01765_db.hashed_dictionary_simple_key_complex_attributes;
|
||||
|
||||
CREATE DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
value_second Nullable(String) DEFAULT 'value_second_default'
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_complex_attributes_source_table'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(HASHED());
|
||||
|
||||
SELECT 'Dictionary sparse_hashed_dictionary_simple_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01765_db.sparse_hashed_dictionary_simple_key_complex_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_complex_attributes;
|
||||
|
||||
DROP TABLE 01765_db.simple_key_complex_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01765_db.simple_key_hierarchy_table
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64
|
||||
) ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO 01765_db.simple_key_hierarchy_table VALUES (1, 0);
|
||||
INSERT INTO 01765_db.simple_key_hierarchy_table VALUES (2, 1);
|
||||
INSERT INTO 01765_db.simple_key_hierarchy_table VALUES (3, 1);
|
||||
INSERT INTO 01765_db.simple_key_hierarchy_table VALUES (4, 2);
|
||||
|
||||
CREATE DICTIONARY 01765_db.hashed_dictionary_simple_key_hierarchy
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_table'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(HASHED());
|
||||
|
||||
SELECT 'Dictionary hashed_dictionary_simple_key_hierarchy';
|
||||
SELECT 'dictGet';
|
||||
SELECT dictGet('01765_db.hashed_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT 'dictGetHierarchy';
|
||||
SELECT dictGetHierarchy('01765_db.hashed_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('01765_db.hashed_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
|
||||
DROP DICTIONARY 01765_db.hashed_dictionary_simple_key_hierarchy;
|
||||
|
||||
CREATE DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_hierarchy
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_table'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(HASHED());
|
||||
|
||||
SELECT 'Dictionary sparse_hashed_dictionary_simple_key_hierarchy';
|
||||
SELECT 'dictGet';
|
||||
SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT 'dictGetHierarchy';
|
||||
SELECT dictGetHierarchy('01765_db.sparse_hashed_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('01765_db.sparse_hashed_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
|
||||
DROP DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_hierarchy;
|
||||
|
||||
DROP TABLE 01765_db.simple_key_hierarchy_table;
|
||||
|
||||
DROP DATABASE 01765_db;
|
@ -0,0 +1,56 @@
|
||||
Dictionary hashed_dictionary_complex_key_simple_attributes
|
||||
dictGet existing value
|
||||
value_0 value_second_0
|
||||
value_1 value_second_1
|
||||
value_2 value_second_2
|
||||
dictGet with non existing value
|
||||
value_0 value_second_0
|
||||
value_1 value_second_1
|
||||
value_2 value_second_2
|
||||
value_first_default value_second_default
|
||||
dictGetOrDefault existing value
|
||||
value_0 value_second_0
|
||||
value_1 value_second_1
|
||||
value_2 value_second_2
|
||||
dictGetOrDefault non existing value
|
||||
value_0 value_second_0
|
||||
value_1 value_second_1
|
||||
value_2 value_second_2
|
||||
default default
|
||||
dictHas
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
select all values as input stream
|
||||
0 id_key_0 value_0 value_second_0
|
||||
1 id_key_1 value_1 value_second_1
|
||||
2 id_key_2 value_2 value_second_2
|
||||
Dictionary hashed_dictionary_complex_key_complex_attributes
|
||||
dictGet existing value
|
||||
value_0 value_second_0
|
||||
value_1 \N
|
||||
value_2 value_second_2
|
||||
dictGet with non existing value
|
||||
value_0 value_second_0
|
||||
value_1 \N
|
||||
value_2 value_second_2
|
||||
value_first_default value_second_default
|
||||
dictGetOrDefault existing value
|
||||
value_0 value_second_0
|
||||
value_1 \N
|
||||
value_2 value_second_2
|
||||
dictGetOrDefault non existing value
|
||||
value_0 value_second_0
|
||||
value_1 \N
|
||||
value_2 value_second_2
|
||||
default default
|
||||
dictHas
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
select all values as input stream
|
||||
0 id_key_0 value_0 value_second_0
|
||||
1 id_key_1 value_1 \N
|
||||
2 id_key_2 value_2 value_second_2
|
@ -0,0 +1,98 @@
|
||||
DROP DATABASE IF EXISTS 01766_db;
|
||||
CREATE DATABASE 01766_db;
|
||||
|
||||
CREATE TABLE 01766_db.complex_key_simple_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
value_first String,
|
||||
value_second String
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01766_db.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO 01766_db.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1');
|
||||
INSERT INTO 01766_db.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01766_db.hashed_dictionary_complex_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
value_second String DEFAULT 'value_second_default'
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_simple_attributes_source_table' DB '01766_db'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(COMPLEX_KEY_HASHED());
|
||||
|
||||
SELECT 'Dictionary hashed_dictionary_complex_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01766_db.hashed_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01766_db.hashed_dictionary_complex_key_simple_attributes ORDER BY (id, id_key);
|
||||
|
||||
DROP DICTIONARY 01766_db.hashed_dictionary_complex_key_simple_attributes;
|
||||
|
||||
DROP TABLE 01766_db.complex_key_simple_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01766_db.complex_key_complex_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
value_first String,
|
||||
value_second Nullable(String)
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01766_db.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO 01766_db.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL);
|
||||
INSERT INTO 01766_db.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01766_db.hashed_dictionary_complex_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
value_second Nullable(String) DEFAULT 'value_second_default'
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_complex_attributes_source_table' DB '01766_db'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(COMPLEX_KEY_HASHED());
|
||||
|
||||
SELECT 'Dictionary hashed_dictionary_complex_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01766_db.hashed_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01766_db.hashed_dictionary_complex_key_complex_attributes ORDER BY (id, id_key);
|
||||
|
||||
DROP DICTIONARY 01766_db.hashed_dictionary_complex_key_complex_attributes;
|
||||
DROP TABLE 01766_db.complex_key_complex_attributes_source_table;
|
||||
|
||||
DROP DATABASE 01766_db;
|
@ -0,0 +1,102 @@
|
||||
Flat dictionary
|
||||
Get hierarchy
|
||||
[]
|
||||
[1]
|
||||
[2,1]
|
||||
[3,1]
|
||||
[4,2,1]
|
||||
[]
|
||||
Get is in hierarchy
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
Get children
|
||||
[1]
|
||||
[2,3]
|
||||
[4]
|
||||
[]
|
||||
[]
|
||||
[]
|
||||
Get all descendants
|
||||
[1,2,3,4]
|
||||
[2,3,4]
|
||||
[4]
|
||||
[]
|
||||
[]
|
||||
[]
|
||||
Get descendants at first level
|
||||
[1]
|
||||
[2,3]
|
||||
[4]
|
||||
[]
|
||||
[]
|
||||
[]
|
||||
Hashed dictionary
|
||||
Get hierarchy
|
||||
[]
|
||||
[1]
|
||||
[2,1]
|
||||
[3,1]
|
||||
[4,2,1]
|
||||
[]
|
||||
Get is in hierarchy
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
Get children
|
||||
[1]
|
||||
[3,2]
|
||||
[4]
|
||||
[]
|
||||
[]
|
||||
[]
|
||||
Get all descendants
|
||||
[1,3,2,4]
|
||||
[3,2,4]
|
||||
[4]
|
||||
[]
|
||||
[]
|
||||
[]
|
||||
Get descendants at first level
|
||||
[1]
|
||||
[3,2]
|
||||
[4]
|
||||
[]
|
||||
[]
|
||||
[]
|
||||
Cache dictionary
|
||||
Get hierarchy
|
||||
[]
|
||||
[1]
|
||||
[2,1]
|
||||
[3,1]
|
||||
[4,2,1]
|
||||
[]
|
||||
Get is in hierarchy
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
Direct dictionary
|
||||
Get hierarchy
|
||||
[]
|
||||
[1]
|
||||
[2,1]
|
||||
[3,1]
|
||||
[4,2,1]
|
||||
[]
|
||||
Get is in hierarchy
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
@ -0,0 +1,95 @@
|
||||
DROP DATABASE IF EXISTS 01778_db;
|
||||
CREATE DATABASE 01778_db;
|
||||
|
||||
CREATE TABLE 01778_db.hierarchy_source_table (id UInt64, parent_id UInt64) ENGINE = TinyLog;
|
||||
INSERT INTO 01778_db.hierarchy_source_table VALUES (1, 0), (2, 1), (3, 1), (4, 2);
|
||||
|
||||
CREATE DICTIONARY 01778_db.hierarchy_flat_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '01778_db'))
|
||||
LAYOUT(FLAT())
|
||||
LIFETIME(MIN 1 MAX 1000);
|
||||
|
||||
SELECT 'Flat dictionary';
|
||||
|
||||
SELECT 'Get hierarchy';
|
||||
SELECT dictGetHierarchy('01778_db.hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get is in hierarchy';
|
||||
SELECT dictIsIn('01778_db.hierarchy_flat_dictionary', number, number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get children';
|
||||
SELECT dictGetChildren('01778_db.hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get all descendants';
|
||||
SELECT dictGetDescendants('01778_db.hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get descendants at first level';
|
||||
SELECT dictGetDescendants('01778_db.hierarchy_flat_dictionary', number, 1) FROM system.numbers LIMIT 6;
|
||||
|
||||
DROP DICTIONARY 01778_db.hierarchy_flat_dictionary;
|
||||
|
||||
CREATE DICTIONARY 01778_db.hierarchy_hashed_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '01778_db'))
|
||||
LAYOUT(HASHED())
|
||||
LIFETIME(MIN 1 MAX 1000);
|
||||
|
||||
SELECT 'Hashed dictionary';
|
||||
|
||||
SELECT 'Get hierarchy';
|
||||
SELECT dictGetHierarchy('01778_db.hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get is in hierarchy';
|
||||
SELECT dictIsIn('01778_db.hierarchy_hashed_dictionary', number, number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get children';
|
||||
SELECT dictGetChildren('01778_db.hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get all descendants';
|
||||
SELECT dictGetDescendants('01778_db.hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get descendants at first level';
|
||||
SELECT dictGetDescendants('01778_db.hierarchy_hashed_dictionary', number, 1) FROM system.numbers LIMIT 6;
|
||||
|
||||
DROP DICTIONARY 01778_db.hierarchy_hashed_dictionary;
|
||||
|
||||
CREATE DICTIONARY 01778_db.hierarchy_cache_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '01778_db'))
|
||||
LAYOUT(CACHE(SIZE_IN_CELLS 10))
|
||||
LIFETIME(MIN 1 MAX 1000);
|
||||
|
||||
SELECT 'Cache dictionary';
|
||||
|
||||
SELECT 'Get hierarchy';
|
||||
SELECT dictGetHierarchy('01778_db.hierarchy_cache_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get is in hierarchy';
|
||||
SELECT dictIsIn('01778_db.hierarchy_cache_dictionary', number, number) FROM system.numbers LIMIT 6;
|
||||
|
||||
DROP DICTIONARY 01778_db.hierarchy_cache_dictionary;
|
||||
|
||||
CREATE DICTIONARY 01778_db.hierarchy_direct_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '01778_db'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT 'Direct dictionary';
|
||||
|
||||
SELECT 'Get hierarchy';
|
||||
SELECT dictGetHierarchy('01778_db.hierarchy_direct_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get is in hierarchy';
|
||||
SELECT dictIsIn('01778_db.hierarchy_direct_dictionary', number, number) FROM system.numbers LIMIT 6;
|
||||
|
||||
DROP DICTIONARY 01778_db.hierarchy_direct_dictionary;
|
||||
|
||||
DROP TABLE 01778_db.hierarchy_source_table;
|
||||
DROP DATABASE 01778_db;
|
@ -679,6 +679,19 @@
|
||||
"live_view",
|
||||
"memory_leak",
|
||||
"memory_limit",
|
||||
"polygon_dicts" // they use an explicitly specified database
|
||||
"polygon_dicts", // they use an explicitly specified database
|
||||
"01658_read_file_to_stringcolumn",
|
||||
"01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test.
|
||||
"01702_system_query_log", // It's ok to execute in parallel with oter tests but not several instances of the same test.
|
||||
"01748_dictionary_table_dot", // creates database
|
||||
"00950_dict_get",
|
||||
"01683_flat_dictionary",
|
||||
"01681_cache_dictionary_simple_key",
|
||||
"01682_cache_dictionary_complex_key",
|
||||
"01684_ssd_cache_dictionary_simple_key",
|
||||
"01685_ssd_cache_dictionary_complex_key",
|
||||
"01760_system_dictionaries",
|
||||
"01760_polygon_dictionaries",
|
||||
"01778_hierarchical_dictionaries"
|
||||
]
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user