提交 689f599e 编写于 作者: A Alexey Milovidov

Fixed error, removed partially duplicate code [#METR-22462].

上级 416a7457
......@@ -10,10 +10,24 @@ namespace DB
class IAST;
class Context;
/** Выполнить константное выражение.
* Используется в редких случаях - для элемента множества в IN, для данных для INSERT.
* Весьма неоптимально.
/** Evaluate constant expression.
* Used in rare cases - for elements of set for IN, for data to INSERT.
* Quite suboptimal.
*/
Field evaluateConstantExpression(std::shared_ptr<IAST> & node, const Context & context);
/** Evaluate constant expression
* and returns ASTLiteral with its value.
*/
std::shared_ptr<IAST> evaluateConstantExpressionAsLiteral(std::shared_ptr<IAST> & node, const Context & context);
/** Evaluate constant expression
* and returns ASTLiteral with its value.
* Also, if AST is identifier, then return string literal with its name.
* Useful in places where some name may be specified as identifier, or as result of a constant expression.
*/
std::shared_ptr<IAST> evaluateConstantExpressionOrIdentidierAsLiteral(std::shared_ptr<IAST> & node, const Context & context);
}
#pragma once
#include <memory>
namespace DB
{
class IAST;
class ASTIdentifier;
class Context;
/** \brief if `expr` is not already ASTIdentifier evaluates it
* and replaces by a new ASTIdentifier with the result of evaluation as its name.
* `expr` must evaluate to a String type */
ASTIdentifier & reinterpretAsIdentifier(std::shared_ptr<IAST> & expr, const Context & context);
}
......@@ -17,7 +17,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
/** Выполнить константное выражение (для элемента множества в IN). Весьма неоптимально. */
Field evaluateConstantExpression(ASTPtr & node, const Context & context)
{
ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(
......@@ -44,4 +44,23 @@ Field evaluateConstantExpression(ASTPtr & node, const Context & context)
return result_column[0];
}
ASTPtr evaluateConstantExpressionAsLiteral(ASTPtr & node, const Context & context)
{
if (typeid_cast<const ASTLiteral *>(node.get()))
return node;
return std::make_shared<ASTLiteral>(node->range,
evaluateConstantExpression(node, context));
}
ASTPtr evaluateConstantExpressionOrIdentidierAsLiteral(ASTPtr & node, const Context & context)
{
if (const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(node.get()))
return std::make_shared<ASTLiteral>(node->range, Field(id->name));
return evaluateConstantExpressionAsLiteral(node, context);
}
}
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/ExpressionAnalyzer.h>
#include <DB/Interpreters/ExpressionActions.h>
#include <DB/Interpreters/reinterpretAsIdentifier.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>
namespace DB
{
namespace
{
ASTPtr reinterpretAsIdentifierImpl(const ASTPtr & expr, const Context & context)
{
/// for string literal return its value
if (const auto literal = typeid_cast<const ASTLiteral *>(expr.get()))
return std::make_shared<ASTIdentifier>(StringRange(), safeGet<const String &>(literal->value));
/// otherwise evaluate the expression
Block block{};
/** pass a dummy column name because ExpressionAnalyzer
* does not work with no columns so far. */
ExpressionAnalyzer{
expr, context, {},
{ { "", std::make_shared<DataTypeString>() } }
}.getActions(false)->execute(block);
const auto & column_name_type = block.getByName(expr->getColumnName());
/// ensure the result of evaluation has String type
if (!typeid_cast<const DataTypeString *>(column_name_type.type.get()))
throw Exception{"Expression must evaluate to a String"};
return std::make_shared<ASTIdentifier>(StringRange(), column_name_type.column->getDataAt(0).toString());
}
}
ASTIdentifier & reinterpretAsIdentifier(ASTPtr & expr, const Context & context)
{
/// for identifier just return its name
if (!typeid_cast<const ASTIdentifier *>(expr.get()))
expr = reinterpretAsIdentifierImpl(expr, context);
return static_cast<ASTIdentifier &>(*expr);
}
}
......@@ -9,7 +9,7 @@
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/reinterpretAsIdentifier.h>
#include <DB/Interpreters/evaluateConstantExpression.h>
#include <DB/Interpreters/getClusterName.h>
#include <DB/Storages/StorageLog.h>
......@@ -350,8 +350,11 @@ StoragePtr StorageFactory::get(
" - name of source database and regexp for table names.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
String source_database = reinterpretAsIdentifier(args[0], local_context).name;
String table_name_regexp = safeGet<const String &>(typeid_cast<ASTLiteral &>(*args[1]).value);
args[0] = evaluateConstantExpressionOrIdentidierAsLiteral(args[0], local_context);
args[1] = evaluateConstantExpressionAsLiteral(args[1], local_context);
String source_database = static_cast<const ASTLiteral &>(*args[0]).value.safeGet<String>();
String table_name_regexp = static_cast<const ASTLiteral &>(*args[1]).value.safeGet<String>();
return StorageMerge::create(
table_name, columns,
......@@ -387,8 +390,11 @@ StoragePtr StorageFactory::get(
String cluster_name = getClusterName(*args[0]);
String remote_database = reinterpretAsIdentifier(args[1], local_context).name;
String remote_table = reinterpretAsIdentifier(args[2], local_context).name;
args[1] = evaluateConstantExpressionOrIdentidierAsLiteral(args[1], local_context);
args[2] = evaluateConstantExpressionOrIdentidierAsLiteral(args[2], local_context);
String remote_database = static_cast<const ASTLiteral &>(*args[1]).value.safeGet<String>();
String remote_table = static_cast<const ASTLiteral &>(*args[2]).value.safeGet<String>();
const auto & sharding_key = args.size() == 4 ? args[3] : nullptr;
......@@ -421,8 +427,11 @@ StoragePtr StorageFactory::get(
" destination_database, destination_table, num_buckets, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
String destination_database = reinterpretAsIdentifier(args[0], local_context).name;
String destination_table = typeid_cast<ASTIdentifier &>(*args[1]).name;
args[1] = evaluateConstantExpressionOrIdentidierAsLiteral(args[1], local_context);
args[2] = evaluateConstantExpressionOrIdentidierAsLiteral(args[2], local_context);
String destination_database = static_cast<const ASTLiteral &>(*args[1]).value.safeGet<String>();
String destination_table = static_cast<const ASTLiteral &>(*args[2]).value.safeGet<String>();
size_t num_buckets = apply_visitor(FieldVisitorConvertToNumber<size_t>(), typeid_cast<ASTLiteral &>(*args[2]).value);
......
......@@ -5,7 +5,7 @@
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/TableFunctions/ITableFunction.h>
#include <DB/Interpreters/reinterpretAsIdentifier.h>
#include <DB/Interpreters/evaluateConstantExpression.h>
#include <DB/Databases/IDatabase.h>
#include <DB/TableFunctions/TableFunctionMerge.h>
......@@ -66,8 +66,11 @@ StoragePtr TableFunctionMerge::execute(ASTPtr ast_function, Context & context) c
" - name of source database and regexp for table names.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
String source_database = reinterpretAsIdentifier(args[0], context).name;
String table_name_regexp = safeGet<const String &>(typeid_cast<ASTLiteral &>(*args[1]).value);
args[0] = evaluateConstantExpressionOrIdentidierAsLiteral(args[1], context);
args[1] = evaluateConstantExpressionAsLiteral(args[2], context);
String source_database = static_cast<const ASTLiteral &>(*args[0]).value.safeGet<String>();
String table_name_regexp = static_cast<const ASTLiteral &>(*args[1]).value.safeGet<String>();
/// В InterpreterSelectQuery будет создан ExpressionAnalzyer, который при обработке запроса наткнется на этот Identifier.
/// Нам необходимо его пометить как имя базы данных, поскольку по умолчанию стоит значение column
......
......@@ -2,7 +2,7 @@
#include <DB/Storages/StorageDistributed.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Interpreters/reinterpretAsIdentifier.h>
#include <DB/Interpreters/evaluateConstantExpression.h>
#include <DB/Interpreters/Cluster.h>
#include <DB/TableFunctions/TableFunctionRemote.h>
......@@ -210,7 +210,8 @@ StoragePtr TableFunctionRemote::execute(ASTPtr ast_function, Context & context)
description = getStringLiteral(*args[arg_num], "Hosts pattern");
++arg_num;
remote_database = reinterpretAsIdentifier(args[arg_num], context).name;
args[arg_num] = evaluateConstantExpressionOrIdentidierAsLiteral(args[arg_num], context);
remote_database = static_cast<const ASTLiteral &>(*args[arg_num]).value.safeGet<String>();
++arg_num;
size_t dot = remote_database.find('.');
......@@ -225,7 +226,8 @@ StoragePtr TableFunctionRemote::execute(ASTPtr ast_function, Context & context)
if (arg_num >= args.size())
throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
remote_table = reinterpretAsIdentifier(args[arg_num], context).name;
args[arg_num] = evaluateConstantExpressionOrIdentidierAsLiteral(args[arg_num], context);
remote_table = static_cast<const ASTLiteral &>(*args[arg_num]).value.safeGet<String>();
++arg_num;
}
......
......@@ -2,7 +2,7 @@
#include <DB/Storages/StorageDistributed.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Interpreters/reinterpretAsIdentifier.h>
#include <DB/Interpreters/evaluateConstantExpression.h>
#include <DB/Interpreters/Cluster.h>
#include <DB/Interpreters/getClusterName.h>
#include <DB/Common/SipHash.h>
......@@ -52,8 +52,12 @@ StoragePtr TableFunctionShardByHash::execute(ASTPtr ast_function, Context & cont
cluster_name = getClusterName(*args[0]);
key = getStringLiteral(*args[1], "Key to hash");
remote_database = reinterpretAsIdentifier(args[2], context).name;
remote_table = reinterpretAsIdentifier(args[3], context).name;
args[2] = evaluateConstantExpressionOrIdentidierAsLiteral(args[2], context);
args[3] = evaluateConstantExpressionOrIdentidierAsLiteral(args[3], context);
remote_database = static_cast<const ASTLiteral &>(*args[2]).value.safeGet<String>();
remote_table = static_cast<const ASTLiteral &>(*args[3]).value.safeGet<String>();
/// Аналогично другим TableFunctions.
for (auto & arg : args)
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册