mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
dbms: array element support non-constant index + test on new functionality [METR-10798]
This commit is contained in:
parent
04c7058d99
commit
e48c3d7b5e
@ -186,13 +186,14 @@ public:
|
||||
};
|
||||
|
||||
|
||||
template <typename T, bool negative>
|
||||
template <typename T>
|
||||
struct ArrayElementNumImpl
|
||||
{
|
||||
/** Если negative = false - передаётся индекс с начала массива, начиная с нуля.
|
||||
* Если negative = true - передаётся индекс с конца массива, начиная с нуля.
|
||||
*/
|
||||
static void vector(
|
||||
template <bool negative>
|
||||
static void vectorConst(
|
||||
const PODArray<T> & data, const ColumnArray::Offsets_t & offsets,
|
||||
const ColumnArray::Offset_t index,
|
||||
PODArray<T> & result)
|
||||
@ -213,12 +214,55 @@ struct ArrayElementNumImpl
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
template <typename index_type>
|
||||
static void vector(
|
||||
const PODArray<T> & data, const ColumnArray::Offsets_t & offsets,
|
||||
const ColumnVector<index_type> & index,
|
||||
PODArray<T> & result)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result.resize(size);
|
||||
|
||||
ColumnArray::Offset_t current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
|
||||
if (index[i].getType() == Field::Types::UInt64)
|
||||
{
|
||||
UInt64 cur_id = safeGet<UInt64>(index[i]);
|
||||
if (cur_id == 0)
|
||||
throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
|
||||
else if (cur_id <= array_size)
|
||||
result[i] = data[current_offset + cur_id - 1];
|
||||
else
|
||||
result[i] = T();
|
||||
}
|
||||
else if (index[i].getType() == Field::Types::Int64)
|
||||
{
|
||||
Int64 cur_id = safeGet<Int64>(index[i]);
|
||||
if (cur_id == 0)
|
||||
throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
|
||||
else if (cur_id > 0 && cur_id <= array_size)
|
||||
result[i] = data[current_offset + cur_id - 1];
|
||||
else if (cur_id < 0 && -cur_id <= array_size)
|
||||
result[i] = data[offsets[i] + cur_id];
|
||||
else
|
||||
result[i] = T();
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <bool negative>
|
||||
struct ArrayElementStringImpl
|
||||
{
|
||||
static void vector(
|
||||
template <bool negative>
|
||||
static void vectorConst(
|
||||
const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets,
|
||||
const ColumnArray::Offset_t index,
|
||||
ColumnString::Chars_t & result_data, ColumnArray::Offsets_t & result_offsets)
|
||||
@ -260,6 +304,70 @@ struct ArrayElementStringImpl
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
template <typename index_type>
|
||||
static void vector(
|
||||
const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets,
|
||||
const ColumnVector<index_type> & index,
|
||||
ColumnString::Chars_t & result_data, ColumnArray::Offsets_t & result_offsets)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result_offsets.resize(size);
|
||||
result_data.reserve(data.size());
|
||||
|
||||
ColumnArray::Offset_t current_offset = 0;
|
||||
ColumnArray::Offset_t current_result_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
size_t adjusted_index;
|
||||
|
||||
if (index[i].getType() == Field::Types::UInt64)
|
||||
{
|
||||
UInt64 cur_id = safeGet<UInt64>(index[i]);
|
||||
if (cur_id == 0)
|
||||
adjusted_index = array_size; /// Индекс не вписывается в рамки массива, заменяем заведомо слишком большим
|
||||
else
|
||||
adjusted_index = cur_id - 1;
|
||||
}
|
||||
else if (index[i].getType() == Field::Types::Int64)
|
||||
{
|
||||
Int64 cur_id = safeGet<Int64>(index[i]);
|
||||
if (cur_id > 0 && cur_id <= array_size)
|
||||
adjusted_index = cur_id - 1;
|
||||
else if (cur_id < 0 && -cur_id <= array_size)
|
||||
adjusted_index = array_size + cur_id;
|
||||
else
|
||||
adjusted_index = array_size; /// Индекс не вписывается в рамки массива, заменяем слишком большим
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (adjusted_index < array_size)
|
||||
{
|
||||
ColumnArray::Offset_t string_pos = current_offset == 0 && adjusted_index == 0
|
||||
? 0
|
||||
: string_offsets[current_offset + adjusted_index - 1];
|
||||
|
||||
ColumnArray::Offset_t string_size = string_offsets[current_offset + adjusted_index] - string_pos;
|
||||
|
||||
result_data.resize(current_result_offset + string_size);
|
||||
memcpy(&result_data[current_result_offset], &data[string_pos], string_size);
|
||||
current_result_offset += string_size;
|
||||
result_offsets[i] = current_result_offset;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Вставим пустую строку.
|
||||
result_data.resize(current_result_offset + 1);
|
||||
result_data[current_result_offset] = 0;
|
||||
current_result_offset += 1;
|
||||
result_offsets[i] = current_result_offset;
|
||||
}
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -267,7 +375,7 @@ class FunctionArrayElement : public IFunction
|
||||
{
|
||||
private:
|
||||
template <typename T>
|
||||
bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
{
|
||||
const ColumnArray * col_array = dynamic_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
@ -283,16 +391,37 @@ private:
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
if (index.getType() == Field::Types::UInt64)
|
||||
ArrayElementNumImpl<T, false>::vector(col_nested->getData(), col_array->getOffsets(), safeGet<UInt64>(index) - 1, col_res->getData());
|
||||
ArrayElementNumImpl<T>::template vectorConst<false>(col_nested->getData(), col_array->getOffsets(), safeGet<UInt64>(index) - 1, col_res->getData());
|
||||
else if (index.getType() == Field::Types::Int64)
|
||||
ArrayElementNumImpl<T, true>::vector(col_nested->getData(), col_array->getOffsets(), -safeGet<Int64>(index) - 1, col_res->getData());
|
||||
ArrayElementNumImpl<T>::template vectorConst<true>(col_nested->getData(), col_array->getOffsets(), -safeGet<Int64>(index) - 1, col_res->getData());
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
template <typename index_type, typename data_type>
|
||||
bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<index_type> & index)
|
||||
{
|
||||
const ColumnArray * col_array = dynamic_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const ColumnVector<data_type> * col_nested = dynamic_cast<const ColumnVector<data_type> *>(&col_array->getData());
|
||||
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
ColumnVector<data_type> * col_res = new ColumnVector<data_type>;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ArrayElementNumImpl<data_type>::template vector<index_type>(col_nested->getData(), col_array->getOffsets(), index, col_res->getData());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
{
|
||||
const ColumnArray * col_array = dynamic_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
@ -308,7 +437,7 @@ private:
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
if (index.getType() == Field::Types::UInt64)
|
||||
ArrayElementStringImpl<false>::vector(
|
||||
ArrayElementStringImpl::vectorConst<false>(
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
@ -316,7 +445,7 @@ private:
|
||||
col_res->getChars(),
|
||||
col_res->getOffsets());
|
||||
else if (index.getType() == Field::Types::Int64)
|
||||
ArrayElementStringImpl<true>::vector(
|
||||
ArrayElementStringImpl::vectorConst<true>(
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
@ -329,7 +458,34 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
template <typename index_type>
|
||||
bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<index_type> & index)
|
||||
{
|
||||
const ColumnArray * col_array = dynamic_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const ColumnString * col_nested = dynamic_cast<const ColumnString *>(&col_array->getData());
|
||||
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
ColumnString * col_res = new ColumnString;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ArrayElementStringImpl::vector<index_type>(
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
index,
|
||||
col_res->getChars(),
|
||||
col_res->getOffsets());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeConstConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
{
|
||||
const ColumnConstArray * col_array = dynamic_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
@ -356,6 +512,72 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename index_type>
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<index_type> & index)
|
||||
{
|
||||
const ColumnConstArray * col_array = dynamic_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const DB::Array & array = col_array->getData();
|
||||
size_t array_size = array.size();
|
||||
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createColumn();
|
||||
|
||||
for (size_t i = 0; i < col_array->size(); ++i)
|
||||
{
|
||||
if (index[i].getType() == Field::Types::UInt64)
|
||||
{
|
||||
UInt64 cur_id = safeGet<UInt64>(index[i]);
|
||||
if (cur_id > 0 && cur_id <= array_size)
|
||||
block.getByPosition(result).column->insert(array[cur_id - 1]);
|
||||
else
|
||||
block.getByPosition(result).column->insertDefault();
|
||||
}
|
||||
else if (index[i].getType() == Field::Types::Int64)
|
||||
{
|
||||
Int64 cur_id = safeGet<Int64>(index[i]);
|
||||
if (cur_id > 0 && cur_id <= array_size)
|
||||
block.getByPosition(result).column->insert(array[cur_id - 1]);
|
||||
else if (cur_id < 0 && -cur_id <= array_size)
|
||||
block.getByPosition(result).column->insert(array[array_size + cur_id]);
|
||||
else
|
||||
block.getByPosition(result).column->insertDefault();
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename index_type>
|
||||
bool executeArgument(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnVector<index_type> * index = dynamic_cast<const ColumnVector<index_type> *> (&*block.getByPosition(arguments[1]).column);
|
||||
|
||||
if (!index)
|
||||
return false;
|
||||
|
||||
if (!( executeNumber<index_type, UInt8> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, UInt16> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, UInt32> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, UInt64> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, Int8> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, Int16> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, Int32> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, Int64> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, Float32> (block, arguments, result, *index)
|
||||
|| executeNumber<index_type, Float64> (block, arguments, result, *index)
|
||||
|| executeConst <index_type> (block, arguments, result, *index)
|
||||
|| executeString<index_type> (block, arguments, result, *index)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
@ -386,28 +608,41 @@ public:
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (!block.getByPosition(arguments[1]).column->isConst())
|
||||
throw Exception("Second argument for function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
{
|
||||
if (!( executeArgument<UInt8> (block, arguments, result)
|
||||
|| executeArgument<UInt16> (block, arguments, result)
|
||||
|| executeArgument<UInt32> (block, arguments, result)
|
||||
|| executeArgument<UInt64> (block, arguments, result)
|
||||
|| executeArgument<Int8> (block, arguments, result)
|
||||
|| executeArgument<Int16> (block, arguments, result)
|
||||
|| executeArgument<Int32> (block, arguments, result)
|
||||
|| executeArgument<Int64> (block, arguments, result)))
|
||||
throw Exception("Second argument for function " + getName() + " must must have UInt or Int type.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
{
|
||||
Field index = (*block.getByPosition(arguments[1]).column)[0];
|
||||
|
||||
Field index = (*block.getByPosition(arguments[1]).column)[0];
|
||||
if (index == UInt64(0))
|
||||
throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
|
||||
|
||||
if (index == UInt64(0))
|
||||
throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
|
||||
|
||||
if (!( executeNumber<UInt8> (block, arguments, result, index)
|
||||
|| executeNumber<UInt16> (block, arguments, result, index)
|
||||
|| executeNumber<UInt32> (block, arguments, result, index)
|
||||
|| executeNumber<UInt64> (block, arguments, result, index)
|
||||
|| executeNumber<Int8> (block, arguments, result, index)
|
||||
|| executeNumber<Int16> (block, arguments, result, index)
|
||||
|| executeNumber<Int32> (block, arguments, result, index)
|
||||
|| executeNumber<Int64> (block, arguments, result, index)
|
||||
|| executeNumber<Float32> (block, arguments, result, index)
|
||||
|| executeNumber<Float64> (block, arguments, result, index)
|
||||
|| executeConst (block, arguments, result, index)
|
||||
|| executeString (block, arguments, result, index)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (!( executeNumberConst<UInt8> (block, arguments, result, index)
|
||||
|| executeNumberConst<UInt16> (block, arguments, result, index)
|
||||
|| executeNumberConst<UInt32> (block, arguments, result, index)
|
||||
|| executeNumberConst<UInt64> (block, arguments, result, index)
|
||||
|| executeNumberConst<Int8> (block, arguments, result, index)
|
||||
|| executeNumberConst<Int16> (block, arguments, result, index)
|
||||
|| executeNumberConst<Int32> (block, arguments, result, index)
|
||||
|| executeNumberConst<Int64> (block, arguments, result, index)
|
||||
|| executeNumberConst<Float32> (block, arguments, result, index)
|
||||
|| executeNumberConst<Float64> (block, arguments, result, index)
|
||||
|| executeConstConst (block, arguments, result, index)
|
||||
|| executeStringConst (block, arguments, result, index)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
30
dbms/tests/queries/0_stateless/00036_array_element.reference
Normal file
30
dbms/tests/queries/0_stateless/00036_array_element.reference
Normal file
@ -0,0 +1,30 @@
|
||||
12
|
||||
0
|
||||
13
|
||||
11
|
||||
0
|
||||
12
|
||||
0
|
||||
11
|
||||
0
|
||||
Df
|
||||
|
||||
ERT
|
||||
Ab
|
||||
|
||||
Df
|
||||
|
||||
ABC
|
||||
|
||||
[1,2,3] 2
|
||||
[1,2,3] 1
|
||||
[1,2,3] 0
|
||||
[1,2,3] 3
|
||||
[1,2,3] 2
|
||||
[1,2,3] 1
|
||||
[1,2,3] 0
|
||||
[1,2,3] 3
|
||||
[1,2,3] 2
|
||||
[1,2,3] 3
|
||||
[1,2,3] 0
|
||||
[1,2,3] 1
|
33
dbms/tests/queries/0_stateless/00036_array_element.sql
Normal file
33
dbms/tests/queries/0_stateless/00036_array_element.sql
Normal file
@ -0,0 +1,33 @@
|
||||
DROP TABLE IF EXISTS array_element_test;
|
||||
CREATE TABLE array_element_test (arr Array(Int32), id Int32) ENGINE = Memory;
|
||||
insert into array_element_test VALUES ([11,12,13], 2), ([11,12], 3), ([11,12,13], -1), ([11,12], -2), ([11,12], -3);
|
||||
select arr[id] from array_element_test;
|
||||
|
||||
DROP TABLE IF EXISTS array_element_test;
|
||||
CREATE TABLE array_element_test (arr Array(Int32), id UInt32) ENGINE = Memory;
|
||||
insert into array_element_test VALUES ([11,12,13], 2), ([11,12], 3), ([11,12,13], 1), ([11,12], 4);
|
||||
select arr[id] from array_element_test;
|
||||
|
||||
DROP TABLE IF EXISTS array_element_test;
|
||||
CREATE TABLE array_element_test (arr Array(String), id Int32) ENGINE = Memory;
|
||||
insert into array_element_test VALUES (['Abc','Df','Q'], 2), (['Abc','DEFQ'], 3), (['ABC','Q','ERT'], -1), (['Ab','ber'], -2), (['AB','asd'], -3);
|
||||
select arr[id] from array_element_test;
|
||||
|
||||
DROP TABLE IF EXISTS array_element_test;
|
||||
CREATE TABLE array_element_test (arr Array(String), id UInt32) ENGINE = Memory;
|
||||
insert into array_element_test VALUES (['Abc','Df','Q'], 2), (['Abc','DEFQ'], 3), (['ABC','Q','ERT'], 1), (['Ab','ber'], 4);
|
||||
select arr[id] from array_element_test;
|
||||
|
||||
DROP TABLE IF EXISTS array_element_test;
|
||||
CREATE TABLE array_element_test (id UInt32) ENGINE = Memory;
|
||||
insert into array_element_test VALUES (2), (1), (4), (3);
|
||||
select [1, 2, 3] as arr, arr[id] from array_element_test;
|
||||
|
||||
DROP TABLE IF EXISTS array_element_test;
|
||||
CREATE TABLE array_element_test (id Int32) ENGINE = Memory;
|
||||
insert into array_element_test VALUES (-2), (1), (-4), (3), (2), (-1), (4), (-3);
|
||||
select [1, 2, 3] as arr, arr[id] from array_element_test;
|
||||
|
||||
DROP TABLE IF EXISTS array_element_test;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user