Make changes by review on #26663 PR

This commit is contained in:
Ildus Kurbangaliev 2021-07-30 20:57:32 +02:00
parent 0f480e6b06
commit a4e9a56e8e
4 changed files with 91 additions and 42 deletions

View File

@ -40,7 +40,7 @@ private:
if (!(which_key.isInt() || which_key.isUInt()))
{
throw Exception(
"Keys for " + getName() + " should be of integer type (signed or unsigned)", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Keys for {} function should be of integer type (signed or unsigned)", getName());
}
if (max_key_type)
@ -49,27 +49,28 @@ private:
if (which_max_key.isNullable())
throw Exception(
"Max key argument in arguments of function " + getName() + " can not be Nullable",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
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("Max key type in " + getName() + " should be same as keys type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
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(getName() + " accepts at least two arrays for key and value", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} accepts at least two arrays for key and value", getName());
if (arguments.size() > 3)
throw Exception("too many arguments in " + getName() + " call", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Too many arguments in {} call", getName());
const DataTypeArray * key_array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
const DataTypeArray * val_array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get());
if (!key_array_type || !val_array_type)
throw Exception(getName() + " accepts two arrays for key and value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {} accepts two arrays for key and value", getName());
const auto & key_type = key_array_type->getNestedType();
@ -89,7 +90,7 @@ private:
else if (arguments.size() == 2)
this->checkTypes(map->getKeyType(), arguments[1]);
else
throw Exception("too many arguments in " + getName() + " call", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Too many arguments in {} call", getName());
return std::make_shared<DataTypeMap>(map->getKeyType(), map->getValueType());
}
@ -97,14 +98,18 @@ private:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.empty())
throw Exception(getName() + " accepts at least one map", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, getName() + " accepts at least one map or two arrays");
if (arguments[0]->getTypeId() == TypeIndex::Array)
return getReturnTypeForTuple(arguments);
else if (arguments[0]->getTypeId() == TypeIndex::Map)
return getReturnTypeForMap(arguments);
else
throw Exception(getName() + " only accepts maps", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Function {} only accepts one map or arrays, but got {}",
getName(),
arguments[0]->getName());
}
// Struct holds input and output columns references,
@ -146,7 +151,8 @@ private:
{
const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(key_column);
if (!const_array)
throw Exception("Expected array column, found " + key_column->getName(), ErrorCodes::ILLEGAL_COLUMN);
throw Exception(
ErrorCodes::ILLEGAL_COLUMN, "Expected array column in function {}, found {}", getName(), key_column->getName());
in_keys_array = checkAndGetColumn<ColumnArray>(const_array->getDataColumnPtr().get());
key_is_const = true;
@ -158,7 +164,8 @@ private:
{
const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(val_column);
if (!const_array)
throw Exception("Expected array column, found " + val_column->getName(), ErrorCodes::ILLEGAL_COLUMN);
throw Exception(
ErrorCodes::ILLEGAL_COLUMN, "Expected array column in function {}, found {}", getName(), val_column->getName());
in_values_array = checkAndGetColumn<ColumnArray>(const_array->getDataColumnPtr().get());
val_is_const = true;
@ -166,7 +173,7 @@ private:
if (!in_keys_array || !in_values_array)
/* something went wrong */
throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
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();
@ -230,8 +237,8 @@ private:
{
MutableColumnPtr res_column = res_type->createColumn();
bool max_key_is_const = false;
auto inout = res_column->getDataType() == TypeIndex::Tuple ? getInOutDataFromArrays<KeyType, ValType>(res_column, arg_columns)
: getInOutDataFromMap<KeyType, ValType>(res_column, arg_columns);
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};
@ -246,36 +253,39 @@ private:
std::map<KeyType, ValType> res_map;
//Iterate through two arrays and fill result values.
for (size_t row = 0; row < inout.row_count; ++row)
for (size_t row = 0; row < columns.row_count; ++row)
{
size_t key_offset = 0, val_offset = 0, items_count = inout.in_key_offsets[0], val_array_size = inout.in_val_offsets[0];
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 (!inout.key_is_const)
if (!columns.key_is_const)
{
key_offset = row > 0 ? inout.in_key_offsets[row - 1] : 0;
items_count = inout.in_key_offsets[row] - key_offset;
key_offset = row > 0 ? columns.in_key_offsets[row - 1] : 0;
items_count = columns.in_key_offsets[row] - key_offset;
}
if (!inout.val_is_const)
if (!columns.val_is_const)
{
val_offset = row > 0 ? inout.in_val_offsets[row - 1] : 0;
val_array_size = inout.in_val_offsets[row] - val_offset;
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("Key and value array should have same amount of elements", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
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)
{
inout.out_keys_offsets.push_back(offset);
columns.out_keys_offsets.push_back(offset);
continue;
}
for (size_t i = 0; i < items_count; ++i)
{
res_map.insert({inout.in_keys_data[key_offset + i], inout.in_vals_data[val_offset + i]});
res_map.insert({columns.in_keys_data[key_offset + i], columns.in_vals_data[val_offset + i]});
}
auto min_key = res_map.begin()->first;
@ -296,7 +306,7 @@ private:
/* no need to add anything, max key is less that first key */
if (max_key < min_key)
{
inout.out_keys_offsets.push_back(offset);
columns.out_keys_offsets.push_back(offset);
continue;
}
}
@ -309,16 +319,16 @@ private:
KeyType key;
for (key = min_key;; ++key)
{
inout.out_keys_data.push_back(key);
columns.out_keys_data.push_back(key);
auto it = res_map.find(key);
if (it != res_map.end())
{
inout.out_vals_data.push_back(it->second);
columns.out_vals_data.push_back(it->second);
}
else
{
inout.out_vals_data.push_back(0);
columns.out_vals_data.push_back(0);
}
++offset;
@ -326,11 +336,11 @@ private:
break;
}
inout.out_keys_offsets.push_back(offset);
columns.out_keys_offsets.push_back(offset);
}
if (inout.out_vals_offsets)
inout.out_vals_offsets->insert(inout.out_keys_offsets.begin(), inout.out_keys_offsets.end());
if (columns.out_vals_offsets)
columns.out_vals_offsets->insert(columns.out_keys_offsets.begin(), columns.out_keys_offsets.end());
return res_column;
}
@ -365,7 +375,7 @@ private:
case TypeIndex::UInt256:
return execute2<KeyType, UInt256>(arg_columns, max_key_column, res_type);
default:
throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns in arguments of function " + getName());
}
}
@ -431,7 +441,7 @@ private:
case TypeIndex::UInt256:
return execute1<UInt256>(arg_columns, max_key_column, res_type, val_type);
default:
throw Exception("Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns in arguments of function " + getName());
}
}
};

