Merge pull request #61543 from ClickHouse/fix-regexp-recompilation-in-parser

Fix recompilation of regexps during backtracking in the parser
This commit is contained in:
Alexey Milovidov 2024-03-21 21:15:17 +03:00 committed by GitHub
commit 8670f1dc91
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
17 changed files with 125 additions and 142 deletions

View File

@ -1,18 +1,18 @@
#include <Analyzer/ColumnTransformers.h> #include <Analyzer/ColumnTransformers.h>
#include <Common/SipHash.h> #include <Common/SipHash.h>
#include <Common/re2.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTColumnsTransformers.h> #include <Parsers/ASTColumnsTransformers.h>
#include <Analyzer/FunctionNode.h> #include <Analyzer/FunctionNode.h>
#include <Analyzer/LambdaNode.h> #include <Analyzer/LambdaNode.h>
namespace DB namespace DB
{ {
@ -133,7 +133,7 @@ ExceptColumnTransformerNode::ExceptColumnTransformerNode(std::shared_ptr<re2::RE
bool ExceptColumnTransformerNode::isColumnMatching(const std::string & column_name) const bool ExceptColumnTransformerNode::isColumnMatching(const std::string & column_name) const
{ {
if (column_matcher) if (column_matcher)
return RE2::PartialMatch(column_name, *column_matcher); return re2::RE2::PartialMatch(column_name, *column_matcher);
for (const auto & name : except_column_names) for (const auto & name : except_column_names)
if (column_name == name) if (column_name == name)

View File

@ -3,7 +3,12 @@
#include <Analyzer/Identifier.h> #include <Analyzer/Identifier.h>
#include <Analyzer/IQueryTreeNode.h> #include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/ListNode.h> #include <Analyzer/ListNode.h>
#include <Common/re2.h>
namespace re2
{
class RE2;
}
namespace DB namespace DB
{ {
@ -102,8 +107,6 @@ enum class ApplyColumnTransformerType
/// Get apply column transformer type name /// Get apply column transformer type name
const char * toString(ApplyColumnTransformerType type); const char * toString(ApplyColumnTransformerType type);
class ApplyColumnTransformerNode;
using ApplyColumnTransformerNodePtr = std::shared_ptr<ApplyColumnTransformerNode>;
/// Apply column transformer /// Apply column transformer
class ApplyColumnTransformerNode final : public IColumnTransformerNode class ApplyColumnTransformerNode final : public IColumnTransformerNode
@ -158,8 +161,6 @@ enum class ExceptColumnTransformerType
const char * toString(ExceptColumnTransformerType type); const char * toString(ExceptColumnTransformerType type);
class ExceptColumnTransformerNode;
using ExceptColumnTransformerNodePtr = std::shared_ptr<ExceptColumnTransformerNode>;
/** Except column transformer. /** Except column transformer.
* Strict EXCEPT column transformer must use all column names during matched nodes transformation. * Strict EXCEPT column transformer must use all column names during matched nodes transformation.
@ -230,8 +231,6 @@ private:
static constexpr size_t children_size = 0; static constexpr size_t children_size = 0;
}; };
class ReplaceColumnTransformerNode;
using ReplaceColumnTransformerNodePtr = std::shared_ptr<ReplaceColumnTransformerNode>;
/** Replace column transformer. /** Replace column transformer.
* Strict replace column transformer must use all replacements during matched nodes transformation. * Strict replace column transformer must use all replacements during matched nodes transformation.

View File

@ -4,7 +4,6 @@
#include <Common/SipHash.h> #include <Common/SipHash.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
@ -14,9 +13,15 @@
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTColumnsTransformers.h> #include <Parsers/ASTColumnsTransformers.h>
namespace DB namespace DB
{ {
namespace ErrorCodes
{
extern const int CANNOT_COMPILE_REGEXP;
}
const char * toString(MatcherNodeType matcher_node_type) const char * toString(MatcherNodeType matcher_node_type)
{ {
switch (matcher_node_type) switch (matcher_node_type)
@ -48,20 +53,20 @@ MatcherNode::MatcherNode(Identifier qualified_identifier_, ColumnTransformersNod
{ {
} }
MatcherNode::MatcherNode(std::shared_ptr<re2::RE2> columns_matcher_, ColumnTransformersNodes column_transformers_) MatcherNode::MatcherNode(String pattern_, ColumnTransformersNodes column_transformers_)
: MatcherNode(MatcherNodeType::COLUMNS_REGEXP, : MatcherNode(MatcherNodeType::COLUMNS_REGEXP,
{} /*qualified_identifier*/, {} /*qualified_identifier*/,
{} /*columns_identifiers*/, {} /*columns_identifiers*/,
std::move(columns_matcher_), std::move(pattern_),
std::move(column_transformers_)) std::move(column_transformers_))
{ {
} }
MatcherNode::MatcherNode(Identifier qualified_identifier_, std::shared_ptr<re2::RE2> columns_matcher_, ColumnTransformersNodes column_transformers_) MatcherNode::MatcherNode(Identifier qualified_identifier_, String pattern_, ColumnTransformersNodes column_transformers_)
: MatcherNode(MatcherNodeType::COLUMNS_REGEXP, : MatcherNode(MatcherNodeType::COLUMNS_REGEXP,
std::move(qualified_identifier_), std::move(qualified_identifier_),
{} /*columns_identifiers*/, {} /*columns_identifiers*/,
std::move(columns_matcher_), std::move(pattern_),
std::move(column_transformers_)) std::move(column_transformers_))
{ {
} }
@ -87,14 +92,21 @@ MatcherNode::MatcherNode(Identifier qualified_identifier_, Identifiers columns_i
MatcherNode::MatcherNode(MatcherNodeType matcher_type_, MatcherNode::MatcherNode(MatcherNodeType matcher_type_,
Identifier qualified_identifier_, Identifier qualified_identifier_,
Identifiers columns_identifiers_, Identifiers columns_identifiers_,
std::shared_ptr<re2::RE2> columns_matcher_, std::optional<String> pattern_,
ColumnTransformersNodes column_transformers_) ColumnTransformersNodes column_transformers_)
: IQueryTreeNode(children_size) : IQueryTreeNode(children_size)
, matcher_type(matcher_type_) , matcher_type(matcher_type_)
, qualified_identifier(qualified_identifier_) , qualified_identifier(qualified_identifier_)
, columns_identifiers(columns_identifiers_) , columns_identifiers(columns_identifiers_)
, columns_matcher(columns_matcher_)
{ {
if (pattern_)
{
columns_matcher = std::make_shared<re2::RE2>(*pattern_, re2::RE2::Quiet);
if (!columns_matcher->ok())
throw DB::Exception(ErrorCodes::CANNOT_COMPILE_REGEXP,
"COLUMNS pattern {} cannot be compiled: {}", *pattern_, columns_matcher->error());
}
auto column_transformers_list_node = std::make_shared<ListNode>(); auto column_transformers_list_node = std::make_shared<ListNode>();
auto & column_transformers_nodes = column_transformers_list_node->getNodes(); auto & column_transformers_nodes = column_transformers_list_node->getNodes();

View File

@ -6,6 +6,7 @@
#include <Parsers/ASTAsterisk.h> #include <Parsers/ASTAsterisk.h>
#include <Common/re2.h> #include <Common/re2.h>
namespace DB namespace DB
{ {
@ -58,10 +59,10 @@ public:
explicit MatcherNode(Identifier qualified_identifier_, ColumnTransformersNodes column_transformers_ = {}); explicit MatcherNode(Identifier qualified_identifier_, ColumnTransformersNodes column_transformers_ = {});
/// Variant unqualified COLUMNS('regexp') /// Variant unqualified COLUMNS('regexp')
explicit MatcherNode(std::shared_ptr<re2::RE2> columns_matcher_, ColumnTransformersNodes column_transformers_ = {}); explicit MatcherNode(String pattern_, ColumnTransformersNodes column_transformers_ = {});
/// Variant qualified COLUMNS('regexp') /// Variant qualified COLUMNS('regexp')
explicit MatcherNode(Identifier qualified_identifier_, std::shared_ptr<re2::RE2> columns_matcher_, ColumnTransformersNodes column_transformers_ = {}); explicit MatcherNode(Identifier qualified_identifier_, String pattern_, ColumnTransformersNodes column_transformers_ = {});
/// Variant unqualified COLUMNS(column_name_1, ...) /// Variant unqualified COLUMNS(column_name_1, ...)
explicit MatcherNode(Identifiers columns_identifiers_, ColumnTransformersNodes column_transformers_ = {}); explicit MatcherNode(Identifiers columns_identifiers_, ColumnTransformersNodes column_transformers_ = {});
@ -81,12 +82,6 @@ public:
return matcher_type == MatcherNodeType::ASTERISK; return matcher_type == MatcherNodeType::ASTERISK;
} }
/// Returns true if matcher is columns regexp or columns list matcher, false otherwise
bool isColumnsMatcher() const
{
return matcher_type == MatcherNodeType::COLUMNS_REGEXP || matcher_type == MatcherNodeType::COLUMNS_LIST;
}
/// Returns true if matcher is qualified, false otherwise /// Returns true if matcher is qualified, false otherwise
bool isQualified() const bool isQualified() const
{ {
@ -152,7 +147,7 @@ private:
explicit MatcherNode(MatcherNodeType matcher_type_, explicit MatcherNode(MatcherNodeType matcher_type_,
Identifier qualified_identifier_, Identifier qualified_identifier_,
Identifiers columns_identifiers_, Identifiers columns_identifiers_,
std::shared_ptr<re2::RE2> columns_matcher_, std::optional<String> pattern_,
ColumnTransformersNodes column_transformers_); ColumnTransformersNodes column_transformers_);
MatcherNodeType matcher_type; MatcherNodeType matcher_type;

View File

@ -2,11 +2,7 @@
#include <Common/FieldVisitorToString.h> #include <Common/FieldVisitorToString.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/FieldToDataType.h> #include <DataTypes/FieldToDataType.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ParserSelectWithUnionQuery.h> #include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h> #include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h> #include <Parsers/ASTSelectIntersectExceptQuery.h>
@ -33,13 +29,11 @@
#include <Analyzer/MatcherNode.h> #include <Analyzer/MatcherNode.h>
#include <Analyzer/ColumnTransformers.h> #include <Analyzer/ColumnTransformers.h>
#include <Analyzer/ConstantNode.h> #include <Analyzer/ConstantNode.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/FunctionNode.h> #include <Analyzer/FunctionNode.h>
#include <Analyzer/LambdaNode.h> #include <Analyzer/LambdaNode.h>
#include <Analyzer/SortNode.h> #include <Analyzer/SortNode.h>
#include <Analyzer/InterpolateNode.h> #include <Analyzer/InterpolateNode.h>
#include <Analyzer/WindowNode.h> #include <Analyzer/WindowNode.h>
#include <Analyzer/TableNode.h>
#include <Analyzer/TableFunctionNode.h> #include <Analyzer/TableFunctionNode.h>
#include <Analyzer/QueryNode.h> #include <Analyzer/QueryNode.h>
#include <Analyzer/ArrayJoinNode.h> #include <Analyzer/ArrayJoinNode.h>
@ -50,7 +44,6 @@
#include <Interpreters/StorageID.h> #include <Interpreters/StorageID.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Functions/FunctionFactory.h>
namespace DB namespace DB
@ -661,7 +654,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
else if (const auto * columns_regexp_matcher = expression->as<ASTColumnsRegexpMatcher>()) else if (const auto * columns_regexp_matcher = expression->as<ASTColumnsRegexpMatcher>())
{ {
auto column_transformers = buildColumnTransformers(columns_regexp_matcher->transformers, context); auto column_transformers = buildColumnTransformers(columns_regexp_matcher->transformers, context);
result = std::make_shared<MatcherNode>(columns_regexp_matcher->getMatcher(), std::move(column_transformers)); result = std::make_shared<MatcherNode>(columns_regexp_matcher->getPattern(), std::move(column_transformers));
} }
else if (const auto * columns_list_matcher = expression->as<ASTColumnsListMatcher>()) else if (const auto * columns_list_matcher = expression->as<ASTColumnsListMatcher>())
{ {
@ -681,7 +674,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
{ {
auto & qualified_identifier = qualified_columns_regexp_matcher->qualifier->as<ASTIdentifier &>(); auto & qualified_identifier = qualified_columns_regexp_matcher->qualifier->as<ASTIdentifier &>();
auto column_transformers = buildColumnTransformers(qualified_columns_regexp_matcher->transformers, context); auto column_transformers = buildColumnTransformers(qualified_columns_regexp_matcher->transformers, context);
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), qualified_columns_regexp_matcher->getMatcher(), std::move(column_transformers)); result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), qualified_columns_regexp_matcher->getPattern(), std::move(column_transformers));
} }
else if (const auto * qualified_columns_list_matcher = expression->as<ASTQualifiedColumnsListMatcher>()) else if (const auto * qualified_columns_list_matcher = expression->as<ASTQualifiedColumnsListMatcher>())
{ {

View File

@ -60,7 +60,7 @@ public:
, replacement(replacement_string) , replacement(replacement_string)
{ {
if (!regexp.ok()) if (!regexp.ok())
throw DB::Exception(DB::ErrorCodes::CANNOT_COMPILE_REGEXP, throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP,
"SensitiveDataMasker: cannot compile re2: {}, error: {}. " "SensitiveDataMasker: cannot compile re2: {}, error: {}. "
"Look at https://github.com/google/re2/wiki/Syntax for reference.", "Look at https://github.com/google/re2/wiki/Syntax for reference.",
regexp_string_, regexp.error()); regexp_string_, regexp.error());

View File

@ -15,11 +15,10 @@
#include <Parsers/ASTColumnsTransformers.h> #include <Parsers/ASTColumnsTransformers.h>
#include <Parsers/ASTQualifiedAsterisk.h> #include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/ParserTablesInSelectQuery.h> #include <Parsers/ParserTablesInSelectQuery.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <IO/WriteHelpers.h>
#include <Core/Defines.h> #include <Core/Defines.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Common/re2.h>
namespace DB namespace DB
{ {
@ -30,6 +29,7 @@ namespace ErrorCodes
extern const int AMBIGUOUS_COLUMN_NAME; extern const int AMBIGUOUS_COLUMN_NAME;
extern const int NOT_IMPLEMENTED; extern const int NOT_IMPLEMENTED;
extern const int UNKNOWN_IDENTIFIER; extern const int UNKNOWN_IDENTIFIER;
extern const int CANNOT_COMPILE_REGEXP;
} }
namespace namespace
@ -204,11 +204,17 @@ private:
{ {
has_asterisks = true; has_asterisks = true;
String pattern = columns_regexp_matcher->getPattern();
re2::RE2 regexp(pattern, re2::RE2::Quiet);
if (!regexp.ok())
throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP,
"COLUMNS pattern {} cannot be compiled: {}", pattern, regexp.error());
for (auto & table_name : data.tables_order) for (auto & table_name : data.tables_order)
data.addTableColumns( data.addTableColumns(
table_name, table_name,
columns, columns,
[&](const String & column_name) { return columns_regexp_matcher->isColumnMatching(column_name); }); [&](const String & column_name) { return re2::RE2::PartialMatch(column_name, regexp); });
if (columns_regexp_matcher->transformers) if (columns_regexp_matcher->transformers)
{ {

View File

@ -1,4 +1,3 @@
#include <cstring>
#include <Poco/String.h> #include <Poco/String.h>
#include <Interpreters/TranslateQualifiedNamesVisitor.h> #include <Interpreters/TranslateQualifiedNamesVisitor.h>
@ -6,7 +5,6 @@
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Core/Names.h>
#include <DataTypes/DataTypeTuple.h> #include <DataTypes/DataTypeTuple.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
@ -21,6 +19,7 @@
#include <Parsers/ASTColumnsMatcher.h> #include <Parsers/ASTColumnsMatcher.h>
#include <Parsers/ASTColumnsTransformers.h> #include <Parsers/ASTColumnsTransformers.h>
#include <Storages/StorageView.h> #include <Storages/StorageView.h>
#include <Common/re2.h>
namespace DB namespace DB
@ -31,7 +30,10 @@ namespace ErrorCodes
extern const int UNKNOWN_IDENTIFIER; extern const int UNKNOWN_IDENTIFIER;
extern const int UNSUPPORTED_JOIN_KEYS; extern const int UNSUPPORTED_JOIN_KEYS;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int CANNOT_COMPILE_REGEXP;
} }
bool TranslateQualifiedNamesMatcher::Data::matchColumnName(std::string_view name, const String & column_name, DataTypePtr column_type) bool TranslateQualifiedNamesMatcher::Data::matchColumnName(std::string_view name, const String & column_name, DataTypePtr column_type)
{ {
if (name.size() < column_name.size()) if (name.size() < column_name.size())
@ -275,12 +277,19 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
} }
else if (const auto * asterisk_regexp_pattern = child->as<ASTColumnsRegexpMatcher>()) else if (const auto * asterisk_regexp_pattern = child->as<ASTColumnsRegexpMatcher>())
{ {
String pattern = asterisk_regexp_pattern->getPattern();
re2::RE2 regexp(pattern, re2::RE2::Quiet);
if (!regexp.ok())
throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP,
"COLUMNS pattern {} cannot be compiled: {}", pattern, regexp.error());
bool first_table = true; bool first_table = true;
for (const auto & table : tables_with_columns) for (const auto & table : tables_with_columns)
{ {
for (const auto & column : table.columns) for (const auto & column : table.columns)
{ {
if (asterisk_regexp_pattern->isColumnMatching(column.name) && (first_table || !data.join_using_columns.contains(column.name))) if (re2::RE2::PartialMatch(column.name, regexp)
&& (first_table || !data.join_using_columns.contains(column.name)))
{ {
addIdentifier(columns, table.table, column.name); addIdentifier(columns, table.table, column.name);
} }

View File

@ -4,17 +4,11 @@
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Common/SipHash.h> #include <Common/SipHash.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Common/re2.h>
namespace DB namespace DB
{ {
namespace ErrorCodes
{
extern const int CANNOT_COMPILE_REGEXP;
}
ASTPtr ASTColumnsRegexpMatcher::clone() const ASTPtr ASTColumnsRegexpMatcher::clone() const
{ {
auto clone = std::make_shared<ASTColumnsRegexpMatcher>(*this); auto clone = std::make_shared<ASTColumnsRegexpMatcher>(*this);
@ -34,14 +28,14 @@ void ASTColumnsRegexpMatcher::appendColumnName(WriteBuffer & ostr) const
writeCString(".", ostr); writeCString(".", ostr);
} }
writeCString("COLUMNS(", ostr); writeCString("COLUMNS(", ostr);
writeQuotedString(original_pattern, ostr); writeQuotedString(pattern, ostr);
writeChar(')', ostr); writeChar(')', ostr);
} }
void ASTColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const void ASTColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(original_pattern.size()); hash_state.update(pattern.size());
hash_state.update(original_pattern); hash_state.update(pattern);
IAST::updateTreeHashImpl(hash_state, ignore_aliases); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
@ -56,7 +50,7 @@ void ASTColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, Format
} }
settings.ostr << "COLUMNS" << (settings.hilite ? hilite_none : "") << "("; settings.ostr << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
settings.ostr << quoteString(original_pattern); settings.ostr << quoteString(pattern);
settings.ostr << ")"; settings.ostr << ")";
if (transformers) if (transformers)
@ -65,28 +59,14 @@ void ASTColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, Format
} }
} }
void ASTColumnsRegexpMatcher::setPattern(String pattern) void ASTColumnsRegexpMatcher::setPattern(String pattern_)
{ {
original_pattern = std::move(pattern); pattern = std::move(pattern_);
column_matcher = std::make_shared<RE2>(original_pattern, RE2::Quiet);
if (!column_matcher->ok())
throw DB::Exception(DB::ErrorCodes::CANNOT_COMPILE_REGEXP,
"COLUMNS pattern {} cannot be compiled: {}", original_pattern, column_matcher->error());
} }
const String & ASTColumnsRegexpMatcher::getPattern() const const String & ASTColumnsRegexpMatcher::getPattern() const
{ {
return original_pattern; return pattern;
}
const std::shared_ptr<re2::RE2> & ASTColumnsRegexpMatcher::getMatcher() const
{
return column_matcher;
}
bool ASTColumnsRegexpMatcher::isColumnMatching(const String & column_name) const
{
return RE2::PartialMatch(column_name, *column_matcher);
} }
ASTPtr ASTColumnsListMatcher::clone() const ASTPtr ASTColumnsListMatcher::clone() const
@ -166,37 +146,24 @@ void ASTQualifiedColumnsRegexpMatcher::appendColumnName(WriteBuffer & ostr) cons
{ {
qualifier->appendColumnName(ostr); qualifier->appendColumnName(ostr);
writeCString(".COLUMNS(", ostr); writeCString(".COLUMNS(", ostr);
writeQuotedString(original_pattern, ostr); writeQuotedString(pattern, ostr);
writeChar(')', ostr); writeChar(')', ostr);
} }
void ASTQualifiedColumnsRegexpMatcher::setPattern(String pattern, bool set_matcher) void ASTQualifiedColumnsRegexpMatcher::setPattern(String pattern_)
{ {
original_pattern = std::move(pattern); pattern = std::move(pattern_);
if (!set_matcher)
return;
column_matcher = std::make_shared<RE2>(original_pattern, RE2::Quiet);
if (!column_matcher->ok())
throw DB::Exception(DB::ErrorCodes::CANNOT_COMPILE_REGEXP,
"COLUMNS pattern {} cannot be compiled: {}", original_pattern, column_matcher->error());
} }
void ASTQualifiedColumnsRegexpMatcher::setMatcher(std::shared_ptr<re2::RE2> matcher) const String & ASTQualifiedColumnsRegexpMatcher::getPattern() const
{ {
column_matcher = std::move(matcher); return pattern;
}
const std::shared_ptr<re2::RE2> & ASTQualifiedColumnsRegexpMatcher::getMatcher() const
{
return column_matcher;
} }
void ASTQualifiedColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const void ASTQualifiedColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(original_pattern.size()); hash_state.update(pattern.size());
hash_state.update(original_pattern); hash_state.update(pattern);
IAST::updateTreeHashImpl(hash_state, ignore_aliases); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
@ -207,7 +174,7 @@ void ASTQualifiedColumnsRegexpMatcher::formatImpl(const FormatSettings & setting
qualifier->formatImpl(settings, state, frame); qualifier->formatImpl(settings, state, frame);
settings.ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "("; settings.ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
settings.ostr << quoteString(original_pattern); settings.ostr << quoteString(pattern);
settings.ostr << ")"; settings.ostr << ")";
if (transformers) if (transformers)

View File

@ -2,11 +2,6 @@
#include <Parsers/IAST.h> #include <Parsers/IAST.h>
namespace re2
{
class RE2;
}
namespace DB namespace DB
{ {
@ -25,8 +20,6 @@ public:
void appendColumnName(WriteBuffer & ostr) const override; void appendColumnName(WriteBuffer & ostr) const override;
void setPattern(String pattern); void setPattern(String pattern);
const String & getPattern() const; const String & getPattern() const;
const std::shared_ptr<re2::RE2> & getMatcher() const;
bool isColumnMatching(const String & column_name) const;
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
ASTPtr expression; ASTPtr expression;
@ -35,8 +28,7 @@ protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
private: private:
std::shared_ptr<re2::RE2> column_matcher; String pattern;
String original_pattern;
}; };
/// Same as the above but use a list of column names to do matching. /// Same as the above but use a list of column names to do matching.
@ -62,9 +54,8 @@ public:
ASTPtr clone() const override; ASTPtr clone() const override;
void appendColumnName(WriteBuffer & ostr) const override; void appendColumnName(WriteBuffer & ostr) const override;
const std::shared_ptr<re2::RE2> & getMatcher() const; void setPattern(String pattern_);
void setPattern(String pattern, bool set_matcher = true); const String & getPattern() const;
void setMatcher(std::shared_ptr<re2::RE2> matcher);
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
ASTPtr qualifier; ASTPtr qualifier;
@ -73,8 +64,7 @@ protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
private: private:
std::shared_ptr<re2::RE2> column_matcher; String pattern;
String original_pattern;
}; };
/// Same as ASTColumnsListMatcher. Qualified identifier is first child. /// Same as ASTColumnsListMatcher. Qualified identifier is first child.

View File

@ -12,6 +12,7 @@
namespace DB namespace DB
{ {
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_TYPE_OF_ARGUMENT;
@ -179,8 +180,8 @@ void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, Fo
(*it)->formatImpl(settings, state, frame); (*it)->formatImpl(settings, state, frame);
} }
if (!original_pattern.empty()) if (pattern)
settings.ostr << quoteString(original_pattern); settings.ostr << quoteString(*pattern);
if (children.size() > 1) if (children.size() > 1)
settings.ostr << ")"; settings.ostr << ")";
@ -202,8 +203,8 @@ void ASTColumnsExceptTransformer::appendColumnName(WriteBuffer & ostr) const
(*it)->appendColumnName(ostr); (*it)->appendColumnName(ostr);
} }
if (!original_pattern.empty()) if (pattern)
writeQuotedString(original_pattern, ostr); writeQuotedString(*pattern, ostr);
if (children.size() > 1) if (children.size() > 1)
writeChar(')', ostr); writeChar(')', ostr);
@ -212,8 +213,11 @@ void ASTColumnsExceptTransformer::appendColumnName(WriteBuffer & ostr) const
void ASTColumnsExceptTransformer::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const void ASTColumnsExceptTransformer::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
{ {
hash_state.update(is_strict); hash_state.update(is_strict);
hash_state.update(original_pattern.size()); if (pattern)
hash_state.update(original_pattern); {
hash_state.update(pattern->size());
hash_state.update(*pattern);
}
IAST::updateTreeHashImpl(hash_state, ignore_aliases); IAST::updateTreeHashImpl(hash_state, ignore_aliases);
} }
@ -221,7 +225,7 @@ void ASTColumnsExceptTransformer::updateTreeHashImpl(SipHash & hash_state, bool
void ASTColumnsExceptTransformer::transform(ASTs & nodes) const void ASTColumnsExceptTransformer::transform(ASTs & nodes) const
{ {
std::set<String> expected_columns; std::set<String> expected_columns;
if (original_pattern.empty()) if (!pattern)
{ {
for (const auto & child : children) for (const auto & child : children)
expected_columns.insert(child->as<const ASTIdentifier &>().name()); expected_columns.insert(child->as<const ASTIdentifier &>().name());
@ -243,11 +247,13 @@ void ASTColumnsExceptTransformer::transform(ASTs & nodes) const
} }
else else
{ {
auto regexp = getMatcher();
for (auto * it = nodes.begin(); it != nodes.end();) for (auto * it = nodes.begin(); it != nodes.end();)
{ {
if (const auto * id = it->get()->as<ASTIdentifier>()) if (const auto * id = it->get()->as<ASTIdentifier>())
{ {
if (isColumnMatching(id->shortName())) if (RE2::PartialMatch(id->shortName(), *regexp))
{ {
it = nodes.erase(it); it = nodes.erase(it);
continue; continue;
@ -268,23 +274,21 @@ void ASTColumnsExceptTransformer::transform(ASTs & nodes) const
} }
} }
void ASTColumnsExceptTransformer::setPattern(String pattern) void ASTColumnsExceptTransformer::setPattern(String pattern_)
{ {
original_pattern = std::move(pattern); pattern = std::move(pattern_);
column_matcher = std::make_shared<RE2>(original_pattern, RE2::Quiet);
if (!column_matcher->ok())
throw DB::Exception(DB::ErrorCodes::CANNOT_COMPILE_REGEXP, "COLUMNS pattern {} cannot be compiled: {}",
original_pattern, column_matcher->error());
} }
const std::shared_ptr<re2::RE2> & ASTColumnsExceptTransformer::getMatcher() const std::shared_ptr<re2::RE2> ASTColumnsExceptTransformer::getMatcher() const
{ {
return column_matcher; if (!pattern)
} return {};
bool ASTColumnsExceptTransformer::isColumnMatching(const String & column_name) const auto regexp = std::make_shared<re2::RE2>(*pattern, re2::RE2::Quiet);
{ if (!regexp->ok())
return RE2::PartialMatch(column_name, *column_matcher); throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP,
"COLUMNS pattern {} cannot be compiled: {}", *pattern, regexp->error());
return regexp;
} }
void ASTColumnsReplaceTransformer::Replacement::formatImpl( void ASTColumnsReplaceTransformer::Replacement::formatImpl(

View File

@ -2,6 +2,7 @@
#include <Parsers/IAST.h> #include <Parsers/IAST.h>
namespace re2 namespace re2
{ {
class RE2; class RE2;
@ -76,16 +77,14 @@ public:
return clone; return clone;
} }
void transform(ASTs & nodes) const override; void transform(ASTs & nodes) const override;
void setPattern(String pattern); void setPattern(String pattern_);
const std::shared_ptr<re2::RE2> & getMatcher() const; std::shared_ptr<re2::RE2> getMatcher() const;
bool isColumnMatching(const String & column_name) const;
void appendColumnName(WriteBuffer & ostr) const override; void appendColumnName(WriteBuffer & ostr) const override;
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
protected: protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
std::shared_ptr<re2::RE2> column_matcher; std::optional<String> pattern;
String original_pattern;
}; };
class ASTColumnsReplaceTransformer : public IASTColumnsTransformer class ASTColumnsReplaceTransformer : public IASTColumnsTransformer

View File

@ -36,7 +36,6 @@
#include <Parsers/ExpressionListParsers.h> #include <Parsers/ExpressionListParsers.h>
#include <Parsers/IAST_fwd.h> #include <Parsers/IAST_fwd.h>
#include <Parsers/ParserSelectWithUnionQuery.h> #include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserCase.h>
#include <Parsers/ExpressionElementParsers.h> #include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserCreateQuery.h> #include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserExplainQuery.h> #include <Parsers/ParserExplainQuery.h>
@ -44,6 +43,7 @@
#include <Interpreters/StorageID.h> #include <Interpreters/StorageID.h>
namespace DB namespace DB
{ {
@ -1628,7 +1628,7 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e
is_strict = true; is_strict = true;
ASTs identifiers; ASTs identifiers;
ASTPtr regex_node; ASTPtr regexp_node;
ParserStringLiteral regex; ParserStringLiteral regex;
auto parse_id = [&identifiers, &pos, &expected] auto parse_id = [&identifiers, &pos, &expected]
{ {
@ -1644,7 +1644,7 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e
{ {
// support one or more parameter // support one or more parameter
++pos; ++pos;
if (!ParserList::parseUtil(pos, expected, parse_id, false) && !regex.parse(pos, regex_node, expected)) if (!ParserList::parseUtil(pos, expected, parse_id, false) && !regex.parse(pos, regexp_node, expected))
return false; return false;
if (pos->type != TokenType::ClosingRoundBracket) if (pos->type != TokenType::ClosingRoundBracket)
@ -1654,13 +1654,13 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e
else else
{ {
// only one parameter // only one parameter
if (!parse_id() && !regex.parse(pos, regex_node, expected)) if (!parse_id() && !regex.parse(pos, regexp_node, expected))
return false; return false;
} }
auto res = std::make_shared<ASTColumnsExceptTransformer>(); auto res = std::make_shared<ASTColumnsExceptTransformer>();
if (regex_node) if (regexp_node)
res->setPattern(regex_node->as<ASTLiteral &>().value.get<String>()); res->setPattern(regexp_node->as<ASTLiteral &>().value.get<String>());
else else
res->children = std::move(identifiers); res->children = std::move(identifiers);
res->is_strict = is_strict; res->is_strict = is_strict;
@ -1794,11 +1794,11 @@ static bool parseColumnsMatcherBody(IParser::Pos & pos, ASTPtr & node, Expected
++pos; ++pos;
ParserList columns_p(std::make_unique<ParserCompoundIdentifier>(false, true), std::make_unique<ParserToken>(TokenType::Comma), false); ParserList columns_p(std::make_unique<ParserCompoundIdentifier>(false, true), std::make_unique<ParserToken>(TokenType::Comma), false);
ParserStringLiteral regex; ParserStringLiteral regexp;
ASTPtr column_list; ASTPtr column_list;
ASTPtr regex_node; ASTPtr regexp_node;
if (!columns_p.parse(pos, column_list, expected) && !regex.parse(pos, regex_node, expected)) if (!columns_p.parse(pos, column_list, expected) && !regexp.parse(pos, regexp_node, expected))
return false; return false;
if (pos->type != TokenType::ClosingRoundBracket) if (pos->type != TokenType::ClosingRoundBracket)
@ -1832,7 +1832,7 @@ static bool parseColumnsMatcherBody(IParser::Pos & pos, ASTPtr & node, Expected
else else
{ {
auto regexp_matcher = std::make_shared<ASTColumnsRegexpMatcher>(); auto regexp_matcher = std::make_shared<ASTColumnsRegexpMatcher>();
regexp_matcher->setPattern(regex_node->as<ASTLiteral &>().value.get<String>()); regexp_matcher->setPattern(regexp_node->as<ASTLiteral &>().value.get<String>());
if (!transformers->children.empty()) if (!transformers->children.empty())
{ {
@ -1895,8 +1895,7 @@ bool ParserQualifiedColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected
else if (auto * column_regexp_matcher = node->as<ASTColumnsRegexpMatcher>()) else if (auto * column_regexp_matcher = node->as<ASTColumnsRegexpMatcher>())
{ {
auto result = std::make_shared<ASTQualifiedColumnsRegexpMatcher>(); auto result = std::make_shared<ASTQualifiedColumnsRegexpMatcher>();
result->setPattern(column_regexp_matcher->getPattern(), false); result->setPattern(column_regexp_matcher->getPattern());
result->setMatcher(column_regexp_matcher->getMatcher());
result->qualifier = std::move(identifier_node); result->qualifier = std::move(identifier_node);
result->children.push_back(result->qualifier); result->children.push_back(result->qualifier);

View File

@ -52,7 +52,7 @@ public:
: string_to_string_regexp(settings_.avro.string_column_pattern) : string_to_string_regexp(settings_.avro.string_column_pattern)
{ {
if (!string_to_string_regexp.ok()) if (!string_to_string_regexp.ok())
throw DB::Exception(DB::ErrorCodes::CANNOT_COMPILE_REGEXP, "Avro: cannot compile re2: {}, error: {}. " throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, "Avro: cannot compile re2: {}, error: {}. "
"Look at https://github.com/google/re2/wiki/Syntax for reference.", "Look at https://github.com/google/re2/wiki/Syntax for reference.",
settings_.avro.string_column_pattern, string_to_string_regexp.error()); settings_.avro.string_column_pattern, string_to_string_regexp.error());
} }

View File

@ -0,0 +1 @@
SYNTAX_ERROR

View File

@ -0,0 +1,9 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# This query is incorrect, but its parsing should not be slow (tens of seconds):
${CLICKHOUSE_LOCAL} < "$CURDIR"/03013_parser_regexp_recompilation.txt 2>&1 | grep --text -o -F 'SYNTAX_ERROR'