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:
alexey-milovidov 2018-02-24 05:34:22 +03:00 committed by GitHub
commit 1497b61b79
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 190 additions and 244 deletions

View File

@ -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,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<DataTypeUInt8>();
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<DataTypeUInt8>(), 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<ASTFunction *>(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<ASTFunction *>(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<DataTypeFunction>(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<DataTypeSet>();
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<DataTypeUInt8>();
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<DataTypeUInt8>(), 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<ASTFunction *>(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<const DataTypeFunction *>(argument_types[i].get());
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(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<DataTypeFunction>(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<DataTypeSet>();
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<ASTIdentifier *>(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<FunctionCapture>(
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<DataTypeFunction>(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<ASTFunction *>(child.get());
if (lambda && lambda->name == "lambda")
{
const DataTypeFunction * lambda_type = typeid_cast<const DataTypeFunction *>(argument_types[i].get());
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(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<ASTIdentifier *>(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<FunctionCapture>(
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<DataTypeFunction>(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<ASTLiteral *>(ast.get()))
{
@ -2201,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<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);

View File

@ -48,8 +48,8 @@ ASTPtr ASTFunction::clone() const
auto res = std::make_shared<ASTFunction>(*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;
}

View File

@ -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_) {}

View File

@ -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)

View File

@ -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
{

View File

@ -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

View File

@ -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);

View File

@ -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