Merge pull request #34318 from kitaisreal/map-populate-series-refactoring

Function mapPopulateSeries refactoring
This commit is contained in:
Maksim Kita 2022-02-05 12:51:02 +01:00 committed by GitHub
commit eff16baaf3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 568 additions and 344 deletions

View File

@ -1,15 +1,20 @@
#include <base/sort.h>
#include <Core/ColumnWithTypeAndName.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeMap.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include "Core/ColumnWithTypeAndName.h"
#include "DataTypes/DataTypeMap.h"
#include "DataTypes/IDataType.h"
#include <Interpreters/castColumn.h>
namespace DB
{
@ -19,6 +24,8 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_LARGE_ARRAY_SIZE;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
class FunctionMapPopulateSeries : public IFunction
@ -35,415 +42,458 @@ private:
bool useDefaultImplementationForConstants() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
void checkTypes(const DataTypePtr & key_type, const DataTypePtr max_key_type) const
void checkTypes(const DataTypePtr & key_type, const DataTypePtr & value_type, const DataTypePtr & max_key_type) const
{
WhichDataType which_key(key_type);
if (!(which_key.isInt() || which_key.isUInt()))
WhichDataType key_data_type(key_type);
WhichDataType value_data_type(value_type);
if (!(key_data_type.isInt() || key_data_type.isUInt()))
{
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Keys for {} function should be of integer type (signed or unsigned)", getName());
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Function {} key argument should be of signed or unsigned integer type. Actual type {}",
getName(),
key_type->getName());
}
if (max_key_type)
if (!(value_data_type.isInt() || value_data_type.isUInt()))
{
WhichDataType which_max_key(max_key_type);
if (which_max_key.isNullable())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Max key argument in arguments of function " + getName() + " can not be Nullable");
if (key_type->getTypeId() != max_key_type->getTypeId())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Max key type in {} should be same as keys type", getName());
}
}
DataTypePtr getReturnTypeForTuple(const DataTypes & arguments) const
{
if (arguments.size() < 2)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} accepts at least two arrays for key and value", getName());
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Function {} key argument should be of signed or unsigned integer type. Actual type {}",
getName(),
key_type->getName());
}
if (arguments.size() > 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Too many arguments in {} call", getName());
if (!max_key_type)
return;
const DataTypeArray * key_array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
const DataTypeArray * val_array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get());
WhichDataType max_key_data_type(max_key_type);
if (!key_array_type || !val_array_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {} accepts two arrays for key and value", getName());
if (max_key_data_type.isNullable())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Function {} max key argument can not be Nullable. Actual type {}",
getName(),
max_key_type->getName());
const auto & key_type = key_array_type->getNestedType();
if (arguments.size() == 3)
this->checkTypes(key_type, arguments[2]);
else
this->checkTypes(key_type, nullptr);
return std::make_shared<DataTypeTuple>(DataTypes{arguments[0], arguments[1]});
}
DataTypePtr getReturnTypeForMap(const DataTypes & arguments) const
{
const auto * map = assert_cast<const DataTypeMap *>(arguments[0].get());
if (arguments.size() == 1)
this->checkTypes(map->getKeyType(), nullptr);
else if (arguments.size() == 2)
this->checkTypes(map->getKeyType(), arguments[1]);
else
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Too many arguments in {} call", getName());
return std::make_shared<DataTypeMap>(map->getKeyType(), map->getValueType());
if (!(max_key_data_type.isInt() || max_key_data_type.isUInt()))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Function {} max key should be of signed or unsigned integer type. Actual type {}.",
getName(),
key_type->getName(),
max_key_type->getName());
}
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, getName() + " accepts at least one map or two arrays");
if (arguments.empty() || arguments.size() > 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Function {} accepts at least one map or two arrays arguments, and optional max key argument",
getName());
if (arguments[0]->getTypeId() == TypeIndex::Array)
return getReturnTypeForTuple(arguments);
else if (arguments[0]->getTypeId() == TypeIndex::Map)
return getReturnTypeForMap(arguments);
WhichDataType key_argument_data_type(arguments[0]);
DataTypePtr key_argument_series_type;
DataTypePtr value_argument_series_type;
size_t max_key_argument_index = 0;
if (key_argument_data_type.isArray())
{
DataTypePtr value_type;
if (1 < arguments.size())
value_type = arguments[1];
if (arguments.size() < 2 || (value_type && !isArray(value_type)))
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Function {} if array argument is passed as key, additional array argument as value must be passed",
getName());
const auto & key_array_type = assert_cast<const DataTypeArray &>(*arguments[0]);
const auto & value_array_type = assert_cast<const DataTypeArray &>(*value_type);
key_argument_series_type = key_array_type.getNestedType();
value_argument_series_type = value_array_type.getNestedType();
max_key_argument_index = 2;
}
else if (key_argument_data_type.isMap())
{
const auto & map_data_type = assert_cast<const DataTypeMap &>(*arguments[0]);
key_argument_series_type = map_data_type.getKeyType();
value_argument_series_type = map_data_type.getValueType();
max_key_argument_index = 1;
}
else
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Function {} only accepts one map or arrays, but got {}",
getName(),
arguments[0]->getName());
DataTypePtr max_key_argument_type;
if (max_key_argument_index < arguments.size())
max_key_argument_type = arguments[max_key_argument_index];
checkTypes(key_argument_series_type, value_argument_series_type, max_key_argument_type);
if (key_argument_data_type.isArray())
return std::make_shared<DataTypeTuple>(DataTypes{arguments[0], arguments[1]});
else
return arguments[0];
}
// Struct holds input and output columns references,
// Both arrays and maps have similar columns to work with but extracted differently
template <typename KeyType, typename ValType>
struct ColumnsInOut
template <typename KeyType, typename ValueType>
void executeImplTyped(
const ColumnPtr & key_column,
const ColumnPtr & value_column,
const ColumnPtr & offsets_column,
const ColumnPtr & max_key_column,
MutableColumnPtr result_key_column,
MutableColumnPtr result_value_column,
MutableColumnPtr result_offset_column) const
{
// inputs
const PaddedPODArray<KeyType> & in_keys_data;
const PaddedPODArray<ValType> & in_vals_data;
const IColumn::Offsets & in_key_offsets;
const IColumn::Offsets & in_val_offsets;
size_t row_count;
bool key_is_const;
bool val_is_const;
const auto & key_column_typed = assert_cast<const ColumnVector<KeyType> &>(*key_column);
const auto & key_column_data = key_column_typed.getData();
// outputs
PaddedPODArray<KeyType> & out_keys_data;
PaddedPODArray<ValType> & out_vals_data;
const auto & offsets_column_typed = assert_cast<const ColumnVector<ColumnArray::Offset> &>(*offsets_column);
const auto & offsets = offsets_column_typed.getData();
IColumn::Offsets & out_keys_offsets;
// with map argument this field will not be used
IColumn::Offsets * out_vals_offsets;
};
const auto & value_column_typed = assert_cast<const ColumnVector<ValueType> &>(*value_column);
const auto & value_column_data = value_column_typed.getData();
template <typename KeyType, typename ValType>
ColumnsInOut<KeyType, ValType> getInOutDataFromArrays(MutableColumnPtr & res_column, ColumnPtr * arg_columns) const
{
auto * out_tuple = assert_cast<ColumnTuple *>(res_column.get());
auto & out_keys_array = assert_cast<ColumnArray &>(out_tuple->getColumn(0));
auto & out_vals_array = assert_cast<ColumnArray &>(out_tuple->getColumn(1));
auto & result_key_column_typed = assert_cast<ColumnVector<KeyType> &>(*result_key_column);
auto & result_key_data = result_key_column_typed.getData();
const auto * key_column = arg_columns[0].get();
const auto * in_keys_array = checkAndGetColumn<ColumnArray>(key_column);
auto & result_value_column_typed = assert_cast<ColumnVector<ValueType> &>(*result_value_column);
auto & result_value_data = result_value_column_typed.getData();
bool key_is_const = false, val_is_const = false;
auto & result_offsets_column_typed = assert_cast<ColumnVector<ColumnArray::Offset> &>(*result_offset_column);
auto & result_offsets_data = result_offsets_column_typed.getData();
if (!in_keys_array)
const PaddedPODArray<KeyType> * max_key_data = max_key_column ? &assert_cast<const ColumnVector<KeyType> &>(*max_key_column).getData() : nullptr;
PaddedPODArray<std::pair<KeyType, ValueType>> sorted_keys_values;
size_t key_offsets_size = offsets.size();
result_key_data.reserve(key_offsets_size);
result_value_data.reserve(key_offsets_size);
for (size_t offset_index = 0; offset_index < key_offsets_size; ++offset_index)
{
const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(key_column);
if (!const_array)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN, "Expected array column in function {}, found {}", getName(), key_column->getName());
size_t start_offset = offsets[offset_index - 1];
size_t end_offset = offsets[offset_index];
in_keys_array = checkAndGetColumn<ColumnArray>(const_array->getDataColumnPtr().get());
key_is_const = true;
}
sorted_keys_values.clear();
const auto * val_column = arg_columns[1].get();
const auto * in_values_array = checkAndGetColumn<ColumnArray>(val_column);
if (!in_values_array)
{
const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(val_column);
if (!const_array)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN, "Expected array column in function {}, found {}", getName(), val_column->getName());
for (; start_offset < end_offset; ++start_offset)
sorted_keys_values.emplace_back(key_column_data[start_offset], value_column_data[start_offset]);
in_values_array = checkAndGetColumn<ColumnArray>(const_array->getDataColumnPtr().get());
val_is_const = true;
}
if (!in_keys_array || !in_values_array)
/* something went wrong */
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns in arguments of function " + getName());
const auto & in_keys_data = assert_cast<const ColumnVector<KeyType> &>(in_keys_array->getData()).getData();
const auto & in_values_data = assert_cast<const ColumnVector<ValType> &>(in_values_array->getData()).getData();
const auto & in_keys_offsets = in_keys_array->getOffsets();
const auto & in_vals_offsets = in_values_array->getOffsets();
auto & out_keys_data = assert_cast<ColumnVector<KeyType> &>(out_keys_array.getData()).getData();
auto & out_vals_data = assert_cast<ColumnVector<ValType> &>(out_vals_array.getData()).getData();
auto & out_keys_offsets = out_keys_array.getOffsets();
size_t row_count = key_is_const ? in_values_array->size() : in_keys_array->size();
IColumn::Offsets * out_vals_offsets = &out_vals_array.getOffsets();
return {
in_keys_data,
in_values_data,
in_keys_offsets,
in_vals_offsets,
row_count,
key_is_const,
val_is_const,
out_keys_data,
out_vals_data,
out_keys_offsets,
out_vals_offsets};
}
template <typename KeyType, typename ValType>
ColumnsInOut<KeyType, ValType> getInOutDataFromMap(MutableColumnPtr & res_column, ColumnPtr * arg_columns) const
{
const auto * in_map = assert_cast<const ColumnMap *>(arg_columns[0].get());
const auto & in_nested_array = in_map->getNestedColumn();
const auto & in_nested_tuple = in_map->getNestedData();
const auto & in_keys_data = assert_cast<const ColumnVector<KeyType> &>(in_nested_tuple.getColumn(0)).getData();
const auto & in_vals_data = assert_cast<const ColumnVector<ValType> &>(in_nested_tuple.getColumn(1)).getData();
const auto & in_keys_offsets = in_nested_array.getOffsets();
auto * out_map = assert_cast<ColumnMap *>(res_column.get());
auto & out_nested_array = out_map->getNestedColumn();
auto & out_nested_tuple = out_map->getNestedData();
auto & out_keys_data = assert_cast<ColumnVector<KeyType> &>(out_nested_tuple.getColumn(0)).getData();
auto & out_vals_data = assert_cast<ColumnVector<ValType> &>(out_nested_tuple.getColumn(1)).getData();
auto & out_keys_offsets = out_nested_array.getOffsets();
return {
in_keys_data,
in_vals_data,
in_keys_offsets,
in_keys_offsets,
in_nested_array.size(),
false,
false,
out_keys_data,
out_vals_data,
out_keys_offsets,
nullptr};
}
template <typename KeyType, typename ValType>
ColumnPtr execute2(ColumnPtr * arg_columns, ColumnPtr max_key_column, const DataTypePtr & res_type) const
{
MutableColumnPtr res_column = res_type->createColumn();
bool max_key_is_const = false;
auto columns = res_column->getDataType() == TypeIndex::Tuple ? getInOutDataFromArrays<KeyType, ValType>(res_column, arg_columns)
: getInOutDataFromMap<KeyType, ValType>(res_column, arg_columns);
KeyType max_key_const{0};
if (max_key_column && isColumnConst(*max_key_column))
{
const auto * column_const = static_cast<const ColumnConst *>(&*max_key_column);
max_key_const = column_const->template getValue<KeyType>();
max_key_is_const = true;
}
IColumn::Offset offset{0};
std::map<KeyType, ValType> res_map;
//Iterate through two arrays and fill result values.
for (size_t row = 0; row < columns.row_count; ++row)
{
size_t key_offset = 0, val_offset = 0, items_count = columns.in_key_offsets[0], val_array_size = columns.in_val_offsets[0];
res_map.clear();
if (!columns.key_is_const)
if unlikely(sorted_keys_values.empty())
{
key_offset = row > 0 ? columns.in_key_offsets[row - 1] : 0;
items_count = columns.in_key_offsets[row] - key_offset;
}
if (!columns.val_is_const)
{
val_offset = row > 0 ? columns.in_val_offsets[row - 1] : 0;
val_array_size = columns.in_val_offsets[row] - val_offset;
}
if (items_count != val_array_size)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Key and value array should have same amount of elements in function {}",
getName());
if (items_count == 0)
{
columns.out_keys_offsets.push_back(offset);
result_offsets_data.emplace_back(result_value_data.size());
continue;
}
for (size_t i = 0; i < items_count; ++i)
::sort(sorted_keys_values.begin(), sorted_keys_values.end());
KeyType min_key = sorted_keys_values.front().first;
KeyType max_key = sorted_keys_values.back().first;
if (max_key_data)
{
res_map.insert({columns.in_keys_data[key_offset + i], columns.in_vals_data[val_offset + i]});
}
max_key = (*max_key_data)[offset_index];
auto min_key = res_map.begin()->first;
auto max_key = res_map.rbegin()->first;
if (max_key_column)
{
/* update the current max key if it's not constant */
if (max_key_is_const)
if (unlikely(max_key < min_key))
{
max_key = max_key_const;
}
else
{
max_key = (static_cast<const ColumnVector<KeyType> *>(max_key_column.get()))->getData()[row];
}
/* no need to add anything, max key is less that first key */
if (max_key < min_key)
{
columns.out_keys_offsets.push_back(offset);
result_offsets_data.emplace_back(result_value_data.size());
continue;
}
}
static constexpr size_t MAX_ARRAY_SIZE = 1ULL << 30;
if (static_cast<size_t>(max_key) - static_cast<size_t>(min_key) > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in the result of function {}", getName());
using KeyTypeUnsigned = ::make_unsigned_t<KeyType>;
KeyTypeUnsigned max_min_key_difference = 0;
/* fill the result arrays */
KeyType key;
for (key = min_key;; ++key)
if constexpr (::is_unsigned_v<KeyType>)
{
columns.out_keys_data.push_back(key);
max_min_key_difference = max_key - min_key;
}
else
{
bool is_max_key_positive = max_key >= 0;
bool is_min_key_positive = min_key >= 0;
auto it = res_map.find(key);
if (it != res_map.end())
if (is_max_key_positive && is_min_key_positive)
{
columns.out_vals_data.push_back(it->second);
max_min_key_difference = static_cast<KeyTypeUnsigned>(max_key - min_key);
}
else if (is_max_key_positive && !is_min_key_positive)
{
KeyTypeUnsigned min_key_unsigned = -static_cast<KeyTypeUnsigned>(min_key);
max_min_key_difference = static_cast<KeyTypeUnsigned>(max_key) + min_key_unsigned;
}
else
{
columns.out_vals_data.push_back(0);
/// Both max and min key are negative
KeyTypeUnsigned min_key_unsigned = -static_cast<KeyTypeUnsigned>(min_key);
KeyTypeUnsigned max_key_unsigned = -static_cast<KeyTypeUnsigned>(max_key);
max_min_key_difference = min_key_unsigned - max_key_unsigned;
}
}
static constexpr size_t MAX_ARRAY_SIZE = 1ULL << 30;
if (max_min_key_difference > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE,
"Function {} too large array size in the result",
getName());
size_t length = static_cast<size_t>(max_min_key_difference);
size_t result_key_data_size = result_key_data.size();
size_t result_value_data_size = result_value_data.size();
size_t sorted_keys_values_size = sorted_keys_values.size();
result_key_data.resize_fill(result_key_data_size + length + 1);
result_value_data.resize_fill(result_value_data_size + length + 1);
size_t sorted_values_index = 0;
for (KeyType current_key = min_key; current_key <= max_key; ++current_key)
{
size_t key_offset_index = current_key - min_key;
size_t insert_index = result_value_data_size + key_offset_index;
result_key_data[insert_index] = current_key;
if (sorted_values_index < sorted_keys_values_size &&
sorted_keys_values[sorted_values_index].first == current_key)
{
auto & sorted_key_value = sorted_keys_values[sorted_values_index];
if (current_key == sorted_key_value.first)
{
result_value_data[insert_index] = sorted_key_value.second;
}
++sorted_values_index;
while (sorted_values_index < sorted_keys_values_size &&
current_key == sorted_keys_values[sorted_values_index].first)
{
++sorted_values_index;
}
}
++offset;
if (key == max_key)
if (current_key == max_key)
break;
}
columns.out_keys_offsets.push_back(offset);
}
if (columns.out_vals_offsets)
columns.out_vals_offsets->insert(columns.out_keys_offsets.begin(), columns.out_keys_offsets.end());
return res_column;
}
template <typename KeyType>
ColumnPtr execute1(ColumnPtr * arg_columns, ColumnPtr max_key_column, const DataTypePtr & res_type, const DataTypePtr & val_type) const
{
switch (val_type->getTypeId())
{
case TypeIndex::Int8:
return execute2<KeyType, Int8>(arg_columns, max_key_column, res_type);
case TypeIndex::Int16:
return execute2<KeyType, Int16>(arg_columns, max_key_column, res_type);
case TypeIndex::Int32:
return execute2<KeyType, Int32>(arg_columns, max_key_column, res_type);
case TypeIndex::Int64:
return execute2<KeyType, Int64>(arg_columns, max_key_column, res_type);
case TypeIndex::Int128:
return execute2<KeyType, Int128>(arg_columns, max_key_column, res_type);
case TypeIndex::Int256:
return execute2<KeyType, Int256>(arg_columns, max_key_column, res_type);
case TypeIndex::UInt8:
return execute2<KeyType, UInt8>(arg_columns, max_key_column, res_type);
case TypeIndex::UInt16:
return execute2<KeyType, UInt16>(arg_columns, max_key_column, res_type);
case TypeIndex::UInt32:
return execute2<KeyType, UInt32>(arg_columns, max_key_column, res_type);
case TypeIndex::UInt64:
return execute2<KeyType, UInt64>(arg_columns, max_key_column, res_type);
case TypeIndex::UInt128:
return execute2<KeyType, UInt128>(arg_columns, max_key_column, res_type);
case TypeIndex::UInt256:
return execute2<KeyType, UInt256>(arg_columns, max_key_column, res_type);
default:
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns in arguments of function " + getName());
result_offsets_data.emplace_back(result_value_data.size());
}
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
struct KeyAndValueInput
{
DataTypePtr res_type, key_type, val_type;
ColumnPtr max_key_column = nullptr;
ColumnPtr arg_columns[] = {arguments[0].column, nullptr};
DataTypePtr key_series_type;
DataTypePtr value_series_type;
if (arguments[0].type->getTypeId() == TypeIndex::Array)
ColumnPtr key_column;
ColumnPtr value_column;
ColumnPtr offsets_column;
/// Optional max key column
ColumnPtr max_key_column;
};
KeyAndValueInput extractKeyAndValueInput(const ColumnsWithTypeAndName & arguments) const
{
KeyAndValueInput input;
size_t max_key_argument_index = 0;
auto first_argument_column = arguments[0].column->convertToFullColumnIfConst();
ColumnPtr second_argument_array_column;
if (const auto * key_argument_array_column = typeid_cast<const ColumnArray *>(first_argument_column.get()))
{
key_type = assert_cast<const DataTypeArray *>(arguments[0].type.get())->getNestedType();
val_type = assert_cast<const DataTypeArray *>(arguments[1].type.get())->getNestedType();
res_type = getReturnTypeImpl(DataTypes{arguments[0].type, arguments[1].type});
const ColumnArray * value_argument_array_column = nullptr;
arg_columns[1] = arguments[1].column;
if (arguments.size() == 3)
if (1 < arguments.size())
{
/* max key provided */
max_key_column = arguments[2].column;
second_argument_array_column = arguments[1].column->convertToFullColumnIfConst();
value_argument_array_column = typeid_cast<const ColumnArray *>(second_argument_array_column.get());
}
if (!value_argument_array_column)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Function {} if array argument is passed as key, additional array argument as value must be passed",
getName());
input.key_series_type = assert_cast<const DataTypeArray &>(*arguments[0].type).getNestedType();
input.key_column = key_argument_array_column->getDataPtr();
const auto & key_offsets = key_argument_array_column->getOffsets();
input.value_series_type = assert_cast<const DataTypeArray &>(*arguments[1].type).getNestedType();
input.value_column = value_argument_array_column->getDataPtr();
const auto & value_offsets = value_argument_array_column->getOffsets();
if (key_offsets != value_offsets)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Function {} key and value array should have same amount of elements",
getName());
input.offsets_column = key_argument_array_column->getOffsetsPtr();
max_key_argument_index = 2;
}
else if (const auto * key_argument_map_column = typeid_cast<const ColumnMap *>(first_argument_column.get()))
{
const auto & nested_array = key_argument_map_column->getNestedColumn();
const auto & nested_data_column = key_argument_map_column->getNestedData();
const auto & map_argument_type = assert_cast<const DataTypeMap &>(*arguments[0].type);
input.key_series_type = map_argument_type.getKeyType();
input.value_series_type = map_argument_type.getValueType();
input.key_column = nested_data_column.getColumnPtr(0);
input.value_column = nested_data_column.getColumnPtr(1);
input.offsets_column = nested_array.getOffsetsPtr();
max_key_argument_index = 1;
}
else
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Function {} only accepts one map or arrays, but got {}",
getName(),
arguments[0].type->getName());
ColumnPtr max_key_column;
if (max_key_argument_index < arguments.size())
{
max_key_column = arguments[max_key_argument_index].column->convertToFullColumnIfConst();
auto max_key_column_type = arguments[max_key_argument_index].type;
if (!max_key_column_type->equals(*input.key_series_type))
{
ColumnWithTypeAndName column_to_cast = {max_key_column, max_key_column_type, ""};
auto casted_column = castColumnAccurate(std::move(column_to_cast), input.key_series_type);
max_key_column = std::move(casted_column);
}
}
input.max_key_column = std::move(max_key_column);
return input;
}
struct ResultColumns
{
MutableColumnPtr result_key_column;
MutableColumnPtr result_value_column;
MutableColumnPtr result_offset_column;
IColumn * result_offset_column_raw;
/// If we return tuple of two arrays, this offset need to be the same as result_offset_column
MutableColumnPtr result_array_additional_offset_column;
};
ResultColumns extractResultColumns(MutableColumnPtr & result_column, const DataTypePtr & result_type) const
{
ResultColumns result;
auto * tuple_column = typeid_cast<ColumnTuple *>(result_column.get());
if (tuple_column && tuple_column->tupleSize() == 2)
{
auto key_array_column = tuple_column->getColumnPtr(0)->assumeMutable();
auto value_array_column = tuple_column->getColumnPtr(1)->assumeMutable();
auto * key_array_column_typed = typeid_cast<ColumnArray *>(key_array_column.get());
auto * value_array_column_typed = typeid_cast<ColumnArray *>(value_array_column.get());
if (!key_array_column_typed || !value_array_column_typed)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Function {} result type should be Tuple with two nested Array columns or Map. Actual {}",
getName(),
result_type->getName());
result.result_key_column = key_array_column_typed->getDataPtr()->assumeMutable();
result.result_value_column = value_array_column_typed->getDataPtr()->assumeMutable();
result.result_offset_column = key_array_column_typed->getOffsetsPtr()->assumeMutable();
result.result_offset_column_raw = result.result_offset_column.get();
result.result_array_additional_offset_column = value_array_column_typed->getOffsetsPtr()->assumeMutable();
}
else if (const auto * map_column = typeid_cast<ColumnMap *>(result_column.get()))
{
result.result_key_column = map_column->getNestedData().getColumnPtr(0)->assumeMutable();
result.result_value_column = map_column->getNestedData().getColumnPtr(1)->assumeMutable();
result.result_offset_column = map_column->getNestedColumn().getOffsetsPtr()->assumeMutable();
result.result_offset_column_raw = result.result_offset_column.get();
result.result_array_additional_offset_column = nullptr;
}
else
{
assert(arguments[0].type->getTypeId() == TypeIndex::Map);
const auto * map_type = assert_cast<const DataTypeMap *>(arguments[0].type.get());
res_type = getReturnTypeImpl(DataTypes{arguments[0].type});
key_type = map_type->getKeyType();
val_type = map_type->getValueType();
if (arguments.size() == 2)
{
/* max key provided */
max_key_column = arguments[1].column;
}
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Function {} result type should be Tuple with two nested Array columns or Map. Actual {}",
getName(),
result_type->getName());
}
switch (key_type->getTypeId())
return result;
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override
{
auto input = extractKeyAndValueInput(arguments);
auto result_column = result_type->createColumn();
auto result_columns = extractResultColumns(result_column, result_type);
auto call = [&](const auto & types)
{
case TypeIndex::Int8:
return execute1<Int8>(arg_columns, max_key_column, res_type, val_type);
case TypeIndex::Int16:
return execute1<Int16>(arg_columns, max_key_column, res_type, val_type);
case TypeIndex::Int32:
return execute1<Int32>(arg_columns, max_key_column, res_type, val_type);
case TypeIndex::Int64:
return execute1<Int64>(arg_columns, max_key_column, res_type, val_type);
case TypeIndex::Int128:
return execute1<Int128>(arg_columns, max_key_column, res_type, val_type);
case TypeIndex::Int256:
return execute1<Int256>(arg_columns, max_key_column, res_type, val_type);
case TypeIndex::UInt8:
return execute1<UInt8>(arg_columns, max_key_column, res_type, val_type);
case TypeIndex::UInt16:
return execute1<UInt16>(arg_columns, max_key_column, res_type, val_type);
case TypeIndex::UInt32:
return execute1<UInt32>(arg_columns, max_key_column, res_type, val_type);
case TypeIndex::UInt64:
return execute1<UInt64>(arg_columns, max_key_column, res_type, val_type);
case TypeIndex::UInt128:
return execute1<UInt128>(arg_columns, max_key_column, res_type, val_type);
case TypeIndex::UInt256:
return execute1<UInt256>(arg_columns, max_key_column, res_type, val_type);
default:
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns in arguments of function " + getName());
using Types = std::decay_t<decltype(types)>;
using KeyType = typename Types::LeftType;
using ValueType = typename Types::RightType;
static constexpr bool key_and_value_are_numbers = IsDataTypeNumber<KeyType> && IsDataTypeNumber<ValueType>;
static constexpr bool key_is_float = std::is_same_v<KeyType, DataTypeFloat32> || std::is_same_v<KeyType, DataTypeFloat64>;
if constexpr (key_and_value_are_numbers && !key_is_float)
{
using KeyFieldType = typename KeyType::FieldType;
using ValueFieldType = typename ValueType::FieldType;
executeImplTyped<KeyFieldType, ValueFieldType>(
input.key_column,
input.value_column,
input.offsets_column,
input.max_key_column,
std::move(result_columns.result_key_column),
std::move(result_columns.result_value_column),
std::move(result_columns.result_offset_column));
return true;
}
return false;
};
if (!callOnTwoTypeIndexes(input.key_series_type->getTypeId(), input.value_series_type->getTypeId(), call))
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Function {} illegal columns passed as arguments",
getName());
if (result_columns.result_array_additional_offset_column)
{
result_columns.result_array_additional_offset_column->insertRangeFrom(
*result_columns.result_offset_column_raw,
0,
result_columns.result_offset_column_raw->size());
}
return result_column;
}
};
@ -451,4 +501,5 @@ void registerFunctionMapPopulateSeries(FunctionFactory & factory)
{
factory.registerFunction<FunctionMapPopulateSeries>();
}
}

