Merge pull request #4405 from yandex/move-to-prewhere-improvement

Move to prewhere improvement
This commit is contained in:
alexey-milovidov 2019-03-03 19:06:43 +03:00 committed by GitHub
commit 3924207040
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 281 additions and 244 deletions

View File

@ -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<Op<int, int>, EqualsOp<int, int>>
|| std::is_same_v<Op<int, int>, LessOrEqualsOp<int, int>>
|| std::is_same_v<Op<int, int>, GreaterOrEqualsOp<int, int>>)
{
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};

View File

@ -10,241 +10,222 @@
#include <Parsers/ASTSubquery.h>
#include <Parsers/formatAST.h>
#include <Interpreters/QueryNormalizer.h>
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
#include <DataTypes/NestedUtils.h>
#include <ext/scope_guard.h>
#include <ext/collection_cast.h>
#include <ext/map.h>
#include <memory>
#include <unordered_map>
#include <map>
#include <limits>
#include <cstddef>
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<std::unordered_set>(data.primary_key_columns)},
table_columns{ext::map<std::unordered_set>(data.getColumns().getAllPhysical(),
: table_columns{ext::map<std::unordered_set>(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<ASTSelectQuery &>(*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<const ASTSubquery *>(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<ASTFunction *>(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<ASTFunction>();
function->name = "and";
function->arguments = std::make_shared<ASTExpressionList>();
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<ASTFunction *>(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<size_t>::max();
/// { first: condition index, second: summary column size }
std::pair<size_t, size_t> lightest_good_condition{no_such_condition, no_such_condition};
std::pair<size_t, size_t> 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<ASTFunction *>(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<float>(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<float>(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<size_t>::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<const ASTFunction *>(condition);
const auto function = typeid_cast<const ASTFunction *>(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<const ASTSubquery *>(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<const ASTFunction *>(ast))
if (const auto func = typeid_cast<const ASTFunction *>(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<const ASTFunction *>(ast))
if (const auto func = typeid_cast<const ASTFunction *>(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<const ASTLiteral *>(expr.get()) ||
(block_with_constants.has(column_name) && block_with_constants.getByName(column_name).column->isColumnConst()))
if (typeid_cast<const ASTLiteral *>(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<const ASTFunction *>(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

View File

@ -17,17 +17,12 @@ class ASTSelectQuery;
class ASTFunction;
class MergeTreeData;
using IdentifierNameSet = std::set<std::string>;
/** 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<Condition>;
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<std::string>;
using StringSet = std::unordered_set<std::string>;
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<std::string, size_t> column_sizes{};
size_t total_column_size{};
std::unordered_map<std::string, UInt64> column_sizes;
UInt64 total_size_of_queried_columns = 0;
NameSet array_joined_names;
};

View File

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

View File

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

View File

@ -0,0 +1,8 @@
0
0
0
0
0
0
0
0

View File

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

View File

@ -0,0 +1,4 @@
417791
417791
0
0

View File

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