mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
fixes
This commit is contained in:
parent
61a01782a6
commit
a197511a96
@ -71,18 +71,9 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
|
||||
{
|
||||
auto type_without_low_cardinality = convertLowCardinalityTypesToNested(argument_types);
|
||||
|
||||
auto with_original_parameters = getImpl(name, type_without_low_cardinality, parameters, out_properties, false);
|
||||
/// If one of the types is Nullable, we apply aggregate function combinator "Null".
|
||||
|
||||
// If one of the types is Nullable, we apply aggregate function combinator "Null".
|
||||
// Pure window functions are not real aggregate functions. Applying
|
||||
// combinators doesn't make sense for them, they must handle the
|
||||
// nullability themselves. Another special case is functions from Nothing
|
||||
// that are rewritten to AggregateFunctionNothing, in this case
|
||||
// nested_function is nullptr.
|
||||
if (!(with_original_parameters
|
||||
&& with_original_parameters->isOnlyWindowFunction())
|
||||
&& std::any_of(type_without_low_cardinality.begin(),
|
||||
type_without_low_cardinality.end(),
|
||||
if (std::any_of(type_without_low_cardinality.begin(), type_without_low_cardinality.end(),
|
||||
[](const auto & type) { return type->isNullable(); }))
|
||||
{
|
||||
AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix("Null");
|
||||
@ -99,12 +90,23 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
|
||||
AggregateFunctionPtr nested_function = getImpl(
|
||||
name, nested_types, nested_parameters, out_properties, has_null_arguments);
|
||||
|
||||
return combinator->transformAggregateFunction(nested_function, out_properties, type_without_low_cardinality, parameters);
|
||||
// Pure window functions are not real aggregate functions. Applying
|
||||
// combinators doesn't make sense for them, they must handle the
|
||||
// nullability themselves. Another special case is functions from Nothing
|
||||
// that are rewritten to AggregateFunctionNothing, in this case
|
||||
// nested_function is nullptr.
|
||||
if (!nested_function || !nested_function->isOnlyWindowFunction())
|
||||
{
|
||||
return combinator->transformAggregateFunction(nested_function,
|
||||
out_properties, type_without_low_cardinality, parameters);
|
||||
}
|
||||
}
|
||||
|
||||
if (!with_original_parameters)
|
||||
auto with_original_arguments = getImpl(name, type_without_low_cardinality, parameters, out_properties, false);
|
||||
|
||||
if (!with_original_arguments)
|
||||
throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR);
|
||||
return with_original_parameters;
|
||||
return with_original_arguments;
|
||||
}
|
||||
|
||||
|
||||
|
@ -1475,12 +1475,21 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction
|
||||
return;
|
||||
}
|
||||
|
||||
if (!argument_types[0]->equals(*argument_types[2]))
|
||||
const auto supertype = getLeastSupertype({argument_types[0], argument_types[2]});
|
||||
if (!supertype)
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"The default value type '{}' is not the same as the argument type '{}'",
|
||||
argument_types[2]->getName(),
|
||||
argument_types[0]->getName());
|
||||
"There is no supertype for the argument type '{}' and the default value type '{}'",
|
||||
argument_types[0]->getName(),
|
||||
argument_types[2]->getName());
|
||||
}
|
||||
if (!argument_types[0]->equals(*supertype))
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"The supertype '{}' for the argument type '{}' and the default value type '{}' is not the same as the argument type",
|
||||
supertype->getName(),
|
||||
argument_types[0]->getName(),
|
||||
argument_types[2]->getName());
|
||||
}
|
||||
|
||||
if (argument_types.size() > 3)
|
||||
@ -1533,9 +1542,13 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction
|
||||
if (argument_types.size() > 2)
|
||||
{
|
||||
// Column with default values is specified.
|
||||
to.insertFrom(*current_block.input_columns[
|
||||
workspace.argument_column_indices[2]],
|
||||
transform->current_row.row);
|
||||
// The conversion through Field is inefficient, but we accept
|
||||
// subtypes of the argument type as a default value (for convenience),
|
||||
// and it's a pain to write conversion that respects ColumnNothing
|
||||
// and ColumnConst and so on.
|
||||
const IColumn & default_column = *current_block.input_columns[
|
||||
workspace.argument_column_indices[2]].get();
|
||||
to.insert(default_column[transform->current_row.row]);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1572,40 +1585,49 @@ void registerWindowFunctions(AggregateFunctionFactory & factory)
|
||||
// to a (rows between unbounded preceding and unbounded following) frame,
|
||||
// instead of adding separate logic for them.
|
||||
|
||||
factory.registerFunction("rank", [](const std::string & name,
|
||||
const AggregateFunctionProperties properties = {
|
||||
// By default, if an aggregate function has a null argument, it will be
|
||||
// replaced with AggregateFunctionNothing. We don't need this behavior
|
||||
// e.g. for lagInFrame(number, 1, null).
|
||||
.returns_default_when_only_null = true,
|
||||
// This probably doesn't make any difference for window functions because
|
||||
// it is an Aggregator-specific setting.
|
||||
.is_order_dependent = true };
|
||||
|
||||
factory.registerFunction("rank", {[](const std::string & name,
|
||||
const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||
{
|
||||
return std::make_shared<WindowFunctionRank>(name, argument_types,
|
||||
parameters);
|
||||
});
|
||||
}, properties});
|
||||
|
||||
factory.registerFunction("dense_rank", [](const std::string & name,
|
||||
factory.registerFunction("dense_rank", {[](const std::string & name,
|
||||
const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||
{
|
||||
return std::make_shared<WindowFunctionDenseRank>(name, argument_types,
|
||||
parameters);
|
||||
});
|
||||
}, properties});
|
||||
|
||||
factory.registerFunction("row_number", [](const std::string & name,
|
||||
factory.registerFunction("row_number", {[](const std::string & name,
|
||||
const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||
{
|
||||
return std::make_shared<WindowFunctionRowNumber>(name, argument_types,
|
||||
parameters);
|
||||
});
|
||||
}, properties});
|
||||
|
||||
factory.registerFunction("lagInFrame", [](const std::string & name,
|
||||
factory.registerFunction("lagInFrame", {[](const std::string & name,
|
||||
const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||
{
|
||||
return std::make_shared<WindowFunctionLagLeadInFrame<false>>(
|
||||
name, argument_types, parameters);
|
||||
});
|
||||
}, properties});
|
||||
|
||||
factory.registerFunction("leadInFrame", [](const std::string & name,
|
||||
factory.registerFunction("leadInFrame", {[](const std::string & name,
|
||||
const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||
{
|
||||
return std::make_shared<WindowFunctionLagLeadInFrame<true>>(
|
||||
name, argument_types, parameters);
|
||||
});
|
||||
}, properties});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -15,3 +15,20 @@ select count() over (rows between 1 + 1 preceding and 1 + 1 following) from numb
|
||||
3
|
||||
-- signed and unsigned in offset do not cause logical error
|
||||
select count() over (rows between 2 following and 1 + -1 following) FROM numbers(10); -- { serverError 36 }
|
||||
-- default arguments of lagInFrame can be a subtype of the argument
|
||||
select number,
|
||||
lagInFrame(toNullable(number), 2, null) over w,
|
||||
lagInFrame(number, 2, 1) over w
|
||||
from numbers(10)
|
||||
window w as (order by number)
|
||||
;
|
||||
0 \N 1
|
||||
1 \N 1
|
||||
2 0 0
|
||||
3 1 1
|
||||
4 2 2
|
||||
5 3 3
|
||||
6 4 4
|
||||
7 5 5
|
||||
8 6 6
|
||||
9 7 7
|
||||
|
@ -7,3 +7,11 @@ select count() over (rows between 1 + 1 preceding and 1 + 1 following) from numb
|
||||
|
||||
-- signed and unsigned in offset do not cause logical error
|
||||
select count() over (rows between 2 following and 1 + -1 following) FROM numbers(10); -- { serverError 36 }
|
||||
|
||||
-- default arguments of lagInFrame can be a subtype of the argument
|
||||
select number,
|
||||
lagInFrame(toNullable(number), 2, null) over w,
|
||||
lagInFrame(number, 2, 1) over w
|
||||
from numbers(10)
|
||||
window w as (order by number)
|
||||
;
|
||||
|
@ -1058,8 +1058,9 @@ settings max_block_size = 3;
|
||||
select lagInFrame(toNullable(1)) over ();
|
||||
\N
|
||||
select lagInFrameOrNull(1) over (); -- { serverError 36 }
|
||||
-- this should give the same error as `select max(Null::Nullable(Nothing))`
|
||||
select intDiv(1, NULL) x, toTypeName(x), max(x) over (); -- { serverError 43 }
|
||||
-- this is the same as `select max(Null::Nullable(Nothing))`
|
||||
select intDiv(1, NULL) x, toTypeName(x), max(x) over ();
|
||||
\N Nullable(Nothing) \N
|
||||
-- to make lagInFrame return null for out-of-frame rows, cast the argument to
|
||||
-- Nullable; otherwise, it returns default values.
|
||||
SELECT
|
||||
|
@ -379,8 +379,8 @@ settings max_block_size = 3;
|
||||
-- careful with auto-application of Null combinator
|
||||
select lagInFrame(toNullable(1)) over ();
|
||||
select lagInFrameOrNull(1) over (); -- { serverError 36 }
|
||||
-- this should give the same error as `select max(Null::Nullable(Nothing))`
|
||||
select intDiv(1, NULL) x, toTypeName(x), max(x) over (); -- { serverError 43 }
|
||||
-- this is the same as `select max(Null::Nullable(Nothing))`
|
||||
select intDiv(1, NULL) x, toTypeName(x), max(x) over ();
|
||||
-- to make lagInFrame return null for out-of-frame rows, cast the argument to
|
||||
-- Nullable; otherwise, it returns default values.
|
||||
SELECT
|
||||
|
Loading…
Reference in New Issue
Block a user