This commit is contained in:
Alexander Kuzmenkov 2021-07-21 01:36:15 +03:00
parent 61a01782a6
commit a197511a96
6 changed files with 85 additions and 35 deletions

View File

@ -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;
}

View File

@ -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});
}
}

View File

@ -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

View File

@ -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)
;

View File

@ -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

View File

@ -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