mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #49666 from hanfei1991/hanfei/regexp-dict-read
Support `dictionary` table function for `RegExpTreeDictionary`
This commit is contained in:
commit
07341a04fd
@ -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:
|
||||
|
@ -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) \
|
||||
\
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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>;
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
};
|
||||
|
@ -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,
|
||||
|
@ -50,6 +50,7 @@ protected:
|
||||
|
||||
friend class StorageSystemDictionaries;
|
||||
friend class DatabaseDictionary;
|
||||
friend class TableFunctionDictionary;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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(
|
||||
|
@ -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']
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user