Add RegExpTree dictionary

This commit is contained in:
Vage Ogannisian 2022-08-31 19:34:50 +00:00
parent 540fa7fe5b
commit 9b2326cc6c
13 changed files with 1113 additions and 129 deletions

View File

@ -1,5 +1,8 @@
include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")
include(configure_config.cmake)
configure_file(config_dictionaries.h.in ${ConfigIncludePath}/config_dictionaries.h)
add_headers_and_sources(clickhouse_dictionaries .)
add_headers_and_sources(clickhouse_dictionaries "${CMAKE_CURRENT_BINARY_DIR}/generated/")
@ -42,5 +45,9 @@ if (TARGET ch_contrib::yaml_cpp)
target_link_libraries(clickhouse_dictionaries PRIVATE ch_contrib::yaml_cpp)
endif()
if (TARGET ch_contrib::vectorscan)
target_link_libraries(clickhouse_dictionaries PRIVATE ch_contrib::vectorscan)
endif()
add_subdirectory(Embedded)
target_link_libraries(clickhouse_dictionaries PRIVATE ch_contrib::sparsehash)

View File

@ -1,19 +1,19 @@
#pragma once
#include <atomic>
#include <optional>
#include <variant>
#include <vector>
#include <optional>
#include <Common/HashTable/HashSet.h>
#include <Common/Arena.h>
#include <DataTypes/IDataType.h>
#include <Core/Block.h>
#include <DataTypes/IDataType.h>
#include <Common/Arena.h>
#include <Common/HashTable/HashSet.h>
#include "DictionaryHelpers.h"
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "IDictionarySource.h"
#include "DictionaryHelpers.h"
namespace DB
{
@ -58,7 +58,8 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<FlatDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), configuration, update_field_loaded_block);
return std::make_shared<FlatDictionary>(
getDictionaryID(), dict_struct, source_ptr->clone(), configuration, update_field_loaded_block);
}
DictionarySourcePtr getSource() const override { return source_ptr; }
@ -67,15 +68,12 @@ public:
const DictionaryStructure & getStructure() const override { return dict_struct; }
bool isInjective(const std::string & attribute_name) const override
{
return dict_struct.getAttribute(attribute_name).injective;
}
bool isInjective(const std::string & attribute_name) const override { return dict_struct.getAttribute(attribute_name).injective; }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Simple; }
ColumnPtr getColumn(
const std::string& attribute_name,
const std::string & attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
@ -87,10 +85,7 @@ public:
ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override;
ColumnUInt8::Ptr isInHierarchy(
ColumnPtr key_column,
ColumnPtr in_key_column,
const DataTypePtr & key_type) const override;
ColumnUInt8::Ptr isInHierarchy(ColumnPtr key_column, ColumnPtr in_key_column, const DataTypePtr & key_type) const override;
DictionaryHierarchicalParentToChildIndexPtr getHierarchicalIndex() const override;

View File

@ -0,0 +1,585 @@
#include "RegExpTreeDictionary.h"
#include <optional>
#include <string_view>
#include <fmt/format.h>
#include <type_traits>
#include <base/defines.h>
#include <Poco/RegularExpression.h>
#include <Common/ArenaUtils.h>
#include <Functions/Regexps.h>
#include <QueryPipeline/QueryPipeline.h>
#include <Dictionaries/DictionaryFactory.h>
#include <Dictionaries/DictionaryHelpers.h>
#include <Dictionaries/DictionaryStructure.h>
#include "config_dictionaries.h"
#if USE_VECTORSCAN
# include <hs.h>
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int CANNOT_ALLOCATE_MEMORY;
extern const int HYPERSCAN_CANNOT_SCAN_TEXT;
extern const int UNSUPPORTED_METHOD;
}
inline const std::string kId = "id";
inline const std::string kParentId = "parent_id";
inline const std::string kRegExp = "regexp";
void RegExpTreeDictionary::createAttributes()
{
for (const auto & pair : structure.attribute_name_to_index)
{
const auto & attribute_name = pair.first;
const auto & attribute_index = pair.second;
const auto & dictionary_attribute = structure.attributes[attribute_index];
auto type_call = [&](const auto & dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
using ValueType = DictionaryValueType<AttributeType>;
Attribute attribute{dictionary_attribute.underlying_type, {}, ContainerType<ValueType>()};
names_to_attributes[attribute_name] = std::move(attribute);
};
callOnDictionaryAttributeType(dictionary_attribute.underlying_type, std::move(type_call));
}
}
void RegExpTreeDictionary::calculateBytesAllocated()
{
for (auto & pair : names_to_attributes)
{
const auto & name = pair.first;
auto & attribute = pair.second;
bytes_allocated += name.size() + sizeof(attribute);
auto type_call = [&](const auto & dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
using ValueType = DictionaryValueType<AttributeType>;
const auto & container = std::get<ContainerType<ValueType>>(attribute.container);
bytes_allocated += sizeof(ContainerType<ValueType>);
if constexpr (std::is_same_v<ValueType, Array>)
{
/// It is not accurate calculations
bytes_allocated += sizeof(Array) * container.size();
}
else
{
bytes_allocated += container.allocated_bytes();
}
bucket_count = container.capacity();
};
callOnDictionaryAttributeType(attribute.type, type_call);
if (!attribute.nullable_set.empty())
{
bytes_allocated += sizeof(*attribute.nullable_set.begin()) * attribute.nullable_set.size();
}
}
for (const auto & regexp : regexps)
{
bytes_allocated += regexp.size();
}
bytes_allocated += 2 * keys_to_ids.size() * sizeof(UInt64);
bytes_allocated += 2 * ids_to_parent_ids.size() * sizeof(UInt64);
bytes_allocated += 2 * ids_to_child_ids.size() * sizeof(UInt64);
bytes_allocated += string_arena.size();
}
void RegExpTreeDictionary::resizeAttributes(const size_t size)
{
for (auto & pair : names_to_attributes)
{
auto & attribute = pair.second;
auto type_call = [&](const auto & dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
using ValueType = DictionaryValueType<AttributeType>;
auto & container = std::get<ContainerType<ValueType>>(attribute.container);
if constexpr (std::is_same_v<ValueType, Array>)
{
container.resize(size, ValueType());
}
else
{
container.resize_fill(size, ValueType());
}
};
callOnDictionaryAttributeType(attribute.type, std::move(type_call));
}
}
void RegExpTreeDictionary::setRegexps(const Block & block)
{
const auto & column = *block.getByName(kRegExp).column;
for (size_t i = 0; i < column.size(); ++i)
{
regexps.push_back(column[i].get<std::string>());
}
}
void RegExpTreeDictionary::setIdToParentId(const Block & block)
{
const auto & column = *block.getByName(kParentId).column;
for (UInt64 i = 0; i < column.size(); ++i)
{
if (column[i].isNull())
{
continue;
}
const auto parent_id = keys_to_ids[column[i].get<UInt64>()];
ids_to_parent_ids[i] = parent_id;
ids_to_child_ids[parent_id] = i;
}
}
void RegExpTreeDictionary::setAttributeValue(Attribute & attribute, const UInt64 key, const Field & value)
{
auto type_call = [&](const auto & dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
using ValueType = DictionaryValueType<AttributeType>;
if (value.isNull())
{
attribute.nullable_set.insert(key);
return;
}
auto & attribute_value = value.safeGet<AttributeType>();
auto & container = std::get<ContainerType<ValueType>>(attribute.container);
if constexpr (std::is_same_v<ValueType, StringRef>)
{
auto arena_value = copyStringInArena(string_arena, attribute_value);
container[key] = arena_value;
}
else
{
container[key] = attribute_value;
}
};
callOnDictionaryAttributeType(attribute.type, std::move(type_call));
}
void RegExpTreeDictionary::blockToAttributes(const Block & block)
{
const auto ids_column = block.getByName(kId).column;
const auto ids_size = ids_column->size();
for (UInt64 i = 0; i < ids_size; ++i)
{
keys_to_ids[(*ids_column)[i].get<UInt64>()] = i;
}
setRegexps(block);
setIdToParentId(block);
resizeAttributes(ids_size);
for (auto & [name, attribute] : names_to_attributes)
{
const auto & attribute_column = *block.getByName(name).column;
for (size_t j = 0; j < ids_size; ++j)
{
setAttributeValue(attribute, j, attribute_column[j]);
}
}
}
void RegExpTreeDictionary::loadData()
{
if (!source_ptr->hasUpdateField())
{
QueryPipeline pipeline(source_ptr->loadAll());
PullingPipelineExecutor executor(pipeline);
Block block;
while (executor.pull(block))
{
blockToAttributes(block);
}
}
else
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary {} does not support updating manual fields", name);
}
}
RegExpTreeDictionary::RegExpTreeDictionary(
const StorageID & id_, const DictionaryStructure & structure_, DictionarySourcePtr source_ptr_, Configuration configuration_)
: IDictionary(id_), structure(structure_), source_ptr(source_ptr_), configuration(configuration_)
{
createAttributes();
loadData();
calculateBytesAllocated();
}
std::unordered_set<UInt64> RegExpTreeDictionary::matchSearchAllIndices(const std::string & key) const
{
#if USE_VECTORSCAN
std::vector<std::string_view> regexps_views(regexps.begin(), regexps.end());
const auto & hyperscan_regex = MultiRegexps::get<true, false>(regexps_views, std::nullopt);
hs_scratch_t * scratch = nullptr;
hs_error_t err = hs_clone_scratch(hyperscan_regex->getScratch(), &scratch);
if (err != HS_SUCCESS)
{
throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not clone scratch space for hyperscan");
}
MultiRegexps::ScratchPtr smart_scratch(scratch);
std::unordered_set<UInt64> matches;
auto on_match = [](unsigned int id,
unsigned long long /* from */, // NOLINT
unsigned long long /* to */, // NOLINT
unsigned int /* flags */,
void * context) -> int
{
static_cast<std::unordered_set<UInt64> *>(context)->insert(id);
return 0;
};
err = hs_scan(hyperscan_regex->getDB(), key.c_str(), key.size(), 0, smart_scratch.get(), on_match, &matches);
if (err != HS_SUCCESS)
{
throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Failed to scan {} with vectorscan", key);
}
return matches;
#else
# void(key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Multi search all indices is not implemented when USE_VECTORSCAN is off");
#endif // USE_VECTORSCAN
}
template <typename AttributeType, typename SetValueFunc, typename DefaultValueExtractor>
void RegExpTreeDictionary::getColumnImpl(
const Attribute & attribute,
const std::optional<UInt64> match_index_opt,
const bool is_nullable,
SetValueFunc && set_value_func,
DefaultValueExtractor & default_value_extractor) const
{
const auto & container = std::get<ContainerType<AttributeType>>(attribute.container);
const auto index = 0;
if (match_index_opt.has_value())
{
const auto & match_index = match_index_opt.value();
if (is_nullable)
{
set_value_func(index, container[match_index], attribute.nullable_set.contains(match_index));
}
else
{
set_value_func(index, container[match_index], false);
}
found_count.fetch_add(1, std::memory_order_relaxed);
}
else
{
if (is_nullable)
{
set_value_func(index, default_value_extractor[index], default_value_extractor.isNullAt(index));
}
else
{
set_value_func(index, default_value_extractor[index], false);
}
}
query_count.fetch_add(1, std::memory_order_relaxed);
}
UInt64 RegExpTreeDictionary::getRoot(const std::unordered_set<UInt64> & indices) const
{
for (const auto & index : indices)
{
if (!ids_to_parent_ids.contains(index - 1))
{
return index - 1;
}
}
return *indices.begin() - 1;
}
std::optional<UInt64> RegExpTreeDictionary::getLastMatchIndex(const std::unordered_set<UInt64> matches, const Attribute & attribute) const
{
if (matches.empty())
{
return std::nullopt;
}
auto match_index = getRoot(matches);
auto last_match_index = match_index;
while (ids_to_child_ids.contains(match_index))
{
match_index = ids_to_child_ids.at(match_index);
if (!matches.contains(match_index))
{
break;
}
if (!attribute.nullable_set.contains(match_index))
{
last_match_index = match_index;
}
}
return last_match_index;
}
std::string format(const std::string & pattern, const std::vector<std::string> & arguments)
{
using context = fmt::format_context;
std::vector<fmt::basic_format_arg<context>> fmt_arguments;
fmt_arguments.reserve(arguments.size());
for (auto const & argument : arguments)
{
fmt_arguments.push_back(fmt::detail::make_arg<context>(argument));
}
return fmt::vformat(pattern, fmt::basic_format_args<context>(fmt_arguments.data(), fmt_arguments.size()));
}
std::string formatMatches(const std::string & key, const std::string & regexp_pattern, const std::string & pattern)
{
Poco::RegularExpression regexp(regexp_pattern);
Poco::RegularExpression::MatchVec matches;
regexp.match(key, 0, matches);
std::vector<std::string> arguments;
for (size_t i = 1; i < matches.size(); ++i)
{
const auto & match = matches[i];
arguments.emplace_back(key.substr(match.offset, match.length));
}
return format(pattern, arguments);
}
ColumnPtr RegExpTreeDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes &,
const ColumnPtr & default_values_column) const
{
if (key_columns.size() != 1)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected 1 key for getColumn, got {}", std::to_string(key_columns.size()));
}
const auto & key_column = key_columns.front();
if (key_column->size() != 1)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected 1 key for getColumn, got {}", std::to_string(key_column->size()));
}
const auto key = (*key_column)[0].get<std::string>();
if (!names_to_attributes.contains(attribute_name))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown attribute {}", attribute_name);
}
ColumnPtr result;
const auto keys_size = 1;
const auto & attribute = names_to_attributes.at(attribute_name);
const auto & dictionary_attribute = structure.getAttribute(attribute_name, result_type);
const auto is_nullable = !attribute.nullable_set.empty();
ColumnUInt8::MutablePtr null_mask;
if (is_nullable)
{
null_mask = ColumnUInt8::create(keys_size, false);
}
const auto matches = matchSearchAllIndices(key);
const auto match_index_opt = getLastMatchIndex(matches, attribute);
auto type_call = [&](const auto & dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
using ValueType = DictionaryValueType<AttributeType>;
using ColumnProvider = DictionaryAttributeColumnProvider<AttributeType>;
DictionaryDefaultValueExtractor<AttributeType> default_value_extractor(dictionary_attribute.null_value, default_values_column);
auto column = ColumnProvider::getColumn(dictionary_attribute, keys_size);
if constexpr (std::is_same_v<ValueType, Array>)
{
getColumnImpl<ValueType>(
attribute,
match_index_opt,
false,
[&](size_t, const Array & value, bool) { column.get()->insert(value); },
default_value_extractor);
}
else if constexpr (std::is_same_v<ValueType, StringRef>)
{
if (is_nullable)
{
getColumnImpl<ValueType>(
attribute,
match_index_opt,
true,
[&](size_t index, StringRef value, bool is_null)
{
if (is_null)
{
null_mask->getData()[index] = true;
}
if (match_index_opt.has_value())
{
const auto formatted_value = formatMatches(key, regexps[match_index_opt.value()], value.toString());
column.get()->insertData(formatted_value.data(), formatted_value.size());
}
else
{
column.get()->insertData(value.data, value.size);
}
},
default_value_extractor);
}
else
{
getColumnImpl<ValueType>(
attribute,
match_index_opt,
false,
[&](size_t, const StringRef value, bool) { column.get()->insertData(value.data, value.size); },
default_value_extractor);
}
}
else
{
if (is_nullable)
{
getColumnImpl<ValueType>(
attribute,
match_index_opt,
true,
[&](size_t index, const auto value, bool is_null)
{
if (is_null)
{
null_mask->getData()[index] = true;
}
column->getData()[index] = value;
},
default_value_extractor);
}
else
{
getColumnImpl<ValueType>(
attribute,
match_index_opt,
false,
[&](size_t index, const auto value, bool) { column->getData()[index] = value; },
default_value_extractor);
}
}
result = std::move(column);
};
callOnDictionaryAttributeType(attribute.type, std::move(type_call));
if (is_nullable)
{
result = ColumnNullable::create(result, std::move(null_mask));
}
return result;
}
void registerDictionaryRegExpTree(DictionaryFactory & factory)
{
auto create_layout = [=](const std::string & full_name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr,
ContextPtr,
bool) -> DictionaryPtr
{
if (dict_struct.id)
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for dictionary of layout `{}`", full_name);
}
String dictionary_layout_prefix = config_prefix + ".layout" + ".reg-exp-tree";
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
RegExpTreeDictionary::Configuration configuration{
.require_nonempty = config.getBool(config_prefix + ".require_nonempty", false), .lifetime = dict_lifetime};
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
return std::make_unique<RegExpTreeDictionary>(dict_id, dict_struct, std::move(source_ptr), configuration);
};
factory.registerLayout("reg-exp-tree", create_layout, false);
}
}

