Support Map type in mapAdd and mapSubtract

This commit is contained in:
Ildus Kurbangaliev 2021-06-22 17:02:42 +02:00
parent 1f07dd06fc
commit 0a937e7377
5 changed files with 376 additions and 143 deletions

View File

@ -73,20 +73,20 @@ Collect all the keys and sum corresponding values.
**Syntax** **Syntax**
``` sql ``` 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** **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** **Example**
Query: Query with a tuple map:
``` sql ``` sql
SELECT mapAdd(([toUInt8(1), 2], [1, 1]), ([toUInt8(1), 2], [1, 1])) as res, toTypeName(res) as type; 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} ## mapSubtract {#function-mapsubtract}
Collect all the keys and subtract corresponding values. Collect all the keys and subtract corresponding values.

View File

@ -1,13 +1,18 @@
#include <cassert> #include <cassert>
#include <Columns/ColumnVector.h> #include <Columns/ColumnFixedString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnTuple.h> #include <Columns/ColumnTuple.h>
#include <Columns/ColumnVector.h>
#include <Columns/IColumn.h>
#include <Core/ColumnWithTypeAndName.h>
#include <DataTypes/DataTypeArray.h> #include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h> #include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h> #include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h> #include <Functions/FunctionHelpers.h>
#include <common/arithmeticOverflow.h> #include <common/arithmeticOverflow.h>
#include <Core/ColumnWithTypeAndName.h> #include "Columns/ColumnMap.h"
#include "DataTypes/DataTypeMap.h"
namespace DB namespace DB
@ -24,8 +29,8 @@ namespace
struct TupArg struct TupArg
{ {
const IColumn & key_column; const ColumnPtr & key_column;
const IColumn & val_column; const ColumnPtr & val_column;
const IColumn::Offsets & key_offsets; const IColumn::Offsets & key_offsets;
const IColumn::Offsets & val_offsets; const IColumn::Offsets & val_offsets;
bool is_const; bool is_const;
@ -52,17 +57,39 @@ private:
bool isVariadic() const override { return true; } bool isVariadic() const override { return true; }
bool useDefaultImplementationForConstants() 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; DataTypePtr key_type, val_type, res;
if (arguments.size() < 2) for (const auto & arg : arguments)
throw Exception{getName() + " accepts at least two map tuples", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
for (const auto & tup_arg : arguments)
{ {
const DataTypeTuple * tup = checkAndGetDataType<DataTypeTuple>(tup_arg.get()); const DataTypeArray * k;
const DataTypeArray * v;
const DataTypeTuple * tup = checkAndGetDataType<DataTypeTuple>(arg.get());
if (!tup) if (!tup)
throw Exception{getName() + " accepts at least two map tuples", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; throw Exception{getName() + " accepts at least two map tuples", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
@ -71,8 +98,8 @@ private:
throw Exception( throw Exception(
"Each tuple in " + getName() + " arguments should consist of two arrays", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); "Each tuple in " + getName() + " arguments should consist of two arrays", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * k = checkAndGetDataType<DataTypeArray>(elems[0].get()); k = checkAndGetDataType<DataTypeArray>(elems[0].get());
const DataTypeArray * v = checkAndGetDataType<DataTypeArray>(elems[1].get()); v = checkAndGetDataType<DataTypeArray>(elems[1].get());
if (!k || !v) if (!k || !v)
throw Exception( throw Exception(
@ -80,62 +107,100 @@ private:
auto result_type = v->getNestedType(); auto result_type = v->getNestedType();
if (!result_type->canBePromoted()) if (!result_type->canBePromoted())
throw Exception{"Values to be summed are expected to be Numeric, Float or Decimal.", throw Exception{
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; "Values to be summed are expected to be Numeric, Float or Decimal.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
WhichDataType which_val(result_type); auto promoted_val_type = result_type->promoteNumericType();
auto promoted_type = result_type->promoteNumericType();
if (!key_type) if (!key_type)
{ {
key_type = k->getNestedType(); key_type = k->getNestedType();
val_type = promoted_type; val_type = promoted_val_type;
is_float = which_val.isFloat(); res = std::make_shared<DataTypeTuple>(
DataTypes{std::make_shared<DataTypeArray>(k->getNestedType()), std::make_shared<DataTypeArray>(promoted_val_type)});
} }
else else
{ checkTypes(key_type, val_type, k->getNestedType(), promoted_val_type);
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<DataTypeTuple>(
DataTypes{std::make_shared<DataTypeArray>(k->getNestedType()), std::make_shared<DataTypeArray>(promoted_type)});
}
} }
return res; return res;
} }
template <typename KeyType, bool is_str_key, typename ValType> DataTypePtr getReturnTypeForMaps(const DataTypes & arguments) const
ColumnPtr execute2(size_t row_count, TupleMaps & args, const DataTypeTuple & res_type) const
{ {
MutableColumnPtr res_tuple = res_type.createColumn(); DataTypePtr key_type, val_type, res;
auto * to_tuple = assert_cast<ColumnTuple *>(res_tuple.get()); for (const auto & arg : arguments)
{
const auto * map = checkAndGetDataType<DataTypeMap>(arg.get());
if (!map)
throw Exception{getName() + " accepts at least two maps", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
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<DataTypeMap>(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 <typename KeyType, typename ValType>
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<ColumnTuple *>(res_column.get());
auto & to_keys_arr = assert_cast<ColumnArray &>(to_tuple->getColumn(0)); auto & to_keys_arr = assert_cast<ColumnArray &>(to_tuple->getColumn(0));
auto & to_keys_data = to_keys_arr.getData(); to_keys_data = &to_keys_arr.getData();
auto & to_keys_offset = to_keys_arr.getOffsets(); to_keys_offset = &to_keys_arr.getOffsets();
auto & to_vals_arr = assert_cast<ColumnArray &>(to_tuple->getColumn(1)); auto & to_vals_arr = assert_cast<ColumnArray &>(to_tuple->getColumn(1));
auto & to_vals_data = to_vals_arr.getData(); 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<ColumnMap *>(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<KeyType, ValType> summing_map; std::map<KeyType, ValType> summing_map;
for (size_t i = 0; i < row_count; i++) for (size_t i = 0; i < row_count; i++)
@ -147,7 +212,7 @@ private:
if (!arg.is_const) 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; len = arg.key_offsets[i] - offset;
if (arg.val_offsets[i] != arg.key_offsets[i]) 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); "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++) for (size_t j = 0; j < len; j++)
{ {
KeyType key; KeyType key;
if constexpr (is_str_key) if constexpr (std::is_same<KeyType, String>::value)
{ {
// have to use Field to get strings if (const auto * col_fixed = checkAndGetColumn<ColumnFixedString>(arg.key_column.get()))
key = arg.key_column[offset + j].get<KeyType>(); key = col_fixed->getDataAt(offset + j).toString();
else if (const auto * col_str = checkAndGetColumn<ColumnString>(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 else
{ {
key = assert_cast<const ColumnVector<KeyType> &>(arg.key_column).getData()[offset + j]; key = assert_cast<const ColumnVector<KeyType> *>(arg.key_column.get())->getData()[offset + j];
} }
ValType value = arg.val_column[offset + j].get<ValType>(); arg.val_column->get(offset + j, temp_val);
ValType value = temp_val.get<ValType>();
if constexpr (op_type == OpTypes::ADD) if constexpr (op_type == OpTypes::ADD)
{ {
@ -190,60 +265,69 @@ private:
for (const auto & elem : summing_map) for (const auto & elem : summing_map)
{ {
res_offset++; to_keys_data->insert(elem.first);
to_keys_data.insert(elem.first); to_vals_data->insert(elem.second);
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(); summing_map.clear();
} }
if (to_vals_offset)
{
// same offsets as in keys // same offsets as in keys
to_vals_arr.getOffsets().insert(to_keys_offset.begin(), to_keys_offset.end()); to_vals_offset->insert(to_keys_offset->begin(), to_keys_offset->end());
return res_tuple;
} }
template <typename KeyType, bool is_str_key> return res_column;
ColumnPtr execute1(size_t row_count, const DataTypeTuple & res_type, TupleMaps & args) const
{
const auto & promoted_type = (assert_cast<const DataTypeArray *>(res_type.getElements()[1].get()))->getNestedType();
#define MATCH_EXECUTE(is_str) \
switch (promoted_type->getTypeId()) \
{ \
case TypeIndex::Int64: return execute2<KeyType, is_str, Int64>(row_count, args, res_type); \
case TypeIndex::UInt64: return execute2<KeyType, is_str, UInt64>(row_count, args, res_type); \
case TypeIndex::Float64: return execute2<KeyType, is_str, Float64>(row_count, args, res_type); \
default: \
throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; \
} }
if constexpr (is_str_key) template <typename KeyType>
ColumnPtr execute1(size_t row_count, const DataTypePtr res_type, const DataTypePtr res_value_type, TupleMaps & args) const
{ {
MATCH_EXECUTE(true) switch (res_value_type->getTypeId())
}
else
{ {
MATCH_EXECUTE(false) case TypeIndex::Int64:
return execute2<KeyType, Int64>(row_count, args, res_type);
case TypeIndex::Int128:
return execute2<KeyType, Int128>(row_count, args, res_type);
case TypeIndex::Int256:
return execute2<KeyType, Int256>(row_count, args, res_type);
case TypeIndex::UInt64:
return execute2<KeyType, UInt64>(row_count, args, res_type);
case TypeIndex::UInt128:
return execute2<KeyType, UInt128>(row_count, args, res_type);
case TypeIndex::UInt256:
return execute2<KeyType, UInt256>(row_count, args, res_type);
case TypeIndex::Float64:
return execute2<KeyType, Float64>(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};
} }
#undef MATCH_EXECUTE
} }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
{ {
DataTypePtr key_type;
size_t row_count;
const DataTypeTuple * tup_type = checkAndGetDataType<DataTypeTuple>((arguments[0]).type.get()); const DataTypeTuple * tup_type = checkAndGetDataType<DataTypeTuple>((arguments[0]).type.get());
const DataTypeArray * key_array_type = checkAndGetDataType<DataTypeArray>(tup_type->getElements()[0].get()); DataTypePtr res_type;
const DataTypeArray * val_array_type = checkAndGetDataType<DataTypeArray>(tup_type->getElements()[1].get()); DataTypePtr res_value_type;
/* determine output type */
const DataTypeTuple & res_type
= DataTypeTuple(DataTypes{std::make_shared<DataTypeArray>(key_array_type->getNestedType()),
std::make_shared<DataTypeArray>(val_array_type->getNestedType()->promoteNumericType())});
TupleMaps args{}; TupleMaps args{};
args.reserve(arguments.size()); args.reserve(arguments.size());
//prepare columns, extract data columns for direct access and put them to the vector //prepare columns, extract data columns for direct access and put them to the vector
if (tup_type)
{
const DataTypeArray * key_array_type = checkAndGetDataType<DataTypeArray>(tup_type->getElements()[0].get());
const DataTypeArray * val_array_type = checkAndGetDataType<DataTypeArray>(tup_type->getElements()[1].get());
/* determine output type */
res_value_type = val_array_type->getNestedType()->promoteNumericType();
res_type = std::make_shared<DataTypeTuple>(DataTypes{
std::make_shared<DataTypeArray>(key_array_type->getNestedType()), std::make_shared<DataTypeArray>(res_value_type)});
for (const auto & col : arguments) for (const auto & col : arguments)
{ {
const ColumnTuple * tup; const ColumnTuple * tup;
@ -260,62 +344,105 @@ private:
const auto & arr2 = assert_cast<const ColumnArray &>(tup->getColumn(1)); const auto & arr2 = assert_cast<const ColumnArray &>(tup->getColumn(1));
const auto & key_offsets = arr1.getOffsets(); const auto & key_offsets = arr1.getOffsets();
const auto & key_column = arr1.getData(); const auto & key_column = arr1.getDataPtr();
const auto & val_offsets = arr2.getOffsets(); const auto & val_offsets = arr2.getOffsets();
const auto & val_column = arr2.getData(); const auto & val_column = arr2.getDataPtr();
// we can check const columns before any processing
if (is_const)
{
if (val_offsets[0] != 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}); args.push_back({key_column, val_column, key_offsets, val_offsets, is_const});
} }
size_t row_count = arguments[0].column->size(); key_type = key_array_type->getNestedType();
auto key_type_id = key_array_type->getNestedType()->getTypeId(); }
else
{
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>((arguments[0]).type.get());
if (map_type)
{
key_type = map_type->getKeyType();
res_value_type = map_type->getValueType()->promoteNumericType();
res_type = std::make_shared<DataTypeMap>(DataTypes{map_type->getKeyType(), res_value_type});
switch (key_type_id) for (const auto & col : arguments)
{
const ColumnMap * map;
bool is_const = isColumnConst(*col.column);
if (is_const)
{
const auto * c = assert_cast<const ColumnConst *>(col.column.get());
map = assert_cast<const ColumnMap *>(c->getDataColumnPtr().get());
}
else
map = assert_cast<const ColumnMap *>(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
throw Exception{
"Illegal column type " + key_type->getName() + " in arguments of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
// we can check const columns before any processing
for (auto & arg : args)
{
if (arg.is_const)
{
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);
}
}
row_count = arguments[0].column->size();
switch (key_type->getTypeId())
{ {
case TypeIndex::Enum8: case TypeIndex::Enum8:
case TypeIndex::Int8: case TypeIndex::Int8:
return execute1<Int8, false>(row_count, res_type, args); return execute1<Int8>(row_count, res_type, res_value_type, args);
case TypeIndex::Enum16: case TypeIndex::Enum16:
case TypeIndex::Int16: case TypeIndex::Int16:
return execute1<Int16, false>(row_count, res_type, args); return execute1<Int16>(row_count, res_type, res_value_type, args);
case TypeIndex::Int32: case TypeIndex::Int32:
return execute1<Int32, false>(row_count, res_type, args); return execute1<Int32>(row_count, res_type, res_value_type, args);
case TypeIndex::Int64: case TypeIndex::Int64:
return execute1<Int64, false>(row_count, res_type, args); return execute1<Int64>(row_count, res_type, res_value_type, args);
case TypeIndex::Int128: case TypeIndex::Int128:
return execute1<Int128, false>(row_count, res_type, args); return execute1<Int128>(row_count, res_type, res_value_type, args);
case TypeIndex::Int256: case TypeIndex::Int256:
return execute1<Int256, false>(row_count, res_type, args); return execute1<Int256>(row_count, res_type, res_value_type, args);
case TypeIndex::UInt8: case TypeIndex::UInt8:
return execute1<UInt8, false>(row_count, res_type, args); return execute1<UInt8>(row_count, res_type, res_value_type, args);
case TypeIndex::Date: case TypeIndex::Date:
case TypeIndex::UInt16: case TypeIndex::UInt16:
return execute1<UInt16, false>(row_count, res_type, args); return execute1<UInt16>(row_count, res_type, res_value_type, args);
case TypeIndex::DateTime: case TypeIndex::DateTime:
case TypeIndex::UInt32: case TypeIndex::UInt32:
return execute1<UInt32, false>(row_count, res_type, args); return execute1<UInt32>(row_count, res_type, res_value_type, args);
case TypeIndex::UInt64: case TypeIndex::UInt64:
return execute1<UInt64, false>(row_count, res_type, args); return execute1<UInt64>(row_count, res_type, res_value_type, args);
case TypeIndex::UInt128: case TypeIndex::UInt128:
return execute1<UInt128, false>(row_count, res_type, args); return execute1<UInt128>(row_count, res_type, res_value_type, args);
case TypeIndex::UInt256: case TypeIndex::UInt256:
return execute1<UInt256, false>(row_count, res_type, args); return execute1<UInt256>(row_count, res_type, res_value_type, args);
case TypeIndex::UUID: case TypeIndex::UUID:
return execute1<UUID, false>(row_count, res_type, args); return execute1<UUID>(row_count, res_type, res_value_type, args);
case TypeIndex::FixedString: case TypeIndex::FixedString:
case TypeIndex::String: case TypeIndex::String:
return execute1<String, true>(row_count, res_type, args); return execute1<String>(row_count, res_type, res_value_type, args);
default: 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};
} }
} }
}; };

View File

@ -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)); create table map_test engine=TinyLog() as (select ([1, number], [toInt32(2),2]) as map from numbers(1, 10));
-- mapAdd -- mapAdd
select mapAdd([1], [1]); -- { serverError 42 } select mapAdd([1], [1]); -- { serverError 43 }
select mapAdd(([1], [1])); -- { serverError 42 } select mapAdd(([1], [1])); -- { serverError 42 }
select mapAdd(([1], [1]), map) from map_test; -- { serverError 43 } select mapAdd(([1], [1]), map) from map_test; -- { serverError 43 }
select mapAdd(([toUInt64(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], [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(([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(([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((['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')); select mapAdd((cast(['a', 'b'], 'Array(FixedString(1))'), [1, 1]), ([key], [1])) as res, toTypeName(res) from values('key FixedString(1)', ('b'), ('c'), ('d'));

View File

@ -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)

View File

@ -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);