fixup! Add groupSortedArray() function

This commit is contained in:
Pablo Alegre 2022-02-08 23:08:44 +01:00
parent de7bbc720c
commit 1e4b504ae2
11 changed files with 98 additions and 98 deletions

View File

@ -2,12 +2,12 @@
toc_priority: 108
---
# groupSortedArray {#groupSortedArray}
# groupArraySorted {#groupArraySorted}
Returns an array with the first N items in ascending order.
``` sql
groupSortedArray(N)(column)
groupArraySorted(N)(column)
```
**Arguments**
@ -26,11 +26,11 @@ If the parameter is omitted, default value 10 is used.
Gets the first 10 numbers:
``` sql
SELECT groupSortedArray(10)(number) FROM numbers(100)
SELECT groupArraySorted(10)(number) FROM numbers(100)
```
``` text
┌─groupSortedArray(10)(number)─┐
┌─groupArraySorted(10)(number)─┐
│ [0,1,2,3,4,5,6,7,8,9] │
└──────────────────────────────┘
```
@ -38,11 +38,11 @@ SELECT groupSortedArray(10)(number) FROM numbers(100)
Or the last 10:
``` sql
SELECT groupSortedArray(10)(number, -number) FROM numbers(100)
SELECT groupArraySorted(10)(number, -number) FROM numbers(100)
```
``` text
┌─groupSortedArray(10)(number, negate(number))─┐
┌─groupArraySorted(10)(number, negate(number))─┐
│ [99,98,97,96,95,94,93,92,91,90] │
└──────────────────────────────────────────────┘
```

View File

@ -35,6 +35,7 @@ ClickHouse-specific aggregate functions:
- [groupArrayInsertAt](../../../sql-reference/aggregate-functions/reference/grouparrayinsertat.md)
- [groupArrayMovingAvg](../../../sql-reference/aggregate-functions/reference/grouparraymovingavg.md)
- [groupArrayMovingSum](../../../sql-reference/aggregate-functions/reference/grouparraymovingsum.md)
- [groupArraySorted](../../../sql-reference/aggregate-functions/reference/grouparraysorted.md)
- [groupBitAnd](../../../sql-reference/aggregate-functions/reference/groupbitand.md)
- [groupBitOr](../../../sql-reference/aggregate-functions/reference/groupbitor.md)
- [groupBitXor](../../../sql-reference/aggregate-functions/reference/groupbitxor.md)
@ -42,7 +43,6 @@ ClickHouse-specific aggregate functions:
- [groupBitmapAnd](../../../sql-reference/aggregate-functions/reference/groupbitmapand.md)
- [groupBitmapOr](../../../sql-reference/aggregate-functions/reference/groupbitmapor.md)
- [groupBitmapXor](../../../sql-reference/aggregate-functions/reference/groupbitmapxor.md)
- [groupSortedArray](../../../sql-reference/aggregate-functions/reference/groupsortedarray.md)
- [sumWithOverflow](../../../sql-reference/aggregate-functions/reference/sumwithoverflow.md)
- [sumMap](../../../sql-reference/aggregate-functions/reference/summap.md)
- [minMap](../../../sql-reference/aggregate-functions/reference/minmap.md)

View File

