ClickHouse/dbms/src/Storages/MergeTree/PKCondition.cpp

638 lines
18 KiB
C++
Raw Normal View History

2013-04-24 10:31:32 +00:00
#include <DB/Storages/MergeTree/PKCondition.h>
2012-12-05 12:44:55 +00:00
#include <DB/DataTypes/DataTypesNumberFixed.h>
2013-06-03 13:17:17 +00:00
#include <DB/Interpreters/ExpressionAnalyzer.h>
2014-03-20 12:25:26 +00:00
#include <DB/Columns/ColumnSet.h>
2014-03-26 18:19:25 +00:00
#include <DB/Columns/ColumnTuple.h>
#include <DB/Parsers/ASTSet.h>
#include <DB/Functions/FunctionFactory.h>
2012-12-05 12:44:55 +00:00
namespace DB
{
const PKCondition::AtomMap PKCondition::atom_map{
{
"notEquals",
[] (RPNElement & out, const Field & value, ASTPtr &)
{
out.function = RPNElement::FUNCTION_NOT_IN_RANGE;
out.range = Range(value);
}
},
{
"equals",
[] (RPNElement & out, const Field & value, ASTPtr &)
{
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = Range(value);
}
},
{
"less",
[] (RPNElement & out, const Field & value, ASTPtr &)
{
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = Range::createRightBounded(value, false);
}
},
{
"greater",
[] (RPNElement & out, const Field & value, ASTPtr &)
{
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = Range::createLeftBounded(value, false);
}
},
{
"lessOrEquals",
[] (RPNElement & out, const Field & value, ASTPtr &)
{
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = Range::createRightBounded(value, true);
}
},
{
"greaterOrEquals",
[] (RPNElement & out, const Field & value, ASTPtr &)
{
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = Range::createLeftBounded(value, true);
}
},
{
"in",
[] (RPNElement & out, const Field & value, ASTPtr & node)
{
out.function = RPNElement::FUNCTION_IN_SET;
out.in_function = node;
}
},
{
"notIn",
[] (RPNElement & out, const Field & value, ASTPtr & node)
{
out.function = RPNElement::FUNCTION_NOT_IN_SET;
out.in_function = node;
}
}
};
inline bool Range::equals(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateEquals(), lhs, rhs); }
inline bool Range::less(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateLess(), lhs, rhs); }
Block PKCondition::getBlockWithConstants(
const ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns)
{
Block result
{
{ new ColumnConstUInt8{1, 0}, new DataTypeUInt8, "_dummy" }
};
const auto expr_for_constant_folding = ExpressionAnalyzer{query, context, nullptr, all_columns}
.getConstActions();
expr_for_constant_folding->execute(result);
return result;
}
PKCondition::PKCondition(ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr_)
2013-05-06 12:15:34 +00:00
: sort_descr(sort_descr_)
2012-12-05 12:44:55 +00:00
{
for (size_t i = 0; i < sort_descr.size(); ++i)
{
std::string name = sort_descr[i].column_name;
if (!pk_columns.count(name))
pk_columns[name] = i;
2012-12-05 12:44:55 +00:00
}
2012-12-05 12:44:55 +00:00
/** Вычисление выражений, зависящих только от констант.
* Чтобы индекс мог использоваться, если написано, например WHERE Date = toDate(now()).
*/
Block block_with_constants = getBlockWithConstants(query, context, all_columns);
2012-12-05 12:44:55 +00:00
/// Преобразуем секцию WHERE в обратную польскую строку.
ASTSelectQuery & select = typeid_cast<ASTSelectQuery &>(*query);
2012-12-05 12:44:55 +00:00
if (select.where_expression)
{
traverseAST(select.where_expression, context, block_with_constants);
if (select.prewhere_expression)
{
traverseAST(select.prewhere_expression, context, block_with_constants);
2015-03-27 03:37:46 +00:00
rpn.emplace_back(RPNElement::FUNCTION_AND);
}
}
else if (select.prewhere_expression)
{
traverseAST(select.prewhere_expression, context, block_with_constants);
2012-12-05 12:44:55 +00:00
}
else
{
2015-03-27 03:37:46 +00:00
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
2012-12-05 12:44:55 +00:00
}
}
bool PKCondition::addCondition(const String & column, const Range & range)
{
if (!pk_columns.count(column))
return false;
2015-03-27 03:37:46 +00:00
rpn.emplace_back(RPNElement::FUNCTION_IN_RANGE, pk_columns[column], range);
rpn.emplace_back(RPNElement::FUNCTION_AND);
return true;
}
2012-12-05 12:44:55 +00:00
/** Получить значение константного выражения.
* Вернуть false, если выражение не константно.
*/
static bool getConstant(const ASTPtr & expr, Block & block_with_constants, Field & value)
2012-12-05 12:44:55 +00:00
{
String column_name = expr->getColumnName();
if (const ASTLiteral * lit = typeid_cast<const ASTLiteral *>(&*expr))
2012-12-05 12:44:55 +00:00
{
/// литерал
value = lit->value;
return true;
}
else if (block_with_constants.has(column_name) && block_with_constants.getByName(column_name).column->isConst())
{
/// выражение, вычислившееся в константу
value = (*block_with_constants.getByName(column_name).column)[0];
return true;
}
else
return false;
}
void PKCondition::traverseAST(ASTPtr & node, const Context & context, Block & block_with_constants)
2012-12-05 12:44:55 +00:00
{
RPNElement element;
if (ASTFunction * func = typeid_cast<ASTFunction *>(&*node))
2012-12-05 12:44:55 +00:00
{
if (operatorFromAST(func, element))
{
auto & args = typeid_cast<ASTExpressionList &>(*func->arguments).children;
for (size_t i = 0, size = args.size(); i < size; ++i)
2012-12-05 12:44:55 +00:00
{
traverseAST(args[i], context, block_with_constants);
/** Первая часть условия - для корректной поддержки функций and и or произвольной арности
* - в этом случае добавляется n - 1 элементов (где n - количество аргументов).
*/
if (i != 0 || element.function == RPNElement::FUNCTION_NOT)
rpn.push_back(element);
2012-12-05 12:44:55 +00:00
}
2012-12-05 12:44:55 +00:00
return;
}
}
if (!atomFromAST(node, context, block_with_constants, element))
2012-12-05 12:44:55 +00:00
{
element.function = RPNElement::FUNCTION_UNKNOWN;
}
2012-12-05 12:44:55 +00:00
rpn.push_back(element);
}
bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
const ASTPtr & node,
const Context & context,
size_t & out_primary_key_column_num,
RPNElement::MonotonicFunctionsChain & out_functions_chain)
2012-12-05 12:44:55 +00:00
{
std::vector<const ASTFunction *> chain_not_tested_for_monotonicity;
if (!isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_primary_key_column_num, chain_not_tested_for_monotonicity))
return false;
for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it)
{
std::cerr << "!!\n";
FunctionPtr func = FunctionFactory::instance().tryGet((*it)->name, context);
if (!func || !func->hasInformationAboutMonotonicity())
return false;
std::cerr << "!!!\n";
out_functions_chain.push_back(func);
}
return true;
}
bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(
const ASTPtr & node,
size_t & out_primary_key_column_num,
std::vector<const ASTFunction *> & out_functions_chain)
{
/** Сам по себе, столбец первичного ключа может быть функциональным выражением. Например, intHash32(UserID).
* Поэтому, используем полное имя выражения для поиска.
*/
String name = node->getColumnName();
auto it = pk_columns.find(name);
if (pk_columns.end() != it)
{
out_primary_key_column_num = it->second;
return true;
}
if (const ASTFunction * func = typeid_cast<const ASTFunction *>(node.get()))
{
const auto & args = func->arguments->children;
if (args.size() != 1)
return false;
out_functions_chain.push_back(func);
if (!isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_primary_key_column_num, out_functions_chain))
return false;
return true;
}
return false;
}
bool PKCondition::atomFromAST(ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out)
{
/** Функции < > = != <= >= in notIn, у которых один агрумент константа, другой - один из столбцов первичного ключа,
* либо он же, завёрнутый в цепочку возможно-монотонных функций.
*/
if (const ASTFunction * func = typeid_cast<const ASTFunction *>(&*node))
2012-12-05 12:44:55 +00:00
{
const ASTs & args = typeid_cast<const ASTExpressionList &>(*func->arguments).children;
2012-12-05 12:44:55 +00:00
if (args.size() != 2)
return false;
2012-12-05 12:44:55 +00:00
/// Если true, слева константа.
bool inverted;
size_t column;
Field value;
RPNElement::MonotonicFunctionsChain chain;
2014-03-20 12:25:26 +00:00
if (getConstant(args[1], block_with_constants, value) && isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[0], context, column, chain))
2012-12-05 12:44:55 +00:00
{
inverted = false;
}
else if (getConstant(args[0], block_with_constants, value) && isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[1], context, column, chain))
2012-12-05 12:44:55 +00:00
{
inverted = true;
}
else if (typeid_cast<const ASTSet *>(args[1].get()) && isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[0], context, column, chain))
2014-03-20 12:25:26 +00:00
{
2014-03-26 18:19:25 +00:00
inverted = false;
2014-03-20 12:25:26 +00:00
}
2012-12-05 12:44:55 +00:00
else
return false;
2012-12-05 12:44:55 +00:00
std::string func_name = func->name;
2012-12-05 12:44:55 +00:00
/// Заменим <const> <sign> <column> на <column> <-sign> <const>
if (inverted)
{
if (func_name == "less")
func_name = "greater";
else if (func_name == "greater")
func_name = "less";
else if (func_name == "greaterOrEquals")
func_name = "lessOrEquals";
else if (func_name == "lessOrEquals")
func_name = "greaterOrEquals";
else if (func_name == "in" || func_name == "notIn")
{
/// const IN x не имеет смысла (в отличие от x IN const).
return false;
}
2012-12-05 12:44:55 +00:00
}
2012-12-05 12:44:55 +00:00
out.key_column = column;
out.monotonic_functions_chain = std::move(chain);
const auto atom_it = atom_map.find(func_name);
if (atom_it == std::end(atom_map))
2012-12-05 12:44:55 +00:00
return false;
atom_it->second(out, value, node);
2012-12-05 12:44:55 +00:00
return true;
}
2012-12-05 12:44:55 +00:00
return false;
}
bool PKCondition::operatorFromAST(const ASTFunction * func, RPNElement & out)
2012-12-05 12:44:55 +00:00
{
/// Функции AND, OR, NOT.
const ASTs & args = typeid_cast<const ASTExpressionList &>(*func->arguments).children;
2012-12-05 12:44:55 +00:00
if (func->name == "not")
{
if (args.size() != 1)
return false;
2012-12-05 12:44:55 +00:00
out.function = RPNElement::FUNCTION_NOT;
}
else
{
if (func->name == "and")
out.function = RPNElement::FUNCTION_AND;
else if (func->name == "or")
out.function = RPNElement::FUNCTION_OR;
else
return false;
}
2012-12-05 12:44:55 +00:00
return true;
}
2015-03-27 03:37:46 +00:00
String PKCondition::toString() const
2012-12-05 12:44:55 +00:00
{
String res;
for (size_t i = 0; i < rpn.size(); ++i)
{
if (i)
res += ", ";
res += rpn[i].toString();
}
return res;
}
static void applyFunction(
FunctionPtr & func,
const DataTypePtr & arg_type, const Field & arg_value,
DataTypePtr & res_type, Field & res_value)
{
res_type = func->getReturnType({arg_type});
Block block
{
{ arg_type->createConstColumn(1, arg_value), arg_type, "x" },
{ nullptr, res_type, "y" }
};
func->execute(block, {0}, 1);
block.getByPosition(1).column->get(0, res_value);
}
bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk, const DataTypes & data_types, bool right_bounded) const
2012-12-05 12:44:55 +00:00
{
/// Найдем диапазоны элементов ключа.
std::vector<Range> key_ranges(sort_descr.size(), Range());
2012-12-06 09:45:09 +00:00
if (right_bounded)
2012-12-05 12:44:55 +00:00
{
2012-12-06 09:45:09 +00:00
for (size_t i = 0; i < sort_descr.size(); ++i)
2012-12-05 12:44:55 +00:00
{
2012-12-06 09:45:09 +00:00
if (left_pk[i] == right_pk[i])
{
key_ranges[i] = Range(left_pk[i]);
}
else
{
key_ranges[i] = Range(left_pk[i], true, right_pk[i], true);
break;
}
2012-12-05 12:44:55 +00:00
}
}
2012-12-06 09:45:09 +00:00
else
{
key_ranges[0] = Range::createLeftBounded(left_pk[0], true);
2012-12-06 09:45:09 +00:00
}
2012-12-05 12:44:55 +00:00
std::vector<BoolMask> rpn_stack;
for (size_t i = 0; i < rpn.size(); ++i)
{
2015-03-27 03:37:46 +00:00
const auto & element = rpn[i];
2012-12-05 12:44:55 +00:00
if (element.function == RPNElement::FUNCTION_UNKNOWN)
{
rpn_stack.emplace_back(true, true);
2012-12-05 12:44:55 +00:00
}
else if (element.function == RPNElement::FUNCTION_IN_RANGE
|| element.function == RPNElement::FUNCTION_NOT_IN_RANGE
|| element.function == RPNElement::FUNCTION_IN_SET
|| element.function == RPNElement::FUNCTION_NOT_IN_SET)
2012-12-05 12:44:55 +00:00
{
const Range * key_range = &key_ranges[element.key_column];
/// Случай, когда столбец обёрнут в цепочку возможно-монотонных функций.
Range key_range_transformed;
bool evaluation_is_not_possible = false;
if (!element.monotonic_functions_chain.empty())
{
key_range_transformed = *key_range;
DataTypePtr current_type = data_types[element.key_column];
for (auto & func : element.monotonic_functions_chain)
{
/// Проверяем монотонность каждой функции на конкретном диапазоне.
IFunction::Monotonicity monotonicity = func->getMonotonicityForRange(key_range_transformed.left, key_range_transformed.right);
if (!monotonicity.is_monotonic)
{
evaluation_is_not_possible = true;
break;
}
/// Вычисляем функцию.
DataTypePtr new_type;
applyFunction(func, current_type, key_range_transformed.left, new_type, key_range_transformed.left);
applyFunction(func, current_type, key_range_transformed.right, new_type, key_range_transformed.right);
current_type.swap(new_type);
if (!monotonicity.is_positive)
key_range_transformed.swapLeftAndRight();
}
if (evaluation_is_not_possible)
{
rpn_stack.emplace_back(true, true);
continue;
}
key_range = &key_range_transformed;
}
2014-03-26 10:56:21 +00:00
if (element.function == RPNElement::FUNCTION_IN_RANGE
|| element.function == RPNElement::FUNCTION_NOT_IN_RANGE)
{
bool intersects = element.range.intersectsRange(*key_range);
bool contains = element.range.containsRange(*key_range);
rpn_stack.emplace_back(intersects, !contains);
if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE)
rpn_stack.back() = !rpn_stack.back();
}
else /// Set
{
auto in_func = typeid_cast<const ASTFunction *>(element.in_function.get());
const ASTs & args = typeid_cast<const ASTExpressionList &>(*in_func->arguments).children;
auto ast_set = typeid_cast<const ASTSet *>(args[1].get());
if (in_func && ast_set)
{
const Range & key_range = key_ranges[element.key_column];
rpn_stack.push_back(ast_set->set->mayBeTrueInRange(key_range));
if (element.function == RPNElement::FUNCTION_NOT_IN_SET)
rpn_stack.back() = !rpn_stack.back();
}
else
{
throw DB::Exception("Set for IN is not created yet!", ErrorCodes::LOGICAL_ERROR);
}
}
2014-03-26 10:56:21 +00:00
}
2012-12-05 12:44:55 +00:00
else if (element.function == RPNElement::FUNCTION_NOT)
{
rpn_stack.back() = !rpn_stack.back();
}
else if (element.function == RPNElement::FUNCTION_AND)
{
auto arg1 = rpn_stack.back();
2012-12-05 12:44:55 +00:00
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
2012-12-05 12:44:55 +00:00
rpn_stack.back() = arg1 & arg2;
}
else if (element.function == RPNElement::FUNCTION_OR)
{
auto arg1 = rpn_stack.back();
2012-12-05 12:44:55 +00:00
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
2012-12-05 12:44:55 +00:00
rpn_stack.back() = arg1 | arg2;
}
else
2012-12-10 10:23:10 +00:00
throw Exception("Unexpected function type in PKCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
2012-12-05 12:44:55 +00:00
}
2012-12-05 12:44:55 +00:00
if (rpn_stack.size() != 1)
throw Exception("Unexpected stack size in PkCondition::mayBeTrueInRange", ErrorCodes::LOGICAL_ERROR);
2012-12-05 12:44:55 +00:00
return rpn_stack[0].can_be_true;
}
bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk, const DataTypes & data_types) const
2012-12-06 09:45:09 +00:00
{
return mayBeTrueInRange(left_pk, right_pk, data_types, true);
2012-12-06 09:45:09 +00:00
}
bool PKCondition::mayBeTrueAfter(const Field * left_pk, const DataTypes & data_types) const
2012-12-06 09:45:09 +00:00
{
return mayBeTrueInRange(left_pk, nullptr, data_types, false);
2012-12-06 09:45:09 +00:00
}
static const ASTSet & inFunctionToSet(const ASTPtr & in_function)
{
const auto & in_func = typeid_cast<const ASTFunction &>(*in_function);
const auto & args = typeid_cast<const ASTExpressionList &>(*in_func.arguments).children;
const auto & ast_set = typeid_cast<const ASTSet &>(*args[1]);
return ast_set;
}
2015-03-27 03:37:46 +00:00
String PKCondition::RPNElement::toString() const
{
auto print_wrapped_column = [this](std::ostringstream & ss)
{
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
ss << (*it)->getName() << "(";
ss << "column " << key_column;
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
ss << ")";
};
std::ostringstream ss;
switch (function)
{
case FUNCTION_AND:
return "and";
case FUNCTION_OR:
return "or";
case FUNCTION_NOT:
return "not";
case FUNCTION_UNKNOWN:
return "unknown";
case FUNCTION_NOT_IN_SET:
case FUNCTION_IN_SET:
{
ss << "(";
print_wrapped_column(ss);
ss << (function == FUNCTION_IN_SET ? " in " : " notIn ") << inFunctionToSet(in_function).set->describe();
ss << ")";
return ss.str();
}
case FUNCTION_IN_RANGE:
case FUNCTION_NOT_IN_RANGE:
{
ss << "(";
print_wrapped_column(ss);
ss << (function == FUNCTION_NOT_IN_RANGE ? " not" : "") << " in " << range.toString();
ss << ")";
return ss.str();
}
default:
throw Exception("Unknown function in RPNElement", ErrorCodes::LOGICAL_ERROR);
}
}
2015-03-27 03:37:46 +00:00
bool PKCondition::alwaysUnknown() const
{
std::vector<UInt8> rpn_stack;
for (size_t i = 0; i < rpn.size(); ++i)
{
2015-03-27 03:37:46 +00:00
const auto & element = rpn[i];
if (element.function == RPNElement::FUNCTION_UNKNOWN)
{
rpn_stack.push_back(true);
}
else if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE
|| element.function == RPNElement::FUNCTION_IN_RANGE
|| element.function == RPNElement::FUNCTION_IN_SET
|| element.function == RPNElement::FUNCTION_NOT_IN_SET)
{
rpn_stack.push_back(false);
}
else if (element.function == RPNElement::FUNCTION_NOT)
{
}
else if (element.function == RPNElement::FUNCTION_AND)
{
auto arg1 = rpn_stack.back();
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
rpn_stack.back() = arg1 & arg2;
}
else if (element.function == RPNElement::FUNCTION_OR)
{
auto arg1 = rpn_stack.back();
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
rpn_stack.back() = arg1 | arg2;
}
else
throw Exception("Unexpected function type in PKCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
}
return rpn_stack[0];
}
2012-12-05 12:44:55 +00:00
}