Merge pull request #27795 from kitaisreal/dictionaries-key-types-refactoring

Dictionaries key types refactoring
This commit is contained in:
Maksim Kita 2021-08-18 10:30:21 +03:00 committed by GitHub
commit 419cecdeaf
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 233 additions and 308 deletions

View File

@ -151,7 +151,7 @@ Columns CacheDictionary<dictionary_key_type>::getColumns(
* use default value. * use default value.
*/ */
if (dictionary_key_type == DictionaryKeyType::complex) if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types); dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder; DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -268,7 +268,7 @@ ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::hasKeys(const Columns & k
* Check that key was fetched during update for that key set true in result array. * Check that key was fetched during update for that key set true in result array.
*/ */
if (dictionary_key_type == DictionaryKeyType::complex) if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types); dict_struct.validateKeyTypes(key_types);
@ -364,7 +364,7 @@ ColumnPtr CacheDictionary<dictionary_key_type>::getHierarchy(
ColumnPtr key_column [[maybe_unused]], ColumnPtr key_column [[maybe_unused]],
const DataTypePtr & key_type [[maybe_unused]]) const const DataTypePtr & key_type [[maybe_unused]]) const
{ {
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
size_t keys_found; size_t keys_found;
auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found); auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found);
@ -382,7 +382,7 @@ ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::isInHierarchy(
ColumnPtr in_key_column [[maybe_unused]], ColumnPtr in_key_column [[maybe_unused]],
const DataTypePtr & key_type [[maybe_unused]]) const const DataTypePtr & key_type [[maybe_unused]]) const
{ {
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
size_t keys_found; size_t keys_found;
auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found); auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found);
@ -492,7 +492,7 @@ Pipe CacheDictionary<dictionary_key_type>::read(const Names & column_names, size
/// Write lock on storage /// Write lock on storage
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
data.emplace(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names); data.emplace(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names);
else else
{ {
@ -534,7 +534,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
std::vector<UInt64> requested_keys_vector; std::vector<UInt64> requested_keys_vector;
std::vector<size_t> requested_complex_key_rows; std::vector<size_t> requested_complex_key_rows;
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
requested_keys_vector.reserve(requested_keys.size()); requested_keys_vector.reserve(requested_keys.size());
else else
requested_complex_key_rows.reserve(requested_keys.size()); requested_complex_key_rows.reserve(requested_keys.size());
@ -546,7 +546,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
if (key_index_to_state_from_storage[i].isExpired() if (key_index_to_state_from_storage[i].isExpired()
|| key_index_to_state_from_storage[i].isNotFound()) || key_index_to_state_from_storage[i].isNotFound())
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
requested_keys_vector.emplace_back(requested_keys[i]); requested_keys_vector.emplace_back(requested_keys[i]);
else else
requested_complex_key_rows.emplace_back(i); requested_complex_key_rows.emplace_back(i);
@ -572,7 +572,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
Stopwatch watch; Stopwatch watch;
QueryPipeline pipeline; QueryPipeline pipeline;
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
pipeline.init(current_source_ptr->loadIds(requested_keys_vector)); pipeline.init(current_source_ptr->loadIds(requested_keys_vector));
else else
pipeline.init(current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows)); pipeline.init(current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows));
@ -684,7 +684,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
} }
} }
template class CacheDictionary<DictionaryKeyType::simple>; template class CacheDictionary<DictionaryKeyType::Simple>;
template class CacheDictionary<DictionaryKeyType::complex>; template class CacheDictionary<DictionaryKeyType::Complex>;
} }

View File

@ -51,8 +51,7 @@ template <DictionaryKeyType dictionary_key_type>
class CacheDictionary final : public IDictionary class CacheDictionary final : public IDictionary
{ {
public: public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; 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 cache dictionary");
CacheDictionary( CacheDictionary(
const StorageID & dict_id_, const StorageID & dict_id_,
@ -118,7 +117,7 @@ public:
DictionaryKeyType getKeyType() const override DictionaryKeyType getKeyType() const override
{ {
return dictionary_key_type == DictionaryKeyType::simple ? DictionaryKeyType::simple : DictionaryKeyType::complex; return dictionary_key_type == DictionaryKeyType::Simple ? DictionaryKeyType::Simple : DictionaryKeyType::Complex;
} }
ColumnPtr getColumn( ColumnPtr getColumn(
@ -141,7 +140,7 @@ public:
std::exception_ptr getLastException() const override; std::exception_ptr getLastException() const override;
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); } bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::Simple && dict_struct.hierarchical_attribute_index.has_value(); }
ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override; ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override;
@ -151,7 +150,7 @@ public:
const DataTypePtr & key_type) const override; const DataTypePtr & key_type) const override;
private: private:
using FetchResult = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, SimpleKeysStorageFetchResult, ComplexKeysStorageFetchResult>; using FetchResult = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, SimpleKeysStorageFetchResult, ComplexKeysStorageFetchResult>;
static MutableColumns aggregateColumnsInOrderOfKeys( static MutableColumns aggregateColumnsInOrderOfKeys(
const PaddedPODArray<KeyType> & keys, const PaddedPODArray<KeyType> & keys,
@ -219,7 +218,7 @@ private:
}; };
extern template class CacheDictionary<DictionaryKeyType::simple>; extern template class CacheDictionary<DictionaryKeyType::Simple>;
extern template class CacheDictionary<DictionaryKeyType::complex>; extern template class CacheDictionary<DictionaryKeyType::Complex>;
} }

View File

@ -41,8 +41,7 @@ class CacheDictionaryStorage final : public ICacheDictionaryStorage
static constexpr size_t max_collision_length = 10; static constexpr size_t max_collision_length = 10;
public: public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; 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 CacheDictionaryStorage");
explicit CacheDictionaryStorage( explicit CacheDictionaryStorage(
const DictionaryStructure & dictionary_structure, const DictionaryStructure & dictionary_structure,
@ -62,19 +61,19 @@ public:
String getName() const override String getName() const override
{ {
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
return "Cache"; return "Cache";
else else
return "ComplexKeyCache"; return "ComplexKeyCache";
} }
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::simple; } bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::Simple; }
SimpleKeysStorageFetchResult fetchColumnsForKeys( SimpleKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<UInt64> & keys, const PaddedPODArray<UInt64> & keys,
const DictionaryStorageFetchRequest & fetch_request) override const DictionaryStorageFetchRequest & fetch_request) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request); return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for complex key storage");
@ -82,7 +81,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertColumnsForKeysImpl(keys, columns); insertColumnsForKeysImpl(keys, columns);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
@ -90,7 +89,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertDefaultKeysImpl(keys); insertDefaultKeysImpl(keys);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage");
@ -98,19 +97,19 @@ public:
PaddedPODArray<UInt64> getCachedSimpleKeys() const override PaddedPODArray<UInt64> getCachedSimpleKeys() const override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return getCachedKeysImpl(); return getCachedKeysImpl();
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage");
} }
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; } bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::Complex; }
ComplexKeysStorageFetchResult fetchColumnsForKeys( ComplexKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<StringRef> & keys, const PaddedPODArray<StringRef> & keys,
const DictionaryStorageFetchRequest & column_fetch_requests) override const DictionaryStorageFetchRequest & column_fetch_requests) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, column_fetch_requests); return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, column_fetch_requests);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage");
@ -118,7 +117,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertColumnsForKeysImpl(keys, columns); insertColumnsForKeysImpl(keys, columns);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage");
@ -126,7 +125,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertDefaultKeysImpl(keys); insertDefaultKeysImpl(keys);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage");
@ -134,7 +133,7 @@ public:
PaddedPODArray<StringRef> getCachedComplexKeys() const override PaddedPODArray<StringRef> getCachedComplexKeys() const override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return getCachedKeysImpl(); return getCachedKeysImpl();
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedComplexKeys is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedComplexKeys is not supported for simple key storage");

