mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Transform single point
This commit is contained in:
parent
30bf5e6d26
commit
aa436a3cb1
@ -53,6 +53,7 @@ using ArrayJoinActionPtr = std::shared_ptr<ArrayJoinAction>;
|
|||||||
*/
|
*/
|
||||||
struct ExpressionAction
|
struct ExpressionAction
|
||||||
{
|
{
|
||||||
|
friend class KeyCondition;
|
||||||
private:
|
private:
|
||||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||||
public:
|
public:
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
#include <Interpreters/TreeRewriter.h>
|
#include <Interpreters/TreeRewriter.h>
|
||||||
#include <Interpreters/ExpressionAnalyzer.h>
|
#include <Interpreters/ExpressionAnalyzer.h>
|
||||||
#include <Interpreters/ExpressionActions.h>
|
#include <Interpreters/ExpressionActions.h>
|
||||||
|
#include <Interpreters/castColumn.h>
|
||||||
#include <Interpreters/misc.h>
|
#include <Interpreters/misc.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
#include <Functions/IFunction.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)
|
static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field)
|
||||||
{
|
{
|
||||||
/// Fallback for fields without block reference.
|
/// Fallback for fields without block reference.
|
||||||
@ -616,6 +640,70 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions(
|
|||||||
return found_transformation;
|
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(
|
bool KeyCondition::tryPrepareSetIndex(
|
||||||
const ASTs & args,
|
const ASTs & args,
|
||||||
const Context & context,
|
const Context & context,
|
||||||
@ -868,27 +956,47 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & cont
|
|||||||
key_arg_pos = 0;
|
key_arg_pos = 0;
|
||||||
is_set_const = true;
|
is_set_const = true;
|
||||||
}
|
}
|
||||||
else if (getConstant(args[1], block_with_constants, const_value, const_type)
|
else if (getConstant(args[1], block_with_constants, const_value, const_type))
|
||||||
&& isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))
|
|
||||||
{
|
{
|
||||||
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)
|
else if (getConstant(args[0], block_with_constants, const_value, const_type))
|
||||||
&& canConstantBeWrappedByMonotonicFunctions(args[0], key_column_num, key_expr_type, const_value, const_type))
|
|
||||||
{
|
{
|
||||||
key_arg_pos = 0;
|
if (isKeyPossiblyWrappedByMonotonicFunctions(args[1], context, key_column_num, key_expr_type, chain))
|
||||||
is_constant_transformed = true;
|
{
|
||||||
}
|
key_arg_pos = 1;
|
||||||
else if (getConstant(args[0], block_with_constants, const_value, const_type)
|
}
|
||||||
&& isKeyPossiblyWrappedByMonotonicFunctions(args[1], context, key_column_num, key_expr_type, chain))
|
else if (canConstantBeWrappedByMonotonicFunctions(args[1], key_column_num, key_expr_type, const_value, const_type))
|
||||||
{
|
{
|
||||||
key_arg_pos = 1;
|
key_arg_pos = 1;
|
||||||
}
|
is_constant_transformed = true;
|
||||||
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))
|
else if (
|
||||||
{
|
single_point && func_name == "equals"
|
||||||
key_arg_pos = 1;
|
&& canConstantBeWrappedByFunctions(args[1], key_column_num, key_expr_type, const_value, const_type))
|
||||||
is_constant_transformed = true;
|
{
|
||||||
|
key_arg_pos = 0;
|
||||||
|
is_constant_transformed = true;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
return false;
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
return false;
|
return false;
|
||||||
|
@ -402,6 +402,9 @@ private:
|
|||||||
Field & out_value,
|
Field & out_value,
|
||||||
DataTypePtr & out_type);
|
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
|
/// If it's possible to make an RPNElement
|
||||||
/// that will filter values (possibly tuples) by the content of 'prepared_set',
|
/// that will filter values (possibly tuples) by the content of 'prepared_set',
|
||||||
/// do it and return true.
|
/// do it and return true.
|
||||||
|
25
src/Storages/MergeTree/PartitionPruner.cpp
Normal file
25
src/Storages/MergeTree/PartitionPruner.cpp
Normal 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;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -9,6 +9,7 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
/// Pruning partitions in verbatim way using KeyCondition
|
||||||
class PartitionPruner
|
class PartitionPruner
|
||||||
{
|
{
|
||||||
private:
|
private:
|
||||||
@ -28,24 +29,7 @@ public:
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
bool canBePruned(DataPartPtr part)
|
bool 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;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool isUseless() const { return useless; }
|
bool isUseless() const { return useless; }
|
||||||
};
|
};
|
||||||
|
@ -85,6 +85,7 @@ SRCS(
|
|||||||
MergeTree/MergeType.cpp
|
MergeTree/MergeType.cpp
|
||||||
MergeTree/MergedBlockOutputStream.cpp
|
MergeTree/MergedBlockOutputStream.cpp
|
||||||
MergeTree/MergedColumnOnlyOutputStream.cpp
|
MergeTree/MergedColumnOnlyOutputStream.cpp
|
||||||
|
MergeTree/PartitionPruner.cpp
|
||||||
MergeTree/ReplicatedFetchList.cpp
|
MergeTree/ReplicatedFetchList.cpp
|
||||||
MergeTree/ReplicatedMergeTreeAddress.cpp
|
MergeTree/ReplicatedMergeTreeAddress.cpp
|
||||||
MergeTree/ReplicatedMergeTreeAltersSequence.cpp
|
MergeTree/ReplicatedMergeTreeAltersSequence.cpp
|
||||||
|
@ -1 +1,2 @@
|
|||||||
1 2
|
1 2
|
||||||
|
1 2
|
||||||
|
@ -8,4 +8,7 @@ SET max_rows_to_read = 1;
|
|||||||
|
|
||||||
select * from xy where intHash64(x) % 100 = intHash64(1) % 100;
|
select * from xy where intHash64(x) % 100 = intHash64(1) % 100;
|
||||||
|
|
||||||
|
-- This works too
|
||||||
|
select * from xy where x = 1;
|
||||||
|
|
||||||
drop table if exists xy;
|
drop table if exists xy;
|
||||||
|
Loading…
Reference in New Issue
Block a user