Revert "Merge pull request #65298 from Algunenano/low_cardinality_wtf"

This reverts commit d3a269c61d, reversing
changes made to 023cf118dc.
This commit is contained in:
Raúl Marín 2024-07-23 19:11:22 +02:00
parent 3df8faf640
commit 73ab9a1197
8 changed files with 56 additions and 265 deletions

View File

@ -4124,9 +4124,7 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo
auto * column_to_interpolate = interpolate_node_typed.getExpression()->as<IdentifierNode>();
if (!column_to_interpolate)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"INTERPOLATE can work only for identifiers, but {} is found",
throw Exception(ErrorCodes::LOGICAL_ERROR, "INTERPOLATE can work only for indentifiers, but {} is found",
interpolate_node_typed.getExpression()->formatASTForErrorMessage());
auto column_to_interpolate_name = column_to_interpolate->getIdentifier().getFullName();

View File

@ -47,63 +47,32 @@ bool allArgumentsAreConstants(const ColumnsWithTypeAndName & args)
return true;
}
/// Replaces single low cardinality column in a function call by its dictionary
/// This can only happen after the arguments have been adapted in IFunctionOverloadResolver::getReturnType
/// as it's only possible if there is one low cardinality column and, optionally, const columns
ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
ColumnsWithTypeAndName & args, bool can_be_executed_on_default_arguments, size_t input_rows_count)
{
/// We return the LC indexes so the LC can be reconstructed with the function result
size_t num_rows = input_rows_count;
ColumnPtr indexes;
size_t number_low_cardinality_columns = 0;
size_t last_low_cardinality = 0;
size_t number_const_columns = 0;
size_t number_full_columns = 0;
for (size_t i = 0; i < args.size(); i++)
/// Find first LowCardinality column and replace it to nested dictionary.
for (auto & column : args)
{
auto const & arg = args[i];
if (checkAndGetColumn<ColumnLowCardinality>(arg.column.get()))
if (const auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(column.column.get()))
{
number_low_cardinality_columns++;
last_low_cardinality = i;
}
else if (checkAndGetColumn<ColumnConst>(arg.column.get()))
number_const_columns++;
else
number_full_columns++;
}
/// Single LowCardinality column is supported now.
if (indexes)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected single dictionary argument for function.");
if (!number_low_cardinality_columns && !number_const_columns)
return nullptr;
const auto * low_cardinality_type = checkAndGetDataType<DataTypeLowCardinality>(column.type.get());
if (number_full_columns > 0 || number_low_cardinality_columns > 1)
{
/// This should not be possible but currently there are multiple tests in CI failing because of it
/// TODO: Fix those cases, then enable this exception
#if 0
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected low cardinality types found. Low cardinality: {}. Full {}. Const {}",
number_low_cardinality_columns, number_full_columns, number_const_columns);
#else
return nullptr;
#endif
}
else if (number_low_cardinality_columns == 1)
{
auto & lc_arg = args[last_low_cardinality];
const auto * low_cardinality_type = checkAndGetDataType<DataTypeLowCardinality>(lc_arg.type.get());
if (!low_cardinality_type)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incompatible type for LowCardinality column: {}", lc_arg.type->getName());
const auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(lc_arg.column.get());
chassert(low_cardinality_column);
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Incompatible type for LowCardinality column: {}",
column.type->getName());
if (can_be_executed_on_default_arguments)
{
/// Normal case, when function can be executed on values' default.
lc_arg.column = low_cardinality_column->getDictionary().getNestedColumn();
column.column = low_cardinality_column->getDictionary().getNestedColumn();
indexes = low_cardinality_column->getIndexesPtr();
}
else
@ -111,21 +80,21 @@ ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
/// Special case when default value can't be used. Example: 1 % LowCardinality(Int).
/// LowCardinality always contains default, so 1 % 0 will throw exception in normal case.
auto dict_encoded = low_cardinality_column->getMinimalDictionaryEncodedColumn(0, low_cardinality_column->size());
lc_arg.column = dict_encoded.dictionary;
column.column = dict_encoded.dictionary;
indexes = dict_encoded.indexes;
}
/// The new column will have a different number of rows, normally less but occasionally it might be more (NULL)
input_rows_count = lc_arg.column->size();
lc_arg.type = low_cardinality_type->getDictionaryType();
num_rows = column.column->size();
column.type = low_cardinality_type->getDictionaryType();
}
}
/// Change size of constants
/// Change size of constants.
for (auto & column : args)
{
if (const auto * column_const = checkAndGetColumn<ColumnConst>(column.column.get()))
{
column.column = ColumnConst::create(recursiveRemoveLowCardinality(column_const->getDataColumnPtr()), input_rows_count);
column.column = ColumnConst::create(recursiveRemoveLowCardinality(column_const->getDataColumnPtr()), num_rows);
column.type = recursiveRemoveLowCardinality(column.type);
}
}
@ -301,8 +270,6 @@ ColumnPtr IExecutableFunction::executeWithoutSparseColumns(const ColumnsWithType
bool can_be_executed_on_default_arguments = canBeExecutedOnDefaultArguments();
const auto & dictionary_type = res_low_cardinality_type->getDictionaryType();
/// The arguments should have been adapted in IFunctionOverloadResolver::getReturnType
/// So there is only one low cardinality column (and optionally some const columns) and no full column
ColumnPtr indexes = replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
columns_without_low_cardinality, can_be_executed_on_default_arguments, input_rows_count);

View File

@ -745,12 +745,7 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
{
auto & interpolate_node_typed = interpolate_node->as<InterpolateNode &>();
PlannerActionsVisitor planner_actions_visitor(
planner_context,
/* use_column_identifier_as_action_node_name_, (default value)*/ true,
/// Prefer the INPUT to CONSTANT nodes (actions must be non constant)
/* always_use_const_column_for_constant_nodes */ false);
PlannerActionsVisitor planner_actions_visitor(planner_context);
auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag,
interpolate_node_typed.getExpression());
if (expression_to_interpolate_expression_nodes.size() != 1)

View File

@ -487,33 +487,16 @@ public:
return node;
}
[[nodiscard]] String addConstantIfNecessary(
const std::string & node_name, const ColumnWithTypeAndName & column, bool always_use_const_column_for_constant_nodes)
const ActionsDAG::Node * addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column)
{
chassert(column.column != nullptr);
auto it = node_name_to_node.find(node_name);
if (it != node_name_to_node.end() && (!always_use_const_column_for_constant_nodes || it->second->column))
return {node_name};
if (it != node_name_to_node.end())
{
/// There is a node with this name, but it doesn't have a column
/// This likely happens because we executed the query until WithMergeableState with a const node in the
/// WHERE clause and, as the results of headers are materialized, the column was removed
/// Let's add a new column and keep this
String dupped_name{node_name + "_dupped"};
if (node_name_to_node.find(dupped_name) != node_name_to_node.end())
return dupped_name;
const auto * node = &actions_dag.addColumn(column);
node_name_to_node[dupped_name] = node;
return dupped_name;
}
return it->second;
const auto * node = &actions_dag.addColumn(column);
node_name_to_node[node->result_name] = node;
return {node_name};
return node;
}
template <typename FunctionOrOverloadResolver>
@ -542,7 +525,7 @@ public:
}
private:
std::unordered_map<String, const ActionsDAG::Node *> node_name_to_node;
std::unordered_map<std::string_view, const ActionsDAG::Node *> node_name_to_node;
ActionsDAG & actions_dag;
QueryTreeNodePtr scope_node;
};
@ -550,11 +533,9 @@ private:
class PlannerActionsVisitorImpl
{
public:
PlannerActionsVisitorImpl(
ActionsDAG & actions_dag,
PlannerActionsVisitorImpl(ActionsDAG & actions_dag,
const PlannerContextPtr & planner_context_,
bool use_column_identifier_as_action_node_name_,
bool always_use_const_column_for_constant_nodes_);
bool use_column_identifier_as_action_node_name_);
ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node);
@ -614,18 +595,14 @@ private:
const PlannerContextPtr planner_context;
ActionNodeNameHelper action_node_name_helper;
bool use_column_identifier_as_action_node_name;
bool always_use_const_column_for_constant_nodes;
};
PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(
ActionsDAG & actions_dag,
PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAG & actions_dag,
const PlannerContextPtr & planner_context_,
bool use_column_identifier_as_action_node_name_,
bool always_use_const_column_for_constant_nodes_)
bool use_column_identifier_as_action_node_name_)
: planner_context(planner_context_)
, action_node_name_helper(node_to_node_name, *planner_context, use_column_identifier_as_action_node_name_)
, use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_)
, always_use_const_column_for_constant_nodes(always_use_const_column_for_constant_nodes_)
{
actions_stack.emplace_back(actions_dag, nullptr);
}
@ -748,16 +725,17 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
column.type = constant_type;
column.column = column.type->createColumnConst(1, constant_literal);
String final_name = actions_stack[0].addConstantIfNecessary(constant_node_name, column, always_use_const_column_for_constant_nodes);
actions_stack[0].addConstantIfNecessary(constant_node_name, column);
size_t actions_stack_size = actions_stack.size();
for (size_t i = 1; i < actions_stack_size; ++i)
{
auto & actions_stack_node = actions_stack[i];
actions_stack_node.addInputConstantColumnIfNecessary(final_name, column);
actions_stack_node.addInputConstantColumnIfNecessary(constant_node_name, column);
}
return {final_name, Levels(0)};
return {constant_node_name, Levels(0)};
}
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitLambda(const QueryTreeNodePtr & node)
@ -886,16 +864,16 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma
else
column.column = std::move(column_set);
String final_name = actions_stack[0].addConstantIfNecessary(column.name, column, always_use_const_column_for_constant_nodes);
actions_stack[0].addConstantIfNecessary(column.name, column);
size_t actions_stack_size = actions_stack.size();
for (size_t i = 1; i < actions_stack_size; ++i)
{
auto & actions_stack_node = actions_stack[i];
actions_stack_node.addInputConstantColumnIfNecessary(final_name, column);
actions_stack_node.addInputConstantColumnIfNecessary(column.name, column);
}
return {final_name, Levels(0)};
return {column.name, Levels(0)};
}
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitIndexHintFunction(const QueryTreeNodePtr & node)
@ -1032,19 +1010,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
}
PlannerActionsVisitor::PlannerActionsVisitor(
const PlannerContextPtr & planner_context_,
bool use_column_identifier_as_action_node_name_,
bool always_use_const_column_for_constant_nodes_)
PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_)
: planner_context(planner_context_)
, use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_)
, always_use_const_column_for_constant_nodes(always_use_const_column_for_constant_nodes_)
{}
ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node)
{
PlannerActionsVisitorImpl actions_visitor_impl(
actions_dag, planner_context, use_column_identifier_as_action_node_name, always_use_const_column_for_constant_nodes);
PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context, use_column_identifier_as_action_node_name);
return actions_visitor_impl.visit(expression_node);
}

