ClickHouse/src/Interpreters/MonotonicityCheckVisitor.h

146 lines
4.5 KiB
C++
Raw Normal View History

2020-07-15 10:10:21 +00:00
#pragma once
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <DataTypes/DataTypeFactory.h>
#include <Functions/FunctionFactory.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/IAST.h>
#include <Common/typeid_cast.h>
namespace DB
{
using Monotonicity = IFunctionBase::Monotonicity;
/// Checks from bottom to top if function composition is monotonous
class MonotonicityCheckMatcher
{
public:
struct Data
{
const TablesWithColumns & tables;
const Context & context;
const std::unordered_set<String> & group_by_function_hashes;
Monotonicity monotonicity{true, true, true};
ASTIdentifier * identifier = nullptr;
DataTypePtr arg_data_type = {};
void reject() { monotonicity.is_monotonic = false; }
bool isRejected() const { return !monotonicity.is_monotonic; }
bool canOptimize(const ASTFunction & ast_function) const
{
/// if GROUP BY contains the same function ORDER BY shouldn't be optimized
auto hash = ast_function.getTreeHash();
String key = toString(hash.first) + '_' + toString(hash.second);
if (group_by_function_hashes.count(key))
return false;
/// if ORDER BY contains aggregate function it shouldn't be optimized
if (AggregateFunctionFactory::instance().isAggregateFunctionName(ast_function.name))
return false;
return true;
}
bool extractIdentifierAndType(const ASTFunction & ast_function)
{
if (identifier)
return true;
identifier = ast_function.arguments->children[0]->as<ASTIdentifier>();
if (!identifier)
return false;
auto pos = IdentifierSemantic::getMembership(*identifier);
if (!pos)
pos = IdentifierSemantic::chooseTableColumnMatch(*identifier, tables, true);
if (!pos)
return false;
if (auto data_type_and_name = tables[*pos].columns.tryGetByName(identifier->shortName()))
{
arg_data_type = data_type_and_name->type;
return true;
}
return false;
}
};
static void visit(const ASTPtr & ast, Data & data)
{
if (const auto * ast_function = ast->as<ASTFunction>())
visit(*ast_function, data);
}
static void visit(const ASTFunction & ast_function, Data & data)
{
if (data.isRejected())
return;
2020-08-08 01:01:47 +00:00
/// TODO: monotonicity for functions of several arguments
2020-07-15 10:10:21 +00:00
auto arguments = ast_function.arguments;
if (arguments->children.size() != 1)
{
data.reject();
return;
}
if (!data.canOptimize(ast_function))
{
data.reject();
return;
}
const auto & function = FunctionFactory::instance().tryGet(ast_function.name, data.context);
if (!function)
{
data.reject();
return;
}
/// First time extract the most enclosed identifier and its data type
if (!data.arg_data_type && !data.extractIdentifierAndType(ast_function))
{
data.reject();
return;
}
ColumnsWithTypeAndName args;
args.emplace_back(data.arg_data_type, "tmp");
auto function_base = function->build(args);
if (function_base && function_base->hasInformationAboutMonotonicity())
{
bool is_positive = data.monotonicity.is_positive;
data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, Field(), Field());
if (!is_positive)
data.monotonicity.is_positive = !data.monotonicity.is_positive;
2020-10-19 18:37:44 +00:00
data.arg_data_type = function_base->getResultType();
2020-07-15 10:10:21 +00:00
}
else
data.reject();
}
static bool needChildVisit(const ASTPtr & parent, const ASTPtr &)
2020-07-15 10:10:21 +00:00
{
if (const auto * func = typeid_cast<const ASTFunction *>(parent.get()))
return func->children.size() < 2;
2020-07-15 10:10:21 +00:00
return true;
}
};
using MonotonicityCheckVisitor = ConstInDepthNodeVisitor<MonotonicityCheckMatcher, false>;
}