View File

@ -1,37 +1,67 @@
-- { echo }
drop table if exists map_test;
set allow_experimental_map_type = 1;
create table map_test engine=TinyLog() as (select (number + 1) as n, map(1, 1, number,2) as m from numbers(1, 5));
select mapPopulateSeries(m) from map_test;
{1:1}
{1:1,2:2}
{1:1,2:0,3:2}
{1:1,2:0,3:0,4:2}
{1:1,2:0,3:0,4:0,5:2}
select mapPopulateSeries(m, toUInt64(3)) from map_test;
{1:1,2:0,3:0}
{1:1,2:2,3:0}
{1:1,2:0,3:2}
{1:1,2:0,3:0}
{1:1,2:0,3:0}
select mapPopulateSeries(m, toUInt64(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, 1000) from map_test; -- { serverError 43 }
select mapPopulateSeries(m, n) from map_test;
{1:1,2:0}
{1:1,2:2,3:0}
{1:1,2:0,3:2,4:0}
{1:1,2:0,3:0,4:2,5:0}
{1:1,2:0,3:0,4:0,5:2,6:0}
drop table map_test;
select mapPopulateSeries(map(toUInt8(1), toUInt8(1), 2, 1)) as res, toTypeName(res);
{1:1,2:1} Map(UInt8,UInt8)
select mapPopulateSeries(map(toUInt16(1), toUInt16(1), 2, 1)) as res, toTypeName(res);
{1:1,2:1} Map(UInt16,UInt16)
select mapPopulateSeries(map(toUInt32(1), toUInt32(1), 2, 1)) as res, toTypeName(res);
{1:1,2:1} Map(UInt32,UInt32)
select mapPopulateSeries(map(toUInt64(1), toUInt64(1), 2, 1)) as res, toTypeName(res);
{1:1,2:1} Map(UInt64,UInt64)
select mapPopulateSeries(map(toUInt128(1), toUInt128(1), 2, 1)) as res, toTypeName(res);
{1:1,2:1} Map(UInt128,UInt128)
select mapPopulateSeries(map(toUInt256(1), toUInt256(1), 2, 1)) as res, toTypeName(res);
{1:1,2:1} Map(UInt256,UInt256)
select mapPopulateSeries(map(toInt8(1), toInt8(1), 2, 1)) as res, toTypeName(res);
{1:1,2:1} Map(Int16,Int16)
select mapPopulateSeries(map(toInt16(1), toInt16(1), 2, 1)) as res, toTypeName(res);
{1:1,2:1} Map(Int16,Int16)
select mapPopulateSeries(map(toInt32(1), toInt32(1), 2, 1)) as res, toTypeName(res);
{1:1,2:1} Map(Int32,Int32)
select mapPopulateSeries(map(toInt64(1), toInt64(1), 2, 1)) as res, toTypeName(res);
{1:1,2:1} Map(Int64,Int64)
select mapPopulateSeries(map(toInt128(1), toInt128(1), 2, 1)) as res, toTypeName(res);
{1:1,2:1} Map(Int128,Int128)
select mapPopulateSeries(map(toInt256(1), toInt256(1), 2, 1)) as res, toTypeName(res);
{1:1,2:1} Map(Int256,Int256)
select mapPopulateSeries(map(toInt8(-10), toInt8(1), 2, 1)) as res, toTypeName(res);
{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16)
select mapPopulateSeries(map(toInt16(-10), toInt16(1), 2, 1)) as res, toTypeName(res);
{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int16,Int16)
select mapPopulateSeries(map(toInt32(-10), toInt32(1), 2, 1)) as res, toTypeName(res);
{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int32,Int32)
select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1)) as res, toTypeName(res);
{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0,-4:0,-3:0,-2:0,-1:0,0:0,1:0,2:1} Map(Int64,Int64)
select mapPopulateSeries(map(toInt64(-10), toInt64(1), 2, 1), toInt64(-5)) as res, toTypeName(res);
{-10:1,-9:0,-8:0,-7:0,-6:0,-5:0} Map(Int64,Int64)
select mapPopulateSeries(); -- { serverError 42 }
select mapPopulateSeries('asdf'); -- { serverError 43 }
select mapPopulateSeries(map('1', 1, '2', 1)) as res, toTypeName(res); -- { serverError 43 }

