mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Implemented function arrayElement for generic case; partially removed bad code [#METR-22008].
This commit is contained in:
parent
f8243864a2
commit
ea0b6b3669
@ -83,7 +83,7 @@ private:
|
||||
template <typename T0, typename T1>
|
||||
bool tryAddField(DataTypePtr type_res, const Field & f, Array & arr) const
|
||||
{
|
||||
if (typeid_cast<const T0 *>(&*type_res))
|
||||
if (typeid_cast<const T0 *>(type_res.get()))
|
||||
{
|
||||
arr.push_back(apply_visitor(FieldVisitorConvertToNumber<typename T1::FieldType>(), f));
|
||||
return true;
|
||||
@ -286,9 +286,9 @@ private:
|
||||
template <typename T>
|
||||
struct ArrayElementNumImpl
|
||||
{
|
||||
/** Процедура для константного идекса
|
||||
* Если negative = false - передаётся индекс с начала массива, начиная с нуля.
|
||||
* Если negative = true - передаётся индекс с конца массива, начиная с нуля.
|
||||
/** Implementation for constant index.
|
||||
* If negative = false - index is from beginning of array, started from 1.
|
||||
* Если negative = true - index is from end of array, started from -1.
|
||||
*/
|
||||
template <bool negative>
|
||||
static void vectorConst(
|
||||
@ -313,13 +313,12 @@ struct ArrayElementNumImpl
|
||||
}
|
||||
}
|
||||
|
||||
/** Процедура для неконстантного идекса
|
||||
* index_type - тип данных идекса
|
||||
/** Implementation for non-constant index.
|
||||
*/
|
||||
template <typename index_type>
|
||||
template <typename TIndex>
|
||||
static void vector(
|
||||
const PaddedPODArray<T> & data, const ColumnArray::Offsets_t & offsets,
|
||||
const ColumnVector<index_type> & index,
|
||||
const PaddedPODArray<TIndex> & indices,
|
||||
PaddedPODArray<T> & result)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
@ -330,26 +329,13 @@ struct ArrayElementNumImpl
|
||||
{
|
||||
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 && 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 && static_cast<UInt64>(cur_id) <= array_size)
|
||||
result[i] = data[current_offset + cur_id - 1];
|
||||
else if (cur_id < 0 && static_cast<UInt64>(-cur_id) <= array_size)
|
||||
result[i] = data[offsets[i] + cur_id];
|
||||
else
|
||||
result[i] = T();
|
||||
}
|
||||
TIndex index = indices[i];
|
||||
if (index > 0 && static_cast<size_t>(index) <= array_size)
|
||||
result[i] = data[current_offset + index - 1];
|
||||
else if (index < 0 && static_cast<size_t>(-index) <= array_size)
|
||||
result[i] = data[offsets[i] + index];
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
result[i] = T();
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
@ -358,9 +344,9 @@ struct ArrayElementNumImpl
|
||||
|
||||
struct ArrayElementStringImpl
|
||||
{
|
||||
/** Процедура для константного идекса
|
||||
* Если negative = false - передаётся индекс с начала массива, начиная с нуля.
|
||||
* Если negative = true - передаётся индекс с конца массива, начиная с нуля.
|
||||
/** Implementation for constant index.
|
||||
* If negative = false - index is from beginning of array, started from 1.
|
||||
* Если negative = true - index is from end of array, started from -1.
|
||||
*/
|
||||
template <bool negative>
|
||||
static void vectorConst(
|
||||
@ -406,13 +392,12 @@ struct ArrayElementStringImpl
|
||||
}
|
||||
}
|
||||
|
||||
/** Процедура для неконстантного идекса
|
||||
* index_type - тип данных идекса
|
||||
/** Implementation for non-constant index.
|
||||
*/
|
||||
template <typename index_type>
|
||||
template <typename TIndex>
|
||||
static void vector(
|
||||
const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets,
|
||||
const ColumnVector<index_type> & index,
|
||||
const PaddedPODArray<TIndex> & indices,
|
||||
ColumnString::Chars_t & result_data, ColumnArray::Offsets_t & result_offsets)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
@ -424,28 +409,15 @@ struct ArrayElementStringImpl
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
size_t adjusted_index;
|
||||
size_t adjusted_index; /// index in array from zero
|
||||
|
||||
if (index[i].getType() == Field::Types::UInt64)
|
||||
{
|
||||
UInt64 cur_id = safeGet<UInt64>(index[i]);
|
||||
if (cur_id > 0 && cur_id <= array_size)
|
||||
adjusted_index = cur_id - 1;
|
||||
else
|
||||
adjusted_index = array_size; /// Индекс не вписывается в рамки массива, заменяем заведомо слишком большим
|
||||
}
|
||||
else if (index[i].getType() == Field::Types::Int64)
|
||||
{
|
||||
Int64 cur_id = safeGet<Int64>(index[i]);
|
||||
if (cur_id > 0 && static_cast<UInt64>(cur_id) <= array_size)
|
||||
adjusted_index = cur_id - 1;
|
||||
else if (cur_id < 0 && static_cast<UInt64>(-cur_id) <= array_size)
|
||||
adjusted_index = array_size + cur_id;
|
||||
else
|
||||
adjusted_index = array_size; /// Индекс не вписывается в рамки массива, заменяем слишком большим
|
||||
}
|
||||
TIndex index = indices[i];
|
||||
if (index > 0 && static_cast<size_t>(index) <= array_size)
|
||||
adjusted_index = index - 1;
|
||||
else if (index < 0 && static_cast<size_t>(-index) <= array_size)
|
||||
adjusted_index = array_size + index;
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
adjusted_index = array_size; /// means no element should be taken
|
||||
|
||||
if (adjusted_index < array_size)
|
||||
{
|
||||
@ -462,7 +434,7 @@ struct ArrayElementStringImpl
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Вставим пустую строку.
|
||||
/// Insert empty string
|
||||
result_data.resize(current_result_offset + 1);
|
||||
result_data[current_result_offset] = 0;
|
||||
current_result_offset += 1;
|
||||
@ -474,6 +446,65 @@ struct ArrayElementStringImpl
|
||||
}
|
||||
};
|
||||
|
||||
/// Generic implementation for other nested types.
|
||||
struct ArrayElementGenericImpl
|
||||
{
|
||||
/** Implementation for constant index.
|
||||
* If negative = false - index is from beginning of array, started from 1.
|
||||
* Если negative = true - index is from end of array, started from -1.
|
||||
*/
|
||||
template <bool negative>
|
||||
static void vectorConst(
|
||||
const IColumn & data, const ColumnArray::Offsets_t & offsets,
|
||||
const ColumnArray::Offset_t index,
|
||||
IColumn & result)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result.reserve(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 < array_size)
|
||||
result.insertFrom(data, !negative ? current_offset + index : offsets[i] - index - 1);
|
||||
else
|
||||
result.insertDefault();
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
/** Implementation for non-constant index.
|
||||
*/
|
||||
template <typename TIndex>
|
||||
static void vector(
|
||||
const IColumn & data, const ColumnArray::Offsets_t & offsets,
|
||||
const PaddedPODArray<TIndex> & indices,
|
||||
IColumn & result)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result.reserve(size);
|
||||
|
||||
ColumnArray::Offset_t current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
|
||||
TIndex index = indices[i];
|
||||
if (index > 0 && static_cast<size_t>(index) <= array_size)
|
||||
result.insertFrom(data, current_offset + index - 1);
|
||||
else if (index < 0 && static_cast<size_t>(-index) <= array_size)
|
||||
result.insertFrom(data, offsets[i] + index);
|
||||
else
|
||||
result.insertDefault();
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionArrayElement : public IFunction
|
||||
{
|
||||
@ -482,56 +513,59 @@ public:
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayElement>(); }
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
template <typename DataType>
|
||||
bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
{
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const ColumnVector<T> * col_nested = typeid_cast<const ColumnVector<T> *>(&col_array->getData());
|
||||
const ColumnVector<DataType> * col_nested = typeid_cast<const ColumnVector<DataType> *>(&col_array->getData());
|
||||
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
auto col_res = std::make_shared<ColumnVector<T>>();
|
||||
auto col_res = std::make_shared<ColumnVector<DataType>>();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
if (index.getType() == Field::Types::UInt64)
|
||||
ArrayElementNumImpl<T>::template vectorConst<false>(col_nested->getData(), col_array->getOffsets(), safeGet<UInt64>(index) - 1, col_res->getData());
|
||||
ArrayElementNumImpl<DataType>::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>::template vectorConst<true>(col_nested->getData(), col_array->getOffsets(), -safeGet<Int64>(index) - 1, col_res->getData());
|
||||
ArrayElementNumImpl<DataType>::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;
|
||||
}
|
||||
|
||||
template <typename index_type, typename data_type>
|
||||
bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<index_type> & index)
|
||||
template <typename IndexType, typename DataType>
|
||||
bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices)
|
||||
{
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const ColumnVector<data_type> * col_nested = typeid_cast<const ColumnVector<data_type> *>(&col_array->getData());
|
||||
const ColumnVector<DataType> * col_nested = typeid_cast<const ColumnVector<DataType> *>(&col_array->getData());
|
||||
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
auto col_res = std::make_shared<ColumnVector<data_type>>();
|
||||
auto col_res = std::make_shared<ColumnVector<DataType>>();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ArrayElementNumImpl<data_type>::template vector<index_type>(col_nested->getData(), col_array->getOffsets(), index, col_res->getData());
|
||||
ArrayElementNumImpl<DataType>::template vector<IndexType>(
|
||||
col_nested->getData(), col_array->getOffsets(), indices, col_res->getData());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
{
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -566,10 +600,10 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename index_type>
|
||||
bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<index_type> & index)
|
||||
template <typename IndexType>
|
||||
bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices)
|
||||
{
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -582,20 +616,61 @@ private:
|
||||
std::shared_ptr<ColumnString> col_res = std::make_shared<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());
|
||||
ArrayElementStringImpl::vector<IndexType>(
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
indices,
|
||||
col_res->getChars(),
|
||||
col_res->getOffsets());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeGenericConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
{
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const auto & col_nested = col_array->getData();
|
||||
auto col_res = col_nested.cloneEmpty();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
if (index.getType() == Field::Types::UInt64)
|
||||
ArrayElementGenericImpl::vectorConst<false>(
|
||||
col_nested, col_array->getOffsets(), safeGet<UInt64>(index) - 1, *col_res);
|
||||
else if (index.getType() == Field::Types::Int64)
|
||||
ArrayElementGenericImpl::vectorConst<true>(
|
||||
col_nested, col_array->getOffsets(), -safeGet<Int64>(index) - 1, *col_res);
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename IndexType>
|
||||
bool executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices)
|
||||
{
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const auto & col_nested = col_array->getData();
|
||||
auto col_res = col_nested.cloneEmpty();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ArrayElementGenericImpl::vector<IndexType>(
|
||||
col_nested, col_array->getOffsets(), indices, *col_res);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeConstConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index)
|
||||
{
|
||||
const ColumnConstArray * col_array = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnConstArray * col_array = typeid_cast<const ColumnConstArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -620,10 +695,10 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename index_type>
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<index_type> & index)
|
||||
template <typename IndexType>
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices)
|
||||
{
|
||||
const ColumnConstArray * col_array = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnConstArray * col_array = typeid_cast<const ColumnConstArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -635,51 +710,41 @@ private:
|
||||
|
||||
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 && static_cast<UInt64>(cur_id) <= array_size)
|
||||
block.getByPosition(result).column->insert(array[cur_id - 1]);
|
||||
else if (cur_id < 0 && static_cast<UInt64>(-cur_id) <= array_size)
|
||||
block.getByPosition(result).column->insert(array[array_size + cur_id]);
|
||||
else
|
||||
block.getByPosition(result).column->insertDefault();
|
||||
}
|
||||
IndexType index = indices[i];
|
||||
if (index > 0 && static_cast<size_t>(index) <= array_size)
|
||||
block.getByPosition(result).column->insert(array[index - 1]);
|
||||
else if (index < 0 && static_cast<size_t>(-index) <= array_size)
|
||||
block.getByPosition(result).column->insert(array[array_size + index]);
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
block.getByPosition(result).column->insertDefault();
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename index_type>
|
||||
template <typename IndexType>
|
||||
bool executeArgument(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnVector<index_type> * index = typeid_cast<const ColumnVector<index_type> *> (&*block.getByPosition(arguments[1]).column);
|
||||
auto index = typeid_cast<const ColumnVector<IndexType> *>(block.getByPosition(arguments[1]).column.get());
|
||||
|
||||
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)))
|
||||
const auto & index_data = index->getData();
|
||||
|
||||
if (!( executeNumber<IndexType, UInt8> (block, arguments, result, index_data)
|
||||
|| executeNumber<IndexType, UInt16> (block, arguments, result, index_data)
|
||||
|| executeNumber<IndexType, UInt32> (block, arguments, result, index_data)
|
||||
|| executeNumber<IndexType, UInt64> (block, arguments, result, index_data)
|
||||
|| executeNumber<IndexType, Int8> (block, arguments, result, index_data)
|
||||
|| executeNumber<IndexType, Int16> (block, arguments, result, index_data)
|
||||
|| executeNumber<IndexType, Int32> (block, arguments, result, index_data)
|
||||
|| executeNumber<IndexType, Int64> (block, arguments, result, index_data)
|
||||
|| executeNumber<IndexType, Float32> (block, arguments, result, index_data)
|
||||
|| executeNumber<IndexType, Float64> (block, arguments, result, index_data)
|
||||
|| executeConst <IndexType> (block, arguments, result, index_data)
|
||||
|| executeString<IndexType> (block, arguments, result, index_data)
|
||||
|| executeGeneric<IndexType> (block, arguments, result, index_data)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
@ -690,7 +755,7 @@ private:
|
||||
*/
|
||||
bool executeTuple(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
ColumnArray * col_array = typeid_cast<ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
ColumnArray * col_array = typeid_cast<ColumnArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -722,8 +787,10 @@ private:
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
{
|
||||
ColumnWithTypeAndName array_of_tuple_section;
|
||||
array_of_tuple_section.column = std::make_shared<ColumnArray>(tuple_block.getByPosition(i).column, col_array->getOffsetsColumn());
|
||||
array_of_tuple_section.type = std::make_shared<DataTypeArray>(tuple_block.getByPosition(i).type);
|
||||
array_of_tuple_section.column = std::make_shared<ColumnArray>(
|
||||
tuple_block.getByPosition(i).column, col_array->getOffsetsColumn());
|
||||
array_of_tuple_section.type = std::make_shared<DataTypeArray>(
|
||||
tuple_block.getByPosition(i).type);
|
||||
block_of_temporary_results.insert(array_of_tuple_section);
|
||||
|
||||
ColumnWithTypeAndName array_elements_of_tuple_section;
|
||||
@ -754,7 +821,7 @@ public:
|
||||
+ toString(arguments.size()) + ", should be 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(&*arguments[0]);
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("First argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -802,10 +869,11 @@ public:
|
||||
|| executeNumberConst<Float32> (block, arguments, result, index)
|
||||
|| executeNumberConst<Float64> (block, arguments, result, index)
|
||||
|| executeConstConst (block, arguments, result, index)
|
||||
|| executeStringConst (block, arguments, result, index)))
|
||||
|| executeStringConst (block, arguments, result, index)
|
||||
|| executeGenericConst (block, arguments, result, index)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
};
|
||||
@ -977,7 +1045,7 @@ private:
|
||||
template <typename T, typename U>
|
||||
bool executeNumberNumber(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -1013,7 +1081,7 @@ private:
|
||||
|
||||
bool executeString(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -1048,7 +1116,7 @@ private:
|
||||
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnConstArray * col_array = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnConstArray * col_array = typeid_cast<const ColumnConstArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -1113,7 +1181,7 @@ public:
|
||||
+ toString(arguments.size()) + ", should be 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(&*arguments[0]);
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("First argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -1168,7 +1236,7 @@ public:
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(&*arguments[0]);
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("First argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -1178,7 +1246,7 @@ public:
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (const ColumnArray * array = typeid_cast<const ColumnArray *>(&*block.getByPosition(arguments[0]).column))
|
||||
if (const ColumnArray * array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
|
||||
@ -1199,7 +1267,7 @@ public:
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
else if (const ColumnConstArray * array = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[0]).column))
|
||||
else if (const ColumnConstArray * array = typeid_cast<const ColumnConstArray *>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
const Array & values = array->getData();
|
||||
|
||||
@ -1246,7 +1314,7 @@ public:
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(&*arguments[i]);
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(arguments[i].get());
|
||||
if (!array_type)
|
||||
throw Exception("All arguments for function " + getName() + " must be arrays; argument " + toString(i + 1) + " isn't.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -1268,16 +1336,17 @@ public:
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
ColumnPtr array_ptr = block.getByPosition(arguments[i]).column;
|
||||
const ColumnArray * array = typeid_cast<const ColumnArray *>(&*array_ptr);
|
||||
const ColumnArray * array = typeid_cast<const ColumnArray *>(array_ptr.get());
|
||||
if (!array)
|
||||
{
|
||||
const ColumnConstArray * const_array = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[i]).column);
|
||||
const ColumnConstArray * const_array = typeid_cast<const ColumnConstArray *>(
|
||||
block.getByPosition(arguments[i]).column.get());
|
||||
if (!const_array)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName()
|
||||
+ " of " + toString(i + 1) + "-th argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
array_ptr = const_array->convertToFullColumn();
|
||||
array = typeid_cast<const ColumnArray *>(&*array_ptr);
|
||||
array = typeid_cast<const ColumnArray *>(array_ptr.get());
|
||||
}
|
||||
array_columns[i] = array_ptr;
|
||||
const ColumnArray::Offsets_t & offsets_i = array->getOffsets();
|
||||
@ -1289,7 +1358,7 @@ public:
|
||||
data_columns[i] = &array->getData();
|
||||
}
|
||||
|
||||
const ColumnArray * first_array = typeid_cast<const ColumnArray *>(&*array_columns[0]);
|
||||
const ColumnArray * first_array = typeid_cast<const ColumnArray *>(array_columns[0].get());
|
||||
auto res = std::make_shared<ColumnUInt32>();
|
||||
block.getByPosition(result).column = res;
|
||||
|
||||
@ -1378,7 +1447,7 @@ private:
|
||||
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnConstArray * array = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnConstArray * array = typeid_cast<const ColumnConstArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
if (!array)
|
||||
return false;
|
||||
const Array & values = array->getData();
|
||||
@ -1476,7 +1545,7 @@ public:
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(&*arguments[i]);
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(arguments[i].get());
|
||||
if (!array_type)
|
||||
throw Exception("All arguments for function " + getName() + " must be arrays; argument " + toString(i + 1) + " isn't.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -1498,16 +1567,17 @@ public:
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
ColumnPtr array_ptr = block.getByPosition(arguments[i]).column;
|
||||
const ColumnArray * array = typeid_cast<const ColumnArray *>(&*array_ptr);
|
||||
const ColumnArray * array = typeid_cast<const ColumnArray *>(array_ptr.get());
|
||||
if (!array)
|
||||
{
|
||||
const ColumnConstArray * const_array = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[i]).column);
|
||||
const ColumnConstArray * const_array = typeid_cast<const ColumnConstArray *>(
|
||||
block.getByPosition(arguments[i]).column.get());
|
||||
if (!const_array)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName()
|
||||
+ " of " + toString(i + 1) + "-th argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
array_ptr = const_array->convertToFullColumn();
|
||||
array = typeid_cast<const ColumnArray *>(&*array_ptr);
|
||||
array = typeid_cast<const ColumnArray *>(array_ptr.get());
|
||||
}
|
||||
array_columns[i] = array_ptr;
|
||||
const ColumnArray::Offsets_t & offsets_i = array->getOffsets();
|
||||
@ -1519,7 +1589,7 @@ public:
|
||||
data_columns[i] = &array->getData();
|
||||
}
|
||||
|
||||
const ColumnArray * first_array = typeid_cast<const ColumnArray *>(&*array_columns[0]);
|
||||
const ColumnArray * first_array = typeid_cast<const ColumnArray *>(array_columns[0].get());
|
||||
auto res_nested = std::make_shared<ColumnUInt32>();
|
||||
auto res_array = std::make_shared<ColumnArray>(res_nested, first_array->getOffsetsColumn());
|
||||
block.getByPosition(result).column = res_array;
|
||||
@ -1610,7 +1680,7 @@ private:
|
||||
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnConstArray * array = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnConstArray * array = typeid_cast<const ColumnConstArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
if (!array)
|
||||
return false;
|
||||
const Array & values = array->getData();
|
||||
|
@ -0,0 +1,132 @@
|
||||
0
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
||||
7
|
||||
8
|
||||
0
|
||||
0
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
||||
7
|
||||
8
|
||||
0
|
||||
0
|
||||
0
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
|
||||
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
|
||||
0
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
||||
7
|
||||
8
|
||||
|
||||
0
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
||||
7
|
||||
8
|
||||
|
||||
0
|
||||
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
[]
|
||||
[]
|
||||
[0]
|
||||
[0]
|
||||
[0]
|
||||
[0]
|
||||
[0]
|
||||
[0]
|
||||
[0]
|
||||
[0]
|
||||
[]
|
||||
[]
|
||||
[0]
|
||||
[0,1]
|
||||
[0,1,2]
|
||||
[0,1,2,3]
|
||||
[0,1,2,3,4]
|
||||
[0,1,2,3,4,5]
|
||||
[0,1,2,3,4,5,6]
|
||||
[0,1,2,3,4,5,6,7]
|
||||
[]
|
||||
[]
|
||||
[0]
|
||||
[0,1]
|
||||
[0,1,2]
|
||||
[0,1,2,3]
|
||||
[0,1,2,3,4]
|
||||
[0,1,2,3,4,5]
|
||||
[0,1,2,3,4,5,6]
|
||||
[0,1,2,3,4,5,6,7]
|
||||
[]
|
||||
[]
|
||||
[]
|
||||
[0,1]
|
||||
[0,1]
|
||||
[0,1]
|
||||
[0,1]
|
||||
[0,1]
|
||||
[0,1]
|
||||
[0,1]
|
||||
[1] [1] [1] [1]
|
||||
['Hello'] ['World'] ['Hello'] ['World']
|
||||
[] []
|
||||
[['a'],['b','c']] ['a'] a
|
||||
[['d','e','f'],['g','h','i','j'],['k','l','m','n','o']] ['g','h','i','j'] h
|
||||
[['p','q','r','s','t','u'],['v','w','x','y','z','aa','bb'],['cc','dd','ee','ff','gg','hh','ii','jj'],['kk','ll','mm','nn','oo','pp','qq','rr','ss']] ['cc','dd','ee','ff','gg','hh','ii','jj'] ee
|
||||
[] []
|
||||
[] []
|
||||
[] []
|
||||
[] []
|
||||
[] []
|
||||
[] []
|
@ -0,0 +1,19 @@
|
||||
SELECT range(number)[2] FROM system.numbers LIMIT 10;
|
||||
SELECT range(number)[-1] FROM system.numbers LIMIT 10;
|
||||
SELECT range(number)[number] FROM system.numbers LIMIT 10;
|
||||
SELECT range(number)[2 - number] FROM system.numbers LIMIT 10;
|
||||
|
||||
SELECT arrayMap(x -> toString(x), range(number))[2] FROM system.numbers LIMIT 10;
|
||||
SELECT arrayMap(x -> toString(x), range(number))[-1] FROM system.numbers LIMIT 10;
|
||||
SELECT arrayMap(x -> toString(x), range(number))[number] FROM system.numbers LIMIT 10;
|
||||
SELECT arrayMap(x -> toString(x), range(number))[2 - number] FROM system.numbers LIMIT 10;
|
||||
|
||||
SELECT arrayMap(x -> range(x), range(number))[2] FROM system.numbers LIMIT 10;
|
||||
SELECT arrayMap(x -> range(x), range(number))[-1] FROM system.numbers LIMIT 10;
|
||||
SELECT arrayMap(x -> range(x), range(number))[number] FROM system.numbers LIMIT 10;
|
||||
SELECT arrayMap(x -> range(x), range(number))[2 - number] FROM system.numbers LIMIT 10;
|
||||
|
||||
SELECT [[1]][1], materialize([[1]])[1], [[1]][materialize(1)], materialize([[1]])[materialize(1)];
|
||||
SELECT [['Hello']][1], materialize([['World']])[1], [['Hello']][materialize(1)], materialize([['World']])[materialize(1)];
|
||||
|
||||
SELECT ([[['a'], ['b', 'c']], [['d', 'e', 'f'], ['g', 'h', 'i', 'j'], ['k', 'l', 'm', 'n', 'o']], [['p', 'q', 'r', 's', 't', 'u'], ['v', 'w', 'x', 'y', 'z', 'aa', 'bb'], ['cc', 'dd', 'ee', 'ff', 'gg', 'hh', 'ii', 'jj'], ['kk', 'll', 'mm', 'nn', 'oo', 'pp', 'qq', 'rr', 'ss']]] AS arr)[number], arr[number][number], arr[number][number][number] FROM system.numbers LIMIT 10;
|
Loading…
Reference in New Issue
Block a user