From 3772c8847213531307bef4286c99b7d0b8de615f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Feb 2018 04:31:42 +0300 Subject: [PATCH 1/2] Removed bad code that may lead to (harmless) race condition [#CLICKHOUSE-2] --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 372 ++++++++---------- dbms/src/Parsers/ASTFunction.cpp | 4 +- dbms/src/Parsers/ASTFunction.h | 13 - dbms/src/Parsers/ASTSelectQuery.cpp | 2 +- dbms/src/Parsers/ParserInsertQuery.cpp | 1 - .../src/Parsers/ParserTablesInSelectQuery.cpp | 18 +- dbms/tests/instructions/sanitizers.txt | 10 +- 7 files changed, 181 insertions(+), 239 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 2072d499317..b890f2fb33c 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -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(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; @@ -1181,7 +1152,6 @@ static ASTPtr addTypeConversion(std::unique_ptr && 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(ast->range); func->arguments = exp_list; @@ -1260,7 +1230,6 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast) auto tuple = std::make_shared(ast->range); tuple->alias = subquery->alias; ast = tuple; - tuple->kind = ASTFunction::FUNCTION; tuple->name = "tuple"; auto exp_list = std::make_shared(ast->range); tuple->arguments = exp_list; @@ -1286,8 +1255,7 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast) */ ASTFunction * func = typeid_cast(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(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(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,193 +1967,193 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries, return; } - if (node->kind == ASTFunction::FUNCTION) + if (functionIsInOrGlobalInOperator(node->name)) { - if (functionIsInOrGlobalInOperator(node->name)) + if (!no_subqueries) { - if (!no_subqueries) - { - /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). - getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack); + /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). + getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack); - /// Transform tuple or subquery into a set. - makeSet(node, actions_stack.getSampleBlock()); + /// Transform tuple or subquery into a set. + makeSet(node, actions_stack.getSampleBlock()); + } + else + { + if (!only_consts) + { + /// We are in the part of the tree that we are not going to compute. You just need to define types. + /// Do not subquery and create sets. We insert an arbitrary column of the correct type. + ColumnWithTypeAndName fake_column; + fake_column.name = node->getColumnName(); + fake_column.type = std::make_shared(); + actions_stack.addAction(ExpressionAction::addColumn(fake_column)); + getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack); + } + return; + } + } + + /// A special function `indexHint`. Everything that is inside it is not calculated + /// (and is used only for index analysis, see PKCondition). + if (node->name == "indexHint") + { + actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName( + ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared(), node->getColumnName()))); + return; + } + + if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name)) + return; + + const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(node->name, context); + + Names argument_names; + DataTypes argument_types; + bool arguments_present = true; + + /// If the function has an argument-lambda expression, you need to determine its type before the recursive call. + bool has_lambda_arguments = false; + + for (auto & child : node->arguments->children) + { + ASTFunction * lambda = typeid_cast(child.get()); + if (lambda && lambda->name == "lambda") + { + /// If the argument is a lambda expression, just remember its approximate type. + if (lambda->arguments->children.size() != 2) + throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + ASTFunction * lambda_args_tuple = typeid_cast(lambda->arguments->children.at(0).get()); + + if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") + throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); + + has_lambda_arguments = true; + argument_types.emplace_back(std::make_shared(DataTypes(lambda_args_tuple->arguments->children.size()))); + /// Select the name in the next cycle. + argument_names.emplace_back(); + } + else if (prepared_sets.count(child.get())) + { + ColumnWithTypeAndName column; + column.type = std::make_shared(); + + const SetPtr & set = prepared_sets[child.get()]; + + /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, + /// so that sets with the same record do not fuse together (they can have different types). + if (!set->empty()) + column.name = getUniqueName(actions_stack.getSampleBlock(), "__set"); + else + column.name = child->getColumnName(); + + if (!actions_stack.getSampleBlock().has(column.name)) + { + column.column = ColumnSet::create(1, set); + + actions_stack.addAction(ExpressionAction::addColumn(column)); + } + + argument_types.push_back(column.type); + argument_names.push_back(column.name); + } + else + { + /// If the argument is not a lambda expression, call it recursively and find out its type. + getActionsImpl(child, no_subqueries, only_consts, actions_stack); + std::string name = child->getColumnName(); + if (actions_stack.getSampleBlock().has(name)) + { + argument_types.push_back(actions_stack.getSampleBlock().getByName(name).type); + argument_names.push_back(name); } else { - if (!only_consts) + if (only_consts) { - /// We are in the part of the tree that we are not going to compute. You just need to define types. - /// Do not subquery and create sets. We insert an arbitrary column of the correct type. - ColumnWithTypeAndName fake_column; - fake_column.name = node->getColumnName(); - fake_column.type = std::make_shared(); - actions_stack.addAction(ExpressionAction::addColumn(fake_column)); - getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack); + arguments_present = false; + } + else + { + throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER); } - return; } } + } - /// A special function `indexHint`. Everything that is inside it is not calculated - /// (and is used only for index analysis, see PKCondition). - if (node->name == "indexHint") + if (only_consts && !arguments_present) + return; + + if (has_lambda_arguments && !only_consts) + { + function_builder->getLambdaArgumentTypes(argument_types); + + /// Call recursively for lambda expressions. + for (size_t i = 0; i < node->arguments->children.size(); ++i) { - actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName( - ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared(), node->getColumnName()))); - return; - } + ASTPtr child = node->arguments->children[i]; - const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(node->name, context); - - Names argument_names; - DataTypes argument_types; - bool arguments_present = true; - - /// If the function has an argument-lambda expression, you need to determine its type before the recursive call. - bool has_lambda_arguments = false; - - for (auto & child : node->arguments->children) - { ASTFunction * lambda = typeid_cast(child.get()); if (lambda && lambda->name == "lambda") { - /// If the argument is a lambda expression, just remember its approximate type. - if (lambda->arguments->children.size() != 2) - throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - + const DataTypeFunction * lambda_type = typeid_cast(argument_types[i].get()); ASTFunction * lambda_args_tuple = typeid_cast(lambda->arguments->children.at(0).get()); + ASTs lambda_arg_asts = lambda_args_tuple->arguments->children; + NamesAndTypesList lambda_arguments; - if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") - throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); - - has_lambda_arguments = true; - argument_types.emplace_back(std::make_shared(DataTypes(lambda_args_tuple->arguments->children.size()))); - /// Select the name in the next cycle. - argument_names.emplace_back(); - } - else if (prepared_sets.count(child.get())) - { - ColumnWithTypeAndName column; - column.type = std::make_shared(); - - const SetPtr & set = prepared_sets[child.get()]; - - /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, - /// so that sets with the same record do not fuse together (they can have different types). - if (!set->empty()) - column.name = getUniqueName(actions_stack.getSampleBlock(), "__set"); - else - column.name = child->getColumnName(); - - if (!actions_stack.getSampleBlock().has(column.name)) + for (size_t j = 0; j < lambda_arg_asts.size(); ++j) { - column.column = ColumnSet::create(1, set); + ASTIdentifier * identifier = typeid_cast(lambda_arg_asts[j].get()); + if (!identifier) + throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); - actions_stack.addAction(ExpressionAction::addColumn(column)); + String arg_name = identifier->name; + + lambda_arguments.emplace_back(arg_name, lambda_type->getArgumentTypes()[j]); } - argument_types.push_back(column.type); - argument_names.push_back(column.name); - } - else - { - /// If the argument is not a lambda expression, call it recursively and find out its type. - getActionsImpl(child, no_subqueries, only_consts, actions_stack); - std::string name = child->getColumnName(); - if (actions_stack.getSampleBlock().has(name)) - { - argument_types.push_back(actions_stack.getSampleBlock().getByName(name).type); - argument_names.push_back(name); - } - else - { - if (only_consts) - { - arguments_present = false; - } - else - { - throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER); - } - } + actions_stack.pushLevel(lambda_arguments); + getActionsImpl(lambda->arguments->children.at(1), no_subqueries, only_consts, actions_stack); + ExpressionActionsPtr lambda_actions = actions_stack.popLevel(); + + String result_name = lambda->arguments->children.at(1)->getColumnName(); + lambda_actions->finalize(Names(1, result_name)); + DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; + + Names captured; + Names required = lambda_actions->getRequiredColumns(); + for (size_t j = 0; j < required.size(); ++j) + if (findColumn(required[j], lambda_arguments) == lambda_arguments.end()) + captured.push_back(required[j]); + + /// We can not name `getColumnName()`, + /// because it does not uniquely define the expression (the types of arguments can be different). + String lambda_name = getUniqueName(actions_stack.getSampleBlock(), "__lambda"); + + auto function_capture = std::make_shared( + lambda_actions, captured, lambda_arguments, result_type, result_name); + actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name)); + + argument_types[i] = std::make_shared(lambda_type->getArgumentTypes(), result_type); + argument_names[i] = lambda_name; } } - - if (only_consts && !arguments_present) - return; - - if (has_lambda_arguments && !only_consts) - { - function_builder->getLambdaArgumentTypes(argument_types); - - /// Call recursively for lambda expressions. - for (size_t i = 0; i < node->arguments->children.size(); ++i) - { - ASTPtr child = node->arguments->children[i]; - - ASTFunction * lambda = typeid_cast(child.get()); - if (lambda && lambda->name == "lambda") - { - const DataTypeFunction * lambda_type = typeid_cast(argument_types[i].get()); - ASTFunction * lambda_args_tuple = typeid_cast(lambda->arguments->children.at(0).get()); - ASTs lambda_arg_asts = lambda_args_tuple->arguments->children; - NamesAndTypesList lambda_arguments; - - for (size_t j = 0; j < lambda_arg_asts.size(); ++j) - { - ASTIdentifier * identifier = typeid_cast(lambda_arg_asts[j].get()); - if (!identifier) - throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); - - String arg_name = identifier->name; - - lambda_arguments.emplace_back(arg_name, lambda_type->getArgumentTypes()[j]); - } - - actions_stack.pushLevel(lambda_arguments); - getActionsImpl(lambda->arguments->children.at(1), no_subqueries, only_consts, actions_stack); - ExpressionActionsPtr lambda_actions = actions_stack.popLevel(); - - String result_name = lambda->arguments->children.at(1)->getColumnName(); - lambda_actions->finalize(Names(1, result_name)); - DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; - - Names captured; - Names required = lambda_actions->getRequiredColumns(); - for (size_t j = 0; j < required.size(); ++j) - if (findColumn(required[j], lambda_arguments) == lambda_arguments.end()) - captured.push_back(required[j]); - - /// We can not name `getColumnName()`, - /// because it does not uniquely define the expression (the types of arguments can be different). - String lambda_name = getUniqueName(actions_stack.getSampleBlock(), "__lambda"); - - auto function_capture = std::make_shared( - lambda_actions, captured, lambda_arguments, result_type, result_name); - actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name)); - - argument_types[i] = std::make_shared(lambda_type->getArgumentTypes(), result_type); - argument_names[i] = lambda_name; - } - } - } - - if (only_consts) - { - for (size_t i = 0; i < argument_names.size(); ++i) - { - if (!actions_stack.getSampleBlock().has(argument_names[i])) - { - arguments_present = false; - break; - } - } - } - - if (arguments_present) - actions_stack.addAction(ExpressionAction::applyFunction(function_builder, argument_names, node->getColumnName())); } + + if (only_consts) + { + for (size_t i = 0; i < argument_names.size(); ++i) + { + if (!actions_stack.getSampleBlock().has(argument_names[i])) + { + arguments_present = false; + break; + } + } + } + + if (arguments_present) + actions_stack.addAction(ExpressionAction::applyFunction(function_builder, argument_names, node->getColumnName())); } else if (ASTLiteral * node = typeid_cast(ast.get())) { @@ -2223,7 +2191,7 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr } const ASTFunction * node = typeid_cast(ast.get()); - if (node && node->kind == ASTFunction::AGGREGATE_FUNCTION) + if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name)) { has_aggregation = true; AggregateDescription aggregate; @@ -2268,7 +2236,7 @@ void ExpressionAnalyzer::assertNoAggregates(const ASTPtr & ast, const char * des { const ASTFunction * node = typeid_cast(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 +2538,7 @@ void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, Express { ASTFunction * node = typeid_cast(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 +2804,7 @@ void ExpressionAnalyzer::getRequiredColumnsImpl(const ASTPtr & ast, if (ASTFunction * node = typeid_cast(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); diff --git a/dbms/src/Parsers/ASTFunction.cpp b/dbms/src/Parsers/ASTFunction.cpp index 434b14673ba..7d73e48b7ea 100644 --- a/dbms/src/Parsers/ASTFunction.cpp +++ b/dbms/src/Parsers/ASTFunction.cpp @@ -48,8 +48,8 @@ ASTPtr ASTFunction::clone() const auto res = std::make_shared(*this); res->children.clear(); - if (arguments) { res->arguments = arguments->clone(); res->children.push_back(res->arguments); } - if (parameters) { res->parameters = parameters->clone(); res->children.push_back(res->parameters); } + if (arguments) { res->arguments = arguments->clone(); res->children.push_back(res->arguments); } + if (parameters) { res->parameters = parameters->clone(); res->children.push_back(res->parameters); } return res; } diff --git a/dbms/src/Parsers/ASTFunction.h b/dbms/src/Parsers/ASTFunction.h index f4e7c32d139..462dc439329 100644 --- a/dbms/src/Parsers/ASTFunction.h +++ b/dbms/src/Parsers/ASTFunction.h @@ -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_) {} diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index d2d5b716d3a..aec8d6345d9 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -28,7 +28,7 @@ ASTSelectQuery::ASTSelectQuery(const StringRange range_) : ASTQueryWithOutput(ra bool ASTSelectQuery::hasArrayJoin(const ASTPtr & ast) { if (const ASTFunction * function = typeid_cast(&*ast)) - if (function->kind == ASTFunction::ARRAY_JOIN) + if (function->name == "arrayJoin") return true; for (const auto & child : ast->children) diff --git a/dbms/src/Parsers/ParserInsertQuery.cpp b/dbms/src/Parsers/ParserInsertQuery.cpp index 0842c1fbd92..2fe8ca46f68 100644 --- a/dbms/src/Parsers/ParserInsertQuery.cpp +++ b/dbms/src/Parsers/ParserInsertQuery.cpp @@ -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(*table_function).kind = ASTFunction::TABLE_FUNCTION; } else { diff --git a/dbms/src/Parsers/ParserTablesInSelectQuery.cpp b/dbms/src/Parsers/ParserTablesInSelectQuery.cpp index aa475fb477c..6b28deeb227 100644 --- a/dbms/src/Parsers/ParserTablesInSelectQuery.cpp +++ b/dbms/src/Parsers/ParserTablesInSelectQuery.cpp @@ -22,21 +22,9 @@ bool ParserTableExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec { auto res = std::make_shared(); - if (ParserWithOptionalAlias(std::make_unique(), true) - .parse(pos, res->subquery, expected)) - { - } - else if (ParserWithOptionalAlias(std::make_unique(), true) - .parse(pos, res->table_function, expected)) - { - static_cast(*res->table_function).kind = ASTFunction::TABLE_FUNCTION; - } - else if (ParserWithOptionalAlias(std::make_unique(), true) - .parse(pos, res->database_and_table_name, expected)) - { - static_cast(*res->database_and_table_name).kind = ASTIdentifier::Table; - } - else + if (!ParserWithOptionalAlias(std::make_unique(), true).parse(pos, res->subquery, expected) + && !ParserWithOptionalAlias(std::make_unique(), true).parse(pos, res->table_function, expected) + && !ParserWithOptionalAlias(std::make_unique(), true).parse(pos, res->database_and_table_name, expected)) return false; /// FINAL diff --git a/dbms/tests/instructions/sanitizers.txt b/dbms/tests/instructions/sanitizers.txt index 8cc67bae05a..552a313e5e2 100644 --- a/dbms/tests/instructions/sanitizers.txt +++ b/dbms/tests/instructions/sanitizers.txt @@ -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 From 2b974554e5092c4cc08e0b1861101393832bd31c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Feb 2018 04:55:43 +0300 Subject: [PATCH 2/2] Fixed test [#CLICKHOUSE-2] --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 11 ++++++++--- dbms/src/Storages/MergeTree/PKCondition.cpp | 3 +-- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index b890f2fb33c..b7f67e8566c 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -368,7 +368,7 @@ void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const String { /// Do not go to FROM, JOIN, UNION. if (!typeid_cast(child.get()) - && child.get() != select_query->next_union_all.get()) + && !typeid_cast(child.get())) { translateQualifiedNamesImpl(child, database_name, table_name, alias); } @@ -1136,7 +1136,7 @@ void ExpressionAnalyzer::executeScalarSubqueries() { /// Do not go to FROM, JOIN, UNION. if (!typeid_cast(child.get()) - && child.get() != select_query->next_union_all.get()) + && !typeid_cast(child.get())) { executeScalarSubqueriesImpl(child); } @@ -2169,7 +2169,12 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries, else { for (auto & child : ast->children) - getActionsImpl(child, no_subqueries, only_consts, actions_stack); + { + /// Do not go to FROM, JOIN, UNION. + if (!typeid_cast(child.get()) + && !typeid_cast(child.get())) + getActionsImpl(child, no_subqueries, only_consts, actions_stack); + } } } diff --git a/dbms/src/Storages/MergeTree/PKCondition.cpp b/dbms/src/Storages/MergeTree/PKCondition.cpp index a7198d5625d..798266e1d15 100644 --- a/dbms/src/Storages/MergeTree/PKCondition.cpp +++ b/dbms/src/Storages/MergeTree/PKCondition.cpp @@ -248,8 +248,7 @@ Block PKCondition::getBlockWithConstants( { DataTypeUInt8().createColumnConstWithDefaultValue(1), std::make_shared(), "_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);