View File

@ -14,8 +14,8 @@ namespace ErrorCodes
extern const int TIMEOUT_EXCEEDED; extern const int TIMEOUT_EXCEEDED;
} }
template class CacheDictionaryUpdateUnit<DictionaryKeyType::simple>; template class CacheDictionaryUpdateUnit<DictionaryKeyType::Simple>;
template class CacheDictionaryUpdateUnit<DictionaryKeyType::complex>; template class CacheDictionaryUpdateUnit<DictionaryKeyType::Complex>;
template <DictionaryKeyType dictionary_key_type> template <DictionaryKeyType dictionary_key_type>
CacheDictionaryUpdateQueue<dictionary_key_type>::CacheDictionaryUpdateQueue( CacheDictionaryUpdateQueue<dictionary_key_type>::CacheDictionaryUpdateQueue(
@ -155,7 +155,7 @@ void CacheDictionaryUpdateQueue<dictionary_key_type>::updateThreadFunction()
} }
} }
template class CacheDictionaryUpdateQueue<DictionaryKeyType::simple>; template class CacheDictionaryUpdateQueue<DictionaryKeyType::Simple>;
template class CacheDictionaryUpdateQueue<DictionaryKeyType::complex>; template class CacheDictionaryUpdateQueue<DictionaryKeyType::Complex>;
} }

View File

