From e953e482ec643830c982e30d0081c62fd9c91834 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 17 Jan 2022 16:34:44 +0800 Subject: [PATCH 01/14] Implement more functions for Map data type --- src/Columns/ColumnMap.h | 4 +- src/Functions/FunctionMapMapped.h | 198 ++++++++++++++++++ src/Functions/map.cpp | 110 ++++++++++ src/Functions/mapFilter.cpp | 124 +++++++++++ .../registerFunctionsHigherOrder.cpp | 4 + .../0_stateless/02169_map_functions.reference | 21 ++ .../0_stateless/02169_map_functions.sql | 8 + 7 files changed, 467 insertions(+), 2 deletions(-) create mode 100644 src/Functions/FunctionMapMapped.h create mode 100644 src/Functions/mapFilter.cpp create mode 100644 tests/queries/0_stateless/02169_map_functions.reference create mode 100644 tests/queries/0_stateless/02169_map_functions.sql diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index fb69541c363..b33630b993c 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -36,8 +36,8 @@ public: static Ptr create(const ColumnPtr & column) { return ColumnMap::create(column->assumeMutable()); } static Ptr create(ColumnPtr && arg) { return create(arg); } - template ::value>::type> - static MutablePtr create(Arg && arg) { return Base::create(std::forward(arg)); } + template ::value>::type> + static MutablePtr create(Args &&... args) { return Base::create(std::forward(args)...); } std::string getName() const override; const char * getFamilyName() const override { return "Map"; } diff --git a/src/Functions/FunctionMapMapped.h b/src/Functions/FunctionMapMapped.h new file mode 100644 index 00000000000..fb92db22325 --- /dev/null +++ b/src/Functions/FunctionMapMapped.h @@ -0,0 +1,198 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +/** Higher-order functions for map. + * These functions optionally apply a map by lambda function, + * and return some result based on that transformation. + * + * Examples: + * mapMap(x1,...,xn -> expression, map) - apply the expression to the map. + * mapFilter((k, v) -> predicate, map) - leave in the map only the kv elements for which the expression is true. + */ +template +class FunctionMapMapped : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + /// Called if at least one function argument is a lambda expression. + /// For argument-lambda expressions, it defines the types of arguments of these expressions. + void getLambdaArgumentTypes(DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception("Function " + getName() + " needs at least one argument; passed " + + toString(arguments.size()) + ".", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (arguments.size() == 1) + throw Exception("Function " + getName() + " needs at least one map argument.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + DataTypes nested_types((arguments.size() - 1) * 2); + for (size_t i = 0; i < arguments.size() - 1; ++i) + { + const DataTypeMap * map_type = checkAndGetDataType(&*arguments[i + 1]); + if (!map_type) + throw Exception("Argument " + toString(i + 2) + " of function " + getName() + " must be map. Found " + + arguments[i + 1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + nested_types[i * 2] = recursiveRemoveLowCardinality(map_type->getKeyType()); + nested_types[i * 2 + 1] = recursiveRemoveLowCardinality(map_type->getValueType()); + } + + const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); + if (!function_type || function_type->getArgumentTypes().size() != nested_types.size()) + throw Exception("First argument for this overload of " + getName() + " must be a function with " + + toString(nested_types.size()) + " arguments. Found " + + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + arguments[0] = std::make_shared(nested_types); + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + size_t min_args = Impl::needExpression() ? 2 : 1; + if (arguments.size() < min_args) + throw Exception("Function " + getName() + " needs at least " + + toString(min_args) + " argument; passed " + + toString(arguments.size()) + ".", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if (arguments.size() > 2 && Impl::needOneMap()) + throw Exception("Function " + getName() + " needs one map argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); + + if (!data_type_function) + throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. + + DataTypePtr return_type = removeLowCardinality(data_type_function->getReturnType()); + if (Impl::needBoolean() && !WhichDataType(return_type).isUInt8()) + throw Exception( + "Expression for function " + getName() + " must return UInt8, found " + return_type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto * map_type = checkAndGetDataType(arguments[1].type.get()); + if (!map_type) + throw Exception("Second argument for function " + getName() + " must be a map.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return Impl::getReturnType(return_type, map_type->getKeyValueTypes()); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const auto & column_with_type_and_name = arguments[0]; + + if (!column_with_type_and_name.column) + throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto * column_function = typeid_cast(column_with_type_and_name.column.get()); + + if (!column_function) + throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + ColumnPtr offsets_column; + + ColumnPtr column_first_map_ptr; + const ColumnMap * column_first_map = nullptr; + + ColumnsWithTypeAndName maps; + maps.reserve(arguments.size() - 1); + + for (size_t i = 1; i < arguments.size(); ++i) + { + const auto & map_with_type_and_name = arguments[i]; + + ColumnPtr column_map_ptr = map_with_type_and_name.column; + const auto * column_map = checkAndGetColumn(column_map_ptr.get()); + + const DataTypePtr & map_type_ptr = map_with_type_and_name.type; + const auto * map_type = checkAndGetDataType(map_type_ptr.get()); + + if (!column_map) + { + const ColumnConst * column_const_map = checkAndGetColumnConst(column_map_ptr.get()); + if (!column_const_map) + throw Exception("Expected map column, found " + column_map_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); + column_map_ptr = recursiveRemoveLowCardinality(column_const_map->convertToFullColumn()); + column_map = checkAndGetColumn(column_map_ptr.get()); + } + + if (!map_type) + throw Exception("Expected map type, found " + map_type_ptr->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!offsets_column) + { + offsets_column = column_map->getNestedColumn().getOffsetsPtr(); + } + else + { + /// The first condition is optimization: do not compare data if the pointers are equal. + if (column_map->getNestedColumn().getOffsetsPtr() != offsets_column + && column_map->getNestedColumn().getOffsets() != typeid_cast(*offsets_column).getData()) + throw Exception("maps passed to " + getName() + " must have equal size", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + } + + if (i == 1) + { + column_first_map_ptr = column_map_ptr; + column_first_map = column_map; + } + + maps.emplace_back(ColumnWithTypeAndName( + column_map->getNestedData().getColumnPtr(0), recursiveRemoveLowCardinality(map_type->getKeyType()), map_with_type_and_name.name+".key")); + maps.emplace_back(ColumnWithTypeAndName( + column_map->getNestedData().getColumnPtr(1), recursiveRemoveLowCardinality(map_type->getValueType()), map_with_type_and_name.name+".value")); + } + + /// Put all the necessary columns multiplied by the sizes of maps into the columns. + auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(column_first_map->getNestedColumn().getOffsets())); + auto * replicated_column_function = typeid_cast(replicated_column_function_ptr.get()); + replicated_column_function->appendArguments(maps); + + auto lambda_result = replicated_column_function->reduce().column; + if (lambda_result->lowCardinality()) + lambda_result = lambda_result->convertToFullColumnIfLowCardinality(); + + return Impl::execute(*column_first_map, lambda_result); + } +}; + +} diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 4e242c4348b..95f464c951f 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -518,6 +518,115 @@ public: } }; +class FunctionMapReplace : public IFunction +{ +public: + static constexpr auto name = "mapReplace"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 2; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const DataTypeMap * left = checkAndGetDataType(arguments[0].type.get()); + const DataTypeMap * right = checkAndGetDataType(arguments[1].type.get()); + + if (!left || !right) + throw Exception{"The two arguments for function " + getName() + " must be both Map type", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + if (!left->getKeyType()->equals(*right->getKeyType()) || !left->getValueType()->equals(*right->getValueType())) + throw Exception{"The Key And Value type of Map for function " + getName() + " must be the same", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(left->getKeyType(), left->getValueType()); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + const ColumnMap * col_map_left = typeid_cast(arguments[0].column.get()); + const auto * col_const_map_left = checkAndGetColumnConst(arguments[0].column.get()); + if (col_const_map_left) + col_map_left = typeid_cast(&col_const_map_left->getDataColumn()); + if (!col_map_left) + return nullptr; + + const ColumnMap * col_map_right = typeid_cast(arguments[1].column.get()); + const auto * col_const_map_right = checkAndGetColumnConst(arguments[1].column.get()); + if (col_const_map_right) + col_map_right = typeid_cast(&col_const_map_right->getDataColumn()); + if (!col_map_right) + return nullptr; + + auto & nested_column_left = col_map_left->getNestedColumn(); + auto & keys_data_left = col_map_left->getNestedData().getColumn(0); + auto & values_data_left = col_map_left->getNestedData().getColumn(1); + auto & offsets_left = nested_column_left.getOffsets(); + + const auto & nested_column_right = col_map_right->getNestedColumn(); + const auto & keys_data_right = col_map_right->getNestedData().getColumn(0); + const auto & values_data_right = col_map_right->getNestedData().getColumn(1); + const auto & offsets_right = nested_column_right.getOffsets(); + + const auto & result_type_map = static_cast(*result_type); + const DataTypePtr & key_type = result_type_map.getKeyType(); + const DataTypePtr & value_type = result_type_map.getValueType(); + MutableColumnPtr keys_data = key_type->createColumn(); + MutableColumnPtr values_data = value_type->createColumn(); + MutableColumnPtr offsets = DataTypeNumber().createColumn(); + + IColumn::Offset current_offset = 0; + for (size_t idx = 0; idx < input_rows_count; ++idx) + { + for (size_t i = offsets_left[idx - 1]; i < offsets_left[idx]; ++i) + { + bool matched = false; + auto key = keys_data_left.getDataAt(i); + for (size_t j = offsets_right[idx - 1]; j < offsets_right[idx]; ++j) + { + if (keys_data_right.getDataAt(j).toString() == key.toString()) + { + matched = true; + break; + } + } + if (!matched) + { + keys_data->insertFrom(keys_data_left, i); + values_data->insertFrom(values_data_left, i); + ++current_offset; + } + } + for (size_t j = offsets_right[idx - 1]; j < offsets_right[idx]; ++j) + { + keys_data->insertFrom(keys_data_right, j); + values_data->insertFrom(values_data_right, j); + ++current_offset; + } + offsets->insert(current_offset); + } + + auto nested_column = ColumnArray::create( + ColumnTuple::create(Columns{std::move(keys_data), std::move(values_data)}), + std::move(offsets)); + + return ColumnMap::create(nested_column); + } +}; + } void registerFunctionsMap(FunctionFactory & factory) @@ -528,6 +637,7 @@ void registerFunctionsMap(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp new file mode 100644 index 00000000000..6b90d2552cc --- /dev/null +++ b/src/Functions/mapFilter.cpp @@ -0,0 +1,124 @@ +#include +#include +#include +#include +#include "FunctionMapMapped.h" +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +/** MapFilter(x -> predicate, array) - leave in the array only the elements for which the expression is true. + */ +struct MapFilterImpl +{ + static bool needBoolean() { return true; } + static bool needExpression() { return true; } + static bool needOneMap() { return true; } + + static DataTypePtr getReturnType(const DataTypePtr & /*expression_return*/, const DataTypes & elems) + { + return std::make_shared(elems); + } + + /// If there are several arrays, the first one is passed here. + static ColumnPtr execute(const ColumnMap & map_column, ColumnPtr mapped) + { + const ColumnUInt8 * column_filter = typeid_cast(&*mapped); + + if (!column_filter) + { + const auto * column_filter_const = checkAndGetColumnConst(&*mapped); + + if (!column_filter_const) + throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + + if (column_filter_const->getValue()) + return map_column.clone(); + else + return ColumnMap::create( + map_column.getNestedColumnPtr()->cloneEmpty()); + } + + const IColumn::Filter & filter = column_filter->getData(); + ColumnPtr filtered = map_column.getNestedColumn().getData().filter(filter, -1); + + const IColumn::Offsets & in_offsets = map_column.getNestedColumn().getOffsets(); + auto column_offsets = ColumnArray::ColumnOffsets::create(in_offsets.size()); + IColumn::Offsets & out_offsets = column_offsets->getData(); + + size_t in_pos = 0; + size_t out_pos = 0; + for (size_t i = 0; i < in_offsets.size(); ++i) + { + for (; in_pos < in_offsets[i]; ++in_pos) + { + if (filter[in_pos]) + ++out_pos; + } + out_offsets[i] = out_pos; + } + + return ColumnMap::create(ColumnArray::create(filtered, std::move(column_offsets))); + } +}; + +struct NameMapFilter { static constexpr auto name = "mapFilter"; }; +using FunctionMapFilter = FunctionMapMapped; + +void registerFunctionMapFilter(FunctionFactory & factory) +{ + factory.registerFunction(); +} + + +/** mapMap((k,v) -> expression, map) - apply the expression to the map. + */ +struct MapMapImpl +{ + /// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean. + static bool needBoolean() { return false; } + /// true if the f(map) overload is unavailable. + static bool needExpression() { return true; } + /// true if the map must be exactly one. + static bool needOneMap() { return true; } + + static DataTypePtr getReturnType(const DataTypePtr & , const DataTypes & elems) + { + return std::make_shared(elems); + } + + static ColumnPtr execute(const ColumnMap & map, ColumnPtr mapped) + { + const auto * column_tuple = checkAndGetColumn(mapped.get()); + if (!column_tuple) + { + const ColumnConst * column_const_tuple = checkAndGetColumnConst(mapped.get()); + if (!column_const_tuple) + throw Exception("Expected tuple column, found " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN); + ColumnPtr column_tuple_ptr = recursiveRemoveLowCardinality(column_const_tuple->convertToFullColumn()); + column_tuple = checkAndGetColumn(column_tuple_ptr.get()); + } + + return ColumnMap::create(column_tuple->getColumnPtr(0), column_tuple->getColumnPtr(1), + map.getNestedColumn().getOffsetsPtr()); + } +}; + +struct NameMapMap { static constexpr auto name = "mapMap"; }; +using FunctionMapMap = FunctionMapMapped; + +void registerFunctionMapMap(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + + diff --git a/src/Functions/registerFunctionsHigherOrder.cpp b/src/Functions/registerFunctionsHigherOrder.cpp index d3621a03ecd..0a5d729facd 100644 --- a/src/Functions/registerFunctionsHigherOrder.cpp +++ b/src/Functions/registerFunctionsHigherOrder.cpp @@ -18,6 +18,8 @@ void registerFunctionsArraySort(FunctionFactory & factory); void registerFunctionArrayCumSum(FunctionFactory & factory); void registerFunctionArrayCumSumNonNegative(FunctionFactory & factory); void registerFunctionArrayDifference(FunctionFactory & factory); +void registerFunctionMapFilter(FunctionFactory & factory); +void registerFunctionMapMap(FunctionFactory & factory); void registerFunctionsHigherOrder(FunctionFactory & factory) { @@ -36,6 +38,8 @@ void registerFunctionsHigherOrder(FunctionFactory & factory) registerFunctionArrayCumSum(factory); registerFunctionArrayCumSumNonNegative(factory); registerFunctionArrayDifference(factory); + registerFunctionMapFilter(factory); + registerFunctionMapMap(factory); } } diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference new file mode 100644 index 00000000000..0f47bdce042 --- /dev/null +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -0,0 +1,21 @@ +{} +{} +{} +{'key3':103} +{} +{} +{} +{'key3':100,'key2':101,'key4':102} {'key4':102} +{'key3':101,'key2':102,'key4':103} {'key2':102,'key4':103} +{'key3':102,'key2':103,'key4':104} {'key3':102,'key2':103,'key4':104} +{'key3':103,'key2':104,'key4':105} {'key3':103,'key2':104,'key4':105} +{'key1':1111,'key2':2222} {'key2':2222} +{'key1':1112,'key2':2224} {'key1':1112,'key2':2224} +{'key1':1113,'key2':2226} {'key1':1113,'key2':2226} +{'key3':101,'key2':102,'key4':103} +{'key3':102,'key2':103,'key4':104} +{'key3':103,'key2':104,'key4':105} +{'key3':104,'key2':105,'key4':106} +{'key1':1112,'key2':2223} +{'key1':1113,'key2':2225} +{'key1':1114,'key2':2227} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql new file mode 100644 index 00000000000..6b4d796043f --- /dev/null +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS table_map; +create TABLE table_map (id UInt32, col Map(String, UInt64)) engine = MergeTree() ORDER BY tuple(); +INSERT INTO table_map SELECT number, map('key1', number, 'key2', number * 2) FROM numbers(1111, 3); +INSERT INTO table_map SELECT number, map('key3', number, 'key2', number + 1, 'key4', number + 2) FROM numbers(100, 4); + +SELECT mapFilter((k,v)->k like '%3' and v > 102, col) FROM table_map ORDER BY id; +SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC; +SELECT mapMap((k,v)->(k,v+1), col) FROM table_map ORDER BY id; From d6d943f4d41da97d4c80cc8e4f4ceedb346324ba Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 17 Jan 2022 16:49:10 +0800 Subject: [PATCH 02/14] Rectify test case --- tests/queries/0_stateless/02169_map_functions.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 6b4d796043f..420bbaaf3d5 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -6,3 +6,4 @@ INSERT INTO table_map SELECT number, map('key3', number, 'key2', number + 1, 'ke SELECT mapFilter((k,v)->k like '%3' and v > 102, col) FROM table_map ORDER BY id; SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC; SELECT mapMap((k,v)->(k,v+1), col) FROM table_map ORDER BY id; +DROP TABLE table_map; From ed39edda676dd6f74deb0a01ea379f2acdf3e807 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 24 Jan 2022 16:17:02 +0800 Subject: [PATCH 03/14] Update src/Functions/mapFilter.cpp Co-authored-by: Vladimir C --- src/Functions/mapFilter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 6b90d2552cc..2a98f0d3fca 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -2,7 +2,7 @@ #include #include #include -#include "FunctionMapMapped.h" +#include #include #include From 36939f1d5cf1d6f343530214db7e43720712dc96 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 24 Jan 2022 16:21:28 +0800 Subject: [PATCH 04/14] Update src/Functions/FunctionMapMapped.h Co-authored-by: Vladimir C --- src/Functions/FunctionMapMapped.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionMapMapped.h b/src/Functions/FunctionMapMapped.h index fb92db22325..3801432029f 100644 --- a/src/Functions/FunctionMapMapped.h +++ b/src/Functions/FunctionMapMapped.h @@ -94,7 +94,7 @@ public: + toString(arguments.size()) + ".", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (arguments.size() > 2 && Impl::needOneMap()) - throw Exception("Function " + getName() + " needs one map argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function '{}' needs one map argument", getName()); const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); From 582e6fa1c55c993c9155ddd5a35e045b6fa222ee Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 24 Jan 2022 16:18:34 +0800 Subject: [PATCH 05/14] Fix build error --- src/Functions/map.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 95f464c951f..4e660f2e0b1 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -571,10 +571,10 @@ public: if (!col_map_right) return nullptr; - auto & nested_column_left = col_map_left->getNestedColumn(); - auto & keys_data_left = col_map_left->getNestedData().getColumn(0); - auto & values_data_left = col_map_left->getNestedData().getColumn(1); - auto & offsets_left = nested_column_left.getOffsets(); + const auto & nested_column_left = col_map_left->getNestedColumn(); + const auto & keys_data_left = col_map_left->getNestedData().getColumn(0); + const auto & values_data_left = col_map_left->getNestedData().getColumn(1); + const auto & offsets_left = nested_column_left.getOffsets(); const auto & nested_column_right = col_map_right->getNestedColumn(); const auto & keys_data_right = col_map_right->getNestedData().getColumn(0); From a435c3a1e81b709a30d314d4dfdc0a3aaa5bfaa8 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 24 Jan 2022 18:33:13 +0800 Subject: [PATCH 06/14] Fix mapFilter bug --- src/Functions/mapFilter.cpp | 9 +++++++-- tests/queries/0_stateless/02169_map_functions.reference | 7 +++++++ tests/queries/0_stateless/02169_map_functions.sql | 1 + 3 files changed, 15 insertions(+), 2 deletions(-) diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 2a98f0d3fca..570c9fe8d36 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -42,8 +42,13 @@ struct MapFilterImpl if (column_filter_const->getValue()) return map_column.clone(); else - return ColumnMap::create( - map_column.getNestedColumnPtr()->cloneEmpty()); + { + const auto * column_array = typeid_cast(map_column.getNestedColumnPtr().get()); + const auto * column_tuple = typeid_cast(column_array->getDataPtr().get()); + ColumnPtr keys = column_tuple->getColumnPtr(0)->cloneEmpty(); + ColumnPtr values = column_tuple->getColumnPtr(1)->cloneEmpty(); + return ColumnMap::create(keys, values, ColumnArray::ColumnOffsets::create(map_column.size(), 0)); + } } const IColumn::Filter & filter = column_filter->getData(); diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index 0f47bdce042..50c35637b84 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -19,3 +19,10 @@ {'key1':1112,'key2':2223} {'key1':1113,'key2':2225} {'key1':1114,'key2':2227} +{} +{} +{} +{} +{} +{} +{} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 420bbaaf3d5..ade266324d7 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -6,4 +6,5 @@ INSERT INTO table_map SELECT number, map('key3', number, 'key2', number + 1, 'ke SELECT mapFilter((k,v)->k like '%3' and v > 102, col) FROM table_map ORDER BY id; SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC; SELECT mapMap((k,v)->(k,v+1), col) FROM table_map ORDER BY id; +SELECT mapFilter((k,v)->0, col) from table_map; DROP TABLE table_map; From 71aa4117460c2e502627abee075ca0a9654cb755 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 15 Feb 2022 17:59:44 +0800 Subject: [PATCH 07/14] Fix bug --- ...nctionMapMapped.h => FunctionMapAdvance.h} | 121 +++++------------- src/Functions/map.cpp | 8 +- src/Functions/mapFilter.cpp | 30 ++--- .../registerFunctionsHigherOrder.cpp | 4 +- .../0_stateless/02169_map_functions.reference | 2 + .../0_stateless/02169_map_functions.sql | 4 +- 6 files changed, 57 insertions(+), 112 deletions(-) rename src/Functions/{FunctionMapMapped.h => FunctionMapAdvance.h} (51%) diff --git a/src/Functions/FunctionMapMapped.h b/src/Functions/FunctionMapAdvance.h similarity index 51% rename from src/Functions/FunctionMapMapped.h rename to src/Functions/FunctionMapAdvance.h index 3801432029f..9f6660d030b 100644 --- a/src/Functions/FunctionMapMapped.h +++ b/src/Functions/FunctionMapAdvance.h @@ -23,7 +23,6 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -33,15 +32,15 @@ namespace ErrorCodes * and return some result based on that transformation. * * Examples: - * mapMap(x1,...,xn -> expression, map) - apply the expression to the map. + * mapApply(x1,...,xn -> expression, map) - apply the expression to the map. * mapFilter((k, v) -> predicate, map) - leave in the map only the kv elements for which the expression is true. */ template -class FunctionMapMapped : public IFunction +class FunctionMapAdvance : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -56,30 +55,21 @@ public: /// For argument-lambda expressions, it defines the types of arguments of these expressions. void getLambdaArgumentTypes(DataTypes & arguments) const override { - if (arguments.empty()) - throw Exception("Function " + getName() + " needs at least one argument; passed " - + toString(arguments.size()) + ".", + if (arguments.size() != 2) + throw Exception("Function " + getName() + " needs 2 arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (arguments.size() == 1) - throw Exception("Function " + getName() + " needs at least one map argument.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - DataTypes nested_types((arguments.size() - 1) * 2); - for (size_t i = 0; i < arguments.size() - 1; ++i) - { - const DataTypeMap * map_type = checkAndGetDataType(&*arguments[i + 1]); - if (!map_type) - throw Exception("Argument " + toString(i + 2) + " of function " + getName() + " must be map. Found " - + arguments[i + 1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - nested_types[i * 2] = recursiveRemoveLowCardinality(map_type->getKeyType()); - nested_types[i * 2 + 1] = recursiveRemoveLowCardinality(map_type->getValueType()); - } + DataTypes nested_types(2); + const DataTypeMap * map_type = checkAndGetDataType(&*arguments[1]); + if (!map_type) + throw Exception("The second argument of function " + getName() + " must be map. Found " + + arguments[1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + nested_types[0] = recursiveRemoveLowCardinality(map_type->getKeyType()); + nested_types[1] = recursiveRemoveLowCardinality(map_type->getValueType()); const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); - if (!function_type || function_type->getArgumentTypes().size() != nested_types.size()) - throw Exception("First argument for this overload of " + getName() + " must be a function with " - + toString(nested_types.size()) + " arguments. Found " + if (!function_type || function_type->getArgumentTypes().size() != 2) + throw Exception("First argument for this overload of " + getName() + " must be a function with 2 arguments. Found " + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); arguments[0] = std::make_shared(nested_types); @@ -87,14 +77,10 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - size_t min_args = Impl::needExpression() ? 2 : 1; - if (arguments.size() < min_args) - throw Exception("Function " + getName() + " needs at least " - + toString(min_args) + " argument; passed " + if (arguments.size() != 2) + throw Exception("Function " + getName() + " needs at least 2 argument; passed " + toString(arguments.size()) + ".", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (arguments.size() > 2 && Impl::needOneMap()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function '{}' needs one map argument", getName()); const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); @@ -102,7 +88,6 @@ public: throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. - DataTypePtr return_type = removeLowCardinality(data_type_function->getReturnType()); if (Impl::needBoolean() && !WhichDataType(return_type).isUInt8()) throw Exception( @@ -119,79 +104,41 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { const auto & column_with_type_and_name = arguments[0]; - if (!column_with_type_and_name.column) throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); const auto * column_function = typeid_cast(column_with_type_and_name.column.get()); - if (!column_function) throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - ColumnPtr offsets_column; - - ColumnPtr column_first_map_ptr; - const ColumnMap * column_first_map = nullptr; - - ColumnsWithTypeAndName maps; - maps.reserve(arguments.size() - 1); - - for (size_t i = 1; i < arguments.size(); ++i) + ColumnPtr column_map_ptr = arguments[1].column; + const auto * column_map = checkAndGetColumn(column_map_ptr.get()); + if (!column_map) { - const auto & map_with_type_and_name = arguments[i]; - - ColumnPtr column_map_ptr = map_with_type_and_name.column; - const auto * column_map = checkAndGetColumn(column_map_ptr.get()); - - const DataTypePtr & map_type_ptr = map_with_type_and_name.type; - const auto * map_type = checkAndGetDataType(map_type_ptr.get()); - - if (!column_map) - { - const ColumnConst * column_const_map = checkAndGetColumnConst(column_map_ptr.get()); - if (!column_const_map) - throw Exception("Expected map column, found " + column_map_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); - column_map_ptr = recursiveRemoveLowCardinality(column_const_map->convertToFullColumn()); - column_map = checkAndGetColumn(column_map_ptr.get()); - } - - if (!map_type) - throw Exception("Expected map type, found " + map_type_ptr->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!offsets_column) - { - offsets_column = column_map->getNestedColumn().getOffsetsPtr(); - } - else - { - /// The first condition is optimization: do not compare data if the pointers are equal. - if (column_map->getNestedColumn().getOffsetsPtr() != offsets_column - && column_map->getNestedColumn().getOffsets() != typeid_cast(*offsets_column).getData()) - throw Exception("maps passed to " + getName() + " must have equal size", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); - } - - if (i == 1) - { - column_first_map_ptr = column_map_ptr; - column_first_map = column_map; - } - - maps.emplace_back(ColumnWithTypeAndName( - column_map->getNestedData().getColumnPtr(0), recursiveRemoveLowCardinality(map_type->getKeyType()), map_with_type_and_name.name+".key")); - maps.emplace_back(ColumnWithTypeAndName( - column_map->getNestedData().getColumnPtr(1), recursiveRemoveLowCardinality(map_type->getValueType()), map_with_type_and_name.name+".value")); + const ColumnConst * column_const_map = checkAndGetColumnConst(column_map_ptr.get()); + if (!column_const_map) + throw Exception("Expected map column, found " + column_map_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); + column_map_ptr = recursiveRemoveLowCardinality(column_const_map->convertToFullColumn()); + column_map = checkAndGetColumn(column_map_ptr.get()); } + const DataTypePtr & map_type_ptr = arguments[1].type; + const auto * map_type = checkAndGetDataType(map_type_ptr.get()); + const auto & offsets_column = column_map->getNestedColumn().getOffsets(); + /// Put all the necessary columns multiplied by the sizes of maps into the columns. - auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(column_first_map->getNestedColumn().getOffsets())); + auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(offsets_column)); auto * replicated_column_function = typeid_cast(replicated_column_function_ptr.get()); - replicated_column_function->appendArguments(maps); + const ColumnsWithTypeAndName args { + {column_map->getNestedData().getColumnPtr(0), recursiveRemoveLowCardinality(map_type->getKeyType()), arguments[1].name + ".key"}, + {column_map->getNestedData().getColumnPtr(1), recursiveRemoveLowCardinality(map_type->getValueType()), arguments[1].name + ".value"}}; + replicated_column_function->appendArguments(args); auto lambda_result = replicated_column_function->reduce().column; if (lambda_result->lowCardinality()) lambda_result = lambda_result->convertToFullColumnIfLowCardinality(); - return Impl::execute(*column_first_map, lambda_result); + return Impl::execute(*column_map, lambda_result); } }; diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 4e660f2e0b1..471d6fc575c 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -518,11 +518,11 @@ public: } }; -class FunctionMapReplace : public IFunction +class FunctionMapUpdate : public IFunction { public: - static constexpr auto name = "mapReplace"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + static constexpr auto name = "mapUpdate"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -637,7 +637,7 @@ void registerFunctionsMap(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 570c9fe8d36..5373b7e3b38 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -1,8 +1,7 @@ -#include -#include +#include #include #include -#include +#include #include #include @@ -19,8 +18,6 @@ namespace ErrorCodes struct MapFilterImpl { static bool needBoolean() { return true; } - static bool needExpression() { return true; } - static bool needOneMap() { return true; } static DataTypePtr getReturnType(const DataTypePtr & /*expression_return*/, const DataTypes & elems) { @@ -75,7 +72,7 @@ struct MapFilterImpl }; struct NameMapFilter { static constexpr auto name = "mapFilter"; }; -using FunctionMapFilter = FunctionMapMapped; +using FunctionMapFilter = FunctionMapAdvance; void registerFunctionMapFilter(FunctionFactory & factory) { @@ -83,20 +80,17 @@ void registerFunctionMapFilter(FunctionFactory & factory) } -/** mapMap((k,v) -> expression, map) - apply the expression to the map. +/** mapApply((k,v) -> expression, map) - apply the expression to the map. */ -struct MapMapImpl +struct MapApplyImpl { /// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean. static bool needBoolean() { return false; } - /// true if the f(map) overload is unavailable. - static bool needExpression() { return true; } - /// true if the map must be exactly one. - static bool needOneMap() { return true; } - static DataTypePtr getReturnType(const DataTypePtr & , const DataTypes & elems) + static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & ) { - return std::make_shared(elems); + const auto * date_type_tuple = typeid_cast(&*expression_return); + return std::make_shared(date_type_tuple->getElements()); } static ColumnPtr execute(const ColumnMap & map, ColumnPtr mapped) @@ -116,12 +110,12 @@ struct MapMapImpl } }; -struct NameMapMap { static constexpr auto name = "mapMap"; }; -using FunctionMapMap = FunctionMapMapped; +struct NameMapApply { static constexpr auto name = "mapApply"; }; +using FunctionMapApply = FunctionMapAdvance; -void registerFunctionMapMap(FunctionFactory & factory) +void registerFunctionMapApply(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/registerFunctionsHigherOrder.cpp b/src/Functions/registerFunctionsHigherOrder.cpp index 0a5d729facd..00b300b18b9 100644 --- a/src/Functions/registerFunctionsHigherOrder.cpp +++ b/src/Functions/registerFunctionsHigherOrder.cpp @@ -19,7 +19,7 @@ void registerFunctionArrayCumSum(FunctionFactory & factory); void registerFunctionArrayCumSumNonNegative(FunctionFactory & factory); void registerFunctionArrayDifference(FunctionFactory & factory); void registerFunctionMapFilter(FunctionFactory & factory); -void registerFunctionMapMap(FunctionFactory & factory); +void registerFunctionMapApply(FunctionFactory & factory); void registerFunctionsHigherOrder(FunctionFactory & factory) { @@ -39,7 +39,7 @@ void registerFunctionsHigherOrder(FunctionFactory & factory) registerFunctionArrayCumSumNonNegative(factory); registerFunctionArrayDifference(factory); registerFunctionMapFilter(factory); - registerFunctionMapMap(factory); + registerFunctionMapApply(factory); } } diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index 50c35637b84..c570ba4e724 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -19,6 +19,7 @@ {'key1':1112,'key2':2223} {'key1':1113,'key2':2225} {'key1':1114,'key2':2227} +{1:2,2:3} {} {} {} @@ -26,3 +27,4 @@ {} {} {} +{3:2,1:0,2:0} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index ade266324d7..c7a11b46357 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -5,6 +5,8 @@ INSERT INTO table_map SELECT number, map('key3', number, 'key2', number + 1, 'ke SELECT mapFilter((k,v)->k like '%3' and v > 102, col) FROM table_map ORDER BY id; SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC; -SELECT mapMap((k,v)->(k,v+1), col) FROM table_map ORDER BY id; +SELECT mapApply((k,v)->(k,v+1), col) FROM table_map ORDER BY id; +SELECT mapApply((x, y) -> (x, x + 1), map(1, 0, 2, 0)); SELECT mapFilter((k,v)->0, col) from table_map; +SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0)); DROP TABLE table_map; From aeec4a62e760938e24245837ce6b77138319d10f Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Wed, 16 Feb 2022 11:28:21 +0800 Subject: [PATCH 08/14] fix bug --- src/Functions/FunctionMapAdvance.h | 6 +++--- src/Functions/mapFilter.cpp | 11 ++++++++--- tests/queries/0_stateless/02169_map_functions.sql | 1 + 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Functions/FunctionMapAdvance.h b/src/Functions/FunctionMapAdvance.h index 9f6660d030b..d72cd4d62e5 100644 --- a/src/Functions/FunctionMapAdvance.h +++ b/src/Functions/FunctionMapAdvance.h @@ -70,7 +70,7 @@ public: const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); if (!function_type || function_type->getArgumentTypes().size() != 2) throw Exception("First argument for this overload of " + getName() + " must be a function with 2 arguments. Found " - + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); arguments[0] = std::make_shared(nested_types); } @@ -79,8 +79,8 @@ public: { if (arguments.size() != 2) throw Exception("Function " + getName() + " needs at least 2 argument; passed " - + toString(arguments.size()) + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + toString(arguments.size()) + ".", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 5373b7e3b38..69bbf0c00f3 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -11,6 +11,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } /** MapFilter(x -> predicate, array) - leave in the array only the elements for which the expression is true. @@ -87,10 +88,14 @@ struct MapApplyImpl /// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean. static bool needBoolean() { return false; } - static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & ) + static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & /*elems*/) { - const auto * date_type_tuple = typeid_cast(&*expression_return); - return std::make_shared(date_type_tuple->getElements()); + const auto & tuple_types = typeid_cast(&*expression_return)->getElements(); + if (tuple_types.size() != 2) + throw Exception("Expected 2 columns as map's key and value, but found " + + toString(tuple_types.size()) + " columns", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return std::make_shared(tuple_types); } static ColumnPtr execute(const ColumnMap & map, ColumnPtr mapped) diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index c7a11b46357..1d8f90e8a90 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -9,4 +9,5 @@ SELECT mapApply((k,v)->(k,v+1), col) FROM table_map ORDER BY id; SELECT mapApply((x, y) -> (x, x + 1), map(1, 0, 2, 0)); SELECT mapFilter((k,v)->0, col) from table_map; SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0)); +SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError 42 } DROP TABLE table_map; From 82a76d47ffa2f47076c181b4beb254fbdc46ab53 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 17 Feb 2022 14:56:29 +0000 Subject: [PATCH 09/14] Use FunctionArrayMapped for Map --- src/Functions/FunctionMapAdvance.h | 145 ------------------ src/Functions/array/FunctionArrayMapped.h | 132 ++++++++++++---- src/Functions/array/arrayAggregation.cpp | 23 ++- src/Functions/array/arrayAll.cpp | 7 +- src/Functions/array/arrayCompact.cpp | 13 +- src/Functions/array/arrayCount.cpp | 8 +- src/Functions/array/arrayCumSum.cpp | 12 +- .../array/arrayCumSumNonNegative.cpp | 11 +- src/Functions/array/arrayDifference.cpp | 12 +- src/Functions/array/arrayExists.cpp | 8 +- src/Functions/array/arrayFill.cpp | 8 +- src/Functions/array/arrayFilter.cpp | 8 +- src/Functions/array/arrayFirst.cpp | 8 +- src/Functions/array/arrayFirstLastIndex.cpp | 3 + src/Functions/array/arrayMap.cpp | 8 +- src/Functions/array/arraySort.cpp | 6 + src/Functions/array/arraySplit.cpp | 8 +- src/Functions/mapFilter.cpp | 30 +++- 18 files changed, 227 insertions(+), 223 deletions(-) delete mode 100644 src/Functions/FunctionMapAdvance.h diff --git a/src/Functions/FunctionMapAdvance.h b/src/Functions/FunctionMapAdvance.h deleted file mode 100644 index d72cd4d62e5..00000000000 --- a/src/Functions/FunctionMapAdvance.h +++ /dev/null @@ -1,145 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - - -/** Higher-order functions for map. - * These functions optionally apply a map by lambda function, - * and return some result based on that transformation. - * - * Examples: - * mapApply(x1,...,xn -> expression, map) - apply the expression to the map. - * mapFilter((k, v) -> predicate, map) - leave in the map only the kv elements for which the expression is true. - */ -template -class FunctionMapAdvance : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - String getName() const override - { - return name; - } - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - /// Called if at least one function argument is a lambda expression. - /// For argument-lambda expressions, it defines the types of arguments of these expressions. - void getLambdaArgumentTypes(DataTypes & arguments) const override - { - if (arguments.size() != 2) - throw Exception("Function " + getName() + " needs 2 arguments.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - DataTypes nested_types(2); - const DataTypeMap * map_type = checkAndGetDataType(&*arguments[1]); - if (!map_type) - throw Exception("The second argument of function " + getName() + " must be map. Found " - + arguments[1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - nested_types[0] = recursiveRemoveLowCardinality(map_type->getKeyType()); - nested_types[1] = recursiveRemoveLowCardinality(map_type->getValueType()); - - const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); - if (!function_type || function_type->getArgumentTypes().size() != 2) - throw Exception("First argument for this overload of " + getName() + " must be a function with 2 arguments. Found " - + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - arguments[0] = std::make_shared(nested_types); - } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - if (arguments.size() != 2) - throw Exception("Function " + getName() + " needs at least 2 argument; passed " - + toString(arguments.size()) + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); - - if (!data_type_function) - throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. - DataTypePtr return_type = removeLowCardinality(data_type_function->getReturnType()); - if (Impl::needBoolean() && !WhichDataType(return_type).isUInt8()) - throw Exception( - "Expression for function " + getName() + " must return UInt8, found " + return_type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const auto * map_type = checkAndGetDataType(arguments[1].type.get()); - if (!map_type) - throw Exception("Second argument for function " + getName() + " must be a map.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return Impl::getReturnType(return_type, map_type->getKeyValueTypes()); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override - { - const auto & column_with_type_and_name = arguments[0]; - if (!column_with_type_and_name.column) - throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const auto * column_function = typeid_cast(column_with_type_and_name.column.get()); - if (!column_function) - throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - ColumnPtr column_map_ptr = arguments[1].column; - const auto * column_map = checkAndGetColumn(column_map_ptr.get()); - if (!column_map) - { - const ColumnConst * column_const_map = checkAndGetColumnConst(column_map_ptr.get()); - if (!column_const_map) - throw Exception("Expected map column, found " + column_map_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); - column_map_ptr = recursiveRemoveLowCardinality(column_const_map->convertToFullColumn()); - column_map = checkAndGetColumn(column_map_ptr.get()); - } - - const DataTypePtr & map_type_ptr = arguments[1].type; - const auto * map_type = checkAndGetDataType(map_type_ptr.get()); - const auto & offsets_column = column_map->getNestedColumn().getOffsets(); - - /// Put all the necessary columns multiplied by the sizes of maps into the columns. - auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(offsets_column)); - auto * replicated_column_function = typeid_cast(replicated_column_function_ptr.get()); - const ColumnsWithTypeAndName args { - {column_map->getNestedData().getColumnPtr(0), recursiveRemoveLowCardinality(map_type->getKeyType()), arguments[1].name + ".key"}, - {column_map->getNestedData().getColumnPtr(1), recursiveRemoveLowCardinality(map_type->getValueType()), arguments[1].name + ".value"}}; - replicated_column_function->appendArguments(args); - - auto lambda_result = replicated_column_function->reduce().column; - if (lambda_result->lowCardinality()) - lambda_result = lambda_result->convertToFullColumnIfLowCardinality(); - - return Impl::execute(*column_map, lambda_result); - } -}; - -} diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 029e33db0cf..32fccd89244 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -1,18 +1,29 @@ #pragma once +#include + +#include +#include +#include +#include +#include + +#include +#include +#include + #include #include #include -#include -#include -#include -#include -#include -#include +#include + #include -#include +#include + #include +#include + namespace DB { @@ -26,6 +37,32 @@ namespace ErrorCodes } +template +ColumnPtr getOffsetsPtr(const T & column) +{ + if constexpr (std::is_same_v) + { + return column.getOffsetsPtr(); + } + else // ColumnMap + { + return column.getNestedColumn().getOffsetsPtr(); + } +} + +template +const IColumn::Offsets & getOffsets(const T & column) +{ + if constexpr (std::is_same_v) + { + return column.getOffsets(); + } + else // ColumnMap + { + return column.getNestedColumn().getOffsets(); + } +} + /** Higher-order functions for arrays. * These functions optionally apply a map (transform) to array (or multiple arrays of identical size) by lambda function, * and return some result based on that transformation. @@ -68,14 +105,23 @@ public: throw Exception("Function " + getName() + " needs at least one array argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - DataTypes nested_types(arguments.size() - 1); - for (size_t i = 0; i < nested_types.size(); ++i) + size_t nested_types_count = std::is_same_v ? (arguments.size() - 1) * 2 : (arguments.size() - 1); + DataTypes nested_types(nested_types_count); + for (size_t i = 0; i < arguments.size() - 1; ++i) { - const DataTypeArray * array_type = checkAndGetDataType(&*arguments[i + 1]); + const auto * array_type = checkAndGetDataType(&*arguments[i + 1]); if (!array_type) throw Exception("Argument " + toString(i + 2) + " of function " + getName() + " must be array. Found " + arguments[i + 1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - nested_types[i] = recursiveRemoveLowCardinality(array_type->getNestedType()); + if constexpr (std::is_same_v) + { + nested_types[2 * i] = recursiveRemoveLowCardinality(array_type->getKeyType()); + nested_types[2 * i + 1] = recursiveRemoveLowCardinality(array_type->getValueType()); + } + else if constexpr (std::is_same_v) + { + nested_types[i] = recursiveRemoveLowCardinality(array_type->getNestedType()); + } } const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); @@ -98,7 +144,7 @@ public: if (arguments.size() == 1) { - const auto * array_type = checkAndGetDataType(arguments[0].type.get()); + const auto * array_type = checkAndGetDataType(arguments[0].type.get()); if (!array_type) throw Exception("The only argument for function " + getName() + " must be array. Found " @@ -110,7 +156,10 @@ public: throw Exception("The only argument for function " + getName() + " must be array of UInt8. Found " + arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return Impl::getReturnType(nested_type, nested_type); + if constexpr (std::is_same_v) + return Impl::getReturnType(nested_type, nested_type); + else + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unreachable code reached"); } else { @@ -131,9 +180,11 @@ public: throw Exception("Expression for function " + getName() + " must return UInt8, found " + return_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - const auto * first_array_type = checkAndGetDataType(arguments[1].type.get()); - - return Impl::getReturnType(return_type, first_array_type->getNestedType()); + const auto * first_array_type = checkAndGetDataType(arguments[1].type.get()); + if constexpr (std::is_same_v) + return Impl::getReturnType(return_type, first_array_type->getNestedType()); + else + return Impl::getReturnType(return_type, first_array_type->getKeyValueTypes()); } } @@ -142,18 +193,25 @@ public: if (arguments.size() == 1) { ColumnPtr column_array_ptr = arguments[0].column; - const auto * column_array = checkAndGetColumn(column_array_ptr.get()); + const auto * column_array = checkAndGetColumn(column_array_ptr.get()); if (!column_array) { - const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); + const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); if (!column_const_array) throw Exception("Expected array column, found " + column_array_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); column_array_ptr = column_const_array->convertToFullColumn(); - column_array = assert_cast(column_array_ptr.get()); + column_array = assert_cast(column_array_ptr.get()); } - return Impl::execute(*column_array, column_array->getDataPtr()); + if constexpr (std::is_same_v) + { + return Impl::execute(*column_array, column_array->getNestedColumn().getDataPtr()); + } + else + { + return Impl::execute(*column_array, column_array->getDataPtr()); + } } else { @@ -172,7 +230,7 @@ public: ColumnPtr offsets_column; ColumnPtr column_first_array_ptr; - const ColumnArray * column_first_array = nullptr; + const typename Impl::column_type * column_first_array = nullptr; ColumnsWithTypeAndName arrays; arrays.reserve(arguments.size() - 1); @@ -182,18 +240,18 @@ public: const auto & array_with_type_and_name = arguments[i]; ColumnPtr column_array_ptr = array_with_type_and_name.column; - const auto * column_array = checkAndGetColumn(column_array_ptr.get()); + const auto * column_array = checkAndGetColumn(column_array_ptr.get()); const DataTypePtr & array_type_ptr = array_with_type_and_name.type; - const auto * array_type = checkAndGetDataType(array_type_ptr.get()); + const auto * array_type = checkAndGetDataType(array_type_ptr.get()); if (!column_array) { - const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); + const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); if (!column_const_array) throw Exception("Expected array column, found " + column_array_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); column_array_ptr = recursiveRemoveLowCardinality(column_const_array->convertToFullColumn()); - column_array = checkAndGetColumn(column_array_ptr.get()); + column_array = checkAndGetColumn(column_array_ptr.get()); } if (!array_type) @@ -201,13 +259,13 @@ public: if (!offsets_column) { - offsets_column = column_array->getOffsetsPtr(); + offsets_column = getOffsetsPtr(*column_array); } else { /// The first condition is optimization: do not compare data if the pointers are equal. - if (column_array->getOffsetsPtr() != offsets_column - && column_array->getOffsets() != typeid_cast(*offsets_column).getData()) + if (getOffsetsPtr(*column_array) != offsets_column + && getOffsets(*column_array) != typeid_cast(*offsets_column).getData()) throw Exception("Arrays passed to " + getName() + " must have equal size", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); } @@ -217,13 +275,23 @@ public: column_first_array = column_array; } - arrays.emplace_back(ColumnWithTypeAndName(column_array->getDataPtr(), - recursiveRemoveLowCardinality(array_type->getNestedType()), - array_with_type_and_name.name)); + if constexpr (std::is_same_v) + { + arrays.emplace_back(ColumnWithTypeAndName( + column_array->getNestedData().getColumnPtr(0), recursiveRemoveLowCardinality(array_type->getKeyType()), array_with_type_and_name.name+".key")); + arrays.emplace_back(ColumnWithTypeAndName( + column_array->getNestedData().getColumnPtr(1), recursiveRemoveLowCardinality(array_type->getValueType()), array_with_type_and_name.name+".value")); + } + else + { + arrays.emplace_back(ColumnWithTypeAndName(column_array->getDataPtr(), + recursiveRemoveLowCardinality(array_type->getNestedType()), + array_with_type_and_name.name)); + } } /// Put all the necessary columns multiplied by the sizes of arrays into the columns. - auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(column_first_array->getOffsets())); + auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(getOffsets(*column_first_array))); auto * replicated_column_function = typeid_cast(replicated_column_function_ptr.get()); replicated_column_function->appendArguments(arrays); diff --git a/src/Functions/array/arrayAggregation.cpp b/src/Functions/array/arrayAggregation.cpp index ee08c4f7f37..97a2f9c4c17 100644 --- a/src/Functions/array/arrayAggregation.cpp +++ b/src/Functions/array/arrayAggregation.cpp @@ -1,12 +1,18 @@ -#include -#include -#include -#include -#include -#include "FunctionArrayMapped.h" -#include #include +#include +#include +#include + +#include +#include +#include +#include + +#include + +#include "FunctionArrayMapped.h" + namespace DB { @@ -83,6 +89,9 @@ using ArrayAggregateResult = typename ArrayAggregateResultImpl struct ArrayAggregateImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayAll.cpp b/src/Functions/array/arrayAll.cpp index 34deafdffdf..0f7ae797dc9 100644 --- a/src/Functions/array/arrayAll.cpp +++ b/src/Functions/array/arrayAll.cpp @@ -1,8 +1,8 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" namespace DB { @@ -16,6 +16,9 @@ namespace ErrorCodes */ struct ArrayAllImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return true; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayCompact.cpp b/src/Functions/array/arrayCompact.cpp index c2908e37e12..aed5e7275d8 100644 --- a/src/Functions/array/arrayCompact.cpp +++ b/src/Functions/array/arrayCompact.cpp @@ -1,10 +1,10 @@ -#include -#include -#include #include -#include -#include +#include +#include +#include #include +#include +#include namespace DB @@ -16,6 +16,9 @@ namespace ErrorCodes struct ArrayCompactImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayCount.cpp b/src/Functions/array/arrayCount.cpp index 377a6eb8fb1..df45783323b 100644 --- a/src/Functions/array/arrayCount.cpp +++ b/src/Functions/array/arrayCount.cpp @@ -1,8 +1,9 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -16,6 +17,9 @@ namespace ErrorCodes */ struct ArrayCountImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return true; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayCumSum.cpp b/src/Functions/array/arrayCumSum.cpp index 467d9ad3951..98ffa09820b 100644 --- a/src/Functions/array/arrayCumSum.cpp +++ b/src/Functions/array/arrayCumSum.cpp @@ -1,10 +1,11 @@ -#include -#include -#include #include -#include "FunctionArrayMapped.h" +#include +#include +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -17,6 +18,9 @@ namespace ErrorCodes struct ArrayCumSumImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayCumSumNonNegative.cpp b/src/Functions/array/arrayCumSumNonNegative.cpp index 476bbd08163..cd8393b7a5f 100644 --- a/src/Functions/array/arrayCumSumNonNegative.cpp +++ b/src/Functions/array/arrayCumSumNonNegative.cpp @@ -1,10 +1,10 @@ -#include -#include -#include #include -#include "FunctionArrayMapped.h" +#include +#include +#include #include +#include "FunctionArrayMapped.h" namespace DB { @@ -19,6 +19,9 @@ namespace ErrorCodes */ struct ArrayCumSumNonNegativeImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayDifference.cpp b/src/Functions/array/arrayDifference.cpp index c5fdf27100b..8af0e8b04f9 100644 --- a/src/Functions/array/arrayDifference.cpp +++ b/src/Functions/array/arrayDifference.cpp @@ -1,10 +1,11 @@ -#include -#include -#include #include -#include "FunctionArrayMapped.h" +#include +#include +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -20,6 +21,9 @@ namespace ErrorCodes */ struct ArrayDifferenceImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayExists.cpp b/src/Functions/array/arrayExists.cpp index 34ea71af259..ea39cc0dc0b 100644 --- a/src/Functions/array/arrayExists.cpp +++ b/src/Functions/array/arrayExists.cpp @@ -1,8 +1,9 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -16,6 +17,9 @@ namespace ErrorCodes */ struct ArrayExistsImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return true; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayFill.cpp b/src/Functions/array/arrayFill.cpp index d4b36a89ba5..22b9e9a657b 100644 --- a/src/Functions/array/arrayFill.cpp +++ b/src/Functions/array/arrayFill.cpp @@ -1,8 +1,9 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -19,6 +20,9 @@ namespace ErrorCodes template struct ArrayFillImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return true; } static bool needExpression() { return true; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayFilter.cpp b/src/Functions/array/arrayFilter.cpp index 1291989f9a2..89a9de44532 100644 --- a/src/Functions/array/arrayFilter.cpp +++ b/src/Functions/array/arrayFilter.cpp @@ -1,8 +1,9 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -15,6 +16,9 @@ namespace ErrorCodes */ struct ArrayFilterImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return true; } static bool needExpression() { return true; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayFirst.cpp b/src/Functions/array/arrayFirst.cpp index edbf7ef6269..693aea746f5 100644 --- a/src/Functions/array/arrayFirst.cpp +++ b/src/Functions/array/arrayFirst.cpp @@ -1,8 +1,9 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -20,6 +21,9 @@ enum class ArrayFirstLastStrategy template struct ArrayFirstLastImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return true; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayFirstLastIndex.cpp b/src/Functions/array/arrayFirstLastIndex.cpp index 467678f3faa..f5570d37ba4 100644 --- a/src/Functions/array/arrayFirstLastIndex.cpp +++ b/src/Functions/array/arrayFirstLastIndex.cpp @@ -20,6 +20,9 @@ enum class ArrayFirstLastIndexStrategy template struct ArrayFirstLastIndexImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return true; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayMap.cpp b/src/Functions/array/arrayMap.cpp index e3afaf7fb66..ec1973d573b 100644 --- a/src/Functions/array/arrayMap.cpp +++ b/src/Functions/array/arrayMap.cpp @@ -1,14 +1,18 @@ -#include "FunctionArrayMapped.h" #include +#include "FunctionArrayMapped.h" + namespace DB { -/** arrayMap(x1,...,xn -> expression, array1,...,arrayn) - apply the expression to each element of the array (or set of parallel arrays). +/** arrayMap(x1, ..., xn -> expression, array1, ..., arrayn) - apply the expression to each element of the array (or set of parallel arrays). */ struct ArrayMapImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + /// true if the expression (for an overload of f(expression, arrays)) or an array (for f(array)) should be boolean. static bool needBoolean() { return false; } /// true if the f(array) overload is unavailable. diff --git a/src/Functions/array/arraySort.cpp b/src/Functions/array/arraySort.cpp index 476dfb46f07..d87b941ab21 100644 --- a/src/Functions/array/arraySort.cpp +++ b/src/Functions/array/arraySort.cpp @@ -1,7 +1,10 @@ #include "FunctionArrayMapped.h" + #include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -11,6 +14,9 @@ namespace DB template struct ArraySortImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arraySplit.cpp b/src/Functions/array/arraySplit.cpp index 2e5f2d8432e..c818be97f60 100644 --- a/src/Functions/array/arraySplit.cpp +++ b/src/Functions/array/arraySplit.cpp @@ -1,8 +1,9 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -14,6 +15,9 @@ namespace ErrorCodes template struct ArraySplitImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return true; } static bool needExpression() { return true; } static bool needOneArray() { return false; } diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 69bbf0c00f3..2308b22d3db 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -1,24 +1,37 @@ -#include -#include +#include #include -#include +#include +#include #include #include +#include namespace DB { + namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -/** MapFilter(x -> predicate, array) - leave in the array only the elements for which the expression is true. +/** Higher-order functions for map. + * These functions optionally apply a map by lambda function, + * and return some result based on that transformation. + */ + + +/** mapFilter((k, v) -> predicate, map) - leave in the map only the kv elements for which the expression is true. */ struct MapFilterImpl { + using data_type = DataTypeMap; + using column_type = ColumnMap; + static bool needBoolean() { return true; } + static bool needExpression() { return true; } + static bool needOneArray() { return false; } static DataTypePtr getReturnType(const DataTypePtr & /*expression_return*/, const DataTypes & elems) { @@ -73,7 +86,7 @@ struct MapFilterImpl }; struct NameMapFilter { static constexpr auto name = "mapFilter"; }; -using FunctionMapFilter = FunctionMapAdvance; +using FunctionMapFilter = FunctionArrayMapped; void registerFunctionMapFilter(FunctionFactory & factory) { @@ -85,8 +98,13 @@ void registerFunctionMapFilter(FunctionFactory & factory) */ struct MapApplyImpl { + using data_type = DataTypeMap; + using column_type = ColumnMap; + /// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean. static bool needBoolean() { return false; } + static bool needExpression() { return true; } + static bool needOneArray() { return false; } static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & /*elems*/) { @@ -116,7 +134,7 @@ struct MapApplyImpl }; struct NameMapApply { static constexpr auto name = "mapApply"; }; -using FunctionMapApply = FunctionMapAdvance; +using FunctionMapApply = FunctionArrayMapped; void registerFunctionMapApply(FunctionFactory & factory) { From 939a15d29a0e2b2d4739d0b7fb32e2bfdc4f0560 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 17 Feb 2022 15:40:26 +0000 Subject: [PATCH 10/14] Upd FunctionArrayMapped for Map --- src/Functions/array/FunctionArrayMapped.h | 19 +++++++++- src/Functions/mapFilter.cpp | 38 +++++++++---------- .../registerFunctionsHigherOrder.cpp | 2 - .../0_stateless/02169_map_functions.reference | 5 ++- .../0_stateless/02169_map_functions.sql | 19 +++++++--- 5 files changed, 53 insertions(+), 30 deletions(-) diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 32fccd89244..28540354b94 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -180,11 +180,28 @@ public: throw Exception("Expression for function " + getName() + " must return UInt8, found " + return_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + static_assert( + std::is_same_v || + std::is_same_v, + "unsupported type"); + + if (arguments.size() < 2) + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "{}", arguments.size()); + } + const auto * first_array_type = checkAndGetDataType(arguments[1].type.get()); + + if (!first_array_type) + throw DB::Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Unsupported type {}", arguments[1].type->getName()); + if constexpr (std::is_same_v) return Impl::getReturnType(return_type, first_array_type->getNestedType()); - else + + if constexpr (std::is_same_v) return Impl::getReturnType(return_type, first_array_type->getKeyValueTypes()); + + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unreachable code reached"); } } diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 2308b22d3db..78a7934b2ba 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -29,6 +30,8 @@ struct MapFilterImpl using data_type = DataTypeMap; using column_type = ColumnMap; + static constexpr auto name = "mapFilter"; + static bool needBoolean() { return true; } static bool needExpression() { return true; } static bool needOneArray() { return false; } @@ -85,14 +88,6 @@ struct MapFilterImpl } }; -struct NameMapFilter { static constexpr auto name = "mapFilter"; }; -using FunctionMapFilter = FunctionArrayMapped; - -void registerFunctionMapFilter(FunctionFactory & factory) -{ - factory.registerFunction(); -} - /** mapApply((k,v) -> expression, map) - apply the expression to the map. */ @@ -101,6 +96,8 @@ struct MapApplyImpl using data_type = DataTypeMap; using column_type = ColumnMap; + static constexpr auto name = "mapApply"; + /// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean. static bool needBoolean() { return false; } static bool needExpression() { return true; } @@ -108,12 +105,15 @@ struct MapApplyImpl static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & /*elems*/) { - const auto & tuple_types = typeid_cast(&*expression_return)->getElements(); - if (tuple_types.size() != 2) - throw Exception("Expected 2 columns as map's key and value, but found " - + toString(tuple_types.size()) + " columns", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + const auto * tuple_types = typeid_cast(expression_return.get()); + if (!tuple_types) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Expected return type is tuple, got {}", expression_return->getName()); + if (tuple_types->getElements().size() != 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Expected 2 columns as map's key and value, but found {}", tuple_types->getElements().size()); - return std::make_shared(tuple_types); + return std::make_shared(tuple_types->getElements()); } static ColumnPtr execute(const ColumnMap & map, ColumnPtr mapped) @@ -123,9 +123,9 @@ struct MapApplyImpl { const ColumnConst * column_const_tuple = checkAndGetColumnConst(mapped.get()); if (!column_const_tuple) - throw Exception("Expected tuple column, found " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN); - ColumnPtr column_tuple_ptr = recursiveRemoveLowCardinality(column_const_tuple->convertToFullColumn()); - column_tuple = checkAndGetColumn(column_tuple_ptr.get()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected tuple column, found {}", mapped->getName()); + auto cols = convertConstTupleToConstantElements(*column_const_tuple); + return ColumnMap::create(cols[0]->convertToFullColumnIfConst(), cols[1]->convertToFullColumnIfConst(), map.getNestedColumn().getOffsetsPtr()); } return ColumnMap::create(column_tuple->getColumnPtr(0), column_tuple->getColumnPtr(1), @@ -133,12 +133,10 @@ struct MapApplyImpl } }; -struct NameMapApply { static constexpr auto name = "mapApply"; }; -using FunctionMapApply = FunctionArrayMapped; - void registerFunctionMapApply(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction>(); + factory.registerFunction>(); } } diff --git a/src/Functions/registerFunctionsHigherOrder.cpp b/src/Functions/registerFunctionsHigherOrder.cpp index 00b300b18b9..00bea58b918 100644 --- a/src/Functions/registerFunctionsHigherOrder.cpp +++ b/src/Functions/registerFunctionsHigherOrder.cpp @@ -18,7 +18,6 @@ void registerFunctionsArraySort(FunctionFactory & factory); void registerFunctionArrayCumSum(FunctionFactory & factory); void registerFunctionArrayCumSumNonNegative(FunctionFactory & factory); void registerFunctionArrayDifference(FunctionFactory & factory); -void registerFunctionMapFilter(FunctionFactory & factory); void registerFunctionMapApply(FunctionFactory & factory); void registerFunctionsHigherOrder(FunctionFactory & factory) @@ -38,7 +37,6 @@ void registerFunctionsHigherOrder(FunctionFactory & factory) registerFunctionArrayCumSum(factory); registerFunctionArrayCumSumNonNegative(factory); registerFunctionArrayDifference(factory); - registerFunctionMapFilter(factory); registerFunctionMapApply(factory); } diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index c570ba4e724..160aebbc852 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -19,7 +19,6 @@ {'key1':1112,'key2':2223} {'key1':1113,'key2':2225} {'key1':1114,'key2':2227} -{1:2,2:3} {} {} {} @@ -28,3 +27,7 @@ {} {} {3:2,1:0,2:0} +{1:2,2:3} +{1:2,2:3} +{'x':'y','x':'y'} +{'x':'y','x':'y'} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 1d8f90e8a90..ee2e70f82cd 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -1,13 +1,20 @@ DROP TABLE IF EXISTS table_map; -create TABLE table_map (id UInt32, col Map(String, UInt64)) engine = MergeTree() ORDER BY tuple(); +CREATE TABLE table_map (id UInt32, col Map(String, UInt64)) engine = MergeTree() ORDER BY tuple(); INSERT INTO table_map SELECT number, map('key1', number, 'key2', number * 2) FROM numbers(1111, 3); INSERT INTO table_map SELECT number, map('key3', number, 'key2', number + 1, 'key4', number + 2) FROM numbers(100, 4); -SELECT mapFilter((k,v)->k like '%3' and v > 102, col) FROM table_map ORDER BY id; +SELECT mapFilter((k, v) -> k like '%3' and v > 102, col) FROM table_map ORDER BY id; SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC; -SELECT mapApply((k,v)->(k,v+1), col) FROM table_map ORDER BY id; -SELECT mapApply((x, y) -> (x, x + 1), map(1, 0, 2, 0)); -SELECT mapFilter((k,v)->0, col) from table_map; +SELECT mapApply((k, v) -> (k, v + 1), col) FROM table_map ORDER BY id; +SELECT mapFilter((k, v) -> 0, col) from table_map; +SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError 42 } + SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0)); -SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError 42 } +SELECT mapApply((x, y) -> (x, x + 1), map(1, 0, 2, 0)); +SELECT mapApply((x, y) -> (x, x + 1), materialize(map(1, 0, 2, 0))); +SELECT mapApply((x, y) -> ('x', 'y'), map(1, 0, 2, 0)); +SELECT mapApply((x, y) -> ('x', 'y'), materialize(map(1, 0, 2, 0))); +SELECT mapApply((x, y) -> (x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapApply((x, y) -> ('x'), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + DROP TABLE table_map; From f7f002139a6d266eb1c9a30c5c6e39d257380a78 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 23 Feb 2022 09:22:40 +0000 Subject: [PATCH 11/14] Upd FunctionArrayMapped --- src/Functions/array/FunctionArrayMapped.h | 2 +- src/Functions/mapFilter.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 28540354b94..e32b4c8e1f3 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -142,7 +142,7 @@ public: + toString(arguments.size()) + ".", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (arguments.size() == 1) + if ((arguments.size() == 1) && std::is_same_v) { const auto * array_type = checkAndGetDataType(arguments[0].type.get()); diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 78a7934b2ba..f38f8f8b4d1 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -34,7 +34,7 @@ struct MapFilterImpl static bool needBoolean() { return true; } static bool needExpression() { return true; } - static bool needOneArray() { return false; } + static bool needOneArray() { return true; } static DataTypePtr getReturnType(const DataTypePtr & /*expression_return*/, const DataTypes & elems) { @@ -101,7 +101,7 @@ struct MapApplyImpl /// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean. static bool needBoolean() { return false; } static bool needExpression() { return true; } - static bool needOneArray() { return false; } + static bool needOneArray() { return true; } static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & /*elems*/) { From 95530d0d70e3ae80c340721c422dbc523ab09a53 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Wed, 23 Feb 2022 15:18:18 +0800 Subject: [PATCH 12/14] fix style error --- src/Functions/array/FunctionArrayMapped.h | 1 + src/Functions/array/arrayCompact.cpp | 5 ++++- src/Functions/array/arrayFirstLastIndex.cpp | 3 ++- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index e32b4c8e1f3..df34abccab9 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -32,6 +32,7 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } diff --git a/src/Functions/array/arrayCompact.cpp b/src/Functions/array/arrayCompact.cpp index aed5e7275d8..a2cfb1cdccd 100644 --- a/src/Functions/array/arrayCompact.cpp +++ b/src/Functions/array/arrayCompact.cpp @@ -1,10 +1,13 @@ #include #include + +#include + #include #include + #include #include -#include namespace DB diff --git a/src/Functions/array/arrayFirstLastIndex.cpp b/src/Functions/array/arrayFirstLastIndex.cpp index f5570d37ba4..9392cbdc840 100644 --- a/src/Functions/array/arrayFirstLastIndex.cpp +++ b/src/Functions/array/arrayFirstLastIndex.cpp @@ -1,8 +1,9 @@ #include #include -#include "FunctionArrayMapped.h" #include +#include "FunctionArrayMapped.h" + namespace DB { From 2aa6685baf9354cc8a71977503a30f8e105b7136 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 3 Mar 2022 18:04:17 +0000 Subject: [PATCH 13/14] Add cases to 02169_map_functions, update error messages in FunctionArrayMapped --- src/Functions/array/FunctionArrayMapped.h | 41 ++++++++++--------- .../0_stateless/02169_map_functions.sql | 21 +++++++++- 2 files changed, 42 insertions(+), 20 deletions(-) diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index df34abccab9..58e6db86f75 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -98,13 +98,16 @@ public: void getLambdaArgumentTypes(DataTypes & arguments) const override { if (arguments.empty()) - throw Exception("Function " + getName() + " needs at least one argument; passed " - + toString(arguments.size()) + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} needs at least one argument, passed {}", getName(), arguments.size()); if (arguments.size() == 1) - throw Exception("Function " + getName() + " needs at least one array argument.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} needs at least one argument with data", getName()); + + if (arguments.size() > 2 && Impl::needOneArray()) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} needs one argument with data", getName()); size_t nested_types_count = std::is_same_v ? (arguments.size() - 1) * 2 : (arguments.size() - 1); DataTypes nested_types(nested_types_count); @@ -127,9 +130,10 @@ public: const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); if (!function_type || function_type->getArgumentTypes().size() != nested_types.size()) - throw Exception("First argument for this overload of " + getName() + " must be a function with " - + toString(nested_types.size()) + " arguments. Found " - + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for this overload of {} must be a function with {} arguments, found {} instead", + getName(), nested_types.size(), arguments[0]->getName()); arguments[0] = std::make_shared(nested_types); } @@ -138,24 +142,23 @@ public: { size_t min_args = Impl::needExpression() ? 2 : 1; if (arguments.size() < min_args) - throw Exception("Function " + getName() + " needs at least " - + toString(min_args) + " argument; passed " - + toString(arguments.size()) + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} needs at least {} argument, passed {}", + getName(), min_args, arguments.size()); if ((arguments.size() == 1) && std::is_same_v) { - const auto * array_type = checkAndGetDataType(arguments[0].type.get()); + const auto * data_type = checkAndGetDataType(arguments[0].type.get()); - if (!array_type) + if (!data_type) throw Exception("The only argument for function " + getName() + " must be array. Found " - + arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + arguments[0].type->getName() + " instead", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - DataTypePtr nested_type = array_type->getNestedType(); + DataTypePtr nested_type = data_type->getNestedType(); if (Impl::needBoolean() && !WhichDataType(nested_type).isUInt8()) throw Exception("The only argument for function " + getName() + " must be array of UInt8. Found " - + arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + arguments[0].type->getName() + " instead", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if constexpr (std::is_same_v) return Impl::getReturnType(nested_type, nested_type); @@ -165,13 +168,13 @@ public: else { if (arguments.size() > 2 && Impl::needOneArray()) - throw Exception("Function " + getName() + " needs one array argument.", + throw Exception("Function " + getName() + " needs one argument with data", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); if (!data_type_function) - throw Exception("First argument for function " + getName() + " must be a function.", + throw Exception("First argument for function " + getName() + " must be a function", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index ee2e70f82cd..4cccaa56722 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -7,14 +7,33 @@ SELECT mapFilter((k, v) -> k like '%3' and v > 102, col) FROM table_map ORDER BY SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC; SELECT mapApply((k, v) -> (k, v + 1), col) FROM table_map ORDER BY id; SELECT mapFilter((k, v) -> 0, col) from table_map; -SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError 42 } +SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0)); SELECT mapApply((x, y) -> (x, x + 1), map(1, 0, 2, 0)); SELECT mapApply((x, y) -> (x, x + 1), materialize(map(1, 0, 2, 0))); SELECT mapApply((x, y) -> ('x', 'y'), map(1, 0, 2, 0)); SELECT mapApply((x, y) -> ('x', 'y'), materialize(map(1, 0, 2, 0))); + +SELECT mapApply(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT mapApply((x, y) -> (x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT mapApply((x, y) -> ('x'), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapApply((x) -> (x, x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapApply((x, y) -> (x, 1, 2), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT mapApply((x, y) -> (x, x + 1)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT mapApply(map(1, 0, 2, 0), (x, y) -> (x, x + 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapApply((x, y) -> (x, x+1), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT mapFilter(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT mapFilter((x, y) -> (toInt32(x)), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapFilter((x, y) -> ('x'), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapFilter((x) -> (x, x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapFilter((x, y) -> (x, 1, 2), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapFilter((x, y) -> (x, x + 1)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT mapFilter(map(1, 0, 2, 0), (x, y) -> (x > 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapFilter((x, y) -> (x, x + 1), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT mapUpdate(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } DROP TABLE table_map; From e9dee63fe2a0a6d533afdfec93d3dfbe4b29b77a Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Mar 2022 10:49:43 +0000 Subject: [PATCH 14/14] style fix --- src/Functions/array/arraySort.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Functions/array/arraySort.cpp b/src/Functions/array/arraySort.cpp index d87b941ab21..5421185211e 100644 --- a/src/Functions/array/arraySort.cpp +++ b/src/Functions/array/arraySort.cpp @@ -3,9 +3,6 @@ #include #include -#include "FunctionArrayMapped.h" - - namespace DB {