mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #34318 from kitaisreal/map-populate-series-refactoring
Function mapPopulateSeries refactoring
This commit is contained in:
commit
eff16baaf3
@ -1,15 +1,20 @@
|
|||||||
|
#include <base/sort.h>
|
||||||
|
|
||||||
|
#include <Core/ColumnWithTypeAndName.h>
|
||||||
#include <Columns/ColumnArray.h>
|
#include <Columns/ColumnArray.h>
|
||||||
#include <Columns/ColumnMap.h>
|
#include <Columns/ColumnMap.h>
|
||||||
#include <Columns/ColumnTuple.h>
|
#include <Columns/ColumnTuple.h>
|
||||||
#include <Columns/ColumnVector.h>
|
#include <Columns/ColumnVector.h>
|
||||||
|
#include <DataTypes/IDataType.h>
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <DataTypes/DataTypeMap.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
#include <Functions/FunctionHelpers.h>
|
#include <Functions/FunctionHelpers.h>
|
||||||
#include <Functions/IFunction.h>
|
#include <Functions/IFunction.h>
|
||||||
#include "Core/ColumnWithTypeAndName.h"
|
#include <Interpreters/castColumn.h>
|
||||||
#include "DataTypes/DataTypeMap.h"
|
|
||||||
#include "DataTypes/IDataType.h"
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -19,6 +24,8 @@ namespace ErrorCodes
|
|||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
extern const int TOO_LARGE_ARRAY_SIZE;
|
extern const int TOO_LARGE_ARRAY_SIZE;
|
||||||
|
extern const int BAD_ARGUMENTS;
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
}
|
}
|
||||||
|
|
||||||
class FunctionMapPopulateSeries : public IFunction
|
class FunctionMapPopulateSeries : public IFunction
|
||||||
@ -35,415 +42,458 @@ private:
|
|||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) 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);
|
WhichDataType key_data_type(key_type);
|
||||||
if (!(which_key.isInt() || which_key.isUInt()))
|
WhichDataType value_data_type(value_type);
|
||||||
|
|
||||||
|
if (!(key_data_type.isInt() || key_data_type.isUInt()))
|
||||||
{
|
{
|
||||||
throw Exception(
|
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(
|
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)
|
if (!max_key_type)
|
||||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Too many arguments in {} call", getName());
|
return;
|
||||||
|
|
||||||
const DataTypeArray * key_array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
|
WhichDataType max_key_data_type(max_key_type);
|
||||||
const DataTypeArray * val_array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get());
|
|
||||||
|
|
||||||
if (!key_array_type || !val_array_type)
|
if (max_key_data_type.isNullable())
|
||||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {} accepts two arrays for key and value", getName());
|
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 (!(max_key_data_type.isInt() || max_key_data_type.isUInt()))
|
||||||
|
throw Exception(
|
||||||
if (arguments.size() == 3)
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||||
this->checkTypes(key_type, arguments[2]);
|
"Function {} max key should be of signed or unsigned integer type. Actual type {}.",
|
||||||
else
|
getName(),
|
||||||
this->checkTypes(key_type, nullptr);
|
key_type->getName(),
|
||||||
|
max_key_type->getName());
|
||||||
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());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
{
|
{
|
||||||
if (arguments.empty())
|
if (arguments.empty() || arguments.size() > 3)
|
||||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, getName() + " accepts at least one map or two arrays");
|
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)
|
WhichDataType key_argument_data_type(arguments[0]);
|
||||||
return getReturnTypeForTuple(arguments);
|
|
||||||
else if (arguments[0]->getTypeId() == TypeIndex::Map)
|
DataTypePtr key_argument_series_type;
|
||||||
return getReturnTypeForMap(arguments);
|
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
|
else
|
||||||
throw Exception(
|
throw Exception(
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||||
"Function {} only accepts one map or arrays, but got {}",
|
"Function {} only accepts one map or arrays, but got {}",
|
||||||
getName(),
|
getName(),
|
||||||
arguments[0]->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,
|
template <typename KeyType, typename ValueType>
|
||||||
// Both arrays and maps have similar columns to work with but extracted differently
|
void executeImplTyped(
|
||||||
template <typename KeyType, typename ValType>
|
const ColumnPtr & key_column,
|
||||||
struct ColumnsInOut
|
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 auto & key_column_typed = assert_cast<const ColumnVector<KeyType> &>(*key_column);
|
||||||
const PaddedPODArray<KeyType> & in_keys_data;
|
const auto & key_column_data = key_column_typed.getData();
|
||||||
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;
|
|
||||||
|
|
||||||
// outputs
|
const auto & offsets_column_typed = assert_cast<const ColumnVector<ColumnArray::Offset> &>(*offsets_column);
|
||||||
PaddedPODArray<KeyType> & out_keys_data;
|
const auto & offsets = offsets_column_typed.getData();
|
||||||
PaddedPODArray<ValType> & out_vals_data;
|
|
||||||
|
|
||||||
IColumn::Offsets & out_keys_offsets;
|
const auto & value_column_typed = assert_cast<const ColumnVector<ValueType> &>(*value_column);
|
||||||
// with map argument this field will not be used
|
const auto & value_column_data = value_column_typed.getData();
|
||||||
IColumn::Offsets * out_vals_offsets;
|
|
||||||
};
|
|
||||||
|
|
||||||
template <typename KeyType, typename ValType>
|
auto & result_key_column_typed = assert_cast<ColumnVector<KeyType> &>(*result_key_column);
|
||||||
ColumnsInOut<KeyType, ValType> getInOutDataFromArrays(MutableColumnPtr & res_column, ColumnPtr * arg_columns) const
|
auto & result_key_data = result_key_column_typed.getData();
|
||||||
{
|
|
||||||
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));
|
|
||||||
|
|
||||||
const auto * key_column = arg_columns[0].get();
|
auto & result_value_column_typed = assert_cast<ColumnVector<ValueType> &>(*result_value_column);
|
||||||
const auto * in_keys_array = checkAndGetColumn<ColumnArray>(key_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);
|
size_t start_offset = offsets[offset_index - 1];
|
||||||
if (!const_array)
|
size_t end_offset = offsets[offset_index];
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::ILLEGAL_COLUMN, "Expected array column in function {}, found {}", getName(), key_column->getName());
|
|
||||||
|
|
||||||
in_keys_array = checkAndGetColumn<ColumnArray>(const_array->getDataColumnPtr().get());
|
sorted_keys_values.clear();
|
||||||
key_is_const = true;
|
|
||||||
}
|
|
||||||
|
|
||||||
const auto * val_column = arg_columns[1].get();
|
for (; start_offset < end_offset; ++start_offset)
|
||||||
const auto * in_values_array = checkAndGetColumn<ColumnArray>(val_column);
|
sorted_keys_values.emplace_back(key_column_data[start_offset], value_column_data[start_offset]);
|
||||||
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());
|
|
||||||
|
|
||||||
in_values_array = checkAndGetColumn<ColumnArray>(const_array->getDataColumnPtr().get());
|
if unlikely(sorted_keys_values.empty())
|
||||||
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)
|
|
||||||
{
|
{
|
||||||
key_offset = row > 0 ? columns.in_key_offsets[row - 1] : 0;
|
result_offsets_data.emplace_back(result_value_data.size());
|
||||||
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);
|
|
||||||
continue;
|
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;
|
if (unlikely(max_key < min_key))
|
||||||
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)
|
|
||||||
{
|
{
|
||||||
max_key = max_key_const;
|
result_offsets_data.emplace_back(result_value_data.size());
|
||||||
}
|
|
||||||
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);
|
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static constexpr size_t MAX_ARRAY_SIZE = 1ULL << 30;
|
using KeyTypeUnsigned = ::make_unsigned_t<KeyType>;
|
||||||
if (static_cast<size_t>(max_key) - static_cast<size_t>(min_key) > MAX_ARRAY_SIZE)
|
KeyTypeUnsigned max_min_key_difference = 0;
|
||||||
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in the result of function {}", getName());
|
|
||||||
|
|
||||||
/* fill the result arrays */
|
if constexpr (::is_unsigned_v<KeyType>)
|
||||||
KeyType key;
|
|
||||||
for (key = min_key;; ++key)
|
|
||||||
{
|
{
|
||||||
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 (is_max_key_positive && is_min_key_positive)
|
||||||
if (it != res_map.end())
|
|
||||||
{
|
{
|
||||||
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
|
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 (current_key == max_key)
|
||||||
if (key == max_key)
|
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
columns.out_keys_offsets.push_back(offset);
|
result_offsets_data.emplace_back(result_value_data.size());
|
||||||
}
|
|
||||||
|
|
||||||
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());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
|
struct KeyAndValueInput
|
||||||
{
|
{
|
||||||
DataTypePtr res_type, key_type, val_type;
|
DataTypePtr key_series_type;
|
||||||
ColumnPtr max_key_column = nullptr;
|
DataTypePtr value_series_type;
|
||||||
ColumnPtr arg_columns[] = {arguments[0].column, nullptr};
|
|
||||||
|
|
||||||
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();
|
const ColumnArray * value_argument_array_column = nullptr;
|
||||||
val_type = assert_cast<const DataTypeArray *>(arguments[1].type.get())->getNestedType();
|
|
||||||
res_type = getReturnTypeImpl(DataTypes{arguments[0].type, arguments[1].type});
|
|
||||||
|
|
||||||
arg_columns[1] = arguments[1].column;
|
if (1 < arguments.size())
|
||||||
if (arguments.size() == 3)
|
|
||||||
{
|
{
|
||||||
/* max key provided */
|
second_argument_array_column = arguments[1].column->convertToFullColumnIfConst();
|
||||||
max_key_column = arguments[2].column;
|
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
|
else
|
||||||
{
|
{
|
||||||
assert(arguments[0].type->getTypeId() == TypeIndex::Map);
|
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||||
|
"Function {} result type should be Tuple with two nested Array columns or Map. Actual {}",
|
||||||
const auto * map_type = assert_cast<const DataTypeMap *>(arguments[0].type.get());
|
getName(),
|
||||||
res_type = getReturnTypeImpl(DataTypes{arguments[0].type});
|
result_type->getName());
|
||||||
key_type = map_type->getKeyType();
|
|
||||||
val_type = map_type->getValueType();
|
|
||||||
|
|
||||||
if (arguments.size() == 2)
|
|
||||||
{
|
|
||||||
/* max key provided */
|
|
||||||
max_key_column = arguments[1].column;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
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:
|
using Types = std::decay_t<decltype(types)>;
|
||||||
return execute1<Int8>(arg_columns, max_key_column, res_type, val_type);
|
using KeyType = typename Types::LeftType;
|
||||||
case TypeIndex::Int16:
|
using ValueType = typename Types::RightType;
|
||||||
return execute1<Int16>(arg_columns, max_key_column, res_type, val_type);
|
|
||||||
case TypeIndex::Int32:
|
static constexpr bool key_and_value_are_numbers = IsDataTypeNumber<KeyType> && IsDataTypeNumber<ValueType>;
|
||||||
return execute1<Int32>(arg_columns, max_key_column, res_type, val_type);
|
static constexpr bool key_is_float = std::is_same_v<KeyType, DataTypeFloat32> || std::is_same_v<KeyType, DataTypeFloat64>;
|
||||||
case TypeIndex::Int64:
|
|
||||||
return execute1<Int64>(arg_columns, max_key_column, res_type, val_type);
|
if constexpr (key_and_value_are_numbers && !key_is_float)
|
||||||
case TypeIndex::Int128:
|
{
|
||||||
return execute1<Int128>(arg_columns, max_key_column, res_type, val_type);
|
using KeyFieldType = typename KeyType::FieldType;
|
||||||
case TypeIndex::Int256:
|
using ValueFieldType = typename ValueType::FieldType;
|
||||||
return execute1<Int256>(arg_columns, max_key_column, res_type, val_type);
|
|
||||||
case TypeIndex::UInt8:
|
executeImplTyped<KeyFieldType, ValueFieldType>(
|
||||||
return execute1<UInt8>(arg_columns, max_key_column, res_type, val_type);
|
input.key_column,
|
||||||
case TypeIndex::UInt16:
|
input.value_column,
|
||||||
return execute1<UInt16>(arg_columns, max_key_column, res_type, val_type);
|
input.offsets_column,
|
||||||
case TypeIndex::UInt32:
|
input.max_key_column,
|
||||||
return execute1<UInt32>(arg_columns, max_key_column, res_type, val_type);
|
std::move(result_columns.result_key_column),
|
||||||
case TypeIndex::UInt64:
|
std::move(result_columns.result_value_column),
|
||||||
return execute1<UInt64>(arg_columns, max_key_column, res_type, val_type);
|
std::move(result_columns.result_offset_column));
|
||||||
case TypeIndex::UInt128:
|
|
||||||
return execute1<UInt128>(arg_columns, max_key_column, res_type, val_type);
|
return true;
|
||||||
case TypeIndex::UInt256:
|
}
|
||||||
return execute1<UInt256>(arg_columns, max_key_column, res_type, val_type);
|
|
||||||
default:
|
return false;
|
||||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns in arguments of function " + getName());
|
};
|
||||||
|
|
||||||
|
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>();
|
factory.registerFunction<FunctionMapPopulateSeries>();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
4
tests/performance/map_populate_series.xml
Normal file
4
tests/performance/map_populate_series.xml
Normal 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>
|
@ -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,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,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,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],[1,0])
|
||||||
([1,2,3],[1,2,0])
|
([1,2,3],[1,2,0])
|
||||||
([1,2,3,4],[1,0,2,0])
|
([1,2,3,4],[1,0,2,0])
|
||||||
|
@ -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) from map_test;
|
||||||
select mapPopulateSeries(map.1, map.2, toUInt64(3)) 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, 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, map.2, n) from map_test;
|
||||||
select mapPopulateSeries(map.1, [11,22]) from map_test;
|
select mapPopulateSeries(map.1, [11,22]) from map_test;
|
||||||
select mapPopulateSeries([3, 4], map.2) 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
|
-- empty
|
||||||
select mapPopulateSeries(cast([], 'Array(UInt8)'), cast([], 'Array(UInt8)'), 5);
|
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'], [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, 3], [1, 1]) as res, toTypeName(res); -- { serverError 36 }
|
||||||
select mapPopulateSeries([1, 2], [1,1,1]) as res, toTypeName(res); -- { serverError 42 }
|
select mapPopulateSeries([1, 2], [1, 1, 1]) as res, toTypeName(res); -- { serverError 36 }
|
||||||
|
@ -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: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: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}
|
{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;
|
select mapPopulateSeries(m, n) from map_test;
|
||||||
{1:1,2:0}
|
{1:1,2:0}
|
||||||
{1:1,2:2,3:0}
|
{1:1,2:2,3:0}
|
||||||
|
@ -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) from map_test;
|
||||||
select mapPopulateSeries(m, toUInt64(3)) from map_test;
|
select mapPopulateSeries(m, toUInt64(3)) from map_test;
|
||||||
select mapPopulateSeries(m, toUInt64(10)) 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;
|
select mapPopulateSeries(m, n) from map_test;
|
||||||
|
|
||||||
drop table map_test;
|
drop table map_test;
|
||||||
|
@ -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])
|
@ -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;
|
Loading…
Reference in New Issue
Block a user