mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #67103 from bigo-sg/fix_map_from_arrays
Function `mapFromArrays`: Allow `Map(K, V)` and `Array(Nullable(T))` as 1st arguments
This commit is contained in:
commit
e4c165e919
@ -43,7 +43,7 @@ Result:
|
||||
|
||||
## mapFromArrays
|
||||
|
||||
Creates a map from an array of keys and an array of values.
|
||||
Creates a map from an array or map of keys and an array or map of values.
|
||||
|
||||
The function is a convenient alternative to syntax `CAST([...], 'Map(key_type, value_type)')`.
|
||||
For example, instead of writing
|
||||
@ -62,8 +62,8 @@ Alias: `MAP_FROM_ARRAYS(keys, values)`
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `keys` — Array of keys to create the map from. [Array(T)](../data-types/array.md) where `T` can be any type supported by [Map](../data-types/map.md) as key type.
|
||||
- `values` - Array or map of values to create the map from. [Array](../data-types/array.md) or [Map](../data-types/map.md).
|
||||
- `keys` — Array or map of keys to create the map from [Array](../data-types/array.md) or [Map](../data-types/map.md). If `keys` is an array, we accept `Array(Nullable(T))` or `Array(LowCardinality(Nullable(T)))` as its type as long as it doesn't contain NULL value.
|
||||
- `values` - Array or map of values to create the map from [Array](../data-types/array.md) or [Map](../data-types/map.md).
|
||||
|
||||
**Returned value**
|
||||
|
||||
@ -99,6 +99,18 @@ Result:
|
||||
└───────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
```sql
|
||||
SELECT mapFromArrays(map('a', 1, 'b', 2, 'c', 3), [1, 2, 3])
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```
|
||||
┌─mapFromArrays(map('a', 1, 'b', 2, 'c', 3), [1, 2, 3])─┐
|
||||
│ {('a',1):1,('b',2):2,('c',3):3} │
|
||||
└───────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## extractKeyValuePairs
|
||||
|
||||
Converts a string of key-value pairs to a [Map(String, String)](../data-types/map.md).
|
||||
|
@ -1,14 +1,17 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
@ -21,6 +24,7 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -155,7 +159,7 @@ private:
|
||||
bool use_variant_as_common_type = false;
|
||||
};
|
||||
|
||||
/// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays
|
||||
/// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays or maps
|
||||
class FunctionMapFromArrays : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -179,21 +183,28 @@ public:
|
||||
getName(),
|
||||
arguments.size());
|
||||
|
||||
/// The first argument should always be Array.
|
||||
/// Because key type can not be nested type of Map, which is Tuple
|
||||
DataTypePtr key_type;
|
||||
if (const auto * keys_type = checkAndGetDataType<DataTypeArray>(arguments[0].get()))
|
||||
key_type = keys_type->getNestedType();
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an Array", getName());
|
||||
auto get_nested_type = [&](const DataTypePtr & type)
|
||||
{
|
||||
DataTypePtr nested;
|
||||
if (const auto * type_as_array = checkAndGetDataType<DataTypeArray>(type.get()))
|
||||
nested = type_as_array->getNestedType();
|
||||
else if (const auto * type_as_map = checkAndGetDataType<DataTypeMap>(type.get()))
|
||||
nested = std::make_shared<DataTypeTuple>(type_as_map->getKeyValueTypes());
|
||||
else
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Arguments of function {} must be Array or Map, but {} is given",
|
||||
getName(),
|
||||
type->getName());
|
||||
|
||||
DataTypePtr value_type;
|
||||
if (const auto * value_array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get()))
|
||||
value_type = value_array_type->getNestedType();
|
||||
else if (const auto * value_map_type = checkAndGetDataType<DataTypeMap>(arguments[1].get()))
|
||||
value_type = std::make_shared<DataTypeTuple>(value_map_type->getKeyValueTypes());
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be Array or Map", getName());
|
||||
return nested;
|
||||
};
|
||||
|
||||
auto key_type = get_nested_type(arguments[0]);
|
||||
auto value_type = get_nested_type(arguments[1]);
|
||||
|
||||
/// We accept Array(Nullable(T)) or Array(LowCardinality(Nullable(T))) as key types as long as the actual array doesn't contain NULL value(this is checked in executeImpl).
|
||||
key_type = removeNullableOrLowCardinalityNullable(key_type);
|
||||
|
||||
DataTypes key_value_types{key_type, value_type};
|
||||
return std::make_shared<DataTypeMap>(key_value_types);
|
||||
@ -202,44 +213,59 @@ public:
|
||||
ColumnPtr executeImpl(
|
||||
const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override
|
||||
{
|
||||
bool is_keys_const = isColumnConst(*arguments[0].column);
|
||||
ColumnPtr holder_keys;
|
||||
const ColumnArray * col_keys;
|
||||
if (is_keys_const)
|
||||
auto get_array_column = [&](const ColumnPtr & column) -> std::pair<const ColumnArray *, ColumnPtr>
|
||||
{
|
||||
holder_keys = arguments[0].column->convertToFullColumnIfConst();
|
||||
col_keys = checkAndGetColumn<ColumnArray>(holder_keys.get());
|
||||
}
|
||||
else
|
||||
bool is_const = isColumnConst(*column);
|
||||
ColumnPtr holder = is_const ? column->convertToFullColumnIfConst() : column;
|
||||
|
||||
const ColumnArray * col_res = nullptr;
|
||||
if (const auto * col_array = checkAndGetColumn<ColumnArray>(holder.get()))
|
||||
col_res = col_array;
|
||||
else if (const auto * col_map = checkAndGetColumn<ColumnMap>(holder.get()))
|
||||
col_res = &col_map->getNestedColumn();
|
||||
else
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Argument columns of function {} must be Array or Map, but {} is given",
|
||||
getName(),
|
||||
holder->getName());
|
||||
|
||||
return {col_res, holder};
|
||||
};
|
||||
|
||||
auto [col_keys, key_holder] = get_array_column(arguments[0].column);
|
||||
auto [col_values, values_holder] = get_array_column(arguments[1].column);
|
||||
|
||||
/// Nullable(T) or LowCardinality(Nullable(T)) are okay as nested key types but actual NULL values are not okay.
|
||||
ColumnPtr data_keys = col_keys->getDataPtr();
|
||||
if (isColumnNullableOrLowCardinalityNullable(*data_keys))
|
||||
{
|
||||
col_keys = checkAndGetColumn<ColumnArray>(arguments[0].column.get());
|
||||
const NullMap * null_map = nullptr;
|
||||
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(data_keys.get()))
|
||||
{
|
||||
null_map = &nullable->getNullMapData();
|
||||
data_keys = nullable->getNestedColumnPtr();
|
||||
}
|
||||
else if (const auto * low_cardinality = checkAndGetColumn<ColumnLowCardinality>(data_keys.get()))
|
||||
{
|
||||
if (const auto * nullable_dict = checkAndGetColumn<ColumnNullable>(low_cardinality->getDictionaryPtr().get()))
|
||||
{
|
||||
null_map = &nullable_dict->getNullMapData();
|
||||
data_keys = ColumnLowCardinality::create(nullable_dict->getNestedColumnPtr(), low_cardinality->getIndexesPtr());
|
||||
}
|
||||
}
|
||||
|
||||
if (null_map && !memoryIsZero(null_map->data(), 0, null_map->size()))
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS, "The nested column of first argument in function {} must not contain NULLs", getName());
|
||||
}
|
||||
|
||||
if (!col_keys)
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The first argument of function {} must be Array", getName());
|
||||
|
||||
bool is_values_const = isColumnConst(*arguments[1].column);
|
||||
ColumnPtr holder_values;
|
||||
if (is_values_const)
|
||||
holder_values = arguments[1].column->convertToFullColumnIfConst();
|
||||
else
|
||||
holder_values = arguments[1].column;
|
||||
|
||||
const ColumnArray * col_values;
|
||||
if (const auto * col_values_array = checkAndGetColumn<ColumnArray>(holder_values.get()))
|
||||
col_values = col_values_array;
|
||||
else if (const auto * col_values_map = checkAndGetColumn<ColumnMap>(holder_values.get()))
|
||||
col_values = &col_values_map->getNestedColumn();
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The second arguments of function {} must be Array or Map", getName());
|
||||
|
||||
if (!col_keys->hasEqualOffsets(*col_values))
|
||||
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments for function {} must have equal sizes", getName());
|
||||
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments of 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);
|
||||
auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{std::move(data_keys), data_values}), offsets);
|
||||
return ColumnMap::create(nested_column);
|
||||
}
|
||||
};
|
||||
@ -250,10 +276,7 @@ public:
|
||||
static constexpr auto name = "mapUpdate";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMapUpdate>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
@ -262,9 +285,11 @@ public:
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 2",
|
||||
getName(), arguments.size());
|
||||
getName(),
|
||||
arguments.size());
|
||||
|
||||
const auto * left = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
|
||||
const auto * right = checkAndGetDataType<DataTypeMap>(arguments[1].type.get());
|
||||
@ -380,7 +405,6 @@ public:
|
||||
return ColumnMap::create(nested_column);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
REGISTER_FUNCTION(Map)
|
||||
|
@ -52,3 +52,10 @@
|
||||
{1:4,2:5}
|
||||
{1:4,2:5}
|
||||
{1:4,2:5}
|
||||
{1:3,2:4}
|
||||
{1:3,2:4}
|
||||
{1:3,2:4} {(1,3):'a',(2,4):'b'}
|
||||
{(1,'a'):'c',(2,'b'):'d'}
|
||||
{(1,'a'):'c',(2,'b'):'d'}
|
||||
{(1,'a'):'c',(2,'b'):'d'}
|
||||
{(1,'a'):'c',(2,'b'):'d'}
|
||||
|
@ -67,12 +67,20 @@ select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_
|
||||
select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
|
||||
select mapFromArrays(['a', 2], [4, 5]); -- { serverError NO_COMMON_TYPE}
|
||||
select mapFromArrays([1, 2], [4, 'a']); -- { serverError NO_COMMON_TYPE}
|
||||
select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
|
||||
select mapFromArrays([1,null]::Array(Nullable(UInt8)), [3,4]); -- { serverError BAD_ARGUMENTS }
|
||||
|
||||
select mapFromArrays(['aa', 'bb'], map('a', 4, 'b', 5));
|
||||
select mapFromArrays(['aa', 'bb'], materialize(map('a', 4, 'b', 5))) from numbers(2);
|
||||
|
||||
select mapFromArrays(map('a', 4, 'b', 4), ['aa', 'bb']) from numbers(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
select mapFromArrays(['aa', 'bb'], map('a', 4)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
|
||||
|
||||
select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], [4, 5]);
|
||||
select mapFromArrays([toLowCardinality(1), toLowCardinality(2)], materialize([4, 5])) from numbers(2);
|
||||
|
||||
select mapFromArrays([1,2], [3,4]);
|
||||
select mapFromArrays([1,2]::Array(Nullable(UInt8)), [3,4]);
|
||||
select mapFromArrays([1,2], [3,4]) as x, mapFromArrays(x, ['a', 'b']);
|
||||
|
||||
select mapFromArrays(map(1, 'a', 2, 'b'), array('c', 'd'));
|
||||
select mapFromArrays(materialize(map(1, 'a', 2, 'b')), array('c', 'd'));
|
||||
select mapFromArrays(map(1, 'a', 2, 'b'), materialize(array('c', 'd')));
|
||||
select mapFromArrays(materialize(map(1, 'a', 2, 'b')), materialize(array('c', 'd')));
|
||||
|
Loading…
Reference in New Issue
Block a user