View File

@ -0,0 +1,197 @@
#pragma once
#include <atomic>
#include <string>
#include <unordered_map>
#include <unordered_set>
#include <variant>
#include <base/types.h>
#include <Common/Arena.h>
#include <Common/Exception.h>
#include <Common/HashTable/Hash.h>
#include <Common/HashTable/HashSet.h>
#include <DataTypes/IDataType.h>
#include <Columns/IColumn.h>
#include <QueryPipeline/Pipe.h>
#include <Core/Block.h>
#include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/IDictionary.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int UNSUPPORTED_METHOD;
}
class RegExpTreeDictionary : public IDictionary
{
public:
struct Configuration
{
bool require_nonempty;
DictionaryLifetime lifetime;
};
const std::string name = "RegExpTree";
RegExpTreeDictionary(
const StorageID & id_, const DictionaryStructure & structure_, DictionarySourcePtr source_ptr_, Configuration configuration_);
std::string getTypeName() const override { return name; }
size_t getBytesAllocated() const override { return bytes_allocated; }
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
double getFoundRate() const override
{
const auto queries = query_count.load(std::memory_order_relaxed);
if (!queries)
return 0;
return static_cast<double>(found_count.load(std::memory_order_relaxed)) / queries;
}
double getHitRate() const override { return 1.0; }
size_t getElementCount() const override { return element_count; }
double getLoadFactor() const override { return static_cast<double>(element_count) / bucket_count; }
DictionarySourcePtr getSource() const override { return source_ptr; }
const DictionaryLifetime & getLifetime() const override { return configuration.lifetime; }
const DictionaryStructure & getStructure() const override { return structure; }
bool isInjective(const std::string & attribute_name) const override { return structure.getAttribute(attribute_name).injective; }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Simple; }
bool hasHierarchy() const override { return false; }
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<RegExpTreeDictionary>(getDictionaryID(), structure, source_ptr->clone(), configuration);
}
ColumnUInt8::Ptr hasKeys(const Columns &, const DataTypes &) const override
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary {} does not support method `hasKeys`", name);
}
Pipe read(const Names &, size_t, size_t) const override
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary {} does not support method `read`", name);
}
ColumnPtr getColumn(
const std::string & attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr & default_values_column) const override;
private:
const DictionaryStructure structure;
const DictionarySourcePtr source_ptr;
const Configuration configuration;
size_t bytes_allocated = 0;
size_t bucket_count = 0;
size_t element_count = 0;
mutable std::atomic<size_t> query_count{0};
mutable std::atomic<size_t> found_count{0};
//////////////////////////////////////////////////////////////////////
template <typename Value>
using ContainerType = std::conditional_t<std::is_same_v<Value, Array>, std::vector<Value>, PaddedPODArray<Value>>;
using NullableSet = HashSet<UInt64, DefaultHash<UInt64>>;
struct Attribute final
{
AttributeUnderlyingType type;
std::unordered_set<UInt64> nullable_set;
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>,
ContainerType<StringRef>,
ContainerType<Array>>
container;
};
std::unordered_map<std::string, Attribute> names_to_attributes;
std::unordered_map<UInt64, UInt64> keys_to_ids;
std::vector<std::string> regexps;
std::unordered_map<UInt64, UInt64> ids_to_parent_ids;
std::unordered_map<UInt64, UInt64> ids_to_child_ids;
Arena string_arena;
//////////////////////////////////////////////////////////////////////
void createAttributes();
void resizeAttributes(size_t size);
void calculateBytesAllocated();
void setRegexps(const Block & block);
void setIdToParentId(const Block & block);
void setAttributeValue(Attribute & attribute, UInt64 key, const Field & value);
void blockToAttributes(const Block & block);
void loadData();
std::unordered_set<UInt64> matchSearchAllIndices(const std::string & key) const;
template <typename AttributeType, typename SetValueFunc, typename DefaultValueExtractor>
void getColumnImpl(
const Attribute & attribute,
std::optional<UInt64> match_index,
bool is_nullable,
SetValueFunc && set_value_func,
DefaultValueExtractor & default_value_extractor) const;
UInt64 getRoot(const std::unordered_set<UInt64> & indices) const;
std::optional<UInt64> getLastMatchIndex(std::unordered_set<UInt64> matches, const Attribute & attribute) const;
};
}

