This commit is contained in:
Alexey Milovidov 2021-04-21 00:22:29 +03:00
parent 022b2e918b
commit 502bc3575c
14 changed files with 106 additions and 317 deletions

View File

@ -6,20 +6,12 @@ toc_priority: 106
Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered.
Tuple version of this function will return the tuple with the maximum `val` value. It is convenient for use with [SimpleAggregateFunction](../../../sql-reference/data-types/simpleaggregatefunction.md).
**Syntax**
``` sql
argMax(arg, val)
```
or
``` sql
argMax(tuple(arg, val))
```
**Arguments**
- `arg` — Argument.
@ -29,13 +21,7 @@ argMax(tuple(arg, val))
- `arg` value that corresponds to maximum `val` value.
Type: matches `arg` type.
For tuple in the input:
- Tuple `(arg, val)`, where `val` is the maximum value and `arg` is a corresponding value.
Type: [Tuple](../../../sql-reference/data-types/tuple.md).
Type: matches `arg` type.
**Example**
@ -52,15 +38,13 @@ Input table:
Query:
``` sql
SELECT argMax(user, salary), argMax(tuple(user, salary), salary), argMax(tuple(user, salary)) FROM salary;
SELECT argMax(user, salary) FROM salary;
```
Result:
``` text
┌─argMax(user, salary)─┬─argMax(tuple(user, salary), salary)─┬─argMax(tuple(user, salary))─
│ director │ ('director',5000) │ ('director',5000) │
└──────────────────────┴─────────────────────────────────────┴─────────────────────────────
┌─argMax(user, salary)─┐
│ director │
└──────────────────────┘
```
[Original article](https://clickhouse.tech/docs/en/sql-reference/aggregate-functions/reference/argmax/) <!--hide-->

View File

@ -6,20 +6,12 @@ toc_priority: 105
Calculates the `arg` value for a minimum `val` value. If there are several different values of `arg` for minimum values of `val`, returns the first of these values encountered.
Tuple version of this function will return the tuple with the minimum `val` value. It is convenient for use with [SimpleAggregateFunction](../../../sql-reference/data-types/simpleaggregatefunction.md).
**Syntax**
``` sql
argMin(arg, val)
```
or
``` sql
argMin(tuple(arg, val))
```
**Arguments**
- `arg` — Argument.
@ -29,13 +21,7 @@ argMin(tuple(arg, val))
- `arg` value that corresponds to minimum `val` value.
Type: matches `arg` type.
For tuple in the input:
- Tuple `(arg, val)`, where `val` is the minimum value and `arg` is a corresponding value.
Type: [Tuple](../../../sql-reference/data-types/tuple.md).
Type: matches `arg` type.
**Example**
@ -52,15 +38,13 @@ Input table:
Query:
``` sql
SELECT argMin(user, salary), argMin(tuple(user, salary)) FROM salary;
SELECT argMin(user, salary) FROM salary
```
Result:
``` text
┌─argMin(user, salary)─┬─argMin(tuple(user, salary))─
│ worker │ ('worker',1000) │
└──────────────────────┴─────────────────────────────
┌─argMin(user, salary)─┐
│ worker │
└──────────────────────┘
```
[Original article](https://clickhouse.tech/docs/en/sql-reference/aggregate-functions/reference/argmin/) <!--hide-->

View File

@ -18,8 +18,6 @@ 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)
!!! note "Note"

View File

@ -1,16 +1,14 @@
#pragma once
#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"
#include <DataTypes/IDataType.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/AggregateFunctionMinMaxAny.h> // SingleValueDataString used in embedded compiler
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
@ -24,53 +22,44 @@ 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 String name() { return StringRef(ValueData_t::name()) == StringRef("min") ? "argMin" : "argMax"; }
static bool allocatesMemoryInArena()
{
return ResultData::allocatesMemoryInArena() || ValueData::allocatesMemoryInArena();
}
};
/// Returns the first arg value found for the minimum/maximum value. Example: argMax(arg, value).
template <typename Data>
class AggregateFunctionArgMinMax final : public IAggregateFunctionTupleArgHelper<Data, AggregateFunctionArgMinMax<Data>, 2>
class AggregateFunctionArgMinMax final : public IAggregateFunctionDataHelper<Data, AggregateFunctionArgMinMax<Data>>
{
private:
const DataTypePtr & type_res;
const DataTypePtr & type_val;
const SerializationPtr serialization_res;
const SerializationPtr serialization_val;
bool tuple_argument;
using Base = IAggregateFunctionTupleArgHelper<Data, AggregateFunctionArgMinMax<Data>, 2>;
using Base = IAggregateFunctionDataHelper<Data, AggregateFunctionArgMinMax<Data>>;
public:
AggregateFunctionArgMinMax(const DataTypePtr & type_res_, const DataTypePtr & type_val_, const bool tuple_argument_)
: Base({type_res_, type_val_}, {}, tuple_argument_)
AggregateFunctionArgMinMax(const DataTypePtr & type_res_, const DataTypePtr & type_val_)
: Base({type_res_, type_val_}, {})
, type_res(this->argument_types[0])
, type_val(this->argument_types[1])
, serialization_res(type_res->getDefaultSerialization())
, serialization_val(type_val->getDefaultSerialization())
{
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);
this->tuple_argument = tuple_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);
}
String getName() const override { return Data::name(); }
String getName() const override { return StringRef(Data::ValueData_t::name()) == StringRef("min") ? "argMin" : "argMax"; }
DataTypePtr getReturnType() const override
{
if (tuple_argument)
{
return std::make_shared<DataTypeTuple>(DataTypes{this->type_res, this->type_val});
}
return type_res;
}
@ -98,21 +87,15 @@ public:
this->data(place).value.read(buf, *serialization_val, arena);
}
bool allocatesMemoryInArena() const override { return Data::allocatesMemoryInArena(); }
bool allocatesMemoryInArena() const override
{
return Data::allocatesMemoryInArena();
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
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);
this->data(place).result.insertResultInto(to);
}
};
}

