mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Introduce mapContains, mapKeys, mapValues functions for Map data type
This commit is contained in:
parent
7f85ae7fa7
commit
31105670a8
@ -1,20 +1,28 @@
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <memory>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#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<FunctionMapContains>(); }
|
||||
|
||||
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<DataTypeMap>(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<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
const ColumnMap * col_map = typeid_cast<const ColumnMap *>(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<DataTypeArray>(result_type),
|
||||
""
|
||||
},
|
||||
arguments[1]
|
||||
};
|
||||
|
||||
return FunctionArrayIndex<HasAction, NameMapContains>().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<FunctionMapKeys>(); }
|
||||
|
||||
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<DataTypeMap>(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<DataTypeArray>(key_type);
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnMap * col_map = typeid_cast<const ColumnMap *>(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<FunctionMapValues>(); }
|
||||
|
||||
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<DataTypeMap>(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<DataTypeArray>(value_type);
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnMap * col_map = typeid_cast<const ColumnMap *>(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<FunctionMap>();
|
||||
factory.registerFunction<FunctionMapContains>();
|
||||
factory.registerFunction<FunctionMapKeys>();
|
||||
factory.registerFunction<FunctionMapValues>();
|
||||
}
|
||||
|
||||
}
|
||||
|
16
tests/queries/0_stateless/01651_map_functions.reference
Normal file
16
tests/queries/0_stateless/01651_map_functions.reference
Normal file
@ -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]
|
21
tests/queries/0_stateless/01651_map_functions.sql
Normal file
21
tests/queries/0_stateless/01651_map_functions.sql
Normal file
@ -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;
|
Loading…
Reference in New Issue
Block a user