View File

@ -1,3 +1,4 @@
-- { echo }
drop table if exists map_test;
set allow_experimental_map_type = 1;
create table map_test engine=TinyLog() as (select (number + 1) as n, map(1, 1, number,2) as m from numbers(1, 5));

View File

@ -357,7 +357,7 @@ def map_func(self, data_type, node=None):
exitcode, message = 0, None
if data_type.startswith("Decimal"):
exitcode, message = 43, "Exception:"
exitcode, message = 43, "Exception:"
node.query(sql, exitcode=exitcode, message=message)
execute_query(f"""SELECT * FROM {table_name} ORDER BY a ASC""")
@ -393,9 +393,13 @@ def map_func(self, data_type, node=None):
execute_query(f"SELECT * FROM {table_name} ORDER BY a ASC")
with Scenario(f"mapPopulateSeries with {data_type}"):
node.query(f"SELECT mapPopulateSeries([1,2,3], [{to_data_type(data_type,1)},"
f"{to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)",
exitcode = 44, message='Exception:')
sql = (f"SELECT mapPopulateSeries([1,2,3], [{to_data_type(data_type,1)},"
f"{to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)")
exitcode, message = 0, None
if data_type.startswith("Decimal"):
exitcode, message = 44, "Exception:"
node.query(sql, exitcode=exitcode, message=message)
with Scenario(f"mapPopulateSeries with {data_type} on a table"):
table_name = get_table_name()
@ -403,9 +407,13 @@ def map_func(self, data_type, node=None):
table(name = table_name, data_type = f'Tuple(Array({data_type}), Array({data_type}))')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3],"
f"[{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)",
exitcode = 44, message='Exception:')
sql = (f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3],",
f"[{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)")
exitcode, message = 0, None
if data_type.startswith("Decimal"):
exitcode, message = 44, "Exception:"
node.query(sql, exitcode=exitcode, message=message)
execute_query(f"SELECT * FROM {table_name} ORDER BY a ASC")