View File

@ -0,0 +1,4 @@
<test>
<query>SELECT mapPopulateSeries(range(number), range(number)) FROM numbers(5000) FORMAT Null;</query>
<query>SELECT mapPopulateSeries(range(number), range(number), 2500) FROM numbers(5000) FORMAT Null;</query>
</test>

View File

@ -13,6 +13,11 @@
([1,2,3,4,5,6,7,8,9,10],[1,0,2,0,0,0,0,0,0,0])
([1,2,3,4,5,6,7,8,9,10],[1,0,0,2,0,0,0,0,0,0])
([1,2,3,4,5,6,7,8,9,10],[1,0,0,0,2,0,0,0,0,0])
([1,2,3,4,5,6,7,8,9,10],[1,0,0,0,0,0,0,0,0,0])
([1,2,3,4,5,6,7,8,9,10],[1,2,0,0,0,0,0,0,0,0])
([1,2,3,4,5,6,7,8,9,10],[1,0,2,0,0,0,0,0,0,0])
([1,2,3,4,5,6,7,8,9,10],[1,0,0,2,0,0,0,0,0,0])
([1,2,3,4,5,6,7,8,9,10],[1,0,0,0,2,0,0,0,0,0])
([1,2],[1,0])
([1,2,3],[1,2,0])
([1,2,3,4],[1,0,2,0])

