mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #31125 from taiyang-li/map_from_arrasy
Implement spark function mapFromArrays()
This commit is contained in:
commit
74fcffb067
@ -66,6 +66,40 @@ Result:
|
||||
|
||||
- [Map(key, value)](../../sql-reference/data-types/map.md) data type
|
||||
|
||||
## mapFromArrays
|
||||
|
||||
mapFromArrays merges an array of keys and an array of values into a Map.
|
||||
|
||||
**Syntax**
|
||||
|
||||
```sql
|
||||
mapFromArrays(keys, values)
|
||||
```
|
||||
|
||||
Alias: `MAP_FROM_ARRAYS(keys, values)`
|
||||
|
||||
**Parameters**
|
||||
- `keys` — Given key array to create a map from. The nested type of array must be: [String](../../sql-reference/data-types/string.md), [Integer](../../sql-reference/data-types/int-uint.md), [LowCardinality](../../sql-reference/data-types/lowcardinality.md), [FixedString](../../sql-reference/data-types/fixedstring.md), [UUID](../../sql-reference/data-types/uuid.md), [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), [Date32](../../sql-reference/data-types/date32.md), [Enum](../../sql-reference/data-types/enum.md)
|
||||
- `values` - Given value array to create a map from.
|
||||
|
||||
**Returned value**
|
||||
|
||||
- A map whose keys and values are constructed from the key and value arrays
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
```sql
|
||||
select mapFromArrays(['a', 'b', 'c'], [1, 2, 3])
|
||||
```
|
||||
|
||||
```text
|
||||
┌─mapFromArrays(['a', 'b', 'c'], [1, 2, 3])─┐
|
||||
│ {'a':1,'b':2,'c':3} │
|
||||
└───────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## mapAdd
|
||||
|
||||
Collect all the keys and sum corresponding values.
|
||||
@ -429,6 +463,8 @@ Result:
|
||||
│ {} │
|
||||
└────────────────────────────┘
|
||||
```
|
||||
|
||||
|
||||
|
||||
## mapApply
|
||||
|
||||
|
@ -26,6 +26,8 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -147,6 +149,84 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
/// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays
|
||||
class FunctionMapFromArrays : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "mapFromArrays";
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMapFromArrays>(); }
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Function {} requires 2 arguments, but {} given",
|
||||
getName(),
|
||||
arguments.size());
|
||||
|
||||
const auto * keys_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
|
||||
if (!keys_type)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an Array", getName());
|
||||
|
||||
const auto * values_type = checkAndGetDataType<DataTypeArray>(arguments[1].get());
|
||||
if (!values_type)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be an Array", getName());
|
||||
|
||||
DataTypes key_value_types{keys_type->getNestedType(), values_type->getNestedType()};
|
||||
return std::make_shared<DataTypeMap>(key_value_types);
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(
|
||||
const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override
|
||||
{
|
||||
ColumnPtr holder_keys;
|
||||
bool is_keys_const = isColumnConst(*arguments[0].column);
|
||||
const ColumnArray * col_keys;
|
||||
if (is_keys_const)
|
||||
{
|
||||
holder_keys = arguments[0].column->convertToFullColumnIfConst();
|
||||
col_keys = checkAndGetColumn<ColumnArray>(holder_keys.get());
|
||||
}
|
||||
else
|
||||
{
|
||||
col_keys = checkAndGetColumn<ColumnArray>(arguments[0].column.get());
|
||||
}
|
||||
|
||||
ColumnPtr holder_values;
|
||||
bool is_values_const = isColumnConst(*arguments[1].column);
|
||||
const ColumnArray * col_values;
|
||||
if (is_values_const)
|
||||
{
|
||||
holder_values = arguments[1].column->convertToFullColumnIfConst();
|
||||
col_values = checkAndGetColumn<ColumnArray>(holder_values.get());
|
||||
}
|
||||
else
|
||||
{
|
||||
col_values = checkAndGetColumn<ColumnArray>(arguments[1].column.get());
|
||||
}
|
||||
|
||||
if (!col_keys || !col_values)
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Arguments of function {} must be array", getName());
|
||||
|
||||
if (!col_keys->hasEqualOffsets(*col_values))
|
||||
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Array arguments for function {} must have equal sizes", getName());
|
||||
|
||||
const auto & data_keys = col_keys->getDataPtr();
|
||||
const auto & data_values = col_values->getDataPtr();
|
||||
const auto & offsets = col_keys->getOffsetsPtr();
|
||||
auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{data_keys, data_values}), offsets);
|
||||
return ColumnMap::create(nested_column);
|
||||
}
|
||||
};
|
||||
|
||||
struct NameMapContains { static constexpr auto name = "mapContains"; };
|
||||
|
||||
@ -649,6 +729,9 @@ REGISTER_FUNCTION(Map)
|
||||
factory.registerFunction<FunctionMapContainsKeyLike>();
|
||||
factory.registerFunction<FunctionExtractKeyLike>();
|
||||
factory.registerFunction<FunctionMapUpdate>();
|
||||
factory.registerFunction<FunctionMapFromArrays>();
|
||||
factory.registerAlias("MAP_FROM_ARRAYS", "mapFromArrays");
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -8,6 +8,8 @@
|
||||
0
|
||||
['name','age']
|
||||
['name','gender']
|
||||
{'name':'zhangsan','age':'10'}
|
||||
{'name':'lisi','gender':'female'}
|
||||
1 0 0
|
||||
1 0 1
|
||||
1 0 0
|
||||
@ -17,7 +19,17 @@
|
||||
[1000]
|
||||
[1001]
|
||||
[1002]
|
||||
{'1000':'2000','1000':'3000','1000':'4000'}
|
||||
{'1001':'2002','1001':'3003','1001':'4004'}
|
||||
{'1002':'2004','1002':'3006','1002':'4008'}
|
||||
{'aa':4,'bb':5} ['aa','bb'] [4,5]
|
||||
{'aa':4,'bb':5} 1 0
|
||||
{0:0} 1
|
||||
{0:0} 0
|
||||
{'aa':4,'bb':5}
|
||||
{'aa':4,'bb':5}
|
||||
{'aa':4,'bb':5}
|
||||
{'aa':4,'bb':5}
|
||||
{'aa':4,'bb':5}
|
||||
{'aa':4,'bb':5}
|
||||
{'aa':4,'bb':5}
|
||||
|
@ -2,23 +2,25 @@ 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');
|
||||
create table table_map (a Map(String, String), b String, c Array(String), d Array(String)) engine = Memory;
|
||||
insert into table_map values ({'name':'zhangsan', 'age':'10'}, 'name', ['name', 'age'], ['zhangsan', '10']), ({'name':'lisi', 'gender':'female'},'age',['name', 'gender'], ['lisi', 'female']);
|
||||
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 386 }
|
||||
select mapContains(a, 10) from table_map; -- { serverError NO_COMMON_TYPE }
|
||||
select mapKeys(a) from table_map;
|
||||
select mapFromArrays(c, d) 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);
|
||||
CREATE TABLE table_map (a Map(UInt8, Int), b UInt8, c UInt32, d Array(String), e Array(String)) engine = MergeTree order by tuple();
|
||||
insert into table_map select map(number, number), number, number, [number, number, number], [number*2, number*3, number*4] 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 386 }
|
||||
select mapContains(b, 'aaa') from table_map; -- { serverError 43 }
|
||||
select mapContains(a, 'aaa') from table_map; -- { serverError NO_COMMON_TYPE }
|
||||
select mapContains(b, 'aaa') from table_map; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
select mapKeys(a) from table_map;
|
||||
select mapValues(a) from table_map;
|
||||
select mapFromArrays(d, e) from table_map;
|
||||
drop table if exists table_map;
|
||||
|
||||
|
||||
@ -27,3 +29,13 @@ select map( 'aa', 4, 'bb' , 5) as m, mapKeys(m), mapValues(m);
|
||||
select map( 'aa', 4, 'bb' , 5) as m, mapContains(m, 'aa'), mapContains(m, 'k');
|
||||
|
||||
select map(0, 0) as m, mapContains(m, number % 2) from numbers(2);
|
||||
|
||||
select mapFromArrays(['aa', 'bb'], [4, 5]);
|
||||
select mapFromArrays(['aa', 'bb'], materialize([4, 5])) from numbers(2);
|
||||
select mapFromArrays(materialize(['aa', 'bb']), [4, 5]) from numbers(2);
|
||||
select mapFromArrays(materialize(['aa', 'bb']), materialize([4, 5])) from numbers(2);
|
||||
select mapFromArrays('aa', [4, 5]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
select mapFromArrays(['aa', 'bb'], 5); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
select mapFromArrays(['aa', 'bb'], [4, 5], [6, 7]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
|
||||
select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError BAD_ARGUMENTS }
|
||||
|
@ -421,6 +421,7 @@ mapContains
|
||||
mapContainsKeyLike
|
||||
mapExtractKeyLike
|
||||
mapFilter
|
||||
mapFromArrays
|
||||
mapKeys
|
||||
mapPopulateSeries
|
||||
mapSubtract
|
||||
|
Loading…
Reference in New Issue
Block a user