mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #33927 from kitaisreal/range-hashed-dictionary-refactoring
RangeHashedDictionary refactoring
This commit is contained in:
commit
662ea9d024
@ -291,6 +291,15 @@ public:
|
|||||||
|
|
||||||
size_t getIntervalsSize() const { return intervals_size; }
|
size_t getIntervalsSize() const { return intervals_size; }
|
||||||
|
|
||||||
|
size_t getSizeInBytes() const
|
||||||
|
{
|
||||||
|
size_t nodes_size_in_bytes = nodes.size() * sizeof(Node);
|
||||||
|
size_t intervals_size_in_bytes = sorted_intervals.size() * sizeof(IntervalWithValue);
|
||||||
|
size_t result = nodes_size_in_bytes + intervals_size_in_bytes;
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
struct Node
|
struct Node
|
||||||
{
|
{
|
||||||
|
@ -523,6 +523,7 @@ inline bool isBool(const DataTypePtr & data_type)
|
|||||||
template <typename DataType> constexpr bool IsDataTypeDecimal = false;
|
template <typename DataType> constexpr bool IsDataTypeDecimal = false;
|
||||||
template <typename DataType> constexpr bool IsDataTypeNumber = false;
|
template <typename DataType> constexpr bool IsDataTypeNumber = false;
|
||||||
template <typename DataType> constexpr bool IsDataTypeDateOrDateTime = false;
|
template <typename DataType> constexpr bool IsDataTypeDateOrDateTime = false;
|
||||||
|
template <typename DataType> constexpr bool IsDataTypeEnum = false;
|
||||||
|
|
||||||
template <typename DataType> constexpr bool IsDataTypeDecimalOrNumber = IsDataTypeDecimal<DataType> || IsDataTypeNumber<DataType>;
|
template <typename DataType> constexpr bool IsDataTypeDecimalOrNumber = IsDataTypeDecimal<DataType> || IsDataTypeNumber<DataType>;
|
||||||
|
|
||||||
@ -547,4 +548,9 @@ template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDate32> = tru
|
|||||||
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime> = true;
|
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime> = true;
|
||||||
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime64> = true;
|
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime64> = true;
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
class DataTypeEnum;
|
||||||
|
|
||||||
|
template <typename T> inline constexpr bool IsDataTypeEnum<DataTypeEnum<T>> = true;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -60,8 +60,8 @@ private:
|
|||||||
const auto & attributes_types_to_read = coordinator->getAttributesTypesToRead();
|
const auto & attributes_types_to_read = coordinator->getAttributesTypesToRead();
|
||||||
const auto & attributes_default_values_columns = coordinator->getAttributesDefaultValuesColumns();
|
const auto & attributes_default_values_columns = coordinator->getAttributesDefaultValuesColumns();
|
||||||
|
|
||||||
const auto & dictionary = coordinator->getDictionary();
|
const auto & read_columns_func = coordinator->getReadColumnsFunc();
|
||||||
auto attributes_columns = dictionary->getColumns(
|
auto attributes_columns = read_columns_func(
|
||||||
attributes_names_to_read,
|
attributes_names_to_read,
|
||||||
attributes_types_to_read,
|
attributes_types_to_read,
|
||||||
key_columns,
|
key_columns,
|
||||||
|
@ -19,6 +19,8 @@ class DictionarySourceCoordinator final : public shared_ptr_helper<DictionarySou
|
|||||||
|
|
||||||
public:
|
public:
|
||||||
|
|
||||||
|
using ReadColumnsFunc = std::function<Columns (const Strings &, const DataTypes &, const Columns &, const DataTypes &, const Columns &)>;
|
||||||
|
|
||||||
Pipe read(size_t num_streams);
|
Pipe read(size_t num_streams);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
@ -31,6 +33,15 @@ private:
|
|||||||
: dictionary(std::move(dictionary_))
|
: dictionary(std::move(dictionary_))
|
||||||
, key_columns_with_type(std::move(key_columns_with_type_))
|
, key_columns_with_type(std::move(key_columns_with_type_))
|
||||||
, max_block_size(max_block_size_)
|
, max_block_size(max_block_size_)
|
||||||
|
, read_columns_func([this](
|
||||||
|
const Strings & attribute_names,
|
||||||
|
const DataTypes & result_types,
|
||||||
|
const Columns & key_columns,
|
||||||
|
const DataTypes & key_types,
|
||||||
|
const Columns & default_values_columns)
|
||||||
|
{
|
||||||
|
return dictionary->getColumns(attribute_names, result_types, key_columns, key_types, default_values_columns);
|
||||||
|
})
|
||||||
{
|
{
|
||||||
initialize(column_names);
|
initialize(column_names);
|
||||||
}
|
}
|
||||||
@ -45,6 +56,31 @@ private:
|
|||||||
, key_columns_with_type(std::move(key_columns_with_type_))
|
, key_columns_with_type(std::move(key_columns_with_type_))
|
||||||
, data_columns_with_type(std::move(data_columns_with_type_))
|
, data_columns_with_type(std::move(data_columns_with_type_))
|
||||||
, max_block_size(max_block_size_)
|
, max_block_size(max_block_size_)
|
||||||
|
, read_columns_func([this](
|
||||||
|
const Strings & attribute_names,
|
||||||
|
const DataTypes & result_types,
|
||||||
|
const Columns & key_columns,
|
||||||
|
const DataTypes & key_types,
|
||||||
|
const Columns & default_values_columns)
|
||||||
|
{
|
||||||
|
return dictionary->getColumns(attribute_names, result_types, key_columns, key_types, default_values_columns);
|
||||||
|
})
|
||||||
|
{
|
||||||
|
initialize(column_names);
|
||||||
|
}
|
||||||
|
|
||||||
|
explicit DictionarySourceCoordinator(
|
||||||
|
std::shared_ptr<const IDictionary> dictionary_,
|
||||||
|
const Names & column_names,
|
||||||
|
ColumnsWithTypeAndName && key_columns_with_type_,
|
||||||
|
ColumnsWithTypeAndName && data_columns_with_type_,
|
||||||
|
size_t max_block_size_,
|
||||||
|
ReadColumnsFunc read_columns_func_)
|
||||||
|
: dictionary(std::move(dictionary_))
|
||||||
|
, key_columns_with_type(std::move(key_columns_with_type_))
|
||||||
|
, data_columns_with_type(std::move(data_columns_with_type_))
|
||||||
|
, max_block_size(max_block_size_)
|
||||||
|
, read_columns_func(std::move(read_columns_func_))
|
||||||
{
|
{
|
||||||
initialize(column_names);
|
initialize(column_names);
|
||||||
}
|
}
|
||||||
@ -61,6 +97,8 @@ private:
|
|||||||
|
|
||||||
const std::vector<ColumnPtr> & getAttributesDefaultValuesColumns() const { return attributes_default_values_columns; }
|
const std::vector<ColumnPtr> & getAttributesDefaultValuesColumns() const { return attributes_default_values_columns; }
|
||||||
|
|
||||||
|
const ReadColumnsFunc & getReadColumnsFunc() const { return read_columns_func; }
|
||||||
|
|
||||||
const std::shared_ptr<const IDictionary> & getDictionary() const { return dictionary; }
|
const std::shared_ptr<const IDictionary> & getDictionary() const { return dictionary; }
|
||||||
|
|
||||||
void initialize(const Names & column_names);
|
void initialize(const Names & column_names);
|
||||||
@ -79,6 +117,8 @@ private:
|
|||||||
std::vector<ColumnPtr> attributes_default_values_columns;
|
std::vector<ColumnPtr> attributes_default_values_columns;
|
||||||
|
|
||||||
const size_t max_block_size;
|
const size_t max_block_size;
|
||||||
|
ReadColumnsFunc read_columns_func;
|
||||||
|
|
||||||
std::atomic<size_t> parallel_read_block_index = 0;
|
std::atomic<size_t> parallel_read_block_index = 0;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -382,7 +382,8 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
|
|||||||
|
|
||||||
void DictionaryStructure::parseRangeConfiguration(const Poco::Util::AbstractConfiguration & config, const std::string & structure_prefix)
|
void DictionaryStructure::parseRangeConfiguration(const Poco::Util::AbstractConfiguration & config, const std::string & structure_prefix)
|
||||||
{
|
{
|
||||||
const char * range_default_type = "Date";
|
static constexpr auto range_default_type = "Date";
|
||||||
|
|
||||||
if (config.has(structure_prefix + ".range_min"))
|
if (config.has(structure_prefix + ".range_min"))
|
||||||
range_min.emplace(makeDictionaryTypedSpecialAttribute(config, structure_prefix + ".range_min", range_default_type));
|
range_min.emplace(makeDictionaryTypedSpecialAttribute(config, structure_prefix + ".range_min", range_default_type));
|
||||||
|
|
||||||
@ -395,7 +396,10 @@ void DictionaryStructure::parseRangeConfiguration(const Poco::Util::AbstractConf
|
|||||||
"Dictionary structure should have both 'range_min' and 'range_max' either specified or not.");
|
"Dictionary structure should have both 'range_min' and 'range_max' either specified or not.");
|
||||||
}
|
}
|
||||||
|
|
||||||
if (range_min && range_max && !range_min->type->equals(*range_max->type))
|
if (!range_min)
|
||||||
|
return;
|
||||||
|
|
||||||
|
if (!range_min->type->equals(*range_max->type))
|
||||||
{
|
{
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||||
"Dictionary structure 'range_min' and 'range_max' should have same type, "
|
"Dictionary structure 'range_min' and 'range_max' should have same type, "
|
||||||
@ -405,15 +409,20 @@ void DictionaryStructure::parseRangeConfiguration(const Poco::Util::AbstractConf
|
|||||||
range_max->type->getName());
|
range_max->type->getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
if (range_min && !range_min->type->isValueRepresentedByInteger())
|
WhichDataType range_type(range_min->type);
|
||||||
|
|
||||||
|
bool valid_range = range_type.isInt() || range_type.isUInt() || range_type.isDecimal() || range_type.isFloat() || range_type.isEnum()
|
||||||
|
|| range_type.isDate() || range_type.isDate32() || range_type.isDateTime() || range_type.isDateTime64();
|
||||||
|
|
||||||
|
if (!valid_range)
|
||||||
{
|
{
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||||
"Dictionary structure type of 'range_min' and 'range_max' should be an integer, Date, DateTime, or Enum."
|
"Dictionary structure type of 'range_min' and 'range_max' should be an Integer, Float, Decimal, Date, Date32, DateTime DateTime64, or Enum."
|
||||||
" Actual 'range_min' and 'range_max' type is {}",
|
" Actual 'range_min' and 'range_max' type is {}",
|
||||||
range_min->type->getName());
|
range_min->type->getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
if ((range_min && !range_min->expression.empty()) || (range_max && !range_max->expression.empty()))
|
if (!range_min->expression.empty() || !range_max->expression.empty())
|
||||||
has_expressions = true;
|
has_expressions = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
File diff suppressed because it is too large
Load Diff
@ -19,7 +19,18 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
using RangeStorageType = Int64;
|
enum class RangeHashedDictionaryLookupStrategy : uint8_t
|
||||||
|
{
|
||||||
|
min,
|
||||||
|
max
|
||||||
|
};
|
||||||
|
|
||||||
|
struct RangeHashedDictionaryConfiguration
|
||||||
|
{
|
||||||
|
bool convert_null_range_bound_to_open;
|
||||||
|
RangeHashedDictionaryLookupStrategy lookup_strategy;
|
||||||
|
bool require_nonempty;
|
||||||
|
};
|
||||||
|
|
||||||
template <DictionaryKeyType dictionary_key_type>
|
template <DictionaryKeyType dictionary_key_type>
|
||||||
class RangeHashedDictionary final : public IDictionary
|
class RangeHashedDictionary final : public IDictionary
|
||||||
@ -31,11 +42,17 @@ public:
|
|||||||
const StorageID & dict_id_,
|
const StorageID & dict_id_,
|
||||||
const DictionaryStructure & dict_struct_,
|
const DictionaryStructure & dict_struct_,
|
||||||
DictionarySourcePtr source_ptr_,
|
DictionarySourcePtr source_ptr_,
|
||||||
const DictionaryLifetime dict_lifetime_,
|
DictionaryLifetime dict_lifetime_,
|
||||||
bool require_nonempty_,
|
RangeHashedDictionaryConfiguration configuration_,
|
||||||
BlockPtr update_field_loaded_block_ = nullptr);
|
BlockPtr update_field_loaded_block_ = nullptr);
|
||||||
|
|
||||||
std::string getTypeName() const override { return "RangeHashed"; }
|
std::string getTypeName() const override
|
||||||
|
{
|
||||||
|
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
|
||||||
|
return "RangeHashed";
|
||||||
|
else
|
||||||
|
return "ComplexKeyRangeHashed";
|
||||||
|
}
|
||||||
|
|
||||||
size_t getBytesAllocated() const override { return bytes_allocated; }
|
size_t getBytesAllocated() const override { return bytes_allocated; }
|
||||||
|
|
||||||
@ -57,7 +74,15 @@ public:
|
|||||||
|
|
||||||
std::shared_ptr<const IExternalLoadable> clone() const override
|
std::shared_ptr<const IExternalLoadable> clone() const override
|
||||||
{
|
{
|
||||||
return std::make_shared<RangeHashedDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, update_field_loaded_block);
|
auto result = std::make_shared<RangeHashedDictionary>(
|
||||||
|
getDictionaryID(),
|
||||||
|
dict_struct,
|
||||||
|
source_ptr->clone(),
|
||||||
|
dict_lifetime,
|
||||||
|
configuration,
|
||||||
|
update_field_loaded_block);
|
||||||
|
|
||||||
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
DictionarySourcePtr getSource() const override { return source_ptr; }
|
DictionarySourcePtr getSource() const override { return source_ptr; }
|
||||||
@ -76,7 +101,7 @@ public:
|
|||||||
DictionarySpecialKeyType getSpecialKeyType() const override { return DictionarySpecialKeyType::Range;}
|
DictionarySpecialKeyType getSpecialKeyType() const override { return DictionarySpecialKeyType::Range;}
|
||||||
|
|
||||||
ColumnPtr getColumn(
|
ColumnPtr getColumn(
|
||||||
const std::string& attribute_name,
|
const std::string & attribute_name,
|
||||||
const DataTypePtr & result_type,
|
const DataTypePtr & result_type,
|
||||||
const Columns & key_columns,
|
const Columns & key_columns,
|
||||||
const DataTypes & key_types,
|
const DataTypes & key_types,
|
||||||
@ -88,52 +113,90 @@ public:
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
|
|
||||||
using RangeInterval = Interval<RangeStorageType>;
|
template <typename RangeStorageType>
|
||||||
|
using IntervalMap = IntervalMap<Interval<RangeStorageType>, size_t>;
|
||||||
|
|
||||||
template <typename T>
|
template <typename RangeStorageType>
|
||||||
using Values = IntervalMap<RangeInterval, std::optional<T>>;
|
using KeyAttributeContainerType = std::conditional_t<
|
||||||
|
dictionary_key_type == DictionaryKeyType::Simple,
|
||||||
|
HashMap<UInt64, IntervalMap<RangeStorageType>, DefaultHash<UInt64>>,
|
||||||
|
HashMapWithSavedHash<StringRef, IntervalMap<RangeStorageType>, DefaultHash<StringRef>>>;
|
||||||
|
|
||||||
template <typename Value>
|
template <typename Value>
|
||||||
using CollectionType = std::conditional_t<
|
using AttributeContainerType = std::conditional_t<std::is_same_v<Value, Array>, std::vector<Value>, PaddedPODArray<Value>>;
|
||||||
dictionary_key_type == DictionaryKeyType::Simple,
|
|
||||||
HashMap<UInt64, Values<Value>, DefaultHash<UInt64>>,
|
|
||||||
HashMapWithSavedHash<StringRef, Values<Value>, DefaultHash<StringRef>>>;
|
|
||||||
|
|
||||||
using NoAttributesCollectionType = std::conditional_t<
|
|
||||||
dictionary_key_type == DictionaryKeyType::Simple,
|
|
||||||
HashMap<UInt64, IntervalSet<RangeInterval>>,
|
|
||||||
HashMapWithSavedHash<StringRef, IntervalSet<RangeInterval>>>;
|
|
||||||
|
|
||||||
struct Attribute final
|
struct Attribute final
|
||||||
{
|
{
|
||||||
public:
|
|
||||||
AttributeUnderlyingType type;
|
AttributeUnderlyingType type;
|
||||||
bool is_nullable;
|
|
||||||
|
|
||||||
std::variant<
|
std::variant<
|
||||||
CollectionType<UInt8>,
|
AttributeContainerType<UInt8>,
|
||||||
CollectionType<UInt16>,
|
AttributeContainerType<UInt16>,
|
||||||
CollectionType<UInt32>,
|
AttributeContainerType<UInt32>,
|
||||||
CollectionType<UInt64>,
|
AttributeContainerType<UInt64>,
|
||||||
CollectionType<UInt128>,
|
AttributeContainerType<UInt128>,
|
||||||
CollectionType<UInt256>,
|
AttributeContainerType<UInt256>,
|
||||||
CollectionType<Int8>,
|
AttributeContainerType<Int8>,
|
||||||
CollectionType<Int16>,
|
AttributeContainerType<Int16>,
|
||||||
CollectionType<Int32>,
|
AttributeContainerType<Int32>,
|
||||||
CollectionType<Int64>,
|
AttributeContainerType<Int64>,
|
||||||
CollectionType<Int128>,
|
AttributeContainerType<Int128>,
|
||||||
CollectionType<Int256>,
|
AttributeContainerType<Int256>,
|
||||||
CollectionType<Decimal32>,
|
AttributeContainerType<Decimal32>,
|
||||||
CollectionType<Decimal64>,
|
AttributeContainerType<Decimal64>,
|
||||||
CollectionType<Decimal128>,
|
AttributeContainerType<Decimal128>,
|
||||||
CollectionType<Decimal256>,
|
AttributeContainerType<Decimal256>,
|
||||||
CollectionType<DateTime64>,
|
AttributeContainerType<DateTime64>,
|
||||||
CollectionType<Float32>,
|
AttributeContainerType<Float32>,
|
||||||
CollectionType<Float64>,
|
AttributeContainerType<Float64>,
|
||||||
CollectionType<UUID>,
|
AttributeContainerType<UUID>,
|
||||||
CollectionType<StringRef>,
|
AttributeContainerType<StringRef>,
|
||||||
CollectionType<Array>>
|
AttributeContainerType<Array>>
|
||||||
maps;
|
container;
|
||||||
|
|
||||||
|
std::optional<std::vector<bool>> is_value_nullable;
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename RangeStorageType>
|
||||||
|
struct InvalidIntervalWithKey
|
||||||
|
{
|
||||||
|
KeyType key;
|
||||||
|
Interval<RangeStorageType> interval;
|
||||||
|
size_t attribute_value_index;
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename RangeStorageType>
|
||||||
|
using InvalidIntervalsContainerType = PaddedPODArray<InvalidIntervalWithKey<RangeStorageType>>;
|
||||||
|
|
||||||
|
template <template<typename> typename ContainerType>
|
||||||
|
using RangeStorageTypeContainer = std::variant<
|
||||||
|
ContainerType<UInt8>,
|
||||||
|
ContainerType<UInt16>,
|
||||||
|
ContainerType<UInt32>,
|
||||||
|
ContainerType<UInt64>,
|
||||||
|
ContainerType<UInt128>,
|
||||||
|
ContainerType<UInt256>,
|
||||||
|
ContainerType<Int8>,
|
||||||
|
ContainerType<Int16>,
|
||||||
|
ContainerType<Int32>,
|
||||||
|
ContainerType<Int64>,
|
||||||
|
ContainerType<Int128>,
|
||||||
|
ContainerType<Int256>,
|
||||||
|
ContainerType<Decimal32>,
|
||||||
|
ContainerType<Decimal64>,
|
||||||
|
ContainerType<Decimal128>,
|
||||||
|
ContainerType<Decimal256>,
|
||||||
|
ContainerType<DateTime64>,
|
||||||
|
ContainerType<Float32>,
|
||||||
|
ContainerType<Float64>,
|
||||||
|
ContainerType<UUID>>;
|
||||||
|
|
||||||
|
struct KeyAttribute final
|
||||||
|
{
|
||||||
|
RangeStorageTypeContainer<KeyAttributeContainerType> container;
|
||||||
|
|
||||||
|
RangeStorageTypeContainer<InvalidIntervalsContainerType> invalid_intervals_container;
|
||||||
|
|
||||||
};
|
};
|
||||||
|
|
||||||
void createAttributes();
|
void createAttributes();
|
||||||
@ -151,43 +214,31 @@ private:
|
|||||||
ValueSetter && set_value,
|
ValueSetter && set_value,
|
||||||
DefaultValueExtractor & default_value_extractor) const;
|
DefaultValueExtractor & default_value_extractor) const;
|
||||||
|
|
||||||
|
ColumnPtr getColumnInternal(
|
||||||
|
const std::string & attribute_name,
|
||||||
|
const DataTypePtr & result_type,
|
||||||
|
const PaddedPODArray<UInt64> & key_to_index) const;
|
||||||
|
|
||||||
|
template <typename AttributeType, bool is_nullable, typename ValueSetter>
|
||||||
|
void getItemsInternalImpl(
|
||||||
|
const Attribute & attribute,
|
||||||
|
const PaddedPODArray<UInt64> & key_to_index,
|
||||||
|
ValueSetter && set_value) const;
|
||||||
|
|
||||||
void updateData();
|
void updateData();
|
||||||
|
|
||||||
void blockToAttributes(const Block & block);
|
void blockToAttributes(const Block & block);
|
||||||
|
|
||||||
void buildAttributeIntervalTrees();
|
void setAttributeValue(Attribute & attribute, const Field & value);
|
||||||
|
|
||||||
template <typename T>
|
|
||||||
void setAttributeValueImpl(Attribute & attribute, KeyType key, const RangeInterval & interval, const Field & value);
|
|
||||||
|
|
||||||
void setAttributeValue(Attribute & attribute, KeyType key, const RangeInterval & interval, const Field & value);
|
|
||||||
|
|
||||||
template <typename RangeType>
|
|
||||||
void getKeysAndDates(
|
|
||||||
PaddedPODArray<KeyType> & keys,
|
|
||||||
PaddedPODArray<RangeType> & start_dates,
|
|
||||||
PaddedPODArray<RangeType> & end_dates) const;
|
|
||||||
|
|
||||||
template <typename T, typename RangeType>
|
|
||||||
void getKeysAndDates(
|
|
||||||
const Attribute & attribute,
|
|
||||||
PaddedPODArray<KeyType> & keys,
|
|
||||||
PaddedPODArray<RangeType> & start_dates,
|
|
||||||
PaddedPODArray<RangeType> & end_dates) const;
|
|
||||||
|
|
||||||
template <typename RangeType>
|
|
||||||
PaddedPODArray<Int64> makeDateKeys(
|
|
||||||
const PaddedPODArray<RangeType> & block_start_dates,
|
|
||||||
const PaddedPODArray<RangeType> & block_end_dates) const;
|
|
||||||
|
|
||||||
const DictionaryStructure dict_struct;
|
const DictionaryStructure dict_struct;
|
||||||
const DictionarySourcePtr source_ptr;
|
const DictionarySourcePtr source_ptr;
|
||||||
const DictionaryLifetime dict_lifetime;
|
const DictionaryLifetime dict_lifetime;
|
||||||
const bool require_nonempty;
|
const RangeHashedDictionaryConfiguration configuration;
|
||||||
BlockPtr update_field_loaded_block;
|
BlockPtr update_field_loaded_block;
|
||||||
|
|
||||||
std::vector<Attribute> attributes;
|
std::vector<Attribute> attributes;
|
||||||
Arena complex_key_arena;
|
KeyAttribute key_attribute;
|
||||||
|
|
||||||
size_t bytes_allocated = 0;
|
size_t bytes_allocated = 0;
|
||||||
size_t element_count = 0;
|
size_t element_count = 0;
|
||||||
@ -195,7 +246,6 @@ private:
|
|||||||
mutable std::atomic<size_t> query_count{0};
|
mutable std::atomic<size_t> query_count{0};
|
||||||
mutable std::atomic<size_t> found_count{0};
|
mutable std::atomic<size_t> found_count{0};
|
||||||
Arena string_arena;
|
Arena string_arena;
|
||||||
NoAttributesCollectionType no_attributes_container;
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -3,3 +3,5 @@ DefaultValue
|
|||||||
1
|
1
|
||||||
0
|
0
|
||||||
0 15 20 Value
|
0 15 20 Value
|
||||||
|
0 10 0 Value
|
||||||
|
0 15 10 Value
|
||||||
|
@ -38,3 +38,7 @@ PolygonDictionary
|
|||||||
1
|
1
|
||||||
0
|
0
|
||||||
[[[(0,0),(0,1),(1,1),(1,0)]]]
|
[[[(0,0),(0,1),(1,1),(1,0)]]]
|
||||||
|
RangeHashedDictionary
|
||||||
|
0 0 1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
@ -170,7 +170,7 @@ CREATE TABLE 02183_range_dictionary_source_table
|
|||||||
)
|
)
|
||||||
ENGINE = TinyLog;
|
ENGINE = TinyLog;
|
||||||
|
|
||||||
INSERT INTO 02183_range_dictionary_source_table VALUES(1, 0, 1);
|
INSERT INTO 02183_range_dictionary_source_table VALUES(0, 0, 1);
|
||||||
|
|
||||||
DROP DICTIONARY IF EXISTS 02183_range_dictionary;
|
DROP DICTIONARY IF EXISTS 02183_range_dictionary;
|
||||||
CREATE DICTIONARY 02183_range_dictionary
|
CREATE DICTIONARY 02183_range_dictionary
|
||||||
@ -185,7 +185,10 @@ LAYOUT(RANGE_HASHED())
|
|||||||
RANGE(MIN start MAX end)
|
RANGE(MIN start MAX end)
|
||||||
LIFETIME(0);
|
LIFETIME(0);
|
||||||
|
|
||||||
SELECT * FROM 02183_range_dictionary; -- {serverError 1}
|
SELECT 'RangeHashedDictionary';
|
||||||
|
SELECT * FROM 02183_range_dictionary;
|
||||||
|
SELECT dictHas('02183_range_dictionary', 0, 0);
|
||||||
|
SELECT dictHas('02183_range_dictionary', 0, 2);
|
||||||
|
|
||||||
DROP DICTIONARY 02183_range_dictionary;
|
DROP DICTIONARY 02183_range_dictionary;
|
||||||
DROP TABLE 02183_range_dictionary_source_table;
|
DROP TABLE 02183_range_dictionary_source_table;
|
||||||
|
@ -0,0 +1,3 @@
|
|||||||
|
1 0 18446744073709551615 value0 value1 value2
|
||||||
|
('value0','value1','value2')
|
||||||
|
1
|
@ -0,0 +1,36 @@
|
|||||||
|
DROP TABLE IF EXISTS 02184_range_dictionary_source_table;
|
||||||
|
CREATE TABLE 02184_range_dictionary_source_table
|
||||||
|
(
|
||||||
|
id UInt64,
|
||||||
|
start UInt64,
|
||||||
|
end UInt64,
|
||||||
|
value_0 String,
|
||||||
|
value_1 String,
|
||||||
|
value_2 String
|
||||||
|
)
|
||||||
|
ENGINE = TinyLog;
|
||||||
|
|
||||||
|
INSERT INTO 02184_range_dictionary_source_table VALUES (1, 0, 18446744073709551615, 'value0', 'value1', 'value2');
|
||||||
|
|
||||||
|
DROP DICTIONARY IF EXISTS 02184_range_dictionary;
|
||||||
|
CREATE DICTIONARY 02184_range_dictionary
|
||||||
|
(
|
||||||
|
id UInt64,
|
||||||
|
start UInt64,
|
||||||
|
end UInt64,
|
||||||
|
value_0 String,
|
||||||
|
value_1 String,
|
||||||
|
value_2 String
|
||||||
|
)
|
||||||
|
PRIMARY KEY id
|
||||||
|
SOURCE(CLICKHOUSE(TABLE '02184_range_dictionary_source_table'))
|
||||||
|
LAYOUT(RANGE_HASHED())
|
||||||
|
RANGE(MIN start MAX end)
|
||||||
|
LIFETIME(0);
|
||||||
|
|
||||||
|
SELECT * FROM 02184_range_dictionary;
|
||||||
|
SELECT dictGet('02184_range_dictionary', ('value_0', 'value_1', 'value_2'), 1, 18446744073709551615);
|
||||||
|
SELECT dictHas('02184_range_dictionary', 1, 18446744073709551615);
|
||||||
|
|
||||||
|
DROP DICTIONARY 02184_range_dictionary;
|
||||||
|
DROP TABLE 02184_range_dictionary_source_table;
|
@ -0,0 +1,22 @@
|
|||||||
|
Source table
|
||||||
|
0 \N 5000 Value0
|
||||||
|
0 5001 10000 Value1
|
||||||
|
0 10001 \N Value2
|
||||||
|
Dictionary convert_null_range_bound_to_open = 1
|
||||||
|
0 5001 10000 Value1
|
||||||
|
0 0 5000 Value0
|
||||||
|
0 10001 18446744073709551615 Value2
|
||||||
|
Value0
|
||||||
|
Value1
|
||||||
|
Value2
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
Dictionary convert_null_range_bound_to_open = 0
|
||||||
|
0 5001 10000 Value1
|
||||||
|
DefaultValue
|
||||||
|
Value1
|
||||||
|
DefaultValue
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
@ -0,0 +1,63 @@
|
|||||||
|
DROP TABLE IF EXISTS 02185_range_dictionary_source_table;
|
||||||
|
CREATE TABLE 02185_range_dictionary_source_table
|
||||||
|
(
|
||||||
|
id UInt64,
|
||||||
|
start Nullable(UInt64),
|
||||||
|
end Nullable(UInt64),
|
||||||
|
value String
|
||||||
|
)
|
||||||
|
ENGINE = TinyLog;
|
||||||
|
|
||||||
|
INSERT INTO 02185_range_dictionary_source_table VALUES (0, NULL, 5000, 'Value0'), (0, 5001, 10000, 'Value1'), (0, 10001, NULL, 'Value2');
|
||||||
|
|
||||||
|
SELECT 'Source table';
|
||||||
|
SELECT * FROM 02185_range_dictionary_source_table;
|
||||||
|
|
||||||
|
DROP DICTIONARY IF EXISTS 02185_range_dictionary;
|
||||||
|
CREATE DICTIONARY 02185_range_dictionary
|
||||||
|
(
|
||||||
|
id UInt64,
|
||||||
|
start UInt64,
|
||||||
|
end UInt64,
|
||||||
|
value String DEFAULT 'DefaultValue'
|
||||||
|
)
|
||||||
|
PRIMARY KEY id
|
||||||
|
SOURCE(CLICKHOUSE(TABLE '02185_range_dictionary_source_table'))
|
||||||
|
LAYOUT(RANGE_HASHED(convert_null_range_bound_to_open 1))
|
||||||
|
RANGE(MIN start MAX end)
|
||||||
|
LIFETIME(0);
|
||||||
|
|
||||||
|
SELECT 'Dictionary convert_null_range_bound_to_open = 1';
|
||||||
|
SELECT * FROM 02185_range_dictionary;
|
||||||
|
SELECT dictGet('02185_range_dictionary', 'value', 0, 0);
|
||||||
|
SELECT dictGet('02185_range_dictionary', 'value', 0, 5001);
|
||||||
|
SELECT dictGet('02185_range_dictionary', 'value', 0, 10001);
|
||||||
|
SELECT dictHas('02185_range_dictionary', 0, 0);
|
||||||
|
SELECT dictHas('02185_range_dictionary', 0, 5001);
|
||||||
|
SELECT dictHas('02185_range_dictionary', 0, 10001);
|
||||||
|
|
||||||
|
DROP DICTIONARY 02185_range_dictionary;
|
||||||
|
|
||||||
|
CREATE DICTIONARY 02185_range_dictionary
|
||||||
|
(
|
||||||
|
id UInt64,
|
||||||
|
start UInt64,
|
||||||
|
end UInt64,
|
||||||
|
value String DEFAULT 'DefaultValue'
|
||||||
|
)
|
||||||
|
PRIMARY KEY id
|
||||||
|
SOURCE(CLICKHOUSE(TABLE '02185_range_dictionary_source_table'))
|
||||||
|
LAYOUT(RANGE_HASHED(convert_null_range_bound_to_open 0))
|
||||||
|
RANGE(MIN start MAX end)
|
||||||
|
LIFETIME(0);
|
||||||
|
|
||||||
|
SELECT 'Dictionary convert_null_range_bound_to_open = 0';
|
||||||
|
SELECT * FROM 02185_range_dictionary;
|
||||||
|
SELECT dictGet('02185_range_dictionary', 'value', 0, 0);
|
||||||
|
SELECT dictGet('02185_range_dictionary', 'value', 0, 5001);
|
||||||
|
SELECT dictGet('02185_range_dictionary', 'value', 0, 10001);
|
||||||
|
SELECT dictHas('02185_range_dictionary', 0, 0);
|
||||||
|
SELECT dictHas('02185_range_dictionary', 0, 5001);
|
||||||
|
SELECT dictHas('02185_range_dictionary', 0, 10001);
|
||||||
|
|
||||||
|
DROP TABLE 02185_range_dictionary_source_table;
|
@ -0,0 +1,18 @@
|
|||||||
|
Source table
|
||||||
|
1 2020-01-01 2100-01-01 Value0
|
||||||
|
1 2020-01-02 2100-01-01 Value1
|
||||||
|
1 2020-01-03 2100-01-01 Value2
|
||||||
|
Dictionary .range_lookup_strategy = min
|
||||||
|
1 2020-01-01 2100-01-01 Value0
|
||||||
|
1 2020-01-02 2100-01-01 Value1
|
||||||
|
1 2020-01-03 2100-01-01 Value2
|
||||||
|
Value0
|
||||||
|
Value0
|
||||||
|
Value0
|
||||||
|
Dictionary .range_lookup_strategy = max
|
||||||
|
1 2020-01-01 2100-01-01 Value0
|
||||||
|
1 2020-01-02 2100-01-01 Value1
|
||||||
|
1 2020-01-03 2100-01-01 Value2
|
||||||
|
Value0
|
||||||
|
Value1
|
||||||
|
Value2
|
@ -0,0 +1,64 @@
|
|||||||
|
DROP TABLE IF EXISTS 02186_range_dictionary_source_table;
|
||||||
|
CREATE TABLE 02186_range_dictionary_source_table
|
||||||
|
(
|
||||||
|
id UInt64,
|
||||||
|
start Date,
|
||||||
|
end Date,
|
||||||
|
value String
|
||||||
|
)
|
||||||
|
Engine = TinyLog;
|
||||||
|
|
||||||
|
INSERT INTO 02186_range_dictionary_source_table VALUES (1, '2020-01-01', '2100-01-01', 'Value0');
|
||||||
|
INSERT INTO 02186_range_dictionary_source_table VALUES (1, '2020-01-02', '2100-01-01', 'Value1');
|
||||||
|
INSERT INTO 02186_range_dictionary_source_table VALUES (1, '2020-01-03', '2100-01-01', 'Value2');
|
||||||
|
|
||||||
|
SELECT 'Source table';
|
||||||
|
SELECT * FROM 02186_range_dictionary_source_table;
|
||||||
|
|
||||||
|
DROP DICTIONARY IF EXISTS 02186_range_dictionary;
|
||||||
|
CREATE DICTIONARY 02186_range_dictionary
|
||||||
|
(
|
||||||
|
id UInt64,
|
||||||
|
start Date,
|
||||||
|
end Date,
|
||||||
|
value String
|
||||||
|
)
|
||||||
|
PRIMARY KEY id
|
||||||
|
SOURCE(CLICKHOUSE(TABLE '02186_range_dictionary_source_table'))
|
||||||
|
LAYOUT(RANGE_HASHED(range_lookup_strategy 'min'))
|
||||||
|
RANGE(MIN start MAX end)
|
||||||
|
LIFETIME(0);
|
||||||
|
|
||||||
|
SELECT 'Dictionary .range_lookup_strategy = min';
|
||||||
|
|
||||||
|
SELECT * FROM 02186_range_dictionary;
|
||||||
|
|
||||||
|
select dictGet('02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-01'));
|
||||||
|
select dictGet('02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-02'));
|
||||||
|
select dictGet('02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-03'));
|
||||||
|
|
||||||
|
DROP DICTIONARY 02186_range_dictionary;
|
||||||
|
|
||||||
|
CREATE DICTIONARY 02186_range_dictionary
|
||||||
|
(
|
||||||
|
id UInt64,
|
||||||
|
start Date,
|
||||||
|
end Date,
|
||||||
|
value String
|
||||||
|
)
|
||||||
|
PRIMARY KEY id
|
||||||
|
SOURCE(CLICKHOUSE(TABLE '02186_range_dictionary_source_table'))
|
||||||
|
LAYOUT(RANGE_HASHED(range_lookup_strategy 'max'))
|
||||||
|
RANGE(MIN start MAX end)
|
||||||
|
LIFETIME(0);
|
||||||
|
|
||||||
|
SELECT 'Dictionary .range_lookup_strategy = max';
|
||||||
|
|
||||||
|
SELECT * FROM 02186_range_dictionary;
|
||||||
|
|
||||||
|
select dictGet('02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-01'));
|
||||||
|
select dictGet('02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-02'));
|
||||||
|
select dictGet('02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-03'));
|
||||||
|
|
||||||
|
DROP DICTIONARY 02186_range_dictionary;
|
||||||
|
DROP TABLE 02186_range_dictionary_source_table;
|
Loading…
Reference in New Issue
Block a user