@ -1,5 +1,5 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionGroupSortedArray.h>
#include <AggregateFunctions/AggregateFunctionGroupArraySorted.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeDate.h>
@ -27,51 +27,51 @@ namespace ErrorCodes
namespace
{
template <typename T, bool expr_sorted, typename TColumnB, bool is_plain_b>
class AggregateFunctionGroupSortedArrayNumeric : public AggregateFunctionGroupSortedArray<T, false, expr_sorted, TColumnB, is_plain_b>
class AggregateFunctionGroupArraySortedNumeric : public AggregateFunctionGroupArraySorted<T, false, expr_sorted, TColumnB, is_plain_b>
{
using AggregateFunctionGroupSortedArray<T, false, expr_sorted, TColumnB, is_plain_b>::AggregateFunctionGroupSortedArray;
using AggregateFunctionGroupArraySorted<T, false, expr_sorted, TColumnB, is_plain_b>::AggregateFunctionGroupArraySorted;
};
template <typename T, bool expr_sorted, typename TColumnB, bool is_plain_b>
class AggregateFunctionGroupSortedArrayFieldType
: public AggregateFunctionGroupSortedArray<typename T::FieldType, false, expr_sorted, TColumnB, is_plain_b>
class AggregateFunctionGroupArraySortedFieldType
: public AggregateFunctionGroupArraySorted<typename T::FieldType, false, expr_sorted, TColumnB, is_plain_b>
{
using AggregateFunctionGroupSortedArray<typename T::FieldType, false, expr_sorted, TColumnB, is_plain_b>::
AggregateFunctionGroupSortedArray;
using AggregateFunctionGroupArraySorted<typename T::FieldType, false, expr_sorted, TColumnB, is_plain_b>::
AggregateFunctionGroupArraySorted;
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeArray>(std::make_shared<T>()); }
};
template <bool expr_sorted = false, typename TColumnB = UInt64, bool is_plain_b = false>
AggregateFunctionPtr
createAggregateFunctionGroupSortedArrayTyped(const DataTypes & argument_types, const Array & params, UInt64 threshold)
createAggregateFunctionGroupArraySortedTyped(const DataTypes & argument_types, const Array & params, UInt64 threshold)
{
#define DISPATCH(A, C, B) \
if (which.idx == TypeIndex::A) \
return AggregateFunctionPtr(new C<B, expr_sorted, TColumnB, is_plain_b>(threshold, argument_types, params));
#define DISPATCH_NUMERIC(A) DISPATCH(A, AggregateFunctionGroupSortedArrayNumeric, A)
#define DISPATCH_NUMERIC(A) DISPATCH(A, AggregateFunctionGroupArraySortedNumeric, A)
WhichDataType which(argument_types[0]);
FOR_NUMERIC_TYPES(DISPATCH_NUMERIC)
DISPATCH(Enum8, AggregateFunctionGroupSortedArrayNumeric, Int8)
DISPATCH(Enum16, AggregateFunctionGroupSortedArrayNumeric, Int16)
DISPATCH(Date, AggregateFunctionGroupSortedArrayFieldType, DataTypeDate)
DISPATCH(DateTime, AggregateFunctionGroupSortedArrayFieldType, DataTypeDateTime)
DISPATCH(Enum8, AggregateFunctionGroupArraySortedNumeric, Int8)
DISPATCH(Enum16, AggregateFunctionGroupArraySortedNumeric, Int16)
DISPATCH(Date, AggregateFunctionGroupArraySortedFieldType, DataTypeDate)
DISPATCH(DateTime, AggregateFunctionGroupArraySortedFieldType, DataTypeDateTime)
#undef DISPATCH
#undef DISPATCH_NUMERIC
if (argument_types[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
{
return AggregateFunctionPtr(new AggregateFunctionGroupSortedArray<StringRef, true, expr_sorted, TColumnB, is_plain_b>(
return AggregateFunctionPtr(new AggregateFunctionGroupArraySorted<StringRef, true, expr_sorted, TColumnB, is_plain_b>(
threshold, argument_types, params));
}
else
{
return AggregateFunctionPtr(new AggregateFunctionGroupSortedArray<StringRef, false, expr_sorted, TColumnB, is_plain_b>(
return AggregateFunctionPtr(new AggregateFunctionGroupArraySorted<StringRef, false, expr_sorted, TColumnB, is_plain_b>(
threshold, argument_types, params));
}
}
AggregateFunctionPtr createAggregateFunctionGroupSortedArray(
AggregateFunctionPtr createAggregateFunctionGroupArraySorted(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
UInt64 threshold = GROUP_SORTED_ARRAY_DEFAULT_THRESHOLD;
@ -101,7 +101,7 @@ namespace
{
#define DISPATCH2(A, B) \
if (which.idx == TypeIndex::A) \
return createAggregateFunctionGroupSortedArrayTyped<true, B>(argument_types, params, threshold);
return createAggregateFunctionGroupArraySortedTyped<true, B>(argument_types, params, threshold);
#define DISPATCH(A) DISPATCH2(A, A)
WhichDataType which(argument_types[1]);
FOR_NUMERIC_TYPES(DISPATCH)
@ -113,16 +113,16 @@ namespace
}
else if (argument_types[1]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
{
return createAggregateFunctionGroupSortedArrayTyped<true, StringRef, true>(argument_types, params, threshold);
return createAggregateFunctionGroupArraySortedTyped<true, StringRef, true>(argument_types, params, threshold);
}
else
{
return createAggregateFunctionGroupSortedArrayTyped<true, StringRef, false>(argument_types, params, threshold);
return createAggregateFunctionGroupArraySortedTyped<true, StringRef, false>(argument_types, params, threshold);
}
}
else if (argument_types.size() == 1)
{
return createAggregateFunctionGroupSortedArrayTyped<>(argument_types, params, threshold);
return createAggregateFunctionGroupArraySortedTyped<>(argument_types, params, threshold);
}
else
{
@ -132,9 +132,9 @@ namespace
}
}
void registerAggregateFunctionGroupSortedArray(AggregateFunctionFactory & factory)
void registerAggregateFunctionGroupArraySorted(AggregateFunctionFactory & factory)
{
AggregateFunctionProperties properties = {.returns_default_when_only_null = false, .is_order_dependent = true};
factory.registerFunction("groupSortedArray", {createAggregateFunctionGroupSortedArray, properties});
factory.registerFunction("groupArraySorted", {createAggregateFunctionGroupArraySorted, properties});
}
}

View File

@ -3,7 +3,7 @@
#include <Columns/ColumnArray.h>
#include <DataTypes/DataTypeArray.h>
#include <AggregateFunctions/AggregateFunctionGroupSortedArrayData.h>
#include <AggregateFunctions/AggregateFunctionGroupArraySortedData.h>
#include <AggregateFunctions/IAggregateFunction.h>
namespace DB
@ -77,15 +77,15 @@ size_t getFirstNElements(const TColumn * data, int num_elements, int threshold,
}
template <typename TColumnA, bool is_plain_a, bool use_column_b, typename TColumnB, bool is_plain_b>
class AggregateFunctionGroupSortedArray : public IAggregateFunctionDataHelper<
AggregateFunctionGroupSortedArrayData<TColumnA, use_column_b, TColumnB>,
AggregateFunctionGroupSortedArray<TColumnA, is_plain_a, use_column_b, TColumnB, is_plain_b>>
class AggregateFunctionGroupArraySorted : public IAggregateFunctionDataHelper<
AggregateFunctionGroupArraySortedData<TColumnA, use_column_b, TColumnB>,
AggregateFunctionGroupArraySorted<TColumnA, is_plain_a, use_column_b, TColumnB, is_plain_b>>
{
protected:
using State = AggregateFunctionGroupSortedArrayData<TColumnA, use_column_b, TColumnB>;
using State = AggregateFunctionGroupArraySortedData<TColumnA, use_column_b, TColumnB>;
using Base = IAggregateFunctionDataHelper<
AggregateFunctionGroupSortedArrayData<TColumnA, use_column_b, TColumnB>,
AggregateFunctionGroupSortedArray>;
AggregateFunctionGroupArraySortedData<TColumnA, use_column_b, TColumnB>,
AggregateFunctionGroupArraySorted>;
UInt64 threshold;
DataTypePtr & input_data_type;
@ -94,10 +94,10 @@ protected:
static void deserializeAndInsert(StringRef str, IColumn & data_to);
public:
AggregateFunctionGroupSortedArray(UInt64 threshold_, const DataTypes & argument_types_, const Array & params)
AggregateFunctionGroupArraySorted(UInt64 threshold_, const DataTypes & argument_types_, const Array & params)
: IAggregateFunctionDataHelper<
AggregateFunctionGroupSortedArrayData<TColumnA, use_column_b, TColumnB>,
AggregateFunctionGroupSortedArray>(argument_types_, params)
AggregateFunctionGroupArraySortedData<TColumnA, use_column_b, TColumnB>,
AggregateFunctionGroupArraySorted>(argument_types_, params)
, threshold(threshold_)
, input_data_type(this->argument_types[0])
{
@ -109,7 +109,7 @@ public:
this->data(place).threshold = threshold;
}
String getName() const override { return "groupSortedArray"; }
String getName() const override { return "groupArraySorted"; }
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeArray>(input_data_type); }

View File

@ -52,19 +52,19 @@ static void readOneItem(ReadBuffer & buf, Arena * arena, StringRef & item)
}
template <typename Storage>
struct AggregateFunctionGroupSortedArrayDataBase
struct AggregateFunctionGroupArraySortedDataBase
{
typedef typename Storage::value_type ValueType;
AggregateFunctionGroupSortedArrayDataBase(UInt64 threshold_ = GROUP_SORTED_DEFAULT_THRESHOLD) : threshold(threshold_) { }
AggregateFunctionGroupArraySortedDataBase(UInt64 threshold_ = GROUP_SORTED_DEFAULT_THRESHOLD) : threshold(threshold_) { }
virtual ~AggregateFunctionGroupSortedArrayDataBase() { }
virtual ~AggregateFunctionGroupArraySortedDataBase() { }
inline void narrowDown()
{
while (values.size() > threshold)
values.erase(--values.end());
}
void merge(const AggregateFunctionGroupSortedArrayDataBase & other)
void merge(const AggregateFunctionGroupArraySortedDataBase & other)
{
values.merge(Storage(other.values));
narrowDown();
@ -101,14 +101,14 @@ struct AggregateFunctionGroupSortedArrayDataBase
};
template <typename T, bool expr_sorted, typename TIndex>
struct AggregateFunctionGroupSortedArrayData
struct AggregateFunctionGroupArraySortedData
{
};
template <typename T, typename TIndex>
struct AggregateFunctionGroupSortedArrayData<T, true, TIndex> : public AggregateFunctionGroupSortedArrayDataBase<std::multimap<TIndex, T>>
struct AggregateFunctionGroupArraySortedData<T, true, TIndex> : public AggregateFunctionGroupArraySortedDataBase<std::multimap<TIndex, T>>
{
using Base = AggregateFunctionGroupSortedArrayDataBase<std::multimap<TIndex, T>>;
using Base = AggregateFunctionGroupArraySortedDataBase<std::multimap<TIndex, T>>;
using Base::Base;
void add(T item, TIndex weight)
@ -137,9 +137,9 @@ struct AggregateFunctionGroupSortedArrayData<T, true, TIndex> : public Aggregate
};
template <typename T, typename TIndex>
struct AggregateFunctionGroupSortedArrayData<T, false, TIndex> : public AggregateFunctionGroupSortedArrayDataBase<std::multiset<T>>
struct AggregateFunctionGroupArraySortedData<T, false, TIndex> : public AggregateFunctionGroupArraySortedDataBase<std::multiset<T>>
{
using Base = AggregateFunctionGroupSortedArrayDataBase<std::multiset<T>>;
using Base = AggregateFunctionGroupArraySortedDataBase<std::multiset<T>>;
using Base::Base;
void add(T item)

View File

@ -59,7 +59,7 @@ void registerAggregateFunctionNothing(AggregateFunctionFactory &);
void registerAggregateFunctionExponentialMovingAverage(AggregateFunctionFactory &);
void registerAggregateFunctionSparkbar(AggregateFunctionFactory &);
void registerAggregateFunctionIntervalLengthSum(AggregateFunctionFactory &);
void registerAggregateFunctionGroupSortedArray(AggregateFunctionFactory & factory);
void registerAggregateFunctionGroupArraySorted(AggregateFunctionFactory & factory);
class AggregateFunctionCombinatorFactory;
void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &);
@ -131,7 +131,7 @@ void registerAggregateFunctions()
registerAggregateFunctionIntervalLengthSum(factory);
registerAggregateFunctionExponentialMovingAverage(factory);
registerAggregateFunctionSparkbar(factory);
registerAggregateFunctionGroupSortedArray(factory);
registerAggregateFunctionGroupArraySorted(factory);
registerWindowFunctions(factory);
}

View File

@ -15,11 +15,11 @@
<create_query>CREATE TABLE test ( `id` UInt64, `value` UInt64, `text` String ) ENGINE = Memory</create_query>
<fill_query>INSERT INTO test SELECT number as id, rand64() as value, toString(number) as text FROM numbers({items})</fill_query>
<query>SELECT groupSortedArray(10)(id, value) FROM test</query>
<query>SELECT groupSortedArray(10)(id, -value) FROM test</query>
<query>SELECT groupSortedArray(10)(text, value) FROM test</query>
<query>SELECT groupSortedArray(10)((id, text), value) FROM test</query>
<query>SELECT groupSortedArray(10)(value) FROM test</query>
<query>SELECT groupSortedArray(10)(text) FROM test</query>
<query>SELECT groupArraySorted(10)(id, value) FROM test</query>
<query>SELECT groupArraySorted(10)(id, -value) FROM test</query>
<query>SELECT groupArraySorted(10)(text, value) FROM test</query>
<query>SELECT groupArraySorted(10)((id, text), value) FROM test</query>
<query>SELECT groupArraySorted(10)(value) FROM test</query>
<query>SELECT groupArraySorted(10)(text) FROM test</query>
<drop_query>DROP TABLE IF EXISTS test</drop_query>
</test>

View File

@ -0,0 +1,41 @@
SELECT groupArraySorted(5)(number) from numbers(100);
SELECT groupArraySorted(number, number) from numbers(100);
SELECT groupArraySorted(100)(number, number) from numbers(1000);
SELECT groupArraySorted(100)(number, -number) from numbers(1000);
SELECT groupArraySorted(5)(str, number) FROM (SELECT toString(number) as str, number FROM numbers(10));
SELECT groupArraySorted(5)(text) FROM (select toString(number) as text from numbers(10));
SELECT groupArraySorted(5)(text, -number) FROM (select toString(number) as text, number from numbers(10));
SELECT groupArraySorted(5)((number,text)) from (SELECT toString(number) as text, number FROM numbers(100));
SELECT groupArraySorted(5)(text,text) from (SELECT toString(number) as text FROM numbers(100));
SELECT groupArraySorted(50)(text,(number,text)) from (SELECT toString(number) as text, number FROM numbers(100));
DROP TABLE IF EXISTS test;
DROP VIEW IF EXISTS mv_test;
CREATE TABLE test (`n` String, `h` Int64) ENGINE = MergeTree ORDER BY n;
CREATE MATERIALIZED VIEW mv_test (`n` String, `h` AggregateFunction(groupArraySorted(2), Int64, Int64)) ENGINE = AggregatingMergeTree ORDER BY n AS SELECT n, groupArraySortedState(2)(h, h) as h FROM test GROUP BY n;
INSERT INTO test VALUES ('pablo',1)('pablo', 2)('luis', 1)('luis', 3)('pablo', 5)('pablo',4)('pablo', 5)('luis', 6)('luis', 7)('pablo', 8)('pablo',9)('pablo',10)('luis',11)('luis',12)('pablo',13);
SELECT n, groupArraySortedMerge(2)(h) from mv_test GROUP BY n;
DROP TABLE IF EXISTS test;
DROP VIEW IF EXISTS mv_test;
CREATE TABLE test (`n` String, `h` Int64) ENGINE = MergeTree ORDER BY n;
CREATE MATERIALIZED VIEW mv_test (`n` String, `h` AggregateFunction(groupArraySorted(2), Int64)) ENGINE = AggregatingMergeTree ORDER BY n AS SELECT n, groupArraySortedState(2)(h) as h FROM test GROUP BY n;
INSERT INTO test VALUES ('pablo',1)('pablo', 2)('luis', 1)('luis', 3)('pablo', 5)('pablo',4)('pablo', 5)('luis', 6)('luis', 7)('pablo', 8)('pablo',9)('pablo',10)('luis',11)('luis',12)('pablo',13);
SELECT n, groupArraySortedMerge(2)(h) from mv_test GROUP BY n;
DROP TABLE test;
DROP VIEW mv_test;
SELECT groupArraySortedIf(5)(number, number, number>3) from numbers(100);
SELECT groupArraySortedIf(5)(number, toString(number), number>3) from numbers(100);
SELECT groupArraySortedIf(5)(toString(number), number>3) from numbers(100);

View File

@ -1,41 +0,0 @@
SELECT groupSortedArray(5)(number) from numbers(100);
SELECT groupSortedArray(number, number) from numbers(100);
SELECT groupSortedArray(100)(number, number) from numbers(1000);
SELECT groupSortedArray(100)(number, -number) from numbers(1000);
SELECT groupSortedArray(5)(str, number) FROM (SELECT toString(number) as str, number FROM numbers(10));
SELECT groupSortedArray(5)(text) FROM (select toString(number) as text from numbers(10));
SELECT groupSortedArray(5)(text, -number) FROM (select toString(number) as text, number from numbers(10));
SELECT groupSortedArray(5)((number,text)) from (SELECT toString(number) as text, number FROM numbers(100));
SELECT groupSortedArray(5)(text,text) from (SELECT toString(number) as text FROM numbers(100));
SELECT groupSortedArray(50)(text,(number,text)) from (SELECT toString(number) as text, number FROM numbers(100));
DROP TABLE IF EXISTS test;
DROP VIEW IF EXISTS mv_test;
CREATE TABLE test (`n` String, `h` Int64) ENGINE = MergeTree ORDER BY n;
CREATE MATERIALIZED VIEW mv_test (`n` String, `h` AggregateFunction(groupSortedArray(2), Int64, Int64)) ENGINE = AggregatingMergeTree ORDER BY n AS SELECT n, groupSortedArrayState(2)(h, h) as h FROM test GROUP BY n;
INSERT INTO test VALUES ('pablo',1)('pablo', 2)('luis', 1)('luis', 3)('pablo', 5)('pablo',4)('pablo', 5)('luis', 6)('luis', 7)('pablo', 8)('pablo',9)('pablo',10)('luis',11)('luis',12)('pablo',13);
SELECT n, groupSortedArrayMerge(2)(h) from mv_test GROUP BY n;
DROP TABLE IF EXISTS test;
DROP VIEW IF EXISTS mv_test;
CREATE TABLE test (`n` String, `h` Int64) ENGINE = MergeTree ORDER BY n;
CREATE MATERIALIZED VIEW mv_test (`n` String, `h` AggregateFunction(groupSortedArray(2), Int64)) ENGINE = AggregatingMergeTree ORDER BY n AS SELECT n, groupSortedArrayState(2)(h) as h FROM test GROUP BY n;
INSERT INTO test VALUES ('pablo',1)('pablo', 2)('luis', 1)('luis', 3)('pablo', 5)('pablo',4)('pablo', 5)('luis', 6)('luis', 7)('pablo', 8)('pablo',9)('pablo',10)('luis',11)('luis',12)('pablo',13);
SELECT n, groupSortedArrayMerge(2)(h) from mv_test GROUP BY n;
DROP TABLE test;
DROP VIEW mv_test;
SELECT groupSortedArrayIf(5)(number, number, number>3) from numbers(100);
SELECT groupSortedArrayIf(5)(number, toString(number), number>3) from numbers(100);
SELECT groupSortedArrayIf(5)(toString(number), number>3) from numbers(100);