Merge pull request #49666 from hanfei1991/hanfei/regexp-dict-read

Support `dictionary` table function for `RegExpTreeDictionary`
This commit is contained in:
Han Fei 2023-05-12 11:40:46 +02:00 committed by GitHub
commit 07341a04fd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 144 additions and 31 deletions

View File

@ -2218,8 +2218,6 @@ LAYOUT(regexp_tree)
...
```
We only allow `YAMLRegExpTree` to work with regexp_tree dicitionary layout. If you want to use other sources, please set variable `regexp_dict_allow_other_sources` true.
**Source**
We introduce a type of source called `YAMLRegExpTree` representing the structure of Regexp Tree dictionary. An Example of a valid yaml config is like:

View File

@ -973,7 +973,6 @@ class IColumn;
M(Bool, output_format_bson_string_as_string, false, "Use BSON String type instead of Binary for String columns.", 0) \
M(Bool, input_format_bson_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format BSON.", 0) \
\
M(Bool, regexp_dict_allow_other_sources, false, "Allow regexp_tree dictionary to use sources other than yaml source.", 0) \
M(Bool, format_display_secrets_in_show_and_select, false, "Do not hide secrets in SHOW and SELECT queries.", IMPORTANT) \
M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \
\

View File

@ -71,11 +71,11 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
: update_time{std::chrono::system_clock::from_time_t(0)}
, dict_struct{dict_struct_}
, configuration{configuration_}
, query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks}
, query_builder(std::make_shared<ExternalQueryBuilder>(dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks))
, sample_block{sample_block_}
, context(context_)
, pool{createPool(configuration)}
, load_all_query{query_builder.composeLoadAllQuery()}
, load_all_query{query_builder->composeLoadAllQuery()}
{
}
@ -84,7 +84,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionar
, dict_struct{other.dict_struct}
, configuration{other.configuration}
, invalidate_query_response{other.invalidate_query_response}
, query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks}
, query_builder(std::make_shared<ExternalQueryBuilder>(dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks))
, sample_block{other.sample_block}
, context(Context::createCopy(other.context))
, pool{createPool(configuration)}
@ -99,12 +99,12 @@ std::string ClickHouseDictionarySource::getUpdateFieldAndDate()
time_t hr_time = std::chrono::system_clock::to_time_t(update_time) - configuration.update_lag;
std::string str_time = DateLUT::instance().timeToString(hr_time);
update_time = std::chrono::system_clock::now();
return query_builder.composeUpdateQuery(configuration.update_field, str_time);
return query_builder->composeUpdateQuery(configuration.update_field, str_time);
}
else
{
update_time = std::chrono::system_clock::now();
return query_builder.composeLoadAllQuery();
return query_builder->composeLoadAllQuery();
}
}
@ -121,13 +121,13 @@ QueryPipeline ClickHouseDictionarySource::loadUpdatedAll()
QueryPipeline ClickHouseDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
return createStreamForQuery(query_builder.composeLoadIdsQuery(ids));
return createStreamForQuery(query_builder->composeLoadIdsQuery(ids));
}
QueryPipeline ClickHouseDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
String query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::IN_WITH_TUPLES);
String query = query_builder->composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::IN_WITH_TUPLES);
return createStreamForQuery(query);
}

View File

@ -78,11 +78,11 @@ private:
const DictionaryStructure dict_struct;
const Configuration configuration;
mutable std::string invalidate_query_response;
ExternalQueryBuilder query_builder;
ExternalQueryBuilderPtr query_builder;
Block sample_block;
ContextMutablePtr context;
ConnectionPoolWithFailoverPtr pool;
const std::string load_all_query;
std::string load_all_query;
Poco::Logger * log = &Poco::Logger::get("ClickHouseDictionarySource");
/// RegExpTreeDictionary is the only dictionary whose structure of attributions differ from the input block.

View File

@ -36,6 +36,10 @@ struct ExternalQueryBuilder
const std::string & where_,
IdentifierQuotingStyle quoting_style_);
ExternalQueryBuilder(const ExternalQueryBuilder &) = default;
virtual ~ExternalQueryBuilder() = default;
/** Generate a query to load all data. */
std::string composeLoadAllQuery() const;
@ -61,10 +65,10 @@ struct ExternalQueryBuilder
std::string composeLoadKeysQuery(const Columns & key_columns, const std::vector<size_t> & requested_rows, LoadKeysMethod method, size_t partition_key_prefix = 0) const;
private:
protected:
const FormatSettings format_settings = {};
void composeLoadAllQuery(WriteBuffer & out) const;
virtual void composeLoadAllQuery(WriteBuffer & out) const;
/// In the following methods `beg` and `end` specifies which columns to write in expression
@ -93,4 +97,6 @@ private:
void writeQuoted(const std::string & s, WriteBuffer & out) const;
};
using ExternalQueryBuilderPtr = std::shared_ptr<ExternalQueryBuilder>;
}

View File

@ -20,6 +20,7 @@
#include <Functions/Regexps.h>
#include <Functions/checkHyperscanRegexp.h>
#include <QueryPipeline/QueryPipeline.h>
#include <Processors/Sources/BlocksListSource.h>
#include <Dictionaries/ClickHouseDictionarySource.h>
#include <Dictionaries/DictionaryFactory.h>
@ -86,6 +87,32 @@ namespace
}
}
struct ExternalRegexpQueryBuilder final : public ExternalQueryBuilder
{
explicit ExternalRegexpQueryBuilder(const ExternalQueryBuilder & builder) : ExternalQueryBuilder(builder) {}
void composeLoadAllQuery(WriteBuffer & out) const override
{
writeString("SELECT id, parent_id, regexp, keys, values FROM ", out);
if (!db.empty())
{
writeQuoted(db, out);
writeChar('.', out);
}
if (!schema.empty())
{
writeQuoted(schema, out);
writeChar('.', out);
}
writeQuoted(table, out);
if (!where.empty())
{
writeString(" WHERE ", out);
writeString(where, out);
}
}
};
struct RegExpTreeDictionary::RegexTreeNode
{
std::vector<UInt64> children;
@ -117,6 +144,7 @@ struct RegExpTreeDictionary::RegexTreeNode
{
Field field;
std::vector<StringPiece> pieces;
String original_value;
constexpr bool containsBackRefs() const { return !pieces.empty(); }
};
@ -208,12 +236,12 @@ void RegExpTreeDictionary::initRegexNodes(Block & block)
auto string_pieces = createStringPieces(value, num_captures, regex, logger);
if (!string_pieces.empty())
{
node->attributes[name_] = RegexTreeNode::AttributeValue{.field = values[j], .pieces = std::move(string_pieces)};
node->attributes[name_] = RegexTreeNode::AttributeValue{.field = values[j], .pieces = std::move(string_pieces), .original_value = value};
}
else
{
Field field = parseStringToField(values[j].safeGet<String>(), attr.type);
node->attributes[name_] = RegexTreeNode::AttributeValue{.field = std::move(field)};
Field field = parseStringToField(value, attr.type);
node->attributes[name_] = RegexTreeNode::AttributeValue{.field = std::move(field), .original_value = value};
}
}
}
@ -383,6 +411,8 @@ RegExpTreeDictionary::RegExpTreeDictionary(
sample_block.insert(ColumnWithTypeAndName(std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), kKeys));
sample_block.insert(ColumnWithTypeAndName(std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), kValues));
ch_source->sample_block = std::move(sample_block);
ch_source->query_builder = std::make_shared<ExternalRegexpQueryBuilder>(*ch_source->query_builder);
ch_source->load_all_query = ch_source->query_builder->composeLoadAllQuery();
}
loadData();
@ -651,6 +681,52 @@ std::unordered_map<String, ColumnPtr> RegExpTreeDictionary::match(
return result;
}
Pipe RegExpTreeDictionary::read(const Names & , size_t max_block_size, size_t) const
{
auto it = regex_nodes.begin();
size_t block_size = 0;
BlocksList result;
for (;;)
{
Block block;
auto col_id = std::make_shared<DataTypeUInt64>()->createColumn();
auto col_pid = std::make_shared<DataTypeUInt64>()->createColumn();
auto col_regex = std::make_shared<DataTypeString>()->createColumn();
auto col_keys = std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())->createColumn();
auto col_values = std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())->createColumn();
for (;it != regex_nodes.end() && block_size < max_block_size; it++, block_size++)
{
col_id->insert(it->first);
const auto & node = it->second;
col_pid->insert(node->parent_id);
col_regex->insert(node->regex);
std::vector<Field> keys, values;
for (const auto & [key, attr] : node->attributes)
{
keys.push_back(key);
values.push_back(attr.original_value);
}
col_keys->insert(Array(keys.begin(), keys.end()));
col_values->insert(Array(values.begin(), values.end()));
}
block.insert(ColumnWithTypeAndName(std::move(col_id),std::make_shared<DataTypeUInt64>(),kId));
block.insert(ColumnWithTypeAndName(std::move(col_pid),std::make_shared<DataTypeUInt64>(),kParentId));
block.insert(ColumnWithTypeAndName(std::move(col_regex),std::make_shared<DataTypeString>(),kRegExp));
block.insert(ColumnWithTypeAndName(std::move(col_keys),std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()),kKeys));
block.insert(ColumnWithTypeAndName(std::move(col_values),std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()),kValues));
result.push_back(std::move(block));
if (it == regex_nodes.end())
break;
block_size = 0;
}
return Pipe(std::make_shared<BlocksListSource>(std::move(result)));
}
Columns RegExpTreeDictionary::getColumns(
const Strings & attribute_names,
const DataTypes & result_types,
@ -717,10 +793,6 @@ void registerDictionaryRegExpTree(DictionaryFactory & factory)
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix);
if (!context->getSettings().regexp_dict_allow_other_sources && typeid_cast<YAMLRegExpTreeDictionarySource *>(source_ptr.get()) == nullptr)
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION,
"regexp_tree dictionary doesn't accept sources other than yaml source. "
"To active it, please set regexp_dict_allow_other_sources=true");
return std::make_unique<RegExpTreeDictionary>(dict_id, dict_struct, std::move(source_ptr), configuration, context->getSettings().regexp_dict_allow_hyperscan);
};

View File

@ -22,6 +22,8 @@
#include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/IDictionary.h>
#include <Storages/ColumnsDescription.h>
namespace DB
{
@ -91,10 +93,7 @@ public:
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);
}
Pipe read(const Names & columns, size_t max_block_size, size_t num_streams) const override;
ColumnPtr getColumn(
const std::string & attribute_name,

View File

@ -50,6 +50,7 @@ protected:
friend class StorageSystemDictionaries;
friend class DatabaseDictionary;
friend class TableFunctionDictionary;
};
}

View File

@ -2,6 +2,10 @@
#include <Parsers/ASTLiteral.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/evaluateConstantExpression.h>
@ -42,10 +46,33 @@ void TableFunctionDictionary::parseArguments(const ASTPtr & ast_function, Contex
ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr context) const
{
const ExternalDictionariesLoader & external_loader = context->getExternalDictionariesLoader();
auto dictionary_structure = external_loader.getDictionaryStructure(dictionary_name, context);
auto result = ColumnsDescription(StorageDictionary::getNamesAndTypes(dictionary_structure));
std::string resolved_name = external_loader.resolveDictionaryName(dictionary_name, context->getCurrentDatabase());
auto load_result = external_loader.load(resolved_name);
if (load_result)
{
/// for regexp tree dictionary, the table structure will be different with dictionary structure. it is:
/// - id. identifier of the tree node
/// - parent_id.
/// - regexp. the regular expression
/// - keys. the names of attributions of dictionary structure
/// - values. the values of each attribution
const auto dictionary = std::static_pointer_cast<const IDictionary>(load_result);
if (dictionary->getTypeName() == "RegExpTree")
{
return ColumnsDescription(NamesAndTypesList({
{"id", std::make_shared<DataTypeUInt64>()},
{"parent_id", std::make_shared<DataTypeUInt64>()},
{"regexp", std::make_shared<DataTypeString>()},
{"keys", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{"values", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}
}));
}
}
/// otherwise, we get table structure by dictionary structure.
auto dictionary_structure = external_loader.getDictionaryStructure(dictionary_name, context);
return ColumnsDescription(StorageDictionary::getNamesAndTypes(dictionary_structure));
return result;
}
StoragePtr TableFunctionDictionary::executeImpl(

View File

@ -1,3 +1,9 @@
1 0 Linux/(\\d+[\\.\\d]*).+tlinux ['version','name'] ['\\1','TencentOS']
2 0 (\\d+)/tclwebkit(\\d+[\\.\\d]*) ['comment','version','name'] ['test $1 and $2','$1','Andriod']
3 2 33/tclwebkit ['version'] ['13']
4 2 3[12]/tclwebkit ['version'] ['12']
5 2 3[12]/tclwebkit ['version'] ['11']
6 2 3[12]/tclwebkit ['version'] ['10']
('TencentOS',101,'nothing')
('Andriod',13,'test 33 and 11.10')
('',NULL,'nothing')
@ -33,3 +39,6 @@
('Andriod',43)
('Andriod',44)
('Andriod1',33,'matched 3')
1 0 (\\d+)/tclwebkit ['version','name'] ['$1','Andriod']
2 0 33/tclwebkit ['comment','version'] ['matched 3','13']
3 1 33/tclwebkit ['name'] ['Andriod1']

View File

@ -29,10 +29,11 @@ create dictionary regexp_dict1
comment String default 'nothing'
)
PRIMARY KEY(regexp)
SOURCE(CLICKHOUSE(QUERY concat('select * from ', currentDatabase() , '.regexp_dictionary_source_table')))
SOURCE(CLICKHOUSE(TABLE 'regexp_dictionary_source_table'))
LIFETIME(0)
LAYOUT(regexp_tree)
SETTINGS(regexp_dict_allow_other_sources = true);
LAYOUT(regexp_tree);
select * from dictionary(regexp_dict1);
select dictGet('regexp_dict1', ('name', 'version', 'comment'), 'Linux/101.tlinux');
select dictGet('regexp_dict1', ('name', 'version', 'comment'), '33/tclwebkit11.10x');
@ -79,6 +80,7 @@ select dictGet(regexp_dict1, ('name', 'version', 'comment'), '33/tclwebkit');
truncate table regexp_dictionary_source_table;
SYSTEM RELOAD dictionary regexp_dict1; -- { serverError 489 }
select * from dictionary(regexp_dict1);
DROP TABLE IF EXISTS regexp_dictionary_source_table;
DROP TABLE IF EXISTS needle_table;