enable length for map, which returns keys number in map

This commit is contained in:
taiyang-li 2021-01-24 22:54:08 +08:00
parent ccb7e6e03f
commit f51c3ee9a3
4 changed files with 27 additions and 1 deletions

View File

@ -597,6 +597,7 @@ inline bool isEnum(const DataTypePtr & data_type) { return WhichDataType(data_ty
inline bool isDecimal(const DataTypePtr & data_type) { return WhichDataType(data_type).isDecimal(); }
inline bool isTuple(const DataTypePtr & data_type) { return WhichDataType(data_type).isTuple(); }
inline bool isArray(const DataTypePtr & data_type) { return WhichDataType(data_type).isArray(); }
inline bool isMap(const DataTypePtr & data_type) {return WhichDataType(data_type).isMap(); }
template <typename T>
inline bool isUInt8(const T & data_type)

View File

@ -7,6 +7,7 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnMap.h>
namespace DB
@ -42,7 +43,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isStringOrFixedString(arguments[0])
&& !isArray(arguments[0]))
&& !isArray(arguments[0]) && !isMap(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeNumber<ResultType>>();
@ -93,6 +94,22 @@ public:
return col_res;
}
else if (const ColumnMap* col_map = checkAndGetColumn<ColumnMap>(column.get()))
{
// only function length is implemented for ColumnMap
// TODO implemented other functions which use FunctionStringOrArrayToT
if (getName() != "length")
throw Exception("Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
auto col_res = ColumnVector<ResultType>::create();
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
vec_res.resize(col_map->size());
const auto & col_nested = col_map->getNestedColumn();
Impl::array(col_nested.getOffsets(), vec_res);
return col_res;
}
else
throw Exception("Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);

View File

@ -0,0 +1,2 @@
2
2

View File

@ -0,0 +1,6 @@
set allow_experimental_map_type = 1;
drop table if exists table_map;
create table table_map (a Map(String, String)) engine = Memory;
insert into table_map values ({'name':'zhangsan', 'gender':'male'}), ({'name':'lisi', 'gender':'female'});
select length(a) from table_map;