mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
Use AST to generate template
This commit is contained in:
parent
2322a88ef0
commit
c7e311bac8
@ -15,6 +15,8 @@
|
|||||||
#include <DataTypes/DataTypeNullable.h>
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
#include <DataTypes/DataTypeString.h>
|
#include <DataTypes/DataTypeString.h>
|
||||||
#include <DataTypes/FieldToDataType.h>
|
#include <DataTypes/FieldToDataType.h>
|
||||||
|
#include <Interpreters/InDepthNodeVisitor.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -26,31 +28,97 @@ namespace ErrorCodes
|
|||||||
extern const int CANNOT_EVALUATE_EXPRESSION_TEMPLATE;
|
extern const int CANNOT_EVALUATE_EXPRESSION_TEMPLATE;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
class ReplaceLiteralsMatcher
|
||||||
ConstantExpressionTemplate::ConstantExpressionTemplate(const IDataType & result_column_type, TokenIterator begin, TokenIterator end,
|
|
||||||
const Context & context)
|
|
||||||
{
|
{
|
||||||
std::pair<String, NamesAndTypesList> expr_template = replaceLiteralsWithDummyIdentifiers(begin, end, result_column_type);
|
public:
|
||||||
for (const auto & col : expr_template.second)
|
using Visitor = InDepthNodeVisitor<ReplaceLiteralsMatcher, true>;
|
||||||
literals.insert({nullptr, col.type, col.name});
|
|
||||||
|
struct LiteralInfo
|
||||||
|
{
|
||||||
|
typedef std::shared_ptr<ASTLiteral> ASTLiteralPtr;
|
||||||
|
LiteralInfo(const ASTLiteralPtr & literal_, const String & column_name_) : literal(literal_), dummy_column_name(column_name_) { }
|
||||||
|
ASTLiteralPtr literal;
|
||||||
|
String dummy_column_name;
|
||||||
|
};
|
||||||
|
|
||||||
|
using Data = std::vector<LiteralInfo>;
|
||||||
|
|
||||||
|
static void visit(ASTPtr & ast, Data & data)
|
||||||
|
{
|
||||||
|
for (auto & child : ast->children)
|
||||||
|
{
|
||||||
|
auto literal = std::dynamic_pointer_cast<ASTLiteral>(child);
|
||||||
|
if (!literal || !literal->begin || !literal->end)
|
||||||
|
continue;
|
||||||
|
|
||||||
|
// TODO don't replace constant arguments of functions such as CAST(x, 'type')
|
||||||
|
// TODO ensure column_name is unique (there was no _dummy_x identifier in expression)
|
||||||
|
String column_name = "_dummy_" + std::to_string(data.size());
|
||||||
|
data.emplace_back(literal, column_name);
|
||||||
|
child = std::make_shared<ASTIdentifier>(column_name);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
static bool needChildVisit(ASTPtr &, const ASTPtr & child) { return !child->as<ASTLiteral>(); }
|
||||||
|
};
|
||||||
|
|
||||||
|
using ReplaceLiteralsVisitor = ReplaceLiteralsMatcher::Visitor;
|
||||||
|
using LiteralInfo = ReplaceLiteralsMatcher::LiteralInfo;
|
||||||
|
|
||||||
|
|
||||||
|
ConstantExpressionTemplate::ConstantExpressionTemplate(const IDataType & result_column_type,
|
||||||
|
TokenIterator expression_begin, TokenIterator expression_end,
|
||||||
|
const ASTPtr & expression_, const Context & context)
|
||||||
|
{
|
||||||
|
ASTPtr expression = expression_->clone();
|
||||||
|
addNodesToCastResult(result_column_type, expression);
|
||||||
|
ReplaceLiteralsVisitor::Data replaced_literals;
|
||||||
|
ReplaceLiteralsVisitor(replaced_literals).visit(expression);
|
||||||
|
|
||||||
|
std::sort(replaced_literals.begin(), replaced_literals.end(), [](const LiteralInfo & a, const LiteralInfo & b)
|
||||||
|
{
|
||||||
|
return a.literal->begin.value() < b.literal->begin.value();
|
||||||
|
});
|
||||||
|
|
||||||
|
bool allow_nulls = result_column_type.isNullable();
|
||||||
|
TokenIterator prev_end = expression_begin;
|
||||||
|
for (const auto & info : replaced_literals)
|
||||||
|
{
|
||||||
|
if (info.literal->begin.value() < prev_end)
|
||||||
|
throw Exception("Cannot replace literals", ErrorCodes::CANNOT_CREATE_EXPRESSION_TEMPLATE);
|
||||||
|
|
||||||
|
while (prev_end < info.literal->begin.value())
|
||||||
|
{
|
||||||
|
tokens.emplace_back(prev_end->begin, prev_end->size());
|
||||||
|
++prev_end;
|
||||||
|
}
|
||||||
|
token_after_literal_idx.push_back(tokens.size());
|
||||||
|
|
||||||
|
DataTypePtr type = applyVisitor(FieldToDataType(), info.literal->value);
|
||||||
|
/// Allow literal to be NULL, if result column has nullable type
|
||||||
|
// TODO also allow NULL literals inside functions, which return not NULL for NULL arguments,
|
||||||
|
// even if result_column_type is not nullable
|
||||||
|
if (allow_nulls)
|
||||||
|
type = makeNullable(type);
|
||||||
|
|
||||||
|
literals.insert({nullptr, type, info.dummy_column_name});
|
||||||
|
|
||||||
|
prev_end = info.literal->end.value();
|
||||||
|
}
|
||||||
|
|
||||||
|
while (prev_end < expression_end)
|
||||||
|
{
|
||||||
|
tokens.emplace_back(prev_end->begin, prev_end->size());
|
||||||
|
++prev_end;
|
||||||
|
}
|
||||||
|
|
||||||
columns = literals.cloneEmptyColumns();
|
columns = literals.cloneEmptyColumns();
|
||||||
|
|
||||||
ParserExpression parser;
|
result_column_name = expression->getColumnName();
|
||||||
Expected expected;
|
|
||||||
Tokens template_tokens(expr_template.first.data(), expr_template.first.data() + expr_template.first.size());
|
|
||||||
TokenIterator token_iterator1(template_tokens);
|
|
||||||
|
|
||||||
ASTPtr ast_template;
|
|
||||||
if (!parser.parse(token_iterator1, ast_template, expected))
|
|
||||||
throw Exception("Cannot parse template after replacing literals: ", ErrorCodes::CANNOT_CREATE_EXPRESSION_TEMPLATE);
|
|
||||||
|
|
||||||
addNodesToCastResult(result_column_type, ast_template);
|
|
||||||
result_column_name = ast_template->getColumnName();
|
|
||||||
|
|
||||||
// TODO convert SyntaxAnalyzer and ExpressionAnalyzer exceptions to CANNOT_CREATE_EXPRESSION_TEMPLATE
|
// TODO convert SyntaxAnalyzer and ExpressionAnalyzer exceptions to CANNOT_CREATE_EXPRESSION_TEMPLATE
|
||||||
auto syntax_result = SyntaxAnalyzer(context).analyze(ast_template, expr_template.second);
|
auto syntax_result = SyntaxAnalyzer(context).analyze(expression, literals.getNamesAndTypesList());
|
||||||
|
|
||||||
actions_on_literals = ExpressionAnalyzer(ast_template, syntax_result, context).getActions(false);
|
actions_on_literals = ExpressionAnalyzer(expression, syntax_result, context).getActions(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
void ConstantExpressionTemplate::parseExpression(ReadBuffer & istr, const FormatSettings & settings)
|
void ConstantExpressionTemplate::parseExpression(ReadBuffer & istr, const FormatSettings & settings)
|
||||||
@ -69,6 +137,7 @@ void ConstantExpressionTemplate::parseExpression(ReadBuffer & istr, const Format
|
|||||||
assertString(tokens[cur_token++], istr);
|
assertString(tokens[cur_token++], istr);
|
||||||
}
|
}
|
||||||
skipWhitespaceIfAny(istr);
|
skipWhitespaceIfAny(istr);
|
||||||
|
// TODO parse numbers more carefully: check actual type of number
|
||||||
const IDataType & type = *literals.getByPosition(cur_column).type;
|
const IDataType & type = *literals.getByPosition(cur_column).type;
|
||||||
type.deserializeAsTextQuoted(*columns[cur_column], istr, settings);
|
type.deserializeAsTextQuoted(*columns[cur_column], istr, settings);
|
||||||
++cur_column;
|
++cur_column;
|
||||||
@ -112,74 +181,6 @@ ColumnPtr ConstantExpressionTemplate::evaluateAll()
|
|||||||
return evaluated.getByName(result_column_name).column->convertToFullColumnIfConst();
|
return evaluated.getByName(result_column_name).column->convertToFullColumnIfConst();
|
||||||
}
|
}
|
||||||
|
|
||||||
std::pair<String, NamesAndTypesList>
|
|
||||||
ConstantExpressionTemplate::replaceLiteralsWithDummyIdentifiers(TokenIterator & begin, TokenIterator & end, const IDataType & result_column_type)
|
|
||||||
{
|
|
||||||
NamesAndTypesList dummy_columns;
|
|
||||||
String result;
|
|
||||||
size_t token_idx = 0;
|
|
||||||
while (begin != end)
|
|
||||||
{
|
|
||||||
const Token & t = *begin;
|
|
||||||
if (t.isError())
|
|
||||||
throw DB::Exception("Error in tokens", ErrorCodes::CANNOT_CREATE_EXPRESSION_TEMPLATE);
|
|
||||||
|
|
||||||
Expected expected;
|
|
||||||
ASTPtr ast;
|
|
||||||
DataTypePtr type;
|
|
||||||
|
|
||||||
// TODO don't convert constant arguments of functions such as CAST(x, 'type')
|
|
||||||
|
|
||||||
if (t.type == TokenType::Number)
|
|
||||||
{
|
|
||||||
ParserNumber parser;
|
|
||||||
if (!parser.parse(begin, ast, expected))
|
|
||||||
throw DB::Exception("Cannot determine literal type: " + String(t.begin, t.size()), ErrorCodes::CANNOT_CREATE_EXPRESSION_TEMPLATE);
|
|
||||||
Field & value = ast->as<ASTLiteral &>().value;
|
|
||||||
// TODO parse numbers more carefully: distinguish unary and binary sign
|
|
||||||
type = DataTypeFactory::instance().get(value.getTypeName());
|
|
||||||
}
|
|
||||||
else if (t.type == TokenType::StringLiteral)
|
|
||||||
{
|
|
||||||
type = std::make_shared<DataTypeString>();
|
|
||||||
++begin;
|
|
||||||
}
|
|
||||||
else if (t.type == TokenType::OpeningSquareBracket)
|
|
||||||
{
|
|
||||||
ParserArrayOfLiterals parser;
|
|
||||||
if (parser.parse(begin, ast, expected))
|
|
||||||
{
|
|
||||||
Field & value = ast->as<ASTLiteral &>().value;
|
|
||||||
type = applyVisitor(FieldToDataType(), value);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (type)
|
|
||||||
{
|
|
||||||
/// Allow literal to be NULL, if result column has nullable type
|
|
||||||
// TODO also allow NULL literals inside functions, which return not NULL for NULL arguments,
|
|
||||||
// even if result_column_type is not nullable
|
|
||||||
if (result_column_type.isNullable())
|
|
||||||
type = makeNullable(type);
|
|
||||||
|
|
||||||
// TODO ensure dummy_col_name is unique (there was no _dummy_x identifier in expression)
|
|
||||||
String dummy_col_name = "_dummy_" + std::to_string(dummy_columns.size());
|
|
||||||
|
|
||||||
dummy_columns.push_back(NameAndTypePair(dummy_col_name, type));
|
|
||||||
token_after_literal_idx.push_back(token_idx);
|
|
||||||
result.append(dummy_col_name);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
tokens.emplace_back(t.begin, t.size());
|
|
||||||
result.append(tokens.back());
|
|
||||||
++begin;
|
|
||||||
++token_idx;
|
|
||||||
}
|
|
||||||
result.append(" ");
|
|
||||||
}
|
|
||||||
return std::make_pair(result, dummy_columns);
|
|
||||||
}
|
|
||||||
|
|
||||||
void ConstantExpressionTemplate::addNodesToCastResult(const IDataType & result_column_type, ASTPtr & expr)
|
void ConstantExpressionTemplate::addNodesToCastResult(const IDataType & result_column_type, ASTPtr & expr)
|
||||||
{
|
{
|
||||||
|
@ -11,15 +11,14 @@ namespace DB
|
|||||||
class ConstantExpressionTemplate
|
class ConstantExpressionTemplate
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
ConstantExpressionTemplate(const IDataType & result_column_type, TokenIterator begin, TokenIterator end, const Context & context);
|
ConstantExpressionTemplate(const IDataType & result_column_type, TokenIterator expression_begin, TokenIterator expression_end,
|
||||||
|
const ASTPtr & expression, const Context & context);
|
||||||
|
|
||||||
void parseExpression(ReadBuffer & istr, const FormatSettings & settings);
|
void parseExpression(ReadBuffer & istr, const FormatSettings & settings);
|
||||||
|
|
||||||
ColumnPtr evaluateAll();
|
ColumnPtr evaluateAll();
|
||||||
|
|
||||||
private:
|
private:
|
||||||
std::pair<String, NamesAndTypesList> replaceLiteralsWithDummyIdentifiers(TokenIterator & begin, TokenIterator & end, const IDataType & result_column_type);
|
|
||||||
|
|
||||||
static void addNodesToCastResult(const IDataType & result_column_type, ASTPtr & expr);
|
static void addNodesToCastResult(const IDataType & result_column_type, ASTPtr & expr);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
@ -204,7 +204,7 @@ ValuesBlockInputStream::parseExpression(IColumn & column, size_t column_idx, boo
|
|||||||
ErrorCodes::LOGICAL_ERROR);
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
templates[column_idx] = ConstantExpressionTemplate(type, TokenIterator(tokens), token_iterator, *context);
|
templates[column_idx] = ConstantExpressionTemplate(type, TokenIterator(tokens), token_iterator, ast, *context);
|
||||||
istr.rollbackToCheckpoint();
|
istr.rollbackToCheckpoint();
|
||||||
templates[column_idx].value().parseExpression(istr, format_settings);
|
templates[column_idx].value().parseExpression(istr, format_settings);
|
||||||
assertDelimAfterValue(column_idx);
|
assertDelimAfterValue(column_idx);
|
||||||
|
Loading…
Reference in New Issue
Block a user