Provide a custom implementation for respect_null aggregations

This commit is contained in:
Raúl Marín 2023-11-27 19:29:20 +01:00
parent 0d0d813a31
commit 3395c7c745
12 changed files with 382 additions and 167 deletions

View File

@ -5,7 +5,12 @@ sidebar_position: 6
# any
Selects the first encountered (non-NULL) value, unless all rows have NULL values in that column.
Selects the first encountered value of a column.
By default, it ignores NULL values and returns the first NOT NULL value found in the column. As [`first_value`](../first_value.md) if supports `RESPECT NULLS`, in which case it will select the first value passed, independently on whether it's NULL or not.
The return type of the function is the same as the input, except for LowCardinality which is discarded). This means that given no rows as input it will return the default value of that type (0 for integers, or Null for a Nullable() column). You might use the `-OrNull` [combinator](../../combinators.md) ) to modify this behaviour.
The query can be executed in any order and even in a different order each time, so the result of this function is indeterminate.
To get a determinate result, you can use the min or max function instead of any.
@ -13,4 +18,4 @@ In some cases, you can rely on the order of execution. This applies to cases whe
When a `SELECT` query has the `GROUP BY` clause or at least one aggregate function, ClickHouse (in contrast to MySQL) requires that all expressions in the `SELECT`, `HAVING`, and `ORDER BY` clauses be calculated from keys or from aggregate functions. In other words, each column selected from the table must be used either in keys or inside aggregate functions. To get behavior like in MySQL, you can put the other columns in the `any` aggregate function.
- Alias: `any_value`
- Alias: `any_value`, `first_value`.

View File

@ -5,9 +5,13 @@ sidebar_position: 7
# first_value
Selects the first encountered value, similar to `any`, but could accept NULL.
Mostly it should be used with [Window Functions](../../window-functions/index.md).
Without Window Functions the result will be random if the source stream is not ordered.
It is an alias for [`any`](../any.md) but it was introduced for compatibility with [Window Functions](../../window-functions/index.md), where sometimes it's necessary to process `NULL` values (by default all ClickHouse
aggregate functions ignore NULLs).
It supports declaring a modifier to respect nulls (`RESPECT NULLS`), both under [Window Functions](../../window-functions/index.md) and in normal aggregations.
As with `any`, without Window Functions the result will be random if the source stream is not ordered and the return type
matches the input type (Null is only returned if the input is Nullable or -OrNull combinator is added).
## examples
@ -23,15 +27,15 @@ INSERT INTO test_data (a, b) Values (1,null), (2,3), (4, 5), (6,null);
```
### example1
The NULL value is ignored at default.
By default, the NULL value is ignored.
```sql
select first_value(b) from test_data;
```
```text
┌─first_value_ignore_nulls(b)─┐
3 │
└─────────────────────────────
┌─any(b)─┐
│ 3 │
└────────┘
```
### example2
@ -41,9 +45,9 @@ select first_value(b) ignore nulls from test_data
```
```text
┌─first_value_ignore_nulls(b)─┐
3 │
└─────────────────────────────
┌─any(b) IGNORE NULLS ─┐
│ 3 │
└──────────────────────┘
```
### example3
@ -53,9 +57,9 @@ select first_value(b) respect nulls from test_data
```
```text
┌─first_value_respect_nulls(b)─┐
ᴺᵁᴸᴸ │
└──────────────────────────────
┌─any(b) RESPECT NULLS ─┐
│ ᴺᵁᴸᴸ │
└───────────────────────┘
```
### example4
@ -73,8 +77,8 @@ FROM
```
```text
┌─first_value_respect_nulls(b)─┬─first_value(b)─┐
ᴺᵁᴸᴸ │ 3 │
└──────────────────────────────────────────────┘
┌─any_respect_nulls(b)─┬─any(b)─┐
│ ᴺᵁᴸᴸ │ 3 │
└──────────────────────┴────────┘
```

View File

@ -1,22 +1,202 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/HelpersMinMaxAny.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <base/defines.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int LOGICAL_ERROR;
}
namespace
{
template <template <typename> class AggregateFunctionTemplate, template <typename, bool> class Data>
struct AggregateFunctionAnyRespectNullsData
{
enum Status : UInt8
{
NotSet = 1,
SetNull = 2,
SetOther = 3
};
Status status = Status::NotSet;
Field value;
bool isSet() const { return status != Status::NotSet; }
void setNull() { status = Status::SetNull; }
void setOther() { status = Status::SetOther; }
};
template <bool First>
class AggregateFunctionAnyRespectNulls final
: public IAggregateFunctionDataHelper<AggregateFunctionAnyRespectNullsData, AggregateFunctionAnyRespectNulls<First>>
{
public:
using Data = AggregateFunctionAnyRespectNullsData;
SerializationPtr serialization;
const bool returns_nullable_type = false;
explicit AggregateFunctionAnyRespectNulls(const DataTypePtr & type)
: IAggregateFunctionDataHelper<Data, AggregateFunctionAnyRespectNulls<First>>({type}, {}, type)
, serialization(type->getDefaultSerialization())
, returns_nullable_type(type->isNullable())
{
}
String getName() const override
{
if constexpr (First)
return "any_respect_nulls";
else
return "anyLast_respect_nulls";
}
bool allocatesMemoryInArena() const override { return false; }
void addNull(AggregateDataPtr __restrict place) const
{
chassert(returns_nullable_type);
auto & d = this->data(place);
if (First && d.isSet())
return;
d.setNull();
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
if (columns[0]->isNullable())
{
if (columns[0]->isNullAt(row_num))
return addNull(place);
}
auto & d = this->data(place);
if (First && d.isSet())
return;
d.setOther();
columns[0]->get(row_num, d.value);
}
void addManyDefaults(AggregateDataPtr __restrict place, const IColumn ** columns, size_t, Arena * arena) const override
{
if (columns[0]->isNullable())
addNull(place);
else
add(place, columns, 0, arena);
}
void addBatchSinglePlace(
size_t row_begin, size_t row_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos)
const override
{
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();
size_t size = row_end - row_begin;
for (size_t i = 0; i < size; ++i)
{
size_t pos = First ? row_begin + i : row_end - 1 - i;
if (flags[pos])
{
add(place, columns, pos, arena);
break;
}
}
}
else
{
size_t pos = First ? row_begin : row_end - 1;
add(place, columns, pos, arena);
}
}
void addBatchSinglePlaceNotNull(
size_t, size_t, AggregateDataPtr __restrict, const IColumn **, const UInt8 *, Arena *, ssize_t) const override
{
/// This should not happen since it means somebody else has preprocessed the data (NULLs or IFs) and might
/// have discarded values that we need (NULLs)
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionAnyRespectNulls::addBatchSinglePlaceNotNull called");
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
auto & d = this->data(place);
if (First && d.isSet())
return;
auto & other = this->data(rhs);
if (other.isSet())
{
d.status = other.status;
d.value = other.value;
}
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
auto & d = this->data(place);
UInt8 k = d.status;
writeBinaryLittleEndian<UInt8>(k, buf);
if (k == Data::Status::SetOther)
serialization->serializeBinary(d.value, buf, {});
}
void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
auto & d = this->data(place);
UInt8 k = Data::Status::NotSet;
readBinaryLittleEndian<UInt8>(k, buf);
d.status = static_cast<Data::Status>(k);
if (d.status == Data::Status::NotSet)
return;
else if (d.status == Data::Status::SetNull)
{
if (!returns_nullable_type)
throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect type (NULL) in non-nullable {}State", getName());
return;
}
else if (d.status == Data::Status::SetOther)
serialization->deserializeBinary(d.value, buf, {});
else
throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect type ({}) in {}State", static_cast<Int8>(k), getName());
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & d = this->data(place);
if (d.status == Data::Status::SetOther)
to.insert(d.value);
else
to.insertDefault();
}
AggregateFunctionPtr getOwnNullAdapter(
const AggregateFunctionPtr & original_function,
const DataTypes & /*arguments*/,
const Array & /*params*/,
const AggregateFunctionProperties & /*properties*/) const override
{
return original_function;
}
};
template <bool First>
IAggregateFunction * createAggregateFunctionSingleValueRespectNulls(
const String & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);
constexpr bool respect_nulls = true;
return new AggregateFunctionTemplate<Data<SingleValueDataGeneric<true>, respect_nulls>>(argument_types[0]);
return new AggregateFunctionAnyRespectNulls<First>(argument_types[0]);
}
AggregateFunctionPtr createAggregateFunctionAny(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
@ -24,11 +204,10 @@ AggregateFunctionPtr createAggregateFunctionAny(const std::string & name, const
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyData>(name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionNullableAny(
AggregateFunctionPtr createAggregateFunctionAnyRespectNulls(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionSingleValueRespectNulls<AggregateFunctionsSingleValue, AggregateFunctionAnyData>(
name, argument_types, parameters, settings));
return AggregateFunctionPtr(createAggregateFunctionSingleValueRespectNulls<true>(name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionAnyLast(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
@ -36,10 +215,10 @@ AggregateFunctionPtr createAggregateFunctionAnyLast(const std::string & name, co
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyLastData>(name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionNullableAnyLast(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
AggregateFunctionPtr createAggregateFunctionAnyLastRespectNulls(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
{
return AggregateFunctionPtr(createAggregateFunctionSingleValueRespectNulls<AggregateFunctionsSingleValue, AggregateFunctionAnyLastData>(
name, argument_types, parameters, settings));
return AggregateFunctionPtr(createAggregateFunctionSingleValueRespectNulls<false>(name, argument_types, parameters, settings));
}
AggregateFunctionPtr createAggregateFunctionAnyHeavy(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
@ -59,14 +238,14 @@ void registerAggregateFunctionsAny(AggregateFunctionFactory & factory)
factory.registerAlias("any_value", "any", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("first_value", "any", AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("any_respect_nulls", {createAggregateFunctionNullableAny, default_properties_for_respect_nulls});
factory.registerFunction("any_respect_nulls", {createAggregateFunctionAnyRespectNulls, default_properties_for_respect_nulls});
factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::CaseInsensitive);
factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("anyLast", {createAggregateFunctionAnyLast, default_properties});
factory.registerAlias("last_value", "anyLast", AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("anyLast_respect_nulls", {createAggregateFunctionNullableAnyLast, default_properties_for_respect_nulls});
factory.registerFunction("anyLast_respect_nulls", {createAggregateFunctionAnyLastRespectNulls, default_properties_for_respect_nulls});
factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::CaseInsensitive);
factory.registerFunction("anyHeavy", {createAggregateFunctionAnyHeavy, default_properties});

View File

@ -771,26 +771,18 @@ static_assert(
/// For any other value types.
template <bool RESULT_IS_NULLABLE = false>
struct SingleValueDataGeneric
{
private:
using Self = SingleValueDataGeneric;
Field value;
bool has_value = false;
public:
static constexpr bool result_is_nullable = RESULT_IS_NULLABLE;
static constexpr bool should_skip_null_arguments = !RESULT_IS_NULLABLE;
static constexpr bool result_is_nullable = false;
static constexpr bool should_skip_null_arguments = true;
static constexpr bool is_any = false;
bool has() const
{
if constexpr (result_is_nullable)
return has_value;
return !value.isNull();
}
bool has() const { return !value.isNull(); }
void insertResultInto(IColumn & to) const
{
@ -820,19 +812,9 @@ public:
serialization.deserializeBinary(value, buf, {});
}
void change(const IColumn & column, size_t row_num, Arena *)
{
column.get(row_num, value);
if constexpr (result_is_nullable)
has_value = true;
}
void change(const IColumn & column, size_t row_num, Arena *) { column.get(row_num, value); }
void change(const Self & to, Arena *)
{
value = to.value;
if constexpr (result_is_nullable)
has_value = true;
}
void change(const Self & to, Arena *) { value = to.value; }
bool changeFirstTime(const IColumn & column, size_t row_num, Arena * arena)
{
@ -847,7 +829,7 @@ public:
bool changeFirstTime(const Self & to, Arena * arena)
{
if (!has() && (result_is_nullable || to.has()))
if (!has() && to.has())
{
change(to, arena);
return true;
@ -882,30 +864,15 @@ public:
}
else
{
if constexpr (result_is_nullable)
Field new_value;
column.get(row_num, new_value);
if (new_value < value)
{
Field new_value;
column.get(row_num, new_value);
if (!value.isNull() && (new_value.isNull() || new_value < value))
{
value = new_value;
return true;
}
else
return false;
value = new_value;
return true;
}
else
{
Field new_value;
column.get(row_num, new_value);
if (new_value < value)
{
value = new_value;
return true;
}
else
return false;
}
return false;
}
}
@ -913,30 +880,13 @@ public:
{
if (!to.has())
return false;
if constexpr (result_is_nullable)
if (!has() || to.value < value)
{
if (!has())
{
change(to, arena);
return true;
}
if (to.value.isNull() || (!value.isNull() && to.value < value))
{
value = to.value;
return true;
}
return false;
change(to, arena);
return true;
}
else
{
if (!has() || to.value < value)
{
change(to, arena);
return true;
}
else
return false;
}
return false;
}
bool changeIfGreater(const IColumn & column, size_t row_num, Arena * arena)
@ -948,29 +898,15 @@ public:
}
else
{
if constexpr (result_is_nullable)
Field new_value;
column.get(row_num, new_value);
if (new_value > value)
{
Field new_value;
column.get(row_num, new_value);
if (!value.isNull() && (new_value.isNull() || value < new_value))
{
value = new_value;
return true;
}
return false;
value = new_value;
return true;
}
else
{
Field new_value;
column.get(row_num, new_value);
if (new_value > value)
{
value = new_value;
return true;
}
else
return false;
}
return false;
}
}
@ -978,36 +914,18 @@ public:
{
if (!to.has())
return false;
if constexpr (result_is_nullable)
if (!has() || to.value > value)
{
if (!value.isNull() && (to.value.isNull() || value < to.value))
{
value = to.value;
return true;
}
return false;
change(to, arena);
return true;
}
else
{
if (!has() || to.value > value)
{
change(to, arena);
return true;
}
else
return false;
}
return false;
}
bool isEqualTo(const IColumn & column, size_t row_num) const
{
return has() && value == column[row_num];
}
bool isEqualTo(const IColumn & column, size_t row_num) const { return has() && value == column[row_num]; }
bool isEqualTo(const Self & to) const
{
return has() && to.value == value;
}
bool isEqualTo(const Self & to) const { return has() && to.value == value; }
static bool allocatesMemoryInArena()
{
@ -1084,7 +1002,7 @@ struct AggregateFunctionMaxData : Data
#endif
};
template <typename Data, bool RespectNulls = false>
template <typename Data>
struct AggregateFunctionAnyData : Data
{
using Self = AggregateFunctionAnyData;
@ -1094,13 +1012,7 @@ struct AggregateFunctionAnyData : Data
bool changeIfBetter(const Self & to, Arena * arena) { return this->changeFirstTime(to, arena); }
void addManyDefaults(const IColumn & column, size_t /*length*/, Arena * arena) { this->changeFirstTime(column, 0, arena); }
static const char * name()
{
if constexpr (RespectNulls)
return "any_respect_nulls";
else
return "any";
}
static const char * name() { return "any"; }
#if USE_EMBEDDED_COMPILER
@ -1119,7 +1031,7 @@ struct AggregateFunctionAnyData : Data
#endif
};
template <typename Data, bool RespectNulls = false>
template <typename Data>
struct AggregateFunctionAnyLastData : Data
{
using Self = AggregateFunctionAnyLastData;
@ -1128,13 +1040,7 @@ struct AggregateFunctionAnyLastData : Data
bool changeIfBetter(const Self & to, Arena * arena) { return this->changeEveryTime(to, arena); }
void addManyDefaults(const IColumn & column, size_t /*length*/, Arena * arena) { this->changeEveryTime(column, 0, arena); }
static const char * name()
{
if constexpr (RespectNulls)
return "anyLast_respect_nulls";
else
return "anyLast";
}
static const char * name() { return "anyLast"; }
#if USE_EMBEDDED_COMPILER

View File

@ -20,7 +20,7 @@ template <template <typename> class Data>
class AggregateFunctionCombinatorArgMinMax final : public IAggregateFunctionCombinator
{
public:
String getName() const override { return Data<SingleValueDataGeneric<>>::name(); }
String getName() const override { return Data<SingleValueDataGeneric>::name(); }
DataTypes transformArguments(const DataTypes & arguments) const override
{
@ -66,7 +66,7 @@ public:
if (which.idx == TypeIndex::String)
return std::make_shared<AggregateFunctionArgMinMax<Data<SingleValueDataString>>>(nested_function, arguments, params);
return std::make_shared<AggregateFunctionArgMinMax<Data<SingleValueDataGeneric<>>>>(nested_function, arguments, params);
return std::make_shared<AggregateFunctionArgMinMax<Data<SingleValueDataGeneric>>>(nested_function, arguments, params);
}
};

View File

@ -45,7 +45,7 @@ createAggregateFunctionSingleValue(const String & name, const DataTypes & argume
if (which.idx == TypeIndex::String)
return new AggregateFunctionTemplate<Data<SingleValueDataString>>(argument_type);
return new AggregateFunctionTemplate<Data<SingleValueDataGeneric<>>>(argument_type);
return new AggregateFunctionTemplate<Data<SingleValueDataGeneric>>(argument_type);
}
/// argMin, argMax
@ -77,7 +77,7 @@ static IAggregateFunction * createAggregateFunctionArgMinMaxSecond(const DataTyp
if (which.idx == TypeIndex::String)
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataString>>>(res_type, val_type);
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataGeneric<>>>>(res_type, val_type);
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataGeneric>>>(res_type, val_type);
}
template <template <typename> class MinMaxData>
@ -113,7 +113,7 @@ static IAggregateFunction * createAggregateFunctionArgMinMax(const String & name
if (which.idx == TypeIndex::String)
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataString>(res_type, val_type);
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataGeneric<>>(res_type, val_type);
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataGeneric>(res_type, val_type);
}
}

View File

@ -0,0 +1,32 @@
-- { echoOn }
Select anyOrNull(tp) FROM (Select (number, number) as tp from numbers(10)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
Select anyOrNull(tp) FROM (Select (number, number) as tp from numbers(10)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT
any(tp) AS default,
toTypeName(default) as default_type,
any(tp) RESPECT NULLS AS respect,
toTypeName(respect) as respect_type
FROM
(
SELECT (toLowCardinality(number), number) AS tp
FROM numbers(10)
);
(0,0) Tuple(UInt64, UInt64) (0,0) Tuple(UInt64, UInt64)
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(number) as t FROM numbers(0));
0
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(number::Nullable(UInt8)) as t FROM numbers(0));
\N
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(number::LowCardinality(Nullable(UInt8))) as t FROM numbers(0)) settings allow_suspicious_low_cardinality_types=1;
\N
SELECT first_value_respect_nullsOrNullMerge(t) FROM (Select first_value_respect_nullsOrNullState(number) as t FROM numbers(0));
\N
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsOrNullState(number) as t FROM numbers(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT first_value_respect_nullsOrNullMerge(t) FROM (Select first_value_respect_nullsState(number) as t FROM numbers(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(dummy) as t FROM system.one);
0
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(dummy::Nullable(UInt8)) as t FROM system.one);
0
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(NULL) as t FROM system.one);
\N
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(NULL::Nullable(UInt8)) as t FROM system.one);
\N

View File

@ -0,0 +1,63 @@
SELECT
*, * APPLY (toTypeName)
FROM
(
SELECT
bl,
anyIf(n, cond) IGNORE NULLS AS any_ignore,
anyIf(n, cond) RESPECT NULLS AS any_respect,
anyLastIf(n, cond) IGNORE NULLS AS last_ignore,
anyLastIf(n, cond) RESPECT NULLS AS last_respect,
anyIf(nullable_n, cond) IGNORE NULLS AS any_nullable_ignore,
anyIf(nullable_n, cond) RESPECT NULLS AS any_nullable_respect,
anyLastIf(nullable_n, cond) IGNORE NULLS AS last_nullable_ignore,
anyLastIf(nullable_n, cond) RESPECT NULLS AS last_nullable_respect
FROM
(
SELECT
number AS n,
rand() > pow(2, 31) as cond,
if(cond, NULL, n) as nullable_n,
blockNumber() AS bl
FROM numbers(10000)
)
GROUP BY bl
)
WHERE
any_ignore != any_respect
OR toTypeName(any_ignore) != toTypeName(any_respect)
OR last_ignore != last_respect
OR toTypeName(last_ignore) != toTypeName(last_respect)
OR any_nullable_ignore != any_nullable_respect
OR toTypeName(any_nullable_ignore) != toTypeName(any_nullable_respect)
OR last_nullable_ignore != last_nullable_respect
OR toTypeName(last_nullable_ignore) != toTypeName(last_nullable_respect);
-- { echoOn }
Select anyOrNull(tp) FROM (Select (number, number) as tp from numbers(10)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
Select anyOrNull(tp) FROM (Select (number, number) as tp from numbers(10)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT
any(tp) AS default,
toTypeName(default) as default_type,
any(tp) RESPECT NULLS AS respect,
toTypeName(respect) as respect_type
FROM
(
SELECT (toLowCardinality(number), number) AS tp
FROM numbers(10)
);
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(number) as t FROM numbers(0));
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(number::Nullable(UInt8)) as t FROM numbers(0));
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(number::LowCardinality(Nullable(UInt8))) as t FROM numbers(0)) settings allow_suspicious_low_cardinality_types=1;
SELECT first_value_respect_nullsOrNullMerge(t) FROM (Select first_value_respect_nullsOrNullState(number) as t FROM numbers(0));
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsOrNullState(number) as t FROM numbers(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT first_value_respect_nullsOrNullMerge(t) FROM (Select first_value_respect_nullsState(number) as t FROM numbers(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(dummy) as t FROM system.one);
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(dummy::Nullable(UInt8)) as t FROM system.one);
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(NULL) as t FROM system.one);
SELECT first_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(NULL::Nullable(UInt8)) as t FROM system.one);

View File

@ -73,17 +73,27 @@ SELECT number, avgIf (number) RESPECT NULLS over (order by number) from numbers(
SELECT number, first_value_respect_nulls (number) RESPECT NULLS over (order by number) from numbers(1); -- { serverError NOT_IMPLEMENTED }
SELECT number, last_value_respect_nulls (number) RESPECT NULLS over (order by number) from numbers(1); -- { serverError NOT_IMPLEMENTED }
-- Aggregate_functions_null_for_empty should work the same way
SELECT toTypeName(any(number) RESPECT NULLS) from numbers(1);
UInt64
SELECT toTypeName(anyOrNull(number) RESPECT NULLS) from numbers(1);
Nullable(UInt64)
SELECT any(number) RESPECT NULLS from numbers(0);
0
SELECT anyOrNull(number) RESPECT NULLS from numbers(0);
\N
SELECT any(number) RESPECT NULLS from (Select NULL::Nullable(UInt8) as number FROM numbers(10));
\N
SELECT anyOrNull(number) RESPECT NULLS from (Select NULL::Nullable(UInt8) as number FROM numbers(10));
\N
SELECT any(number) RESPECT NULLS from (Select if(number > 8, NULL, number) as number FROM numbers(10));
0
SELECT anyOrNull(number) RESPECT NULLS from (Select if(number > 8, NULL, number) as number FROM numbers(10));
0
SELECT any(number) RESPECT NULLS from (Select if(number < 8, NULL, number) as number FROM numbers(10));
\N
SELECT anyOrNull(number) RESPECT NULLS from (Select if(number < 8, NULL, number) as number FROM numbers(10));
\N
SELECT toTypeName(any(number) RESPECT NULLS) from numbers(1) SETTINGS aggregate_functions_null_for_empty = 1;
Nullable(UInt64)
SELECT any(number) RESPECT NULLS from numbers(0) SETTINGS aggregate_functions_null_for_empty = 1;;
SELECT any(number) RESPECT NULLS from numbers(0) SETTINGS aggregate_functions_null_for_empty = 1;
\N

View File

@ -16,19 +16,16 @@ Select number, LAST_VALUE (number) over (order by number) from numbers(1);
Select number, last_value_respect_nulls (number) over (order by number) from numbers(1);
Select number, anyLast (number) RESPECT NULLS over (order by number) from numbers(1);
-- IGNORE NULLS should be accepted too
Select number, FIRST_VALUE (number) IGNORE NULLS over (order by number) from numbers(1);
Select number, LAST_VALUE (number) IGNORE NULLS over (order by number) from numbers(1);
-- When applying IGNORE NULLs to first_value_respect_nulls we go back to the original function (any)
Select first_value_respect_nulls (number) IGNORE NULLS from (SELECT if(number < 2, NULL, number) as number FROM numbers(10));
Select FIRST_VALUE_respect_nulls (number) IGNORE NULLS from (SELECT if(number < 2, NULL, number) as number FROM numbers(10));
Select last_value_respect_nulls (number) IGNORE NULLS from (SELECT if(number < 2, NULL, number) as number FROM numbers(10));
Select LAST_VALUE_respect_nulls (number) IGNORE NULLS from (SELECT if(number < 2, NULL, number) as number FROM numbers(10));
-- IGNORE/RESPECT NULLS should work with combinators because we can do it
SELECT first_valueIf (number, NOT isNull(number) AND (assumeNotNull(number) > 5)) RESPECT NULLS from (SELECT if(number < 2, NULL, number) as number FROM numbers(10));
SELECT last_valueIf (number, NOT isNull(number) AND (assumeNotNull(number) > 5)) RESPECT NULLS from (SELECT if(number < 2, NULL, number) as number FROM numbers(10));
@ -43,7 +40,6 @@ SELECT anyLastIf (number, isNull(number)) RESPECT NULLS from (SELECT if(number >
SELECT toTypeName(FIRST_VALUEIfState(number, isNull(number)) RESPECT NULLS) from (SELECT if(number > 8, NULL, number) as number FROM numbers(10));
SELECT toTypeName(LAST_VALUEIfState(number, isNull(number)) RESPECT NULLS) from (SELECT if(number > 8, NULL, number) as number FROM numbers(10));
-- Unsupported functions should throw in the server
SELECT number, sum (number) RESPECT NULLS over (order by number) from numbers(1); -- { serverError NOT_IMPLEMENTED }
SELECT number, avgIf (number) RESPECT NULLS over (order by number) from numbers(1); -- { serverError NOT_IMPLEMENTED }
@ -52,11 +48,16 @@ SELECT number, first_value_respect_nulls (number) RESPECT NULLS over (order by n
SELECT number, last_value_respect_nulls (number) RESPECT NULLS over (order by number) from numbers(1); -- { serverError NOT_IMPLEMENTED }
-- Aggregate_functions_null_for_empty should work the same way
SELECT toTypeName(any(number) RESPECT NULLS) from numbers(1);
SELECT toTypeName(anyOrNull(number) RESPECT NULLS) from numbers(1);
SELECT any(number) RESPECT NULLS from numbers(0);
SELECT anyOrNull(number) RESPECT NULLS from numbers(0);
SELECT any(number) RESPECT NULLS from (Select NULL::Nullable(UInt8) as number FROM numbers(10));
SELECT anyOrNull(number) RESPECT NULLS from (Select NULL::Nullable(UInt8) as number FROM numbers(10));
SELECT any(number) RESPECT NULLS from (Select if(number > 8, NULL, number) as number FROM numbers(10));
SELECT anyOrNull(number) RESPECT NULLS from (Select if(number > 8, NULL, number) as number FROM numbers(10));
SELECT any(number) RESPECT NULLS from (Select if(number < 8, NULL, number) as number FROM numbers(10));
SELECT anyOrNull(number) RESPECT NULLS from (Select if(number < 8, NULL, number) as number FROM numbers(10));
SELECT toTypeName(any(number) RESPECT NULLS) from numbers(1) SETTINGS aggregate_functions_null_for_empty = 1;
SELECT any(number) RESPECT NULLS from numbers(0) SETTINGS aggregate_functions_null_for_empty = 1;;
SELECT any(number) RESPECT NULLS from numbers(0) SETTINGS aggregate_functions_null_for_empty = 1;

View File

@ -7,4 +7,7 @@ AggregateFunction(any_respect_nulls, UInt8) AggregateFunction(anyLast_respect_nu
\N
4
\N
0
0
\N
\N

View File

@ -14,4 +14,16 @@ SELECT first_value_respect_nullsMerge(t) FROM (Select first_valueState(number) a
SELECT first_value_respect_nullsMerge(t) FROM (Select last_value_respect_nullsState(number) as t from numbers(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT last_value_respect_nullsMerge(t) FROM (Select first_value_respect_nullsState(number) as t from numbers(1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT first_value_respect_nullsMerge(CAST(unhex('00'), 'AggregateFunction(any, UInt64)')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT first_value_respect_nullsMerge(CAST(unhex('00'), 'AggregateFunction(any_respect_nulls, UInt64)'));
-- Invalid (starts at 1)
SELECT first_value_respect_nullsMerge(CAST(unhex('00'), 'AggregateFunction(any_respect_nulls, UInt64)')); -- { serverError INCORRECT_DATA }
-- Not set (Default value)
SELECT first_value_respect_nullsMerge(CAST(unhex('01'), 'AggregateFunction(any_respect_nulls, UInt64)'));
SELECT finalizeAggregation(CAST(unhex('01'), 'AggregateFunction(any_respect_nulls, UInt64)'));
-- Set to NULL
SELECT first_value_respect_nullsMerge(CAST(unhex('02'), 'AggregateFunction(any_respect_nulls, UInt64)')); -- { serverError INCORRECT_DATA }
SELECT first_value_respect_nullsMerge(CAST(unhex('02'), 'AggregateFunction(any_respect_nulls, Nullable(UInt64))'));
SELECT finalizeAggregation(CAST(unhex('02'), 'AggregateFunction(any_respect_nulls, Nullable(UInt64))'));
-- Set to other value, but without providing value
SELECT first_value_respect_nullsMerge(CAST(unhex('03'), 'AggregateFunction(any_respect_nulls, UInt64)')); -- { serverError CANNOT_READ_ALL_DATA }