mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Refactored hierarchy dictionaries interface
This commit is contained in:
parent
ac53d30e53
commit
9f2f0d1095
@ -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,
|
||||
@ -526,6 +367,32 @@ 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, const DataTypePtr & key_type) const
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
{
|
||||
auto result = getHierarchyDefaultImplementation(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, ColumnPtr in_key_column, const DataTypePtr & key_type) const
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
{
|
||||
auto result = isInHierarchyDefaultImplementation(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 +485,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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -130,23 +130,14 @@ 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>;
|
||||
@ -171,8 +162,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 +182,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 +204,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{}};
|
||||
|
@ -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});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -370,9 +370,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())
|
||||
|
@ -152,6 +152,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>
|
||||
@ -179,7 +54,7 @@ ColumnPtr DirectDictionary<dictionary_key_type>::getColumn(
|
||||
|
||||
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();
|
||||
@ -310,6 +185,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 = getHierarchyDefaultImplementation(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 = isInHierarchyDefaultImplementation(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 +248,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,90 @@ 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 is_in_hierarchy_result = isInKeysHierarchy(keys, keys_in, null_value, is_key_valid_func, get_parent_key_func);
|
||||
|
||||
auto result = ColumnUInt8::create();
|
||||
result->getData() = std::move(is_in_hierarchy_result);
|
||||
|
||||
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 +276,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 +285,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 +390,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 +412,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 +429,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 +444,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 +452,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 +488,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 +503,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,22 @@ 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:
|
||||
template <typename Value>
|
||||
using ContainerType = PaddedPODArray<Value>;
|
||||
|
||||
using NullableSet = HashSet<Key, DefaultHash<Key>>;
|
||||
using NullableSet = HashSet<UInt64, DefaultHash<UInt64>>;
|
||||
|
||||
struct Attribute final
|
||||
{
|
||||
@ -151,24 +151,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 +177,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 +184,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,22 @@
|
||||
#include <memory>
|
||||
#include <variant>
|
||||
#include <optional>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Core/Block.h>
|
||||
|
||||
#include <sparsehash/sparse_hash_map>
|
||||
#include <absl/container/flat_hash_map.h>
|
||||
#include <ext/range.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 <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 +29,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 +68,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 +79,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,88 +93,57 @@ 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 hierarchy_attribute_column, const DataTypePtr & hierarchy_attribute_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:
|
||||
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, absl::flat_hash_map<UInt64, Value>, absl::flat_hash_map<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>>;
|
||||
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>>;
|
||||
|
||||
struct Attribute final
|
||||
{
|
||||
AttributeUnderlyingType type;
|
||||
std::optional<NullableSet> nullable_set;
|
||||
|
||||
bool is_complex_type;
|
||||
std::variant<
|
||||
UInt8,
|
||||
UInt16,
|
||||
UInt32,
|
||||
UInt64,
|
||||
UInt128,
|
||||
Int8,
|
||||
Int16,
|
||||
Int32,
|
||||
Int64,
|
||||
Decimal32,
|
||||
Decimal64,
|
||||
Decimal128,
|
||||
Float32,
|
||||
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>,
|
||||
CollectionType<Field>>
|
||||
container;
|
||||
std::unique_ptr<Arena> string_arena;
|
||||
};
|
||||
|
||||
@ -172,76 +155,39 @@ private:
|
||||
|
||||
void loadData();
|
||||
|
||||
template <typename T>
|
||||
void addAttributeSize(const Attribute & attribute);
|
||||
|
||||
void calculateBytesAllocated();
|
||||
|
||||
template <typename T>
|
||||
void createAttributeImpl(Attribute & attribute, const Field & null_value);
|
||||
template <typename GetContainerFunc>
|
||||
void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func);
|
||||
|
||||
Attribute createAttribute(const DictionaryAttribute& attribute, const Field & null_value);
|
||||
template <typename GetContainerFunc>
|
||||
void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func) const;
|
||||
|
||||
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>
|
||||
void getItemsImpl(
|
||||
const Attribute & attribute,
|
||||
const PaddedPODArray<Key> & ids,
|
||||
ValueSetter && set_value,
|
||||
DefaultValueExtractor & default_value_extractor) const;
|
||||
|
||||
template <typename T>
|
||||
bool setAttributeValueImpl(Attribute & attribute, const Key id, const T value);
|
||||
|
||||
bool setAttributeValue(Attribute & attribute, const Key id, const Field & value);
|
||||
|
||||
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>;
|
||||
|
||||
}
|
||||
|
150
src/Dictionaries/HierarchyDictionariesUtils.cpp
Normal file
150
src/Dictionaries/HierarchyDictionariesUtils.cpp
Normal file
@ -0,0 +1,150 @@
|
||||
#include "HierarchyDictionariesUtils.h"
|
||||
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
HashMap<UInt64, UInt64> getHierarchyMapImpl(const IDictionary * dictionary, const DictionaryAttribute & dictionary_attribute, const PaddedPODArray<UInt64> & initial_keys_to_request, const DataTypePtr & key_type)
|
||||
{
|
||||
UInt64 null_value = dictionary_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> key_to_parent_key;
|
||||
|
||||
while (!keys_to_request.empty())
|
||||
{
|
||||
key_to_parent_key.reserve(key_to_parent_key.size() + keys_to_request.size());
|
||||
|
||||
auto parent_key_column
|
||||
= dictionary->getColumn(dictionary_attribute.name, dictionary_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");
|
||||
|
||||
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;
|
||||
|
||||
key_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 key_to_parent_key;
|
||||
}
|
||||
}
|
||||
|
||||
ColumnPtr getHierarchyDefaultImplementation(const IDictionary * dictionary, ColumnPtr key_column, const DataTypePtr & key_type)
|
||||
{
|
||||
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();
|
||||
const auto & dictionary_attribute = dictionary_structure.attributes[0];
|
||||
|
||||
const PaddedPODArray<UInt64> & requested_keys = key_column_typed->getData();
|
||||
HashMap<UInt64, UInt64> key_to_parent_key = getHierarchyMapImpl(dictionary, dictionary_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)
|
||||
{
|
||||
std::optional<UInt64> result;
|
||||
|
||||
auto it = key_to_parent_key.find(key);
|
||||
|
||||
if (it != nullptr)
|
||||
result = it->getMapped();
|
||||
|
||||
return result;
|
||||
};
|
||||
|
||||
UInt64 null_value = dictionary_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 isInHierarchyDefaultImplementation(
|
||||
const IDictionary * dictionary,
|
||||
ColumnPtr key_column,
|
||||
ColumnPtr in_key_column,
|
||||
const DataTypePtr & key_type)
|
||||
{
|
||||
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();
|
||||
const auto & dictionary_attribute = dictionary_structure.attributes[0];
|
||||
|
||||
const PaddedPODArray<UInt64> & requested_keys = key_column_typed->getData();
|
||||
HashMap<UInt64, UInt64> key_to_parent_key = getHierarchyMapImpl(dictionary, dictionary_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)
|
||||
{
|
||||
std::optional<UInt64> result;
|
||||
|
||||
auto it = key_to_parent_key.find(key);
|
||||
|
||||
if (it != nullptr)
|
||||
result = it->getMapped();
|
||||
|
||||
return result;
|
||||
};
|
||||
|
||||
UInt64 null_value = dictionary_attribute.null_value.get<UInt64>();
|
||||
const auto & in_keys = in_key_column_typed->getData();
|
||||
|
||||
auto is_in_hierarchy_result = isInKeysHierarchy(requested_keys, in_keys, null_value, is_key_valid_func, get_parent_key_func);
|
||||
|
||||
auto result = ColumnUInt8::create();
|
||||
result->getData() = std::move(is_in_hierarchy_result);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
197
src/Dictionaries/HierarchyDictionariesUtils.h
Normal file
197
src/Dictionaries/HierarchyDictionariesUtils.h
Normal file
@ -0,0 +1,197 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/types.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
|
||||
#include <Dictionaries/IDictionary.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
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 {}; }
|
||||
};
|
||||
|
||||
template <typename KeyType, typename IsKeyValidFunc, typename GetParentKeyFunc>
|
||||
ElementsAndOffsets<KeyType> getKeysHierarchy(
|
||||
const PaddedPODArray<KeyType> & hierarchy_keys,
|
||||
const KeyType & hierarchy_null_value,
|
||||
IsKeyValidFunc && is_key_valid_func,
|
||||
GetParentKeyFunc && get_parent_func)
|
||||
{
|
||||
size_t hierarchy_keys_size = hierarchy_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 = hierarchy_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];
|
||||
|
||||
current_hierarchy_depth += end_index - start_index;
|
||||
|
||||
/// TODO: Insert part of pod array into itself
|
||||
while (start_index < end_index)
|
||||
{
|
||||
elements.emplace_back(elements[start_index]);
|
||||
++start_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_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;
|
||||
}
|
||||
|
||||
template <typename KeyType, typename IsKeyValidFunc, typename GetParentKeyFunc>
|
||||
ColumnPtr getKeysHierarchyArray(
|
||||
const PaddedPODArray<KeyType> & hierarchy_keys,
|
||||
const KeyType & hierarchy_null_value,
|
||||
IsKeyValidFunc && is_key_valid_func,
|
||||
GetParentKeyFunc && get_parent_func)
|
||||
{
|
||||
auto elements_and_offsets = getKeysHierarchy(hierarchy_keys, hierarchy_null_value, std::forward<IsKeyValidFunc>(is_key_valid_func), std::forward<GetParentKeyFunc>(get_parent_func));
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
template <typename KeyType, typename IsKeyValidFunc, typename GetParentKeyFunc>
|
||||
PaddedPODArray<UInt8> isInKeysHierarchy(
|
||||
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)
|
||||
{
|
||||
assert(hierarchy_keys.size() == hierarchy_in_keys.size());
|
||||
|
||||
PaddedPODArray<UInt8> result;
|
||||
result.resize_fill(hierarchy_keys.size());
|
||||
|
||||
ElementsAndOffsets<KeyType> hierarchy = getKeysHierarchy(hierarchy_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 = hierarchy_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;
|
||||
}
|
||||
|
||||
ColumnPtr getHierarchyDefaultImplementation(const IDictionary * dictionary, ColumnPtr key_column, const DataTypePtr & key_type);
|
||||
|
||||
ColumnUInt8::Ptr isInHierarchyDefaultImplementation(
|
||||
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,55 @@ 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,
|
||||
"Hierarchy 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,
|
||||
"Hierarchy 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];
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -248,7 +248,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_)
|
||||
@ -857,9 +857,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)
|
||||
@ -880,12 +877,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);
|
||||
|
@ -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
|
||||
|
@ -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,7 +79,7 @@ public:
|
||||
return dict;
|
||||
}
|
||||
|
||||
std::shared_ptr<const IDictionaryBase> getDictionary(const ColumnWithTypeAndName & column)
|
||||
std::shared_ptr<const IDictionary> getDictionary(const ColumnWithTypeAndName & column)
|
||||
{
|
||||
const auto * dict_name_col = checkAndGetColumnConst<ColumnString>(column.column.get());
|
||||
return getDictionary(dict_name_col->getValue<String>());
|
||||
@ -744,109 +733,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]);
|
||||
|
||||
/// 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;
|
||||
@ -900,105 +795,14 @@ private:
|
||||
|
||||
auto dict = helper.getDictionary(arguments[0]);
|
||||
|
||||
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
|
||||
{
|
||||
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};
|
||||
}
|
||||
|
||||
template <typename DictionaryType>
|
||||
ColumnPtr execute(const DictionaryType * dict,
|
||||
const ColumnUInt64 * child_id_col, const IColumn * ancestor_id_col_untyped) const
|
||||
{
|
||||
if (const auto * ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped))
|
||||
{
|
||||
auto out = ColumnUInt8::create();
|
||||
|
||||
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);
|
||||
|
||||
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};
|
||||
}
|
||||
}
|
||||
|
||||
template <typename DictionaryType>
|
||||
ColumnPtr execute(const DictionaryType * dict, const ColumnConst * child_id_col, const IColumn * ancestor_id_col_untyped) const
|
||||
{
|
||||
if (const auto * ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped))
|
||||
{
|
||||
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;
|
||||
}
|
||||
else if (const auto * ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
|
||||
{
|
||||
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);
|
||||
}
|
||||
else
|
||||
throw Exception{"Illegal column " + ancestor_id_col_untyped->getName()
|
||||
+ " of third argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
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();
|
||||
|
@ -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;
|
Loading…
Reference in New Issue
Block a user