Transform single point

This commit is contained in:
Amos Bird 2020-11-06 11:50:58 +08:00
parent 30bf5e6d26
commit aa436a3cb1
No known key found for this signature in database
GPG Key ID: 80D430DCBECFEDB4
8 changed files with 162 additions and 36 deletions

View File

@ -53,6 +53,7 @@ using ArrayJoinActionPtr = std::shared_ptr<ArrayJoinAction>;
*/
struct ExpressionAction
{
friend class KeyCondition;
private:
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
public:

View File

@ -5,6 +5,7 @@
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/castColumn.h>
#include <Interpreters/misc.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
@ -485,6 +486,29 @@ static std::pair<Field, DataTypePtr> applyFunctionForFieldOfUnknownType(
}
/// Same as above but for binary operators
static std::pair<Field, DataTypePtr> applyBinaryFunctionForFieldOfUnknownType(
const FunctionOverloadResolverPtr & func,
const DataTypePtr & arg_type,
const Field & arg_value,
const DataTypePtr & arg_type2,
const Field & arg_value2)
{
ColumnsWithTypeAndName arguments{
{arg_type->createColumnConst(1, arg_value), arg_type, "x"}, {arg_type2->createColumnConst(1, arg_value2), arg_type2, "y"}};
FunctionBasePtr func_base = func->build(arguments);
DataTypePtr return_type = func_base->getResultType();
auto col = func_base->execute(arguments, return_type, 1);
Field result = (*col)[0];
return {std::move(result), std::move(return_type)};
}
static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field)
{
/// Fallback for fields without block reference.
@ -616,6 +640,70 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions(
return found_transformation;
}
/// Looking for possible transformation of `column = constant` into `partition_expr = function(constant)`
bool KeyCondition::canConstantBeWrappedByFunctions(
const ASTPtr & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type)
{
String expr_name = node->getColumnName();
const auto & sample_block = key_expr->getSampleBlock();
if (!sample_block.has(expr_name))
return false;
bool found_transformation = false;
auto input_column = sample_block.getByName(expr_name);
auto const_column = out_type->createColumnConst(1, out_value);
out_value = (*castColumn({const_column, out_type, "c"}, input_column.type))[0];
out_type = input_column.type;
Block transform({{input_column.type->createColumn(), input_column.type, input_column.name}});
for (const ExpressionAction & action : key_expr->getActions())
{
const auto & argument_names = action.argument_names;
if (action.type == ExpressionAction::Type::APPLY_FUNCTION)
{
if (!action.function_base->isDeterministic())
return false;
if (argument_names.size() == 1 && argument_names[0] == expr_name)
{
std::tie(out_value, out_type) = applyFunctionForFieldOfUnknownType(action.function_builder, out_type, out_value);
}
else if (argument_names.size() == 2)
{
if (!transform.has(argument_names[0]) || !transform.has(argument_names[1]))
return false;
auto left = transform.getByName(argument_names[0]);
auto right = transform.getByName(argument_names[1]);
if (isColumnConst(*left.column))
{
auto left_arg_type = left.type;
auto left_arg_value = (*left.column)[0];
std::tie(out_value, out_type) = applyBinaryFunctionForFieldOfUnknownType(
action.function_builder, left_arg_type, left_arg_value, out_type, out_value);
}
else if (isColumnConst(*right.column))
{
auto right_arg_type = right.type;
auto right_arg_value = (*right.column)[0];
std::tie(out_value, out_type) = applyBinaryFunctionForFieldOfUnknownType(
action.function_builder, out_type, out_value, right_arg_type, right_arg_value);
}
}
expr_name = action.result_name;
auto it = key_columns.find(expr_name);
if (key_columns.end() != it)
{
out_key_column_num = it->second;
out_key_column_type = sample_block.getByName(it->first).type;
found_transformation = true;
break;
}
}
action.execute(transform, true);
}
return found_transformation;
}
bool KeyCondition::tryPrepareSetIndex(
const ASTs & args,
const Context & context,
@ -868,27 +956,47 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & cont
key_arg_pos = 0;
is_set_const = true;
}
else if (getConstant(args[1], block_with_constants, const_value, const_type)
&& isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))
else if (getConstant(args[1], block_with_constants, const_value, const_type))
{
key_arg_pos = 0;
if (isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))
{
key_arg_pos = 0;
}
else if (canConstantBeWrappedByMonotonicFunctions(args[0], key_column_num, key_expr_type, const_value, const_type))
{
key_arg_pos = 0;
is_constant_transformed = true;
}
else if (
single_point && func_name == "equals"
&& canConstantBeWrappedByFunctions(args[0], key_column_num, key_expr_type, const_value, const_type))
{
key_arg_pos = 0;
is_constant_transformed = true;
}
else
return false;
}
else if (getConstant(args[1], block_with_constants, const_value, const_type)
&& canConstantBeWrappedByMonotonicFunctions(args[0], key_column_num, key_expr_type, const_value, const_type))
else if (getConstant(args[0], block_with_constants, const_value, const_type))
{
key_arg_pos = 0;
is_constant_transformed = true;
}
else if (getConstant(args[0], block_with_constants, const_value, const_type)
&& isKeyPossiblyWrappedByMonotonicFunctions(args[1], context, key_column_num, key_expr_type, chain))
{
key_arg_pos = 1;
}
else if (getConstant(args[0], block_with_constants, const_value, const_type)
&& canConstantBeWrappedByMonotonicFunctions(args[1], key_column_num, key_expr_type, const_value, const_type))
{
key_arg_pos = 1;
is_constant_transformed = true;
if (isKeyPossiblyWrappedByMonotonicFunctions(args[1], context, key_column_num, key_expr_type, chain))
{
key_arg_pos = 1;
}
else if (canConstantBeWrappedByMonotonicFunctions(args[1], key_column_num, key_expr_type, const_value, const_type))
{
key_arg_pos = 1;
is_constant_transformed = true;
}
else if (
single_point && func_name == "equals"
&& canConstantBeWrappedByFunctions(args[1], key_column_num, key_expr_type, const_value, const_type))
{
key_arg_pos = 0;
is_constant_transformed = true;
}
else
return false;
}
else
return false;

