mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Dictionaries key types refactoring
This commit is contained in:
parent
c0e1c42562
commit
b4f41bd824
@ -151,7 +151,7 @@ Columns CacheDictionary<dictionary_key_type>::getColumns(
|
||||
* use default value.
|
||||
*/
|
||||
|
||||
if (dictionary_key_type == DictionaryKeyType::complex)
|
||||
if (dictionary_key_type == DictionaryKeyType::Complex)
|
||||
dict_struct.validateKeyTypes(key_types);
|
||||
|
||||
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.
|
||||
*/
|
||||
|
||||
if (dictionary_key_type == DictionaryKeyType::complex)
|
||||
if (dictionary_key_type == DictionaryKeyType::Complex)
|
||||
dict_struct.validateKeyTypes(key_types);
|
||||
|
||||
|
||||
@ -364,7 +364,7 @@ ColumnPtr CacheDictionary<dictionary_key_type>::getHierarchy(
|
||||
ColumnPtr key_column [[maybe_unused]],
|
||||
const DataTypePtr & key_type [[maybe_unused]]) const
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
{
|
||||
size_t 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]],
|
||||
const DataTypePtr & key_type [[maybe_unused]]) const
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
{
|
||||
size_t 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
|
||||
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);
|
||||
else
|
||||
{
|
||||
@ -534,7 +534,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
|
||||
std::vector<UInt64> requested_keys_vector;
|
||||
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());
|
||||
else
|
||||
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()
|
||||
|| 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]);
|
||||
else
|
||||
requested_complex_key_rows.emplace_back(i);
|
||||
@ -572,7 +572,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
|
||||
Stopwatch watch;
|
||||
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));
|
||||
else
|
||||
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::complex>;
|
||||
template class CacheDictionary<DictionaryKeyType::Simple>;
|
||||
template class CacheDictionary<DictionaryKeyType::Complex>;
|
||||
|
||||
}
|
||||
|
@ -51,8 +51,7 @@ template <DictionaryKeyType dictionary_key_type>
|
||||
class CacheDictionary 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 cache dictionary");
|
||||
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
|
||||
|
||||
CacheDictionary(
|
||||
const StorageID & dict_id_,
|
||||
@ -118,7 +117,7 @@ public:
|
||||
|
||||
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(
|
||||
@ -141,7 +140,7 @@ public:
|
||||
|
||||
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;
|
||||
|
||||
@ -151,7 +150,7 @@ public:
|
||||
const DataTypePtr & key_type) const override;
|
||||
|
||||
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(
|
||||
const PaddedPODArray<KeyType> & keys,
|
||||
@ -219,7 +218,7 @@ private:
|
||||
|
||||
};
|
||||
|
||||
extern template class CacheDictionary<DictionaryKeyType::simple>;
|
||||
extern template class CacheDictionary<DictionaryKeyType::complex>;
|
||||
extern template class CacheDictionary<DictionaryKeyType::Simple>;
|
||||
extern template class CacheDictionary<DictionaryKeyType::Complex>;
|
||||
|
||||
}
|
||||
|
@ -41,8 +41,7 @@ class CacheDictionaryStorage final : public ICacheDictionaryStorage
|
||||
static constexpr size_t max_collision_length = 10;
|
||||
|
||||
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 CacheDictionaryStorage");
|
||||
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
|
||||
|
||||
explicit CacheDictionaryStorage(
|
||||
const DictionaryStructure & dictionary_structure,
|
||||
@ -62,19 +61,19 @@ public:
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
return "Cache";
|
||||
else
|
||||
return "ComplexKeyCache";
|
||||
}
|
||||
|
||||
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::simple; }
|
||||
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::Simple; }
|
||||
|
||||
SimpleKeysStorageFetchResult fetchColumnsForKeys(
|
||||
const PaddedPODArray<UInt64> & keys,
|
||||
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);
|
||||
else
|
||||
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
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
insertColumnsForKeysImpl(keys, columns);
|
||||
else
|
||||
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
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
insertDefaultKeysImpl(keys);
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage");
|
||||
@ -98,19 +97,19 @@ public:
|
||||
|
||||
PaddedPODArray<UInt64> getCachedSimpleKeys() const override
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
return getCachedKeysImpl();
|
||||
else
|
||||
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(
|
||||
const PaddedPODArray<StringRef> & keys,
|
||||
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);
|
||||
else
|
||||
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
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
|
||||
insertColumnsForKeysImpl(keys, columns);
|
||||
else
|
||||
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
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
|
||||
insertDefaultKeysImpl(keys);
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage");
|
||||
@ -134,7 +133,7 @@ public:
|
||||
|
||||
PaddedPODArray<StringRef> getCachedComplexKeys() const override
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
|
||||
return getCachedKeysImpl();
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedComplexKeys is not supported for simple key storage");
|
||||
|
@ -14,8 +14,8 @@ namespace ErrorCodes
|
||||
extern const int TIMEOUT_EXCEEDED;
|
||||
}
|
||||
|
||||
template class CacheDictionaryUpdateUnit<DictionaryKeyType::simple>;
|
||||
template class CacheDictionaryUpdateUnit<DictionaryKeyType::complex>;
|
||||
template class CacheDictionaryUpdateUnit<DictionaryKeyType::Simple>;
|
||||
template class CacheDictionaryUpdateUnit<DictionaryKeyType::Complex>;
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
CacheDictionaryUpdateQueue<dictionary_key_type>::CacheDictionaryUpdateQueue(
|
||||
@ -155,7 +155,7 @@ void CacheDictionaryUpdateQueue<dictionary_key_type>::updateThreadFunction()
|
||||
}
|
||||
}
|
||||
|
||||
template class CacheDictionaryUpdateQueue<DictionaryKeyType::simple>;
|
||||
template class CacheDictionaryUpdateQueue<DictionaryKeyType::complex>;
|
||||
template class CacheDictionaryUpdateQueue<DictionaryKeyType::Simple>;
|
||||
template class CacheDictionaryUpdateQueue<DictionaryKeyType::Complex>;
|
||||
|
||||
}
|
||||
|
@ -39,7 +39,7 @@ template <DictionaryKeyType dictionary_key_type>
|
||||
class CacheDictionaryUpdateUnit
|
||||
{
|
||||
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
|
||||
explicit CacheDictionaryUpdateUnit(
|
||||
@ -85,8 +85,8 @@ private:
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
using CacheDictionaryUpdateUnitPtr = std::shared_ptr<CacheDictionaryUpdateUnit<dictionary_key_type>>;
|
||||
|
||||
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::simple>;
|
||||
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::complex>;
|
||||
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::Simple>;
|
||||
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::Complex>;
|
||||
|
||||
struct CacheDictionaryUpdateQueueConfiguration
|
||||
{
|
||||
@ -110,7 +110,6 @@ class CacheDictionaryUpdateQueue
|
||||
public:
|
||||
/// 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>)>;
|
||||
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionaryUpdateQueue");
|
||||
|
||||
CacheDictionaryUpdateQueue(
|
||||
String dictionary_name_for_logs_,
|
||||
@ -167,7 +166,7 @@ private:
|
||||
std::atomic<bool> finished{false};
|
||||
};
|
||||
|
||||
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::simple>;
|
||||
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::complex>;
|
||||
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::Simple>;
|
||||
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::Complex>;
|
||||
|
||||
}
|
||||
|
@ -380,14 +380,14 @@ template <DictionaryKeyType key_type>
|
||||
class DictionaryKeysArenaHolder;
|
||||
|
||||
template <>
|
||||
class DictionaryKeysArenaHolder<DictionaryKeyType::simple>
|
||||
class DictionaryKeysArenaHolder<DictionaryKeyType::Simple>
|
||||
{
|
||||
public:
|
||||
static Arena * getComplexKeyArena() { return nullptr; }
|
||||
};
|
||||
|
||||
template <>
|
||||
class DictionaryKeysArenaHolder<DictionaryKeyType::complex>
|
||||
class DictionaryKeysArenaHolder<DictionaryKeyType::Complex>
|
||||
{
|
||||
public:
|
||||
|
||||
@ -402,8 +402,7 @@ template <DictionaryKeyType key_type>
|
||||
class DictionaryKeysExtractor
|
||||
{
|
||||
public:
|
||||
using KeyType = std::conditional_t<key_type == DictionaryKeyType::simple, UInt64, StringRef>;
|
||||
static_assert(key_type != DictionaryKeyType::range, "Range key type is not supported by DictionaryKeysExtractor");
|
||||
using KeyType = std::conditional_t<key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
|
||||
|
||||
explicit DictionaryKeysExtractor(const Columns & key_columns_, Arena * complex_key_arena_)
|
||||
: key_columns(key_columns_)
|
||||
@ -411,7 +410,7 @@ public:
|
||||
{
|
||||
assert(!key_columns.empty());
|
||||
|
||||
if constexpr (key_type == DictionaryKeyType::simple)
|
||||
if constexpr (key_type == DictionaryKeyType::Simple)
|
||||
{
|
||||
key_columns[0] = key_columns[0]->convertToFullColumnIfConst();
|
||||
|
||||
@ -437,7 +436,7 @@ public:
|
||||
{
|
||||
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 & data = column_vector.getData();
|
||||
@ -465,7 +464,7 @@ public:
|
||||
|
||||
void rollbackCurrentKey() const
|
||||
{
|
||||
if constexpr (key_type == DictionaryKeyType::complex)
|
||||
if constexpr (key_type == DictionaryKeyType::Complex)
|
||||
complex_key_arena->rollback(current_complex_key.size);
|
||||
}
|
||||
|
||||
@ -521,8 +520,7 @@ void mergeBlockWithPipe(
|
||||
Block & block_to_update,
|
||||
Pipe pipe)
|
||||
{
|
||||
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");
|
||||
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
|
||||
|
||||
Columns saved_block_key_columns;
|
||||
saved_block_key_columns.reserve(key_columns_size);
|
||||
|
@ -132,7 +132,7 @@ Block DictionarySourceData::fillBlock(
|
||||
{
|
||||
ColumnPtr column;
|
||||
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
{
|
||||
column = dictionary->getColumn(
|
||||
attribute.name,
|
||||
|
@ -40,7 +40,7 @@ Columns DirectDictionary<dictionary_key_type>::getColumns(
|
||||
const DataTypes & key_types [[maybe_unused]],
|
||||
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);
|
||||
|
||||
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
|
||||
@ -161,7 +161,7 @@ ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::hasKeys(
|
||||
const Columns & key_columns,
|
||||
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);
|
||||
|
||||
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
|
||||
@ -230,7 +230,7 @@ ColumnPtr DirectDictionary<dictionary_key_type>::getHierarchy(
|
||||
ColumnPtr key_column,
|
||||
const DataTypePtr & key_type) const
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
{
|
||||
size_t 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,
|
||||
const DataTypePtr & key_type) const
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
{
|
||||
size_t keys_found = 0;
|
||||
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;
|
||||
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
{
|
||||
std::vector<UInt64> ids;
|
||||
ids.reserve(requested_keys_size);
|
||||
@ -310,9 +310,9 @@ namespace
|
||||
ContextPtr /* context */,
|
||||
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)
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
|
||||
@ -344,13 +344,13 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
template class DirectDictionary<DictionaryKeyType::simple>;
|
||||
template class DirectDictionary<DictionaryKeyType::complex>;
|
||||
template class DirectDictionary<DictionaryKeyType::Simple>;
|
||||
template class DirectDictionary<DictionaryKeyType::Complex>;
|
||||
|
||||
void registerDictionaryDirect(DictionaryFactory & factory)
|
||||
{
|
||||
factory.registerLayout("direct", createDirectDictionary<DictionaryKeyType::simple>, false);
|
||||
factory.registerLayout("complex_key_direct", createDirectDictionary<DictionaryKeyType::complex>, true);
|
||||
factory.registerLayout("direct", createDirectDictionary<DictionaryKeyType::Simple>, false);
|
||||
factory.registerLayout("complex_key_direct", createDirectDictionary<DictionaryKeyType::Complex>, true);
|
||||
}
|
||||
|
||||
|
||||
|
@ -20,8 +20,7 @@ template <DictionaryKeyType dictionary_key_type>
|
||||
class DirectDictionary final : public IDictionary
|
||||
{
|
||||
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(
|
||||
const StorageID & dict_id_,
|
||||
@ -30,7 +29,7 @@ public:
|
||||
|
||||
std::string getTypeName() const override
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
return "Direct";
|
||||
else
|
||||
return "ComplexKeyDirect";
|
||||
@ -110,7 +109,7 @@ private:
|
||||
mutable std::atomic<size_t> found_count{0};
|
||||
};
|
||||
|
||||
extern template class DirectDictionary<DictionaryKeyType::simple>;
|
||||
extern template class DirectDictionary<DictionaryKeyType::complex>;
|
||||
extern template class DirectDictionary<DictionaryKeyType::Simple>;
|
||||
extern template class DirectDictionary<DictionaryKeyType::Complex>;
|
||||
|
||||
}
|
||||
|
@ -289,8 +289,8 @@ void FlatDictionary::blockToAttributes(const Block & block)
|
||||
{
|
||||
const auto keys_column = block.safeGetByPosition(0).column;
|
||||
|
||||
DictionaryKeysArenaHolder<DictionaryKeyType::simple> arena_holder;
|
||||
DictionaryKeysExtractor<DictionaryKeyType::simple> keys_extractor({ keys_column }, arena_holder.getComplexKeyArena());
|
||||
DictionaryKeysArenaHolder<DictionaryKeyType::Simple> arena_holder;
|
||||
DictionaryKeysExtractor<DictionaryKeyType::Simple> keys_extractor({ keys_column }, arena_holder.getComplexKeyArena());
|
||||
auto keys = keys_extractor.extractAllKeys();
|
||||
|
||||
HashSet<UInt64> already_processed_keys;
|
||||
@ -344,7 +344,7 @@ void FlatDictionary::updateData()
|
||||
else
|
||||
{
|
||||
Pipe pipe(source_ptr->loadUpdatedAll());
|
||||
mergeBlockWithPipe<DictionaryKeyType::simple>(
|
||||
mergeBlockWithPipe<DictionaryKeyType::Simple>(
|
||||
dict_struct.getKeysSize(),
|
||||
*update_field_loaded_block,
|
||||
std::move(pipe));
|
||||
|
@ -72,7 +72,7 @@ public:
|
||||
return dict_struct.getAttribute(attribute_name).injective;
|
||||
}
|
||||
|
||||
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::simple; }
|
||||
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Simple; }
|
||||
|
||||
ColumnPtr getColumn(
|
||||
const std::string& attribute_name,
|
||||
|
@ -61,7 +61,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getColumn(
|
||||
const DataTypes & key_types [[maybe_unused]],
|
||||
const ColumnPtr & default_values_column) const
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::complex)
|
||||
if (dictionary_key_type == DictionaryKeyType::Complex)
|
||||
dict_struct.validateKeyTypes(key_types);
|
||||
|
||||
ColumnPtr result;
|
||||
@ -163,7 +163,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getColumn(
|
||||
template <DictionaryKeyType dictionary_key_type, bool sparse>
|
||||
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);
|
||||
|
||||
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>
|
||||
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;
|
||||
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]],
|
||||
const DataTypePtr &) const
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
{
|
||||
PaddedPODArray<UInt64> 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 &,
|
||||
size_t level [[maybe_unused]]) const
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
{
|
||||
PaddedPODArray<UInt64> 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));
|
||||
else
|
||||
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, false>;
|
||||
template class HashedDictionary<DictionaryKeyType::complex, true>;
|
||||
template class HashedDictionary<DictionaryKeyType::complex, false>;
|
||||
template class HashedDictionary<DictionaryKeyType::Simple, true>;
|
||||
template class HashedDictionary<DictionaryKeyType::Simple, false>;
|
||||
template class HashedDictionary<DictionaryKeyType::Complex, true>;
|
||||
template class HashedDictionary<DictionaryKeyType::Complex, false>;
|
||||
|
||||
void registerDictionaryHashed(DictionaryFactory & factory)
|
||||
{
|
||||
@ -717,9 +717,9 @@ void registerDictionaryHashed(DictionaryFactory & factory)
|
||||
DictionaryKeyType dictionary_key_type,
|
||||
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");
|
||||
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");
|
||||
|
||||
if (dict_struct.range_min || dict_struct.range_max)
|
||||
@ -737,32 +737,32 @@ void registerDictionaryHashed(DictionaryFactory & factory)
|
||||
|
||||
HashedDictionaryStorageConfiguration configuration{preallocate, require_nonempty, dict_lifetime};
|
||||
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
{
|
||||
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
|
||||
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
|
||||
{
|
||||
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
|
||||
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;
|
||||
|
||||
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",
|
||||
[=](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",
|
||||
[=](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",
|
||||
[=](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);
|
||||
|
||||
}
|
||||
|
||||
|
@ -35,8 +35,7 @@ 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");
|
||||
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
|
||||
|
||||
HashedDictionary(
|
||||
const StorageID & dict_id_,
|
||||
@ -47,11 +46,11 @@ public:
|
||||
|
||||
std::string getTypeName() const override
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple && sparse)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple && sparse)
|
||||
return "SparseHashed";
|
||||
else if constexpr (dictionary_key_type == DictionaryKeyType::simple && !sparse)
|
||||
else if constexpr (dictionary_key_type == DictionaryKeyType::Simple && !sparse)
|
||||
return "Hashed";
|
||||
else if constexpr (dictionary_key_type == DictionaryKeyType::complex && sparse)
|
||||
else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && sparse)
|
||||
return "ComplexKeySparseHashed";
|
||||
else
|
||||
return "ComplexKeyHashed";
|
||||
@ -102,7 +101,7 @@ public:
|
||||
|
||||
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;
|
||||
|
||||
@ -121,13 +120,13 @@ public:
|
||||
private:
|
||||
template <typename Value>
|
||||
using CollectionTypeNonSparse = std::conditional_t<
|
||||
dictionary_key_type == DictionaryKeyType::simple,
|
||||
dictionary_key_type == DictionaryKeyType::Simple,
|
||||
HashMap<UInt64, Value>,
|
||||
HashMapWithSavedHash<StringRef, Value, DefaultHash<StringRef>>>;
|
||||
|
||||
template <typename Value>
|
||||
using CollectionTypeSparse = std::conditional_t<
|
||||
dictionary_key_type == DictionaryKeyType::simple,
|
||||
dictionary_key_type == DictionaryKeyType::Simple,
|
||||
SparseHashMap<UInt64, Value>,
|
||||
SparseHashMap<StringRef, Value>>;
|
||||
|
||||
@ -211,10 +210,10 @@ private:
|
||||
Arena complex_key_arena;
|
||||
};
|
||||
|
||||
extern template class HashedDictionary<DictionaryKeyType::simple, false>;
|
||||
extern template class HashedDictionary<DictionaryKeyType::simple, true>;
|
||||
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>;
|
||||
extern template class HashedDictionary<DictionaryKeyType::Complex, false>;
|
||||
extern template class HashedDictionary<DictionaryKeyType::Complex, true>;
|
||||
|
||||
}
|
||||
|
@ -33,15 +33,20 @@ using DictionaryPtr = std::unique_ptr<IDictionary>;
|
||||
* Simple is for dictionaries that support UInt64 key column.
|
||||
*
|
||||
* 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
|
||||
{
|
||||
simple,
|
||||
complex,
|
||||
range
|
||||
Simple,
|
||||
Complex
|
||||
};
|
||||
|
||||
/** 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; }
|
||||
|
||||
StorageID getDictionaryID() const
|
||||
{
|
||||
std::lock_guard lock{name_mutex};
|
||||
@ -109,6 +115,8 @@ struct IDictionary : public IExternalLoadable
|
||||
*/
|
||||
virtual DictionaryKeyType getKeyType() const = 0;
|
||||
|
||||
virtual DictionarySpecialKeyType getSpecialKeyType() const { return DictionarySpecialKeyType::None;}
|
||||
|
||||
/** Subclass must validate key columns and keys types
|
||||
* and return column representation of dictionary attribute.
|
||||
*
|
||||
|
@ -67,7 +67,7 @@ public:
|
||||
return dict_struct.getAttribute(attribute_name).injective;
|
||||
}
|
||||
|
||||
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; }
|
||||
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Complex; }
|
||||
|
||||
ColumnPtr getColumn(
|
||||
const std::string& attribute_name,
|
||||
|
@ -86,7 +86,7 @@ public:
|
||||
|
||||
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(
|
||||
const std::string& attribute_name,
|
||||
|
@ -14,18 +14,12 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
enum class RangeDictionaryType
|
||||
{
|
||||
simple,
|
||||
complex
|
||||
};
|
||||
|
||||
template <RangeDictionaryType range_dictionary_type, typename RangeType>
|
||||
template <DictionaryKeyType dictionary_key_type, typename RangeType>
|
||||
class RangeDictionarySourceData
|
||||
{
|
||||
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(
|
||||
std::shared_ptr<const IDictionary> dictionary,
|
||||
@ -58,8 +52,8 @@ private:
|
||||
};
|
||||
|
||||
|
||||
template <RangeDictionaryType range_dictionary_type, typename RangeType>
|
||||
RangeDictionarySourceData<range_dictionary_type, RangeType>::RangeDictionarySourceData(
|
||||
template <DictionaryKeyType dictionary_key_type, typename RangeType>
|
||||
RangeDictionarySourceData<dictionary_key_type, RangeType>::RangeDictionarySourceData(
|
||||
std::shared_ptr<const IDictionary> dictionary_,
|
||||
const Names & column_names_,
|
||||
PaddedPODArray<KeyType> && keys,
|
||||
@ -73,8 +67,8 @@ RangeDictionarySourceData<range_dictionary_type, RangeType>::RangeDictionarySour
|
||||
{
|
||||
}
|
||||
|
||||
template <RangeDictionaryType range_dictionary_type, typename RangeType>
|
||||
Block RangeDictionarySourceData<range_dictionary_type, RangeType>::getBlock(size_t start, size_t length) const
|
||||
template <DictionaryKeyType dictionary_key_type, typename RangeType>
|
||||
Block RangeDictionarySourceData<dictionary_key_type, RangeType>::getBlock(size_t start, size_t length) const
|
||||
{
|
||||
PaddedPODArray<KeyType> block_keys;
|
||||
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);
|
||||
}
|
||||
|
||||
template <RangeDictionaryType range_dictionary_type, typename RangeType>
|
||||
PaddedPODArray<Int64> RangeDictionarySourceData<range_dictionary_type, RangeType>::makeDateKeys(
|
||||
template <DictionaryKeyType dictionary_key_type, typename RangeType>
|
||||
PaddedPODArray<Int64> RangeDictionarySourceData<dictionary_key_type, RangeType>::makeDateKeys(
|
||||
const PaddedPODArray<RangeType> & block_start_dates,
|
||||
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>
|
||||
Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
|
||||
template <DictionaryKeyType dictionary_key_type, typename RangeType>
|
||||
Block RangeDictionarySourceData<dictionary_key_type, RangeType>::fillBlock(
|
||||
const PaddedPODArray<KeyType> & keys_to_fill,
|
||||
const PaddedPODArray<RangeType> & block_start_dates,
|
||||
const PaddedPODArray<RangeType> & block_end_dates,
|
||||
size_t start,
|
||||
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;
|
||||
const DictionaryStructure & dictionary_structure = dictionary->getStructure();
|
||||
|
||||
@ -137,7 +121,7 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
|
||||
Columns keys_columns;
|
||||
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_types = {std::make_shared<DataTypeUInt64>()};
|
||||
@ -154,9 +138,6 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
|
||||
|
||||
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_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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
template <RangeDictionaryType range_dictionary_type, typename RangeType>
|
||||
template <DictionaryKeyType dictionary_key_type, typename RangeType>
|
||||
class RangeDictionarySource : public DictionarySourceBase
|
||||
{
|
||||
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"; }
|
||||
|
||||
protected:
|
||||
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>
|
||||
RangeDictionarySource<range_dictionary_type, RangeType>::RangeDictionarySource(RangeDictionarySourceData<range_dictionary_type, RangeType> data_, size_t max_block_size)
|
||||
template <DictionaryKeyType dictionary_key_type, typename RangeType>
|
||||
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)
|
||||
, data(std::move(data_))
|
||||
{
|
||||
}
|
||||
|
||||
template <RangeDictionaryType range_dictionary_type, typename RangeType>
|
||||
Block RangeDictionarySource<range_dictionary_type, RangeType>::getBlock(size_t start, size_t length) const
|
||||
template <DictionaryKeyType dictionary_key_type, typename RangeType>
|
||||
Block RangeDictionarySource<dictionary_key_type, RangeType>::getBlock(size_t start, size_t length) const
|
||||
{
|
||||
return data.getBlock(start, length);
|
||||
}
|
||||
|
@ -95,6 +95,13 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::getColumn(
|
||||
const DataTypes & key_types,
|
||||
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;
|
||||
|
||||
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>
|
||||
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_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);
|
||||
PaddedPODArray<RangeStorageType> 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);
|
||||
}
|
||||
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
|
||||
bytes_allocated += complex_key_arena.size();
|
||||
}
|
||||
|
||||
@ -607,10 +620,9 @@ Pipe RangeHashedDictionary<dictionary_key_type>::readImpl(const Names & column_n
|
||||
PaddedPODArray<RangeType> 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<range_dictionary_type, RangeType>;
|
||||
using RangeDictionarySourceType = RangeDictionarySource<dictionary_key_type, RangeType>;
|
||||
|
||||
auto source_data = RangeDictionarySourceData<range_dictionary_type, RangeType>(
|
||||
auto source_data = RangeDictionarySourceData<dictionary_key_type, RangeType>(
|
||||
shared_from_this(),
|
||||
column_names,
|
||||
std::move(keys),
|
||||
@ -690,7 +702,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
|
||||
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<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);
|
||||
|
||||
@ -713,7 +725,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
|
||||
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<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);
|
||||
}
|
||||
|
@ -32,8 +32,7 @@ template <DictionaryKeyType dictionary_key_type>
|
||||
class RangeHashedDictionary 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");
|
||||
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
|
||||
|
||||
RangeHashedDictionary(
|
||||
const StorageID & dict_id_,
|
||||
@ -78,7 +77,9 @@ public:
|
||||
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(
|
||||
const std::string& attribute_name,
|
||||
@ -104,7 +105,7 @@ private:
|
||||
|
||||
template <typename Value>
|
||||
using CollectionType = std::conditional_t<
|
||||
dictionary_key_type == DictionaryKeyType::simple,
|
||||
dictionary_key_type == DictionaryKeyType::Simple,
|
||||
HashMap<UInt64, Values<Value>>,
|
||||
HashMapWithSavedHash<StringRef, Values<Value>, DefaultHash<StringRef>>>;
|
||||
|
||||
|
@ -823,8 +823,8 @@ template <DictionaryKeyType dictionary_key_type>
|
||||
class SSDCacheDictionaryStorage final : public ICacheDictionaryStorage
|
||||
{
|
||||
public:
|
||||
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 SSDCacheKeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, SSDCacheSimpleKey, SSDCacheComplexKey>;
|
||||
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
|
||||
|
||||
explicit SSDCacheDictionaryStorage(const SSDCacheDictionaryStorageConfiguration & configuration_)
|
||||
: configuration(configuration_)
|
||||
@ -838,19 +838,19 @@ public:
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
return "SSDCache";
|
||||
else
|
||||
return "SSDComplexKeyCache";
|
||||
}
|
||||
|
||||
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::simple; }
|
||||
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::Simple; }
|
||||
|
||||
SimpleKeysStorageFetchResult fetchColumnsForKeys(
|
||||
const PaddedPODArray<UInt64> & keys,
|
||||
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);
|
||||
else
|
||||
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
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
insertColumnsForKeysImpl(keys, columns);
|
||||
else
|
||||
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
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
insertDefaultKeysImpl(keys);
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage");
|
||||
@ -874,19 +874,19 @@ public:
|
||||
|
||||
PaddedPODArray<UInt64> getCachedSimpleKeys() const override
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
return getCachedKeysImpl();
|
||||
else
|
||||
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(
|
||||
const PaddedPODArray<StringRef> & keys,
|
||||
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);
|
||||
else
|
||||
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
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
|
||||
insertColumnsForKeysImpl(keys, columns);
|
||||
else
|
||||
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
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
|
||||
insertDefaultKeysImpl(keys);
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage");
|
||||
@ -910,7 +910,7 @@ public:
|
||||
|
||||
PaddedPODArray<StringRef> getCachedComplexKeys() const override
|
||||
{
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
|
||||
return getCachedKeysImpl();
|
||||
else
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for simple key storage");
|
||||
@ -1134,7 +1134,7 @@ private:
|
||||
Cell cell;
|
||||
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
|
||||
size_t key_size = key.size;
|
||||
@ -1166,7 +1166,7 @@ private:
|
||||
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
|
||||
size_t key_size = key.size;
|
||||
@ -1382,7 +1382,7 @@ private:
|
||||
using ComplexKeyHashMap = HashMapWithSavedHash<StringRef, Cell>;
|
||||
|
||||
using CacheMap = std::conditional_t<
|
||||
dictionary_key_type == DictionaryKeyType::simple,
|
||||
dictionary_key_type == DictionaryKeyType::Simple,
|
||||
SimpleKeyHashMap,
|
||||
ComplexKeyHashMap>;
|
||||
|
||||
|
@ -157,24 +157,23 @@ DictionaryPtr createCacheDictionaryLayout(
|
||||
ContextPtr context [[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;
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple && !ssd)
|
||||
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple && !ssd)
|
||||
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";
|
||||
else if constexpr (dictionary_key_type == DictionaryKeyType::complex && !ssd)
|
||||
else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && !ssd)
|
||||
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";
|
||||
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
{
|
||||
if (dict_struct.key)
|
||||
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)
|
||||
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,
|
||||
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);
|
||||
@ -256,7 +255,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
ContextPtr context,
|
||||
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);
|
||||
@ -271,7 +270,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
ContextPtr context,
|
||||
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);
|
||||
@ -283,7 +282,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr context,
|
||||
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);
|
||||
|
@ -182,15 +182,16 @@ public:
|
||||
|
||||
auto dictionary = helper.getDictionary(arguments[0].column);
|
||||
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_type = key_column_with_type.type;
|
||||
|
||||
ColumnPtr range_col = nullptr;
|
||||
DataTypePtr range_col_type = nullptr;
|
||||
ColumnPtr range_col;
|
||||
DataTypePtr range_col_type;
|
||||
|
||||
if (dictionary_key_type == DictionaryKeyType::range)
|
||||
if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
|
||||
{
|
||||
if (arguments.size() != 3)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
@ -207,7 +208,10 @@ public:
|
||||
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())
|
||||
throw Exception(
|
||||
@ -216,16 +220,23 @@ public:
|
||||
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.
|
||||
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)
|
||||
{
|
||||
throw Exception(
|
||||
@ -237,41 +248,24 @@ public:
|
||||
}
|
||||
else
|
||||
{
|
||||
Columns tuple_columns = {std::move(key_column)};
|
||||
key_column = ColumnTuple::create(tuple_columns);
|
||||
|
||||
DataTypes tuple_types = {key_column_type};
|
||||
key_column_type = std::make_shared<DataTypeTuple>(tuple_types);
|
||||
key_columns = {key_column};
|
||||
key_types = {key_column_type};
|
||||
}
|
||||
}
|
||||
|
||||
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.
|
||||
ColumnPtr key_column = key_column_with_type.column->convertToFullColumnIfConst();
|
||||
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});
|
||||
key_columns.emplace_back(range_col);
|
||||
key_types.emplace_back(range_col_type);
|
||||
}
|
||||
|
||||
std::cerr << "FunctionDictHas::executeImpl" << std::endl;
|
||||
|
||||
for (auto & key_type : key_types)
|
||||
std::cerr << "Key type " << key_type->getName() << std::endl;
|
||||
|
||||
return dictionary->hasKeys(key_columns, key_types);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -369,13 +363,14 @@ public:
|
||||
|
||||
auto dictionary = helper.getDictionary(dictionary_name);
|
||||
auto dictionary_key_type = dictionary->getKeyType();
|
||||
auto dictionary_special_key_type = dictionary->getSpecialKeyType();
|
||||
|
||||
size_t current_arguments_index = 3;
|
||||
|
||||
ColumnPtr range_col = 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())
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
@ -437,12 +432,13 @@ public:
|
||||
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];
|
||||
const auto key_column = key_col_with_type.column;
|
||||
Columns key_columns;
|
||||
DataTypes key_types;
|
||||
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
if (dictionary_key_type == DictionaryKeyType::Simple)
|
||||
{
|
||||
if (!WhichDataType(key_col_with_type.type).isUInt64())
|
||||
throw Exception(
|
||||
@ -451,24 +447,24 @@ public:
|
||||
getName(),
|
||||
key_col_with_type.type->getName());
|
||||
|
||||
result = executeDictionaryRequest(
|
||||
dictionary,
|
||||
attribute_names,
|
||||
{key_column},
|
||||
{std::make_shared<DataTypeUInt64>()},
|
||||
result_type,
|
||||
default_cols);
|
||||
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.
|
||||
ColumnPtr key_column = key_col_with_type.column->convertToFullColumnIfConst();
|
||||
key_column = key_column->convertToFullColumnIfConst();
|
||||
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)
|
||||
{
|
||||
throw Exception(
|
||||
@ -480,60 +476,19 @@ public:
|
||||
}
|
||||
else
|
||||
{
|
||||
Columns tuple_columns = {std::move(key_column)};
|
||||
key_column = ColumnTuple::create(tuple_columns);
|
||||
|
||||
DataTypes tuple_types = {key_column_type};
|
||||
key_column_type = std::make_shared<DataTypeTuple>(tuple_types);
|
||||
key_columns = {std::move(key_column)};
|
||||
key_types = {std::move(key_column_type)};
|
||||
}
|
||||
}
|
||||
|
||||
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_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:
|
||||
|
Loading…
Reference in New Issue
Block a user