View File

@ -6,20 +6,25 @@
#include <optional>
#include <string>
#include <unordered_map>
#include <unordered_set>
#if USE_YAML_CPP
#include <yaml-cpp/exceptions.h>
#include <yaml-cpp/node/node.h>
#include <yaml-cpp/node/parse.h>
#include <yaml-cpp/yaml.h>
# include <yaml-cpp/exceptions.h>
# include <yaml-cpp/node/node.h>
# include <yaml-cpp/node/parse.h>
# include <yaml-cpp/yaml.h>
#endif
#include <base/types.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/IColumn.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/Serializations/ISerialization.h>
@ -69,7 +74,7 @@ void registerDictionarySourceYAMLRegExpTree(DictionarySourceFactory & factory)
ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `{}` does not support attribute expressions", kYAMLRegExpTree);
}
const auto filepath = config.getString(config_prefix + ".file.path");
const auto filepath = config.getString(config_prefix + ".YAMLRegExpTree.path");
const auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix);
@ -90,6 +95,7 @@ void registerDictionarySourceYAMLRegExpTree(DictionarySourceFactory & factory)
#if USE_CASSANDRA
namespace DB
{
/**
@ -108,6 +114,7 @@ namespace DB
* ...
* ```
*/
inline const std::string kAttributes = "attributes";
inline const std::string kConfiguration = "configuration";
@ -118,17 +125,17 @@ inline const std::string kConfiguration = "configuration";
* ```
* attr1:
* type: String
* default: "demogorgon" # optional field
* attr2:
* type: Float
* ```
*/
inline const std::string kType = "type";
inline const std::string kDefault = "default";
/**
* Allowed data types
*/
* Allowed data types
*/
inline const std::string kUInt = "UInt";
inline const std::string kInt = "Int";
inline const std::string kFloat = "Float";
@ -149,6 +156,7 @@ inline const std::string kString = "String";
* ...
* ```
*/
inline const std::string kMatch = "match";
inline const std::string kRegExp = "regexp";
inline const std::string kSet = "set";
@ -164,6 +172,7 @@ inline const std::string kSet = "set";
* attr_n DataType
* ```
*/
inline const std::string kId = "id";
inline const std::string kParentId = "parent_id";
@ -179,97 +188,118 @@ namespace ErrorCodes
//////////////////////////////////////////////////////////////////////
Field toUInt(const std::string & value)
{
const auto result = static_cast<UInt64>(std::strtoul(value.c_str(), nullptr, 10));
return Field(result);
}
Field toInt(const std::string & value)
{
const auto result = static_cast<Int64>(std::strtol(value.c_str(), nullptr, 10));
return Field(result);
}
Field toFloat(const std::string & value)
{
const auto result = static_cast<Float64>(std::strtof(value.c_str(), nullptr));
return Field(result);
}
Field toString(const std::string & value)
{
return Field(value);
}
class Attribute
{
public:
std::string name;
DataTypePtr data_type;
MutableColumnPtr column;
std::optional<std::string> default_value;
explicit Attribute(const std::string & name_, const DataTypePtr & data_type_, const std::optional<std::string> & default_value_)
: name(name_), data_type(data_type_), column(data_type->createColumn()), default_value(default_value_)
explicit Attribute(
const std::string & name_, const bool is_nullable_, const DataTypePtr & data_type_, const std::string & attribute_type_)
: name(name_)
, is_nullable(is_nullable_)
, data_type(data_type_)
, attribute_type(attribute_type_)
, column_ptr(data_type_->createColumn())
{
}
Attribute(Attribute && other) noexcept
void insert(const std::string & value)
{
column = std::move(other.column);
default_value = std::move(other.default_value);
}
virtual void insert(const std::string &) { }
virtual void insertDefault()
{
if (default_value.has_value())
if (attribute_type == kUInt)
{
insert(default_value.value());
column_ptr->insert(toUInt(value));
}
else if (attribute_type == kInt)
{
column_ptr->insert(toInt(value));
}
else if (attribute_type == kFloat)
{
column_ptr->insert(toFloat(value));
}
else if (attribute_type == kString)
{
column_ptr->insert(toString(value));
}
else
{
column->insertDefault();
throw Exception(ErrorCodes::INVALID_REGEXP_TREE_CONFIGURATION, "Unkown type for attribute {}", attribute_type);
}
}
virtual ~Attribute() = default;
};
class UIntAttribute : public Attribute
{
public:
explicit UIntAttribute(const std::string & name_, const DataTypePtr & data_type_, const std::optional<std::string> & default_value_)
: Attribute(name_, data_type_, default_value_)
void insertNull()
{
if (!is_nullable)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying insert null to not nullable column {} of type {}", name, attribute_type);
}
null_indices.insert(column_ptr->size());
column_ptr->insertDefault();
}
void insert(const std::string & value) override { column->insert(Field(fromString(value))); }
std::string getName() const { return name; }
private:
static UInt64 fromString(const std::string & value) { return static_cast<UInt64>(std::strtoul(value.c_str(), nullptr, 10)); }
};
class IntAttribute : public Attribute
{
public:
explicit IntAttribute(const std::string & name_, const DataTypePtr & data_type_, const std::optional<std::string> & default_value_)
: Attribute(name_, data_type_, default_value_)
DataTypePtr getDataType() const
{
if (isNullable())
{
return makeNullable(data_type);
}
return data_type;
}
void insert(const std::string & value) override { column->insert(Field(fromString(value))); }
private:
static Int64 fromString(const std::string & value) { return static_cast<Int64>(std::strtol(value.c_str(), nullptr, 10)); }
};
class FloatAttribute : public Attribute
{
public:
explicit FloatAttribute(const std::string & name_, const DataTypePtr & data_type_, const std::optional<std::string> & default_value_)
: Attribute(name_, data_type_, default_value_)
ColumnPtr getColumn()
{
auto result = column_ptr->convertToFullIfNeeded();
if (isNullable())
{
auto null_mask = ColumnUInt8::create(column_ptr->size(), false);
for (size_t i = 0; i < column_ptr->size(); ++i)
{
null_mask->getData()[i] = null_indices.contains(i);
}
return ColumnNullable::create(result, std::move(null_mask));
}
return result;
}
void insert(const std::string & value) override { column->insert(Field(fromString(value))); }
private:
static Float64 fromString(const std::string & value) { return static_cast<Float64>(std::strtof(value.c_str(), nullptr)); }
};
std::string name;
class StringAttribute : public Attribute
{
public:
explicit StringAttribute(const std::string & name_, const DataTypePtr & data_type_, const std::optional<std::string> & default_value_)
: Attribute(name_, data_type_, default_value_)
{
}
bool is_nullable;
std::unordered_set<UInt64> null_indices;
void insert(const std::string & value) override { column->insert(Field(fromString(value))); }
DataTypePtr data_type;
std::string attribute_type;
private:
static String fromString(const std::string & value) { return value; }
MutableColumnPtr column_ptr;
bool isNullable() const { return is_nullable && !null_indices.empty(); }
};
//////////////////////////////////////////////////////////////////////
@ -307,26 +337,32 @@ StringToNode parseYAMLMap(const YAML::Node & node)
return result;
}
Attribute makeAttribute(const std::string & name, const std::string & type, const std::optional<std::string> & default_value)
Attribute makeAttribute(const std::string & name, const std::string & type)
{
DataTypePtr data_type;
if (type == kUInt)
{
return std::move(UIntAttribute(name, std::make_shared<DataTypeUInt64>(), default_value));
data_type = std::make_shared<DataTypeUInt64>();
}
if (type == kInt)
else if (type == kInt)
{
return std::move(IntAttribute(name, std::make_shared<DataTypeInt64>(), default_value));
data_type = std::make_shared<DataTypeInt64>();
}
if (type == kFloat)
else if (type == kFloat)
{
return std::move(FloatAttribute(name, std::make_shared<DataTypeFloat64>(), default_value));
data_type = std::make_shared<DataTypeFloat64>();
}
if (type == kString)
else if (type == kString)
{
return std::move(StringAttribute(name, std::make_shared<DataTypeString>(), default_value));
data_type = std::make_shared<DataTypeString>();
}
else
{
throw Exception(ErrorCodes::INVALID_REGEXP_TREE_CONFIGURATION, "Unkown type for attribute {}", type);
}
throw Exception(ErrorCodes::INVALID_REGEXP_TREE_CONFIGURATION, "Unsupported data type {}", type);
return Attribute(name, true, data_type, type);
}
Attribute makeAttribute(const std::string & name, const YAML::Node & node)
@ -334,11 +370,7 @@ Attribute makeAttribute(const std::string & name, const YAML::Node & node)
if (!node.IsMap())
{
throw Exception(
ErrorCodes::INVALID_REGEXP_TREE_CONFIGURATION,
"Invalid structure for attribute {}, expected `{}` and `{}` (optional) mapping",
name,
kType,
kDefault);
ErrorCodes::INVALID_REGEXP_TREE_CONFIGURATION, "Invalid structure for attribute {}, expected `{}` mapping", name, kType);
}
auto attribute_params = parseYAMLMap(node);
@ -348,19 +380,6 @@ Attribute makeAttribute(const std::string & name, const YAML::Node & node)
throw Exception(ErrorCodes::INVALID_REGEXP_TREE_CONFIGURATION, "Missing type for attribute {}", name);
}
std::optional<std::string> default_value;
if (attribute_params.contains(kDefault))
{
const auto default_value_node = attribute_params[kDefault];
if (!default_value_node.IsScalar())
{
throw Exception(ErrorCodes::INVALID_REGEXP_TREE_CONFIGURATION, "Value for `default` must be scalar, attribute {}", kDefault);
}
default_value = default_value_node.as<std::string>();
}
const auto type_node = attribute_params[kType];
if (!type_node.IsScalar())
{
@ -369,7 +388,7 @@ Attribute makeAttribute(const std::string & name, const YAML::Node & node)
const auto type = type_node.as<std::string>();
return makeAttribute(name, type, default_value);
return makeAttribute(name, type);
}
StringToAttribute getAttributes(const YAML::Node & node)
@ -389,7 +408,7 @@ void getValuesFromSet(const YAML::Node & node, StringToString & attributes_to_in
{
if (!node.IsMap())
{
throw Exception(ErrorCodes::INVALID_REGEXP_TREE_CONFIGURATION, "`` must be mapping", kSet);
throw Exception(ErrorCodes::INVALID_REGEXP_TREE_CONFIGURATION, "`{}` must be mapping", kSet);
}
const auto attributes = parseYAMLMap(node);
@ -414,12 +433,12 @@ void insertValues(StringToString & attributes_to_insert, StringToAttribute & nam
}
else
{
attribute.insertDefault();
attribute.insertNull();
}
}
}
UInt64 processMatch(UInt64 parent_id, const bool is_root, const YAML::Node & node, StringToAttribute & names_to_attributes)
UInt64 processMatch(const bool is_root, UInt64 parent_id, const YAML::Node & node, StringToAttribute & names_to_attributes)
{
if (!node.IsMap())
{
@ -430,7 +449,10 @@ UInt64 processMatch(UInt64 parent_id, const bool is_root, const YAML::Node & nod
StringToString attributes_to_insert;
attributes_to_insert[kParentId] = is_root ? "0" : std::to_string(parent_id);
if (!is_root)
{
attributes_to_insert[kParentId] = std::to_string(parent_id);
}
attributes_to_insert[kId] = std::to_string(++parent_id);
if (!match.contains(kRegExp))
@ -457,14 +479,14 @@ UInt64 processMatch(UInt64 parent_id, const bool is_root, const YAML::Node & nod
return parent_id;
}
return processMatch(parent_id, false, match[kMatch], names_to_attributes);
return processMatch(false, parent_id, match[kMatch], names_to_attributes);
}
void parseConfiguration(const YAML::Node & node, StringToAttribute & names_to_attributes)
{
names_to_attributes.insert({kId, UIntAttribute(kId, std::make_shared<DataTypeUInt64>(), std::nullopt)});
names_to_attributes.insert({kParentId, UIntAttribute(kParentId, std::make_shared<DataTypeUInt64>(), std::nullopt)});
names_to_attributes.insert({kRegExp, StringAttribute(kRegExp, std::make_shared<DataTypeString>(), std::nullopt)});
names_to_attributes.insert({kId, Attribute(kId, false, std::make_shared<DataTypeUInt64>(), kUInt)});
names_to_attributes.insert({kParentId, Attribute(kParentId, true, std::make_shared<DataTypeUInt64>(), kUInt)});
names_to_attributes.insert({kRegExp, Attribute(kRegExp, false, std::make_shared<DataTypeString>(), kString)});
if (!node.IsSequence())
{
@ -477,15 +499,17 @@ void parseConfiguration(const YAML::Node & node, StringToAttribute & names_to_at
{
if (!child_node.IsMap())
{
throw Exception(ErrorCodes::INVALID_REGEXP_TREE_CONFIGURATION, "Element of configuration sequence must be mapping", kMatch);
throw Exception(ErrorCodes::INVALID_REGEXP_TREE_CONFIGURATION, "Element of {} configuration sequence must be mapping", kMatch);
}
if (child_node.first.as<std::string>() != kMatch)
auto match = parseYAMLMap(child_node);
if (!match.contains(kMatch))
{
continue;
throw Exception(ErrorCodes::INVALID_REGEXP_TREE_CONFIGURATION, "Match mapping should contain key {}", kMatch);
}
uid = processMatch(uid, true, child_node.second, names_to_attributes);
uid = processMatch(true, uid, match[kMatch], names_to_attributes);
}
}
@ -515,8 +539,7 @@ Block parseYAMLAsRegExpTree(const YAML::Node & node)
for (auto & [name, attribute] : names_to_attributes)
{
auto column = ColumnWithTypeAndName(attribute.column->convertToFullIfNeeded(), attribute.data_type, name);
auto column = ColumnWithTypeAndName(attribute.getColumn(), attribute.getDataType(), attribute.getName());
columns.push_back(std::move(column));
}

View File

@ -0,0 +1,5 @@
#pragma once
// .h autogenerated by cmake!
#cmakedefine01 USE_VECTORSCAN

View File

@ -0,0 +1,3 @@
if (TARGET ch_contrib::vectorscan)
set(USE_VECTORSCAN 1)
endif()

View File

@ -26,6 +26,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory);
void registerDictionaryComplexKeyHashed(DictionaryFactory & factory);
void registerDictionaryTrie(DictionaryFactory & factory);
void registerDictionaryFlat(DictionaryFactory & factory);
void registerDictionaryRegExpTree(DictionaryFactory & factory);
void registerDictionaryHashed(DictionaryFactory & factory);
void registerDictionaryArrayHashed(DictionaryFactory & factory);
void registerDictionaryCache(DictionaryFactory & factory);
@ -58,6 +59,7 @@ void registerDictionaries()
registerDictionaryRangeHashed(factory);
registerDictionaryTrie(factory);
registerDictionaryFlat(factory);
registerDictionaryRegExpTree(factory);
registerDictionaryHashed(factory);
registerDictionaryArrayHashed(factory);
registerDictionaryCache(factory);

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<clickhouse>
<dictionaries_config>/etc/clickhouse-server/config.d/regexp_tree_dict.xml</dictionaries_config>
</clickhouse>

View File

@ -0,0 +1,38 @@
<?xml version="1.0"?>
<clickhouse>
<dictionary>
<name>ext-dict-yaml</name>
<source>
<YAMLRegExpTree>
<path>/etc/clickhouse-server/config.d/regexp_tree_config.yaml</path>
</YAMLRegExpTree>
</source>
<layout>
<reg-exp-tree />
</layout>
<lifetime>0</lifetime>
<structure>
<key>
<attribute>
<name>id</name>
<type>String</type>
</attribute>
</key>
<attribute>
<name>name</name>
<type>Nullable(String)</type>
<null_value></null_value>
</attribute>
<attribute>
<name>version</name>
<type>Nullable(String)</type>
<null_value></null_value>
</attribute>
<attribute>
<name>code</name>
<type>Nullable(String)</type>
<null_value></null_value>
</attribute>
</structure>
</dictionary>
</clickhouse>

View File

@ -0,0 +1,43 @@
attributes:
name:
type: String
version:
type: String
code:
type: String
lucky:
type: UInt
configuration:
# direct match
- match:
regexp: "InspectBrowser"
set:
name: "Inspect Browser"
# nested matches
- match:
regexp: "Google Chrome (.*) (.*)"
set:
name: "{0} {1}"
version: "{1} amd64"
match:
regexp: "(.*)"
set:
code: "{0}"
# collecting attributes
- match:
regexp: "Mozilla(?:/(\\d+[\\.\\d]+))"
set:
name: "Mozilla"
match:
regexp: ".*ozilla(?:/(\\d+[\\.\\d]+))"
set:
name: "Mozilla Wrong"
version: "{0}"
match:
regexp: ".*zilla(?:/(\\d+[\\.\\d]+))"
set:
name: "Mozilla Correct"
code: "{0}"

View File

@ -0,0 +1,82 @@
import pytest
from helpers.cluster import ClickHouseCluster
DICTIONARIES = ["configs/dictionaries/regexp_tree_dict.xml"]
CONFIG_FILES = ["configs/regexp_tree_config.yaml"]
DICT_NAME = 'ext-dict-yaml'
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance(
"instance", main_configs=CONFIG_FILES, dictionaries=DICTIONARIES
)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_simple_get(started_cluster):
query = instance.query
result = query(
f"""
SELECT dictGet('{DICT_NAME}', 'name', 'InspectBrowser');
"""
)
assert result.strip() == 'Inspect Browser'
def test_nested_get(started_cluster):
query = instance.query
result = query(
f"""
SELECT dictGet('{DICT_NAME}', ('name', 'version', 'code'), 'Google Chrome Chromium 1994');
"""
)
assert result.strip() == "('Chromium 1994','1994 amd64','Google Chrome Chromium 1994')"
def test_collecting_get(started_cluster):
query = instance.query
result = query(
f"""
SELECT dictGet('{DICT_NAME}', ('name', 'version', 'code'), 'Mozilla/5.0');
"""
)
assert result.strip() == "('Mozilla Correct','5.0','5.0')"
def test_not_found(started_cluster):
query = instance.query
result = query(
f"""
SELECT dictGet('{DICT_NAME}', 'name', 'Some weird key');
"""
)
assert result.strip() == '\\N'
def test_not_all_attributes_found(started_cluster):
query = instance.query
result = query(
f"""
SELECT dictGet('{DICT_NAME}', ('name', 'version'), 'InspectBrowser');
"""
)
assert result.strip() == "('Inspect Browser',NULL)"