mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
dbms: added support for Array arguments of function 'if' (incomplete) [#METR-16700].
This commit is contained in:
parent
73072b58c7
commit
bb83c867fd
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
#include <DB/Columns/ColumnVector.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
@ -14,7 +15,7 @@ namespace DB
|
||||
|
||||
/** Функция выбора по условию: if(cond, then, else).
|
||||
* cond - UInt8
|
||||
* then, else - либо числа/даты/даты-с-временем, либо строки.
|
||||
* then, else - числовые типы, для которых есть общий тип, либо даты, даты-с-временем, либо строки, либо массивы таких типов.
|
||||
*/
|
||||
|
||||
|
||||
@ -275,6 +276,224 @@ struct StringIfImpl
|
||||
};
|
||||
|
||||
|
||||
template <typename A, typename B, typename ResultType>
|
||||
struct NumArrayIfImpl
|
||||
{
|
||||
template <typename FromT>
|
||||
static ALWAYS_INLINE void copy_from_vector(
|
||||
size_t i,
|
||||
const PODArray<FromT> & from_data, const ColumnArray::Offsets_t & from_offsets, ColumnArray::Offset_t from_prev_offset,
|
||||
PODArray<ResultType> & to_data, ColumnArray::Offsets_t & to_offsets, ColumnArray::Offset_t & to_prev_offset)
|
||||
{
|
||||
size_t size_to_write = from_offsets[i] - from_prev_offset;
|
||||
to_data.resize(to_data.size() + size_to_write);
|
||||
|
||||
for (size_t i = 0; i < size_to_write; ++i)
|
||||
to_data[to_prev_offset + i] = static_cast<ResultType>(from_data[from_prev_offset + i]);
|
||||
|
||||
to_prev_offset += size_to_write;
|
||||
to_offsets[i] = to_prev_offset;
|
||||
}
|
||||
|
||||
template <typename FromT>
|
||||
static ALWAYS_INLINE void copy_from_constant(
|
||||
size_t i,
|
||||
const PODArray<FromT> & from_data,
|
||||
PODArray<ResultType> & to_data, ColumnArray::Offsets_t & to_offsets, ColumnArray::Offset_t & to_prev_offset)
|
||||
{
|
||||
size_t size_to_write = from_data.size();
|
||||
to_data.resize(to_data.size() + size_to_write);
|
||||
memcpy(&to_data[to_prev_offset], from_data.data(), size_to_write * sizeof(from_data[0]));
|
||||
to_prev_offset += size_to_write;
|
||||
to_offsets[i] = to_prev_offset;
|
||||
}
|
||||
|
||||
static void create_result_column(
|
||||
Block & block, size_t result,
|
||||
PODArray<ResultType> ** c_data, ColumnArray::Offsets_t ** c_offsets)
|
||||
{
|
||||
ColumnVector<ResultType> * col_res_vec = new ColumnVector<ResultType>;
|
||||
ColumnArray * col_res_array = new ColumnArray(col_res_vec);
|
||||
block.getByPosition(result).column = col_res_array;
|
||||
|
||||
*c_data = &col_res_vec->getData();
|
||||
*c_offsets = &col_res_array->getOffsets();
|
||||
}
|
||||
|
||||
|
||||
static void vector_vector(
|
||||
const PODArray<UInt8> & cond,
|
||||
const PODArray<A> & a_data, const ColumnArray::Offsets_t & a_offsets,
|
||||
const PODArray<B> & b_data, const ColumnArray::Offsets_t & b_offsets,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
PODArray<ResultType> * c_data = nullptr;
|
||||
ColumnArray::Offsets_t * c_offsets = nullptr;
|
||||
create_result_column(block, result, &c_data, &c_offsets);
|
||||
|
||||
size_t size = cond.size();
|
||||
c_offsets->resize(size);
|
||||
c_data->reserve(std::max(a_data.size(), b_data.size()));
|
||||
|
||||
ColumnArray::Offset_t a_prev_offset = 0;
|
||||
ColumnArray::Offset_t b_prev_offset = 0;
|
||||
ColumnArray::Offset_t c_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (cond[i])
|
||||
copy_from_vector(i, a_data, a_offsets, a_prev_offset, *c_data, *c_offsets, c_prev_offset);
|
||||
else
|
||||
copy_from_vector(i, b_data, b_offsets, b_prev_offset, *c_data, *c_offsets, c_prev_offset);
|
||||
|
||||
a_prev_offset = a_offsets[i];
|
||||
b_prev_offset = b_offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_constant(
|
||||
const PODArray<UInt8> & cond,
|
||||
const PODArray<A> & a_data, const ColumnArray::Offsets_t & a_offsets,
|
||||
const Array & b,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
PODArray<ResultType> * c_data = nullptr;
|
||||
ColumnArray::Offsets_t * c_offsets = nullptr;
|
||||
create_result_column(block, result, &c_data, &c_offsets);
|
||||
|
||||
PODArray<B> b_converted(b.size());
|
||||
for (size_t i = 0, size = b.size(); i < size; ++i)
|
||||
b_converted[i] = b[i].get<typename NearestFieldType<B>::Type>();
|
||||
|
||||
size_t size = cond.size();
|
||||
c_offsets->resize(size);
|
||||
c_data->reserve(a_data.size());
|
||||
|
||||
ColumnArray::Offset_t a_prev_offset = 0;
|
||||
ColumnArray::Offset_t c_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (cond[i])
|
||||
copy_from_vector(i, a_data, a_offsets, a_prev_offset, *c_data, *c_offsets, c_prev_offset);
|
||||
else
|
||||
copy_from_constant(i, b_converted, *c_data, *c_offsets, c_prev_offset);
|
||||
|
||||
a_prev_offset = a_offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static void constant_vector(
|
||||
const PODArray<UInt8> & cond,
|
||||
const Array & a,
|
||||
const PODArray<B> & b_data, const ColumnArray::Offsets_t & b_offsets,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
PODArray<ResultType> * c_data = nullptr;
|
||||
ColumnArray::Offsets_t * c_offsets = nullptr;
|
||||
create_result_column(block, result, &c_data, &c_offsets);
|
||||
|
||||
PODArray<A> a_converted(a.size());
|
||||
for (size_t i = 0, size = a.size(); i < size; ++i)
|
||||
a_converted[i] = a[i].get<typename NearestFieldType<A>::Type>();
|
||||
|
||||
size_t size = cond.size();
|
||||
c_offsets->resize(size);
|
||||
c_data->reserve(b_data.size());
|
||||
|
||||
ColumnArray::Offset_t b_prev_offset = 0;
|
||||
ColumnArray::Offset_t c_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (cond[i])
|
||||
copy_from_constant(i, a_converted, *c_data, *c_offsets, c_prev_offset);
|
||||
else
|
||||
copy_from_vector(i, b_data, b_offsets, b_prev_offset, *c_data, *c_offsets, c_prev_offset);
|
||||
|
||||
b_prev_offset = b_offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static void constant_constant(
|
||||
const PODArray<UInt8> & cond,
|
||||
const Array & a, const Array & b,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
PODArray<ResultType> * c_data = nullptr;
|
||||
ColumnArray::Offsets_t * c_offsets = nullptr;
|
||||
create_result_column(block, result, &c_data, &c_offsets);
|
||||
|
||||
PODArray<A> a_converted(a.size());
|
||||
for (size_t i = 0, size = a.size(); i < size; ++i)
|
||||
a_converted[i] = a[i].get<typename NearestFieldType<A>::Type>();
|
||||
|
||||
PODArray<B> b_converted(b.size());
|
||||
for (size_t i = 0, size = b.size(); i < size; ++i)
|
||||
b_converted[i] = b[i].get<typename NearestFieldType<B>::Type>();
|
||||
|
||||
size_t size = cond.size();
|
||||
c_offsets->resize(size);
|
||||
c_data->reserve((std::max(a.size(), b.size())) * size);
|
||||
|
||||
ColumnArray::Offset_t c_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (cond[i])
|
||||
copy_from_constant(i, a_converted, *c_data, *c_offsets, c_prev_offset);
|
||||
else
|
||||
copy_from_constant(i, b_converted, *c_data, *c_offsets, c_prev_offset);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <typename A, typename B>
|
||||
struct NumArrayIfImpl<A, B, NumberTraits::Error>
|
||||
{
|
||||
private:
|
||||
static void throw_error()
|
||||
{
|
||||
throw Exception("Internal logic error: invalid types of arguments 2 and 3 of if", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
public:
|
||||
static void vector_vector(
|
||||
const PODArray<UInt8> & cond,
|
||||
const PODArray<A> & a_data, const ColumnArray::Offsets_t & a_offsets,
|
||||
const PODArray<B> & b_data, const ColumnArray::Offsets_t & b_offsets,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
throw_error();
|
||||
}
|
||||
|
||||
static void vector_constant(
|
||||
const PODArray<UInt8> & cond,
|
||||
const PODArray<A> & a_data, const ColumnArray::Offsets_t & a_offsets,
|
||||
const Array & b,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
throw_error();
|
||||
}
|
||||
|
||||
static void constant_vector(
|
||||
const PODArray<UInt8> & cond,
|
||||
const Array & a,
|
||||
const PODArray<B> & b_data, const ColumnArray::Offsets_t & b_offsets,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
throw_error();
|
||||
}
|
||||
|
||||
static void constant_constant(
|
||||
const PODArray<UInt8> & cond,
|
||||
const Array & a, const Array & b,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
throw_error();
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename T>
|
||||
struct DataTypeFromFieldTypeOrError
|
||||
{
|
||||
@ -347,8 +566,8 @@ private:
|
||||
size_t result,
|
||||
const ColumnVector<T0> * col_left)
|
||||
{
|
||||
ColumnVector<T1> * col_right_vec = typeid_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
ColumnConst<T1> * col_right_const = typeid_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
const ColumnVector<T1> * col_right_vec = typeid_cast<const ColumnVector<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
const ColumnConst<T1> * col_right_const = typeid_cast<const ColumnConst<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
|
||||
if (!col_right_vec && !col_right_const)
|
||||
return false;
|
||||
@ -371,8 +590,8 @@ private:
|
||||
size_t result,
|
||||
const ColumnConst<T0> * col_left)
|
||||
{
|
||||
ColumnVector<T1> * col_right_vec = typeid_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
ColumnConst<T1> * col_right_const = typeid_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
const ColumnVector<T1> * col_right_vec = typeid_cast<const ColumnVector<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
const ColumnConst<T1> * col_right_const = typeid_cast<const ColumnConst<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
|
||||
if (!col_right_vec && !col_right_const)
|
||||
return false;
|
||||
@ -387,10 +606,131 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeRightTypeArray(
|
||||
const ColumnVector<UInt8> * cond_col,
|
||||
Block & block,
|
||||
const ColumnNumbers & arguments,
|
||||
size_t result,
|
||||
const ColumnArray * col_left_array,
|
||||
const ColumnVector<T0> * col_left)
|
||||
{
|
||||
const IColumn * col_right_untyped = block.getByPosition(arguments[2]).column.get();
|
||||
|
||||
const ColumnArray * col_right_array = typeid_cast<const ColumnArray *>(col_right_untyped);
|
||||
const ColumnConstArray * col_right_const_array = typeid_cast<const ColumnConstArray *>(col_right_untyped);
|
||||
|
||||
if (!col_right_array && !col_right_const_array)
|
||||
return false;
|
||||
|
||||
typedef typename NumberTraits::ResultOfIf<T0, T1>::Type ResultType;
|
||||
|
||||
if (col_right_array)
|
||||
{
|
||||
std::cerr << "col_right_array\n";
|
||||
|
||||
const ColumnVector<T1> * col_right_vec = typeid_cast<const ColumnVector<T1> *>(&col_right_array->getData());
|
||||
|
||||
if (!col_right_vec)
|
||||
return false;
|
||||
|
||||
std::cerr << "!\n";
|
||||
|
||||
NumArrayIfImpl<T0, T1, ResultType>::vector_vector(
|
||||
cond_col->getData(),
|
||||
col_left->getData(), col_left_array->getOffsets(),
|
||||
col_right_vec->getData(), col_right_array->getOffsets(),
|
||||
block, result);
|
||||
}
|
||||
else
|
||||
{
|
||||
std::cerr << "col_right_const_array\n";
|
||||
|
||||
NumArrayIfImpl<T0, T1, ResultType>::vector_constant(
|
||||
cond_col->getData(),
|
||||
col_left->getData(), col_left_array->getOffsets(),
|
||||
col_right_const_array->getData(),
|
||||
block, result);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeConstRightTypeArray(
|
||||
const ColumnVector<UInt8> * cond_col,
|
||||
Block & block,
|
||||
const ColumnNumbers & arguments,
|
||||
size_t result,
|
||||
const ColumnConstArray * col_left_const_array)
|
||||
{
|
||||
const IColumn * col_right_untyped = block.getByPosition(arguments[2]).column.get();
|
||||
|
||||
const ColumnArray * col_right_array = typeid_cast<const ColumnArray *>(col_right_untyped);
|
||||
const ColumnConstArray * col_right_const_array = typeid_cast<const ColumnConstArray *>(col_right_untyped);
|
||||
|
||||
if (!col_right_array && !col_right_const_array)
|
||||
return false;
|
||||
|
||||
typedef typename NumberTraits::ResultOfIf<T0, T1>::Type ResultType;
|
||||
|
||||
if (col_right_array)
|
||||
{
|
||||
std::cerr << "col_right_array\n";
|
||||
|
||||
const ColumnVector<T1> * col_right_vec = typeid_cast<const ColumnVector<T1> *>(&col_right_array->getData());
|
||||
|
||||
if (!col_right_vec)
|
||||
return false;
|
||||
|
||||
std::cerr << "!\n";
|
||||
|
||||
NumArrayIfImpl<T0, T1, ResultType>::constant_vector(
|
||||
cond_col->getData(),
|
||||
col_left_const_array->getData(),
|
||||
col_right_vec->getData(), col_right_array->getOffsets(),
|
||||
block, result);
|
||||
}
|
||||
else
|
||||
{
|
||||
std::cerr << "col_right_const_array\n";
|
||||
|
||||
NumArrayIfImpl<T0, T1, ResultType>::constant_constant(
|
||||
cond_col->getData(),
|
||||
col_left_const_array->getData(),
|
||||
col_right_const_array->getData(),
|
||||
block, result);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool executeLeftType(const ColumnVector<UInt8> * cond_col, Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (ColumnVector<T0> * col_left = typeid_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[1]).column))
|
||||
const IColumn * col_left_untyped = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
const ColumnVector<T0> * col_left = nullptr;
|
||||
const ColumnConst<T0> * col_const_left = nullptr;
|
||||
const ColumnArray * col_arr_left = nullptr;
|
||||
const ColumnConstArray * col_const_arr_left = nullptr;
|
||||
|
||||
col_left = typeid_cast<const ColumnVector<T0> *>(col_left_untyped);
|
||||
if (!col_left)
|
||||
{
|
||||
col_const_left = typeid_cast<const ColumnConst<T0> *>(col_left_untyped);
|
||||
if (!col_const_left)
|
||||
{
|
||||
col_arr_left = typeid_cast<const ColumnArray *>(col_left_untyped);
|
||||
|
||||
if (col_arr_left)
|
||||
col_left = typeid_cast<const ColumnVector<T0> *>(&col_arr_left->getData());
|
||||
else
|
||||
col_const_arr_left = typeid_cast<const ColumnConstArray *>(col_left_untyped);
|
||||
}
|
||||
}
|
||||
|
||||
if (col_left)
|
||||
{
|
||||
if ( executeRightType<T0, UInt8>(cond_col, block, arguments, result, col_left)
|
||||
|| executeRightType<T0, UInt16>(cond_col, block, arguments, result, col_left)
|
||||
@ -405,21 +745,61 @@ private:
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
||||
+ " of third argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
+ " of third argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (ColumnConst<T0> * col_left = typeid_cast<ColumnConst<T0> *>(&*block.getByPosition(arguments[1]).column))
|
||||
else if (col_const_left)
|
||||
{
|
||||
if ( executeConstRightType<T0, UInt8>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, UInt16>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, UInt32>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, UInt64>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int8>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int16>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int32>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int64>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Float32>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Float64>(cond_col, block, arguments, result, col_left))
|
||||
if ( executeConstRightType<T0, UInt8>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, UInt16>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, UInt32>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, UInt64>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, Int8>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, Int16>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, Int32>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, Int64>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, Float32>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, Float64>(cond_col, block, arguments, result, col_const_left))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
||||
+ " of third argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (col_arr_left && col_left)
|
||||
{
|
||||
std::cerr << "col_arr_left\n";
|
||||
|
||||
if ( executeRightTypeArray<T0, UInt8>(cond_col, block, arguments, result, col_arr_left, col_left)
|
||||
|| executeRightTypeArray<T0, UInt16>(cond_col, block, arguments, result, col_arr_left, col_left)
|
||||
|| executeRightTypeArray<T0, UInt32>(cond_col, block, arguments, result, col_arr_left, col_left)
|
||||
|| executeRightTypeArray<T0, UInt64>(cond_col, block, arguments, result, col_arr_left, col_left)
|
||||
|| executeRightTypeArray<T0, Int8>(cond_col, block, arguments, result, col_arr_left, col_left)
|
||||
|| executeRightTypeArray<T0, Int16>(cond_col, block, arguments, result, col_arr_left, col_left)
|
||||
|| executeRightTypeArray<T0, Int32>(cond_col, block, arguments, result, col_arr_left, col_left)
|
||||
|| executeRightTypeArray<T0, Int64>(cond_col, block, arguments, result, col_arr_left, col_left)
|
||||
|| executeRightTypeArray<T0, Float32>(cond_col, block, arguments, result, col_arr_left, col_left)
|
||||
|| executeRightTypeArray<T0, Float64>(cond_col, block, arguments, result, col_arr_left, col_left))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
||||
+ " of third argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (col_const_arr_left)
|
||||
{
|
||||
std::cerr << "col_const_arr_left\n";
|
||||
|
||||
if ( executeConstRightTypeArray<T0, UInt8>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, UInt16>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, UInt32>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, UInt64>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, Int8>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, Int16>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, Int32>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, Int64>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, Float32>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, Float64>(cond_col, block, arguments, result, col_const_arr_left))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
||||
@ -432,10 +812,10 @@ private:
|
||||
|
||||
bool executeString(const ColumnVector<UInt8> * cond_col, Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
ColumnString * col_then = typeid_cast<ColumnString *>(&*block.getByPosition(arguments[1]).column);
|
||||
ColumnString * col_else = typeid_cast<ColumnString *>(&*block.getByPosition(arguments[2]).column);
|
||||
ColumnConstString * col_then_const = typeid_cast<ColumnConstString *>(&*block.getByPosition(arguments[1]).column);
|
||||
ColumnConstString * col_else_const = typeid_cast<ColumnConstString *>(&*block.getByPosition(arguments[2]).column);
|
||||
const ColumnString * col_then = typeid_cast<const ColumnString *>(&*block.getByPosition(arguments[1]).column);
|
||||
const ColumnString * col_else = typeid_cast<const ColumnString *>(&*block.getByPosition(arguments[2]).column);
|
||||
const ColumnConstString * col_then_const = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[1]).column);
|
||||
const ColumnConstString * col_else_const = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[2]).column);
|
||||
|
||||
ColumnString * col_res = new ColumnString;
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -446,31 +826,31 @@ private:
|
||||
if (col_then && col_else)
|
||||
StringIfImpl::vector_vector(
|
||||
cond_col->getData(),
|
||||
col_then->getChars(), col_then->getOffsets(),
|
||||
col_else->getChars(), col_else->getOffsets(),
|
||||
res_vec, res_offsets);
|
||||
else if (col_then && col_else_const)
|
||||
StringIfImpl::vector_constant(
|
||||
cond_col->getData(),
|
||||
col_then->getChars(), col_then->getOffsets(),
|
||||
col_else_const->getData(),
|
||||
res_vec, res_offsets);
|
||||
else if (col_then_const && col_else)
|
||||
StringIfImpl::constant_vector(
|
||||
cond_col->getData(),
|
||||
col_then_const->getData(),
|
||||
col_else->getChars(), col_else->getOffsets(),
|
||||
res_vec, res_offsets);
|
||||
else if (col_then_const && col_else_const)
|
||||
StringIfImpl::constant_constant(
|
||||
cond_col->getData(),
|
||||
col_then_const->getData(),
|
||||
col_else_const->getData(),
|
||||
res_vec, res_offsets);
|
||||
else
|
||||
return false;
|
||||
col_then->getChars(), col_then->getOffsets(),
|
||||
col_else->getChars(), col_else->getOffsets(),
|
||||
res_vec, res_offsets);
|
||||
else if (col_then && col_else_const)
|
||||
StringIfImpl::vector_constant(
|
||||
cond_col->getData(),
|
||||
col_then->getChars(), col_then->getOffsets(),
|
||||
col_else_const->getData(),
|
||||
res_vec, res_offsets);
|
||||
else if (col_then_const && col_else)
|
||||
StringIfImpl::constant_vector(
|
||||
cond_col->getData(),
|
||||
col_then_const->getData(),
|
||||
col_else->getChars(), col_else->getOffsets(),
|
||||
res_vec, res_offsets);
|
||||
else if (col_then_const && col_else_const)
|
||||
StringIfImpl::constant_constant(
|
||||
cond_col->getData(),
|
||||
col_then_const->getData(),
|
||||
col_else_const->getData(),
|
||||
res_vec, res_offsets);
|
||||
else
|
||||
return false;
|
||||
|
||||
return true;
|
||||
return true;
|
||||
}
|
||||
|
||||
public:
|
||||
@ -492,6 +872,9 @@ public:
|
||||
throw Exception("Illegal type of first argument (condition) of function if. Must be UInt8.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const DataTypeArray * type_arr1 = typeid_cast<const DataTypeArray *>(arguments[1].get());
|
||||
const DataTypeArray * type_arr2 = typeid_cast<const DataTypeArray *>(arguments[2].get());
|
||||
|
||||
if (arguments[1]->behavesAsNumber() && arguments[2]->behavesAsNumber())
|
||||
{
|
||||
DataTypePtr type_res;
|
||||
@ -509,6 +892,11 @@ public:
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
return type_res;
|
||||
}
|
||||
else if (type_arr1 && type_arr2)
|
||||
{
|
||||
/// NOTE Сообщения об ошибках будут относится к типам элементов массивов, что немного некорректно.
|
||||
return new DataTypeArray(getReturnType({arguments[0], type_arr1->getNestedType(), type_arr2->getNestedType()}));
|
||||
}
|
||||
else if (arguments[1]->getName() != arguments[2]->getName())
|
||||
{
|
||||
throw Exception("Incompatible second and third arguments for function " + getName() + ": "
|
||||
@ -542,6 +930,7 @@ public:
|
||||
cond_col = typeid_cast<const ColumnVector<UInt8> *>(&*materialized_cond_col);
|
||||
}
|
||||
}
|
||||
|
||||
if (cond_col)
|
||||
{
|
||||
if (!( executeLeftType<UInt8>(cond_col, block, arguments, result)
|
||||
@ -558,11 +947,12 @@ public:
|
||||
throw Exception("Illegal columns " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " and " + block.getByPosition(arguments[2]).column->getName()
|
||||
+ " of second (then) and third (else) arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + cond_col->getName() + " of first argument of function " + getName() + ". Must be ColumnUInt8 or ColumnConstUInt8.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("Illegal column " + cond_col->getName() + " of first argument of function " + getName()
|
||||
+ ". Must be ColumnUInt8 or ColumnConstUInt8.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user