View File

@ -31,12 +31,6 @@
M(Float32) \
M(Float64)
#define FOR_DECIMAL_TYPES(M) \
M(Decimal32) \
M(Decimal64) \
M(Decimal128)
namespace DB
{

View File

@ -8,14 +8,10 @@
#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>
@ -30,7 +26,6 @@ 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)
@ -39,6 +34,12 @@ 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);
@ -48,77 +49,66 @@ 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, bool is_tuple)
static IAggregateFunction * createAggregateFunctionArgMinMaxSecond(const DataTypePtr & res_type, const DataTypePtr & val_type)
{
WhichDataType which(val_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) \
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<TYPE>>>>(res_type, val_type, is_tuple);
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<TYPE>>>>(res_type, val_type);
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, is_tuple);
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDate::FieldType>>>>(res_type, val_type);
if (which.idx == TypeIndex::DateTime)
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDateTime::FieldType>>>>(res_type, val_type, is_tuple);
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DataTypeDateTime::FieldType>>>>(res_type, val_type);
if (which.idx == TypeIndex::DateTime64)
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<DateTime64>>>>(res_type, val_type, is_tuple);
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);
if (which.idx == TypeIndex::String)
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<SingleValueDataString>>>(res_type, val_type);
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);
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];
}
const DataTypePtr & res_type = argument_types[0];
const DataTypePtr & 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, is_tuple);
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<TYPE>>(res_type, val_type);
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, is_tuple);
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDate::FieldType>>(res_type, val_type);
if (which.idx == TypeIndex::DateTime)
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDateTime::FieldType>>(res_type, val_type, is_tuple);
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DataTypeDateTime::FieldType>>(res_type, val_type);
if (which.idx == TypeIndex::DateTime64)
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<DateTime64>>(res_type, val_type, is_tuple);
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);
if (which.idx == TypeIndex::String)
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataString>(res_type, val_type, is_tuple);
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataString>(res_type, val_type);
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataGeneric>(res_type, val_type, is_tuple);
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataGeneric>(res_type, val_type);
}
}

