From 030e5e625ea4143d98c3fd56d522d862241c8474 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 5 May 2021 17:11:56 +0200 Subject: [PATCH 01/10] add Map combinator --- .../AggregateFunctionMap.cpp | 125 +++++++++++ src/AggregateFunctions/AggregateFunctionMap.h | 206 ++++++++++++++++++ .../AggregateFunctionSumMap.cpp | 19 +- .../AggregateFunctionSumMap.h | 15 +- .../registerAggregateFunctions.cpp | 2 + 5 files changed, 357 insertions(+), 10 deletions(-) create mode 100644 src/AggregateFunctions/AggregateFunctionMap.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionMap.h diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp new file mode 100644 index 00000000000..581587e91d3 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -0,0 +1,125 @@ +#include "AggregateFunctionMap.h" +#include "AggregateFunctions/AggregateFunctionCombinatorFactory.h" +#include "Functions/FunctionHelpers.h" + +namespace DB +{ +class AggregateFunctionCombinatorMap final : public IAggregateFunctionCombinator +{ +public: + String getName() const override { return "Map"; } + + DataTypes transformArguments(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Incorrect number of arguments for aggregate function with " + getName() + " suffix"); + + const auto * map_type = checkAndGetDataType(arguments[0].get()); + if (map_type) + return DataTypes({map_type->getValueType()}); + + // we need this part just to pass to redirection for mapped arrays + const auto * tup_type = checkAndGetDataType(arguments[0].get()); + if (tup_type) + { + const auto * val_array_type = checkAndGetDataType(tup_type->getElements()[1].get()); + if (val_array_type) + return DataTypes({val_array_type->getNestedType()}); + } + + if (arguments.size() >= 2) + { + const auto * val_array_type = checkAndGetDataType(arguments[1].get()); + if (val_array_type) + return DataTypes({val_array_type->getNestedType()}); + } + + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function " + getName() + " requires map as argument"); + } + + AggregateFunctionPtr transformAggregateFunction( + const AggregateFunctionPtr & nested_function, + const AggregateFunctionProperties &, + const DataTypes & arguments, + const Array & params) const override + { + const auto * map_type = checkAndGetDataType(arguments[0].get()); + if (map_type) + { + auto key_type_id = map_type->getKeyType()->getTypeId(); + + switch (key_type_id) + { + case TypeIndex::Enum8: + case TypeIndex::Int8: + return std::make_shared>(nested_function, arguments); + case TypeIndex::Enum16: + case TypeIndex::Int16: + return std::make_shared>(nested_function, arguments); + case TypeIndex::Int32: + return std::make_shared>(nested_function, arguments); + case TypeIndex::Int64: + return std::make_shared>(nested_function, arguments); + case TypeIndex::UInt8: + return std::make_shared>(nested_function, arguments); + case TypeIndex::Date: + case TypeIndex::UInt16: + return std::make_shared>(nested_function, arguments); + case TypeIndex::DateTime: + case TypeIndex::UInt32: + return std::make_shared>(nested_function, arguments); + case TypeIndex::UInt64: + return std::make_shared>(nested_function, arguments); + case TypeIndex::UUID: + return std::make_shared>(nested_function, arguments); + case TypeIndex::FixedString: + case TypeIndex::String: + return std::make_shared>(nested_function, arguments); + default: + throw Exception{"Illegal columns in arguments for combinator " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + } + else if (!arguments.empty()) + { + // check if we got tuple of arrays or just arrays and if so, try to redirect to sum/min/max-MappedArrays to implement old behavior + auto nested_func_name = nested_function->getName(); + if (nested_func_name == "sum" || nested_func_name == "min" || nested_func_name == "max") + { + bool match; + const auto * tup_type = checkAndGetDataType(arguments[0].get()); + auto check_func = [](DataTypePtr t) { + return t->getTypeId() == TypeIndex::Array; + }; + + if (tup_type) + { + const auto & types = tup_type->getElements(); + match = arguments.size() == 1 && types.size() >= 2 && std::all_of(types.begin(), types.end(), check_func); + } + else + { + // sumMappedArrays and others support more than 2 mapped arrays + match = arguments.size() >= 2 && std::all_of(arguments.begin(), arguments.end(), check_func); + } + + if (match) + { + AggregateFunctionProperties out_properties; + auto & aggr_func_factory = AggregateFunctionFactory::instance(); + return aggr_func_factory.get(nested_func_name + "MappedArrays", arguments, params, out_properties); + } + } + } + + throw Exception{"Illegal columns in arguments for combinator " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } +}; + +void registerAggregateFunctionCombinatorMap(AggregateFunctionCombinatorFactory & factory) +{ + factory.registerCombinator(std::make_shared()); +} + +} diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h new file mode 100644 index 00000000000..02715b40c90 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -0,0 +1,206 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "AggregateFunctions/AggregateFunctionFactory.h" + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +template +struct AggregateFunctionMapCombinatorData +{ + std::unordered_map merged_maps; + + static void writeKey(KeyType key, WriteBuffer & buf) { writeBinary(key, buf); } + static void readKey(KeyType & key, ReadBuffer & buf) { readBinary(key, buf); } +}; + +template <> +struct AggregateFunctionMapCombinatorData +{ + std::unordered_map merged_maps; + + static void writeKey(String key, WriteBuffer & buf) { writeString(key, buf); } + static void readKey(String & key, ReadBuffer & buf) { readString(key, buf); } +}; + +template +class AggregateFunctionMap final + : public IAggregateFunctionDataHelper, AggregateFunctionMap> +{ +private: + DataTypePtr key_type; + AggregateFunctionPtr nested_func; + using Base = IAggregateFunctionDataHelper, AggregateFunctionMap>; + +public: + AggregateFunctionMap(AggregateFunctionPtr nested, const DataTypes & types) : Base(types, nested->getParameters()), nested_func(nested) + { + if (types.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function " + getName() + " require at least one argument"); + + const auto * map_type = checkAndGetDataType(types[0].get()); + if (!map_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function " + getName() + " requires map as argument"); + + key_type = map_type->getKeyType(); + } + + String getName() const override { return nested_func->getName() + "Map"; } + + DataTypePtr getReturnType() const override { return std::make_shared(DataTypes{key_type, nested_func->getReturnType()}); } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + const auto & map_column = assert_cast(*columns[0]); + const auto & map_nested_tuple = map_column.getNestedData(); + const IColumn::Offsets & map_array_offsets = map_column.getNestedColumn().getOffsets(); + + const size_t offset = map_array_offsets[row_num - 1]; + const size_t size = (map_array_offsets[row_num] - offset); + + const auto & key_column = map_nested_tuple.getColumn(0); + const auto & val_column = map_nested_tuple.getColumn(1); + + auto & merged_maps = this->data(place).merged_maps; + + for (size_t i = 0; i < size; ++i) + { + KeyType key; + if constexpr (std::is_same::value) + { + key = key_column.operator[](offset + i).get(); + } + else + { + key = assert_cast &>(key_column).getData()[offset + i]; + } + + AggregateDataPtr nested_place; + auto it = merged_maps.find(key); + + if (it == merged_maps.end()) + { + // create a new place for each key + nested_place = arena->alloc(sizeof(AggregateDataPtr)); + nested_func->create(nested_place); + merged_maps.emplace(key, nested_place); + } + else + nested_place = it->second; + + const IColumn * nested_columns[1] = {&val_column}; + nested_func->add(nested_place, nested_columns, offset + i, arena); + } + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override + { + auto & merged_maps = this->data(place).merged_maps; + const auto & rhs_maps = this->data(rhs).merged_maps; + + for (const auto & elem : rhs_maps) + { + const auto & it = merged_maps.find(elem.first); + + if (it != merged_maps.end()) + { + nested_func->merge(it->second, elem.second, arena); + } + else + merged_maps[elem.first] = elem.second; + } + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + { + auto & merged_maps = this->data(place).merged_maps; + writeVarUInt(merged_maps.size(), buf); + + for (const auto & elem : merged_maps) + { + this->data(place).writeKey(elem.first, buf); + nested_func->serialize(elem.second, buf); + } + } + + void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override + { + auto & merged_maps = this->data(place).merged_maps; + size_t size; + + readVarUInt(size, buf); + for (size_t i = 0; i < size; ++i) + { + KeyType key; + AggregateDataPtr nested_place; + + this->data(place).readKey(key, buf); + nested_place = arena->alloc(sizeof(AggregateDataPtr)); + nested_func->create(nested_place); + merged_maps.emplace(key, nested_place); + nested_func->deserialize(nested_place, buf, arena); + } + } + + void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * arena) const override + { + auto & map_column = assert_cast(to); + auto & nested_column = map_column.getNestedColumn(); + auto & nested_data_column = map_column.getNestedData(); + + auto & key_column = nested_data_column.getColumn(0); + auto & val_column = nested_data_column.getColumn(1); + + auto & merged_maps = this->data(place).merged_maps; + + size_t res_offset = 0; + + // sort the keys + std::vector keys; + keys.reserve(merged_maps.size()); + for (auto & it : merged_maps) + { + keys.push_back(it.first); + } + std::sort(keys.begin(), keys.end()); + + // insert using sorted keys to result column + for (auto & key : keys) + { + res_offset++; + key_column.insert(key); + nested_func->insertResultInto(merged_maps[key], val_column, arena); + } + + IColumn::Offsets & res_offsets = nested_column.getOffsets(); + res_offsets.push_back(res_offset); + } + + bool allocatesMemoryInArena() const override { return true; } + + AggregateFunctionPtr getNestedFunction() const override { return nested_func; } +}; + +} diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 245b089108b..32c51d5f746 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -145,9 +145,20 @@ struct MaxMapDispatchOnTupleArgument void registerAggregateFunctionSumMap(AggregateFunctionFactory & factory) { - factory.registerFunction("sumMap", createAggregateFunctionMap< + // these functions used to be called *Map, with now these names occupied by + // Map combinator, which redirects calls here if was called with + // array or tuple arguments. + factory.registerFunction("sumMappedArrays", createAggregateFunctionMap< SumMapVariants::DispatchOnTupleArgument>); + factory.registerFunction("minMappedArrays", + createAggregateFunctionMap); + + factory.registerFunction("maxMappedArrays", + createAggregateFunctionMap); + + // these functions could be renamed to *MappedArrays too, but it would + // break backward compatibility factory.registerFunction("sumMapWithOverflow", createAggregateFunctionMap< SumMapVariants::DispatchOnTupleArgument>); @@ -157,12 +168,6 @@ void registerAggregateFunctionSumMap(AggregateFunctionFactory & factory) factory.registerFunction("sumMapFilteredWithOverflow", createAggregateFunctionMap< SumMapVariants::DispatchOnTupleArgument>); - - factory.registerFunction("minMap", - createAggregateFunctionMap); - - factory.registerFunction("maxMap", - createAggregateFunctionMap); } } diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index b103f42fcc5..9bc507f5b5c 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -377,7 +377,16 @@ public: assertNoParameters(getName(), params_); } - String getName() const override { return "sumMap"; } + String getName() const override { + if constexpr (overflow) + { + return "sumMapWithOverflow"; + } + else + { + return "sumMappedArrays"; + } + } bool keepKey(const T &) const { return true; } }; @@ -528,7 +537,7 @@ public: assertNoParameters(getName(), params_); } - String getName() const override { return "minMap"; } + String getName() const override { return "minMappedArrays"; } bool keepKey(const T &) const { return true; } }; @@ -552,7 +561,7 @@ public: assertNoParameters(getName(), params_); } - String getName() const override { return "maxMap"; } + String getName() const override { return "maxMappedArrays"; } bool keepKey(const T &) const { return true; } }; diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 383f10ac24b..35efb41cf21 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -61,6 +61,7 @@ void registerAggregateFunctionCombinatorNull(AggregateFunctionCombinatorFactory void registerAggregateFunctionCombinatorOrFill(AggregateFunctionCombinatorFactory &); void registerAggregateFunctionCombinatorResample(AggregateFunctionCombinatorFactory &); void registerAggregateFunctionCombinatorDistinct(AggregateFunctionCombinatorFactory &); +void registerAggregateFunctionCombinatorMap(AggregateFunctionCombinatorFactory & factory); void registerWindowFunctions(AggregateFunctionFactory & factory); @@ -132,6 +133,7 @@ void registerAggregateFunctions() registerAggregateFunctionCombinatorOrFill(factory); registerAggregateFunctionCombinatorResample(factory); registerAggregateFunctionCombinatorDistinct(factory); + registerAggregateFunctionCombinatorMap(factory); } } From 03be2c31330b0f0ce674793857a37f1af2686709 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Mon, 10 May 2021 16:06:59 +0200 Subject: [PATCH 02/10] *MappedArrays will return old name for backwards compability --- src/AggregateFunctions/AggregateFunctionMap.cpp | 10 +++++++++- src/AggregateFunctions/AggregateFunctionSumMap.h | 9 +++++---- .../DataTypeCustomSimpleAggregateFunction.cpp | 3 ++- 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index 581587e91d3..c9c9072ff6c 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -4,6 +4,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + class AggregateFunctionCombinatorMap final : public IAggregateFunctionCombinator { public: @@ -89,7 +95,9 @@ public: { bool match; const auto * tup_type = checkAndGetDataType(arguments[0].get()); - auto check_func = [](DataTypePtr t) { + + auto check_func = [](DataTypePtr t) + { return t->getTypeId() == TypeIndex::Array; }; diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 9bc507f5b5c..2c97ad387b8 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -377,14 +377,15 @@ public: assertNoParameters(getName(), params_); } - String getName() const override { + String getName() const override + { if constexpr (overflow) { return "sumMapWithOverflow"; } else { - return "sumMappedArrays"; + return "sumMap"; } } @@ -537,7 +538,7 @@ public: assertNoParameters(getName(), params_); } - String getName() const override { return "minMappedArrays"; } + String getName() const override { return "minMap"; } bool keepKey(const T &) const { return true; } }; @@ -561,7 +562,7 @@ public: assertNoParameters(getName(), params_); } - String getName() const override { return "maxMappedArrays"; } + String getName() const override { return "maxMap"; } bool keepKey(const T &) const { return true; } }; diff --git a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp index 023629fc699..dc0411331e6 100644 --- a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp +++ b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp @@ -31,7 +31,8 @@ void DataTypeCustomSimpleAggregateFunction::checkSupportedFunctions(const Aggreg /// TODO Make it sane. static const std::vector supported_functions{"any", "anyLast", "min", "max", "sum", "sumWithOverflow", "groupBitAnd", "groupBitOr", "groupBitXor", - "sumMap", "minMap", "maxMap", "groupArrayArray", "groupUniqArrayArray"}; + "sumMap", "minMap", "maxMap", "groupArrayArray", "groupUniqArrayArray", + "sumMappedArrays", "minMappedArrays", "maxMappedArrays"}; // check function if (std::find(std::begin(supported_functions), std::end(supported_functions), function->getName()) == std::end(supported_functions)) From 0570c9db3f59e6169d5a4eb0b096cfd452f40ca7 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Tue, 18 May 2021 10:42:27 +0200 Subject: [PATCH 03/10] Fix allocation in Map combinator --- src/AggregateFunctions/AggregateFunctionMap.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 02715b40c90..c2b2784881a 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -103,7 +103,7 @@ public: if (it == merged_maps.end()) { // create a new place for each key - nested_place = arena->alloc(sizeof(AggregateDataPtr)); + nested_place = arena->alloc(nested_func->sizeOfData()); nested_func->create(nested_place); merged_maps.emplace(key, nested_place); } @@ -157,7 +157,7 @@ public: AggregateDataPtr nested_place; this->data(place).readKey(key, buf); - nested_place = arena->alloc(sizeof(AggregateDataPtr)); + nested_place = arena->alloc(nested_func->sizeOfData()); nested_func->create(nested_place); merged_maps.emplace(key, nested_place); nested_func->deserialize(nested_place, buf, arena); From c85aa0a4d670a413e994f311c23428cbd76a923c Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Tue, 18 May 2021 11:48:34 +0200 Subject: [PATCH 04/10] Fix offsets in insertResultInto for map combinator --- src/AggregateFunctions/AggregateFunctionMap.h | 3 +- .../01852_map_combinator.reference | 30 +++++++++++++ .../0_stateless/01852_map_combinator.sql | 42 +++++++++++++++++++ 3 files changed, 74 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01852_map_combinator.reference create mode 100644 tests/queries/0_stateless/01852_map_combinator.sql diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index c2b2784881a..1ae836a13d8 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -195,7 +195,8 @@ public: } IColumn::Offsets & res_offsets = nested_column.getOffsets(); - res_offsets.push_back(res_offset); + auto last_offset = res_offsets[res_offsets.size() - 1]; + res_offsets.push_back(last_offset + res_offset); } bool allocatesMemoryInArena() const override { return true; } diff --git a/tests/queries/0_stateless/01852_map_combinator.reference b/tests/queries/0_stateless/01852_map_combinator.reference new file mode 100644 index 00000000000..59a2d22933b --- /dev/null +++ b/tests/queries/0_stateless/01852_map_combinator.reference @@ -0,0 +1,30 @@ +1 {1:10,2:10,3:10} +1 {3:10,4:10,5:10} +2 {4:10,5:10,6:10} +2 {6:10,7:10,8:10} +3 {1:10,2:10,3:10} +4 {3:10,4:10,5:10} +5 {4:10,5:10,6:10} +5 {6:10,7:10,8:10} +Map(UInt16,UInt64) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} +Map(UInt16,UInt32) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} +Map(UInt16,UInt64) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} +{1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} +{1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} +Map(UInt16,Float64) {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} +{1:2,2:2,3:4,4:4,5:4,6:4,7:2,8:2} +1 {1:10,2:10,3:20,4:10,5:10} +2 {4:10,5:10,6:20,7:10,8:10} +3 {1:10,2:10,3:10} +4 {3:10,4:10,5:10} +5 {4:10,5:10,6:20,7:10,8:10} +{'01234567-89ab-cdef-0123-456789abcdef':1} +{'1':'1'} +{'1':'1'} +{1:1} +{'1970-01-02':1} +{'1970-01-01 03:00:01':1} +{'a':1} +{'1':'2'} +{1:1.00000,2:2.00000,3:6.00000,4:8.00000,5:10.00000,6:12.00000,7:7.00000,8:8.00000} +{1:1.00000,2:2.00000,3:6.00000,4:8.00000,5:10.00000,6:12.00000,7:7.00000,8:8.00000} diff --git a/tests/queries/0_stateless/01852_map_combinator.sql b/tests/queries/0_stateless/01852_map_combinator.sql new file mode 100644 index 00000000000..26911b983ae --- /dev/null +++ b/tests/queries/0_stateless/01852_map_combinator.sql @@ -0,0 +1,42 @@ +SET send_logs_level = 'fatal'; +SET allow_experimental_map_type = 1; + +DROP TABLE IF EXISTS map_comb; +CREATE TABLE map_comb(a int, statusMap Map(UInt16, UInt32)) ENGINE = Log; + +INSERT INTO map_comb VALUES (1, map(1, 10, 2, 10, 3, 10)),(1, map(3, 10, 4, 10, 5, 10)),(2, map(4, 10, 5, 10, 6, 10)),(2, map(6, 10, 7, 10, 8, 10)),(3, map(1, 10, 2, 10, 3, 10)),(4, map(3, 10, 4, 10, 5, 10)),(5, map(4, 10, 5, 10, 6, 10)),(5, map(6, 10, 7, 10, 8, 10)); + +SELECT * FROM map_comb ORDER BY a; +SELECT toTypeName(res), sumMap(statusMap) as res FROM map_comb; +SELECT toTypeName(res), sumWithOverflowMap(statusMap) as res FROM map_comb; +SELECT toTypeName(res), sumMapMerge(s) as res FROM (SELECT sumMapState(statusMap) AS s FROM map_comb); +SELECT minMap(statusMap) FROM map_comb; +SELECT maxMap(statusMap) FROM map_comb; +SELECT toTypeName(res), avgMap(statusMap) as res FROM map_comb; +SELECT countMap(statusMap) FROM map_comb; +SELECT a, sumMap(statusMap) FROM map_comb GROUP BY a ORDER BY a; + +DROP TABLE map_comb; + +-- check different types +select minMap(val) from values ('val Map(UUID, Int32)', + (map('01234567-89ab-cdef-0123-456789abcdef', 1)), + (map('01234567-89ab-cdef-0123-456789abcdef', 2))); +select minMap(val) from values ('val Map(String, String)', (map('1', '1')), (map('1', '2'))); +select minMap(val) from values ('val Map(FixedString(1), FixedString(1))', (map('1', '1')), (map('1', '2'))); +select minMap(val) from values ('val Map(UInt64, UInt64)', (map(1, 1)), (map(1, 2))); +select minMap(val) from values ('val Map(Date, Int16)', (map(1, 1)), (map(1, 2))); +select minMap(val) from values ('val Map(DateTime(\'Europe/Moscow\'), Int32)', (map(1, 1)), (map(1, 2))); +select minMap(val) from values ('val Map(Enum16(\'a\'=1), Int16)', (map('a', 1)), (map('a', 2))); +select maxMap(val) from values ('val Map(String, String)', (map('1', '1')), (map('1', '2'))); + +DROP TABLE IF EXISTS sum_map_decimal; + +CREATE TABLE sum_map_decimal(statusMap Map(UInt16,Decimal32(5))) ENGINE = Log; + +INSERT INTO sum_map_decimal VALUES (map(1,'1.0',2,'2.0',3,'3.0')), (map(3,'3.0',4,'4.0',5,'5.0')), (map(4,'4.0',5,'5.0',6,'6.0')), (map(6,'6.0',7,'7.0',8,'8.0')); + +SELECT sumMap(statusMap) FROM sum_map_decimal; +SELECT sumWithOverflowMap(statusMap) FROM sum_map_decimal; + +DROP TABLE sum_map_decimal; From 715766d7ec651c7b1ef23e2bb7993206249660ab Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Tue, 25 May 2021 17:00:27 +0200 Subject: [PATCH 05/10] Move validations to transformArguments --- .../AggregateFunctionMap.cpp | 64 +++++++++---------- 1 file changed, 30 insertions(+), 34 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index c9c9072ff6c..31505b89fe2 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -24,22 +24,40 @@ public: const auto * map_type = checkAndGetDataType(arguments[0].get()); if (map_type) + { + if (arguments->size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + getName() + " combinator takes only one map argument"); + return DataTypes({map_type->getValueType()}); + } // we need this part just to pass to redirection for mapped arrays + auto check_func = [](DataTypePtr t) + { + return t->getTypeId() == TypeIndex::Array; + }; + const auto * tup_type = checkAndGetDataType(arguments[0].get()); if (tup_type) { - const auto * val_array_type = checkAndGetDataType(tup_type->getElements()[1].get()); - if (val_array_type) - return DataTypes({val_array_type->getNestedType()}); + const auto & types = tup_type->getElements(); + bool arrays_match = arguments.size() == 1 && types.size() >= 2 && std::all_of(types.begin(), types.end(), check_func); + if (arrays_match) + { + const auto & val_array_type = assert_cast(types[1]); + return DataTypes({val_array_type.getNestedType()}); + } } - - if (arguments.size() >= 2) + else { - const auto * val_array_type = checkAndGetDataType(arguments[1].get()); - if (val_array_type) + bool arrays_match = arguments.size() >= 2 && std::all_of(arguments.begin(), arguments.end(), check_func); + if (arrays_match) + { + const auto & val_array_type = assert_cast(arguments[1]); return DataTypes({val_array_type->getNestedType()}); + } } throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function " + getName() + " requires map as argument"); @@ -87,37 +105,15 @@ public: throw Exception{"Illegal columns in arguments for combinator " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } } - else if (!arguments.empty()) + else { - // check if we got tuple of arrays or just arrays and if so, try to redirect to sum/min/max-MappedArrays to implement old behavior + // in case of tuple of arrays or just arrays (checked in transformArguments), try to redirect to sum/min/max-MappedArrays to implement old behavior auto nested_func_name = nested_function->getName(); if (nested_func_name == "sum" || nested_func_name == "min" || nested_func_name == "max") { - bool match; - const auto * tup_type = checkAndGetDataType(arguments[0].get()); - - auto check_func = [](DataTypePtr t) - { - return t->getTypeId() == TypeIndex::Array; - }; - - if (tup_type) - { - const auto & types = tup_type->getElements(); - match = arguments.size() == 1 && types.size() >= 2 && std::all_of(types.begin(), types.end(), check_func); - } - else - { - // sumMappedArrays and others support more than 2 mapped arrays - match = arguments.size() >= 2 && std::all_of(arguments.begin(), arguments.end(), check_func); - } - - if (match) - { - AggregateFunctionProperties out_properties; - auto & aggr_func_factory = AggregateFunctionFactory::instance(); - return aggr_func_factory.get(nested_func_name + "MappedArrays", arguments, params, out_properties); - } + AggregateFunctionProperties out_properties; + auto & aggr_func_factory = AggregateFunctionFactory::instance(); + return aggr_func_factory.get(nested_func_name + "MappedArrays", arguments, params, out_properties); } } From 5a5cb238d8b29b8de37baceb290547a2790231fc Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 26 May 2021 16:47:58 +0200 Subject: [PATCH 06/10] Map combinator: add more arguments checks, fix memory align errors, support more key types, add some docs and tests --- .../aggregate-functions/combinators.md | 6 +++ .../AggregateFunctionMap.cpp | 40 +++++++++++-------- src/AggregateFunctions/AggregateFunctionMap.h | 16 ++++---- .../01852_map_combinator.reference | 4 ++ .../0_stateless/01852_map_combinator.sql | 15 +++++++ 5 files changed, 56 insertions(+), 25 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index 3fc5121ebcc..44615628eef 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -25,6 +25,12 @@ Example 2: `uniqArray(arr)` – Counts the number of unique elements in all ‘a -If and -Array can be combined. However, ‘Array’ must come first, then ‘If’. Examples: `uniqArrayIf(arr, cond)`, `quantilesTimingArrayIf(level1, level2)(arr, cond)`. Due to this order, the ‘cond’ argument won’t be an array. +## -Map {#agg-functions-combinator-map} + +The -Map suffix can be appended to any aggregate function. This will create an aggregate function which gets Map type as an argument, and aggregates values of each key of the map separately using the specified aggregate function. The result is also of a Map type. + +Examples: `sumMap(map(1,1))`, `avgMap(map('a', 1))`. + ## -SimpleState {#agg-functions-combinator-simplestate} If you apply this combinator, the aggregate function returns the same value but with a different type. This is a [SimpleAggregateFunction(...)](../../sql-reference/data-types/simpleaggregatefunction.md) that can be stored in a table to work with [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) tables. diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index 31505b89fe2..09214427ad6 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -25,19 +25,14 @@ public: const auto * map_type = checkAndGetDataType(arguments[0].get()); if (map_type) { - if (arguments->size() > 1) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - getName() + " combinator takes only one map argument"); + if (arguments.size() > 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, getName() + " combinator takes only one map argument"); return DataTypes({map_type->getValueType()}); } // we need this part just to pass to redirection for mapped arrays - auto check_func = [](DataTypePtr t) - { - return t->getTypeId() == TypeIndex::Array; - }; + auto check_func = [](DataTypePtr t) { return t->getTypeId() == TypeIndex::Array; }; const auto * tup_type = checkAndGetDataType(arguments[0].get()); if (tup_type) @@ -46,8 +41,8 @@ public: bool arrays_match = arguments.size() == 1 && types.size() >= 2 && std::all_of(types.begin(), types.end(), check_func); if (arrays_match) { - const auto & val_array_type = assert_cast(types[1]); - return DataTypes({val_array_type.getNestedType()}); + const auto * val_array_type = assert_cast(types[1].get()); + return DataTypes({val_array_type->getNestedType()}); } } else @@ -55,7 +50,7 @@ public: bool arrays_match = arguments.size() >= 2 && std::all_of(arguments.begin(), arguments.end(), check_func); if (arrays_match) { - const auto & val_array_type = assert_cast(arguments[1]); + const auto * val_array_type = assert_cast(arguments[1].get()); return DataTypes({val_array_type->getNestedType()}); } } @@ -72,9 +67,9 @@ public: const auto * map_type = checkAndGetDataType(arguments[0].get()); if (map_type) { - auto key_type_id = map_type->getKeyType()->getTypeId(); + const auto & key_type = map_type->getKeyType(); - switch (key_type_id) + switch (key_type->getTypeId()) { case TypeIndex::Enum8: case TypeIndex::Int8: @@ -86,6 +81,10 @@ public: return std::make_shared>(nested_function, arguments); case TypeIndex::Int64: return std::make_shared>(nested_function, arguments); + case TypeIndex::Int128: + return std::make_shared>(nested_function, arguments); + case TypeIndex::Int256: + return std::make_shared>(nested_function, arguments); case TypeIndex::UInt8: return std::make_shared>(nested_function, arguments); case TypeIndex::Date: @@ -96,13 +95,19 @@ public: return std::make_shared>(nested_function, arguments); case TypeIndex::UInt64: return std::make_shared>(nested_function, arguments); - case TypeIndex::UUID: + case TypeIndex::UInt128: return std::make_shared>(nested_function, arguments); + case TypeIndex::UInt256: + return std::make_shared>(nested_function, arguments); + case TypeIndex::UUID: + return std::make_shared>(nested_function, arguments); case TypeIndex::FixedString: case TypeIndex::String: return std::make_shared>(nested_function, arguments); default: - throw Exception{"Illegal columns in arguments for combinator " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception{ + "Map key type " + key_type->getName() + " is not is not supported by combinator " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } } else @@ -115,9 +120,10 @@ public: auto & aggr_func_factory = AggregateFunctionFactory::instance(); return aggr_func_factory.get(nested_func_name + "MappedArrays", arguments, params, out_properties); } + else + throw Exception{ + "Aggregation '" + nested_func_name + "Map' is not implemented for mapped arrays", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } - - throw Exception{"Illegal columns in arguments for combinator " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } }; diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 1ae836a13d8..75bb2e75840 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -58,7 +58,11 @@ public: { if (types.empty()) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function " + getName() + " require at least one argument"); + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function " + getName() + " requires at least one argument"); + + if (types.size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function " + getName() + " requires only one map argument"); const auto * map_type = checkAndGetDataType(types[0].get()); if (!map_type) @@ -103,7 +107,7 @@ public: if (it == merged_maps.end()) { // create a new place for each key - nested_place = arena->alloc(nested_func->sizeOfData()); + nested_place = arena->alignedAlloc(nested_func->sizeOfData(), nested_func->alignOfData()); nested_func->create(nested_place); merged_maps.emplace(key, nested_place); } @@ -157,7 +161,7 @@ public: AggregateDataPtr nested_place; this->data(place).readKey(key, buf); - nested_place = arena->alloc(nested_func->sizeOfData()); + nested_place = arena->alignedAlloc(nested_func->sizeOfData(), nested_func->alignOfData()); nested_func->create(nested_place); merged_maps.emplace(key, nested_place); nested_func->deserialize(nested_place, buf, arena); @@ -175,8 +179,6 @@ public: auto & merged_maps = this->data(place).merged_maps; - size_t res_offset = 0; - // sort the keys std::vector keys; keys.reserve(merged_maps.size()); @@ -189,14 +191,12 @@ public: // insert using sorted keys to result column for (auto & key : keys) { - res_offset++; key_column.insert(key); nested_func->insertResultInto(merged_maps[key], val_column, arena); } IColumn::Offsets & res_offsets = nested_column.getOffsets(); - auto last_offset = res_offsets[res_offsets.size() - 1]; - res_offsets.push_back(last_offset + res_offset); + res_offsets.push_back(val_column.size()); } bool allocatesMemoryInArena() const override { return true; } diff --git a/tests/queries/0_stateless/01852_map_combinator.reference b/tests/queries/0_stateless/01852_map_combinator.reference index 59a2d22933b..7c0648ccb65 100644 --- a/tests/queries/0_stateless/01852_map_combinator.reference +++ b/tests/queries/0_stateless/01852_map_combinator.reference @@ -26,5 +26,9 @@ Map(UInt16,Float64) {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} {'1970-01-01 03:00:01':1} {'a':1} {'1':'2'} +{1:1} +{1:1} +{1:1} +{1:1} {1:1.00000,2:2.00000,3:6.00000,4:8.00000,5:10.00000,6:12.00000,7:7.00000,8:8.00000} {1:1.00000,2:2.00000,3:6.00000,4:8.00000,5:10.00000,6:12.00000,7:7.00000,8:8.00000} diff --git a/tests/queries/0_stateless/01852_map_combinator.sql b/tests/queries/0_stateless/01852_map_combinator.sql index 26911b983ae..20923460eb6 100644 --- a/tests/queries/0_stateless/01852_map_combinator.sql +++ b/tests/queries/0_stateless/01852_map_combinator.sql @@ -29,6 +29,21 @@ select minMap(val) from values ('val Map(Date, Int16)', (map(1, 1)), (map(1, 2) select minMap(val) from values ('val Map(DateTime(\'Europe/Moscow\'), Int32)', (map(1, 1)), (map(1, 2))); select minMap(val) from values ('val Map(Enum16(\'a\'=1), Int16)', (map('a', 1)), (map('a', 2))); select maxMap(val) from values ('val Map(String, String)', (map('1', '1')), (map('1', '2'))); +select minMap(val) from values ('val Map(Int128, Int128)', (map(1, 1)), (map(1, 2))); +select minMap(val) from values ('val Map(Int256, Int256)', (map(1, 1)), (map(1, 2))); +select minMap(val) from values ('val Map(UInt128, UInt128)', (map(1, 1)), (map(1, 2))); +select minMap(val) from values ('val Map(UInt256, UInt256)', (map(1, 1)), (map(1, 2))); + +select sumMap(map(1,2), 1, 2); -- { serverError 42 } +select sumMap(map(1,2), map(1,3)); -- { serverError 42 } + +-- array and tuple arguments +select avgMap([1,1,1], [2,2,2]); -- { serverError 43 } +select minMap((1,1)); -- { serverError 43 } +select minMap(([1,1,1],1)); -- { serverError 43 } +select minMap([1,1,1],1); -- { serverError 43 } +select minMap([1,1,1]); -- { serverError 43 } +select minMap(([1,1,1])); -- { serverError 43 } DROP TABLE IF EXISTS sum_map_decimal; From 95f5a4d0fae94cc8bc601d03006eb37864972b0e Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Mon, 14 Jun 2021 22:11:34 +0500 Subject: [PATCH 07/10] Optimize string keys and fix string serialization in map combinator --- src/AggregateFunctions/AggregateFunctionMap.h | 48 +++++++++++++++---- 1 file changed, 40 insertions(+), 8 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 75bb2e75840..8ac6a5d6eb6 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include #include @@ -15,6 +17,7 @@ #include #include #include +#include "common/types.h" #include #include "AggregateFunctions/AggregateFunctionFactory.h" @@ -29,6 +32,7 @@ namespace ErrorCodes template struct AggregateFunctionMapCombinatorData { + using SearchType = KeyType; std::unordered_map merged_maps; static void writeKey(KeyType key, WriteBuffer & buf) { writeBinary(key, buf); } @@ -38,10 +42,29 @@ struct AggregateFunctionMapCombinatorData template <> struct AggregateFunctionMapCombinatorData { - std::unordered_map merged_maps; + struct StringHash + { + using hash_type = std::hash; + using is_transparent = void; - static void writeKey(String key, WriteBuffer & buf) { writeString(key, buf); } - static void readKey(String & key, ReadBuffer & buf) { readString(key, buf); } + size_t operator()(std::string_view str) const { return hash_type{}(str); } + }; + + using SearchType = std::string_view; + std::unordered_map> merged_maps; + + static void writeKey(String key, WriteBuffer & buf) + { + writeVarUInt(key.size(), buf); + writeString(key, buf); + } + static void readKey(String & key, ReadBuffer & buf) + { + UInt64 size; + readVarUInt(size, buf); + key.resize(size); + buf.readStrict(key.data(), size); + } }; template @@ -51,7 +74,9 @@ class AggregateFunctionMap final private: DataTypePtr key_type; AggregateFunctionPtr nested_func; - using Base = IAggregateFunctionDataHelper, AggregateFunctionMap>; + + using Data = AggregateFunctionMapCombinatorData; + using Base = IAggregateFunctionDataHelper>; public: AggregateFunctionMap(AggregateFunctionPtr nested, const DataTypes & types) : Base(types, nested->getParameters()), nested_func(nested) @@ -91,10 +116,17 @@ public: for (size_t i = 0; i < size; ++i) { - KeyType key; + typename Data::SearchType key; + if constexpr (std::is_same::value) { - key = key_column.operator[](offset + i).get(); + StringRef key_ref; + if (key_type->getTypeId() == TypeIndex::FixedString) + key_ref = assert_cast(key_column).getDataAt(offset + i); + else + key_ref = assert_cast(key_column).getDataAt(offset + i); + + key = static_cast(key_ref); } else { @@ -152,10 +184,10 @@ public: void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override { auto & merged_maps = this->data(place).merged_maps; - size_t size; + UInt64 size; readVarUInt(size, buf); - for (size_t i = 0; i < size; ++i) + for (UInt64 i = 0; i < size; ++i) { KeyType key; AggregateDataPtr nested_place; From fd15fc5e4ab6101996bed18ff07a95300177c3e1 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Tue, 15 Jun 2021 16:20:42 +0500 Subject: [PATCH 08/10] Fix style errors, disable heterogeneous comparison lookup where it's not supported --- src/AggregateFunctions/AggregateFunctionMap.cpp | 10 +++++----- src/AggregateFunctions/AggregateFunctionMap.h | 14 +++++++++++--- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index 09214427ad6..1808e1c2ffa 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -105,9 +105,9 @@ public: case TypeIndex::String: return std::make_shared>(nested_function, arguments); default: - throw Exception{ - "Map key type " + key_type->getName() + " is not is not supported by combinator " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Map key type " + key_type->getName() + " is not is not supported by combinator " + getName()); } } else @@ -121,8 +121,8 @@ public: return aggr_func_factory.get(nested_func_name + "MappedArrays", arguments, params, out_properties); } else - throw Exception{ - "Aggregation '" + nested_func_name + "Map' is not implemented for mapped arrays", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregation '" + nested_func_name + "Map' is not implemented for mapped arrays"); } } }; diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 8ac6a5d6eb6..63974091a01 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -3,11 +3,11 @@ #include #include #include +#include #include +#include #include #include -#include -#include #include #include #include @@ -47,10 +47,14 @@ struct AggregateFunctionMapCombinatorData using hash_type = std::hash; using is_transparent = void; - size_t operator()(std::string_view str) const { return hash_type{}(str); } + size_t operator()(std::string_view str) const { return hash_type{}(str); } }; +#ifdef __cpp_lib_generic_unordered_lookup using SearchType = std::string_view; +#else + using SearchType = std::string; +#endif std::unordered_map> merged_maps; static void writeKey(String key, WriteBuffer & buf) @@ -126,7 +130,11 @@ public: else key_ref = assert_cast(key_column).getDataAt(offset + i); +#ifdef __cpp_lib_generic_unordered_lookup key = static_cast(key_ref); +#else + key = key_ref.toString(); +#endif } else { From 9f6af8501eeb91c1c68ce871ebd1add033df3a3a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 1 Nov 2021 11:58:49 +0300 Subject: [PATCH 09/10] Fix build --- src/AggregateFunctions/AggregateFunctionMap.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 63974091a01..e5367ac4f9f 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -17,7 +17,7 @@ #include #include #include -#include "common/types.h" +#include "base/types.h" #include #include "AggregateFunctions/AggregateFunctionFactory.h" From 1a228e8071a949b28755994069dd2befecc55715 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 2 Nov 2021 12:14:00 +0300 Subject: [PATCH 10/10] Update 01852_map_combinator.reference --- .../0_stateless/01852_map_combinator.reference | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01852_map_combinator.reference b/tests/queries/0_stateless/01852_map_combinator.reference index 7c0648ccb65..4ad4ea8071e 100644 --- a/tests/queries/0_stateless/01852_map_combinator.reference +++ b/tests/queries/0_stateless/01852_map_combinator.reference @@ -6,12 +6,12 @@ 4 {3:10,4:10,5:10} 5 {4:10,5:10,6:10} 5 {6:10,7:10,8:10} -Map(UInt16,UInt64) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} -Map(UInt16,UInt32) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} -Map(UInt16,UInt64) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} +Map(UInt16, UInt64) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} +Map(UInt16, UInt32) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} +Map(UInt16, UInt64) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} -Map(UInt16,Float64) {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} +Map(UInt16, Float64) {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} {1:2,2:2,3:4,4:4,5:4,6:4,7:2,8:2} 1 {1:10,2:10,3:20,4:10,5:10} 2 {4:10,5:10,6:20,7:10,8:10} @@ -30,5 +30,5 @@ Map(UInt16,Float64) {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} {1:1} {1:1} {1:1} -{1:1.00000,2:2.00000,3:6.00000,4:8.00000,5:10.00000,6:12.00000,7:7.00000,8:8.00000} -{1:1.00000,2:2.00000,3:6.00000,4:8.00000,5:10.00000,6:12.00000,7:7.00000,8:8.00000} +{1:1,2:2,3:6,4:8,5:10,6:12,7:7,8:8} +{1:1,2:2,3:6,4:8,5:10,6:12,7:7,8:8}