2022-02-17 14:56:29 +00:00
|
|
|
#include <Columns/ColumnMap.h>
|
2022-01-17 08:34:44 +00:00
|
|
|
#include <Columns/ColumnTuple.h>
|
2022-02-17 14:56:29 +00:00
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <DataTypes/DataTypeTuple.h>
|
2022-01-17 08:34:44 +00:00
|
|
|
#include <Functions/FunctionFactory.h>
|
|
|
|
#include <Functions/FunctionHelpers.h>
|
2022-02-17 14:56:29 +00:00
|
|
|
#include <Functions/array/FunctionArrayMapped.h>
|
2022-01-17 08:34:44 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2022-02-17 14:56:29 +00:00
|
|
|
|
2022-01-17 08:34:44 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int ILLEGAL_COLUMN;
|
2022-02-17 15:40:26 +00:00
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
2022-02-16 03:28:21 +00:00
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2022-01-17 08:34:44 +00:00
|
|
|
}
|
|
|
|
|
2022-02-17 14:56:29 +00:00
|
|
|
/** 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.
|
2022-01-17 08:34:44 +00:00
|
|
|
*/
|
|
|
|
struct MapFilterImpl
|
|
|
|
{
|
2022-02-17 14:56:29 +00:00
|
|
|
using data_type = DataTypeMap;
|
|
|
|
using column_type = ColumnMap;
|
|
|
|
|
2022-02-17 15:40:26 +00:00
|
|
|
static constexpr auto name = "mapFilter";
|
|
|
|
|
2022-01-17 08:34:44 +00:00
|
|
|
static bool needBoolean() { return true; }
|
2022-02-17 14:56:29 +00:00
|
|
|
static bool needExpression() { return true; }
|
2022-02-23 09:22:40 +00:00
|
|
|
static bool needOneArray() { return true; }
|
2022-01-17 08:34:44 +00:00
|
|
|
|
|
|
|
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("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
|
|
|
if (column_filter_const->getValue<UInt8>())
|
|
|
|
return map_column.clone();
|
|
|
|
else
|
2022-01-24 10:33:13 +00:00
|
|
|
{
|
|
|
|
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));
|
|
|
|
}
|
2022-01-17 08:34:44 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
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)));
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
|
2022-02-15 09:59:44 +00:00
|
|
|
/** mapApply((k,v) -> expression, map) - apply the expression to the map.
|
2022-01-17 08:34:44 +00:00
|
|
|
*/
|
2022-02-15 09:59:44 +00:00
|
|
|
struct MapApplyImpl
|
2022-01-17 08:34:44 +00:00
|
|
|
{
|
2022-02-17 14:56:29 +00:00
|
|
|
using data_type = DataTypeMap;
|
|
|
|
using column_type = ColumnMap;
|
|
|
|
|
2022-02-17 15:40:26 +00:00
|
|
|
static constexpr auto name = "mapApply";
|
|
|
|
|
2022-01-17 08:34:44 +00:00
|
|
|
/// 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; }
|
2022-02-17 14:56:29 +00:00
|
|
|
static bool needExpression() { return true; }
|
2022-02-23 09:22:40 +00:00
|
|
|
static bool needOneArray() { return true; }
|
2022-01-17 08:34:44 +00:00
|
|
|
|
2022-02-16 03:28:21 +00:00
|
|
|
static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & /*elems*/)
|
2022-01-17 08:34:44 +00:00
|
|
|
{
|
2022-02-17 15:40:26 +00:00
|
|
|
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());
|
2022-01-17 08:34:44 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
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)
|
2022-02-17 15:40:26 +00:00
|
|
|
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());
|
2022-01-17 08:34:44 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return ColumnMap::create(column_tuple->getColumnPtr(0), column_tuple->getColumnPtr(1),
|
|
|
|
map.getNestedColumn().getOffsetsPtr());
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2022-02-15 09:59:44 +00:00
|
|
|
void registerFunctionMapApply(FunctionFactory & factory)
|
2022-01-17 08:34:44 +00:00
|
|
|
{
|
2022-02-17 15:40:26 +00:00
|
|
|
factory.registerFunction<FunctionArrayMapped<MapFilterImpl, MapFilterImpl>>();
|
|
|
|
factory.registerFunction<FunctionArrayMapped<MapApplyImpl, MapApplyImpl>>();
|
2022-01-17 08:34:44 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|