mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 09:02:00 +00:00
Merge pull request #1958 from yandex/remove-ast-function-kind
Removed bad code that may lead to (harmless) race condition
This commit is contained in:
commit
1497b61b79
@ -368,7 +368,7 @@ void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const String
|
||||
{
|
||||
/// Do not go to FROM, JOIN, UNION.
|
||||
if (!typeid_cast<const ASTTableExpression *>(child.get())
|
||||
&& child.get() != select_query->next_union_all.get())
|
||||
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
|
||||
{
|
||||
translateQualifiedNamesImpl(child, database_name, table_name, alias);
|
||||
}
|
||||
@ -1108,35 +1108,6 @@ void ExpressionAnalyzer::normalizeTreeImpl(
|
||||
normalizeTreeImpl(select->having_expression, finished_asts, current_asts, current_alias, level + 1);
|
||||
}
|
||||
|
||||
/// Actions to be performed from the bottom up.
|
||||
|
||||
if (ASTFunction * node = typeid_cast<ASTFunction *>(ast.get()))
|
||||
{
|
||||
if (node->kind == ASTFunction::TABLE_FUNCTION)
|
||||
{
|
||||
}
|
||||
else if (node->name == "lambda")
|
||||
{
|
||||
node->kind = ASTFunction::LAMBDA_EXPRESSION;
|
||||
}
|
||||
else if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||
{
|
||||
node->kind = ASTFunction::AGGREGATE_FUNCTION;
|
||||
}
|
||||
else if (node->name == "arrayJoin")
|
||||
{
|
||||
node->kind = ASTFunction::ARRAY_JOIN;
|
||||
}
|
||||
else
|
||||
{
|
||||
node->kind = ASTFunction::FUNCTION;
|
||||
}
|
||||
|
||||
if (node->parameters && node->kind != ASTFunction::AGGREGATE_FUNCTION)
|
||||
throw Exception("The only parametric functions (functions with two separate parenthesis pairs) are aggregate functions"
|
||||
", and '" + node->name + "' is not an aggregate function.", ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS);
|
||||
}
|
||||
|
||||
current_asts.erase(initial_ast.get());
|
||||
current_asts.erase(ast.get());
|
||||
finished_asts[initial_ast] = ast;
|
||||
@ -1165,7 +1136,7 @@ void ExpressionAnalyzer::executeScalarSubqueries()
|
||||
{
|
||||
/// Do not go to FROM, JOIN, UNION.
|
||||
if (!typeid_cast<const ASTTableExpression *>(child.get())
|
||||
&& child.get() != select_query->next_union_all.get())
|
||||
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
|
||||
{
|
||||
executeScalarSubqueriesImpl(child);
|
||||
}
|
||||
@ -1181,7 +1152,6 @@ static ASTPtr addTypeConversion(std::unique_ptr<ASTLiteral> && ast, const String
|
||||
func->alias = ast->alias;
|
||||
func->prefer_alias_to_column_name = ast->prefer_alias_to_column_name;
|
||||
ast->alias.clear();
|
||||
func->kind = ASTFunction::FUNCTION;
|
||||
func->name = "CAST";
|
||||
auto exp_list = std::make_shared<ASTExpressionList>(ast->range);
|
||||
func->arguments = exp_list;
|
||||
@ -1260,7 +1230,6 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast)
|
||||
auto tuple = std::make_shared<ASTFunction>(ast->range);
|
||||
tuple->alias = subquery->alias;
|
||||
ast = tuple;
|
||||
tuple->kind = ASTFunction::FUNCTION;
|
||||
tuple->name = "tuple";
|
||||
auto exp_list = std::make_shared<ASTExpressionList>(ast->range);
|
||||
tuple->arguments = exp_list;
|
||||
@ -1286,8 +1255,7 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast)
|
||||
*/
|
||||
ASTFunction * func = typeid_cast<ASTFunction *>(ast.get());
|
||||
|
||||
if (func && func->kind == ASTFunction::FUNCTION
|
||||
&& functionIsInOrGlobalInOperator(func->name))
|
||||
if (func && functionIsInOrGlobalInOperator(func->name))
|
||||
{
|
||||
for (auto & child : ast->children)
|
||||
{
|
||||
@ -1495,7 +1463,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block &
|
||||
}
|
||||
|
||||
const ASTFunction * func = typeid_cast<const ASTFunction *>(node.get());
|
||||
if (func && func->kind == ASTFunction::FUNCTION && functionIsInOperator(func->name))
|
||||
if (func && functionIsInOperator(func->name))
|
||||
{
|
||||
const IAST & args = *func->arguments;
|
||||
const ASTPtr & arg = args.children.at(1);
|
||||
@ -1976,11 +1944,11 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
|
||||
}
|
||||
else if (ASTFunction * node = typeid_cast<ASTFunction *>(ast.get()))
|
||||
{
|
||||
if (node->kind == ASTFunction::LAMBDA_EXPRESSION)
|
||||
if (node->name == "lambda")
|
||||
throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION);
|
||||
|
||||
/// Function arrayJoin.
|
||||
if (node->kind == ASTFunction::ARRAY_JOIN)
|
||||
if (node->name == "arrayJoin")
|
||||
{
|
||||
if (node->arguments->children.size() != 1)
|
||||
throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH);
|
||||
@ -1999,8 +1967,6 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
|
||||
return;
|
||||
}
|
||||
|
||||
if (node->kind == ASTFunction::FUNCTION)
|
||||
{
|
||||
if (functionIsInOrGlobalInOperator(node->name))
|
||||
{
|
||||
if (!no_subqueries)
|
||||
@ -2036,6 +2002,9 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
|
||||
return;
|
||||
}
|
||||
|
||||
if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||
return;
|
||||
|
||||
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(node->name, context);
|
||||
|
||||
Names argument_names;
|
||||
@ -2186,7 +2155,6 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
|
||||
if (arguments_present)
|
||||
actions_stack.addAction(ExpressionAction::applyFunction(function_builder, argument_names, node->getColumnName()));
|
||||
}
|
||||
}
|
||||
else if (ASTLiteral * node = typeid_cast<ASTLiteral *>(ast.get()))
|
||||
{
|
||||
DataTypePtr type = applyVisitor(FieldToDataType(), node->value);
|
||||
@ -2201,8 +2169,13 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
|
||||
else
|
||||
{
|
||||
for (auto & child : ast->children)
|
||||
{
|
||||
/// Do not go to FROM, JOIN, UNION.
|
||||
if (!typeid_cast<const ASTTableExpression *>(child.get())
|
||||
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
|
||||
getActionsImpl(child, no_subqueries, only_consts, actions_stack);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -2223,7 +2196,7 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr
|
||||
}
|
||||
|
||||
const ASTFunction * node = typeid_cast<const ASTFunction *>(ast.get());
|
||||
if (node && node->kind == ASTFunction::AGGREGATE_FUNCTION)
|
||||
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||
{
|
||||
has_aggregation = true;
|
||||
AggregateDescription aggregate;
|
||||
@ -2268,7 +2241,7 @@ void ExpressionAnalyzer::assertNoAggregates(const ASTPtr & ast, const char * des
|
||||
{
|
||||
const ASTFunction * node = typeid_cast<const ASTFunction *>(ast.get());
|
||||
|
||||
if (node && node->kind == ASTFunction::AGGREGATE_FUNCTION)
|
||||
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||
throw Exception("Aggregate function " + node->getColumnName()
|
||||
+ " is found " + String(description) + " in query", ErrorCodes::ILLEGAL_AGGREGATION);
|
||||
|
||||
@ -2570,7 +2543,7 @@ void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, Express
|
||||
{
|
||||
ASTFunction * node = typeid_cast<ASTFunction *>(ast.get());
|
||||
|
||||
if (node && node->kind == ASTFunction::AGGREGATE_FUNCTION)
|
||||
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||
for (auto & argument : node->arguments->children)
|
||||
getRootActions(argument, no_subqueries, false, actions);
|
||||
else
|
||||
@ -2836,7 +2809,7 @@ void ExpressionAnalyzer::getRequiredColumnsImpl(const ASTPtr & ast,
|
||||
|
||||
if (ASTFunction * node = typeid_cast<ASTFunction *>(ast.get()))
|
||||
{
|
||||
if (node->kind == ASTFunction::LAMBDA_EXPRESSION)
|
||||
if (node->name == "lambda")
|
||||
{
|
||||
if (node->arguments->children.size() != 2)
|
||||
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
@ -12,24 +12,11 @@ namespace DB
|
||||
class ASTFunction : public ASTWithAlias
|
||||
{
|
||||
public:
|
||||
/// TODO This is semantic, not syntax. Remove it.
|
||||
enum FunctionKind
|
||||
{
|
||||
UNKNOWN,
|
||||
TABLE_FUNCTION,
|
||||
FUNCTION,
|
||||
AGGREGATE_FUNCTION,
|
||||
LAMBDA_EXPRESSION,
|
||||
ARRAY_JOIN,
|
||||
};
|
||||
|
||||
String name;
|
||||
ASTPtr arguments;
|
||||
/// parameters - for parametric aggregate function. Example: quantile(0.9)(x) - what in first parens are 'parameters'.
|
||||
ASTPtr parameters;
|
||||
|
||||
FunctionKind kind{UNKNOWN};
|
||||
|
||||
public:
|
||||
ASTFunction() = default;
|
||||
ASTFunction(const StringRange range_) : ASTWithAlias(range_) {}
|
||||
|
@ -28,7 +28,7 @@ ASTSelectQuery::ASTSelectQuery(const StringRange range_) : ASTQueryWithOutput(ra
|
||||
bool ASTSelectQuery::hasArrayJoin(const ASTPtr & ast)
|
||||
{
|
||||
if (const ASTFunction * function = typeid_cast<const ASTFunction *>(&*ast))
|
||||
if (function->kind == ASTFunction::ARRAY_JOIN)
|
||||
if (function->name == "arrayJoin")
|
||||
return true;
|
||||
|
||||
for (const auto & child : ast->children)
|
||||
|
@ -58,7 +58,6 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
if (!table_function_p.parse(pos, table_function, expected))
|
||||
return false;
|
||||
static_cast<ASTFunction &>(*table_function).kind = ASTFunction::TABLE_FUNCTION;
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -22,21 +22,9 @@ bool ParserTableExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
{
|
||||
auto res = std::make_shared<ASTTableExpression>();
|
||||
|
||||
if (ParserWithOptionalAlias(std::make_unique<ParserSubquery>(), true)
|
||||
.parse(pos, res->subquery, expected))
|
||||
{
|
||||
}
|
||||
else if (ParserWithOptionalAlias(std::make_unique<ParserFunction>(), true)
|
||||
.parse(pos, res->table_function, expected))
|
||||
{
|
||||
static_cast<ASTFunction &>(*res->table_function).kind = ASTFunction::TABLE_FUNCTION;
|
||||
}
|
||||
else if (ParserWithOptionalAlias(std::make_unique<ParserCompoundIdentifier>(), true)
|
||||
.parse(pos, res->database_and_table_name, expected))
|
||||
{
|
||||
static_cast<ASTIdentifier &>(*res->database_and_table_name).kind = ASTIdentifier::Table;
|
||||
}
|
||||
else
|
||||
if (!ParserWithOptionalAlias(std::make_unique<ParserSubquery>(), true).parse(pos, res->subquery, expected)
|
||||
&& !ParserWithOptionalAlias(std::make_unique<ParserFunction>(), true).parse(pos, res->table_function, expected)
|
||||
&& !ParserWithOptionalAlias(std::make_unique<ParserCompoundIdentifier>(), true).parse(pos, res->database_and_table_name, expected))
|
||||
return false;
|
||||
|
||||
/// FINAL
|
||||
|
@ -248,8 +248,7 @@ Block PKCondition::getBlockWithConstants(
|
||||
{ DataTypeUInt8().createColumnConstWithDefaultValue(1), std::make_shared<DataTypeUInt8>(), "_dummy" }
|
||||
};
|
||||
|
||||
const auto expr_for_constant_folding = ExpressionAnalyzer{query, context, nullptr, all_columns}
|
||||
.getConstActions();
|
||||
const auto expr_for_constant_folding = ExpressionAnalyzer{query, context, nullptr, all_columns}.getConstActions();
|
||||
|
||||
expr_for_constant_folding->execute(result);
|
||||
|
||||
|
@ -13,11 +13,11 @@ make -j24
|
||||
|
||||
# Copy binary to your server
|
||||
|
||||
scp ./dbms/src/Server/clickhouse yourserver:~/clickhouse-libcxx-asan
|
||||
scp ./dbms/src/Server/clickhouse yourserver:~/clickhouse-asan
|
||||
|
||||
# Start ClickHouse and run tests
|
||||
|
||||
sudo -u clickhouse ./clickhouse-libcxx-asan --config /etc/clickhouse-server/config.xml
|
||||
sudo -u clickhouse ./clickhouse-asan --config /etc/clickhouse-server/config.xml
|
||||
|
||||
|
||||
# How to use Thread Sanitizer
|
||||
@ -26,13 +26,13 @@ mkdir build && cd build
|
||||
|
||||
# Note: All parameters are mandatory.
|
||||
|
||||
CC=clang CXX=clang++ cmake -D CCACHE_FOUND=0 -D CMAKE_BUILD_TYPE=TSan -D ENABLE_TCMALLOC=0 ..
|
||||
CC=clang CXX=clang++ cmake -D CMAKE_BUILD_TYPE=TSan -D ENABLE_TCMALLOC=0 ..
|
||||
make -j24
|
||||
|
||||
# Copy binary to your server
|
||||
|
||||
scp ./dbms/src/Server/clickhouse yourserver:~/clickhouse-libcxx-tsan
|
||||
scp ./dbms/src/Server/clickhouse yourserver:~/clickhouse-tsan
|
||||
|
||||
# Start ClickHouse and run tests
|
||||
|
||||
sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1' ./clickhouse-libcxx-tsan --config /etc/clickhouse-server/config.xml
|
||||
sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1' ./clickhouse-tsan --config /etc/clickhouse-server/config.xml
|
||||
|
Loading…
Reference in New Issue
Block a user