mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge COLUMNS matcher (incomplete)
This commit is contained in:
parent
3793c3cd13
commit
ef6c7ea5be
@ -4,7 +4,7 @@
|
||||
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTQualifiedAsterisk.h>
|
||||
#include "../Parsers/ASTColumnsClause.h"
|
||||
#include <Parsers/ASTColumnsClause.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTQualifiedAsterisk.h>
|
||||
#include <Parsers/ASTColumnsClause.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
@ -24,7 +25,7 @@
|
||||
#include <Interpreters/FindIdentifierBestTableVisitor.h>
|
||||
#include <Interpreters/ExtractFunctionDataVisitor.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include "../Parsers/ASTColumnsClause.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -267,7 +268,7 @@ std::vector<ASTPtr> PredicateExpressionsOptimizer::splitConjunctionPredicate(con
|
||||
continue;
|
||||
}
|
||||
}
|
||||
idx++;
|
||||
++idx;
|
||||
}
|
||||
}
|
||||
return predicate_expressions;
|
||||
@ -486,7 +487,7 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que
|
||||
const auto block = storage->getSampleBlock();
|
||||
if (const auto * asterisk_pattern = asterisk->as<ASTColumnsClause>())
|
||||
{
|
||||
for (size_t idx = 0; idx < block.columns(); idx++)
|
||||
for (size_t idx = 0; idx < block.columns(); ++idx)
|
||||
{
|
||||
auto & col = block.getByPosition(idx);
|
||||
if (asterisk_pattern->isColumnMatching(col.name))
|
||||
@ -495,7 +496,7 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t idx = 0; idx < block.columns(); idx++)
|
||||
for (size_t idx = 0; idx < block.columns(); ++idx)
|
||||
projection_columns.emplace_back(std::make_shared<ASTIdentifier>(block.getByPosition(idx).name));
|
||||
}
|
||||
}
|
||||
|
@ -16,8 +16,7 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <iostream>
|
||||
#include "../Parsers/ASTColumnsClause.h"
|
||||
#include <Parsers/ASTColumnsClause.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,12 @@
|
||||
#include "ASTColumnsClause.h"
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include "ASTColumnsClause.h"
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
#include <re2/re2.h>
|
||||
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -12,11 +18,28 @@ ASTPtr ASTColumnsClause::clone() const
|
||||
return clone;
|
||||
}
|
||||
|
||||
void ASTColumnsClause::appendColumnName(WriteBuffer & ostr) const { writeString(originalPattern, ostr); }
|
||||
void ASTColumnsClause::appendColumnName(WriteBuffer & ostr) const { writeString(original_pattern, ostr); }
|
||||
|
||||
void ASTColumnsClause::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << " '" << originalPattern << "'";
|
||||
WriteBufferFromOwnString pattern_quoted;
|
||||
writeQuotedString(original_pattern, pattern_quoted);
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(" << pattern_quoted.str() << ")";
|
||||
}
|
||||
|
||||
void ASTColumnsClause::setPattern(String pattern)
|
||||
{
|
||||
original_pattern = std::move(pattern);
|
||||
column_matcher = std::make_shared<RE2>(pattern, RE2::Quiet);
|
||||
if (!column_matcher->ok())
|
||||
throw DB::Exception("COLUMNS pattern " + original_pattern + " cannot be compiled: " + column_matcher->error(), DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
|
||||
}
|
||||
|
||||
bool ASTColumnsClause::isColumnMatching(const String & column_name) const
|
||||
{
|
||||
return RE2::FullMatch(column_name, *column_matcher);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -1,13 +1,19 @@
|
||||
#pragma once
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <re2/re2.h>
|
||||
|
||||
|
||||
namespace re2
|
||||
{
|
||||
class RE2;
|
||||
}
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class WriteBuffer;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_COMPILE_REGEXP;
|
||||
@ -17,31 +23,24 @@ struct AsteriskSemantic;
|
||||
struct AsteriskSemanticImpl;
|
||||
|
||||
|
||||
/** SELECT COLUMNS('regexp') is expanded to multiple columns like * (asterisk).
|
||||
*/
|
||||
class ASTColumnsClause : public IAST
|
||||
{
|
||||
public:
|
||||
|
||||
String getID(char) const override { return "ColumnsClause"; }
|
||||
ASTPtr clone() const override;
|
||||
|
||||
void appendColumnName(WriteBuffer & ostr) const override;
|
||||
void setPattern(String pattern)
|
||||
{
|
||||
originalPattern = pattern;
|
||||
columnMatcher = std::make_shared<RE2>(pattern, RE2::Quiet);
|
||||
if (!columnMatcher->ok())
|
||||
throw DB::Exception("COLUMNS pattern " + originalPattern + " cannot be compiled: " + columnMatcher->error(), DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
|
||||
}
|
||||
bool isColumnMatching(String columnName) const
|
||||
{
|
||||
return RE2::FullMatch(columnName, *columnMatcher);
|
||||
}
|
||||
void setPattern(String pattern);
|
||||
bool isColumnMatching(const String & column_name) const;
|
||||
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<RE2> columnMatcher;
|
||||
String originalPattern;
|
||||
std::shared_ptr<re2::RE2> column_matcher;
|
||||
String original_pattern;
|
||||
std::shared_ptr<AsteriskSemanticImpl> semantic; /// pimpl
|
||||
|
||||
friend struct AsteriskSemantic;
|
||||
|
@ -1177,10 +1177,18 @@ bool ParserColumnsClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
|
||||
if (!columns.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (pos->type != TokenType::OpeningRoundBracket)
|
||||
return false;
|
||||
++pos;
|
||||
|
||||
ASTPtr regex_node;
|
||||
if (!regex.parse(pos, regex_node, expected))
|
||||
return false;
|
||||
|
||||
if (pos->type != TokenType::ClosingRoundBracket)
|
||||
return false;
|
||||
++pos;
|
||||
|
||||
auto res = std::make_shared<ASTColumnsClause>();
|
||||
res->setPattern(regex_node->as<ASTLiteral &>().value.get<String>());
|
||||
res->children.push_back(regex_node);
|
||||
@ -1188,6 +1196,7 @@ bool ParserColumnsClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected &)
|
||||
{
|
||||
if (pos->type == TokenType::Asterisk)
|
||||
@ -1282,10 +1291,10 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
||||
|| ParserLeftExpression().parse(pos, node, expected)
|
||||
|| ParserRightExpression().parse(pos, node, expected)
|
||||
|| ParserCase().parse(pos, node, expected)
|
||||
|| ParserColumnsClause().parse(pos, node, expected) /// before ParserFunction because it can be also parsed as a function.
|
||||
|| ParserFunction().parse(pos, node, expected)
|
||||
|| ParserQualifiedAsterisk().parse(pos, node, expected)
|
||||
|| ParserAsterisk().parse(pos, node, expected)
|
||||
|| ParserColumnsClause().parse(pos, node, expected)
|
||||
|| ParserCompoundIdentifier().parse(pos, node, expected)
|
||||
|| ParserSubstitution().parse(pos, node, expected);
|
||||
}
|
||||
|
@ -73,12 +73,12 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
/** COLUMNS '<regular expression>'
|
||||
/** COLUMNS('<regular expression>')
|
||||
*/
|
||||
class ParserColumnsClause : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const { return "COLUMNS clause"; }
|
||||
const char * getName() const { return "COLUMNS matcher"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
|
@ -1,3 +1,3 @@
|
||||
CREATE TABLE IF NOT EXISTS ColumnsClauseTest (product_price Int64, product_weight Int16, amount Int64) Engine=TinyLog;
|
||||
INSERT INTO ColumnsClauseTest VALUES (100, 10, 324), (120, 8, 23);
|
||||
SELECT COLUMNS 'product.*' from ColumnsClauseTest ORDER BY product_price;
|
||||
SELECT COLUMNS('product.*') from ColumnsClauseTest ORDER BY product_price;
|
||||
|
Loading…
Reference in New Issue
Block a user