View File

@ -4,7 +4,7 @@ create table map_test engine=TinyLog() as (select (number + 1) as n, ([1, number
select mapPopulateSeries(map.1, map.2) from map_test;
select mapPopulateSeries(map.1, map.2, toUInt64(3)) from map_test;
select mapPopulateSeries(map.1, map.2, toUInt64(10)) from map_test;
select mapPopulateSeries(map.1, map.2, 1000) from map_test; -- { serverError 43 }
select mapPopulateSeries(map.1, map.2, 10) from map_test;
select mapPopulateSeries(map.1, map.2, n) from map_test;
select mapPopulateSeries(map.1, [11,22]) from map_test;
select mapPopulateSeries([3, 4], map.2) from map_test;
@ -31,6 +31,6 @@ select mapPopulateSeries([toInt64(-10), 2], [toInt64(1), 1], toInt64(-5)) as res
-- empty
select mapPopulateSeries(cast([], 'Array(UInt8)'), cast([], 'Array(UInt8)'), 5);
select mapPopulateSeries(['1', '2'], [1,1]) as res, toTypeName(res); -- { serverError 43 }
select mapPopulateSeries([1, 2, 3], [1,1]) as res, toTypeName(res); -- { serverError 42 }
select mapPopulateSeries([1, 2], [1,1,1]) as res, toTypeName(res); -- { serverError 42 }
select mapPopulateSeries(['1', '2'], [1, 1]) as res, toTypeName(res); -- { serverError 43 }
select mapPopulateSeries([1, 2, 3], [1, 1]) as res, toTypeName(res); -- { serverError 36 }
select mapPopulateSeries([1, 2], [1, 1, 1]) as res, toTypeName(res); -- { serverError 36 }

View File

@ -20,7 +20,12 @@ select mapPopulateSeries(m, toUInt64(10)) from map_test;
{1:1,2:0,3:2,4:0,5:0,6:0,7:0,8:0,9:0,10:0}
{1:1,2:0,3:0,4:2,5:0,6:0,7:0,8:0,9:0,10:0}
{1:1,2:0,3:0,4:0,5:2,6:0,7:0,8:0,9:0,10:0}
select mapPopulateSeries(m, 1000) from map_test; -- { serverError 43 }
select mapPopulateSeries(m, 10) from map_test;
{1:1,2:0,3:0,4:0,5:0,6:0,7:0,8:0,9:0,10:0}
{1:1,2:2,3:0,4:0,5:0,6:0,7:0,8:0,9:0,10:0}
{1:1,2:0,3:2,4:0,5:0,6:0,7:0,8:0,9:0,10:0}
{1:1,2:0,3:0,4:2,5:0,6:0,7:0,8:0,9:0,10:0}
{1:1,2:0,3:0,4:0,5:2,6:0,7:0,8:0,9:0,10:0}
select mapPopulateSeries(m, n) from map_test;
{1:1,2:0}
{1:1,2:2,3:0}

View File

@ -6,7 +6,7 @@ create table map_test engine=TinyLog() as (select (number + 1) as n, map(1, 1, n
select mapPopulateSeries(m) from map_test;
select mapPopulateSeries(m, toUInt64(3)) from map_test;
select mapPopulateSeries(m, toUInt64(10)) from map_test;
select mapPopulateSeries(m, 1000) from map_test; -- { serverError 43 }
select mapPopulateSeries(m, 10) from map_test;
select mapPopulateSeries(m, n) from map_test;
drop table map_test;

View File

@ -0,0 +1,34 @@
mapPopulateSeries with map
Without max key
{0:5}
{0:5,1:0,2:0,3:0,4:0,5:10}
{-5:-5,-4:0,-3:0,-2:0,-1:0,0:5,1:0,2:0,3:0,4:0,5:10}
{-5:-5,-4:0,-3:0,-2:0,-1:0,0:5,1:0,2:0,3:0,4:0,5:10,6:0,7:0,8:0,9:0,10:15}
With max key
{0:5,1:0,2:0,3:0,4:0,5:0,6:0,7:0,8:0,9:0,10:0,11:0,12:0,13:0,14:0,15:0,16:0,17:0,18:0,19:0,20:0}
{0:5,1:0,2:0,3:0,4:0,5:10,6:0,7:0,8:0,9:0,10:0,11:0,12:0,13:0,14:0,15:0,16:0,17:0,18:0,19:0,20:0}
{-5:-5,-4:0,-3:0,-2:0,-1:0,0:5,1:0,2:0,3:0,4:0,5:10}
{-5:-5,-4:0,-3:0,-2:0,-1:0,0:5,1:0,2:0,3:0,4:0,5:10,6:0,7:0,8:0,9:0,10:15,11:0,12:0,13:0,14:0,15:0,16:0,17:0,18:0,19:0,20:0}
Possible verflow
{18446744073709551610:5,18446744073709551611:0,18446744073709551612:0,18446744073709551613:0,18446744073709551614:0,18446744073709551615:0}
{18446744073709551615:5}
Duplicate keys
{1:4,2:0,3:0,4:0,5:6}
{1:4,2:0,3:0,4:0,5:6,6:0,7:0,8:0,9:0,10:0}
mapPopulateSeries with two arrays
Without max key
([0],[5])
([0,1,2,3,4,5],[5,0,0,0,0,10])
([-5,-4,-3,-2,-1,0,1,2,3,4,5],[-5,0,0,0,0,5,0,0,0,0,10])
([-5,-4,-3,-2,-1,0,1,2,3,4,5,6,7,8,9,10],[-5,0,0,0,0,5,0,0,0,0,10,0,0,0,0,15])
With max key
([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0])
([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[5,0,0,0,0,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0])
([-5,-4,-3,-2,-1,0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[-5,0,0,0,0,5,0,0,0,0,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0])
([-5,-4,-3,-2,-1,0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20],[-5,0,0,0,0,5,0,0,0,0,10,0,0,0,0,15,0,0,0,0,0,0,0,0,0,0])
Possible verflow
([18446744073709551610,18446744073709551611,18446744073709551612,18446744073709551613,18446744073709551614,18446744073709551615],[5,0,0,0,0,0])
([18446744073709551615],[5])
Duplicate keys
([1,2,3,4,5],[4,0,0,0,6])
([1,2,3,4,5,6,7,8,9,10],[4,0,0,0,6,0,0,0,0,0])

View File

@ -0,0 +1,125 @@
DROP TABLE IF EXISTS 02005_test_table;
CREATE TABLE 02005_test_table
(
value Map(Int64, Int64)
)
ENGINE = TinyLog;
SELECT 'mapPopulateSeries with map';
SELECT 'Without max key';
SELECT mapPopulateSeries(value) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES (map(0, 5));
SELECT mapPopulateSeries(value) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES (map(0, 5, 5, 10));
SELECT mapPopulateSeries(value) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES (map(-5, -5, 0, 5, 5, 10));
SELECT mapPopulateSeries(value) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES (map(-5, -5, 0, 5, 5, 10, 10, 15));
SELECT mapPopulateSeries(value) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
SELECT 'With max key';
SELECT mapPopulateSeries(value, materialize(20)) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES (map(0, 5));
SELECT mapPopulateSeries(value, materialize(20)) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES (map(0, 5, 5, 10));
SELECT mapPopulateSeries(value, materialize(20)) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES (map(-5, -5, 0, 5, 5, 10));
SELECT mapPopulateSeries(value) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES (map(-5, -5, 0, 5, 5, 10, 10, 15));
SELECT mapPopulateSeries(value, materialize(20)) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
SELECT 'Possible verflow';
SELECT mapPopulateSeries(map(toUInt64(18446744073709551610), toUInt64(5)), 18446744073709551615);
SELECT mapPopulateSeries(map(toUInt64(18446744073709551615), toUInt64(5)), 18446744073709551615);
SELECT 'Duplicate keys';
SELECT mapPopulateSeries(map(1, 4, 1, 5, 5, 6));
SELECT mapPopulateSeries(map(1, 4, 1, 5, 5, 6), materialize(10));
DROP TABLE 02005_test_table;
DROP TABLE IF EXISTS 02005_test_table;
CREATE TABLE 02005_test_table
(
key Array(Int64),
value Array(Int64)
)
ENGINE = TinyLog;
SELECT 'mapPopulateSeries with two arrays';
SELECT 'Without max key';
SELECT mapPopulateSeries(key, value) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES ([0], [5]);
SELECT mapPopulateSeries(key, value) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES ([0, 5], [5, 10]);
SELECT mapPopulateSeries(key, value) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES ([-5, 0, 5], [-5, 5, 10]);
SELECT mapPopulateSeries(key, value) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES ([-5, 0, 5, 10], [-5, 5, 10, 15]);
SELECT mapPopulateSeries(key, value) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
SELECT 'With max key';
SELECT mapPopulateSeries(key, value, materialize(20)) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES ([0], [5]);
SELECT mapPopulateSeries(key, value, materialize(20)) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES ([0, 5], [5, 10]);
SELECT mapPopulateSeries(key, value, materialize(20)) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES ([-5, 0, 5], [-5, 5, 10]);
SELECT mapPopulateSeries(key, value, materialize(20)) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
INSERT INTO 02005_test_table VALUES ([-5, 0, 5, 10], [-5, 5, 10, 15]);
SELECT mapPopulateSeries(key, value, materialize(20)) FROM 02005_test_table;
TRUNCATE TABLE 02005_test_table;
SELECT 'Possible verflow';
SELECT mapPopulateSeries([18446744073709551610], [5], 18446744073709551615);
SELECT mapPopulateSeries([18446744073709551615], [5], 18446744073709551615);
SELECT 'Duplicate keys';
SELECT mapPopulateSeries([1, 1, 5], [4, 5, 6]);
SELECT mapPopulateSeries([1, 1, 5], [4, 5, 6], materialize(10));
DROP TABLE 02005_test_table;