mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
fixup! Add groupSortedArray() function
This commit is contained in:
parent
de7bbc720c
commit
1e4b504ae2
@ -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] │
|
||||
└──────────────────────────────────────────────┘
|
||||
```
|
@ -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)
|
||||
|
@ -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});
|
||||
}
|
||||
}
|
@ -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); }
|
||||
|
@ -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)
|
@ -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);
|
||||
}
|
||||
|
@ -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>
|
41
tests/queries/0_stateless/02158_grouparraysorted.sql
Normal file
41
tests/queries/0_stateless/02158_grouparraysorted.sql
Normal 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);
|
@ -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);
|
Loading…
Reference in New Issue
Block a user