From d6fe824340cb4ac7956f72899a0ef498be6c352a Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 21 Jul 2021 09:20:17 +0200 Subject: [PATCH 1/7] Add Map type support in mapPopulateSeries --- src/Functions/array/mapPopulateSeries.cpp | 351 ++++++++++++------ ...01925_map_populate_series_on_map.reference | 37 ++ .../01925_map_populate_series_on_map.sql | 35 ++ 3 files changed, 319 insertions(+), 104 deletions(-) create mode 100644 tests/queries/0_stateless/01925_map_populate_series_on_map.reference create mode 100644 tests/queries/0_stateless/01925_map_populate_series_on_map.sql diff --git a/src/Functions/array/mapPopulateSeries.cpp b/src/Functions/array/mapPopulateSeries.cpp index eb2f6192346..f8bcbf8d451 100644 --- a/src/Functions/array/mapPopulateSeries.cpp +++ b/src/Functions/array/mapPopulateSeries.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -7,6 +8,7 @@ #include #include #include "Core/ColumnWithTypeAndName.h" +#include "DataTypes/DataTypeMap.h" #include "DataTypes/IDataType.h" namespace DB @@ -32,31 +34,17 @@ private: bool isVariadic() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + void checkTypes(const DataTypePtr & key_type, const DataTypePtr max_key_type) const { - if (arguments.size() < 2) - throw Exception{getName() + " accepts at least two arrays for key and value", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - if (arguments.size() > 3) - throw Exception{"too many arguments in " + getName() + " call", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - const DataTypeArray * key_array_type = checkAndGetDataType(arguments[0].get()); - const DataTypeArray * val_array_type = checkAndGetDataType(arguments[1].get()); - - if (!key_array_type || !val_array_type) - throw Exception{getName() + " accepts two arrays for key and value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - DataTypePtr keys_type = key_array_type->getNestedType(); - WhichDataType which_key(keys_type); - if (!(which_key.isNativeInt() || which_key.isNativeUInt())) + WhichDataType which_key(key_type); + if (!(which_key.isInt() || which_key.isUInt())) { throw Exception( - "Keys for " + getName() + " should be of native integer type (signed or unsigned)", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + "Keys for " + getName() + " should be of integer type (signed or unsigned)", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } - if (arguments.size() == 3) + if (max_key_type) { - DataTypePtr max_key_type = arguments[2]; WhichDataType which_max_key(max_key_type); if (which_max_key.isNullable()) @@ -64,53 +52,186 @@ private: "Max key argument in arguments of function " + getName() + " can not be Nullable", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (keys_type->getTypeId() != max_key_type->getTypeId()) + if (key_type->getTypeId() != max_key_type->getTypeId()) throw Exception("Max key type in " + getName() + " should be same as keys type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } + } + + DataTypePtr getReturnTypeForTuple(const DataTypes & arguments) const + { + if (arguments.size() < 2) + throw Exception(getName() + " accepts at least two arrays for key and value", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (arguments.size() > 3) + throw Exception("too many arguments in " + getName() + " call", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const DataTypeArray * key_array_type = checkAndGetDataType(arguments[0].get()); + const DataTypeArray * val_array_type = checkAndGetDataType(arguments[1].get()); + + if (!key_array_type || !val_array_type) + throw Exception(getName() + " accepts two arrays for key and value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto & key_type = key_array_type->getNestedType(); + + if (arguments.size() == 3) + this->checkTypes(key_type, arguments[2]); + else + this->checkTypes(key_type, nullptr); return std::make_shared(DataTypes{arguments[0], arguments[1]}); } - template - ColumnPtr execute2(ColumnPtr key_column, ColumnPtr val_column, ColumnPtr max_key_column, const DataTypeTuple & res_type) const + DataTypePtr getReturnTypeForMap(const DataTypes & arguments) const { - MutableColumnPtr res_tuple = res_type.createColumn(); + const auto * map = assert_cast(arguments[0].get()); + if (arguments.size() == 1) + this->checkTypes(map->getKeyType(), nullptr); + else if (arguments.size() == 2) + this->checkTypes(map->getKeyType(), arguments[1]); + else + throw Exception("too many arguments in " + getName() + " call", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - 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_offsets = to_keys_arr.getOffsets(); + return std::make_shared(map->getKeyType(), map->getValueType()); + } - auto & to_vals_arr = assert_cast(to_tuple->getColumn(1)); - auto & to_values_data = to_vals_arr.getData(); + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception(getName() + " accepts at least one map", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - bool max_key_is_const = false, key_is_const = false, val_is_const = false; + if (arguments[0]->getTypeId() == TypeIndex::Array) + return getReturnTypeForTuple(arguments); + else if (arguments[0]->getTypeId() == TypeIndex::Map) + return getReturnTypeForMap(arguments); + else + throw Exception(getName() + " only accepts maps", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } - const auto * keys_array = checkAndGetColumn(key_column.get()); - if (!keys_array) + // Struct holds input and output columns references, + // Both arrays and maps have similar columns to work with but extracted differently + template + struct ColumnsInOut + { + // inputs + const PaddedPODArray & in_keys_data; + const PaddedPODArray & in_vals_data; + const IColumn::Offsets & in_key_offsets; + const IColumn::Offsets & in_val_offsets; + size_t row_count; + bool key_is_const; + bool val_is_const; + + // outputs + PaddedPODArray & out_keys_data; + PaddedPODArray & out_vals_data; + + IColumn::Offsets & out_keys_offsets; + // with map argument this field will not be used + IColumn::Offsets * out_vals_offsets; + }; + + template + ColumnsInOut getInOutDataFromArrays(MutableColumnPtr & res_column, ColumnPtr * arg_columns) const + { + auto * out_tuple = assert_cast(res_column.get()); + auto & out_keys_array = assert_cast(out_tuple->getColumn(0)); + auto & out_vals_array = assert_cast(out_tuple->getColumn(1)); + + const auto * key_column = arg_columns[0].get(); + const auto * in_keys_array = checkAndGetColumn(key_column); + + bool key_is_const = false, val_is_const = false; + + if (!in_keys_array) { - const ColumnConst * const_array = checkAndGetColumnConst(key_column.get()); + const ColumnConst * const_array = checkAndGetColumnConst(key_column); if (!const_array) throw Exception("Expected array column, found " + key_column->getName(), ErrorCodes::ILLEGAL_COLUMN); - keys_array = checkAndGetColumn(const_array->getDataColumnPtr().get()); + in_keys_array = checkAndGetColumn(const_array->getDataColumnPtr().get()); key_is_const = true; } - const auto * values_array = checkAndGetColumn(val_column.get()); - if (!values_array) + const auto * val_column = arg_columns[1].get(); + const auto * in_values_array = checkAndGetColumn(val_column); + if (!in_values_array) { - const ColumnConst * const_array = checkAndGetColumnConst(val_column.get()); + const ColumnConst * const_array = checkAndGetColumnConst(val_column); if (!const_array) throw Exception("Expected array column, found " + val_column->getName(), ErrorCodes::ILLEGAL_COLUMN); - values_array = checkAndGetColumn(const_array->getDataColumnPtr().get()); + in_values_array = checkAndGetColumn(const_array->getDataColumnPtr().get()); val_is_const = true; } - if (!keys_array || !values_array) + if (!in_keys_array || !in_values_array) /* something went wrong */ - throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + + const auto & in_keys_data = assert_cast &>(in_keys_array->getData()).getData(); + const auto & in_values_data = assert_cast &>(in_values_array->getData()).getData(); + const auto & in_keys_offsets = in_keys_array->getOffsets(); + const auto & in_vals_offsets = in_values_array->getOffsets(); + + auto & out_keys_data = assert_cast &>(out_keys_array.getData()).getData(); + auto & out_vals_data = assert_cast &>(out_vals_array.getData()).getData(); + auto & out_keys_offsets = out_keys_array.getOffsets(); + + size_t row_count = key_is_const ? in_values_array->size() : in_keys_array->size(); + IColumn::Offsets * out_vals_offsets = &out_vals_array.getOffsets(); + + return { + in_keys_data, + in_values_data, + in_keys_offsets, + in_vals_offsets, + row_count, + key_is_const, + val_is_const, + out_keys_data, + out_vals_data, + out_keys_offsets, + out_vals_offsets}; + } + + template + ColumnsInOut getInOutDataFromMap(MutableColumnPtr & res_column, ColumnPtr * arg_columns) const + { + const auto * in_map = assert_cast(arg_columns[0].get()); + const auto & in_nested_array = in_map->getNestedColumn(); + const auto & in_nested_tuple = in_map->getNestedData(); + const auto & in_keys_data = assert_cast &>(in_nested_tuple.getColumn(0)).getData(); + const auto & in_vals_data = assert_cast &>(in_nested_tuple.getColumn(1)).getData(); + const auto & in_keys_offsets = in_nested_array.getOffsets(); + + auto * out_map = assert_cast(res_column.get()); + auto & out_nested_array = out_map->getNestedColumn(); + auto & out_nested_tuple = out_map->getNestedData(); + auto & out_keys_data = assert_cast &>(out_nested_tuple.getColumn(0)).getData(); + auto & out_vals_data = assert_cast &>(out_nested_tuple.getColumn(1)).getData(); + auto & out_keys_offsets = out_nested_array.getOffsets(); + + return { + in_keys_data, + in_vals_data, + in_keys_offsets, + in_keys_offsets, + in_nested_array.size(), + false, + false, + out_keys_data, + out_vals_data, + out_keys_offsets, + nullptr}; + } + + template + ColumnPtr execute2(ColumnPtr * arg_columns, ColumnPtr max_key_column, const DataTypePtr & res_type) const + { + MutableColumnPtr res_column = res_type->createColumn(); + bool max_key_is_const = false; + auto inout = res_column->getDataType() == TypeIndex::Tuple ? getInOutDataFromArrays(res_column, arg_columns) + : getInOutDataFromMap(res_column, arg_columns); KeyType max_key_const{0}; @@ -121,49 +242,40 @@ private: max_key_is_const = true; } - auto & keys_data = assert_cast &>(keys_array->getData()).getData(); - auto & values_data = assert_cast &>(values_array->getData()).getData(); - - // Original offsets - const IColumn::Offsets & key_offsets = keys_array->getOffsets(); - const IColumn::Offsets & val_offsets = values_array->getOffsets(); - IColumn::Offset offset{0}; - size_t row_count = key_is_const ? values_array->size() : keys_array->size(); - std::map res_map; //Iterate through two arrays and fill result values. - for (size_t row = 0; row < row_count; ++row) + for (size_t row = 0; row < inout.row_count; ++row) { - size_t key_offset = 0, val_offset = 0, array_size = key_offsets[0], val_array_size = val_offsets[0]; + size_t key_offset = 0, val_offset = 0, items_count = inout.in_key_offsets[0], val_array_size = inout.in_val_offsets[0]; res_map.clear(); - if (!key_is_const) + if (!inout.key_is_const) { - key_offset = row > 0 ? key_offsets[row - 1] : 0; - array_size = key_offsets[row] - key_offset; + key_offset = row > 0 ? inout.in_key_offsets[row - 1] : 0; + items_count = inout.in_key_offsets[row] - key_offset; } - if (!val_is_const) + if (!inout.val_is_const) { - val_offset = row > 0 ? val_offsets[row - 1] : 0; - val_array_size = val_offsets[row] - val_offset; + val_offset = row > 0 ? inout.in_val_offsets[row - 1] : 0; + val_array_size = inout.in_val_offsets[row] - val_offset; } - if (array_size != val_array_size) + if (items_count != val_array_size) throw Exception("Key and value array should have same amount of elements", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (array_size == 0) + if (items_count == 0) { - to_keys_offsets.push_back(offset); + inout.out_keys_offsets.push_back(offset); continue; } - for (size_t i = 0; i < array_size; ++i) + for (size_t i = 0; i < items_count; ++i) { - res_map.insert({keys_data[key_offset + i], values_data[val_offset + i]}); + res_map.insert({inout.in_keys_data[key_offset + i], inout.in_vals_data[val_offset + i]}); } auto min_key = res_map.begin()->first; @@ -184,7 +296,7 @@ private: /* no need to add anything, max key is less that first key */ if (max_key < min_key) { - to_keys_offsets.push_back(offset); + inout.out_keys_offsets.push_back(offset); continue; } } @@ -197,16 +309,16 @@ private: KeyType key; for (key = min_key;; ++key) { - to_keys_data.insert(key); + inout.out_keys_data.push_back(key); auto it = res_map.find(key); if (it != res_map.end()) { - to_values_data.insert(it->second); + inout.out_vals_data.push_back(it->second); } else { - to_values_data.insertDefault(); + inout.out_vals_data.push_back(0); } ++offset; @@ -214,80 +326,112 @@ private: break; } - to_keys_offsets.push_back(offset); + inout.out_keys_offsets.push_back(offset); } - to_vals_arr.getOffsets().insert(to_keys_offsets.begin(), to_keys_offsets.end()); - return res_tuple; + if (inout.out_vals_offsets) + inout.out_vals_offsets->insert(inout.out_keys_offsets.begin(), inout.out_keys_offsets.end()); + + return res_column; } template - ColumnPtr execute1(ColumnPtr key_column, ColumnPtr val_column, ColumnPtr max_key_column, const DataTypeTuple & res_type) const + ColumnPtr execute1(ColumnPtr * arg_columns, ColumnPtr max_key_column, const DataTypePtr & res_type, const DataTypePtr & val_type) const { - const auto & val_type = (assert_cast(res_type.getElements()[1].get()))->getNestedType(); switch (val_type->getTypeId()) { case TypeIndex::Int8: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::Int16: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::Int32: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::Int64: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); + case TypeIndex::Int128: + return execute2(arg_columns, max_key_column, res_type); + case TypeIndex::Int256: + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::UInt8: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::UInt16: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::UInt32: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); case TypeIndex::UInt64: - return execute2(key_column, val_column, max_key_column, res_type); + return execute2(arg_columns, max_key_column, res_type); + case TypeIndex::UInt128: + return execute2(arg_columns, max_key_column, res_type); + case TypeIndex::UInt256: + return execute2(arg_columns, max_key_column, res_type); default: - throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override { - auto col1 = arguments[0]; - auto col2 = arguments[1]; - - const auto * k = assert_cast(col1.type.get()); - const auto * v = assert_cast(col2.type.get()); - - /* determine output type */ - const DataTypeTuple & res_type = DataTypeTuple( - DataTypes{std::make_shared(k->getNestedType()), std::make_shared(v->getNestedType())}); - + DataTypePtr res_type, key_type, val_type; ColumnPtr max_key_column = nullptr; + ColumnPtr arg_columns[] = {arguments[0].column, nullptr}; - if (arguments.size() == 3) + if (arguments[0].type->getTypeId() == TypeIndex::Array) { - /* max key provided */ - max_key_column = arguments[2].column; + key_type = assert_cast(arguments[0].type.get())->getNestedType(); + val_type = assert_cast(arguments[1].type.get())->getNestedType(); + res_type = getReturnTypeImpl(DataTypes{arguments[0].type, arguments[1].type}); + + arg_columns[1] = arguments[1].column; + if (arguments.size() == 3) + { + /* max key provided */ + max_key_column = arguments[2].column; + } + } + else + { + assert(arguments[0].type->getTypeId() == TypeIndex::Map); + + const auto * map_type = assert_cast(arguments[0].type.get()); + res_type = getReturnTypeImpl(DataTypes{arguments[0].type}); + key_type = map_type->getKeyType(); + val_type = map_type->getValueType(); + + if (arguments.size() == 2) + { + /* max key provided */ + max_key_column = arguments[1].column; + } } - switch (k->getNestedType()->getTypeId()) + switch (key_type->getTypeId()) { case TypeIndex::Int8: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::Int16: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::Int32: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::Int64: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); + case TypeIndex::Int128: + return execute1(arg_columns, max_key_column, res_type, val_type); + case TypeIndex::Int256: + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::UInt8: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::UInt16: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::UInt32: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); case TypeIndex::UInt64: - return execute1(col1.column, col2.column, max_key_column, res_type); + return execute1(arg_columns, max_key_column, res_type, val_type); + case TypeIndex::UInt128: + return execute1(arg_columns, max_key_column, res_type, val_type); + case TypeIndex::UInt256: + return execute1(arg_columns, max_key_column, res_type, val_type); default: - throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } } }; @@ -296,5 +440,4 @@ void registerFunctionMapPopulateSeries(FunctionFactory & factory) { factory.registerFunction(); } - } diff --git a/tests/queries/0_stateless/01925_map_populate_series_on_map.reference b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference new file mode 100644 index 00000000000..74ed35f0638 --- /dev/null +++ b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference @@ -0,0 +1,37 @@ +{1:1} +{1:1,2:2} +{1:1,2:0,3:2} +{1:1,2:0,3:0,4:2} +{1:1,2:0,3:0,4:0,5:2} +{1:1,2:0,3:0} +{1:1,2:2,3:0} +{1:1,2:0,3:2} +{1:1,2:0,3:0} +{1:1,2:0,3:0} +{1:1,2:0,3:0,4:0,5:0,6:0,7:0,8:0,9:0,10:0} +{1:1,2:2,3:0,4:0,5:0,6:0,7:0,8:0,9:0,10:0} +{1:1,2:0,3:2,4:0,5:0,6:0,7:0,8:0,9:0,10:0} +{1:1,2:0,3:0,4:2,5:0,6:0,7:0,8:0,9:0,10:0} +{1:1,2:0,3:0,4:0,5:2,6:0,7:0,8:0,9:0,10:0} +{1:1,2:0} +{1:1,2:2,3:0} +{1:1,2:0,3:2,4:0} +{1:1,2:0,3:0,4:2,5:0} +{1:1,2:0,3:0,4:0,5:2,6:0} +{1:1,2:1} Map(UInt8,UInt8) +{1:1,2:1} Map(UInt16,UInt16) +{1:1,2:1} Map(UInt32,UInt32) +{1:1,2:1} Map(UInt64,UInt64) +{1:1,2:1} Map(UInt128,UInt128) +{1:1,2:1} Map(UInt256,UInt256) +{1:1,2:1} Map(Int16,Int16) +{1:1,2:1} Map(Int16,Int16) +{1:1,2:1} Map(Int32,Int32) +{1:1,2:1} Map(Int64,Int64) +{1:1,2:1} Map(Int128,Int128) +{1:1,2:1} Map(Int256,Int256) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int32,Int32) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int64,Int64) +{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0} Map(Int64,Int64) diff --git a/tests/queries/0_stateless/01925_map_populate_series_on_map.sql b/tests/queries/0_stateless/01925_map_populate_series_on_map.sql new file mode 100644 index 00000000000..f45543a8dba --- /dev/null +++ b/tests/queries/0_stateless/01925_map_populate_series_on_map.sql @@ -0,0 +1,35 @@ +drop table if exists map_test; +set allow_experimental_map_type = 1; +create table map_test engine=TinyLog() as (select (number + 1) as n, map(1, 1, number,2) as m from numbers(1, 5)); + +select mapPopulateSeries(m) from map_test; +select mapPopulateSeries(m, toUInt64(3)) from map_test; +select mapPopulateSeries(m, toUInt64(10)) from map_test; +select mapPopulateSeries(m, 1000) from map_test; -- { serverError 43 } +select mapPopulateSeries(m, n) from map_test; + +drop table map_test; + +select mapPopulateSeries(map(toUInt8(1), toUInt8(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toUInt16(1), toUInt16(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toUInt32(1), toUInt32(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toUInt64(1), toUInt64(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toUInt128(1), toUInt128(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toUInt256(1), toUInt256(1), 2, 1)) as res, toTypeName(res); + +select mapPopulateSeries(map(toInt8(1), toInt8(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt16(1), toInt16(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt32(1), toInt32(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt64(1), toInt64(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt128(1), toInt128(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt256(1), toInt256(1), 2, 1)) as res, toTypeName(res); + +select mapPopulateSeries(map(toInt8(-10), toInt8(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt16(-10), toInt16(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt32(-10), toInt32(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1)) as res, toTypeName(res); +select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1), toInt64(-5)) as res, toTypeName(res); + +select mapPopulateSeries(); -- { serverError 42 } +select mapPopulateSeries('asdf'); -- { serverError 43 } +select mapPopulateSeries(map('1', 1, '2', 1)) as res, toTypeName(res); -- { serverError 43 } From 0f480e6b06b2d9ddfeb8a900fb0f6fbf8782757c Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 21 Jul 2021 10:08:29 +0200 Subject: [PATCH 2/7] Fix docs for map functions --- .../functions/tuple-map-functions.md | 77 +++++++++++++++---- 1 file changed, 61 insertions(+), 16 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index dcfa18e04bf..908d3e1bbd3 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -103,81 +103,126 @@ Result: Query with `Map` type: ``` sql +SELECT mapAdd(map(1,1), map(1,1)); +``` + +Result: + +``` text +┌─mapAdd(map(1, 1), map(1, 1))─┐ +│ {1:2} │ +└──────────────────────────────┘ ``` ## mapSubtract {#function-mapsubtract} Collect all the keys and subtract corresponding values. -**Syntax** +**Syntax** ``` sql mapSubtract(Tuple(Array, Array), Tuple(Array, Array) [, ...]) ``` -**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) or [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 +``` sql SELECT mapSubtract(([toUInt8(1), 2], [toInt32(1), 1]), ([toUInt8(1), 2], [toInt32(2), 1])) as res, toTypeName(res) as type; ``` Result: -```text +``` text ┌─res────────────┬─type──────────────────────────────┐ │ ([1,2],[-1,0]) │ Tuple(Array(UInt8), Array(Int64)) │ └────────────────┴───────────────────────────────────┘ ``` +Query with `Map` type: + +``` sql +SELECT mapSubtract(map(1,1), map(1,1)); +``` + +Result: + +``` text +┌─mapSubtract(map(1, 1), map(1, 1))─┐ +│ {1:0} │ +└───────────────────────────────────┘ +``` + ## mapPopulateSeries {#function-mappopulateseries} Fills missing keys in the maps (key and value array pair), where keys are integers. Also, it supports specifying the max key, which is used to extend the keys array. +Arguments are [maps](../../sql-reference/data-types/map.md) or two [arrays](../../sql-reference/data-types/array.md#data-type-array), where the first array represent keys, and the second array contains values for the each key. -**Syntax** +For array arguments the number of elements in `keys` and `values` must be the same for each row. + +**Syntax** ``` sql mapPopulateSeries(keys, values[, max]) +mapPopulateSeries(map[, max]) ``` -Generates a map, where keys are a series of numbers, from minimum to maximum keys (or `max` argument if it specified) taken from `keys` array with a step size of one, and corresponding values taken from `values` array. If the value is not specified for the key, then it uses the default value in the resulting map. For repeated keys, only the first value (in order of appearing) gets associated with the key. - -The number of elements in `keys` and `values` must be the same for each row. +Generates a map (a tuple with two arrays or a value of `Map` type, depending on the arguments), where keys are a series of numbers, from minimum to maximum keys (or `max` argument if it specified) taken from the map with a step size of one, and corresponding values. If the value is not specified for the key, then it uses the default value in the resulting map. For repeated keys, only the first value (in order of appearing) gets associated with the key. **Arguments** +Mapped arrays: + - `keys` — Array of keys. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#uint-ranges)). - `values` — Array of values. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#uint-ranges)). +or + +- `map` — Map with integer keys. [Map](../../sql-reference/data-types/map.md). + **Returned value** -- Returns a [tuple](../../sql-reference/data-types/tuple.md#tuplet1-t2) of two [arrays](../../sql-reference/data-types/array.md#data-type-array): keys in sorted order, and values the corresponding keys. +- Depending on the arguments returns a [map](../../sql-reference/data-types/map.md) or a [tuple](../../sql-reference/data-types/tuple.md#tuplet1-t2) of two [arrays](../../sql-reference/data-types/array.md#data-type-array): keys in sorted order, and values the corresponding keys. **Example** -Query: +Query with mapped arrays: -```sql +``` sql select mapPopulateSeries([1,2,4], [11,22,44], 5) as res, toTypeName(res) as type; ``` Result: -```text +``` text ┌─res──────────────────────────┬─type──────────────────────────────┐ │ ([1,2,3,4,5],[11,22,0,44,0]) │ Tuple(Array(UInt8), Array(UInt8)) │ └──────────────────────────────┴───────────────────────────────────┘ ``` +Query with `Map` type: + +``` sql +SELECT mapPopulateSeries(map(1, 10, 5, 20), 6); +``` + +Result: + +``` text +┌─mapPopulateSeries(map(1, 10, 5, 20), 6)─┐ +│ {1:10,2:0,3:0,4:0,5:20,6:0} │ +└─────────────────────────────────────────┘ +``` + ## mapContains {#mapcontains} Determines whether the `map` contains the `key` parameter. @@ -188,7 +233,7 @@ Determines whether the `map` contains the `key` parameter. mapContains(map, key) ``` -**Parameters** +**Parameters** - `map` — Map. [Map](../../sql-reference/data-types/map.md). - `key` — Key. Type matches the type of keys of `map` parameter. From a4e9a56e8e2bc786edd78158650ceb36f63155bb Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Fri, 30 Jul 2021 20:57:32 +0200 Subject: [PATCH 3/7] Make changes by review on #26663 PR --- src/Functions/array/mapPopulateSeries.cpp | 80 +++++++++++-------- ...01925_map_populate_series_on_map.reference | 30 +++++++ .../01925_map_populate_series_on_map.sql | 1 + .../tests/array_tuple_map.py | 22 +++-- 4 files changed, 91 insertions(+), 42 deletions(-) diff --git a/src/Functions/array/mapPopulateSeries.cpp b/src/Functions/array/mapPopulateSeries.cpp index f8bcbf8d451..51e436e8022 100644 --- a/src/Functions/array/mapPopulateSeries.cpp +++ b/src/Functions/array/mapPopulateSeries.cpp @@ -40,7 +40,7 @@ private: if (!(which_key.isInt() || which_key.isUInt())) { throw Exception( - "Keys for " + getName() + " should be of integer type (signed or unsigned)", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Keys for {} function should be of integer type (signed or unsigned)", getName()); } if (max_key_type) @@ -49,27 +49,28 @@ private: if (which_max_key.isNullable()) throw Exception( - "Max key argument in arguments of function " + getName() + " can not be Nullable", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Max key argument in arguments of function " + getName() + " can not be Nullable"); if (key_type->getTypeId() != max_key_type->getTypeId()) - throw Exception("Max key type in " + getName() + " should be same as keys type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Max key type in {} should be same as keys type", getName()); } } DataTypePtr getReturnTypeForTuple(const DataTypes & arguments) const { if (arguments.size() < 2) - throw Exception(getName() + " accepts at least two arrays for key and value", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} accepts at least two arrays for key and value", getName()); if (arguments.size() > 3) - throw Exception("too many arguments in " + getName() + " call", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Too many arguments in {} call", getName()); const DataTypeArray * key_array_type = checkAndGetDataType(arguments[0].get()); const DataTypeArray * val_array_type = checkAndGetDataType(arguments[1].get()); if (!key_array_type || !val_array_type) - throw Exception(getName() + " accepts two arrays for key and value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {} accepts two arrays for key and value", getName()); const auto & key_type = key_array_type->getNestedType(); @@ -89,7 +90,7 @@ private: else if (arguments.size() == 2) this->checkTypes(map->getKeyType(), arguments[1]); else - throw Exception("too many arguments in " + getName() + " call", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Too many arguments in {} call", getName()); return std::make_shared(map->getKeyType(), map->getValueType()); } @@ -97,14 +98,18 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.empty()) - throw Exception(getName() + " accepts at least one map", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, getName() + " accepts at least one map or two arrays"); if (arguments[0]->getTypeId() == TypeIndex::Array) return getReturnTypeForTuple(arguments); else if (arguments[0]->getTypeId() == TypeIndex::Map) return getReturnTypeForMap(arguments); else - throw Exception(getName() + " only accepts maps", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Function {} only accepts one map or arrays, but got {}", + getName(), + arguments[0]->getName()); } // Struct holds input and output columns references, @@ -146,7 +151,8 @@ private: { const ColumnConst * const_array = checkAndGetColumnConst(key_column); if (!const_array) - throw Exception("Expected array column, found " + key_column->getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Expected array column in function {}, found {}", getName(), key_column->getName()); in_keys_array = checkAndGetColumn(const_array->getDataColumnPtr().get()); key_is_const = true; @@ -158,7 +164,8 @@ private: { const ColumnConst * const_array = checkAndGetColumnConst(val_column); if (!const_array) - throw Exception("Expected array column, found " + val_column->getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Expected array column in function {}, found {}", getName(), val_column->getName()); in_values_array = checkAndGetColumn(const_array->getDataColumnPtr().get()); val_is_const = true; @@ -166,7 +173,7 @@ private: if (!in_keys_array || !in_values_array) /* something went wrong */ - throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns in arguments of function " + getName()); const auto & in_keys_data = assert_cast &>(in_keys_array->getData()).getData(); const auto & in_values_data = assert_cast &>(in_values_array->getData()).getData(); @@ -230,8 +237,8 @@ private: { MutableColumnPtr res_column = res_type->createColumn(); bool max_key_is_const = false; - auto inout = res_column->getDataType() == TypeIndex::Tuple ? getInOutDataFromArrays(res_column, arg_columns) - : getInOutDataFromMap(res_column, arg_columns); + auto columns = res_column->getDataType() == TypeIndex::Tuple ? getInOutDataFromArrays(res_column, arg_columns) + : getInOutDataFromMap(res_column, arg_columns); KeyType max_key_const{0}; @@ -246,36 +253,39 @@ private: std::map res_map; //Iterate through two arrays and fill result values. - for (size_t row = 0; row < inout.row_count; ++row) + for (size_t row = 0; row < columns.row_count; ++row) { - size_t key_offset = 0, val_offset = 0, items_count = inout.in_key_offsets[0], val_array_size = inout.in_val_offsets[0]; + size_t key_offset = 0, val_offset = 0, items_count = columns.in_key_offsets[0], val_array_size = columns.in_val_offsets[0]; res_map.clear(); - if (!inout.key_is_const) + if (!columns.key_is_const) { - key_offset = row > 0 ? inout.in_key_offsets[row - 1] : 0; - items_count = inout.in_key_offsets[row] - key_offset; + key_offset = row > 0 ? columns.in_key_offsets[row - 1] : 0; + items_count = columns.in_key_offsets[row] - key_offset; } - if (!inout.val_is_const) + if (!columns.val_is_const) { - val_offset = row > 0 ? inout.in_val_offsets[row - 1] : 0; - val_array_size = inout.in_val_offsets[row] - val_offset; + val_offset = row > 0 ? columns.in_val_offsets[row - 1] : 0; + val_array_size = columns.in_val_offsets[row] - val_offset; } if (items_count != val_array_size) - throw Exception("Key and value array should have same amount of elements", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Key and value array should have same amount of elements in function {}", + getName()); if (items_count == 0) { - inout.out_keys_offsets.push_back(offset); + columns.out_keys_offsets.push_back(offset); continue; } for (size_t i = 0; i < items_count; ++i) { - res_map.insert({inout.in_keys_data[key_offset + i], inout.in_vals_data[val_offset + i]}); + res_map.insert({columns.in_keys_data[key_offset + i], columns.in_vals_data[val_offset + i]}); } auto min_key = res_map.begin()->first; @@ -296,7 +306,7 @@ private: /* no need to add anything, max key is less that first key */ if (max_key < min_key) { - inout.out_keys_offsets.push_back(offset); + columns.out_keys_offsets.push_back(offset); continue; } } @@ -309,16 +319,16 @@ private: KeyType key; for (key = min_key;; ++key) { - inout.out_keys_data.push_back(key); + columns.out_keys_data.push_back(key); auto it = res_map.find(key); if (it != res_map.end()) { - inout.out_vals_data.push_back(it->second); + columns.out_vals_data.push_back(it->second); } else { - inout.out_vals_data.push_back(0); + columns.out_vals_data.push_back(0); } ++offset; @@ -326,11 +336,11 @@ private: break; } - inout.out_keys_offsets.push_back(offset); + columns.out_keys_offsets.push_back(offset); } - if (inout.out_vals_offsets) - inout.out_vals_offsets->insert(inout.out_keys_offsets.begin(), inout.out_keys_offsets.end()); + if (columns.out_vals_offsets) + columns.out_vals_offsets->insert(columns.out_keys_offsets.begin(), columns.out_keys_offsets.end()); return res_column; } @@ -365,7 +375,7 @@ private: case TypeIndex::UInt256: return execute2(arg_columns, max_key_column, res_type); default: - throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns in arguments of function " + getName()); } } @@ -431,7 +441,7 @@ private: case TypeIndex::UInt256: return execute1(arg_columns, max_key_column, res_type, val_type); default: - throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns in arguments of function " + getName()); } } }; diff --git a/tests/queries/0_stateless/01925_map_populate_series_on_map.reference b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference index 74ed35f0638..235a227f548 100644 --- a/tests/queries/0_stateless/01925_map_populate_series_on_map.reference +++ b/tests/queries/0_stateless/01925_map_populate_series_on_map.reference @@ -1,37 +1,67 @@ +-- { echo } +drop table if exists map_test; +set allow_experimental_map_type = 1; +create table map_test engine=TinyLog() as (select (number + 1) as n, map(1, 1, number,2) as m from numbers(1, 5)); +select mapPopulateSeries(m) from map_test; {1:1} {1:1,2:2} {1:1,2:0,3:2} {1:1,2:0,3:0,4:2} {1:1,2:0,3:0,4:0,5:2} +select mapPopulateSeries(m, toUInt64(3)) from map_test; {1:1,2:0,3:0} {1:1,2:2,3:0} {1:1,2:0,3:2} {1:1,2:0,3:0} {1:1,2:0,3:0} +select mapPopulateSeries(m, toUInt64(10)) from map_test; {1:1,2:0,3:0,4:0,5:0,6:0,7:0,8:0,9:0,10:0} {1:1,2:2,3:0,4:0,5:0,6:0,7:0,8:0,9:0,10:0} {1:1,2:0,3:2,4:0,5:0,6:0,7:0,8:0,9:0,10:0} {1:1,2:0,3:0,4:2,5:0,6:0,7:0,8:0,9:0,10:0} {1:1,2:0,3:0,4:0,5:2,6:0,7:0,8:0,9:0,10:0} +select mapPopulateSeries(m, 1000) from map_test; -- { serverError 43 } +select mapPopulateSeries(m, n) from map_test; {1:1,2:0} {1:1,2:2,3:0} {1:1,2:0,3:2,4:0} {1:1,2:0,3:0,4:2,5:0} {1:1,2:0,3:0,4:0,5:2,6:0} +drop table map_test; +select mapPopulateSeries(map(toUInt8(1), toUInt8(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(UInt8,UInt8) +select mapPopulateSeries(map(toUInt16(1), toUInt16(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(UInt16,UInt16) +select mapPopulateSeries(map(toUInt32(1), toUInt32(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(UInt32,UInt32) +select mapPopulateSeries(map(toUInt64(1), toUInt64(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(UInt64,UInt64) +select mapPopulateSeries(map(toUInt128(1), toUInt128(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(UInt128,UInt128) +select mapPopulateSeries(map(toUInt256(1), toUInt256(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(UInt256,UInt256) +select mapPopulateSeries(map(toInt8(1), toInt8(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(Int16,Int16) +select mapPopulateSeries(map(toInt16(1), toInt16(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(Int16,Int16) +select mapPopulateSeries(map(toInt32(1), toInt32(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(Int32,Int32) +select mapPopulateSeries(map(toInt64(1), toInt64(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(Int64,Int64) +select mapPopulateSeries(map(toInt128(1), toInt128(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(Int128,Int128) +select mapPopulateSeries(map(toInt256(1), toInt256(1), 2, 1)) as res, toTypeName(res); {1:1,2:1} Map(Int256,Int256) +select mapPopulateSeries(map(toInt8(-10), toInt8(1), 2, 1)) as res, toTypeName(res); {-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16) +select mapPopulateSeries(map(toInt16(-10), toInt16(1), 2, 1)) as res, toTypeName(res); {-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16) +select mapPopulateSeries(map(toInt32(-10), toInt32(1), 2, 1)) as res, toTypeName(res); {-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int32,Int32) +select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1)) as res, toTypeName(res); {-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int64,Int64) +select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1), toInt64(-5)) as res, toTypeName(res); {-10:1,-9:0,-8:0,-7:0,-6:0,-5:0} Map(Int64,Int64) +select mapPopulateSeries(); -- { serverError 42 } +select mapPopulateSeries('asdf'); -- { serverError 43 } +select mapPopulateSeries(map('1', 1, '2', 1)) as res, toTypeName(res); -- { serverError 43 } diff --git a/tests/queries/0_stateless/01925_map_populate_series_on_map.sql b/tests/queries/0_stateless/01925_map_populate_series_on_map.sql index f45543a8dba..ac78280ec1d 100644 --- a/tests/queries/0_stateless/01925_map_populate_series_on_map.sql +++ b/tests/queries/0_stateless/01925_map_populate_series_on_map.sql @@ -1,3 +1,4 @@ +-- { echo } drop table if exists map_test; set allow_experimental_map_type = 1; create table map_test engine=TinyLog() as (select (number + 1) as n, map(1, 1, number,2) as m from numbers(1, 5)); diff --git a/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py b/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py index 938beabfff4..04150d945f6 100644 --- a/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py +++ b/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py @@ -357,7 +357,7 @@ def map_func(self, data_type, node=None): exitcode, message = 0, None if data_type.startswith("Decimal"): - exitcode, message = 43, "Exception:" + exitcode, message = 43, "Exception:" node.query(sql, exitcode=exitcode, message=message) execute_query(f"""SELECT * FROM {table_name} ORDER BY a ASC""") @@ -393,9 +393,13 @@ def map_func(self, data_type, node=None): execute_query(f"SELECT * FROM {table_name} ORDER BY a ASC") with Scenario(f"mapPopulateSeries with {data_type}"): - node.query(f"SELECT mapPopulateSeries([1,2,3], [{to_data_type(data_type,1)}," - f"{to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)", - exitcode = 44, message='Exception:') + sql = (f"SELECT mapPopulateSeries([1,2,3], [{to_data_type(data_type,1)}," + f"{to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)") + + exitcode, message = 0, None + if data_type.startswith("Decimal"): + exitcode, message = 44, "Exception:" + node.query(sql, exitcode=exitcode, message=message) with Scenario(f"mapPopulateSeries with {data_type} on a table"): table_name = get_table_name() @@ -403,9 +407,13 @@ def map_func(self, data_type, node=None): table(name = table_name, data_type = f'Tuple(Array({data_type}), Array({data_type}))') with When("I insert the output into a table"): - node.query(f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3]," - f"[{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)", - exitcode = 44, message='Exception:') + sql = (f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3],", + f"[{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)") + + exitcode, message = 0, None + if data_type.startswith("Decimal"): + exitcode, message = 44, "Exception:" + node.query(sql, exitcode=exitcode, message=message) execute_query(f"SELECT * FROM {table_name} ORDER BY a ASC") From df0d9be04b4c32e4a21744c8f667e53f44b7e6fa Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Mon, 2 Aug 2021 12:59:13 +0300 Subject: [PATCH 4/7] Update docs/en/sql-reference/functions/tuple-map-functions.md --- docs/en/sql-reference/functions/tuple-map-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 908d3e1bbd3..34935a10d8c 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -197,7 +197,7 @@ or Query with mapped arrays: -``` sql +```sql select mapPopulateSeries([1,2,4], [11,22,44], 5) as res, toTypeName(res) as type; ``` From d8ce46ea747a62c65c3c589d88c1d2f576fb3be5 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 4 Aug 2021 17:20:35 +0500 Subject: [PATCH 5/7] Show more details in testflows checks --- .../tests/array_tuple_map.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py b/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py index 04150d945f6..698e81ca1ac 100644 --- a/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py +++ b/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py @@ -1,5 +1,6 @@ import uuid +from testflows.asserts import error from extended_precision_data_types.requirements import * from extended_precision_data_types.common import * @@ -410,10 +411,11 @@ def map_func(self, data_type, node=None): sql = (f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3],", f"[{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)") - exitcode, message = 0, None + r = node.query(sql, exitcode=None, message=None) if data_type.startswith("Decimal"): - exitcode, message = 44, "Exception:" - node.query(sql, exitcode=exitcode, message=message) + assert r.exitcode == 44, error(r.output) + else: + assert r.exitcode == 0, error(f"sql: {sql}\noutput: {r.output}") execute_query(f"SELECT * FROM {table_name} ORDER BY a ASC") From 8e347bab99d0bf1d46d4a03dfab8250379ec4c0b Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 4 Aug 2021 20:21:37 +0500 Subject: [PATCH 6/7] Fix error in testflows related with mapPopulateSeries --- .../tests/array_tuple_map.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py b/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py index 698e81ca1ac..c39574ba75e 100644 --- a/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py +++ b/tests/testflows/extended_precision_data_types/tests/array_tuple_map.py @@ -1,6 +1,5 @@ import uuid -from testflows.asserts import error from extended_precision_data_types.requirements import * from extended_precision_data_types.common import * @@ -408,14 +407,13 @@ def map_func(self, data_type, node=None): table(name = table_name, data_type = f'Tuple(Array({data_type}), Array({data_type}))') with When("I insert the output into a table"): - sql = (f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3],", + sql = (f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3]," f"[{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)") - r = node.query(sql, exitcode=None, message=None) + exitcode, message = 0, None if data_type.startswith("Decimal"): - assert r.exitcode == 44, error(r.output) - else: - assert r.exitcode == 0, error(f"sql: {sql}\noutput: {r.output}") + exitcode, message = 44, "Exception:" + node.query(sql, exitcode=exitcode, message=message) execute_query(f"SELECT * FROM {table_name} ORDER BY a ASC") From 2ed7e02079cbd2aa939e83a313e8302540baab70 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Thu, 5 Aug 2021 11:11:41 +0500 Subject: [PATCH 7/7] Fix snapshots for mapPopulateSeries function --- .../snapshots/common.py.tests.snapshot | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot b/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot index 18b58b0cfdc..e0414393111 100644 --- a/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot +++ b/tests/testflows/extended_precision_data_types/snapshots/common.py.tests.snapshot @@ -1045,6 +1045,7 @@ a mapPopulateSeries_with_Int128_on_a_table = r""" a +([1,2,3,4,5],[1,2,3,0,0]) """ mapContains_with_Int128_on_a_table = r""" @@ -1575,6 +1576,7 @@ a mapPopulateSeries_with_Int256_on_a_table = r""" a +([1,2,3,4,5],[1,2,3,0,0]) """ mapContains_with_Int256_on_a_table = r""" @@ -2105,6 +2107,7 @@ a mapPopulateSeries_with_UInt128_on_a_table = r""" a +([1,2,3,4,5],[1,2,3,0,0]) """ mapContains_with_UInt128_on_a_table = r""" @@ -2635,6 +2638,7 @@ a mapPopulateSeries_with_UInt256_on_a_table = r""" a +([1,2,3,4,5],[1,2,3,0,0]) """ mapContains_with_UInt256_on_a_table = r"""