View File

@ -11,8 +11,8 @@
#include <cstddef>
#include <memory>
#include <type_traits>
#include <vector>
#include <type_traits>
namespace DB
@ -52,9 +52,7 @@ 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;
@ -180,8 +178,12 @@ 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.
@ -202,10 +204,8 @@ 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;
}
@ -250,9 +250,7 @@ private:
public:
IAggregateFunctionHelper(const DataTypes & argument_types_, const Array & parameters_)
: IAggregateFunction(argument_types_, parameters_)
{
}
: IAggregateFunction(argument_types_, parameters_) {}
AddFunc getAddressOfAddFunction() const override { return &addFree; }
@ -414,19 +412,31 @@ public:
static constexpr bool DateTime64Supported = true;
IAggregateFunctionDataHelper(const DataTypes & argument_types_, const Array & parameters_)
: IAggregateFunctionHelper<Derived>(argument_types_, parameters_)
: IAggregateFunctionHelper<Derived>(argument_types_, parameters_) {}
void create(AggregateDataPtr place) const override
{
new (place) Data;
}
void create(AggregateDataPtr __restrict place) const override { new (place) Data; }
void destroy(AggregateDataPtr __restrict place) const noexcept override { data(place).~Data(); }
bool hasTrivialDestructor() const override { return std::is_trivially_destructible_v<Data>; }
bool hasTrivialDestructor() const override
{
return std::is_trivially_destructible_v<Data>;
}
size_t sizeOfData() const override { return sizeof(Data); }
size_t sizeOfData() const override
{
return sizeof(Data);
}
size_t alignOfData() const override { return alignof(Data); }
size_t alignOfData() const override
{
return alignof(Data);
}
void addBatchLookupTable8(
size_t batch_size,
@ -505,142 +515,6 @@ 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:
ssize_t extractColumns(const IColumn ** columns, const IColumn ** aggr_columns, ssize_t if_argument_pos) 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];
}
if (if_argument_pos >= 0)
{
columns[args_count] = aggr_columns[if_argument_pos];
return args_count;
}
else
return -1;
}
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 + (if_argument_pos >= 0)];
if_argument_pos = extractColumns(ex_columns, columns, if_argument_pos);
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 + (if_argument_pos >= 0)];
if_argument_pos = extractColumns(ex_columns, columns, if_argument_pos);
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 + (if_argument_pos >= 0)];
if_argument_pos = extractColumns(ex_columns, columns, if_argument_pos);
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 + (if_argument_pos >= 0)];
if_argument_pos = extractColumns(ex_columns, columns, if_argument_pos);
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, -1);
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, -1);
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

View File

@ -29,8 +29,7 @@ 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",
"argMin", "argMax"};
"sumMap", "minMap", "maxMap", "groupArrayArray", "groupUniqArrayArray"};
// check function
if (std::find(std::begin(supported_functions), std::end(supported_functions), function->getName()) == std::end(supported_functions))

View File

@ -1,5 +0,0 @@
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

View File

@ -1,8 +0,0 @@
-- 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);

View File

@ -0,0 +1 @@
4 1

View File

@ -0,0 +1 @@
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)

View File

@ -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] (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))
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))
with_overflow 1 0

View File

@ -31,22 +31,16 @@ 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)),
arg_min SimpleAggregateFunction(argMin, Tuple(Int32, Int64)),
arg_max SimpleAggregateFunction(argMax, Tuple(Int32, Int64))
uniq_arr SimpleAggregateFunction(groupUniqArrayArray, Array(Int32))
) 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], (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));
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]);
-- 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]), [], [], (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));
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]), [], []);
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), toTypeName(arg_min), toTypeName(arg_max)
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) from simple limit 1;
optimize table simple final;