@ -39,7 +39,7 @@ template <DictionaryKeyType dictionary_key_type>
class CacheDictionaryUpdateUnit class CacheDictionaryUpdateUnit
{ {
public: public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
/// Constructor for complex keys update request /// Constructor for complex keys update request
explicit CacheDictionaryUpdateUnit( explicit CacheDictionaryUpdateUnit(
@ -85,8 +85,8 @@ private:
template <DictionaryKeyType dictionary_key_type> template <DictionaryKeyType dictionary_key_type>
using CacheDictionaryUpdateUnitPtr = std::shared_ptr<CacheDictionaryUpdateUnit<dictionary_key_type>>; using CacheDictionaryUpdateUnitPtr = std::shared_ptr<CacheDictionaryUpdateUnit<dictionary_key_type>>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::simple>; extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::Simple>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::complex>; extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::Complex>;
struct CacheDictionaryUpdateQueueConfiguration struct CacheDictionaryUpdateQueueConfiguration
{ {
@ -110,7 +110,6 @@ class CacheDictionaryUpdateQueue
public: public:
/// Client of update queue must provide this function in constructor and perform update using update unit. /// Client of update queue must provide this function in constructor and perform update using update unit.
using UpdateFunction = std::function<void (CacheDictionaryUpdateUnitPtr<dictionary_key_type>)>; using UpdateFunction = std::function<void (CacheDictionaryUpdateUnitPtr<dictionary_key_type>)>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionaryUpdateQueue");
CacheDictionaryUpdateQueue( CacheDictionaryUpdateQueue(
String dictionary_name_for_logs_, String dictionary_name_for_logs_,
@ -167,7 +166,7 @@ private:
std::atomic<bool> finished{false}; std::atomic<bool> finished{false};
}; };
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::simple>; extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::Simple>;
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::complex>; extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::Complex>;
} }

View File

@ -380,14 +380,14 @@ template <DictionaryKeyType key_type>
class DictionaryKeysArenaHolder; class DictionaryKeysArenaHolder;
template <> template <>
class DictionaryKeysArenaHolder<DictionaryKeyType::simple> class DictionaryKeysArenaHolder<DictionaryKeyType::Simple>
{ {
public: public:
static Arena * getComplexKeyArena() { return nullptr; } static Arena * getComplexKeyArena() { return nullptr; }
}; };
template <> template <>
class DictionaryKeysArenaHolder<DictionaryKeyType::complex> class DictionaryKeysArenaHolder<DictionaryKeyType::Complex>
{ {
public: public:
@ -402,8 +402,7 @@ template <DictionaryKeyType key_type>
class DictionaryKeysExtractor class DictionaryKeysExtractor
{ {
public: public:
using KeyType = std::conditional_t<key_type == DictionaryKeyType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
static_assert(key_type != DictionaryKeyType::range, "Range key type is not supported by DictionaryKeysExtractor");
explicit DictionaryKeysExtractor(const Columns & key_columns_, Arena * complex_key_arena_) explicit DictionaryKeysExtractor(const Columns & key_columns_, Arena * complex_key_arena_)
: key_columns(key_columns_) : key_columns(key_columns_)
@ -411,7 +410,7 @@ public:
{ {
assert(!key_columns.empty()); assert(!key_columns.empty());
if constexpr (key_type == DictionaryKeyType::simple) if constexpr (key_type == DictionaryKeyType::Simple)
{ {
key_columns[0] = key_columns[0]->convertToFullColumnIfConst(); key_columns[0] = key_columns[0]->convertToFullColumnIfConst();
@ -437,7 +436,7 @@ public:
{ {
assert(current_key_index < keys_size); assert(current_key_index < keys_size);
if constexpr (key_type == DictionaryKeyType::simple) if constexpr (key_type == DictionaryKeyType::Simple)
{ {
const auto & column_vector = static_cast<const ColumnVector<UInt64> &>(*key_columns[0]); const auto & column_vector = static_cast<const ColumnVector<UInt64> &>(*key_columns[0]);
const auto & data = column_vector.getData(); const auto & data = column_vector.getData();
@ -465,7 +464,7 @@ public:
void rollbackCurrentKey() const void rollbackCurrentKey() const
{ {
if constexpr (key_type == DictionaryKeyType::complex) if constexpr (key_type == DictionaryKeyType::Complex)
complex_key_arena->rollback(current_complex_key.size); complex_key_arena->rollback(current_complex_key.size);
} }
@ -521,8 +520,7 @@ void mergeBlockWithPipe(
Block & block_to_update, Block & block_to_update,
Pipe pipe) Pipe pipe)
{ {
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; 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 updatePreviousyLoadedBlockWithStream");
Columns saved_block_key_columns; Columns saved_block_key_columns;
saved_block_key_columns.reserve(key_columns_size); saved_block_key_columns.reserve(key_columns_size);

View File

@ -132,7 +132,7 @@ Block DictionarySourceData::fillBlock(
{ {
ColumnPtr column; ColumnPtr column;
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
column = dictionary->getColumn( column = dictionary->getColumn(
attribute.name, attribute.name,

View File

@ -40,7 +40,7 @@ Columns DirectDictionary<dictionary_key_type>::getColumns(
const DataTypes & key_types [[maybe_unused]], const DataTypes & key_types [[maybe_unused]],
const Columns & default_values_columns) const const Columns & default_values_columns) const
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types); dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder; DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -161,7 +161,7 @@ ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::hasKeys(
const Columns & key_columns, const Columns & key_columns,
const DataTypes & key_types [[maybe_unused]]) const const DataTypes & key_types [[maybe_unused]]) const
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types); dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder; DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -230,7 +230,7 @@ ColumnPtr DirectDictionary<dictionary_key_type>::getHierarchy(
ColumnPtr key_column, ColumnPtr key_column,
const DataTypePtr & key_type) const const DataTypePtr & key_type) const
{ {
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
size_t keys_found; size_t keys_found;
auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found); auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found);
@ -248,7 +248,7 @@ ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::isInHierarchy(
ColumnPtr in_key_column, ColumnPtr in_key_column,
const DataTypePtr & key_type) const const DataTypePtr & key_type) const
{ {
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
size_t keys_found = 0; size_t keys_found = 0;
auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found); auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found);
@ -269,7 +269,7 @@ Pipe DirectDictionary<dictionary_key_type>::getSourceBlockInputStream(
Pipe pipe; Pipe pipe;
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
std::vector<UInt64> ids; std::vector<UInt64> ids;
ids.reserve(requested_keys_size); ids.reserve(requested_keys_size);
@ -310,9 +310,9 @@ namespace
ContextPtr /* context */, ContextPtr /* context */,
bool /* created_from_ddl */) bool /* created_from_ddl */)
{ {
const auto * layout_name = dictionary_key_type == DictionaryKeyType::simple ? "direct" : "complex_key_direct"; const auto * layout_name = dictionary_key_type == DictionaryKeyType::Simple ? "direct" : "complex_key_direct";
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
if (dict_struct.key) if (dict_struct.key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
@ -344,13 +344,13 @@ namespace
} }
} }
template class DirectDictionary<DictionaryKeyType::simple>; template class DirectDictionary<DictionaryKeyType::Simple>;
template class DirectDictionary<DictionaryKeyType::complex>; template class DirectDictionary<DictionaryKeyType::Complex>;
void registerDictionaryDirect(DictionaryFactory & factory) void registerDictionaryDirect(DictionaryFactory & factory)
{ {
factory.registerLayout("direct", createDirectDictionary<DictionaryKeyType::simple>, false); factory.registerLayout("direct", createDirectDictionary<DictionaryKeyType::Simple>, false);
factory.registerLayout("complex_key_direct", createDirectDictionary<DictionaryKeyType::complex>, true); factory.registerLayout("complex_key_direct", createDirectDictionary<DictionaryKeyType::Complex>, true);
} }

View File

@ -20,8 +20,7 @@ template <DictionaryKeyType dictionary_key_type>
class DirectDictionary final : public IDictionary class DirectDictionary final : public IDictionary
{ {
public: public:
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by direct dictionary"); using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
DirectDictionary( DirectDictionary(
const StorageID & dict_id_, const StorageID & dict_id_,
@ -30,7 +29,7 @@ public:
std::string getTypeName() const override std::string getTypeName() const override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return "Direct"; return "Direct";
else else
return "ComplexKeyDirect"; return "ComplexKeyDirect";
@ -110,7 +109,7 @@ private:
mutable std::atomic<size_t> found_count{0}; mutable std::atomic<size_t> found_count{0};
}; };
extern template class DirectDictionary<DictionaryKeyType::simple>; extern template class DirectDictionary<DictionaryKeyType::Simple>;
extern template class DirectDictionary<DictionaryKeyType::complex>; extern template class DirectDictionary<DictionaryKeyType::Complex>;
} }

View File

@ -289,8 +289,8 @@ void FlatDictionary::blockToAttributes(const Block & block)
{ {
const auto keys_column = block.safeGetByPosition(0).column; const auto keys_column = block.safeGetByPosition(0).column;
DictionaryKeysArenaHolder<DictionaryKeyType::simple> arena_holder; DictionaryKeysArenaHolder<DictionaryKeyType::Simple> arena_holder;
DictionaryKeysExtractor<DictionaryKeyType::simple> keys_extractor({ keys_column }, arena_holder.getComplexKeyArena()); DictionaryKeysExtractor<DictionaryKeyType::Simple> keys_extractor({ keys_column }, arena_holder.getComplexKeyArena());
auto keys = keys_extractor.extractAllKeys(); auto keys = keys_extractor.extractAllKeys();
HashSet<UInt64> already_processed_keys; HashSet<UInt64> already_processed_keys;
@ -344,7 +344,7 @@ void FlatDictionary::updateData()
else else
{ {
Pipe pipe(source_ptr->loadUpdatedAll()); Pipe pipe(source_ptr->loadUpdatedAll());
mergeBlockWithPipe<DictionaryKeyType::simple>( mergeBlockWithPipe<DictionaryKeyType::Simple>(
dict_struct.getKeysSize(), dict_struct.getKeysSize(),
*update_field_loaded_block, *update_field_loaded_block,
std::move(pipe)); std::move(pipe));

View File

@ -72,7 +72,7 @@ public:
return dict_struct.getAttribute(attribute_name).injective; return dict_struct.getAttribute(attribute_name).injective;
} }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::simple; } DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Simple; }
ColumnPtr getColumn( ColumnPtr getColumn(
const std::string& attribute_name, const std::string& attribute_name,

View File

@ -61,7 +61,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getColumn(
const DataTypes & key_types [[maybe_unused]], const DataTypes & key_types [[maybe_unused]],
const ColumnPtr & default_values_column) const const ColumnPtr & default_values_column) const
{ {
if (dictionary_key_type == DictionaryKeyType::complex) if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types); dict_struct.validateKeyTypes(key_types);
ColumnPtr result; ColumnPtr result;
@ -163,7 +163,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getColumn(
template <DictionaryKeyType dictionary_key_type, bool sparse> template <DictionaryKeyType dictionary_key_type, bool sparse>
ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::hasKeys(const Columns & key_columns, const DataTypes & key_types) const ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::hasKeys(const Columns & key_columns, const DataTypes & key_types) const
{ {
if (dictionary_key_type == DictionaryKeyType::complex) if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types); dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder; DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -210,7 +210,7 @@ ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::hasKeys(const Co
template <DictionaryKeyType dictionary_key_type, bool sparse> template <DictionaryKeyType dictionary_key_type, bool sparse>
ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getHierarchy(ColumnPtr key_column [[maybe_unused]], const DataTypePtr &) const ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getHierarchy(ColumnPtr key_column [[maybe_unused]], const DataTypePtr &) const
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
PaddedPODArray<UInt64> keys_backup_storage; PaddedPODArray<UInt64> keys_backup_storage;
const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage);
@ -258,7 +258,7 @@ ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::isInHierarchy(
ColumnPtr in_key_column [[maybe_unused]], ColumnPtr in_key_column [[maybe_unused]],
const DataTypePtr &) const const DataTypePtr &) const
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
PaddedPODArray<UInt64> keys_backup_storage; PaddedPODArray<UInt64> keys_backup_storage;
const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage);
@ -309,7 +309,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getDescendants(
const DataTypePtr &, const DataTypePtr &,
size_t level [[maybe_unused]]) const size_t level [[maybe_unused]]) const
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
PaddedPODArray<UInt64> keys_backup; PaddedPODArray<UInt64> keys_backup;
const auto & keys = getColumnVectorData(this, key_column, keys_backup); const auto & keys = getColumnVectorData(this, key_column, keys_backup);
@ -665,7 +665,7 @@ Pipe HashedDictionary<dictionary_key_type, sparse>::read(const Names & column_na
}); });
} }
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return Pipe(std::make_shared<DictionarySource>(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); return Pipe(std::make_shared<DictionarySource>(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size));
else else
return Pipe(std::make_shared<DictionarySource>(DictionarySourceData(shared_from_this(), keys, column_names), max_block_size)); return Pipe(std::make_shared<DictionarySource>(DictionarySourceData(shared_from_this(), keys, column_names), max_block_size));
@ -702,10 +702,10 @@ void HashedDictionary<dictionary_key_type, sparse>::getAttributeContainer(size_t
}); });
} }
template class HashedDictionary<DictionaryKeyType::simple, true>; template class HashedDictionary<DictionaryKeyType::Simple, true>;
template class HashedDictionary<DictionaryKeyType::simple, false>; template class HashedDictionary<DictionaryKeyType::Simple, false>;
template class HashedDictionary<DictionaryKeyType::complex, true>; template class HashedDictionary<DictionaryKeyType::Complex, true>;
template class HashedDictionary<DictionaryKeyType::complex, false>; template class HashedDictionary<DictionaryKeyType::Complex, false>;
void registerDictionaryHashed(DictionaryFactory & factory) void registerDictionaryHashed(DictionaryFactory & factory)
{ {
@ -717,9 +717,9 @@ void registerDictionaryHashed(DictionaryFactory & factory)
DictionaryKeyType dictionary_key_type, DictionaryKeyType dictionary_key_type,
bool sparse) -> DictionaryPtr bool sparse) -> DictionaryPtr
{ {
if (dictionary_key_type == DictionaryKeyType::simple && dict_struct.key) if (dictionary_key_type == DictionaryKeyType::Simple && dict_struct.key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for simple key hashed dictionary"); throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for simple key hashed dictionary");
else if (dictionary_key_type == DictionaryKeyType::complex && dict_struct.id) else if (dictionary_key_type == DictionaryKeyType::Complex && dict_struct.id)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for complex key hashed dictionary"); throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for complex key hashed dictionary");
if (dict_struct.range_min || dict_struct.range_max) if (dict_struct.range_min || dict_struct.range_max)
@ -737,32 +737,32 @@ void registerDictionaryHashed(DictionaryFactory & factory)
HashedDictionaryStorageConfiguration configuration{preallocate, require_nonempty, dict_lifetime}; HashedDictionaryStorageConfiguration configuration{preallocate, require_nonempty, dict_lifetime};
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
if (sparse) if (sparse)
return std::make_unique<HashedDictionary<DictionaryKeyType::simple, true>>(dict_id, dict_struct, std::move(source_ptr), configuration); return std::make_unique<HashedDictionary<DictionaryKeyType::Simple, true>>(dict_id, dict_struct, std::move(source_ptr), configuration);
else else
return std::make_unique<HashedDictionary<DictionaryKeyType::simple, false>>(dict_id, dict_struct, std::move(source_ptr), configuration); return std::make_unique<HashedDictionary<DictionaryKeyType::Simple, false>>(dict_id, dict_struct, std::move(source_ptr), configuration);
} }
else else
{ {
if (sparse) if (sparse)
return std::make_unique<HashedDictionary<DictionaryKeyType::complex, true>>(dict_id, dict_struct, std::move(source_ptr), configuration); return std::make_unique<HashedDictionary<DictionaryKeyType::Complex, true>>(dict_id, dict_struct, std::move(source_ptr), configuration);
else else
return std::make_unique<HashedDictionary<DictionaryKeyType::complex, false>>(dict_id, dict_struct, std::move(source_ptr), configuration); return std::make_unique<HashedDictionary<DictionaryKeyType::Complex, false>>(dict_id, dict_struct, std::move(source_ptr), configuration);
} }
}; };
using namespace std::placeholders; using namespace std::placeholders;
factory.registerLayout("hashed", factory.registerLayout("hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false); [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ false); }, false);
factory.registerLayout("sparse_hashed", factory.registerLayout("sparse_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false); [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ true); }, false);
factory.registerLayout("complex_key_hashed", factory.registerLayout("complex_key_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true); [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ false); }, true);
factory.registerLayout("complex_key_sparse_hashed", factory.registerLayout("complex_key_sparse_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true); [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ true); }, true);
} }

View File

@ -35,8 +35,7 @@ template <DictionaryKeyType dictionary_key_type, bool sparse>
class HashedDictionary final : public IDictionary class HashedDictionary final : public IDictionary
{ {
public: public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; 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( HashedDictionary(
const StorageID & dict_id_, const StorageID & dict_id_,
@ -47,11 +46,11 @@ public:
std::string getTypeName() const override std::string getTypeName() const override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple && sparse) if constexpr (dictionary_key_type == DictionaryKeyType::Simple && sparse)
return "SparseHashed"; return "SparseHashed";
else if constexpr (dictionary_key_type == DictionaryKeyType::simple && !sparse) else if constexpr (dictionary_key_type == DictionaryKeyType::Simple && !sparse)
return "Hashed"; return "Hashed";
else if constexpr (dictionary_key_type == DictionaryKeyType::complex && sparse) else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && sparse)
return "ComplexKeySparseHashed"; return "ComplexKeySparseHashed";
else else
return "ComplexKeyHashed"; return "ComplexKeyHashed";
@ -102,7 +101,7 @@ public:
ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override;
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); } bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::Simple && dict_struct.hierarchical_attribute_index.has_value(); }
ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & hierarchy_attribute_type) const override; ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & hierarchy_attribute_type) const override;
@ -121,13 +120,13 @@ public:
private: private:
template <typename Value> template <typename Value>
using CollectionTypeNonSparse = std::conditional_t< using CollectionTypeNonSparse = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple, dictionary_key_type == DictionaryKeyType::Simple,
HashMap<UInt64, Value>, HashMap<UInt64, Value>,
HashMapWithSavedHash<StringRef, Value, DefaultHash<StringRef>>>; HashMapWithSavedHash<StringRef, Value, DefaultHash<StringRef>>>;
template <typename Value> template <typename Value>
using CollectionTypeSparse = std::conditional_t< using CollectionTypeSparse = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple, dictionary_key_type == DictionaryKeyType::Simple,
SparseHashMap<UInt64, Value>, SparseHashMap<UInt64, Value>,
SparseHashMap<StringRef, Value>>; SparseHashMap<StringRef, Value>>;
@ -211,10 +210,10 @@ private:
Arena complex_key_arena; Arena complex_key_arena;
}; };
extern template class HashedDictionary<DictionaryKeyType::simple, false>; extern template class HashedDictionary<DictionaryKeyType::Simple, false>;
extern template class HashedDictionary<DictionaryKeyType::simple, true>; extern template class HashedDictionary<DictionaryKeyType::Simple, true>;
extern template class HashedDictionary<DictionaryKeyType::complex, false>; extern template class HashedDictionary<DictionaryKeyType::Complex, false>;
extern template class HashedDictionary<DictionaryKeyType::complex, true>; extern template class HashedDictionary<DictionaryKeyType::Complex, true>;
} }

View File

@ -33,15 +33,20 @@ using DictionaryPtr = std::unique_ptr<IDictionary>;
* Simple is for dictionaries that support UInt64 key column. * Simple is for dictionaries that support UInt64 key column.
* *
* Complex is for dictionaries that support any combination of key columns. * Complex is for dictionaries that support any combination of key columns.
*
* Range is for dictionary that support combination of UInt64 key column,
* and numeric representable range key column.
*/ */
enum class DictionaryKeyType enum class DictionaryKeyType
{ {
simple, Simple,
complex, Complex
range };
/** DictionarySpecialKeyType provides IDictionary client information about
* which special key type is supported by dictionary.
*/
enum class DictionarySpecialKeyType
{
None,
Range
}; };
/** /**
@ -56,6 +61,7 @@ struct IDictionary : public IExternalLoadable
} }
const std::string & getFullName() const{ return full_name; } const std::string & getFullName() const{ return full_name; }
StorageID getDictionaryID() const StorageID getDictionaryID() const
{ {
std::lock_guard lock{name_mutex}; std::lock_guard lock{name_mutex};
@ -109,6 +115,8 @@ struct IDictionary : public IExternalLoadable
*/ */
virtual DictionaryKeyType getKeyType() const = 0; virtual DictionaryKeyType getKeyType() const = 0;
virtual DictionarySpecialKeyType getSpecialKeyType() const { return DictionarySpecialKeyType::None;}
/** Subclass must validate key columns and keys types /** Subclass must validate key columns and keys types
* and return column representation of dictionary attribute. * and return column representation of dictionary attribute.
* *

View File

@ -67,7 +67,7 @@ public:
return dict_struct.getAttribute(attribute_name).injective; return dict_struct.getAttribute(attribute_name).injective;
} }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; } DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Complex; }
ColumnPtr getColumn( ColumnPtr getColumn(
const std::string& attribute_name, const std::string& attribute_name,

View File

@ -86,7 +86,7 @@ public:
bool isInjective(const std::string & attribute_name) const override { return dict_struct.getAttribute(attribute_name).injective; } bool isInjective(const std::string & attribute_name) const override { return dict_struct.getAttribute(attribute_name).injective; }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; } DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Complex; }
ColumnPtr getColumn( ColumnPtr getColumn(
const std::string& attribute_name, const std::string& attribute_name,

View File

@ -14,18 +14,12 @@
namespace DB namespace DB
{ {
enum class RangeDictionaryType template <DictionaryKeyType dictionary_key_type, typename RangeType>
{
simple,
complex
};
template <RangeDictionaryType range_dictionary_type, typename RangeType>
class RangeDictionarySourceData class RangeDictionarySourceData
{ {
public: public:
using KeyType = std::conditional_t<range_dictionary_type == RangeDictionaryType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
RangeDictionarySourceData( RangeDictionarySourceData(
std::shared_ptr<const IDictionary> dictionary, std::shared_ptr<const IDictionary> dictionary,
@ -58,8 +52,8 @@ private:
}; };
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
RangeDictionarySourceData<range_dictionary_type, RangeType>::RangeDictionarySourceData( RangeDictionarySourceData<dictionary_key_type, RangeType>::RangeDictionarySourceData(
std::shared_ptr<const IDictionary> dictionary_, std::shared_ptr<const IDictionary> dictionary_,
const Names & column_names_, const Names & column_names_,
PaddedPODArray<KeyType> && keys, PaddedPODArray<KeyType> && keys,
@ -73,8 +67,8 @@ RangeDictionarySourceData<range_dictionary_type, RangeType>::RangeDictionarySour
{ {
} }
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
Block RangeDictionarySourceData<range_dictionary_type, RangeType>::getBlock(size_t start, size_t length) const Block RangeDictionarySourceData<dictionary_key_type, RangeType>::getBlock(size_t start, size_t length) const
{ {
PaddedPODArray<KeyType> block_keys; PaddedPODArray<KeyType> block_keys;
PaddedPODArray<RangeType> block_start_dates; PaddedPODArray<RangeType> block_start_dates;
@ -93,8 +87,8 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::getBlock(size
return fillBlock(block_keys, block_start_dates, block_end_dates, start, start + length); return fillBlock(block_keys, block_start_dates, block_end_dates, start, start + length);
} }
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
PaddedPODArray<Int64> RangeDictionarySourceData<range_dictionary_type, RangeType>::makeDateKeys( PaddedPODArray<Int64> RangeDictionarySourceData<dictionary_key_type, RangeType>::makeDateKeys(
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_start_dates,
const PaddedPODArray<RangeType> & block_end_dates) const const PaddedPODArray<RangeType> & block_end_dates) const
{ {
@ -112,24 +106,14 @@ PaddedPODArray<Int64> RangeDictionarySourceData<range_dictionary_type, RangeType
} }
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock( Block RangeDictionarySourceData<dictionary_key_type, RangeType>::fillBlock(
const PaddedPODArray<KeyType> & keys_to_fill, const PaddedPODArray<KeyType> & keys_to_fill,
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_start_dates,
const PaddedPODArray<RangeType> & block_end_dates, const PaddedPODArray<RangeType> & block_end_dates,
size_t start, size_t start,
size_t end) const size_t end) const
{ {
std::cerr << "RangeDictionarySourceData::fillBlock keys_to_fill " << keys_to_fill.size() << std::endl;
if constexpr (range_dictionary_type == RangeDictionaryType::simple)
{
for (auto & key : keys_to_fill)
{
std::cerr << key << std::endl;
}
}
ColumnsWithTypeAndName columns; ColumnsWithTypeAndName columns;
const DictionaryStructure & dictionary_structure = dictionary->getStructure(); const DictionaryStructure & dictionary_structure = dictionary->getStructure();
@ -137,7 +121,7 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
Columns keys_columns; Columns keys_columns;
Strings keys_names = dictionary_structure.getKeysNames(); Strings keys_names = dictionary_structure.getKeysNames();
if constexpr (range_dictionary_type == RangeDictionaryType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
keys_columns = {getColumnFromPODArray(keys_to_fill)}; keys_columns = {getColumnFromPODArray(keys_to_fill)};
keys_types = {std::make_shared<DataTypeUInt64>()}; keys_types = {std::make_shared<DataTypeUInt64>()};
@ -154,9 +138,6 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
size_t keys_size = keys_names.size(); size_t keys_size = keys_names.size();
std::cerr << "Keys size " << keys_size << " key columns size " << keys_columns.size();
std::cerr << " keys types size " << keys_types.size() << std::endl;
assert(keys_columns.size() == keys_size); assert(keys_columns.size() == keys_size);
assert(keys_types.size() == keys_size); assert(keys_types.size() == keys_size);
@ -204,51 +185,33 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
columns.emplace_back(std::move(column), attribute.type, attribute.name); columns.emplace_back(std::move(column), attribute.type, attribute.name);
} }
auto result = Block(columns);
Field value;
std::cerr << "RangeDictionarySourceData::fillBlock result" << std::endl;
for (auto & block_column : result)
{
std::cerr << "Column name " << block_column.name << " type " << block_column.type->getName() << std::endl;
auto & column = block_column.column;
size_t column_size = column->size();
for (size_t i = 0; i < column_size; ++i)
{
column->get(i, value);
std::cerr << "Index " << i << " value " << value.dump() << std::endl;
}
}
return Block(columns); return Block(columns);
} }
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
class RangeDictionarySource : public DictionarySourceBase class RangeDictionarySource : public DictionarySourceBase
{ {
public: public:
RangeDictionarySource(RangeDictionarySourceData<range_dictionary_type, RangeType> data_, size_t max_block_size); RangeDictionarySource(RangeDictionarySourceData<dictionary_key_type, RangeType> data_, size_t max_block_size);
String getName() const override { return "RangeDictionarySource"; } String getName() const override { return "RangeDictionarySource"; }
protected: protected:
Block getBlock(size_t start, size_t length) const override; Block getBlock(size_t start, size_t length) const override;
RangeDictionarySourceData<range_dictionary_type, RangeType> data; RangeDictionarySourceData<dictionary_key_type, RangeType> data;
}; };
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
RangeDictionarySource<range_dictionary_type, RangeType>::RangeDictionarySource(RangeDictionarySourceData<range_dictionary_type, RangeType> data_, size_t max_block_size) RangeDictionarySource<dictionary_key_type, RangeType>::RangeDictionarySource(RangeDictionarySourceData<dictionary_key_type, RangeType> data_, size_t max_block_size)
: DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size) : DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size)
, data(std::move(data_)) , data(std::move(data_))
{ {
} }
template <RangeDictionaryType range_dictionary_type, typename RangeType> template <DictionaryKeyType dictionary_key_type, typename RangeType>
Block RangeDictionarySource<range_dictionary_type, RangeType>::getBlock(size_t start, size_t length) const Block RangeDictionarySource<dictionary_key_type, RangeType>::getBlock(size_t start, size_t length) const
{ {
return data.getBlock(start, length); return data.getBlock(start, length);
} }

View File

@ -95,6 +95,13 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::getColumn(
const DataTypes & key_types, const DataTypes & key_types,
const ColumnPtr & default_values_column) const const ColumnPtr & default_values_column) const
{ {
if (dictionary_key_type == DictionaryKeyType::Complex)
{
auto key_types_copy = key_types;
key_types_copy.pop_back();
dict_struct.validateKeyTypes(key_types_copy);
}
ColumnPtr result; ColumnPtr result;
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
@ -206,9 +213,15 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::getColumn(
template <DictionaryKeyType dictionary_key_type> template <DictionaryKeyType dictionary_key_type>
ColumnUInt8::Ptr RangeHashedDictionary<dictionary_key_type>::hasKeys(const Columns & key_columns, const DataTypes & key_types) const ColumnUInt8::Ptr RangeHashedDictionary<dictionary_key_type>::hasKeys(const Columns & key_columns, const DataTypes & key_types) const
{ {
if (dictionary_key_type == DictionaryKeyType::Complex)
{
auto key_types_copy = key_types;
key_types_copy.pop_back();
dict_struct.validateKeyTypes(key_types_copy);
}
auto range_column_storage_type = std::make_shared<DataTypeInt64>(); auto range_column_storage_type = std::make_shared<DataTypeInt64>();
auto range_storage_column = key_columns.back(); auto range_storage_column = key_columns.back();
ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types[1], ""}; ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types.back(), ""};
auto range_column_updated = castColumnAccurate(column_to_cast, range_column_storage_type); auto range_column_updated = castColumnAccurate(column_to_cast, range_column_storage_type);
PaddedPODArray<RangeStorageType> range_backup_storage; PaddedPODArray<RangeStorageType> range_backup_storage;
const PaddedPODArray<RangeStorageType> & dates = getColumnVectorData(this, range_column_updated, range_backup_storage); const PaddedPODArray<RangeStorageType> & dates = getColumnVectorData(this, range_column_updated, range_backup_storage);
@ -383,7 +396,7 @@ void RangeHashedDictionary<dictionary_key_type>::calculateBytesAllocated()
callOnDictionaryAttributeType(attribute.type, type_call); callOnDictionaryAttributeType(attribute.type, type_call);
} }
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
bytes_allocated += complex_key_arena.size(); bytes_allocated += complex_key_arena.size();
} }
@ -607,10 +620,9 @@ Pipe RangeHashedDictionary<dictionary_key_type>::readImpl(const Names & column_n
PaddedPODArray<RangeType> end_dates; PaddedPODArray<RangeType> end_dates;
getKeysAndDates(keys, start_dates, end_dates); getKeysAndDates(keys, start_dates, end_dates);
static constexpr RangeDictionaryType range_dictionary_type = (dictionary_key_type == DictionaryKeyType::simple) ? RangeDictionaryType::simple : RangeDictionaryType::complex; using RangeDictionarySourceType = RangeDictionarySource<dictionary_key_type, RangeType>;
using RangeDictionarySourceType = RangeDictionarySource<range_dictionary_type, RangeType>;
auto source_data = RangeDictionarySourceData<range_dictionary_type, RangeType>( auto source_data = RangeDictionarySourceData<dictionary_key_type, RangeType>(
shared_from_this(), shared_from_this(),
column_names, column_names,
std::move(keys), std::move(keys),
@ -690,7 +702,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<RangeHashedDictionary<DictionaryKeyType::simple>>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); return std::make_unique<RangeHashedDictionary<DictionaryKeyType::Simple>>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
}; };
factory.registerLayout("range_hashed", create_layout_simple, false); factory.registerLayout("range_hashed", create_layout_simple, false);
@ -713,7 +725,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<RangeHashedDictionary<DictionaryKeyType::complex>>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); return std::make_unique<RangeHashedDictionary<DictionaryKeyType::Complex>>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
}; };
factory.registerLayout("complex_key_range_hashed", create_layout_complex, true); factory.registerLayout("complex_key_range_hashed", create_layout_complex, true);
} }

View File

@ -32,8 +32,7 @@ template <DictionaryKeyType dictionary_key_type>
class RangeHashedDictionary final : public IDictionary class RangeHashedDictionary final : public IDictionary
{ {
public: public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; 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");
RangeHashedDictionary( RangeHashedDictionary(
const StorageID & dict_id_, const StorageID & dict_id_,
@ -78,7 +77,9 @@ public:
return dict_struct.getAttribute(attribute_name).injective; return dict_struct.getAttribute(attribute_name).injective;
} }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::range; } DictionaryKeyType getKeyType() const override { return dictionary_key_type; }
DictionarySpecialKeyType getSpecialKeyType() const override { return DictionarySpecialKeyType::Range;}
ColumnPtr getColumn( ColumnPtr getColumn(
const std::string& attribute_name, const std::string& attribute_name,
@ -104,7 +105,7 @@ private:
template <typename Value> template <typename Value>
using CollectionType = std::conditional_t< using CollectionType = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple, dictionary_key_type == DictionaryKeyType::Simple,
HashMap<UInt64, Values<Value>>, HashMap<UInt64, Values<Value>>,
HashMapWithSavedHash<StringRef, Values<Value>, DefaultHash<StringRef>>>; HashMapWithSavedHash<StringRef, Values<Value>, DefaultHash<StringRef>>>;

View File

@ -823,8 +823,8 @@ template <DictionaryKeyType dictionary_key_type>
class SSDCacheDictionaryStorage final : public ICacheDictionaryStorage class SSDCacheDictionaryStorage final : public ICacheDictionaryStorage
{ {
public: public:
using SSDCacheKeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, SSDCacheSimpleKey, SSDCacheComplexKey>; using SSDCacheKeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, SSDCacheSimpleKey, SSDCacheComplexKey>;
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>; using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
explicit SSDCacheDictionaryStorage(const SSDCacheDictionaryStorageConfiguration & configuration_) explicit SSDCacheDictionaryStorage(const SSDCacheDictionaryStorageConfiguration & configuration_)
: configuration(configuration_) : configuration(configuration_)
@ -838,19 +838,19 @@ public:
String getName() const override String getName() const override
{ {
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
return "SSDCache"; return "SSDCache";
else else
return "SSDComplexKeyCache"; return "SSDComplexKeyCache";
} }
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::simple; } bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::Simple; }
SimpleKeysStorageFetchResult fetchColumnsForKeys( SimpleKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<UInt64> & keys, const PaddedPODArray<UInt64> & keys,
const DictionaryStorageFetchRequest & fetch_request) override const DictionaryStorageFetchRequest & fetch_request) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request); return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
@ -858,7 +858,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertColumnsForKeysImpl(keys, columns); insertColumnsForKeysImpl(keys, columns);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
@ -866,7 +866,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertDefaultKeysImpl(keys); insertDefaultKeysImpl(keys);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage");
@ -874,19 +874,19 @@ public:
PaddedPODArray<UInt64> getCachedSimpleKeys() const override PaddedPODArray<UInt64> getCachedSimpleKeys() const override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return getCachedKeysImpl(); return getCachedKeysImpl();
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage");
} }
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; } bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::Complex; }
ComplexKeysStorageFetchResult fetchColumnsForKeys( ComplexKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<StringRef> & keys, const PaddedPODArray<StringRef> & keys,
const DictionaryStorageFetchRequest & fetch_request) override const DictionaryStorageFetchRequest & fetch_request) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, fetch_request); return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, fetch_request);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage");
@ -894,7 +894,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertColumnsForKeysImpl(keys, columns); insertColumnsForKeysImpl(keys, columns);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage");
@ -902,7 +902,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertDefaultKeysImpl(keys); insertDefaultKeysImpl(keys);
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage");
@ -910,7 +910,7 @@ public:
PaddedPODArray<StringRef> getCachedComplexKeys() const override PaddedPODArray<StringRef> getCachedComplexKeys() const override
{ {
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return getCachedKeysImpl(); return getCachedKeysImpl();
else else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for simple key storage"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for simple key storage");
@ -1134,7 +1134,7 @@ private:
Cell cell; Cell cell;
setCellDeadline(cell, now); setCellDeadline(cell, now);
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
{ {
/// Copy complex key into arena and put in cache /// Copy complex key into arena and put in cache
size_t key_size = key.size; size_t key_size = key.size;
@ -1166,7 +1166,7 @@ private:
cell.state = Cell::default_value; cell.state = Cell::default_value;
if constexpr (dictionary_key_type == DictionaryKeyType::complex) if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
{ {
/// Copy complex key into arena and put in cache /// Copy complex key into arena and put in cache
size_t key_size = key.size; size_t key_size = key.size;
@ -1382,7 +1382,7 @@ private:
using ComplexKeyHashMap = HashMapWithSavedHash<StringRef, Cell>; using ComplexKeyHashMap = HashMapWithSavedHash<StringRef, Cell>;
using CacheMap = std::conditional_t< using CacheMap = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple, dictionary_key_type == DictionaryKeyType::Simple,
SimpleKeyHashMap, SimpleKeyHashMap,
ComplexKeyHashMap>; ComplexKeyHashMap>;

View File

@ -157,24 +157,23 @@ DictionaryPtr createCacheDictionaryLayout(
ContextPtr context [[maybe_unused]], ContextPtr context [[maybe_unused]],
bool created_from_ddl [[maybe_unused]]) bool created_from_ddl [[maybe_unused]])
{ {
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionary");
String layout_type; String layout_type;
if constexpr (dictionary_key_type == DictionaryKeyType::simple && !ssd)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple && !ssd)
layout_type = "cache"; layout_type = "cache";
else if constexpr (dictionary_key_type == DictionaryKeyType::simple && ssd) else if constexpr (dictionary_key_type == DictionaryKeyType::Simple && ssd)
layout_type = "ssd_cache"; layout_type = "ssd_cache";
else if constexpr (dictionary_key_type == DictionaryKeyType::complex && !ssd) else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && !ssd)
layout_type = "complex_key_cache"; layout_type = "complex_key_cache";
else if constexpr (dictionary_key_type == DictionaryKeyType::complex && ssd) else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && ssd)
layout_type = "complex_key_ssd_cache"; layout_type = "complex_key_ssd_cache";
if constexpr (dictionary_key_type == DictionaryKeyType::simple) if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{ {
if (dict_struct.key) if (dict_struct.key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: dictionary of layout '{}' 'key' is not supported", full_name, layout_type); throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: dictionary of layout '{}' 'key' is not supported", full_name, layout_type);
} }
else if constexpr (dictionary_key_type == DictionaryKeyType::complex) else if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
{ {
if (dict_struct.id) if (dict_struct.id)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: dictionary of layout '{}' 'id' is not supported", full_name, layout_type); throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: dictionary of layout '{}' 'id' is not supported", full_name, layout_type);
@ -243,7 +242,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
ContextPtr context, ContextPtr context,
bool created_from_ddl) -> DictionaryPtr bool created_from_ddl) -> DictionaryPtr
{ {
return createCacheDictionaryLayout<DictionaryKeyType::simple, false/* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); return createCacheDictionaryLayout<DictionaryKeyType::Simple, false/* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
}; };
factory.registerLayout("cache", create_simple_cache_layout, false); factory.registerLayout("cache", create_simple_cache_layout, false);
@ -256,7 +255,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
ContextPtr context, ContextPtr context,
bool created_from_ddl) -> DictionaryPtr bool created_from_ddl) -> DictionaryPtr
{ {
return createCacheDictionaryLayout<DictionaryKeyType::complex, false /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); return createCacheDictionaryLayout<DictionaryKeyType::Complex, false /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
}; };
factory.registerLayout("complex_key_cache", create_complex_key_cache_layout, true); factory.registerLayout("complex_key_cache", create_complex_key_cache_layout, true);
@ -271,7 +270,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
ContextPtr context, ContextPtr context,
bool created_from_ddl) -> DictionaryPtr bool created_from_ddl) -> DictionaryPtr
{ {
return createCacheDictionaryLayout<DictionaryKeyType::simple, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); return createCacheDictionaryLayout<DictionaryKeyType::Simple, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
}; };
factory.registerLayout("ssd_cache", create_simple_ssd_cache_layout, false); factory.registerLayout("ssd_cache", create_simple_ssd_cache_layout, false);
@ -283,7 +282,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr context, ContextPtr context,
bool created_from_ddl) -> DictionaryPtr { bool created_from_ddl) -> DictionaryPtr {
return createCacheDictionaryLayout<DictionaryKeyType::complex, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); return createCacheDictionaryLayout<DictionaryKeyType::Complex, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
}; };
factory.registerLayout("complex_key_ssd_cache", create_complex_key_ssd_cache_layout, true); factory.registerLayout("complex_key_ssd_cache", create_complex_key_ssd_cache_layout, true);

View File

@ -42,7 +42,6 @@ namespace ErrorCodes
extern const int UNSUPPORTED_METHOD; extern const int UNSUPPORTED_METHOD;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
extern const int TYPE_MISMATCH; extern const int TYPE_MISMATCH;
} }
@ -182,15 +181,16 @@ public:
auto dictionary = helper.getDictionary(arguments[0].column); auto dictionary = helper.getDictionary(arguments[0].column);
auto dictionary_key_type = dictionary->getKeyType(); auto dictionary_key_type = dictionary->getKeyType();
auto dictionary_special_key_type = dictionary->getSpecialKeyType();
const ColumnWithTypeAndName & key_column_with_type = arguments[1]; const auto & key_column_with_type = arguments[1];
auto key_column = key_column_with_type.column; auto key_column = key_column_with_type.column;
auto key_column_type = key_column_with_type.type; auto key_column_type = key_column_with_type.type;
ColumnPtr range_col = nullptr; ColumnPtr range_col;
DataTypePtr range_col_type = nullptr; DataTypePtr range_col_type;
if (dictionary_key_type == DictionaryKeyType::range) if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
{ {
if (arguments.size() != 3) if (arguments.size() != 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
@ -207,7 +207,10 @@ public:
getName()); getName());
} }
if (dictionary_key_type == DictionaryKeyType::simple) Columns key_columns;
DataTypes key_types;
if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
if (!WhichDataType(key_column_type).isUInt64()) if (!WhichDataType(key_column_type).isUInt64())
throw Exception( throw Exception(
@ -216,16 +219,23 @@ public:
getName(), getName(),
key_column_with_type.type->getName()); key_column_with_type.type->getName());
return dictionary->hasKeys({key_column}, {std::make_shared<DataTypeUInt64>()}); key_columns = {key_column};
key_types = {std::make_shared<DataTypeUInt64>()};
} }
else if (dictionary_key_type == DictionaryKeyType::complex) else if (dictionary_key_type == DictionaryKeyType::Complex)
{ {
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. /// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
key_column = key_column->convertToFullColumnIfConst(); key_column = key_column->convertToFullColumnIfConst();
size_t keys_size = dictionary->getStructure().getKeysSize();
if (!isTuple(key_column_type)) if (isTuple(key_column_type))
{ {
key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
}
else
{
size_t keys_size = dictionary->getStructure().getKeysSize();
if (keys_size > 1) if (keys_size > 1)
{ {
throw Exception( throw Exception(
@ -237,41 +247,19 @@ public:
} }
else else
{ {
Columns tuple_columns = {std::move(key_column)}; key_columns = {key_column};
key_column = ColumnTuple::create(tuple_columns); key_types = {key_column_type};
DataTypes tuple_types = {key_column_type};
key_column_type = std::make_shared<DataTypeTuple>(tuple_types);
} }
} }
const auto & key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
const auto & key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
return dictionary->hasKeys(key_columns, key_types);
} }
else
if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
{ {
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. key_columns.emplace_back(range_col);
ColumnPtr key_column = key_column_with_type.column->convertToFullColumnIfConst(); key_types.emplace_back(range_col_type);
DataTypePtr key_column_type = key_column_with_type.type;
Columns key_columns;
DataTypes key_types;
if (isTuple(key_column_type))
{
key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
}
else
{
key_columns = {key_column, range_col};
key_types = {std::make_shared<DataTypeUInt64>(), range_col_type};
}
return dictionary->hasKeys({key_column, range_col}, {std::make_shared<DataTypeUInt64>(), range_col_type});
} }
return dictionary->hasKeys(key_columns, key_types);
} }
private: private:
@ -369,13 +357,14 @@ public:
auto dictionary = helper.getDictionary(dictionary_name); auto dictionary = helper.getDictionary(dictionary_name);
auto dictionary_key_type = dictionary->getKeyType(); auto dictionary_key_type = dictionary->getKeyType();
auto dictionary_special_key_type = dictionary->getSpecialKeyType();
size_t current_arguments_index = 3; size_t current_arguments_index = 3;
ColumnPtr range_col = nullptr; ColumnPtr range_col = nullptr;
DataTypePtr range_col_type = nullptr; DataTypePtr range_col_type = nullptr;
if (dictionary_key_type == DictionaryKeyType::range) if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
{ {
if (current_arguments_index >= arguments.size()) if (current_arguments_index >= arguments.size())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
@ -437,12 +426,13 @@ public:
default_cols.emplace_back(nullptr); default_cols.emplace_back(nullptr);
} }
ColumnPtr result; const auto & key_col_with_type = arguments[2];
auto key_column = key_col_with_type.column;
const ColumnWithTypeAndName & key_col_with_type = arguments[2]; Columns key_columns;
const auto key_column = key_col_with_type.column; DataTypes key_types;
if (dictionary_key_type == DictionaryKeyType::simple) if (dictionary_key_type == DictionaryKeyType::Simple)
{ {
if (!WhichDataType(key_col_with_type.type).isUInt64()) if (!WhichDataType(key_col_with_type.type).isUInt64())
throw Exception( throw Exception(
@ -451,24 +441,24 @@ public:
getName(), getName(),
key_col_with_type.type->getName()); key_col_with_type.type->getName());
result = executeDictionaryRequest( key_columns = {key_column};
dictionary, key_types = {std::make_shared<DataTypeUInt64>()};
attribute_names,
{key_column},
{std::make_shared<DataTypeUInt64>()},
result_type,
default_cols);
} }
else if (dictionary_key_type == DictionaryKeyType::complex) else if (dictionary_key_type == DictionaryKeyType::Complex)
{ {
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. /// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column = key_col_with_type.column->convertToFullColumnIfConst(); key_column = key_column->convertToFullColumnIfConst();
DataTypePtr key_column_type = key_col_with_type.type; DataTypePtr key_column_type = key_col_with_type.type;
size_t keys_size = dictionary->getStructure().getKeysSize(); if (isTuple(key_column_type))
if (!isTuple(key_column_type))
{ {
key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
}
else if (!isTuple(key_column_type))
{
size_t keys_size = dictionary->getStructure().getKeysSize();
if (keys_size > 1) if (keys_size > 1)
{ {
throw Exception( throw Exception(
@ -480,60 +470,19 @@ public:
} }
else else
{ {
Columns tuple_columns = {std::move(key_column)}; key_columns = {std::move(key_column)};
key_column = ColumnTuple::create(tuple_columns); key_types = {std::move(key_column_type)};
DataTypes tuple_types = {key_column_type};
key_column_type = std::make_shared<DataTypeTuple>(tuple_types);
} }
} }
const auto & key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
const auto & key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
result = executeDictionaryRequest(
dictionary,
attribute_names,
key_columns,
key_types,
result_type,
default_cols);
} }
else if (dictionary_key_type == DictionaryKeyType::range)
if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
{ {
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column = key_col_with_type.column->convertToFullColumnIfConst();
DataTypePtr key_column_type = key_col_with_type.type;
Columns key_columns;
DataTypes key_types;
if (isTuple(key_column_type))
{
key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
}
else
{
key_columns = {key_column};
key_types = {std::make_shared<DataTypeUInt64>()};
}
key_columns.emplace_back(range_col); key_columns.emplace_back(range_col);
key_types.emplace_back(range_col_type); key_types.emplace_back(range_col_type);
result = executeDictionaryRequest(
dictionary,
attribute_names,
key_columns,
key_types,
result_type,
default_cols);
} }
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown dictionary identifier type");
return result; return executeDictionaryRequest(dictionary, attribute_names, key_columns, key_types, result_type, default_cols);
} }
private: private: