mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Add tuple argument support for argMin and argMax
This commit is contained in:
parent
fe6b964b32
commit
47a0f4e162
@ -4,6 +4,28 @@ toc_priority: 106
|
||||
|
||||
# argMax {#agg-function-argmax}
|
||||
|
||||
Syntax: `argMax(arg, val)`
|
||||
Syntax: `argMax(arg, val)` or `argMax(tuple(arg, val))`
|
||||
|
||||
Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, the first of these values encountered is output.
|
||||
|
||||
Tuple version of this function will return the tuple with the maximum `val` value. It is convinient for use with `SimpleAggregateFunction`.
|
||||
|
||||
**Example:**
|
||||
|
||||
``` text
|
||||
┌─user─────┬─salary─┐
|
||||
│ director │ 5000 │
|
||||
│ manager │ 3000 │
|
||||
│ worker │ 1000 │
|
||||
└──────────┴────────┘
|
||||
```
|
||||
|
||||
``` sql
|
||||
SELECT argMax(user, salary), argMax(tuple(user, salary)) FROM salary
|
||||
```
|
||||
|
||||
``` text
|
||||
┌─argMax(user, salary)─┬─argMax(tuple(user, salary))─┐
|
||||
│ director │ ('director',5000) │
|
||||
└──────────────────────┴─────────────────────────────┘
|
||||
```
|
||||
|
@ -4,10 +4,12 @@ toc_priority: 105
|
||||
|
||||
# argMin {#agg-function-argmin}
|
||||
|
||||
Syntax: `argMin(arg, val)`
|
||||
Syntax: `argMin(arg, val)` or `argMin(tuple(arg, val))`
|
||||
|
||||
Calculates the `arg` value for a minimal `val` value. If there are several different values of `arg` for minimal values of `val`, the first of these values encountered is output.
|
||||
|
||||
Tuple version of this function will return the tuple with the minimal `val` value. It is convinient for use with `SimpleAggregateFunction`.
|
||||
|
||||
**Example:**
|
||||
|
||||
``` text
|
||||
@ -19,11 +21,11 @@ Calculates the `arg` value for a minimal `val` value. If there are several diffe
|
||||
```
|
||||
|
||||
``` sql
|
||||
SELECT argMin(user, salary) FROM salary
|
||||
SELECT argMin(user, salary), argMin(tuple(user, salary)) FROM salary
|
||||
```
|
||||
|
||||
``` text
|
||||
┌─argMin(user, salary)─┐
|
||||
│ worker │
|
||||
└──────────────────────┘
|
||||
┌─argMin(user, salary)─┬─argMin(tuple(user, salary))─┐
|
||||
│ worker │ ('worker',1000) │
|
||||
└──────────────────────┴─────────────────────────────┘
|
||||
```
|
||||
|
@ -18,6 +18,8 @@ The following aggregate functions are supported:
|
||||
- [`sumMap`](../../sql-reference/aggregate-functions/reference/summap.md#agg_functions-summap)
|
||||
- [`minMap`](../../sql-reference/aggregate-functions/reference/minmap.md#agg_functions-minmap)
|
||||
- [`maxMap`](../../sql-reference/aggregate-functions/reference/maxmap.md#agg_functions-maxmap)
|
||||
- [`argMin`](../../sql-reference/aggregate-functions/reference/argmin.md)
|
||||
- [`argMax`](../../sql-reference/aggregate-functions/reference/argmax.md)
|
||||
|
||||
Values of the `SimpleAggregateFunction(func, Type)` look and stored the same way as `Type`, so you do not need to apply functions with `-Merge`/`-State` suffixes. `SimpleAggregateFunction` has better performance than `AggregateFunction` with same aggregation function.
|
||||
|
||||
|
@ -1,14 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/StringRef.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/AggregateFunctionMinMaxAny.h> // SingleValueDataString used in embedded compiler
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <common/StringRef.h>
|
||||
#include "Columns/IColumn.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
@ -22,37 +24,49 @@ struct AggregateFunctionArgMinMaxData
|
||||
using ResultData_t = ResultData;
|
||||
using ValueData_t = ValueData;
|
||||
|
||||
ResultData result; // the argument at which the minimum/maximum value is reached.
|
||||
ValueData value; // value for which the minimum/maximum is calculated.
|
||||
ResultData result; // the argument at which the minimum/maximum value is reached.
|
||||
ValueData value; // value for which the minimum/maximum is calculated.
|
||||
|
||||
static bool allocatesMemoryInArena()
|
||||
{
|
||||
return ResultData::allocatesMemoryInArena() || ValueData::allocatesMemoryInArena();
|
||||
}
|
||||
static bool allocatesMemoryInArena() { return ResultData::allocatesMemoryInArena() || ValueData::allocatesMemoryInArena(); }
|
||||
|
||||
static String name() { return StringRef(ValueData_t::name()) == StringRef("min") ? "argMin" : "argMax"; }
|
||||
};
|
||||
|
||||
/// Returns the first arg value found for the minimum/maximum value. Example: argMax(arg, value).
|
||||
template <typename Data>
|
||||
class AggregateFunctionArgMinMax final : public IAggregateFunctionDataHelper<Data, AggregateFunctionArgMinMax<Data>>
|
||||
class AggregateFunctionArgMinMax final : public IAggregateFunctionTupleArgHelper<Data, AggregateFunctionArgMinMax<Data>, 2>
|
||||
{
|
||||
private:
|
||||
const DataTypePtr & type_res;
|
||||
const DataTypePtr & type_val;
|
||||
bool tuple_argument;
|
||||
|
||||
using Base = IAggregateFunctionTupleArgHelper<Data, AggregateFunctionArgMinMax<Data>, 2>;
|
||||
|
||||
public:
|
||||
AggregateFunctionArgMinMax(const DataTypePtr & type_res_, const DataTypePtr & type_val_)
|
||||
: IAggregateFunctionDataHelper<Data, AggregateFunctionArgMinMax<Data>>({type_res_, type_val_}, {}),
|
||||
type_res(this->argument_types[0]), type_val(this->argument_types[1])
|
||||
AggregateFunctionArgMinMax(const DataTypePtr & type_res_, const DataTypePtr & type_val_, const bool tuple_argument_)
|
||||
: Base({type_res_, type_val_}, {}, tuple_argument_)
|
||||
, type_res(this->argument_types[0])
|
||||
, type_val(this->argument_types[1])
|
||||
{
|
||||
if (!type_val->isComparable())
|
||||
throw Exception("Illegal type " + type_val->getName() + " of second argument of aggregate function " + getName()
|
||||
+ " because the values of that data type are not comparable", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(
|
||||
"Illegal type " + type_val->getName() + " of second argument of aggregate function " + getName()
|
||||
+ " because the values of that data type are not comparable",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
this->tuple_argument = tuple_argument_;
|
||||
}
|
||||
|
||||
String getName() const override { return StringRef(Data::ValueData_t::name()) == StringRef("min") ? "argMin" : "argMax"; }
|
||||
String getName() const override { return Data::name(); }
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
if (tuple_argument)
|
||||
{
|
||||
return std::make_shared<DataTypeTuple>(DataTypes{this->type_res, this->type_val});
|
||||
}
|
||||
|
||||
return type_res;
|
||||
}
|
||||
|
||||
@ -80,15 +94,21 @@ public:
|
||||
this->data(place).value.read(buf, *type_val, arena);
|
||||
}
|
||||
|
||||
bool allocatesMemoryInArena() const override
|
||||
{
|
||||
return Data::allocatesMemoryInArena();
|
||||
}
|
||||
bool allocatesMemoryInArena() const override { return Data::allocatesMemoryInArena(); }
|
||||
|
||||
void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override
|
||||
{
|
||||
this->data(place).result.insertResultInto(to);
|
||||
if (tuple_argument)
|
||||
{
|
||||
auto & tup = assert_cast<ColumnTuple &>(to);
|
||||
|
||||
this->data(place).result.insertResultInto(tup.getColumn(0));
|
||||
this->data(place).value.insertResultInto(tup.getColumn(1));
|
||||
}
|
||||
else
|
||||
this->data(place).result.insertResultInto(to);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
|
@ -31,6 +31,12 @@
|
||||
M(Float32) \
|
||||
M(Float64)
|
||||
|
||||
#define FOR_DECIMAL_TYPES(M) \
|
||||
M(Decimal32) \
|
||||
M(Decimal64) \
|
||||
M(Decimal128)
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -8,10 +8,14 @@
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
/// min, max, any, anyLast, anyHeavy, etc...
|
||||
template <template <typename> class AggregateFunctionTemplate, template <typename> class Data>
|
||||
@ -26,6 +30,7 @@ static IAggregateFunction * createAggregateFunctionSingleValue(const String & na
|
||||
#define DISPATCH(TYPE) \
|
||||
if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<TYPE>>>(argument_type);
|
||||
FOR_NUMERIC_TYPES(DISPATCH)
|
||||
FOR_DECIMAL_TYPES(DISPATCH)
|
||||
#undef DISPATCH
|
||||
|
||||
if (which.idx == TypeIndex::Date)
|
||||
@ -34,12 +39,6 @@ static IAggregateFunction * createAggregateFunctionSingleValue(const String & na
|
||||
return new AggregateFunctionTemplate<Data<SingleValueDataFixed<DataTypeDateTime::FieldType>>>(argument_type);
|
||||
if (which.idx == TypeIndex::DateTime64)
|
||||
return new AggregateFunctionTemplate<Data<SingleValueDataFixed<DateTime64>>>(argument_type);
|
||||
if (which.idx == TypeIndex::Decimal32)
|
||||
return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Decimal32>>>(argument_type);
|
||||
if (which.idx == TypeIndex::Decimal64)
|
||||
return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Decimal64>>>(argument_type);
|
||||
if (which.idx == TypeIndex::Decimal128)
|
||||
return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Decimal128>>>(argument_type);
|
||||
if (which.idx == TypeIndex::String)
|
||||
return new AggregateFunctionTemplate<Data<SingleValueDataString>>(argument_type);
|
||||
|
||||
@ -49,66 +48,77 @@ static IAggregateFunction * createAggregateFunctionSingleValue(const String & na
|
||||
|
||||
/// argMin, argMax
|
||||
template <template <typename> class MinMaxData, typename ResData>
|
||||
static IAggregateFunction * createAggregateFunctionArgMinMaxSecond(const DataTypePtr & res_type, const DataTypePtr & val_type)
|
||||
static IAggregateFunction * createAggregateFunctionArgMinMaxSecond(const DataTypePtr & res_type, const DataTypePtr & val_type, bool is_tuple)
|
||||
{
|
||||
WhichDataType which(val_type);
|
||||
|
||||
#define DISPATCH(TYPE) \
|
||||
if (which.idx == TypeIndex::TYPE) \
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<TYPE>>>>(res_type, val_type);
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<TYPE>>>>(res_type, val_type, is_tuple);
|
||||
FOR_NUMERIC_TYPES(DISPATCH)
|
||||
FOR_DECIMAL_TYPES(DISPATCH)
|
||||
#undef DISPATCH
|
||||
|
||||
if (which.idx == TypeIndex::Date)
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDate::FieldType>>>>(res_type, val_type);
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDate::FieldType>>>>(res_type, val_type, is_tuple);
|
||||
if (which.idx == TypeIndex::DateTime)
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDateTime::FieldType>>>>(res_type, val_type);
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDateTime::FieldType>>>>(res_type, val_type, is_tuple);
|
||||
if (which.idx == TypeIndex::DateTime64)
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DateTime64>>>>(res_type, val_type);
|
||||
if (which.idx == TypeIndex::Decimal32)
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Decimal32>>>>(res_type, val_type);
|
||||
if (which.idx == TypeIndex::Decimal64)
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Decimal64>>>>(res_type, val_type);
|
||||
if (which.idx == TypeIndex::Decimal128)
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<Decimal128>>>>(res_type, val_type);
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DateTime64>>>>(res_type, val_type, is_tuple);
|
||||
if (which.idx == TypeIndex::String)
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataString>>>(res_type, val_type);
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataString>>>(res_type, val_type, is_tuple);
|
||||
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataGeneric>>>(res_type, val_type, is_tuple);
|
||||
|
||||
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataGeneric>>>(res_type, val_type);
|
||||
}
|
||||
|
||||
template <template <typename> class MinMaxData>
|
||||
static IAggregateFunction * createAggregateFunctionArgMinMax(const String & name, const DataTypes & argument_types, const Array & parameters)
|
||||
{
|
||||
assertNoParameters(name, parameters);
|
||||
assertBinary(name, argument_types);
|
||||
|
||||
const DataTypePtr & res_type = argument_types[0];
|
||||
const DataTypePtr & val_type = argument_types[1];
|
||||
DataTypePtr res_type, val_type;
|
||||
bool is_tuple = false;
|
||||
|
||||
// argMin and argMax could get tuple of two as arguments
|
||||
if (argument_types.size() == 1 && argument_types[0]->getTypeId() == TypeIndex::Tuple)
|
||||
{
|
||||
const auto * tuple_type = assert_cast<const DataTypeTuple *>(argument_types[0].get());
|
||||
|
||||
if (tuple_type->getElements().size() != 2)
|
||||
{
|
||||
throw Exception("Aggregate function " + name + " expects two elements in tuple argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
res_type = tuple_type->getElements()[0];
|
||||
val_type = tuple_type->getElements()[1];
|
||||
is_tuple = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
assertBinary(name, argument_types);
|
||||
res_type = argument_types[0];
|
||||
val_type = argument_types[1];
|
||||
}
|
||||
|
||||
WhichDataType which(res_type);
|
||||
#define DISPATCH(TYPE) \
|
||||
if (which.idx == TypeIndex::TYPE) \
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<TYPE>>(res_type, val_type);
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<TYPE>>(res_type, val_type, is_tuple);
|
||||
FOR_NUMERIC_TYPES(DISPATCH)
|
||||
FOR_DECIMAL_TYPES(DISPATCH)
|
||||
#undef DISPATCH
|
||||
|
||||
if (which.idx == TypeIndex::Date)
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDate::FieldType>>(res_type, val_type);
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDate::FieldType>>(res_type, val_type, is_tuple);
|
||||
if (which.idx == TypeIndex::DateTime)
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDateTime::FieldType>>(res_type, val_type);
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDateTime::FieldType>>(res_type, val_type, is_tuple);
|
||||
if (which.idx == TypeIndex::DateTime64)
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DateTime64>>(res_type, val_type);
|
||||
if (which.idx == TypeIndex::Decimal32)
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Decimal32>>(res_type, val_type);
|
||||
if (which.idx == TypeIndex::Decimal64)
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Decimal64>>(res_type, val_type);
|
||||
if (which.idx == TypeIndex::Decimal128)
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<Decimal128>>(res_type, val_type);
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DateTime64>>(res_type, val_type, is_tuple);
|
||||
if (which.idx == TypeIndex::String)
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataString>(res_type, val_type);
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataString>(res_type, val_type, is_tuple);
|
||||
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataGeneric>(res_type, val_type);
|
||||
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataGeneric>(res_type, val_type, is_tuple);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,14 +2,15 @@
|
||||
|
||||
#include <cstddef>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
#include <type_traits>
|
||||
#include <vector>
|
||||
|
||||
#include <common/types.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
|
||||
@ -48,7 +49,9 @@ class IAggregateFunction
|
||||
{
|
||||
public:
|
||||
IAggregateFunction(const DataTypes & argument_types_, const Array & parameters_)
|
||||
: argument_types(argument_types_), parameters(parameters_) {}
|
||||
: argument_types(argument_types_), parameters(parameters_)
|
||||
{
|
||||
}
|
||||
|
||||
/// Get main function name.
|
||||
virtual String getName() const = 0;
|
||||
@ -100,10 +103,7 @@ public:
|
||||
virtual void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const = 0;
|
||||
|
||||
/// Returns true if a function requires Arena to handle own states (see add(), merge(), deserialize()).
|
||||
virtual bool allocatesMemoryInArena() const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
virtual bool allocatesMemoryInArena() const { return false; }
|
||||
|
||||
/// Inserts results into a column. This method might modify the state (e.g.
|
||||
/// sort an array), so must be called once, from single thread. The state
|
||||
@ -177,12 +177,8 @@ public:
|
||||
* "places" contains a large number of same values consecutively.
|
||||
*/
|
||||
virtual void addBatchArray(
|
||||
size_t batch_size,
|
||||
AggregateDataPtr * places,
|
||||
size_t place_offset,
|
||||
const IColumn ** columns,
|
||||
const UInt64 * offsets,
|
||||
Arena * arena) const = 0;
|
||||
size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, const UInt64 * offsets, Arena * arena)
|
||||
const = 0;
|
||||
|
||||
/** The case when the aggregation key is UInt8
|
||||
* and pointers to aggregation states are stored in AggregateDataPtr[256] lookup table.
|
||||
@ -203,8 +199,10 @@ public:
|
||||
* arguments and params are for nested_function.
|
||||
*/
|
||||
virtual AggregateFunctionPtr getOwnNullAdapter(
|
||||
const AggregateFunctionPtr & /*nested_function*/, const DataTypes & /*arguments*/,
|
||||
const Array & /*params*/, const AggregateFunctionProperties & /*properties*/) const
|
||||
const AggregateFunctionPtr & /*nested_function*/,
|
||||
const DataTypes & /*arguments*/,
|
||||
const Array & /*params*/,
|
||||
const AggregateFunctionProperties & /*properties*/) const
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
@ -235,7 +233,9 @@ private:
|
||||
|
||||
public:
|
||||
IAggregateFunctionHelper(const DataTypes & argument_types_, const Array & parameters_)
|
||||
: IAggregateFunction(argument_types_, parameters_) {}
|
||||
: IAggregateFunction(argument_types_, parameters_)
|
||||
{
|
||||
}
|
||||
|
||||
AddFunc getAddressOfAddFunction() const override { return &addFree; }
|
||||
|
||||
@ -387,40 +387,27 @@ class IAggregateFunctionDataHelper : public IAggregateFunctionHelper<Derived>
|
||||
protected:
|
||||
using Data = T;
|
||||
|
||||
static Data & data(AggregateDataPtr place) { return *reinterpret_cast<Data*>(place); }
|
||||
static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast<const Data*>(place); }
|
||||
static Data & data(AggregateDataPtr place) { return *reinterpret_cast<Data *>(place); }
|
||||
static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast<const Data *>(place); }
|
||||
|
||||
public:
|
||||
// Derived class can `override` this to flag that DateTime64 is not supported.
|
||||
static constexpr bool DateTime64Supported = true;
|
||||
|
||||
IAggregateFunctionDataHelper(const DataTypes & argument_types_, const Array & parameters_)
|
||||
: IAggregateFunctionHelper<Derived>(argument_types_, parameters_) {}
|
||||
|
||||
void create(AggregateDataPtr place) const override
|
||||
: IAggregateFunctionHelper<Derived>(argument_types_, parameters_)
|
||||
{
|
||||
new (place) Data;
|
||||
}
|
||||
|
||||
void destroy(AggregateDataPtr place) const noexcept override
|
||||
{
|
||||
data(place).~Data();
|
||||
}
|
||||
void create(AggregateDataPtr place) const override { new (place) Data; }
|
||||
|
||||
bool hasTrivialDestructor() const override
|
||||
{
|
||||
return std::is_trivially_destructible_v<Data>;
|
||||
}
|
||||
void destroy(AggregateDataPtr place) const noexcept override { data(place).~Data(); }
|
||||
|
||||
size_t sizeOfData() const override
|
||||
{
|
||||
return sizeof(Data);
|
||||
}
|
||||
bool hasTrivialDestructor() const override { return std::is_trivially_destructible_v<Data>; }
|
||||
|
||||
size_t alignOfData() const override
|
||||
{
|
||||
return alignof(Data);
|
||||
}
|
||||
size_t sizeOfData() const override { return sizeof(Data); }
|
||||
|
||||
size_t alignOfData() const override { return alignof(Data); }
|
||||
|
||||
void addBatchLookupTable8(
|
||||
size_t batch_size,
|
||||
@ -499,6 +486,135 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
/// Implements tuple argument unwrapper when the tuple just masks arguments
|
||||
template <typename T, typename Derived, size_t args_count>
|
||||
class IAggregateFunctionTupleArgHelper : public IAggregateFunctionDataHelper<T, Derived>
|
||||
{
|
||||
private:
|
||||
using Base = IAggregateFunctionDataHelper<T, Derived>;
|
||||
|
||||
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns_, size_t row_num, Arena * arena)
|
||||
{
|
||||
if (const auto * col = checkAndGetColumn<ColumnTuple>(*columns_[0]))
|
||||
{
|
||||
const IColumn * columns[args_count];
|
||||
const auto & tup_columns = col->getColumns();
|
||||
|
||||
assert(tup_columns.size == args_count);
|
||||
for (size_t i = 0; i < tup_columns.size(); ++i)
|
||||
{
|
||||
columns[i] = tup_columns[i].get();
|
||||
}
|
||||
|
||||
static_cast<const Derived &>(*that).add(place, columns, row_num, arena);
|
||||
}
|
||||
else
|
||||
static_cast<const Derived &>(*that).add(place, columns_, row_num, arena);
|
||||
}
|
||||
|
||||
protected:
|
||||
void extractColumns(const IColumn ** columns, const IColumn ** aggr_columns) const
|
||||
{
|
||||
if (tuple_argument)
|
||||
{
|
||||
auto tup_columns = assert_cast<const ColumnTuple *>(aggr_columns[0])->getColumns();
|
||||
for (size_t i = 0; i < args_count; ++i)
|
||||
columns[i] = tup_columns[i].get();
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < args_count; ++i)
|
||||
columns[i] = aggr_columns[i];
|
||||
}
|
||||
}
|
||||
|
||||
bool tuple_argument;
|
||||
|
||||
public:
|
||||
IAggregateFunctionTupleArgHelper(const DataTypes & argument_types_, const Array & parameters_, bool tuple_argument_)
|
||||
: Base(argument_types_, parameters_)
|
||||
{
|
||||
tuple_argument = tuple_argument_;
|
||||
}
|
||||
|
||||
IAggregateFunction::AddFunc getAddressOfAddFunction() const override { return &addFree; }
|
||||
|
||||
/*
|
||||
* We're overriding addBatch* functions just to avoid extracting columns
|
||||
* in 'add' functions
|
||||
*/
|
||||
void addBatch(
|
||||
size_t batch_size,
|
||||
AggregateDataPtr * places,
|
||||
size_t place_offset,
|
||||
const IColumn ** columns,
|
||||
Arena * arena,
|
||||
ssize_t if_argument_pos = -1) const override
|
||||
{
|
||||
const IColumn * ex_columns[args_count];
|
||||
extractColumns(ex_columns, columns);
|
||||
|
||||
Base::addBatch(batch_size, places, place_offset, ex_columns, arena, if_argument_pos);
|
||||
}
|
||||
|
||||
void addBatchSinglePlace(
|
||||
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const override
|
||||
{
|
||||
const IColumn * ex_columns[args_count];
|
||||
extractColumns(ex_columns, columns);
|
||||
|
||||
Base::addBatchSinglePlace(batch_size, place, ex_columns, arena, if_argument_pos);
|
||||
}
|
||||
|
||||
void addBatchSinglePlaceNotNull(
|
||||
size_t batch_size,
|
||||
AggregateDataPtr place,
|
||||
const IColumn ** columns,
|
||||
const UInt8 * null_map,
|
||||
Arena * arena,
|
||||
ssize_t if_argument_pos = -1) const override
|
||||
{
|
||||
const IColumn * ex_columns[args_count];
|
||||
extractColumns(ex_columns, columns);
|
||||
|
||||
Base::addBatchSinglePlaceNotNull(batch_size, place, ex_columns, null_map, arena, if_argument_pos);
|
||||
}
|
||||
|
||||
void addBatchSinglePlaceFromInterval(
|
||||
size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1)
|
||||
const override
|
||||
{
|
||||
const IColumn * ex_columns[args_count];
|
||||
extractColumns(ex_columns, columns);
|
||||
|
||||
Base::addBatchSinglePlaceFromInterval(batch_begin, batch_end, place, ex_columns, arena, if_argument_pos);
|
||||
}
|
||||
|
||||
void addBatchArray(
|
||||
size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, const UInt64 * offsets, Arena * arena)
|
||||
const override
|
||||
{
|
||||
const IColumn * ex_columns[args_count];
|
||||
extractColumns(ex_columns, columns);
|
||||
|
||||
Base::addBatchArray(batch_size, places, place_offset, ex_columns, offsets, arena);
|
||||
}
|
||||
|
||||
void addBatchLookupTable8(
|
||||
size_t batch_size,
|
||||
AggregateDataPtr * map,
|
||||
size_t place_offset,
|
||||
std::function<void(AggregateDataPtr &)> init,
|
||||
const UInt8 * key,
|
||||
const IColumn ** columns,
|
||||
Arena * arena) const override
|
||||
{
|
||||
const IColumn * ex_columns[args_count];
|
||||
extractColumns(ex_columns, columns);
|
||||
|
||||
Base::addBatchLookupTable8(batch_size, map, place_offset, init, key, ex_columns, arena);
|
||||
}
|
||||
};
|
||||
|
||||
/// Properties of aggregate function that are independent of argument types and parameters.
|
||||
struct AggregateFunctionProperties
|
||||
|
@ -29,7 +29,8 @@ void DataTypeCustomSimpleAggregateFunction::checkSupportedFunctions(const Aggreg
|
||||
{
|
||||
static const std::vector<String> supported_functions{"any", "anyLast", "min",
|
||||
"max", "sum", "sumWithOverflow", "groupBitAnd", "groupBitOr", "groupBitXor",
|
||||
"sumMap", "minMap", "maxMap", "groupArrayArray", "groupUniqArrayArray"};
|
||||
"sumMap", "minMap", "maxMap", "groupArrayArray", "groupUniqArrayArray",
|
||||
"argMin", "argMax"};
|
||||
|
||||
// check function
|
||||
if (std::find(std::begin(supported_functions), std::end(supported_functions), function->getName()) == std::end(supported_functions))
|
||||
|
5
tests/queries/0_stateless/00027_argMinMax.reference
Normal file
5
tests/queries/0_stateless/00027_argMinMax.reference
Normal file
@ -0,0 +1,5 @@
|
||||
0 (0,1) 9 (9,10)
|
||||
0 ('0',1) 9 ('9',10)
|
||||
1970-01-01 ('1970-01-01','1970-01-01 00:00:01') 1970-01-10 ('1970-01-10','1970-01-01 00:00:10')
|
||||
0.00 (0.00,1.00) 9.00 (9.00,10.00)
|
||||
4 1
|
8
tests/queries/0_stateless/00027_argMinMax.sql
Normal file
8
tests/queries/0_stateless/00027_argMinMax.sql
Normal file
@ -0,0 +1,8 @@
|
||||
-- types
|
||||
select argMin(x.1, x.2), argMin(x), argMax(x.1, x.2), argMax(x) from (select (number, number + 1) as x from numbers(10));
|
||||
select argMin(x.1, x.2), argMin(x), argMax(x.1, x.2), argMax(x) from (select (toString(number), toInt32(number) + 1) as x from numbers(10));
|
||||
select argMin(x.1, x.2), argMin(x), argMax(x.1, x.2), argMax(x) from (select (toDate(number, 'UTC'), toDateTime(number, 'UTC') + 1) as x from numbers(10));
|
||||
select argMin(x.1, x.2), argMin(x), argMax(x.1, x.2), argMax(x) from (select (toDecimal32(number, 2), toDecimal64(number, 2) + 1) as x from numbers(10));
|
||||
|
||||
-- array
|
||||
SELECT argMinArray(id, num), argMaxArray(id, num) FROM (SELECT arrayJoin([[10, 4, 3], [7, 5, 6], [8, 8, 2]]) AS num, arrayJoin([[1, 2, 4], [2, 3, 3]]) AS id);
|
@ -1 +0,0 @@
|
||||
4 1
|
@ -1 +0,0 @@
|
||||
SELECT argMinArray(id, num), argMaxArray(id, num) FROM (SELECT arrayJoin([[10, 4, 3], [7, 5, 6], [8, 8, 2]]) AS num, arrayJoin([[1, 2, 4], [2, 3, 3]]) AS id)
|
@ -39,7 +39,7 @@ SimpleAggregateFunction(sum, Float64)
|
||||
7 14
|
||||
8 16
|
||||
9 18
|
||||
1 1 2 2.2.2.2 3 ([1,2,3],[2,1,1]) ([1,2,3],[1,1,2]) ([1,2,3],[2,1,2]) [1,2,2,3,4] [4,2,1,3]
|
||||
10 2222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222 20 20.20.20.20 5 ([2,3,4],[2,1,1]) ([2,3,4],[3,3,4]) ([2,3,4],[4,3,4]) [] []
|
||||
SimpleAggregateFunction(anyLast, Nullable(String)) SimpleAggregateFunction(anyLast, LowCardinality(Nullable(String))) SimpleAggregateFunction(anyLast, IPv4) SimpleAggregateFunction(groupBitOr, UInt32) SimpleAggregateFunction(sumMap, Tuple(Array(Int32), Array(Int64))) SimpleAggregateFunction(minMap, Tuple(Array(Int32), Array(Int64))) SimpleAggregateFunction(maxMap, Tuple(Array(Int32), Array(Int64))) SimpleAggregateFunction(groupArrayArray, Array(Int32)) SimpleAggregateFunction(groupUniqArrayArray, Array(Int32))
|
||||
1 1 2 2.2.2.2 3 ([1,2,3],[2,1,1]) ([1,2,3],[1,1,2]) ([1,2,3],[2,1,2]) [1,2,2,3,4] [4,2,1,3] (1,1) (2,2)
|
||||
10 2222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222 20 20.20.20.20 5 ([2,3,4],[2,1,1]) ([2,3,4],[3,3,4]) ([2,3,4],[4,3,4]) [] [] (3,3) (4,4)
|
||||
SimpleAggregateFunction(anyLast, Nullable(String)) SimpleAggregateFunction(anyLast, LowCardinality(Nullable(String))) SimpleAggregateFunction(anyLast, IPv4) SimpleAggregateFunction(groupBitOr, UInt32) SimpleAggregateFunction(sumMap, Tuple(Array(Int32), Array(Int64))) SimpleAggregateFunction(minMap, Tuple(Array(Int32), Array(Int64))) SimpleAggregateFunction(maxMap, Tuple(Array(Int32), Array(Int64))) SimpleAggregateFunction(groupArrayArray, Array(Int32)) SimpleAggregateFunction(groupUniqArrayArray, Array(Int32)) SimpleAggregateFunction(argMin, Tuple(Int32, Int64)) SimpleAggregateFunction(argMax, Tuple(Int32, Int64))
|
||||
with_overflow 1 0
|
||||
|
@ -31,16 +31,22 @@ create table simple (
|
||||
tup_min SimpleAggregateFunction(minMap, Tuple(Array(Int32), Array(Int64))),
|
||||
tup_max SimpleAggregateFunction(maxMap, Tuple(Array(Int32), Array(Int64))),
|
||||
arr SimpleAggregateFunction(groupArrayArray, Array(Int32)),
|
||||
uniq_arr SimpleAggregateFunction(groupUniqArrayArray, Array(Int32))
|
||||
uniq_arr SimpleAggregateFunction(groupUniqArrayArray, Array(Int32)),
|
||||
arg_min SimpleAggregateFunction(argMin, Tuple(Int32, Int64)),
|
||||
arg_max SimpleAggregateFunction(argMax, Tuple(Int32, Int64))
|
||||
) engine=AggregatingMergeTree order by id;
|
||||
insert into simple values(1,'1','1','1.1.1.1', 1, ([1,2], [1,1]), ([1,2], [1,1]), ([1,2], [1,1]), [1,2], [1,2]);
|
||||
insert into simple values(1,null,'2','2.2.2.2', 2, ([1,3], [1,1]), ([1,3], [2,2]), ([1,3], [2,2]), [2,3,4], [2,3,4]);
|
||||
|
||||
insert into simple values(1,'1','1','1.1.1.1', 1, ([1,2], [1,1]), ([1,2], [1,1]), ([1,2], [1,1]), [1,2], [1,2], (1,1), (1,1));
|
||||
insert into simple values(1,null,'2','2.2.2.2', 2, ([1,3], [1,1]), ([1,3], [2,2]), ([1,3], [2,2]), [2,3,4], [2,3,4], (2,2), (2,2));
|
||||
-- String longer then MAX_SMALL_STRING_SIZE (actual string length is 100)
|
||||
insert into simple values(10,'10','10','10.10.10.10', 4, ([2,3], [1,1]), ([2,3], [3,3]), ([2,3], [3,3]), [], []);
|
||||
insert into simple values(10,'2222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222','20','20.20.20.20', 1, ([2, 4], [1,1]), ([2, 4], [4,4]), ([2, 4], [4,4]), [], []);
|
||||
insert into simple values(10,'10','10','10.10.10.10', 4, ([2,3], [1,1]), ([2,3], [3,3]), ([2,3], [3,3]), [], [], (3,3), (3,3));
|
||||
insert into simple values(10,'2222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222','20','20.20.20.20', 1, ([2, 4], [1,1]), ([2, 4], [4,4]), ([2, 4], [4,4]), [], [], (4,4), (4,4));
|
||||
|
||||
select * from simple final order by id;
|
||||
select toTypeName(nullable_str),toTypeName(low_str),toTypeName(ip),toTypeName(status), toTypeName(tup), toTypeName(tup_min), toTypeName(tup_max), toTypeName(arr), toTypeName(uniq_arr) from simple limit 1;
|
||||
select toTypeName(nullable_str), toTypeName(low_str), toTypeName(ip), toTypeName(status),
|
||||
toTypeName(tup), toTypeName(tup_min), toTypeName(tup_max), toTypeName(arr),
|
||||
toTypeName(uniq_arr), toTypeName(arg_min), toTypeName(arg_max)
|
||||
from simple limit 1;
|
||||
|
||||
optimize table simple final;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user