mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Revert "Merge pull request #65298 from Algunenano/low_cardinality_wtf"
This reverts commitd3a269c61d
, reversing changes made to023cf118dc
.
This commit is contained in:
parent
3df8faf640
commit
73ab9a1197
@ -4124,9 +4124,7 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo
|
|||||||
|
|
||||||
auto * column_to_interpolate = interpolate_node_typed.getExpression()->as<IdentifierNode>();
|
auto * column_to_interpolate = interpolate_node_typed.getExpression()->as<IdentifierNode>();
|
||||||
if (!column_to_interpolate)
|
if (!column_to_interpolate)
|
||||||
throw Exception(
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "INTERPOLATE can work only for indentifiers, but {} is found",
|
||||||
ErrorCodes::LOGICAL_ERROR,
|
|
||||||
"INTERPOLATE can work only for identifiers, but {} is found",
|
|
||||||
interpolate_node_typed.getExpression()->formatASTForErrorMessage());
|
interpolate_node_typed.getExpression()->formatASTForErrorMessage());
|
||||||
auto column_to_interpolate_name = column_to_interpolate->getIdentifier().getFullName();
|
auto column_to_interpolate_name = column_to_interpolate->getIdentifier().getFullName();
|
||||||
|
|
||||||
|
@ -47,63 +47,32 @@ bool allArgumentsAreConstants(const ColumnsWithTypeAndName & args)
|
|||||||
return true;
|
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(
|
ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
|
||||||
ColumnsWithTypeAndName & args, bool can_be_executed_on_default_arguments, size_t input_rows_count)
|
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;
|
ColumnPtr indexes;
|
||||||
|
|
||||||
size_t number_low_cardinality_columns = 0;
|
/// Find first LowCardinality column and replace it to nested dictionary.
|
||||||
size_t last_low_cardinality = 0;
|
for (auto & column : args)
|
||||||
size_t number_const_columns = 0;
|
|
||||||
size_t number_full_columns = 0;
|
|
||||||
|
|
||||||
for (size_t i = 0; i < args.size(); i++)
|
|
||||||
{
|
{
|
||||||
auto const & arg = args[i];
|
if (const auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(column.column.get()))
|
||||||
if (checkAndGetColumn<ColumnLowCardinality>(arg.column.get()))
|
|
||||||
{
|
{
|
||||||
number_low_cardinality_columns++;
|
/// Single LowCardinality column is supported now.
|
||||||
last_low_cardinality = i;
|
if (indexes)
|
||||||
}
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected single dictionary argument for function.");
|
||||||
else if (checkAndGetColumn<ColumnConst>(arg.column.get()))
|
|
||||||
number_const_columns++;
|
|
||||||
else
|
|
||||||
number_full_columns++;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!number_low_cardinality_columns && !number_const_columns)
|
const auto * low_cardinality_type = checkAndGetDataType<DataTypeLowCardinality>(column.type.get());
|
||||||
return nullptr;
|
|
||||||
|
|
||||||
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)
|
if (!low_cardinality_type)
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incompatible type for LowCardinality column: {}", lc_arg.type->getName());
|
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||||
|
"Incompatible type for LowCardinality column: {}",
|
||||||
const auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(lc_arg.column.get());
|
column.type->getName());
|
||||||
chassert(low_cardinality_column);
|
|
||||||
|
|
||||||
if (can_be_executed_on_default_arguments)
|
if (can_be_executed_on_default_arguments)
|
||||||
{
|
{
|
||||||
/// Normal case, when function can be executed on values' default.
|
/// 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();
|
indexes = low_cardinality_column->getIndexesPtr();
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
@ -111,21 +80,21 @@ ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
|
|||||||
/// Special case when default value can't be used. Example: 1 % LowCardinality(Int).
|
/// 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.
|
/// 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());
|
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;
|
indexes = dict_encoded.indexes;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// The new column will have a different number of rows, normally less but occasionally it might be more (NULL)
|
num_rows = column.column->size();
|
||||||
input_rows_count = lc_arg.column->size();
|
column.type = low_cardinality_type->getDictionaryType();
|
||||||
lc_arg.type = low_cardinality_type->getDictionaryType();
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Change size of constants
|
/// Change size of constants.
|
||||||
for (auto & column : args)
|
for (auto & column : args)
|
||||||
{
|
{
|
||||||
if (const auto * column_const = checkAndGetColumn<ColumnConst>(column.column.get()))
|
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);
|
column.type = recursiveRemoveLowCardinality(column.type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -301,8 +270,6 @@ ColumnPtr IExecutableFunction::executeWithoutSparseColumns(const ColumnsWithType
|
|||||||
bool can_be_executed_on_default_arguments = canBeExecutedOnDefaultArguments();
|
bool can_be_executed_on_default_arguments = canBeExecutedOnDefaultArguments();
|
||||||
|
|
||||||
const auto & dictionary_type = res_low_cardinality_type->getDictionaryType();
|
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(
|
ColumnPtr indexes = replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
|
||||||
columns_without_low_cardinality, can_be_executed_on_default_arguments, input_rows_count);
|
columns_without_low_cardinality, can_be_executed_on_default_arguments, input_rows_count);
|
||||||
|
|
||||||
|
@ -745,12 +745,7 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
|
|||||||
{
|
{
|
||||||
auto & interpolate_node_typed = interpolate_node->as<InterpolateNode &>();
|
auto & interpolate_node_typed = interpolate_node->as<InterpolateNode &>();
|
||||||
|
|
||||||
PlannerActionsVisitor planner_actions_visitor(
|
PlannerActionsVisitor planner_actions_visitor(planner_context);
|
||||||
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);
|
|
||||||
|
|
||||||
auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag,
|
auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag,
|
||||||
interpolate_node_typed.getExpression());
|
interpolate_node_typed.getExpression());
|
||||||
if (expression_to_interpolate_expression_nodes.size() != 1)
|
if (expression_to_interpolate_expression_nodes.size() != 1)
|
||||||
|
@ -487,33 +487,16 @@ public:
|
|||||||
return node;
|
return node;
|
||||||
}
|
}
|
||||||
|
|
||||||
[[nodiscard]] String addConstantIfNecessary(
|
const ActionsDAG::Node * addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column)
|
||||||
const std::string & node_name, const ColumnWithTypeAndName & column, bool always_use_const_column_for_constant_nodes)
|
|
||||||
{
|
{
|
||||||
chassert(column.column != nullptr);
|
|
||||||
auto it = node_name_to_node.find(node_name);
|
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())
|
if (it != node_name_to_node.end())
|
||||||
{
|
return it->second;
|
||||||
/// 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;
|
|
||||||
}
|
|
||||||
|
|
||||||
const auto * node = &actions_dag.addColumn(column);
|
const auto * node = &actions_dag.addColumn(column);
|
||||||
node_name_to_node[node->result_name] = node;
|
node_name_to_node[node->result_name] = node;
|
||||||
|
|
||||||
return {node_name};
|
return node;
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename FunctionOrOverloadResolver>
|
template <typename FunctionOrOverloadResolver>
|
||||||
@ -542,7 +525,7 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
private:
|
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;
|
ActionsDAG & actions_dag;
|
||||||
QueryTreeNodePtr scope_node;
|
QueryTreeNodePtr scope_node;
|
||||||
};
|
};
|
||||||
@ -550,11 +533,9 @@ private:
|
|||||||
class PlannerActionsVisitorImpl
|
class PlannerActionsVisitorImpl
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
PlannerActionsVisitorImpl(
|
PlannerActionsVisitorImpl(ActionsDAG & actions_dag,
|
||||||
ActionsDAG & actions_dag,
|
|
||||||
const PlannerContextPtr & planner_context_,
|
const PlannerContextPtr & planner_context_,
|
||||||
bool use_column_identifier_as_action_node_name_,
|
bool use_column_identifier_as_action_node_name_);
|
||||||
bool always_use_const_column_for_constant_nodes_);
|
|
||||||
|
|
||||||
ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node);
|
ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node);
|
||||||
|
|
||||||
@ -614,18 +595,14 @@ private:
|
|||||||
const PlannerContextPtr planner_context;
|
const PlannerContextPtr planner_context;
|
||||||
ActionNodeNameHelper action_node_name_helper;
|
ActionNodeNameHelper action_node_name_helper;
|
||||||
bool use_column_identifier_as_action_node_name;
|
bool use_column_identifier_as_action_node_name;
|
||||||
bool always_use_const_column_for_constant_nodes;
|
|
||||||
};
|
};
|
||||||
|
|
||||||
PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(
|
PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAG & actions_dag,
|
||||||
ActionsDAG & actions_dag,
|
|
||||||
const PlannerContextPtr & planner_context_,
|
const PlannerContextPtr & planner_context_,
|
||||||
bool use_column_identifier_as_action_node_name_,
|
bool use_column_identifier_as_action_node_name_)
|
||||||
bool always_use_const_column_for_constant_nodes_)
|
|
||||||
: planner_context(planner_context_)
|
: planner_context(planner_context_)
|
||||||
, action_node_name_helper(node_to_node_name, *planner_context, use_column_identifier_as_action_node_name_)
|
, 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_)
|
, 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);
|
actions_stack.emplace_back(actions_dag, nullptr);
|
||||||
}
|
}
|
||||||
@ -748,16 +725,17 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
|
|||||||
column.type = constant_type;
|
column.type = constant_type;
|
||||||
column.column = column.type->createColumnConst(1, constant_literal);
|
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();
|
size_t actions_stack_size = actions_stack.size();
|
||||||
for (size_t i = 1; i < actions_stack_size; ++i)
|
for (size_t i = 1; i < actions_stack_size; ++i)
|
||||||
{
|
{
|
||||||
auto & actions_stack_node = actions_stack[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)
|
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitLambda(const QueryTreeNodePtr & node)
|
||||||
@ -886,16 +864,16 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma
|
|||||||
else
|
else
|
||||||
column.column = std::move(column_set);
|
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();
|
size_t actions_stack_size = actions_stack.size();
|
||||||
for (size_t i = 1; i < actions_stack_size; ++i)
|
for (size_t i = 1; i < actions_stack_size; ++i)
|
||||||
{
|
{
|
||||||
auto & actions_stack_node = actions_stack[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)
|
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitIndexHintFunction(const QueryTreeNodePtr & node)
|
||||||
@ -1032,19 +1010,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
|
|||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
PlannerActionsVisitor::PlannerActionsVisitor(
|
PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_)
|
||||||
const PlannerContextPtr & planner_context_,
|
|
||||||
bool use_column_identifier_as_action_node_name_,
|
|
||||||
bool always_use_const_column_for_constant_nodes_)
|
|
||||||
: planner_context(planner_context_)
|
: planner_context(planner_context_)
|
||||||
, use_column_identifier_as_action_node_name(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_)
|
|
||||||
{}
|
{}
|
||||||
|
|
||||||
ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node)
|
ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node)
|
||||||
{
|
{
|
||||||
PlannerActionsVisitorImpl actions_visitor_impl(
|
PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context, use_column_identifier_as_action_node_name);
|
||||||
actions_dag, planner_context, use_column_identifier_as_action_node_name, always_use_const_column_for_constant_nodes);
|
|
||||||
return actions_visitor_impl.visit(expression_node);
|
return actions_visitor_impl.visit(expression_node);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -27,17 +27,11 @@ using PlannerContextPtr = std::shared_ptr<PlannerContext>;
|
|||||||
* During actions build, there is special handling for following functions:
|
* 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.
|
* 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.
|
* 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
|
class PlannerActionsVisitor
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
explicit PlannerActionsVisitor(
|
explicit PlannerActionsVisitor(const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_ = true);
|
||||||
const PlannerContextPtr & planner_context_,
|
|
||||||
bool use_column_identifier_as_action_node_name_ = true,
|
|
||||||
bool always_use_const_column_for_constant_nodes_ = true);
|
|
||||||
|
|
||||||
/** Add actions necessary to calculate expression node into expression dag.
|
/** Add actions necessary to calculate expression node into expression dag.
|
||||||
* Necessary actions are not added in actions dag output.
|
* Necessary actions are not added in actions dag output.
|
||||||
@ -48,7 +42,6 @@ public:
|
|||||||
private:
|
private:
|
||||||
const PlannerContextPtr planner_context;
|
const PlannerContextPtr planner_context;
|
||||||
bool use_column_identifier_as_action_node_name = true;
|
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.
|
/** Calculate query tree expression node action dag name and add them into node to name map.
|
||||||
|
@ -8,13 +8,13 @@
|
|||||||
40
|
40
|
||||||
41
|
41
|
||||||
|
|
||||||
41
|
0
|
||||||
2 42
|
2 42
|
||||||
|
|
||||||
2 42
|
2 42
|
||||||
43
|
43
|
||||||
|
|
||||||
43
|
0
|
||||||
11
|
11
|
||||||
|
|
||||||
11
|
11
|
||||||
|
@ -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;
|
|
@ -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;
|
|
Loading…
Reference in New Issue
Block a user