Use AST to generate template

This commit is contained in:
Alexander Tokmakov 2019-05-22 06:29:32 +03:00 committed by Alexander Tokmakov
parent 2322a88ef0
commit c7e311bac8
3 changed files with 91 additions and 91 deletions

View File

@ -15,6 +15,8 @@
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/FieldToDataType.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Parsers/ASTIdentifier.h>
namespace DB
{
@ -26,31 +28,97 @@ namespace ErrorCodes
extern const int CANNOT_EVALUATE_EXPRESSION_TEMPLATE;
}
ConstantExpressionTemplate::ConstantExpressionTemplate(const IDataType & result_column_type, TokenIterator begin, TokenIterator end,
const Context & context)
class ReplaceLiteralsMatcher
{
std::pair<String, NamesAndTypesList> expr_template = replaceLiteralsWithDummyIdentifiers(begin, end, result_column_type);
for (const auto & col : expr_template.second)
literals.insert({nullptr, col.type, col.name});
public:
using Visitor = InDepthNodeVisitor<ReplaceLiteralsMatcher, true>;
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();
ParserExpression parser;
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();
result_column_name = expression->getColumnName();
// 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)
@ -69,6 +137,7 @@ void ConstantExpressionTemplate::parseExpression(ReadBuffer & istr, const Format
assertString(tokens[cur_token++], istr);
}
skipWhitespaceIfAny(istr);
// TODO parse numbers more carefully: check actual type of number
const IDataType & type = *literals.getByPosition(cur_column).type;
type.deserializeAsTextQuoted(*columns[cur_column], istr, settings);
++cur_column;
@ -112,74 +181,6 @@ ColumnPtr ConstantExpressionTemplate::evaluateAll()
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)
{

View File

@ -11,15 +11,14 @@ namespace DB
class ConstantExpressionTemplate
{
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);
ColumnPtr evaluateAll();
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);
private:

View File

@ -204,7 +204,7 @@ ValuesBlockInputStream::parseExpression(IColumn & column, size_t column_idx, boo
ErrorCodes::LOGICAL_ERROR);
try
{
templates[column_idx] = ConstantExpressionTemplate(type, TokenIterator(tokens), token_iterator, *context);
templates[column_idx] = ConstantExpressionTemplate(type, TokenIterator(tokens), token_iterator, ast, *context);
istr.rollbackToCheckpoint();
templates[column_idx].value().parseExpression(istr, format_settings);
assertDelimAfterValue(column_idx);