mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Fix early constant folding for isNull/isNotNul and analyzer.
This commit is contained in:
parent
c6dda8291a
commit
c9ce1362a1
@ -3,6 +3,7 @@
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeVariant.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Core/Field.h>
|
||||
@ -174,4 +175,17 @@ DataTypePtr removeNullableOrLowCardinalityNullable(const DataTypePtr & type)
|
||||
|
||||
}
|
||||
|
||||
bool canContainNull(const IDataType & type)
|
||||
{
|
||||
if (type.isNullable() || type.isLowCardinalityNullable() || isDynamic(type))
|
||||
return true;
|
||||
|
||||
if (const auto * variant = typeid_cast<const DataTypeVariant *>(&type))
|
||||
for (const auto & elem : variant->getVariants())
|
||||
if (canContainNull(*elem))
|
||||
return true;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -62,4 +62,6 @@ DataTypePtr makeNullableOrLowCardinalityNullableSafe(const DataTypePtr & type);
|
||||
/// Nullable(T) -> T, LowCardinality(Nullable(T)) -> T
|
||||
DataTypePtr removeNullableOrLowCardinalityNullable(const DataTypePtr & type);
|
||||
|
||||
bool canContainNull(const IDataType & type);
|
||||
|
||||
}
|
||||
|
@ -29,6 +29,18 @@ public:
|
||||
return name;
|
||||
}
|
||||
|
||||
ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & elem = arguments[0];
|
||||
if (elem.type->onlyNull())
|
||||
return result_type->createColumnConst(1, UInt8(0));
|
||||
|
||||
if (canContainNull(*elem.type))
|
||||
return nullptr;
|
||||
|
||||
return result_type->createColumnConst(1, UInt8(1));
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
@ -31,6 +31,18 @@ public:
|
||||
return name;
|
||||
}
|
||||
|
||||
ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & elem = arguments[0];
|
||||
if (elem.type->onlyNull())
|
||||
return result_type->createColumnConst(1, UInt8(1));
|
||||
|
||||
if (canContainNull(*elem.type))
|
||||
return nullptr;
|
||||
|
||||
return result_type->createColumnConst(1, UInt8(0));
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -23,6 +24,15 @@ public:
|
||||
return name;
|
||||
}
|
||||
|
||||
ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & elem = arguments[0];
|
||||
if (elem.type->onlyNull() || canContainNull(*elem.type))
|
||||
return result_type->createColumnConst(1, UInt8(1));
|
||||
|
||||
return result_type->createColumnConst(1, UInt8(0));
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
bool useDefaultImplementationForNothing() const override { return false; }
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Planner/PlannerExpressionAnalysis.h>
|
||||
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/FilterDescription.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
@ -37,7 +38,7 @@ namespace
|
||||
* Actions before filter are added into into actions chain.
|
||||
* It is client responsibility to update filter analysis result if filter column must be removed after chain is finalized.
|
||||
*/
|
||||
FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_node,
|
||||
std::optional<FilterAnalysisResult> analyzeFilter(const QueryTreeNodePtr & filter_expression_node,
|
||||
const ColumnsWithTypeAndName & input_columns,
|
||||
const PlannerContextPtr & planner_context,
|
||||
ActionsChain & actions_chain)
|
||||
@ -45,7 +46,11 @@ FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_no
|
||||
FilterAnalysisResult result;
|
||||
|
||||
result.filter_actions = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context);
|
||||
result.filter_column_name = result.filter_actions->getOutputs().at(0)->result_name;
|
||||
const auto * output = result.filter_actions->getOutputs().at(0);
|
||||
if (output->column && ConstantFilterDescription(*output->column).always_true)
|
||||
return {};
|
||||
|
||||
result.filter_column_name = output->result_name;
|
||||
actions_chain.addStep(std::make_unique<ActionsChainStep>(result.filter_actions));
|
||||
|
||||
return result;
|
||||
|
@ -0,0 +1 @@
|
||||
ReadFromPreparedSource (Optimized trivial count)
|
@ -0,0 +1,30 @@
|
||||
CREATE TABLE checks
|
||||
(
|
||||
`pull_request_number` UInt32,
|
||||
`commit_sha` LowCardinality(String),
|
||||
`check_name` LowCardinality(String),
|
||||
`check_status` LowCardinality(String),
|
||||
`check_duration_ms` UInt64,
|
||||
`check_start_time` DateTime,
|
||||
`test_name` LowCardinality(String),
|
||||
`test_status` LowCardinality(String),
|
||||
`test_duration_ms` UInt64,
|
||||
`report_url` String,
|
||||
`pull_request_url` String,
|
||||
`commit_url` String,
|
||||
`task_url` String,
|
||||
`base_ref` String,
|
||||
`base_repo` String,
|
||||
`head_ref` String,
|
||||
`head_repo` String,
|
||||
`test_context_raw` String,
|
||||
`instance_type` LowCardinality(String),
|
||||
`instance_id` String,
|
||||
`date` Date MATERIALIZED toDate(check_start_time)
|
||||
)
|
||||
ENGINE = MergeTree ORDER BY (date, pull_request_number, commit_sha, check_name, test_name, check_start_time);
|
||||
|
||||
insert into checks select * from generateRandom() limit 1;
|
||||
|
||||
|
||||
select trimLeft(explain) from (explain SELECT count(1) FROM checks WHERE test_name IS NOT NULL SETTINGS allow_experimental_analyzer = 1, allow_experimental_parallel_reading_from_replicas = 0) where explain like '%ReadFromPreparedSource%';
|
Loading…
Reference in New Issue
Block a user