mirror of https://github.com/ByConity/ByConity
Merge 'lc_null_not_in' into 'cnch-dev'
fix(clickhousech@m-5382919052): 【CP】fix some bugs for lc See merge request: !25981
This commit is contained in:
parent
76f29c9ec1
commit
10552bf704
|
@ -3,12 +3,10 @@
|
||||||
#include <Functions/FunctionHelpers.h>
|
#include <Functions/FunctionHelpers.h>
|
||||||
#include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <DataTypes/DataTypeLowCardinality.h>
|
|
||||||
#include <Columns/ColumnConst.h>
|
#include <Columns/ColumnConst.h>
|
||||||
#include <Columns/ColumnsNumber.h>
|
#include <Columns/ColumnsNumber.h>
|
||||||
#include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
#include <Columns/ColumnSet.h>
|
#include <Columns/ColumnSet.h>
|
||||||
#include <Columns/ColumnLowCardinality.h>
|
|
||||||
#include <Interpreters/Set.h>
|
#include <Interpreters/Set.h>
|
||||||
|
|
||||||
|
|
||||||
|
@ -69,12 +67,6 @@ public:
|
||||||
return 2;
|
return 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Do not use default implementation for LowCardinality.
|
|
||||||
/// For now, Set may be const or non const column, depending on how it was created.
|
|
||||||
/// But we will return UInt8 for any case.
|
|
||||||
/// TODO: we could use special implementation later.
|
|
||||||
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
|
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
|
||||||
{
|
{
|
||||||
return std::make_shared<DataTypeUInt8>();
|
return std::make_shared<DataTypeUInt8>();
|
||||||
|
@ -138,35 +130,16 @@ public:
|
||||||
else
|
else
|
||||||
columns_of_key_columns.insert(left_arg);
|
columns_of_key_columns.insert(left_arg);
|
||||||
|
|
||||||
/// Replace single LowCardinality column to it's dictionary if possible.
|
|
||||||
ColumnPtr lc_indexes = nullptr;
|
|
||||||
if (columns_of_key_columns.columns() == 1)
|
if (columns_of_key_columns.columns() == 1)
|
||||||
{
|
{
|
||||||
auto & arg = columns_of_key_columns.safeGetByPosition(0);
|
auto & arg = columns_of_key_columns.safeGetByPosition(0);
|
||||||
const auto * col = arg.column.get();
|
const auto * col = arg.column.get();
|
||||||
if (const auto * const_col = typeid_cast<const ColumnConst *>(col))
|
if (const auto * const_col = typeid_cast<const ColumnConst *>(col))
|
||||||
col = &const_col->getDataColumn();
|
col = &const_col->getDataColumn();
|
||||||
|
|
||||||
if (const auto * lc = typeid_cast<const ColumnLowCardinality *>(col))
|
|
||||||
{
|
|
||||||
if (!lc->isFullState())
|
|
||||||
{
|
|
||||||
lc_indexes = lc->getIndexesPtr();
|
|
||||||
arg.column = lc->getDictionary().getNestedColumn();
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
arg.column = lc->getNestedColumnPtr();
|
|
||||||
}
|
|
||||||
arg.type = removeLowCardinality(arg.type);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
auto res = set->execute(columns_of_key_columns, negative);
|
auto res = set->execute(columns_of_key_columns, negative);
|
||||||
|
|
||||||
if (lc_indexes)
|
|
||||||
return res->index(*lc_indexes, 0);
|
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
|
@ -37,9 +37,10 @@
|
||||||
#include <DataTypes/DataTypeString.h>
|
#include <DataTypes/DataTypeString.h>
|
||||||
#include <Functions/FunctionsConversion.h>
|
#include <Functions/FunctionsConversion.h>
|
||||||
|
|
||||||
|
#include <Interpreters/Aggregator.h>
|
||||||
#include <Interpreters/ArrayJoinAction.h>
|
#include <Interpreters/ArrayJoinAction.h>
|
||||||
#include <Interpreters/Context.h>
|
|
||||||
#include <Interpreters/ConcurrentHashJoin.h>
|
#include <Interpreters/ConcurrentHashJoin.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
#include <Interpreters/DictionaryReader.h>
|
#include <Interpreters/DictionaryReader.h>
|
||||||
#include <Interpreters/ExpressionActions.h>
|
#include <Interpreters/ExpressionActions.h>
|
||||||
#include <Interpreters/ExpressionAnalyzer.h>
|
#include <Interpreters/ExpressionAnalyzer.h>
|
||||||
|
@ -54,6 +55,7 @@
|
||||||
#include <Interpreters/evaluateConstantExpression.h>
|
#include <Interpreters/evaluateConstantExpression.h>
|
||||||
#include <Interpreters/replaceForPositionalArguments.h>
|
#include <Interpreters/replaceForPositionalArguments.h>
|
||||||
|
|
||||||
|
#include <QueryPlan/AggregatingStep.h>
|
||||||
#include <QueryPlan/ExpressionStep.h>
|
#include <QueryPlan/ExpressionStep.h>
|
||||||
|
|
||||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
|
@ -2110,7 +2112,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||||
bool finalized = false;
|
bool finalized = false;
|
||||||
size_t where_step_num = 0;
|
size_t where_step_num = 0;
|
||||||
|
|
||||||
auto finalize_chain = [&](ExpressionActionsChain & chain) {
|
auto finalize_chain = [&](ExpressionActionsChain & chain) -> ColumnsWithTypeAndName {
|
||||||
chain.finalize();
|
chain.finalize();
|
||||||
|
|
||||||
if (!finalized)
|
if (!finalized)
|
||||||
|
@ -2119,9 +2121,9 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||||
finalized = true;
|
finalized = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
// fmt::print("After finallize ---------- \n{}\n", chain.dumpChain());
|
auto res = chain.getLastStep().getResultColumns();
|
||||||
|
|
||||||
chain.clear();
|
chain.clear();
|
||||||
|
return res;
|
||||||
};
|
};
|
||||||
|
|
||||||
if (storage)
|
if (storage)
|
||||||
|
@ -2270,7 +2272,56 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||||
query_analyzer.appendAggregateFunctionsArguments(chain, only_types || !first_stage);
|
query_analyzer.appendAggregateFunctionsArguments(chain, only_types || !first_stage);
|
||||||
before_aggregation = chain.getLastActions();
|
before_aggregation = chain.getLastActions();
|
||||||
|
|
||||||
finalize_chain(chain);
|
auto columns_before_aggregation = finalize_chain(chain);
|
||||||
|
|
||||||
|
/// Here we want to check that columns after aggregation have the same type as
|
||||||
|
/// were promised in query_analyzer.aggregated_columns
|
||||||
|
/// Ideally, they should be equal. In practice, this may be not true.
|
||||||
|
/// As an example, we don't build sets for IN inside ExpressionAnalysis::analyzeAggregation,
|
||||||
|
/// so that constant folding for expression (1 in 1) will not work. This may change the return type
|
||||||
|
/// for functions with LowCardinality argument: function "substr(toLowCardinality('abc'), 1 IN 1)"
|
||||||
|
/// should usually return LowCardinality(String) when (1 IN 1) is constant, but without built set
|
||||||
|
/// for (1 IN 1) constant is not propagated and "substr" returns String type.
|
||||||
|
/// See 02503_in_lc_const_args_bug.sql
|
||||||
|
///
|
||||||
|
/// As a temporary solution, we add converting actions to the next chain.
|
||||||
|
/// Hopefully, later we can
|
||||||
|
/// * use a new analyzer where this issue is absent
|
||||||
|
/// * or remove ExpressionActionsChain completely and re-implement its logic on top of the query plan
|
||||||
|
{
|
||||||
|
for (auto & col : columns_before_aggregation)
|
||||||
|
if (!col.column)
|
||||||
|
col.column = col.type->createColumn();
|
||||||
|
|
||||||
|
Block header_before_aggregation(std::move(columns_before_aggregation));
|
||||||
|
|
||||||
|
auto names = query_analyzer.aggregationKeys().getNames();
|
||||||
|
ColumnNumbers keys;
|
||||||
|
for (const auto & name : names)
|
||||||
|
keys.push_back(header_before_aggregation.getPositionByName(name));
|
||||||
|
const auto & aggregates = query_analyzer.aggregates();
|
||||||
|
|
||||||
|
bool has_grouping = query_analyzer.group_by_kind != GroupByKind::ORDINARY;
|
||||||
|
auto actual_header = Aggregator::Params::getHeader(header_before_aggregation, {}, keys, aggregates, /*final*/ true);
|
||||||
|
actual_header = AggregatingStep::appendGroupingColumn(std::move(actual_header), has_grouping);
|
||||||
|
|
||||||
|
Block expected_header;
|
||||||
|
for (const auto & expected : query_analyzer.aggregated_columns)
|
||||||
|
expected_header.insert(ColumnWithTypeAndName(expected.type, expected.name));
|
||||||
|
|
||||||
|
if (!blocksHaveEqualStructure(actual_header, expected_header))
|
||||||
|
{
|
||||||
|
auto converting = ActionsDAG::makeConvertingActions(
|
||||||
|
actual_header.getColumnsWithTypeAndName(),
|
||||||
|
expected_header.getColumnsWithTypeAndName(),
|
||||||
|
ActionsDAG::MatchColumnsMode::Name,
|
||||||
|
true);
|
||||||
|
|
||||||
|
auto & step = chain.lastStep(query_analyzer.aggregated_columns);
|
||||||
|
auto & actions = step.actions();
|
||||||
|
actions = ActionsDAG::merge(std::move(*actions), std::move(*converting));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
if (query_analyzer.appendHaving(chain, only_types || !second_stage))
|
if (query_analyzer.appendHaving(chain, only_types || !second_stage))
|
||||||
{
|
{
|
||||||
|
|
|
@ -196,6 +196,19 @@ appendGroupingColumns(Block block, const GroupingSetsParamsList & grouping_set_p
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Block generateOutputHeader(const Block & input_header)
|
||||||
|
{
|
||||||
|
return appendGroupingSetColumn(input_header);
|
||||||
|
}
|
||||||
|
|
||||||
|
Block AggregatingStep::appendGroupingColumn(Block block, bool has_grouping)
|
||||||
|
{
|
||||||
|
if (!has_grouping)
|
||||||
|
return block;
|
||||||
|
|
||||||
|
return generateOutputHeader(block);
|
||||||
|
}
|
||||||
|
|
||||||
Aggregator::Params
|
Aggregator::Params
|
||||||
AggregatingStep::createParams(Block header_before_aggregation, AggregateDescriptions aggregates, Names group_by_keys, bool overflow_row)
|
AggregatingStep::createParams(Block header_before_aggregation, AggregateDescriptions aggregates, Names group_by_keys, bool overflow_row)
|
||||||
{
|
{
|
||||||
|
|
|
@ -86,6 +86,7 @@ enum class AggregateStagePolicy : UInt8
|
||||||
};
|
};
|
||||||
|
|
||||||
Block appendGroupingSetColumn(Block header);
|
Block appendGroupingSetColumn(Block header);
|
||||||
|
Block generateOutputHeader(const Block & input_header);
|
||||||
|
|
||||||
void computeGroupingFunctions(
|
void computeGroupingFunctions(
|
||||||
QueryPipeline & pipeline,
|
QueryPipeline & pipeline,
|
||||||
|
@ -197,6 +198,8 @@ public:
|
||||||
bool streaming_for_cache_ = false,
|
bool streaming_for_cache_ = false,
|
||||||
PlanHints hints_ = {});
|
PlanHints hints_ = {});
|
||||||
|
|
||||||
|
static Block appendGroupingColumn(Block block, bool has_grouping);
|
||||||
|
|
||||||
String getName() const override { return "Aggregating"; }
|
String getName() const override { return "Aggregating"; }
|
||||||
|
|
||||||
Type getType() const override { return Type::Aggregating; }
|
Type getType() const override { return Type::Aggregating; }
|
||||||
|
|
|
@ -0,0 +1 @@
|
||||||
|
abc
|
|
@ -0,0 +1,2 @@
|
||||||
|
SELECT substr(toLowCardinality('abc'), 1 in 1) AS x GROUP BY x;
|
||||||
|
|
|
@ -0,0 +1,4 @@
|
||||||
|
pure nullable result:
|
||||||
|
qwe
|
||||||
|
wrapping in LC:
|
||||||
|
qwe
|
|
@ -0,0 +1,17 @@
|
||||||
|
-- https://github.com/ClickHouse/ClickHouse/issues/50570
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS tnul SYNC;
|
||||||
|
DROP TABLE IF EXISTS tlc SYNC;
|
||||||
|
|
||||||
|
CREATE TABLE tnul (lc Nullable(String)) ENGINE = CnchMergeTree ORDER BY tuple();
|
||||||
|
INSERT INTO tnul VALUES (NULL), ('qwe');
|
||||||
|
SELECT 'pure nullable result:';
|
||||||
|
SELECT lc FROM tnul WHERE notIn(lc, ('rty', 'uiop'));
|
||||||
|
DROP TABLE tnul SYNC;
|
||||||
|
|
||||||
|
|
||||||
|
CREATE TABLE tlc (lc LowCardinality(Nullable(String))) ENGINE = CnchMergeTree ORDER BY tuple();
|
||||||
|
INSERT INTO tlc VALUES (NULL), ('qwe');
|
||||||
|
SELECT 'wrapping in LC:';
|
||||||
|
SELECT lc FROM tlc WHERE notIn(lc, ('rty', 'uiop'));
|
||||||
|
DROP TABLE tlc SYNC;
|
Loading…
Reference in New Issue