diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 8b0710c0182..ff2f11322a4 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -70,23 +70,23 @@ Result: Collect all the keys and sum corresponding values. -**Syntax** +**Syntax** ``` sql -mapAdd(Tuple(Array, Array), Tuple(Array, Array) [, ...]) +mapAdd(arg1, arg2 [, ...]) ``` -**Arguments** +**Arguments** -Arguments are [tuples](../../sql-reference/data-types/tuple.md#tuplet1-t2) of two [arrays](../../sql-reference/data-types/array.md#data-type-array), where items in the first array represent keys, and the second array contains values for the each key. All key arrays should have same type, and all value arrays should contain items which are promote to the one type ([Int64](../../sql-reference/data-types/int-uint.md#int-ranges), [UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges) or [Float64](../../sql-reference/data-types/float.md#float32-float64)). The common promoted type is used as a type for the result array. +Arguments are [maps](../../sql-reference/data-types/map.md) or [tuples](../../sql-reference/data-types/tuple.md#tuplet1-t2) of two [arrays](../../sql-reference/data-types/array.md#data-type-array), where items in the first array represent keys, and the second array contains values for the each key. All key arrays should have same type, and all value arrays should contain items which are promote to the one type ([Int64](../../sql-reference/data-types/int-uint.md#int-ranges), [UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges) or [Float64](../../sql-reference/data-types/float.md#float32-float64)). The common promoted type is used as a type for the result array. **Returned value** -- Returns one [tuple](../../sql-reference/data-types/tuple.md#tuplet1-t2), where the first array contains the sorted keys and the second array contains values. +- Depending on the arguments returns one [map](../../sql-reference/data-types/map.md) o [tuple](../../sql-reference/data-types/tuple.md#tuplet1-t2), where the first array contains the sorted keys and the second array contains values. **Example** -Query: +Query with a tuple map: ``` sql SELECT mapAdd(([toUInt8(1), 2], [1, 1]), ([toUInt8(1), 2], [1, 1])) as res, toTypeName(res) as type; @@ -100,6 +100,11 @@ Result: └───────────────┴────────────────────────────────────┘ ``` +Query with `Map` type: + +``` sql +``` + ## mapSubtract {#function-mapsubtract} Collect all the keys and subtract corresponding values. diff --git a/src/Functions/array/mapOp.cpp b/src/Functions/array/mapOp.cpp index 1a19ee41d2f..da394c47f80 100644 --- a/src/Functions/array/mapOp.cpp +++ b/src/Functions/array/mapOp.cpp @@ -1,13 +1,18 @@ #include -#include +#include +#include #include +#include +#include +#include #include #include #include #include #include #include -#include +#include "Columns/ColumnMap.h" +#include "DataTypes/DataTypeMap.h" namespace DB @@ -24,8 +29,8 @@ namespace struct TupArg { - const IColumn & key_column; - const IColumn & val_column; + const ColumnPtr & key_column; + const ColumnPtr & val_column; const IColumn::Offsets & key_offsets; const IColumn::Offsets & val_offsets; bool is_const; @@ -52,17 +57,39 @@ private: bool isVariadic() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + void checkTypes( + DataTypePtr & key_type, DataTypePtr & promoted_val_type, const DataTypePtr & check_key_type, DataTypePtr & check_val_type) const + { + if (!(check_key_type->equals(*key_type))) + throw Exception( + "Expected same " + key_type->getName() + " type for all keys in " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + WhichDataType which_val(promoted_val_type); + WhichDataType which_ch_val(check_val_type); + + if (which_ch_val.isFloat() != which_val.isFloat()) + throw Exception( + "All value types in " + getName() + " should be ether or float or integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!(check_val_type->equals(*promoted_val_type))) + { + throw Exception( + "All value types in " + getName() + " should be promotable to " + promoted_val_type->getName() + ", got " + + check_val_type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + + DataTypePtr getReturnTypeForTuples(const DataTypes & arguments) const { - bool is_float = false; DataTypePtr key_type, val_type, res; - if (arguments.size() < 2) - throw Exception{getName() + " accepts at least two map tuples", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - for (const auto & tup_arg : arguments) + for (const auto & arg : arguments) { - const DataTypeTuple * tup = checkAndGetDataType(tup_arg.get()); + const DataTypeArray * k; + const DataTypeArray * v; + + const DataTypeTuple * tup = checkAndGetDataType(arg.get()); if (!tup) throw Exception{getName() + " accepts at least two map tuples", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; @@ -71,8 +98,8 @@ private: throw Exception( "Each tuple in " + getName() + " arguments should consist of two arrays", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - const DataTypeArray * k = checkAndGetDataType(elems[0].get()); - const DataTypeArray * v = checkAndGetDataType(elems[1].get()); + k = checkAndGetDataType(elems[0].get()); + v = checkAndGetDataType(elems[1].get()); if (!k || !v) throw Exception( @@ -80,62 +107,100 @@ private: auto result_type = v->getNestedType(); if (!result_type->canBePromoted()) - throw Exception{"Values to be summed are expected to be Numeric, Float or Decimal.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception{ + "Values to be summed are expected to be Numeric, Float or Decimal.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - WhichDataType which_val(result_type); - - auto promoted_type = result_type->promoteNumericType(); + auto promoted_val_type = result_type->promoteNumericType(); if (!key_type) { key_type = k->getNestedType(); - val_type = promoted_type; - is_float = which_val.isFloat(); + val_type = promoted_val_type; + res = std::make_shared( + DataTypes{std::make_shared(k->getNestedType()), std::make_shared(promoted_val_type)}); } else - { - if (!(k->getNestedType()->equals(*key_type))) - throw Exception( - "All key types in " + getName() + " should be same: " + key_type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (is_float != which_val.isFloat()) - throw Exception( - "All value types in " + getName() + " should be or float or integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!(promoted_type->equals(*val_type))) - { - throw Exception( - "All value types in " + getName() + " should be promotable to " + val_type->getName() + ", got " - + promoted_type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - } - - if (!res) - { - res = std::make_shared( - DataTypes{std::make_shared(k->getNestedType()), std::make_shared(promoted_type)}); - } + checkTypes(key_type, val_type, k->getNestedType(), promoted_val_type); } return res; } - template - ColumnPtr execute2(size_t row_count, TupleMaps & args, const DataTypeTuple & res_type) const + DataTypePtr getReturnTypeForMaps(const DataTypes & arguments) const { - MutableColumnPtr res_tuple = res_type.createColumn(); + DataTypePtr key_type, val_type, res; - auto * to_tuple = assert_cast(res_tuple.get()); - auto & to_keys_arr = assert_cast(to_tuple->getColumn(0)); - auto & to_keys_data = to_keys_arr.getData(); - auto & to_keys_offset = to_keys_arr.getOffsets(); + for (const auto & arg : arguments) + { + const auto * map = checkAndGetDataType(arg.get()); + if (!map) + throw Exception{getName() + " accepts at least two maps", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - auto & to_vals_arr = assert_cast(to_tuple->getColumn(1)); - auto & to_vals_data = to_vals_arr.getData(); + const auto & v = map->getValueType(); + + if (!v->canBePromoted()) + throw Exception{ + "Values to be summed are expected to be Numeric, Float or Decimal.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + auto promoted_val_type = v->promoteNumericType(); + if (!key_type) + { + key_type = map->getKeyType(); + val_type = promoted_val_type; + res = std::make_shared(DataTypes({key_type, promoted_val_type})); + } + else + checkTypes(key_type, val_type, map->getKeyType(), promoted_val_type); + } + + return res; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() < 2) + throw Exception{getName() + " accepts at least two maps", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + + if (arguments[0]->getTypeId() == TypeIndex::Tuple) + return getReturnTypeForTuples(arguments); + else if (arguments[0]->getTypeId() == TypeIndex::Map) + return getReturnTypeForMaps(arguments); + else + throw Exception{getName() + " only accepts maps", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + + template + ColumnPtr execute2(size_t row_count, TupleMaps & args, const DataTypePtr res_type) const + { + MutableColumnPtr res_column = res_type->createColumn(); + IColumn *to_keys_data, *to_vals_data; + ColumnArray::Offsets * to_keys_offset; + ColumnArray::Offsets * to_vals_offset = nullptr; + + // prepare output destinations + if (res_type->getTypeId() == TypeIndex::Tuple) + { + auto * to_tuple = assert_cast(res_column.get()); + auto & to_keys_arr = assert_cast(to_tuple->getColumn(0)); + to_keys_data = &to_keys_arr.getData(); + to_keys_offset = &to_keys_arr.getOffsets(); + + auto & to_vals_arr = assert_cast(to_tuple->getColumn(1)); + to_vals_data = &to_vals_arr.getData(); + to_vals_offset = &to_vals_arr.getOffsets(); + } + else + { + assert(res_type->getTypeId() == TypeIndex::Map); + + auto * to_map = assert_cast(res_column.get()); + auto & to_wrapper_arr = to_map->getNestedColumn(); + to_keys_offset = &to_wrapper_arr.getOffsets(); + + auto & to_map_tuple = to_map->getNestedData(); + to_keys_data = &to_map_tuple.getColumn(0); + to_vals_data = &to_map_tuple.getColumn(1); + } - size_t res_offset = 0; std::map summing_map; for (size_t i = 0; i < row_count; i++) @@ -147,7 +212,7 @@ private: if (!arg.is_const) { - offset = i > 0 ? arg.key_offsets[i - 1] : 0; + offset = arg.key_offsets[i - 1]; len = arg.key_offsets[i] - offset; if (arg.val_offsets[i] != arg.key_offsets[i]) @@ -155,20 +220,30 @@ private: "Key and value array should have same amount of elements", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } + Field temp_val; for (size_t j = 0; j < len; j++) { KeyType key; - if constexpr (is_str_key) + if constexpr (std::is_same::value) { - // have to use Field to get strings - key = arg.key_column[offset + j].get(); + if (const auto * col_fixed = checkAndGetColumn(arg.key_column.get())) + key = col_fixed->getDataAt(offset + j).toString(); + else if (const auto * col_str = checkAndGetColumn(arg.key_column.get())) + key = col_str->getDataAt(offset + j).toString(); + else + // should not happen + throw Exception( + "Expected String or FixedString, got " + std::string(getTypeName(arg.key_column->getDataType())) + + " in " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } else { - key = assert_cast &>(arg.key_column).getData()[offset + j]; + key = assert_cast *>(arg.key_column.get())->getData()[offset + j]; } - ValType value = arg.val_column[offset + j].get(); + arg.val_column->get(offset + j, temp_val); + ValType value = temp_val.get(); if constexpr (op_type == OpTypes::ADD) { @@ -190,132 +265,184 @@ private: for (const auto & elem : summing_map) { - res_offset++; - to_keys_data.insert(elem.first); - to_vals_data.insert(elem.second); + to_keys_data->insert(elem.first); + to_vals_data->insert(elem.second); } - to_keys_offset.push_back(res_offset); + to_keys_offset->push_back(to_keys_data->size()); summing_map.clear(); } - // same offsets as in keys - to_vals_arr.getOffsets().insert(to_keys_offset.begin(), to_keys_offset.end()); + if (to_vals_offset) + { + // same offsets as in keys + to_vals_offset->insert(to_keys_offset->begin(), to_keys_offset->end()); + } - return res_tuple; + return res_column; } - template - ColumnPtr execute1(size_t row_count, const DataTypeTuple & res_type, TupleMaps & args) const + template + ColumnPtr execute1(size_t row_count, const DataTypePtr res_type, const DataTypePtr res_value_type, TupleMaps & args) const { - const auto & promoted_type = (assert_cast(res_type.getElements()[1].get()))->getNestedType(); -#define MATCH_EXECUTE(is_str) \ - switch (promoted_type->getTypeId()) \ - { \ - case TypeIndex::Int64: return execute2(row_count, args, res_type); \ - case TypeIndex::UInt64: return execute2(row_count, args, res_type); \ - case TypeIndex::Float64: return execute2(row_count, args, res_type); \ - default: \ - throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; \ - } - - if constexpr (is_str_key) + switch (res_value_type->getTypeId()) { - MATCH_EXECUTE(true) + case TypeIndex::Int64: + return execute2(row_count, args, res_type); + case TypeIndex::Int128: + return execute2(row_count, args, res_type); + case TypeIndex::Int256: + return execute2(row_count, args, res_type); + case TypeIndex::UInt64: + return execute2(row_count, args, res_type); + case TypeIndex::UInt128: + return execute2(row_count, args, res_type); + case TypeIndex::UInt256: + return execute2(row_count, args, res_type); + case TypeIndex::Float64: + return execute2(row_count, args, res_type); + default: + throw Exception{ + "Illegal column type " + res_value_type->getName() + " for values in arguments of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } - else - { - MATCH_EXECUTE(false) - } -#undef MATCH_EXECUTE } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override { + DataTypePtr key_type; + size_t row_count; const DataTypeTuple * tup_type = checkAndGetDataType((arguments[0]).type.get()); - const DataTypeArray * key_array_type = checkAndGetDataType(tup_type->getElements()[0].get()); - const DataTypeArray * val_array_type = checkAndGetDataType(tup_type->getElements()[1].get()); - - /* determine output type */ - const DataTypeTuple & res_type - = DataTypeTuple(DataTypes{std::make_shared(key_array_type->getNestedType()), - std::make_shared(val_array_type->getNestedType()->promoteNumericType())}); - + DataTypePtr res_type; + DataTypePtr res_value_type; TupleMaps args{}; args.reserve(arguments.size()); //prepare columns, extract data columns for direct access and put them to the vector - for (const auto & col : arguments) + if (tup_type) { - const ColumnTuple * tup; - bool is_const = isColumnConst(*col.column); - if (is_const) + const DataTypeArray * key_array_type = checkAndGetDataType(tup_type->getElements()[0].get()); + const DataTypeArray * val_array_type = checkAndGetDataType(tup_type->getElements()[1].get()); + + /* determine output type */ + res_value_type = val_array_type->getNestedType()->promoteNumericType(); + res_type = std::make_shared(DataTypes{ + std::make_shared(key_array_type->getNestedType()), std::make_shared(res_value_type)}); + + for (const auto & col : arguments) { - const auto * c = assert_cast(col.column.get()); - tup = assert_cast(c->getDataColumnPtr().get()); + const ColumnTuple * tup; + bool is_const = isColumnConst(*col.column); + if (is_const) + { + const auto * c = assert_cast(col.column.get()); + tup = assert_cast(c->getDataColumnPtr().get()); + } + else + tup = assert_cast(col.column.get()); + + const auto & arr1 = assert_cast(tup->getColumn(0)); + const auto & arr2 = assert_cast(tup->getColumn(1)); + + const auto & key_offsets = arr1.getOffsets(); + const auto & key_column = arr1.getDataPtr(); + + const auto & val_offsets = arr2.getOffsets(); + const auto & val_column = arr2.getDataPtr(); + + args.push_back({key_column, val_column, key_offsets, val_offsets, is_const}); + } + + key_type = key_array_type->getNestedType(); + } + else + { + const DataTypeMap * map_type = checkAndGetDataType((arguments[0]).type.get()); + if (map_type) + { + key_type = map_type->getKeyType(); + res_value_type = map_type->getValueType()->promoteNumericType(); + res_type = std::make_shared(DataTypes{map_type->getKeyType(), res_value_type}); + + for (const auto & col : arguments) + { + const ColumnMap * map; + bool is_const = isColumnConst(*col.column); + if (is_const) + { + const auto * c = assert_cast(col.column.get()); + map = assert_cast(c->getDataColumnPtr().get()); + } + else + map = assert_cast(col.column.get()); + + const auto & map_arr = map->getNestedColumn(); + const auto & key_offsets = map_arr.getOffsets(); + const auto & val_offsets = key_offsets; + + const auto & map_tup = map->getNestedData(); + const auto & key_column = map_tup.getColumnPtr(0); + const auto & val_column = map_tup.getColumnPtr(1); + + args.push_back({key_column, val_column, key_offsets, val_offsets, is_const}); + } } else - tup = assert_cast(col.column.get()); + throw Exception{ + "Illegal column type " + key_type->getName() + " in arguments of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } - const auto & arr1 = assert_cast(tup->getColumn(0)); - const auto & arr2 = assert_cast(tup->getColumn(1)); - - const auto & key_offsets = arr1.getOffsets(); - const auto & key_column = arr1.getData(); - - const auto & val_offsets = arr2.getOffsets(); - const auto & val_column = arr2.getData(); - - // we can check const columns before any processing - if (is_const) + // we can check const columns before any processing + for (auto & arg : args) + { + if (arg.is_const) { - if (val_offsets[0] != key_offsets[0]) + if (arg.val_offsets[0] != arg.key_offsets[0]) throw Exception( "Key and value array should have same amount of elements", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } - - args.push_back({key_column, val_column, key_offsets, val_offsets, is_const}); } - size_t row_count = arguments[0].column->size(); - auto key_type_id = key_array_type->getNestedType()->getTypeId(); - - switch (key_type_id) + row_count = arguments[0].column->size(); + switch (key_type->getTypeId()) { case TypeIndex::Enum8: case TypeIndex::Int8: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::Enum16: case TypeIndex::Int16: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::Int32: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::Int64: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::Int128: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::Int256: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::UInt8: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::Date: case TypeIndex::UInt16: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::DateTime: case TypeIndex::UInt32: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::UInt64: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::UInt128: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::UInt256: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::UUID: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); case TypeIndex::FixedString: case TypeIndex::String: - return execute1(row_count, res_type, args); + return execute1(row_count, res_type, res_value_type, args); default: - throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + throw Exception{ + "Illegal column type " + key_type->getName() + " for keys in arguments of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } } }; diff --git a/tests/queries/0_stateless/01318_map_add_map_subtract.sql b/tests/queries/0_stateless/01318_map_add_map_subtract.sql index 40c08e0a147..6ead7a2db46 100644 --- a/tests/queries/0_stateless/01318_map_add_map_subtract.sql +++ b/tests/queries/0_stateless/01318_map_add_map_subtract.sql @@ -2,7 +2,7 @@ drop table if exists map_test; create table map_test engine=TinyLog() as (select ([1, number], [toInt32(2),2]) as map from numbers(1, 10)); -- mapAdd -select mapAdd([1], [1]); -- { serverError 42 } +select mapAdd([1], [1]); -- { serverError 43 } select mapAdd(([1], [1])); -- { serverError 42 } select mapAdd(([1], [1]), map) from map_test; -- { serverError 43 } select mapAdd(([toUInt64(1)], [1]), map) from map_test; -- { serverError 43 } @@ -27,7 +27,7 @@ select mapAdd(([toInt64(1), 2], [toInt64(1), 1]), ([toInt64(1), 2], [toInt64(1), select mapAdd(([1, 2], [toFloat32(1.1), 1]), ([1, 2], [2.2, 1])) as res, toTypeName(res); select mapAdd(([1, 2], [toFloat64(1.1), 1]), ([1, 2], [2.2, 1])) as res, toTypeName(res); -select mapAdd(([toFloat32(1), 2], [toFloat64(1.1), 1]), ([toFloat32(1), 2], [2.2, 1])) as res, toTypeName(res); -- { serverError 44 } +select mapAdd(([toFloat32(1), 2], [toFloat64(1.1), 1]), ([toFloat32(1), 2], [2.2, 1])) as res, toTypeName(res); -- { serverError 43 } select mapAdd(([1, 2], [toFloat64(1.1), 1]), ([1, 2], [1, 1])) as res, toTypeName(res); -- { serverError 43 } select mapAdd((['a', 'b'], [1, 1]), ([key], [1])) from values('key String', ('b'), ('c'), ('d')); select mapAdd((cast(['a', 'b'], 'Array(FixedString(1))'), [1, 1]), ([key], [1])) as res, toTypeName(res) from values('key FixedString(1)', ('b'), ('c'), ('d')); diff --git a/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference b/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference new file mode 100644 index 00000000000..96bafc2c79c --- /dev/null +++ b/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.reference @@ -0,0 +1,55 @@ +{1:5} +{1:3,2:2} +{1:3,3:2} +{1:3,4:2} +{1:3,5:2} +{1:3,6:2} +{1:3,7:2} +{1:3,8:2} +{1:3,9:2} +{1:3,10:2} +{1:5,2:2} +{1:3,2:4} +{1:3,2:2,3:2} +{1:3,2:2,4:2} +{1:3,2:2,5:2} +{1:3,2:2,6:2} +{1:3,2:2,7:2} +{1:3,2:2,8:2} +{1:3,2:2,9:2} +{1:3,2:2,10:2} +{1:2,2:2} Map(UInt8,UInt64) +{1:2,2:2} Map(UInt16,UInt64) +{1:2,2:2} Map(UInt32,UInt64) +{1:2,2:2} Map(UInt64,UInt64) +{1:2,2:2} Map(UInt128,UInt128) +{1:2,2:2} Map(UInt256,UInt256) +{1:2,2:2} Map(Int16,UInt64) +{1:2,2:2} Map(Int16,Int64) +{1:2,2:2} Map(Int32,Int64) +{1:2,2:2} Map(Int64,Int64) +{1:2,2:2} Map(Int128,Int128) +{1:2,2:2} Map(Int256,Int256) +{1:3.300000023841858,2:2} Map(UInt8,Float64) +{1:3.3000000000000003,2:2} Map(UInt8,Float64) +{'a':1,'b':2} +{'a':1,'b':1,'c':1} +{'a':1,'b':1,'d':1} +{'a':1,'b':2} Map(String,UInt64) +{'a':1,'b':1,'c':1} Map(String,UInt64) +{'a':1,'b':1,'d':1} Map(String,UInt64) +{'a':1,'b':2} +{'a':1,'b':1,'c':1} +{'a':1,'b':1,'d':1} +{'a':2} Map(Enum16(\'a\' = 1, \'b\' = 2),Int64) +{'b':2} Map(Enum16(\'a\' = 1, \'b\' = 2),Int64) +{'a':2} Map(Enum8(\'a\' = 1, \'b\' = 2),Int64) +{'b':2} Map(Enum8(\'a\' = 1, \'b\' = 2),Int64) +{'00000000-89ab-cdef-0123-456789abcdef':2} Map(UUID,Int64) +{'11111111-89ab-cdef-0123-456789abcdef':4} Map(UUID,Int64) +{1:0,2:0} Map(UInt8,UInt64) +{1:18446744073709551615,2:18446744073709551615} Map(UInt8,UInt64) +{1:-1,2:-1} Map(UInt8,Int64) +{1:-1.0999999761581423,2:0} Map(UInt8,Float64) +{1:-1,2:-1} Map(UInt8,Int64) +{1:-2,2:-2,3:1} Map(UInt8,Int64) diff --git a/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.sql b/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.sql new file mode 100644 index 00000000000..9f0f1cb0489 --- /dev/null +++ b/tests/queries/0_stateless/01318_map_add_map_subtract_on_map_type.sql @@ -0,0 +1,46 @@ +drop table if exists mapop_test; +set allow_experimental_map_type = 1; +create table mapop_test engine=TinyLog() as (select map(1, toInt32(2), number, 2) as m from numbers(1, 10)); + +-- mapAdd +select mapAdd(map(1, 1)); -- { serverError 42 } +select mapAdd(map(1, 1), m) from mapop_test; -- { serverError 43 } + +select mapAdd(map(toUInt64(1), toInt32(1)), m) from mapop_test; +select mapAdd(cast(m, 'Map(UInt8, UInt8)'), map(1, 1), map(2,2)) from mapop_test; + +-- cleanup +drop table mapop_test; + +-- check types +select mapAdd(map(toUInt8(1), 1, 2, 1), map(toUInt8(1), 1, 2, 1)) as res, toTypeName(res); +select mapAdd(map(toUInt16(1), toUInt16(1), 2, 1), map(toUInt16(1), toUInt16(1), 2, 1)) as res, toTypeName(res); +select mapAdd(map(toUInt32(1), toUInt32(1), 2, 1), map(toUInt32(1), toUInt32(1), 2, 1)) as res, toTypeName(res); +select mapAdd(map(toUInt64(1), toUInt64(1), 2, 1), map(toUInt64(1), toUInt64(1), 2, 1)) as res, toTypeName(res); +select mapAdd(map(toUInt128(1), toUInt128(1), 2, 1), map(toUInt128(1), toUInt128(1), 2, 1)) as res, toTypeName(res); +select mapAdd(map(toUInt256(1), toUInt256(1), 2, 1), map(toUInt256(1), toUInt256(1), 2, 1)) as res, toTypeName(res); + +select mapAdd(map(toInt8(1), 1, 2, 1), map(toInt8(1), 1, 2, 1)) as res, toTypeName(res); +select mapAdd(map(toInt16(1), toInt16(1), 2, 1), map(toInt16(1), toInt16(1), 2, 1)) as res, toTypeName(res); +select mapAdd(map(toInt32(1), toInt32(1), 2, 1), map(toInt32(1), toInt32(1), 2, 1)) as res, toTypeName(res); +select mapAdd(map(toInt64(1), toInt64(1), 2, 1), map(toInt64(1), toInt64(1), 2, 1)) as res, toTypeName(res); +select mapAdd(map(toInt128(1), toInt128(1), 2, 1), map(toInt128(1), toInt128(1), 2, 1)) as res, toTypeName(res); +select mapAdd(map(toInt256(1), toInt256(1), 2, 1), map(toInt256(1), toInt256(1), 2, 1)) as res, toTypeName(res); + +select mapAdd(map(1, toFloat32(1.1), 2, 1), map(1, 2.2, 2, 1)) as res, toTypeName(res); +select mapAdd(map(1, toFloat64(1.1), 2, 1), map(1, 2.2, 2, 1)) as res, toTypeName(res); +select mapAdd(map(1, toFloat64(1.1), 2, 1), map(1, 1, 2, 1)) as res, toTypeName(res); -- { serverError 43 } +select mapAdd(map('a', 1, 'b', 1), map(key, 1)) from values('key String', ('b'), ('c'), ('d')); +select mapAdd(map(cast('a', 'FixedString(1)'), 1, 'b', 1), map(key, 1)) as res, toTypeName(res) from values('key String', ('b'), ('c'), ('d')); +select mapAdd(map(cast('a', 'LowCardinality(String)'), 1, 'b', 1), map(key, 1)) from values('key String', ('b'), ('c'), ('d')); +select mapAdd(map(key, val), map(key, val)) as res, toTypeName(res) from values ('key Enum16(\'a\'=1, \'b\'=2), val Int16', ('a', 1), ('b', 1)); +select mapAdd(map(key, val), map(key, val)) as res, toTypeName(res) from values ('key Enum8(\'a\'=1, \'b\'=2), val Int16', ('a', 1), ('b', 1)); +select mapAdd(map(key, val), map(key, val)) as res, toTypeName(res) from values ('key UUID, val Int32', ('00000000-89ab-cdef-0123-456789abcdef', 1), ('11111111-89ab-cdef-0123-456789abcdef', 2)); + +-- mapSubtract, same rules as mapAdd +select mapSubtract(map(toUInt8(1), 1, 2, 1), map(toUInt8(1), 1, 2, 1)) as res, toTypeName(res); +select mapSubtract(map(toUInt8(1), 1, 2, 1), map(toUInt8(1), 2, 2, 2)) as res, toTypeName(res); -- overflow +select mapSubtract(map(toUInt8(1), toInt32(1), 2, 1), map(toUInt8(1), toInt16(2), 2, 2)) as res, toTypeName(res); +select mapSubtract(map(1, toFloat32(1.1), 2, 1), map(1, 2.2, 2, 1)) as res, toTypeName(res); +select mapSubtract(map(toUInt8(1), toInt32(1), 2, 1), map(toUInt8(1), toInt16(2), 2, 2)) as res, toTypeName(res); +select mapSubtract(map(toUInt8(3), toInt32(1)), map(toUInt8(1), toInt32(2), 2, 2)) as res, toTypeName(res);