View File

@ -27,17 +27,11 @@ using PlannerContextPtr = std::shared_ptr<PlannerContext>;
* During actions build, there is special handling for following functions:
* 1. Aggregate functions are added in actions dag as INPUT nodes. Aggregate functions arguments are not added.
* 2. For function `in` and its variants, already collected sets from planner context are used.
* 3. When building actions that use CONSTANT nodes, by default we ignore pre-existing INPUTs if those don't have
* a column (a const column always has a column). This is for compatibility with previous headers. We disable this
* behaviour when we explicitly want to override CONSTANT nodes with the input (resolving InterpolateNode for example)
*/
class PlannerActionsVisitor
{
public:
explicit PlannerActionsVisitor(
const PlannerContextPtr & planner_context_,
bool use_column_identifier_as_action_node_name_ = true,
bool always_use_const_column_for_constant_nodes_ = true);
explicit PlannerActionsVisitor(const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_ = true);
/** Add actions necessary to calculate expression node into expression dag.
* Necessary actions are not added in actions dag output.
@ -48,7 +42,6 @@ public:
private:
const PlannerContextPtr planner_context;
bool use_column_identifier_as_action_node_name = true;
bool always_use_const_column_for_constant_nodes = true;
};
/** Calculate query tree expression node action dag name and add them into node to name map.

View File

@ -8,13 +8,13 @@
40
41
41
0
2 42
2 42
43
43
0
11
11

View File

@ -1,55 +0,0 @@
-- { echoOn }
SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8)))
FROM system.one
GROUP BY '666';
6.666.8
SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8)))
FROM remote('127.0.0.{1,1}', 'system.one')
GROUP BY '666';
6.666.8
-- https://github.com/ClickHouse/ClickHouse/issues/63006
SELECT
6,
concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a,
concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b
FROM system.one
GROUP BY toNullable(6)
WITH ROLLUP
WITH TOTALS;
6 World666666 \N
6 World666666 \N
6 World666666 \N
SELECT
6,
concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a,
concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b
FROM remote('127.0.0.1')
GROUP BY toNullable(6)
WITH ROLLUP
WITH TOTALS;
6 World666666 \N
6 World666666 \N
6 World666666 \N
-- { echoOn }
SELECT
'%',
tuple(concat('%', 1, toLowCardinality(toLowCardinality(toNullable(materialize(1)))), currentDatabase(), 101., toNullable(13), '%AS%id_02%', toNullable(toNullable(10)), toLowCardinality(toNullable(10)), 10, 10)),
(toDecimal128(99.67, 6), 36, 61, 14)
FROM dist_03174
WHERE dummy IN (0, '255')
GROUP BY
toNullable(13),
(99.67, 61, toLowCardinality(14));
% ('%11default10113%AS%id_02%10101010') (99.67,36,61,14)
-- { echoOn }
SELECT
38,
concat(position(concat(concat(position(concat(toUInt256(3)), 'ca', 2), 3), NULLIF(1, materialize(toLowCardinality(1)))), toLowCardinality(toNullable('ca'))), concat(NULLIF(1, 1), concat(3), toNullable(3)))
FROM set_index_not__fuzz_0
GROUP BY
toNullable(3),
concat(concat(CAST(NULL, 'Nullable(Int8)'), toNullable(3)))
FORMAT Null
SETTINGS max_threads = 1, allow_experimental_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 3, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1;

View File

@ -1,80 +0,0 @@
-- There are various tests that check that group by keys don't propagate into functions replacing const arguments
-- by full (empty) columns
DROP TABLE IF EXISTS dist_03174;
DROP TABLE IF EXISTS set_index_not__fuzz_0;
-- https://github.com/ClickHouse/ClickHouse/issues/63006
SET allow_experimental_analyzer=1;
-- { echoOn }
SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8)))
FROM system.one
GROUP BY '666';
SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8)))
FROM remote('127.0.0.{1,1}', 'system.one')
GROUP BY '666';
-- https://github.com/ClickHouse/ClickHouse/issues/63006
SELECT
6,
concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a,
concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b
FROM system.one
GROUP BY toNullable(6)
WITH ROLLUP
WITH TOTALS;
SELECT
6,
concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a,
concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b
FROM remote('127.0.0.1')
GROUP BY toNullable(6)
WITH ROLLUP
WITH TOTALS;
-- https://github.com/ClickHouse/ClickHouse/issues/64945
-- { echoOff }
CREATE TABLE dist_03174 AS system.one ENGINE = Distributed(test_cluster_two_shards, system, one, dummy);
-- { echoOn }
SELECT
'%',
tuple(concat('%', 1, toLowCardinality(toLowCardinality(toNullable(materialize(1)))), currentDatabase(), 101., toNullable(13), '%AS%id_02%', toNullable(toNullable(10)), toLowCardinality(toNullable(10)), 10, 10)),
(toDecimal128(99.67, 6), 36, 61, 14)
FROM dist_03174
WHERE dummy IN (0, '255')
GROUP BY
toNullable(13),
(99.67, 61, toLowCardinality(14));
-- Parallel replicas
-- { echoOff }
CREATE TABLE set_index_not__fuzz_0
(
`name` String,
`status` Enum8('alive' = 0, 'rip' = 1),
INDEX idx_status status TYPE set(2) GRANULARITY 1
)
ENGINE = MergeTree()
ORDER BY name;
INSERT INTO set_index_not__fuzz_0 SELECT * FROM generateRandom() LIMIT 10;
-- { echoOn }
SELECT
38,
concat(position(concat(concat(position(concat(toUInt256(3)), 'ca', 2), 3), NULLIF(1, materialize(toLowCardinality(1)))), toLowCardinality(toNullable('ca'))), concat(NULLIF(1, 1), concat(3), toNullable(3)))
FROM set_index_not__fuzz_0
GROUP BY
toNullable(3),
concat(concat(CAST(NULL, 'Nullable(Int8)'), toNullable(3)))
FORMAT Null
SETTINGS max_threads = 1, allow_experimental_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 3, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1;
-- { echoOff }
DROP TABLE IF EXISTS dist_03174;
DROP TABLE IF EXISTS set_index_not__fuzz_0;