diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index 054ca094bdc..47fa401f926 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -1142,9 +1142,30 @@ public: const auto & col_with_type_and_name_right = block.getByPosition(arguments[1]); const IColumn * col_left_untyped = col_with_type_and_name_left.column.get(); const IColumn * col_right_untyped = col_with_type_and_name_right.column.get(); + const DataTypePtr & left_type = col_with_type_and_name_left.type; const DataTypePtr & right_type = col_with_type_and_name_right.type; + /// The case when arguments are the same (tautological comparison). Return constant. + /// NOTE: Nullable types are special case. (BTW, this function use default implementation for Nullable, so Nullable types cannot be here. Check just in case.) + /// NOTE: We consider NaN comparison to be implementation specific (and in our implementation NaNs are sometimes equal sometimes not). + if (left_type->equals(*right_type) && !left_type->isNullable() && col_left_untyped == col_right_untyped) + { + /// Always true: =, <=, >= + if constexpr (std::is_same_v, EqualsOp> + || std::is_same_v, LessOrEqualsOp> + || std::is_same_v, GreaterOrEqualsOp>) + { + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, 1u); + return; + } + else + { + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, 0u); + return; + } + } + WhichDataType which_left{left_type}; WhichDataType which_right{right_type}; diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index d52a855e3f3..89a499b710a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -10,241 +10,222 @@ #include #include #include -#include #include #include -#include -#include #include -#include -#include -#include -#include -#include namespace DB { -static constexpr auto threshold = 10; -/// We decided to remove the restriction due to the absence of a penalty for the transfer in PREWHERE -static constexpr auto max_columns_relative_size = 1.0f; -static constexpr auto and_function_name = "and"; -static constexpr auto equals_function_name = "equals"; -static constexpr auto array_join_function_name = "arrayJoin"; -static constexpr auto global_in_function_name = "globalIn"; -static constexpr auto global_not_in_function_name = "globalNotIn"; +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +/// Conditions like "x = N" are considered good if abs(N) > threshold. +/// This is used to assume that condition is likely to have good selectivity. +static constexpr auto threshold = 2; MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( SelectQueryInfo & query_info, const Context & context, const MergeTreeData & data, - const Names & column_names, + const Names & queried_columns, Logger * log) - : primary_key_columns{ext::collection_cast(data.primary_key_columns)}, - table_columns{ext::map(data.getColumns().getAllPhysical(), + : table_columns{ext::map(data.getColumns().getAllPhysical(), [] (const NameAndTypePair & col) { return col.name; })}, + queried_columns{queried_columns}, block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}, log{log} { - calculateColumnSizes(data, column_names); + if (!data.primary_key_columns.empty()) + first_primary_key_column = data.primary_key_columns[0]; + + calculateColumnSizes(data, queried_columns); auto & select = typeid_cast(*query_info.query); determineArrayJoinedNames(select); optimize(select); } +void MergeTreeWhereOptimizer::calculateColumnSizes(const MergeTreeData & data, const Names & column_names) +{ + for (const auto & column_name : column_names) + { + UInt64 size = data.getColumnCompressedSize(column_name); + column_sizes[column_name] = size; + total_size_of_queried_columns += size; + } +} + + +static void collectIdentifiersNoSubqueries(const ASTPtr & ast, NameSet & set) +{ + if (auto opt_name = getIdentifierName(ast)) + return (void)set.insert(*opt_name); + + if (typeid_cast(ast.get())) + return; + + for (const auto & child : ast->children) + collectIdentifiersNoSubqueries(child, set); +} + +void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const ASTPtr & node) const +{ + if (const auto func_and = typeid_cast(node.get()); func_and && func_and->name == "and") + { + for (const auto & elem : func_and->arguments->children) + analyzeImpl(res, elem); + } + else + { + Condition cond; + cond.node = node; + + collectIdentifiersNoSubqueries(node, cond.identifiers); + + cond.viable = + /// Condition depend on some column. Constant expressions are not moved. + !cond.identifiers.empty() + && !cannotBeMoved(node) + /// Do not take into consideration the conditions consisting only of the first primary key column + && !hasPrimaryKeyAtoms(node) + /// Only table columns are considered. Not array joined columns. NOTE We're assuming that aliases was expanded. + && isSubsetOfTableColumns(cond.identifiers) + /// Do not move conditions involving all queried columns. + && cond.identifiers.size() < queried_columns.size(); + + if (cond.viable) + { + cond.columns_size = getIdentifiersColumnSize(cond.identifiers); + cond.good = isConditionGood(node); + } + + res.emplace_back(std::move(cond)); + } +} + +/// Transform conjunctions chain in WHERE expression to Conditions list. +MergeTreeWhereOptimizer::Conditions MergeTreeWhereOptimizer::analyze(const ASTPtr & expression) const +{ + Conditions res; + analyzeImpl(res, expression); + return res; +} + +/// Transform Conditions list to WHERE or PREWHERE expression. +ASTPtr MergeTreeWhereOptimizer::reconstruct(const Conditions & conditions) const +{ + if (conditions.empty()) + return {}; + + if (conditions.size() == 1) + return conditions.front().node; + + const auto function = std::make_shared(); + + function->name = "and"; + function->arguments = std::make_shared(); + function->children.push_back(function->arguments); + + for (const auto & elem : conditions) + function->arguments->children.push_back(elem.node); + + return function; +} + + void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const { if (!select.where_expression || select.prewhere_expression) return; - const auto function = typeid_cast(select.where_expression.get()); - if (function && function->name == and_function_name) - optimizeConjunction(select, function); + Conditions where_conditions = analyze(select.where_expression); + Conditions prewhere_conditions; + + UInt64 total_size_of_moved_conditions = 0; + + /// Move condition and all other conditions depend on the same set of columns. + auto move_condition = [&](Conditions::iterator cond_it) + { + prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, cond_it); + total_size_of_moved_conditions += cond_it->columns_size; + + /// Move all other conditions that depend on the same set of columns. + for (auto jt = where_conditions.begin(); jt != where_conditions.end();) + { + if (jt->columns_size == cond_it->columns_size && jt->identifiers == cond_it->identifiers) + prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, jt++); + else + ++jt; + } + }; + + /// Move conditions unless the ratio of total_size_of_moved_conditions to the total_size_of_queried_columns is less than some threshold. + while (!where_conditions.empty()) + { + /// Move the best condition to PREWHERE if it is viable. + + auto it = std::min_element(where_conditions.begin(), where_conditions.end()); + + if (!it->viable) + break; + + /// 10% ratio is just a guess. + if (total_size_of_moved_conditions > 0 && (total_size_of_moved_conditions + it->columns_size) * 10 > total_size_of_queried_columns) + break; + + move_condition(it); + } + + /// Nothing was moved. + if (prewhere_conditions.empty()) + return; + + /// Rewrite the SELECT query. + + auto old_where = std::find(std::begin(select.children), std::end(select.children), select.where_expression); + if (old_where == select.children.end()) + throw Exception("Logical error: cannot find WHERE expression in the list of children of SELECT query", ErrorCodes::LOGICAL_ERROR); + + select.where_expression = reconstruct(where_conditions); + select.prewhere_expression = reconstruct(prewhere_conditions); + + if (select.where_expression) + *old_where = select.where_expression; else - optimizeArbitrary(select); + select.children.erase(old_where); + + select.children.push_back(select.prewhere_expression); + + LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"" << select.prewhere_expression << "\" moved to PREWHERE"); } -void MergeTreeWhereOptimizer::calculateColumnSizes(const MergeTreeData & data, const Names & column_names) +UInt64 MergeTreeWhereOptimizer::getIdentifiersColumnSize(const NameSet & identifiers) const { - for (const auto & column_name : column_names) - { - const auto column_size = data.getColumnCompressedSize(column_name); - - column_sizes[column_name] = column_size; - total_column_size += column_size; - } -} - - -void MergeTreeWhereOptimizer::optimizeConjunction(ASTSelectQuery & select, ASTFunction * const fun) const -{ - /// used as max possible size and indicator that appropriate condition has not been found - const auto no_such_condition = std::numeric_limits::max(); - - /// { first: condition index, second: summary column size } - std::pair lightest_good_condition{no_such_condition, no_such_condition}; - std::pair lightest_viable_condition{no_such_condition, no_such_condition}; - - auto & conditions = fun->arguments->children; - - /// remove condition by swapping it with the last one and calling ::pop_back() - const auto remove_condition_at_index = [&conditions] (const size_t idx) - { - if (idx < conditions.size() - 1) - std::swap(conditions[idx], conditions.back()); - conditions.pop_back(); - }; - - /// linearize conjunction and divide conditions into "good" and not-"good" ones - for (size_t idx = 0; idx < conditions.size();) - { - const auto condition = conditions[idx].get(); - - /// linearize sub-conjunctions - if (const auto function = typeid_cast(condition)) - { - if (function->name == and_function_name) - { - for (auto & child : function->arguments->children) - conditions.emplace_back(std::move(child)); - - /// remove the condition corresponding to conjunction - remove_condition_at_index(idx); - - /// continue iterating without increment to ensure the just added conditions are processed - continue; - } - } - - SCOPE_EXIT(++idx); - - if (cannotBeMoved(conditions[idx])) - continue; - - IdentifierNameSet identifiers{}; - collectIdentifiersNoSubqueries(condition, identifiers); - - /// do not take into consideration the conditions consisting only of primary key columns - if (!hasPrimaryKeyAtoms(condition) && isSubsetOfTableColumns(identifiers)) - { - /// calculate size of columns involved in condition - const auto cond_columns_size = getIdentifiersColumnSize(identifiers); - - /// place condition either in good or viable conditions set - auto & good_or_viable_condition = isConditionGood(condition) ? lightest_good_condition : lightest_viable_condition; - if (good_or_viable_condition.second > cond_columns_size) - { - good_or_viable_condition.first = idx; - good_or_viable_condition.second = cond_columns_size; - } - } - } - - const auto move_condition_to_prewhere = [&] (const size_t idx) - { - select.prewhere_expression = conditions[idx]; - select.children.push_back(select.prewhere_expression); - LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition `" << select.prewhere_expression << "` moved to PREWHERE"); - - /** Replace conjunction with the only remaining argument if only two conditions were present, - * remove selected condition from conjunction otherwise. */ - if (conditions.size() == 2) - { - /// find old where_expression in children of select - const auto it = std::find(std::begin(select.children), std::end(select.children), select.where_expression); - /// replace where_expression with the remaining argument - select.where_expression = std::move(conditions[idx == 0 ? 1 : 0]); - /// overwrite child entry with the new where_expression - *it = select.where_expression; - } - else - remove_condition_at_index(idx); - }; - - /// if there is a "good" condition - move it to PREWHERE - if (lightest_good_condition.first != no_such_condition) - { - move_condition_to_prewhere(lightest_good_condition.first); - } - else if (lightest_viable_condition.first != no_such_condition) - { - /// check that the relative column size is less than max - if (total_column_size != 0) - { - /// calculate relative size of condition's columns - const auto cond_columns_size = lightest_viable_condition.second; - const auto columns_relative_size = static_cast(cond_columns_size) / total_column_size; - - /// do nothing if it exceeds max relative size - if (columns_relative_size > max_columns_relative_size) - return; - } - - move_condition_to_prewhere(lightest_viable_condition.first); - } -} - - -void MergeTreeWhereOptimizer::optimizeArbitrary(ASTSelectQuery & select) const -{ - auto & condition = select.where_expression; - - /// do not optimize restricted expressions - if (cannotBeMoved(select.where_expression)) - return; - - IdentifierNameSet identifiers{}; - collectIdentifiersNoSubqueries(condition.get(), identifiers); - - if (hasPrimaryKeyAtoms(condition.get()) || !isSubsetOfTableColumns(identifiers)) - return; - - /// if condition is not "good" - check that it can be moved - if (!isConditionGood(condition.get()) && total_column_size != 0) - { - const auto cond_columns_size = getIdentifiersColumnSize(identifiers); - const auto columns_relative_size = static_cast(cond_columns_size) / total_column_size; - - if (columns_relative_size > max_columns_relative_size) - return; - } - - /// add the condition to PREWHERE, remove it from WHERE - std::swap(select.prewhere_expression, condition); - LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition `" << select.prewhere_expression << "` moved to PREWHERE"); -} - - -size_t MergeTreeWhereOptimizer::getIdentifiersColumnSize(const IdentifierNameSet & identifiers) const -{ - /** for expressions containing no columns (or where columns could not be determined otherwise) assume maximum - * possible size so they do not have priority in eligibility over other expressions. */ - if (identifiers.empty()) - return std::numeric_limits::max(); - - size_t size{}; + UInt64 size = 0; for (const auto & identifier : identifiers) if (column_sizes.count(identifier)) - size += column_sizes.find(identifier)->second; + size += column_sizes.at(identifier); return size; } -bool MergeTreeWhereOptimizer::isConditionGood(const IAST * condition) const +bool MergeTreeWhereOptimizer::isConditionGood(const ASTPtr & condition) const { - const auto function = typeid_cast(condition); + const auto function = typeid_cast(condition.get()); if (!function) return false; /** we are only considering conditions of form `equals(one, another)` or `one = another`, * especially if either `one` or `another` is ASTIdentifier */ - if (function->name != equals_function_name) + if (function->name != "equals") return false; auto left_arg = function->arguments->children.front().get(); @@ -285,29 +266,16 @@ bool MergeTreeWhereOptimizer::isConditionGood(const IAST * condition) const } -void MergeTreeWhereOptimizer::collectIdentifiersNoSubqueries(const IAST * const ast, IdentifierNameSet & set) +bool MergeTreeWhereOptimizer::hasPrimaryKeyAtoms(const ASTPtr & ast) const { - if (auto opt_name = getIdentifierName(ast)) - return (void) set.insert(*opt_name); - - if (typeid_cast(ast)) - return; - - for (const auto & child : ast->children) - collectIdentifiersNoSubqueries(child.get(), set); -} - - -bool MergeTreeWhereOptimizer::hasPrimaryKeyAtoms(const IAST * ast) const -{ - if (const auto func = typeid_cast(ast)) + if (const auto func = typeid_cast(ast.get())) { const auto & args = func->arguments->children; if ((func->name == "not" && 1 == args.size()) || func->name == "and" || func->name == "or") { for (const auto & arg : args) - if (hasPrimaryKeyAtoms(arg.get())) + if (hasPrimaryKeyAtoms(arg)) return true; return false; @@ -318,9 +286,9 @@ bool MergeTreeWhereOptimizer::hasPrimaryKeyAtoms(const IAST * ast) const } -bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const +bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const ASTPtr & ast) const { - if (const auto func = typeid_cast(ast)) + if (const auto func = typeid_cast(ast.get())) { if (!KeyCondition::atom_map.count(func->name)) return false; @@ -332,9 +300,9 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const const auto & first_arg_name = args.front()->getColumnName(); const auto & second_arg_name = args.back()->getColumnName(); - if ((primary_key_columns.count(first_arg_name) && isConstant(args[1])) || - (primary_key_columns.count(second_arg_name) && isConstant(args[0])) || - (primary_key_columns.count(first_arg_name) && functionIsInOrGlobalInOperator(func->name))) + if ((first_primary_key_column == first_arg_name && isConstant(args[1])) + || (first_primary_key_column == second_arg_name && isConstant(args[0])) + || (first_primary_key_column == first_arg_name && functionIsInOrGlobalInOperator(func->name))) return true; } @@ -346,15 +314,15 @@ bool MergeTreeWhereOptimizer::isConstant(const ASTPtr & expr) const { const auto column_name = expr->getColumnName(); - if (typeid_cast(expr.get()) || - (block_with_constants.has(column_name) && block_with_constants.getByName(column_name).column->isColumnConst())) + if (typeid_cast(expr.get()) + || (block_with_constants.has(column_name) && block_with_constants.getByName(column_name).column->isColumnConst())) return true; return false; } -bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const IdentifierNameSet & identifiers) const +bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const NameSet & identifiers) const { for (const auto & identifier : identifiers) if (table_columns.count(identifier) == 0) @@ -369,12 +337,12 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr) const if (const auto function_ptr = typeid_cast(ptr.get())) { /// disallow arrayJoin expressions to be moved to PREWHERE for now - if (array_join_function_name == function_ptr->name) + if ("arrayJoin" == function_ptr->name) return true; /// disallow GLOBAL IN, GLOBAL NOT IN - if (global_in_function_name == function_ptr->name - || global_not_in_function_name == function_ptr->name) + if ("globalIn" == function_ptr->name + || "globalNotIn" == function_ptr->name) return true; /// indexHint is a special function that it does not make sense to transfer to PREWHERE diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 5f8dd587a92..ccc1195cada 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -17,17 +17,12 @@ class ASTSelectQuery; class ASTFunction; class MergeTreeData; -using IdentifierNameSet = std::set; - - /** Identifies WHERE expressions that can be placed in PREWHERE by calculating respective * sizes of columns used in particular expression and identifying "good" conditions of * form "column_name = constant", where "constant" is outside some `threshold` specified in advance. * - * If there are "good" conditions present in WHERE, the one with minimal summary column size is - * transferred to PREWHERE. - * Otherwise any condition with minimal summary column size can be transferred to PREWHERE, if only - * its relative size (summary column size divided by query column size) is less than `max_columns_relative_size`. + * If there are "good" conditions present in WHERE, the one with minimal summary column size is transferred to PREWHERE. + * Otherwise any condition with minimal summary column size can be transferred to PREWHERE. */ class MergeTreeWhereOptimizer : private boost::noncopyable { @@ -36,31 +31,59 @@ public: SelectQueryInfo & query_info, const Context & context, const MergeTreeData & data, - const Names & column_names, + const Names & queried_column_names, Poco::Logger * log); private: void optimize(ASTSelectQuery & select) const; + struct Condition + { + ASTPtr node; + UInt64 columns_size = 0; + NameSet identifiers; + bool viable = false; + bool good = false; + + auto tuple() const + { + return std::make_tuple(!viable, !good, columns_size); + } + + /// Is condition a better candidate for moving to PREWHERE? + bool operator< (const Condition & rhs) const + { + return tuple() < rhs.tuple(); + } + }; + + using Conditions = std::list; + + void analyzeImpl(Conditions & res, const ASTPtr & node) const; + + /// Transform conjunctions chain in WHERE expression to Conditions list. + Conditions analyze(const ASTPtr & expression) const; + + /// Transform Conditions list to WHERE or PREWHERE expression. + ASTPtr reconstruct(const Conditions & conditions) const; + void calculateColumnSizes(const MergeTreeData & data, const Names & column_names); void optimizeConjunction(ASTSelectQuery & select, ASTFunction * const fun) const; void optimizeArbitrary(ASTSelectQuery & select) const; - size_t getIdentifiersColumnSize(const IdentifierNameSet & identifiers) const; + UInt64 getIdentifiersColumnSize(const NameSet & identifiers) const; - bool isConditionGood(const IAST * condition) const; + bool isConditionGood(const ASTPtr & condition) const; - static void collectIdentifiersNoSubqueries(const IAST * const ast, IdentifierNameSet & set); + bool hasPrimaryKeyAtoms(const ASTPtr & ast) const; - bool hasPrimaryKeyAtoms(const IAST * ast) const; - - bool isPrimaryKeyAtom(const IAST * const ast) const; + bool isPrimaryKeyAtom(const ASTPtr & ast) const; bool isConstant(const ASTPtr & expr) const; - bool isSubsetOfTableColumns(const IdentifierNameSet & identifiers) const; + bool isSubsetOfTableColumns(const NameSet & identifiers) const; /** ARRAY JOIN'ed columns as well as arrayJoin() result cannot be used in PREWHERE, therefore expressions * containing said columns should not be moved to PREWHERE at all. @@ -72,14 +95,15 @@ private: void determineArrayJoinedNames(ASTSelectQuery & select); - using string_set_t = std::unordered_set; + using StringSet = std::unordered_set; - const string_set_t primary_key_columns; - const string_set_t table_columns; + String first_primary_key_column; + const StringSet table_columns; + const Names queried_columns; const Block block_with_constants; Poco::Logger * log; - std::unordered_map column_sizes{}; - size_t total_column_size{}; + std::unordered_map column_sizes; + UInt64 total_size_of_queried_columns = 0; NameSet array_joined_names; }; diff --git a/dbms/tests/queries/0_stateless/00712_nan_comparison.reference b/dbms/tests/queries/0_stateless/00712_nan_comparison.reference index 2129ac0b80f..1631046f6dc 100644 --- a/dbms/tests/queries/0_stateless/00712_nan_comparison.reference +++ b/dbms/tests/queries/0_stateless/00712_nan_comparison.reference @@ -18,11 +18,6 @@ 0 1 0 0 0 0 0 1 0 0 0 0 0 1 0 0 0 0 -0 1 0 1 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan diff --git a/dbms/tests/queries/0_stateless/00712_nan_comparison.sql b/dbms/tests/queries/0_stateless/00712_nan_comparison.sql index b8fd06aa0ff..a1eee82ae29 100644 --- a/dbms/tests/queries/0_stateless/00712_nan_comparison.sql +++ b/dbms/tests/queries/0_stateless/00712_nan_comparison.sql @@ -20,12 +20,6 @@ SELECT -nan = toInt64(0), -nan != toInt64(0), -nan < toInt64(0), -nan > toInt64( SELECT -nan = toFloat32(0.0), -nan != toFloat32(0.0), -nan < toFloat32(0.0), -nan > toFloat32(0.0), -nan <= toFloat32(0.0), -nan >= toFloat32(0.0); SELECT -nan = toFloat64(0.0), -nan != toFloat64(0.0), -nan < toFloat64(0.0), -nan > toFloat64(0.0), -nan <= toFloat64(0.0), -nan >= toFloat64(0.0); -SELECT nan = nan, nan != nan, nan = -nan, nan != -nan; -SELECT nan < nan, nan <= nan, nan < -nan, nan <= -nan; -SELECT nan > nan, nan >= nan, nan > -nan, nan >= -nan; -SELECT -nan < -nan, -nan <= -nan, -nan < nan, -nan <= nan; -SELECT -nan > -nan, -nan >= -nan, -nan > nan, -nan >= nan; - --SELECT 1 % nan, nan % 1, pow(x, 1), pow(1, x); -- TODO SELECT 1 + nan, 1 - nan, nan - 1, 1 * nan, 1 / nan, nan / 1; SELECT nan AS x, exp(x), exp2(x), exp10(x), log(x), log2(x), log10(x), sqrt(x), cbrt(x); diff --git a/dbms/tests/queries/0_stateless/00911_tautological_compare.reference b/dbms/tests/queries/0_stateless/00911_tautological_compare.reference new file mode 100644 index 00000000000..405d3348775 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00911_tautological_compare.reference @@ -0,0 +1,8 @@ +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/dbms/tests/queries/0_stateless/00911_tautological_compare.sql b/dbms/tests/queries/0_stateless/00911_tautological_compare.sql new file mode 100644 index 00000000000..34c95d73716 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00911_tautological_compare.sql @@ -0,0 +1,10 @@ +SELECT count() FROM system.numbers WHERE number != number; +SELECT count() FROM system.numbers WHERE number < number; +SELECT count() FROM system.numbers WHERE number > number; + +SELECT count() FROM system.numbers WHERE NOT (number = number); +SELECT count() FROM system.numbers WHERE NOT (number <= number); +SELECT count() FROM system.numbers WHERE NOT (number >= number); + +SELECT count() FROM system.numbers WHERE SHA256(toString(number)) != SHA256(toString(number)); +SELECT count() FROM system.numbers WHERE SHA256(toString(number)) != SHA256(toString(number)) AND rand() > 10; diff --git a/dbms/tests/queries/1_stateful/00091_prewhere_two_conditions.reference b/dbms/tests/queries/1_stateful/00091_prewhere_two_conditions.reference new file mode 100644 index 00000000000..5f884bdcd11 --- /dev/null +++ b/dbms/tests/queries/1_stateful/00091_prewhere_two_conditions.reference @@ -0,0 +1,4 @@ +417791 +417791 +0 +0 diff --git a/dbms/tests/queries/1_stateful/00091_prewhere_two_conditions.sql b/dbms/tests/queries/1_stateful/00091_prewhere_two_conditions.sql new file mode 100644 index 00000000000..cc660ed3f24 --- /dev/null +++ b/dbms/tests/queries/1_stateful/00091_prewhere_two_conditions.sql @@ -0,0 +1,13 @@ +SET max_bytes_to_read = 200000000; + +SET optimize_move_to_prewhere = 1; + +SELECT uniq(URL) FROM test.hits WHERE EventTime >= '2014-03-20 00:00:00' AND EventTime < '2014-03-21 00:00:00'; +SELECT uniq(URL) FROM test.hits WHERE EventTime >= '2014-03-20 00:00:00' AND URL != '' AND EventTime < '2014-03-21 00:00:00'; +SELECT uniq(*) FROM test.hits WHERE EventTime >= '2014-03-20 00:00:00' AND EventTime < '2014-03-21 00:00:00' AND EventDate = '2014-03-21'; +WITH EventTime AS xyz SELECT uniq(*) FROM test.hits WHERE xyz >= '2014-03-20 00:00:00' AND xyz < '2014-03-21 00:00:00' AND EventDate = '2014-03-21'; + +SET optimize_move_to_prewhere = 0; + +SELECT uniq(URL) FROM test.hits WHERE EventTime >= '2014-03-20 00:00:00' AND EventTime < '2014-03-21 00:00:00'; -- { serverError 307 } +SELECT uniq(URL) FROM test.hits WHERE EventTime >= '2014-03-20 00:00:00' AND URL != '' AND EventTime < '2014-03-21 00:00:00'; -- { serverError 307 }