mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
rework functions that work with Map type
This commit is contained in:
parent
0f01725d8b
commit
7165da1cc6
@ -258,12 +258,11 @@ void ColumnFunction::appendArguments(const ColumnsWithTypeAndName & columns)
|
||||
|
||||
void ColumnFunction::appendArgument(const ColumnWithTypeAndName & column)
|
||||
{
|
||||
const auto & argumnet_types = function->getArgumentTypes();
|
||||
|
||||
const auto & argument_types = function->getArgumentTypes();
|
||||
auto index = captured_columns.size();
|
||||
if (!is_short_circuit_argument && !column.type->equals(*argumnet_types[index]))
|
||||
if (!is_short_circuit_argument && !column.type->equals(*argument_types[index]))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot capture column {} because it has incompatible type: "
|
||||
"got {}, but {} is expected.", argumnet_types.size(), column.type->getName(), argumnet_types[index]->getName());
|
||||
"got {}, but {} is expected.", argument_types.size(), column.type->getName(), argument_types[index]->getName());
|
||||
|
||||
captured_columns.push_back(column);
|
||||
}
|
||||
|
@ -37,7 +37,6 @@ public:
|
||||
template <typename Function>
|
||||
void registerFunction(const std::string & name, Documentation doc = {}, CaseSensitiveness case_sensitiveness = CaseSensitive)
|
||||
{
|
||||
|
||||
if constexpr (std::is_base_of_v<IFunction, Function>)
|
||||
registerFunction(name, &adaptFunctionToOverloadResolver<Function>, std::move(doc), case_sensitiveness);
|
||||
else
|
||||
|
@ -3,6 +3,10 @@ add_headers_and_sources(clickhouse_functions_array .)
|
||||
add_library(clickhouse_functions_array OBJECT ${clickhouse_functions_array_sources} ${clickhouse_functions_array_headers})
|
||||
target_link_libraries(clickhouse_functions_array PRIVATE dbms clickhouse_functions_gatherutils)
|
||||
|
||||
if (TARGET ch_contrib::vectorscan)
|
||||
target_link_libraries(clickhouse_functions_array PRIVATE ch_contrib::vectorscan)
|
||||
endif()
|
||||
|
||||
if (OMIT_HEAVY_DEBUG_SYMBOLS)
|
||||
target_compile_options(clickhouse_functions_array PRIVATE "-g0")
|
||||
endif()
|
||||
|
@ -19,6 +19,7 @@
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
@ -133,6 +134,10 @@ public:
|
||||
|
||||
size_t nested_types_count = (arguments.size() - num_fixed_params - 1) * (is_argument_type_map ? 2 : 1);
|
||||
DataTypes nested_types(nested_types_count);
|
||||
|
||||
size_t num_array_tuple_arguments = 0;
|
||||
size_t tuple_argument_size = 0;
|
||||
|
||||
for (size_t i = 0; i < arguments.size() - 1 - num_fixed_params; ++i)
|
||||
{
|
||||
const auto * array_type = checkAndGetDataType<typename Impl::data_type>(&*arguments[i + 1 + num_fixed_params]);
|
||||
@ -144,6 +149,13 @@ public:
|
||||
getName(),
|
||||
argument_type_name,
|
||||
arguments[i + 1 + num_fixed_params]->getName());
|
||||
|
||||
if (const auto * tuple_type = checkAndGetDataType<DataTypeTuple>(array_type->getNestedType().get()))
|
||||
{
|
||||
++num_array_tuple_arguments;
|
||||
tuple_argument_size = tuple_type->getElements().size();
|
||||
}
|
||||
|
||||
if constexpr (is_argument_type_map)
|
||||
{
|
||||
nested_types[2 * i] = recursiveRemoveLowCardinality(array_type->getKeyType());
|
||||
@ -155,8 +167,31 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
const DataTypeFunction * function_type = checkAndGetDataType<DataTypeFunction>(arguments[0].get());
|
||||
if (!function_type || function_type->getArgumentTypes().size() != nested_types.size())
|
||||
const auto * function_type = checkAndGetDataType<DataTypeFunction>(arguments[0].get());
|
||||
if (!function_type)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"First argument for this overload of {} must be a function with {} arguments, found {} instead",
|
||||
getName(),
|
||||
nested_types.size(),
|
||||
arguments[0]->getName());
|
||||
|
||||
size_t num_function_arguments = function_type->getArgumentTypes().size();
|
||||
if (num_array_tuple_arguments == 1 && tuple_argument_size == num_function_arguments)
|
||||
{
|
||||
assert(nested_types.size() == 1);
|
||||
|
||||
auto argument_type = nested_types[0];
|
||||
const auto & tuple_type = assert_cast<const DataTypeTuple &>(*argument_type);
|
||||
|
||||
nested_types.clear();
|
||||
nested_types.reserve(tuple_argument_size);
|
||||
|
||||
for (const auto & element : tuple_type.getElements())
|
||||
nested_types.push_back(element);
|
||||
}
|
||||
|
||||
if (num_function_arguments != nested_types.size())
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"First argument for this overload of {} must be a function with {} arguments, found {} instead",
|
||||
@ -315,18 +350,20 @@ public:
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function.", getName());
|
||||
|
||||
const auto * column_function = typeid_cast<const ColumnFunction *>(column_with_type_and_name.column.get());
|
||||
|
||||
if (!column_function)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function.", getName());
|
||||
|
||||
ColumnPtr offsets_column;
|
||||
const auto & type_function = assert_cast<const DataTypeFunction &>(*arguments[0].type);
|
||||
size_t num_function_arguments = type_function.getArgumentTypes().size();
|
||||
|
||||
ColumnPtr offsets_column;
|
||||
ColumnPtr column_first_array_ptr;
|
||||
const typename Impl::column_type * column_first_array = nullptr;
|
||||
|
||||
ColumnsWithTypeAndName arrays;
|
||||
arrays.reserve(arguments.size() - 1);
|
||||
arrays.reserve(arguments.size() - 1 - num_fixed_params);
|
||||
|
||||
bool is_single_array_argument = arguments.size() == num_fixed_params + 2;
|
||||
for (size_t i = 1 + num_fixed_params; i < arguments.size(); ++i)
|
||||
{
|
||||
const auto & array_with_type_and_name = arguments[i];
|
||||
@ -367,12 +404,6 @@ public:
|
||||
getName());
|
||||
}
|
||||
|
||||
if (i == 1 + num_fixed_params)
|
||||
{
|
||||
column_first_array_ptr = column_array_ptr;
|
||||
column_first_array = column_array;
|
||||
}
|
||||
|
||||
if constexpr (is_argument_type_map)
|
||||
{
|
||||
arrays.emplace_back(ColumnWithTypeAndName(
|
||||
@ -382,9 +413,35 @@ public:
|
||||
}
|
||||
else
|
||||
{
|
||||
arrays.emplace_back(ColumnWithTypeAndName(column_array->getDataPtr(),
|
||||
recursiveRemoveLowCardinality(array_type->getNestedType()),
|
||||
array_with_type_and_name.name));
|
||||
const auto * column_tuple = checkAndGetColumn<ColumnTuple>(&column_array->getData());
|
||||
if (is_single_array_argument && column_tuple && column_tuple->getColumns().size() == num_function_arguments)
|
||||
{
|
||||
const auto & type_tuple = assert_cast<const DataTypeTuple &>(*array_type->getNestedType());
|
||||
const auto & tuple_names = type_tuple.getElementNames();
|
||||
|
||||
size_t tuple_size = column_tuple->getColumns().size();
|
||||
arrays.reserve(column_tuple->getColumns().size());
|
||||
for (size_t j = 0; j < tuple_size; ++j)
|
||||
{
|
||||
arrays.emplace_back(
|
||||
column_tuple->getColumnPtr(j),
|
||||
recursiveRemoveLowCardinality(type_tuple.getElement(j)),
|
||||
array_with_type_and_name.name + "." + tuple_names[j]);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
arrays.emplace_back(
|
||||
column_array->getDataPtr(),
|
||||
recursiveRemoveLowCardinality(array_type->getNestedType()),
|
||||
array_with_type_and_name.name);
|
||||
}
|
||||
}
|
||||
|
||||
if (i == 1 + num_fixed_params)
|
||||
{
|
||||
column_first_array_ptr = column_array_ptr;
|
||||
column_first_array = column_array;
|
||||
}
|
||||
}
|
||||
|
||||
|
334
src/Functions/array/FunctionsMapMiscellaneous.cpp
Normal file
334
src/Functions/array/FunctionsMapMiscellaneous.cpp
Normal file
@ -0,0 +1,334 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnFunction.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeFunction.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/like.h>
|
||||
#include <Functions/array/arrayConcat.h>
|
||||
#include <Functions/array/arrayFilter.h>
|
||||
#include <Functions/array/arrayMap.h>
|
||||
#include <Functions/array/arrayExists.h>
|
||||
#include <Functions/array/arraySort.h>
|
||||
#include <Functions/array/arrayIndex.h>
|
||||
#include <Functions/array/arrayExists.h>
|
||||
#include <Functions/identity.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename Impl, typename Adapter, typename Name>
|
||||
class FunctionMapToArrayAdapter : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMapToArrayAdapter>(); }
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return impl.isVariadic(); }
|
||||
size_t getNumberOfArguments() const override { return impl.getNumberOfArguments(); }
|
||||
bool useDefaultImplementationForConstants() const override { return impl.useDefaultImplementationForConstants(); }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return false; }
|
||||
|
||||
void getLambdaArgumentTypes(DataTypes & arguments) const override
|
||||
{
|
||||
auto nested_arguments = arguments;
|
||||
Adapter::extractNestedTypes(arguments);
|
||||
impl.getLambdaArgumentTypes(arguments);
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.empty())
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Function {} requires at least one argument, passed {}", getName(), arguments.size());
|
||||
|
||||
auto nested_arguments = arguments;
|
||||
Adapter::extractNestedTypesAndColumns(nested_arguments);
|
||||
|
||||
constexpr bool impl_has_get_return_type = requires
|
||||
{
|
||||
impl.getReturnTypeImpl(nested_arguments);
|
||||
};
|
||||
|
||||
if constexpr (impl_has_get_return_type)
|
||||
return Adapter::wrapType(impl.getReturnTypeImpl(nested_arguments));
|
||||
else
|
||||
return Adapter::wrapType(dynamic_cast<const IFunction &>(impl).getReturnTypeImpl(nested_arguments));
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
auto nested_arguments = arguments;
|
||||
Adapter::extractNestedTypesAndColumns(nested_arguments);
|
||||
return Adapter::wrapColumn(impl.executeImpl(nested_arguments, Adapter::extractResultType(result_type), input_rows_count));
|
||||
}
|
||||
|
||||
private:
|
||||
Impl impl;
|
||||
};
|
||||
|
||||
|
||||
template <typename Derived, typename Name>
|
||||
struct MapAdapterBase
|
||||
{
|
||||
static void extractNestedTypes(DataTypes & types)
|
||||
{
|
||||
bool has_map_column = false;
|
||||
for (auto & type : types)
|
||||
{
|
||||
if (const auto * type_map = typeid_cast<const DataTypeMap *>(type.get()))
|
||||
{
|
||||
has_map_column = true;
|
||||
type = Derived::extractNestedType(*type_map);
|
||||
}
|
||||
}
|
||||
|
||||
if (!has_map_column)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Function {} requires at least one argument of type Map", Name::name);
|
||||
}
|
||||
|
||||
static void extractNestedTypesAndColumns(ColumnsWithTypeAndName & arguments)
|
||||
{
|
||||
bool has_map_column = false;
|
||||
for (auto & argument : arguments)
|
||||
{
|
||||
if (const auto * type_map = typeid_cast<const DataTypeMap *>(argument.type.get()))
|
||||
{
|
||||
has_map_column = true;
|
||||
argument.type = Derived::extractNestedType(*type_map);
|
||||
|
||||
if (argument.column)
|
||||
{
|
||||
if (const auto * const_map = checkAndGetColumnConstData<ColumnMap>(argument.column.get()))
|
||||
argument.column = ColumnConst::create(Derived::extractNestedColumn(*const_map), argument.column->size());
|
||||
else
|
||||
argument.column = Derived::extractNestedColumn(assert_cast<const ColumnMap &>(*argument.column));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!has_map_column)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Function {} requires at least one argument of type Map", Name::name);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Name, bool returns_map = true>
|
||||
struct MapToNestedAdapter : public MapAdapterBase<MapToNestedAdapter<Name, returns_map>, Name>
|
||||
{
|
||||
using MapAdapterBase<MapToNestedAdapter, Name>::extractNestedTypes;
|
||||
using MapAdapterBase<MapToNestedAdapter, Name>::extractNestedTypesAndColumns;
|
||||
|
||||
static DataTypePtr extractNestedType(const DataTypeMap & type_map)
|
||||
{
|
||||
return type_map.getNestedType();
|
||||
}
|
||||
|
||||
static ColumnPtr extractNestedColumn(const ColumnMap & column_map)
|
||||
{
|
||||
return column_map.getNestedColumnPtr();
|
||||
}
|
||||
|
||||
static DataTypePtr extractResultType(const DataTypePtr & result_type)
|
||||
{
|
||||
if constexpr (returns_map)
|
||||
return assert_cast<const DataTypeMap &>(*result_type).getNestedType();
|
||||
return result_type;
|
||||
}
|
||||
|
||||
static DataTypePtr wrapType(DataTypePtr type)
|
||||
{
|
||||
if constexpr (returns_map)
|
||||
return std::make_shared<DataTypeMap>(std::move(type));
|
||||
return type;
|
||||
}
|
||||
|
||||
static ColumnPtr wrapColumn(ColumnPtr column)
|
||||
{
|
||||
if constexpr (returns_map)
|
||||
return ColumnMap::create(std::move(column));
|
||||
return column;
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Name, size_t position>
|
||||
struct MapToSubcolumnAdapter : public MapAdapterBase<MapToSubcolumnAdapter<Name, position>, Name>
|
||||
{
|
||||
using MapAdapterBase<MapToSubcolumnAdapter, Name>::extractNestedTypes;
|
||||
using MapAdapterBase<MapToSubcolumnAdapter, Name>::extractNestedTypesAndColumns;
|
||||
|
||||
static DataTypePtr extractNestedType(const DataTypeMap & type_map)
|
||||
{
|
||||
const auto & array_type = assert_cast<const DataTypeArray &>(*type_map.getNestedType());
|
||||
const auto & tuple_type = assert_cast<const DataTypeTuple &>(*array_type.getNestedType());
|
||||
return std::make_shared<DataTypeArray>(tuple_type.getElement(position));
|
||||
}
|
||||
|
||||
static ColumnPtr extractNestedColumn(const ColumnMap & column_map)
|
||||
{
|
||||
const auto & array_column = column_map.getNestedColumn();
|
||||
const auto & tuple_column = column_map.getNestedData();
|
||||
return ColumnArray::create(tuple_column.getColumnPtr(position), array_column.getOffsetsPtr());
|
||||
}
|
||||
|
||||
static DataTypePtr extractResultType(const DataTypePtr & result_type) { return result_type; }
|
||||
static DataTypePtr wrapType(DataTypePtr type) { return type; }
|
||||
static ColumnPtr wrapColumn(ColumnPtr column) { return column; }
|
||||
};
|
||||
|
||||
class FunctionMapKeyLike : public IFunction
|
||||
{
|
||||
public:
|
||||
String getName() const override { return "mapKeyLike"; }
|
||||
size_t getNumberOfArguments() const override { return 3; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
DataTypes new_arguments{arguments[1], arguments[0]};
|
||||
return impl.getReturnTypeImpl(new_arguments);
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
ColumnsWithTypeAndName new_arguments{arguments[1], arguments[0]};
|
||||
return impl.executeImpl(new_arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
private:
|
||||
FunctionLike impl;
|
||||
};
|
||||
|
||||
template <typename Name, bool returns_map>
|
||||
struct MapKeyLikeAdapter
|
||||
{
|
||||
static void extractNestedTypes(DataTypes & types)
|
||||
{
|
||||
if (types.size() != 2)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 2",
|
||||
Name::name, types.size());
|
||||
|
||||
const auto * map_type = checkAndGetDataType<DataTypeMap>(types[0].get());
|
||||
if (!map_type)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a Map", Name::name);
|
||||
|
||||
if (!isStringOrFixedString(types[1]))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be String or FixedString", Name::name);
|
||||
|
||||
if (!isStringOrFixedString(map_type->getKeyType()))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Key type of map for function {} must be String or FixedString", Name::name);
|
||||
|
||||
DataTypes argument_types{map_type->getKeyType(), map_type->getValueType()};
|
||||
auto function_type = std::make_shared<DataTypeFunction>(argument_types, std::make_shared<DataTypeUInt8>());
|
||||
|
||||
types = {function_type, types[0]};
|
||||
MapToNestedAdapter<Name, returns_map>::extractNestedTypes(types);
|
||||
}
|
||||
|
||||
static void extractNestedTypesAndColumns(ColumnsWithTypeAndName & arguments)
|
||||
{
|
||||
const auto & map_type = assert_cast<const DataTypeMap &>(*arguments[0].type);
|
||||
const auto & pattern_arg = arguments[1];
|
||||
|
||||
ColumnPtr function_column;
|
||||
|
||||
DataTypePtr return_type = std::make_shared<DataTypeUInt8>();
|
||||
DataTypes argument_types{map_type.getKeyType(), map_type.getValueType()};
|
||||
auto function_type = std::make_shared<DataTypeFunction>(argument_types, return_type);
|
||||
|
||||
if (pattern_arg.column)
|
||||
{
|
||||
DataTypes function_argument_types{pattern_arg.type, map_type.getKeyType(), map_type.getValueType()};
|
||||
|
||||
auto function = std::make_shared<FunctionMapKeyLike>();
|
||||
auto function_base = std::make_shared<FunctionToFunctionBaseAdaptor>(function, function_argument_types, return_type);
|
||||
function_column = ColumnFunction::create(pattern_arg.column->size(), std::move(function_base), ColumnsWithTypeAndName{pattern_arg});
|
||||
}
|
||||
|
||||
ColumnWithTypeAndName function_arg{std::move(function_column), std::move(function_type), "__function_map_key_like"};
|
||||
arguments = {function_arg, arguments[0]};
|
||||
MapToNestedAdapter<Name, returns_map>::extractNestedTypesAndColumns(arguments);
|
||||
}
|
||||
|
||||
static DataTypePtr extractResultType(const DataTypePtr & result_type)
|
||||
{
|
||||
return MapToNestedAdapter<Name, returns_map>::extractResultType(result_type);
|
||||
}
|
||||
|
||||
static DataTypePtr wrapType(DataTypePtr type)
|
||||
{
|
||||
return MapToNestedAdapter<Name, returns_map>::wrapType(std::move(type));
|
||||
}
|
||||
|
||||
static ColumnPtr wrapColumn(ColumnPtr column)
|
||||
{
|
||||
return MapToNestedAdapter<Name, returns_map>::wrapColumn(std::move(column));
|
||||
}
|
||||
};
|
||||
|
||||
struct NameMapConcat { static constexpr auto name = "mapConcat"; };
|
||||
using FunctionMapConcat = FunctionMapToArrayAdapter<FunctionArrayConcat, MapToNestedAdapter<NameMapConcat>, NameMapConcat>;
|
||||
|
||||
struct NameMapKeys { static constexpr auto name = "mapKeys"; };
|
||||
using FunctionMapKeys = FunctionMapToArrayAdapter<FunctionIdentity, MapToSubcolumnAdapter<NameMapKeys, 0>, NameMapKeys>;
|
||||
|
||||
struct NameMapValues { static constexpr auto name = "mapValues"; };
|
||||
using FunctionMapValues = FunctionMapToArrayAdapter<FunctionIdentity, MapToSubcolumnAdapter<NameMapValues, 1>, NameMapValues>;
|
||||
|
||||
struct NameMapContains { static constexpr auto name = "mapContains"; };
|
||||
using FunctionMapContains = FunctionMapToArrayAdapter<FunctionArrayIndex<HasAction, NameMapContains>, MapToSubcolumnAdapter<NameMapKeys, 0>, NameMapContains>;
|
||||
|
||||
struct NameMapFilter { static constexpr auto name = "mapFilter"; };
|
||||
using FunctionMapFilter = FunctionMapToArrayAdapter<FunctionArrayFilter, MapToNestedAdapter<NameMapFilter>, NameMapFilter>;
|
||||
|
||||
struct NameMapApply { static constexpr auto name = "mapApply"; };
|
||||
using FunctionMapApply = FunctionMapToArrayAdapter<FunctionArrayMap, MapToNestedAdapter<NameMapApply>, NameMapApply>;
|
||||
|
||||
struct NameMapExists { static constexpr auto name = "mapExists"; };
|
||||
using FunctionMapExists = FunctionMapToArrayAdapter<FunctionArrayExists, MapToNestedAdapter<NameMapExists, false>, NameMapExists>;
|
||||
|
||||
struct NameMapContainsKeyLike { static constexpr auto name = "mapContainsKeyLike"; };
|
||||
using FunctionMapContainsKeyLike = FunctionMapToArrayAdapter<FunctionArrayExists, MapKeyLikeAdapter<NameMapContainsKeyLike, false>, NameMapContainsKeyLike>;
|
||||
|
||||
struct NameMapExtractKeyLike { static constexpr auto name = "mapExtractKeyLike"; };
|
||||
using FunctionMapExtractKeyLike = FunctionMapToArrayAdapter<FunctionArrayFilter, MapKeyLikeAdapter<NameMapExtractKeyLike, true>, NameMapExtractKeyLike>;
|
||||
|
||||
struct NameMapSort { static constexpr auto name = "mapSort"; };
|
||||
struct NameMapReverseSort { static constexpr auto name = "mapReverseSort"; };
|
||||
struct NameMapPartialSort { static constexpr auto name = "mapPartialSort"; };
|
||||
struct NameMapPartialReverseSort { static constexpr auto name = "mapPartialReverseSort"; };
|
||||
|
||||
using FunctionMapSort = FunctionMapToArrayAdapter<FunctionArraySort, MapToNestedAdapter<NameMapSort>, NameMapSort>;
|
||||
using FunctionMapReverseSort = FunctionMapToArrayAdapter<FunctionArrayReverseSort, MapToNestedAdapter<NameMapReverseSort>, NameMapReverseSort>;
|
||||
using FunctionMapPartialSort = FunctionMapToArrayAdapter<FunctionArrayPartialSort, MapToNestedAdapter<NameMapPartialSort>, NameMapPartialSort>;
|
||||
using FunctionMapPartialReverseSort = FunctionMapToArrayAdapter<FunctionArrayPartialReverseSort, MapToNestedAdapter<NameMapPartialReverseSort>, NameMapPartialReverseSort>;
|
||||
|
||||
REGISTER_FUNCTION(MapMiscellaneous)
|
||||
{
|
||||
factory.registerFunction<FunctionMapConcat>();
|
||||
factory.registerFunction<FunctionMapKeys>();
|
||||
factory.registerFunction<FunctionMapValues>();
|
||||
factory.registerFunction<FunctionMapContains>();
|
||||
factory.registerFunction<FunctionMapFilter>();
|
||||
factory.registerFunction<FunctionMapApply>();
|
||||
factory.registerFunction<FunctionMapExists>();
|
||||
factory.registerFunction<FunctionMapSort>();
|
||||
factory.registerFunction<FunctionMapReverseSort>();
|
||||
factory.registerFunction<FunctionMapPartialSort>();
|
||||
factory.registerFunction<FunctionMapPartialReverseSort>();
|
||||
factory.registerFunction<FunctionMapContainsKeyLike>();
|
||||
factory.registerFunction<FunctionMapExtractKeyLike>();
|
||||
}
|
||||
|
||||
}
|
@ -2,22 +2,11 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <memory>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include "array/arrayIndex.h"
|
||||
#include "Functions/like.h"
|
||||
#include "Functions/FunctionsStringSearch.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -228,372 +217,6 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
struct NameMapContains { static constexpr auto name = "mapContains"; };
|
||||
|
||||
class FunctionMapContains : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = NameMapContains::name;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMapContains>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return NameMapContains::name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return impl.getNumberOfArguments(); }
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & arguments) const override
|
||||
{
|
||||
return impl.isSuitableForShortCircuitArgumentsExecution(arguments);
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
return impl.getReturnTypeImpl(arguments);
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
return impl.executeImpl(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
private:
|
||||
FunctionArrayIndex<HasAction, NameMapContains> impl;
|
||||
};
|
||||
|
||||
|
||||
class FunctionMapKeys : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "mapKeys";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMapKeys>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 1",
|
||||
getName(), arguments.size());
|
||||
|
||||
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
|
||||
|
||||
if (!map_type)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a map", getName());
|
||||
|
||||
auto key_type = map_type->getKeyType();
|
||||
|
||||
return std::make_shared<DataTypeArray>(key_type);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
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(ContextPtr) { return std::make_shared<FunctionMapValues>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 1",
|
||||
getName(), arguments.size());
|
||||
|
||||
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
|
||||
|
||||
if (!map_type)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a map", getName());
|
||||
|
||||
auto value_type = map_type->getValueType();
|
||||
|
||||
return std::make_shared<DataTypeArray>(value_type);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
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());
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionMapContainsKeyLike : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "mapContainsKeyLike";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionMapContainsKeyLike>(); }
|
||||
String getName() const override { return name; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*info*/) const override { return true; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
bool is_const = isColumnConst(*arguments[0].column);
|
||||
const ColumnMap * col_map = is_const ? checkAndGetColumnConstData<ColumnMap>(arguments[0].column.get())
|
||||
: checkAndGetColumn<ColumnMap>(arguments[0].column.get());
|
||||
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
|
||||
if (!col_map || !map_type)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a map", getName());
|
||||
|
||||
auto col_res = ColumnVector<UInt8>::create();
|
||||
typename ColumnVector<UInt8>::Container & vec_res = col_res->getData();
|
||||
|
||||
if (input_rows_count == 0)
|
||||
return col_res;
|
||||
|
||||
vec_res.resize(input_rows_count);
|
||||
|
||||
const auto & column_array = typeid_cast<const ColumnArray &>(col_map->getNestedColumn());
|
||||
const auto & column_tuple = typeid_cast<const ColumnTuple &>(column_array.getData());
|
||||
|
||||
const ColumnString * column_string = checkAndGetColumn<ColumnString>(column_tuple.getColumn(0));
|
||||
const ColumnFixedString * column_fixed_string = checkAndGetColumn<ColumnFixedString>(column_tuple.getColumn(0));
|
||||
|
||||
FunctionLike func_like;
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
size_t element_start_row = row != 0 ? column_array.getOffsets()[row-1] : 0;
|
||||
size_t elem_size = column_array.getOffsets()[row]- element_start_row;
|
||||
|
||||
ColumnPtr sub_map_column;
|
||||
DataTypePtr data_type;
|
||||
|
||||
//The keys of one row map will be processed as a single ColumnString
|
||||
if (column_string)
|
||||
{
|
||||
sub_map_column = column_string->cut(element_start_row, elem_size);
|
||||
data_type = std::make_shared<DataTypeString>();
|
||||
}
|
||||
else
|
||||
{
|
||||
sub_map_column = column_fixed_string->cut(element_start_row, elem_size);
|
||||
data_type = std::make_shared<DataTypeFixedString>(checkAndGetColumn<ColumnFixedString>(sub_map_column.get())->getN());
|
||||
}
|
||||
|
||||
size_t col_key_size = sub_map_column->size();
|
||||
auto column = is_const ? ColumnConst::create(std::move(sub_map_column), std::move(col_key_size)) : std::move(sub_map_column);
|
||||
|
||||
ColumnsWithTypeAndName new_arguments =
|
||||
{
|
||||
{
|
||||
column,
|
||||
data_type,
|
||||
""
|
||||
},
|
||||
arguments[1]
|
||||
};
|
||||
|
||||
auto res = func_like.executeImpl(new_arguments, result_type, input_rows_count);
|
||||
const auto & container = checkAndGetColumn<ColumnUInt8>(res.get())->getData();
|
||||
|
||||
const auto it = std::find_if(container.begin(), container.end(), [](int element){ return element == 1; }); // NOLINT
|
||||
vec_res[row] = it == container.end() ? 0 : 1;
|
||||
}
|
||||
|
||||
return col_res;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 2",
|
||||
getName(), arguments.size());
|
||||
|
||||
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
|
||||
const DataTypeString * pattern_type = checkAndGetDataType<DataTypeString>(arguments[1].type.get());
|
||||
|
||||
if (!map_type)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a Map", getName());
|
||||
if (!pattern_type)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be String", getName());
|
||||
|
||||
if (!isStringOrFixedString(map_type->getKeyType()))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Key type of map for function {} must be `String` or `FixedString`", getName());
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
};
|
||||
|
||||
class FunctionExtractKeyLike : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "mapExtractKeyLike";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionExtractKeyLike>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*info*/) const override { return true; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 2",
|
||||
getName(), arguments.size());
|
||||
|
||||
|
||||
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
|
||||
|
||||
if (!map_type)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a map", getName());
|
||||
|
||||
|
||||
auto key_type = map_type->getKeyType();
|
||||
|
||||
WhichDataType which(key_type);
|
||||
|
||||
if (!which.isStringOrFixedString())
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}only support the map with String or FixedString key",
|
||||
getName());
|
||||
|
||||
if (!isStringOrFixedString(arguments[1].type))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument passed to function {} must be String or FixedString", getName());
|
||||
|
||||
return std::make_shared<DataTypeMap>(map_type->getKeyType(), map_type->getValueType());
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
bool is_const = isColumnConst(*arguments[0].column);
|
||||
const ColumnMap * col_map = typeid_cast<const ColumnMap *>(arguments[0].column.get());
|
||||
|
||||
//It may not be necessary to check this condition, cause it will be checked in getReturnTypeImpl function
|
||||
if (!col_map)
|
||||
return nullptr;
|
||||
|
||||
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
|
||||
auto key_type = map_type->getKeyType();
|
||||
auto value_type = map_type->getValueType();
|
||||
|
||||
const auto & nested_column = col_map->getNestedColumn();
|
||||
const auto & keys_column = col_map->getNestedData().getColumn(0);
|
||||
const auto & values_column = col_map->getNestedData().getColumn(1);
|
||||
const ColumnString * keys_string_column = checkAndGetColumn<ColumnString>(keys_column);
|
||||
const ColumnFixedString * keys_fixed_string_column = checkAndGetColumn<ColumnFixedString>(keys_column);
|
||||
|
||||
FunctionLike func_like;
|
||||
|
||||
//create result data
|
||||
MutableColumnPtr keys_data = key_type->createColumn();
|
||||
MutableColumnPtr values_data = value_type->createColumn();
|
||||
MutableColumnPtr offsets = DataTypeNumber<IColumn::Offset>().createColumn();
|
||||
|
||||
IColumn::Offset current_offset = 0;
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
size_t element_start_row = row != 0 ? nested_column.getOffsets()[row-1] : 0;
|
||||
size_t element_size = nested_column.getOffsets()[row]- element_start_row;
|
||||
|
||||
ColumnsWithTypeAndName new_arguments;
|
||||
ColumnPtr sub_map_column;
|
||||
DataTypePtr data_type;
|
||||
|
||||
if (keys_string_column)
|
||||
{
|
||||
sub_map_column = keys_string_column->cut(element_start_row, element_size);
|
||||
data_type = std::make_shared<DataTypeString>();
|
||||
}
|
||||
else
|
||||
{
|
||||
sub_map_column = keys_fixed_string_column->cut(element_start_row, element_size);
|
||||
data_type =std::make_shared<DataTypeFixedString>(checkAndGetColumn<ColumnFixedString>(sub_map_column.get())->getN());
|
||||
}
|
||||
|
||||
size_t col_key_size = sub_map_column->size();
|
||||
auto column = is_const? ColumnConst::create(std::move(sub_map_column), std::move(col_key_size)) : std::move(sub_map_column);
|
||||
|
||||
new_arguments = {
|
||||
{
|
||||
column,
|
||||
data_type,
|
||||
""
|
||||
},
|
||||
arguments[1]
|
||||
};
|
||||
|
||||
auto res = func_like.executeImpl(new_arguments, result_type, input_rows_count);
|
||||
const auto & container = checkAndGetColumn<ColumnUInt8>(res.get())->getData();
|
||||
|
||||
for (size_t row_num = 0; row_num < element_size; ++row_num)
|
||||
{
|
||||
if (container[row_num] == 1)
|
||||
{
|
||||
auto key_ref = keys_string_column ?
|
||||
keys_string_column->getDataAt(element_start_row + row_num) :
|
||||
keys_fixed_string_column->getDataAt(element_start_row + row_num);
|
||||
auto value_ref = values_column.getDataAt(element_start_row + row_num);
|
||||
|
||||
keys_data->insertData(key_ref.data, key_ref.size);
|
||||
values_data->insertData(value_ref.data, value_ref.size);
|
||||
current_offset += 1;
|
||||
}
|
||||
}
|
||||
|
||||
offsets->insert(current_offset);
|
||||
}
|
||||
|
||||
auto result_nested_column = ColumnArray::create(
|
||||
ColumnTuple::create(Columns{std::move(keys_data), std::move(values_data)}),
|
||||
std::move(offsets));
|
||||
|
||||
return ColumnMap::create(result_nested_column);
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionMapUpdate : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -723,15 +346,9 @@ public:
|
||||
REGISTER_FUNCTION(Map)
|
||||
{
|
||||
factory.registerFunction<FunctionMap>();
|
||||
factory.registerFunction<FunctionMapContains>();
|
||||
factory.registerFunction<FunctionMapKeys>();
|
||||
factory.registerFunction<FunctionMapValues>();
|
||||
factory.registerFunction<FunctionMapContainsKeyLike>();
|
||||
factory.registerFunction<FunctionExtractKeyLike>();
|
||||
factory.registerFunction<FunctionMapUpdate>();
|
||||
factory.registerFunction<FunctionMapFromArrays>();
|
||||
factory.registerAlias("MAP_FROM_ARRAYS", "mapFromArrays");
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,144 +0,0 @@
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/array/FunctionArrayMapped.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
/** Higher-order functions for map.
|
||||
* These functions optionally apply a map by lambda function,
|
||||
* and return some result based on that transformation.
|
||||
*/
|
||||
|
||||
|
||||
/** mapFilter((k, v) -> predicate, map) - leave in the map only the kv elements for which the expression is true.
|
||||
*/
|
||||
struct MapFilterImpl
|
||||
{
|
||||
using data_type = DataTypeMap;
|
||||
using column_type = ColumnMap;
|
||||
|
||||
static constexpr auto name = "mapFilter";
|
||||
|
||||
static bool needBoolean() { return true; }
|
||||
static bool needExpression() { return true; }
|
||||
static bool needOneArray() { return true; }
|
||||
|
||||
static DataTypePtr getReturnType(const DataTypePtr & /*expression_return*/, const DataTypes & elems)
|
||||
{
|
||||
return std::make_shared<DataTypeMap>(elems);
|
||||
}
|
||||
|
||||
/// If there are several arrays, the first one is passed here.
|
||||
static ColumnPtr execute(const ColumnMap & map_column, ColumnPtr mapped)
|
||||
{
|
||||
const ColumnUInt8 * column_filter = typeid_cast<const ColumnUInt8 *>(&*mapped);
|
||||
|
||||
if (!column_filter)
|
||||
{
|
||||
const auto * column_filter_const = checkAndGetColumnConst<ColumnUInt8>(&*mapped);
|
||||
|
||||
if (!column_filter_const)
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Unexpected type of filter column");
|
||||
|
||||
if (column_filter_const->getValue<UInt8>())
|
||||
return map_column.clone();
|
||||
else
|
||||
{
|
||||
const auto * column_array = typeid_cast<const ColumnArray *>(map_column.getNestedColumnPtr().get());
|
||||
const auto * column_tuple = typeid_cast<const ColumnTuple *>(column_array->getDataPtr().get());
|
||||
ColumnPtr keys = column_tuple->getColumnPtr(0)->cloneEmpty();
|
||||
ColumnPtr values = column_tuple->getColumnPtr(1)->cloneEmpty();
|
||||
return ColumnMap::create(keys, values, ColumnArray::ColumnOffsets::create(map_column.size(), 0));
|
||||
}
|
||||
}
|
||||
|
||||
const IColumn::Filter & filter = column_filter->getData();
|
||||
ColumnPtr filtered = map_column.getNestedColumn().getData().filter(filter, -1);
|
||||
|
||||
const IColumn::Offsets & in_offsets = map_column.getNestedColumn().getOffsets();
|
||||
auto column_offsets = ColumnArray::ColumnOffsets::create(in_offsets.size());
|
||||
IColumn::Offsets & out_offsets = column_offsets->getData();
|
||||
|
||||
size_t in_pos = 0;
|
||||
size_t out_pos = 0;
|
||||
for (size_t i = 0; i < in_offsets.size(); ++i)
|
||||
{
|
||||
for (; in_pos < in_offsets[i]; ++in_pos)
|
||||
{
|
||||
if (filter[in_pos])
|
||||
++out_pos;
|
||||
}
|
||||
out_offsets[i] = out_pos;
|
||||
}
|
||||
|
||||
return ColumnMap::create(ColumnArray::create(filtered, std::move(column_offsets)));
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** mapApply((k,v) -> expression, map) - apply the expression to the map.
|
||||
*/
|
||||
struct MapApplyImpl
|
||||
{
|
||||
using data_type = DataTypeMap;
|
||||
using column_type = ColumnMap;
|
||||
|
||||
static constexpr auto name = "mapApply";
|
||||
|
||||
/// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean.
|
||||
static bool needBoolean() { return false; }
|
||||
static bool needExpression() { return true; }
|
||||
static bool needOneArray() { return true; }
|
||||
|
||||
static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & /*elems*/)
|
||||
{
|
||||
const auto * tuple_types = typeid_cast<const DataTypeTuple *>(expression_return.get());
|
||||
if (!tuple_types)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Expected return type is tuple, got {}", expression_return->getName());
|
||||
if (tuple_types->getElements().size() != 2)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Expected 2 columns as map's key and value, but found {}", tuple_types->getElements().size());
|
||||
|
||||
return std::make_shared<DataTypeMap>(tuple_types->getElements());
|
||||
}
|
||||
|
||||
static ColumnPtr execute(const ColumnMap & map, ColumnPtr mapped)
|
||||
{
|
||||
const auto * column_tuple = checkAndGetColumn<ColumnTuple>(mapped.get());
|
||||
if (!column_tuple)
|
||||
{
|
||||
const ColumnConst * column_const_tuple = checkAndGetColumnConst<ColumnTuple>(mapped.get());
|
||||
if (!column_const_tuple)
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected tuple column, found {}", mapped->getName());
|
||||
auto cols = convertConstTupleToConstantElements(*column_const_tuple);
|
||||
return ColumnMap::create(cols[0]->convertToFullColumnIfConst(), cols[1]->convertToFullColumnIfConst(), map.getNestedColumn().getOffsetsPtr());
|
||||
}
|
||||
|
||||
return ColumnMap::create(column_tuple->getColumnPtr(0), column_tuple->getColumnPtr(1),
|
||||
map.getNestedColumn().getOffsetsPtr());
|
||||
}
|
||||
};
|
||||
|
||||
REGISTER_FUNCTION(MapApply)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayMapped<MapFilterImpl, MapFilterImpl>>();
|
||||
factory.registerFunction<FunctionArrayMapped<MapApplyImpl, MapApplyImpl>>();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -4,3 +4,7 @@
|
||||
4 {'4-K1':'4-V1','4-K2':'4-V2'}
|
||||
5 {'5-K1':'5-V1','5-K2':'5-V2'}
|
||||
6 {'6-K1':'6-V1','6-K2':'6-V2'}
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
|
@ -10,3 +10,8 @@ SELECT id, map FROM map_containsKeyLike_test WHERE mapContainsKeyLike(map, '1-%'
|
||||
SELECT id, map FROM map_containsKeyLike_test WHERE mapContainsKeyLike(map, '3-%') = 0 order by id;
|
||||
|
||||
DROP TABLE map_containsKeyLike_test;
|
||||
|
||||
SELECT mapContainsKeyLike(map('aa', 1, 'bb', 2), 'a%');
|
||||
SELECT mapContainsKeyLike(map('aa', 1, 'bb', 2), materialize('a%'));
|
||||
SELECT mapContainsKeyLike(materialize(map('aa', 1, 'bb', 2)), 'a%');
|
||||
SELECT mapContainsKeyLike(materialize(map('aa', 1, 'bb', 2)), materialize('a%'));
|
||||
|
@ -21,3 +21,7 @@ The results of query: SELECT id, mapExtractKeyLike(map, \'5-K1\') FROM map_extra
|
||||
4 {}
|
||||
5 {'5-K1':'5-V1'}
|
||||
6 {}
|
||||
{'aa':1}
|
||||
{'aa':1}
|
||||
{'aa':1}
|
||||
{'aa':1}
|
||||
|
@ -7,7 +7,7 @@ INSERT INTO map_extractKeyLike_test VALUES (3, {'P1-K1':'3-V1','P2-K2':'3-V2'}),
|
||||
INSERT INTO map_extractKeyLike_test VALUES (5, {'5-K1':'5-V1','5-K2':'5-V2'}),(6, {'P3-K1':'6-V1','P4-K2':'6-V2'});
|
||||
|
||||
SELECT 'The data of table:';
|
||||
SELECT * FROM map_extractKeyLike_test ORDER BY id;
|
||||
SELECT * FROM map_extractKeyLike_test ORDER BY id;
|
||||
|
||||
SELECT '';
|
||||
|
||||
@ -20,3 +20,8 @@ SELECT 'The results of query: SELECT id, mapExtractKeyLike(map, \'5-K1\') FROM m
|
||||
SELECT id, mapExtractKeyLike(map, '5-K1') FROM map_extractKeyLike_test ORDER BY id;
|
||||
|
||||
DROP TABLE map_extractKeyLike_test;
|
||||
|
||||
SELECT mapExtractKeyLike(map('aa', 1, 'bb', 2), 'a%');
|
||||
SELECT mapExtractKeyLike(map('aa', 1, 'bb', 2), materialize('a%'));
|
||||
SELECT mapExtractKeyLike(materialize(map('aa', 1, 'bb', 2)), 'a%');
|
||||
SELECT mapExtractKeyLike(materialize(map('aa', 1, 'bb', 2)), materialize('a%'));
|
||||
|
@ -7,7 +7,7 @@ SELECT mapFilter((k, v) -> k like '%3' and v > 102, col) FROM table_map ORDER BY
|
||||
SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC;
|
||||
SELECT mapApply((k, v) -> (k, v + 1), col) FROM table_map ORDER BY id;
|
||||
SELECT mapFilter((k, v) -> 0, col) from table_map;
|
||||
SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError BAD_ARGUMENTS }
|
||||
|
||||
SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0));
|
||||
SELECT mapApply((x, y) -> (x, x + 1), map(1, 0, 2, 0));
|
||||
@ -16,22 +16,22 @@ SELECT mapApply((x, y) -> ('x', 'y'), map(1, 0, 2, 0));
|
||||
SELECT mapApply((x, y) -> ('x', 'y'), materialize(map(1, 0, 2, 0)));
|
||||
|
||||
SELECT mapApply(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT mapApply((x, y) -> (x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT mapApply((x, y) -> ('x'), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT mapApply((x) -> (x, x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT mapApply((x, y) -> (x, 1, 2), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT mapApply((x, y) -> (x, x + 1)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT mapApply((x, y) -> (x), map(1, 0, 2, 0)); -- { serverError BAD_ARGUMENTS }
|
||||
SELECT mapApply((x, y) -> ('x'), map(1, 0, 2, 0)); -- { serverError BAD_ARGUMENTS }
|
||||
SELECT mapApply((x) -> (x, x), map(1, 0, 2, 0)); -- { serverError BAD_ARGUMENTS }
|
||||
SELECT mapApply((x, y) -> (x, 1, 2), map(1, 0, 2, 0)); -- { serverError BAD_ARGUMENTS }
|
||||
SELECT mapApply((x, y) -> (x, x + 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT mapApply(map(1, 0, 2, 0), (x, y) -> (x, x + 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT mapApply((x, y) -> (x, x+1), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT mapApply((x, y) -> (x, x+1), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
|
||||
SELECT mapFilter(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT mapFilter((x, y) -> (toInt32(x)), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT mapFilter((x, y) -> ('x'), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT mapFilter((x) -> (x, x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT mapFilter((x, y) -> (x, 1, 2), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT mapFilter((x, y) -> (x, x + 1)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT mapFilter((x, y) -> (x, x + 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT mapFilter(map(1, 0, 2, 0), (x, y) -> (x > 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT mapFilter((x, y) -> (x, x + 1), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT mapFilter((x, y) -> (x, x + 1), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
|
||||
SELECT mapUpdate(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
|
Loading…
Reference in New Issue
Block a user