From 31105670a8a62050f87780f595ac9c2fb2e8492d Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Wed, 6 Jan 2021 18:53:14 +0800 Subject: [PATCH] Introduce mapContains, mapKeys, mapValues functions for Map data type --- src/Functions/map.cpp | 161 ++++++++++++++++++ .../0_stateless/01651_map_functions.reference | 16 ++ .../0_stateless/01651_map_functions.sql | 21 +++ 3 files changed, 198 insertions(+) create mode 100644 tests/queries/0_stateless/01651_map_functions.reference create mode 100644 tests/queries/0_stateless/01651_map_functions.sql diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 5993ab3706e..dd74d3efa47 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -1,20 +1,28 @@ #include #include +#include #include #include #include #include #include +#include +#include #include #include #include +#include +#include +#include "array/arrayIndex.h" + namespace DB { namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace @@ -130,11 +138,164 @@ public: } }; +struct NameMapContains { static constexpr auto name = "mapContains"; }; + +class FunctionMapContains : public IFunction +{ +public: + static constexpr auto name = NameMapContains::name; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override + { + return NameMapContains::name ; + } + + size_t getNumberOfArguments() const override { return 2; } + + 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 * map_type = checkAndGetDataType(arguments[0].type.get()); + + if (!map_type) + throw Exception{"First argument for function " + getName() + " must be a map.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + auto key_type = map_type->getKeyType(); + + if (!(isNumber(arguments[1].type) && isNumber(key_type)) + && key_type->getName() != arguments[1].type->getName()) + throw Exception{"Second argument for function " + getName() + " must be a " + key_type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + const ColumnMap * col_map = typeid_cast(arguments[0].column.get()); + if (!col_map) + return nullptr; + + const auto & nested_column = col_map->getNestedColumn(); + const auto & keys_data = col_map->getNestedData().getColumn(0); + + /// Prepare arguments to call arrayIndex for check has the array element. + ColumnsWithTypeAndName new_arguments = + { + { + ColumnArray::create(keys_data.getPtr(), nested_column.getOffsetsPtr()), + std::make_shared(result_type), + "" + }, + arguments[1] + }; + + return FunctionArrayIndex().executeImpl(new_arguments, result_type, input_rows_count); + } +}; + +class FunctionMapKeys : public IFunction +{ +public: + static constexpr auto name = "mapKeys"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override + { + return name ; + } + + size_t getNumberOfArguments() const override { return 1; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 1) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const DataTypeMap * map_type = checkAndGetDataType(arguments[0].type.get()); + + if (!map_type) + throw Exception{"First argument for function " + getName() + " must be a map.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + auto key_type = map_type->getKeyType(); + + return std::make_shared(key_type); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override + { + const ColumnMap * col_map = typeid_cast(arguments[0].column.get()); + if (!col_map) + return nullptr; + + const auto & nested_column = col_map->getNestedColumn(); + const auto & keys_data = col_map->getNestedData().getColumn(0); + + return ColumnArray::create(keys_data.getPtr(), nested_column.getOffsetsPtr()); + } +}; + +class FunctionMapValues : public IFunction +{ +public: + static constexpr auto name = "mapValues"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override + { + return name ; + } + + size_t getNumberOfArguments() const override { return 1; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 1) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const DataTypeMap * map_type = checkAndGetDataType(arguments[0].type.get()); + + if (!map_type) + throw Exception{"First argument for function " + getName() + " must be a map.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + auto value_type = map_type->getValueType(); + + return std::make_shared(value_type); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override + { + const ColumnMap * col_map = typeid_cast(arguments[0].column.get()); + if (!col_map) + return nullptr; + + const auto & nested_column = col_map->getNestedColumn(); + const auto & values_data = col_map->getNestedData().getColumn(1); + + return ColumnArray::create(values_data.getPtr(), nested_column.getOffsetsPtr()); + } +}; + } void registerFunctionsMap(FunctionFactory & factory) { factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference new file mode 100644 index 00000000000..efcd9ce8bcd --- /dev/null +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -0,0 +1,16 @@ +1 +1 +0 +1 +0 +0 +1 +0 +['name','age'] +['name','gender'] +1 0 0 +1 0 1 +1 0 0 +[232] +[233] +[234] diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql new file mode 100644 index 00000000000..30ca3a4aeea --- /dev/null +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -0,0 +1,21 @@ +set allow_experimental_map_type = 1; + +-- String type +drop table if exists table_map; +create table table_map (a Map(String, String), b String) engine = Memory; +insert into table_map values ({'name':'zhangsan', 'age':'10'}, 'name'), ({'name':'lisi', 'gender':'female'},'age'); +select mapContains(a, 'name') from table_map; +select mapContains(a, 'gender') from table_map; +select mapContains(a, 'abc') from table_map; +select mapContains(a, b) from table_map; +select mapContains(a, 10) from table_map; -- { serverError 43 } +select mapKeys(a) from table_map; +drop table if exists table_map; + +CREATE TABLE table_map (a Map(UInt8, Int), b UInt8, c UInt32) engine = MergeTree order by tuple(); +insert into table_map select map(number, number), number, number from numbers(1000, 3); +select mapContains(a, b), mapContains(a, c), mapContains(a, 233) from table_map; +select mapContains(a, 'aaa') from table_map; -- { serverError 43 } +select mapContains(b, 'aaa') from table_map; -- { serverError 43 } +select mapKeys(a) from table_map; +drop table if exists table_map;