View File

@ -402,6 +402,9 @@ private:
Field & out_value,
DataTypePtr & out_type);
bool canConstantBeWrappedByFunctions(
const ASTPtr & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type);
/// If it's possible to make an RPNElement
/// that will filter values (possibly tuples) by the content of 'prepared_set',
/// do it and return true.

View File

@ -0,0 +1,25 @@
#include <Storages/MergeTree/PartitionPruner.h>
namespace DB
{
bool PartitionPruner::canBePruned(const DataPartPtr & part)
{
if (part->isEmpty())
return true;
const auto & partition_id = part->info.partition_id;
bool is_valid;
if (auto it = partition_filter_map.find(partition_id); it != partition_filter_map.end())
is_valid = it->second;
else
{
const auto & partition_value = part->partition.value;
std::vector<FieldRef> index_value(partition_value.begin(), partition_value.end());
is_valid = partition_condition.mayBeTrueInRange(
partition_value.size(), index_value.data(), index_value.data(), partition_key.data_types);
partition_filter_map.emplace(partition_id, is_valid);
}
return !is_valid;
}
}

View File

@ -9,6 +9,7 @@
namespace DB
{
/// Pruning partitions in verbatim way using KeyCondition
class PartitionPruner
{
private:
@ -28,24 +29,7 @@ public:
{
}
bool canBePruned(DataPartPtr part)
{
if (part->isEmpty())
return true;
const auto & partition_id = part->info.partition_id;
bool is_valid;
if (auto it = partition_filter_map.find(partition_id); it != partition_filter_map.end())
is_valid = it->second;
else
{
const auto & partition_value = part->partition.value;
std::vector<FieldRef> index_value(partition_value.begin(), partition_value.end());
is_valid = partition_condition.mayBeTrueInRange(
partition_value.size(), index_value.data(), index_value.data(), partition_key.data_types);
partition_filter_map.emplace(partition_id, is_valid);
}
return !is_valid;
}
bool canBePruned(const DataPartPtr & part);
bool isUseless() const { return useless; }
};

View File

@ -85,6 +85,7 @@ SRCS(
MergeTree/MergeType.cpp
MergeTree/MergedBlockOutputStream.cpp
MergeTree/MergedColumnOnlyOutputStream.cpp
MergeTree/PartitionPruner.cpp
MergeTree/ReplicatedFetchList.cpp
MergeTree/ReplicatedMergeTreeAddress.cpp
MergeTree/ReplicatedMergeTreeAltersSequence.cpp

View File

@ -8,4 +8,7 @@ SET max_rows_to_read = 1;
select * from xy where intHash64(x) % 100 = intHash64(1) % 100;
-- This works too
select * from xy where x = 1;
drop table if exists xy;