2012-09-24 02:12:59 +00:00
|
|
|
|
#pragma once
|
|
|
|
|
|
|
|
|
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
|
|
|
|
#include <DB/Columns/ColumnVector.h>
|
2012-09-24 04:11:55 +00:00
|
|
|
|
#include <DB/Columns/ColumnString.h>
|
|
|
|
|
#include <DB/Columns/ColumnConst.h>
|
|
|
|
|
#include <DB/Columns/ColumnArray.h>
|
2012-09-24 02:12:59 +00:00
|
|
|
|
#include <DB/Functions/IFunction.h>
|
2013-10-08 12:30:43 +00:00
|
|
|
|
#include <DB/Functions/NumberTraits.h>
|
2012-09-24 02:12:59 +00:00
|
|
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
/** Функция выбора по условию: if(cond, then, else).
|
2012-09-24 04:11:55 +00:00
|
|
|
|
* cond - UInt8
|
2013-10-08 12:30:43 +00:00
|
|
|
|
* then, else - либо числа/даты/даты-с-временем, либо строки.
|
2012-09-24 02:12:59 +00:00
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
template <typename A, typename B, typename ResultType>
|
2012-09-24 02:12:59 +00:00
|
|
|
|
struct NumIfImpl
|
|
|
|
|
{
|
2013-10-08 12:30:43 +00:00
|
|
|
|
private:
|
2013-12-08 02:29:40 +00:00
|
|
|
|
static PODArray<ResultType> & result_vector(Block & block, size_t result, size_t size)
|
2013-10-08 12:30:43 +00:00
|
|
|
|
{
|
|
|
|
|
ColumnVector<ResultType> * col_res = new ColumnVector<ResultType>;
|
|
|
|
|
block.getByPosition(result).column = col_res;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
typename ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
|
|
|
|
|
vec_res.resize(size);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
return vec_res;
|
|
|
|
|
}
|
|
|
|
|
public:
|
2012-09-24 02:12:59 +00:00
|
|
|
|
static void vector_vector(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt8> & cond,
|
|
|
|
|
const PODArray<A> & a, const PODArray<B> & b,
|
2013-10-08 12:30:43 +00:00
|
|
|
|
Block & block,
|
|
|
|
|
size_t result)
|
2012-09-24 02:12:59 +00:00
|
|
|
|
{
|
|
|
|
|
size_t size = cond.size();
|
2013-12-08 02:29:40 +00:00
|
|
|
|
PODArray<ResultType> & res = result_vector(block, result, size);
|
2012-09-24 02:12:59 +00:00
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
2013-10-08 12:30:43 +00:00
|
|
|
|
res[i] = cond[i] ? static_cast<ResultType>(a[i]) : static_cast<ResultType>(b[i]);
|
2012-09-24 02:12:59 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
static void vector_constant(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt8> & cond,
|
|
|
|
|
const PODArray<A> & a, B b,
|
2013-10-08 12:30:43 +00:00
|
|
|
|
Block & block,
|
|
|
|
|
size_t result)
|
2012-09-24 02:12:59 +00:00
|
|
|
|
{
|
|
|
|
|
size_t size = cond.size();
|
2013-12-08 02:29:40 +00:00
|
|
|
|
PODArray<ResultType> & res = result_vector(block, result, size);
|
2012-09-24 02:12:59 +00:00
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
2013-10-08 12:30:43 +00:00
|
|
|
|
res[i] = cond[i] ? static_cast<ResultType>(a[i]) : static_cast<ResultType>(b);
|
2012-09-24 02:12:59 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
static void constant_vector(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt8> & cond,
|
|
|
|
|
A a, const PODArray<B> & b,
|
2013-10-08 12:30:43 +00:00
|
|
|
|
Block & block,
|
|
|
|
|
size_t result)
|
2012-09-24 02:12:59 +00:00
|
|
|
|
{
|
|
|
|
|
size_t size = cond.size();
|
2013-12-08 02:29:40 +00:00
|
|
|
|
PODArray<ResultType> & res = result_vector(block, result, size);
|
2012-09-24 02:12:59 +00:00
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
2013-10-08 12:30:43 +00:00
|
|
|
|
res[i] = cond[i] ? static_cast<ResultType>(a) : static_cast<ResultType>(b[i]);
|
2012-09-24 02:12:59 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
static void constant_constant(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt8> & cond,
|
2013-10-08 12:30:43 +00:00
|
|
|
|
A a, B b,
|
|
|
|
|
Block & block,
|
|
|
|
|
size_t result)
|
2012-09-24 02:12:59 +00:00
|
|
|
|
{
|
|
|
|
|
size_t size = cond.size();
|
2013-12-08 02:29:40 +00:00
|
|
|
|
PODArray<ResultType> & res = result_vector(block, result, size);
|
2012-09-24 02:12:59 +00:00
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
2013-10-08 12:30:43 +00:00
|
|
|
|
res[i] = cond[i] ? static_cast<ResultType>(a) : static_cast<ResultType>(b);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
template <typename A, typename B>
|
|
|
|
|
struct NumIfImpl<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(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt8> & cond,
|
|
|
|
|
const PODArray<A> & a, const PODArray<B> & b,
|
2013-10-08 12:30:43 +00:00
|
|
|
|
Block & block,
|
|
|
|
|
size_t result)
|
|
|
|
|
{
|
|
|
|
|
throw_error();
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
static void vector_constant(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt8> & cond,
|
|
|
|
|
const PODArray<A> & a, B b,
|
2013-10-08 12:30:43 +00:00
|
|
|
|
Block & block,
|
|
|
|
|
size_t result)
|
|
|
|
|
{
|
|
|
|
|
throw_error();
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
static void constant_vector(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt8> & cond,
|
|
|
|
|
A a, const PODArray<B> & b,
|
2013-10-08 12:30:43 +00:00
|
|
|
|
Block & block,
|
|
|
|
|
size_t result)
|
|
|
|
|
{
|
|
|
|
|
throw_error();
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
static void constant_constant(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt8> & cond,
|
2013-10-08 12:30:43 +00:00
|
|
|
|
A a, B b,
|
|
|
|
|
Block & block,
|
|
|
|
|
size_t result)
|
|
|
|
|
{
|
|
|
|
|
throw_error();
|
2012-09-24 02:12:59 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2012-09-24 04:11:55 +00:00
|
|
|
|
struct StringIfImpl
|
|
|
|
|
{
|
|
|
|
|
static void vector_vector(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt8> & cond,
|
2013-09-15 05:51:43 +00:00
|
|
|
|
const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets,
|
|
|
|
|
const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets,
|
|
|
|
|
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets)
|
2012-09-24 04:11:55 +00:00
|
|
|
|
{
|
|
|
|
|
size_t size = cond.size();
|
|
|
|
|
c_offsets.resize(size);
|
|
|
|
|
c_data.reserve(std::max(a_data.size(), b_data.size()));
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-05-05 15:25:25 +00:00
|
|
|
|
ColumnString::Offset_t a_prev_offset = 0;
|
|
|
|
|
ColumnString::Offset_t b_prev_offset = 0;
|
|
|
|
|
ColumnString::Offset_t c_prev_offset = 0;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-09-24 04:11:55 +00:00
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
|
{
|
|
|
|
|
if (cond[i])
|
|
|
|
|
{
|
|
|
|
|
size_t size_to_write = a_offsets[i] - a_prev_offset;
|
|
|
|
|
c_data.resize(c_data.size() + size_to_write);
|
|
|
|
|
memcpy(&c_data[c_prev_offset], &a_data[a_prev_offset], size_to_write);
|
|
|
|
|
c_prev_offset += size_to_write;
|
|
|
|
|
c_offsets[i] = c_prev_offset;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
size_t size_to_write = b_offsets[i] - b_prev_offset;
|
|
|
|
|
c_data.resize(c_data.size() + size_to_write);
|
|
|
|
|
memcpy(&c_data[c_prev_offset], &b_data[b_prev_offset], size_to_write);
|
|
|
|
|
c_prev_offset += size_to_write;
|
|
|
|
|
c_offsets[i] = c_prev_offset;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
a_prev_offset = a_offsets[i];
|
|
|
|
|
b_prev_offset = b_offsets[i];
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
static void vector_constant(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt8> & cond,
|
2013-09-15 05:51:43 +00:00
|
|
|
|
const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets,
|
2012-09-24 04:11:55 +00:00
|
|
|
|
const String & b,
|
2013-09-15 05:51:43 +00:00
|
|
|
|
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets)
|
2012-09-24 04:11:55 +00:00
|
|
|
|
{
|
|
|
|
|
size_t size = cond.size();
|
|
|
|
|
c_offsets.resize(size);
|
|
|
|
|
c_data.reserve(a_data.size());
|
|
|
|
|
|
2013-05-05 15:25:25 +00:00
|
|
|
|
ColumnString::Offset_t a_prev_offset = 0;
|
|
|
|
|
ColumnString::Offset_t c_prev_offset = 0;
|
2012-09-24 04:11:55 +00:00
|
|
|
|
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
|
{
|
|
|
|
|
if (cond[i])
|
|
|
|
|
{
|
|
|
|
|
size_t size_to_write = a_offsets[i] - a_prev_offset;
|
|
|
|
|
c_data.resize(c_data.size() + size_to_write);
|
|
|
|
|
memcpy(&c_data[c_prev_offset], &a_data[a_prev_offset], size_to_write);
|
|
|
|
|
c_prev_offset += size_to_write;
|
|
|
|
|
c_offsets[i] = c_prev_offset;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
size_t size_to_write = b.size() + 1;
|
|
|
|
|
c_data.resize(c_data.size() + size_to_write);
|
|
|
|
|
memcpy(&c_data[c_prev_offset], b.data(), size_to_write);
|
|
|
|
|
c_prev_offset += size_to_write;
|
|
|
|
|
c_offsets[i] = c_prev_offset;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
a_prev_offset = a_offsets[i];
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
static void constant_vector(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt8> & cond,
|
2012-09-24 04:11:55 +00:00
|
|
|
|
const String & a,
|
2013-09-15 05:51:43 +00:00
|
|
|
|
const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets,
|
|
|
|
|
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets)
|
2012-09-24 04:11:55 +00:00
|
|
|
|
{
|
|
|
|
|
size_t size = cond.size();
|
|
|
|
|
c_offsets.resize(size);
|
|
|
|
|
c_data.reserve(b_data.size());
|
|
|
|
|
|
2013-05-05 15:25:25 +00:00
|
|
|
|
ColumnString::Offset_t b_prev_offset = 0;
|
|
|
|
|
ColumnString::Offset_t c_prev_offset = 0;
|
2012-09-24 04:11:55 +00:00
|
|
|
|
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
|
{
|
|
|
|
|
if (cond[i])
|
|
|
|
|
{
|
|
|
|
|
size_t size_to_write = a.size() + 1;
|
|
|
|
|
c_data.resize(c_data.size() + size_to_write);
|
|
|
|
|
memcpy(&c_data[c_prev_offset], a.data(), size_to_write);
|
|
|
|
|
c_prev_offset += size_to_write;
|
|
|
|
|
c_offsets[i] = c_prev_offset;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
size_t size_to_write = b_offsets[i] - b_prev_offset;
|
|
|
|
|
c_data.resize(c_data.size() + size_to_write);
|
|
|
|
|
memcpy(&c_data[c_prev_offset], &b_data[b_prev_offset], size_to_write);
|
|
|
|
|
c_prev_offset += size_to_write;
|
|
|
|
|
c_offsets[i] = c_prev_offset;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
b_prev_offset = b_offsets[i];
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
static void constant_constant(
|
2013-12-08 02:29:40 +00:00
|
|
|
|
const PODArray<UInt8> & cond,
|
2012-09-24 04:11:55 +00:00
|
|
|
|
const String & a, const String & b,
|
2013-09-15 05:51:43 +00:00
|
|
|
|
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_offsets)
|
2012-09-24 04:11:55 +00:00
|
|
|
|
{
|
|
|
|
|
size_t size = cond.size();
|
|
|
|
|
c_offsets.resize(size);
|
|
|
|
|
c_data.reserve((std::max(a.size(), b.size()) + 1) * size);
|
|
|
|
|
|
2013-05-05 15:25:25 +00:00
|
|
|
|
ColumnString::Offset_t c_prev_offset = 0;
|
2012-09-24 04:11:55 +00:00
|
|
|
|
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
|
{
|
|
|
|
|
if (cond[i])
|
|
|
|
|
{
|
|
|
|
|
size_t size_to_write = a.size() + 1;
|
|
|
|
|
c_data.resize(c_data.size() + size_to_write);
|
|
|
|
|
memcpy(&c_data[c_prev_offset], a.data(), size_to_write);
|
|
|
|
|
c_prev_offset += size_to_write;
|
|
|
|
|
c_offsets[i] = c_prev_offset;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
size_t size_to_write = b.size() + 1;
|
|
|
|
|
c_data.resize(c_data.size() + size_to_write);
|
|
|
|
|
memcpy(&c_data[c_prev_offset], b.data(), size_to_write);
|
|
|
|
|
c_prev_offset += size_to_write;
|
|
|
|
|
c_offsets[i] = c_prev_offset;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
template <typename T>
|
|
|
|
|
struct DataTypeFromFieldTypeOrError
|
2012-09-24 02:12:59 +00:00
|
|
|
|
{
|
2013-10-08 12:30:43 +00:00
|
|
|
|
static DataTypePtr getDataType()
|
2012-09-24 02:12:59 +00:00
|
|
|
|
{
|
2013-10-08 12:30:43 +00:00
|
|
|
|
return new typename DataTypeFromFieldType<T>::Type;
|
|
|
|
|
}
|
|
|
|
|
};
|
2012-09-24 02:12:59 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
template <>
|
|
|
|
|
struct DataTypeFromFieldTypeOrError<NumberTraits::Error>
|
|
|
|
|
{
|
|
|
|
|
static DataTypePtr getDataType()
|
|
|
|
|
{
|
2014-04-08 07:58:53 +00:00
|
|
|
|
return nullptr;
|
2013-10-08 12:30:43 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
2012-09-24 02:12:59 +00:00
|
|
|
|
|
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
class FunctionIf : public IFunction
|
|
|
|
|
{
|
2014-11-12 17:23:26 +00:00
|
|
|
|
public:
|
|
|
|
|
static constexpr auto name = "if";
|
|
|
|
|
static IFunction * create(const Context & context) { return new FunctionIf; }
|
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
private:
|
|
|
|
|
template <typename T0, typename T1>
|
|
|
|
|
bool checkRightType(const DataTypes & arguments, DataTypePtr & type_res) const
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (typeid_cast<const T1 *>(&*arguments[2]))
|
2013-10-08 12:30:43 +00:00
|
|
|
|
{
|
|
|
|
|
typedef typename NumberTraits::ResultOfIf<typename T0::FieldType, typename T1::FieldType>::Type ResultType;
|
|
|
|
|
type_res = DataTypeFromFieldTypeOrError<ResultType>::getDataType();
|
|
|
|
|
if (!type_res)
|
|
|
|
|
throw Exception("Arguments 2 and 3 of function " + getName() + " are not upscalable to a common type without loss of precision: "
|
|
|
|
|
+ arguments[1]->getName() + " and " + arguments[2]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
return false;
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
template <typename T0>
|
|
|
|
|
bool checkLeftType(const DataTypes & arguments, DataTypePtr & type_res) const
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (typeid_cast<const T0 *>(&*arguments[1]))
|
2013-10-08 12:30:43 +00:00
|
|
|
|
{
|
|
|
|
|
if ( checkRightType<T0, DataTypeUInt8>(arguments, type_res)
|
|
|
|
|
|| checkRightType<T0, DataTypeUInt16>(arguments, type_res)
|
|
|
|
|
|| checkRightType<T0, DataTypeUInt32>(arguments, type_res)
|
|
|
|
|
|| checkRightType<T0, DataTypeUInt64>(arguments, type_res)
|
|
|
|
|
|| checkRightType<T0, DataTypeInt8>(arguments, type_res)
|
|
|
|
|
|| checkRightType<T0, DataTypeInt16>(arguments, type_res)
|
|
|
|
|
|| checkRightType<T0, DataTypeInt32>(arguments, type_res)
|
|
|
|
|
|| checkRightType<T0, DataTypeInt64>(arguments, type_res)
|
|
|
|
|
|| checkRightType<T0, DataTypeFloat32>(arguments, type_res)
|
|
|
|
|
|| checkRightType<T0, DataTypeFloat64>(arguments, type_res))
|
|
|
|
|
return true;
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal type " + arguments[2]->getName() + " of third argument of function " + getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
}
|
|
|
|
|
return false;
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
template <typename T0, typename T1>
|
|
|
|
|
bool executeRightType(
|
|
|
|
|
const ColumnVector<UInt8> * cond_col,
|
|
|
|
|
Block & block,
|
|
|
|
|
const ColumnNumbers & arguments,
|
|
|
|
|
size_t result,
|
|
|
|
|
const ColumnVector<T0> * col_left)
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
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);
|
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
if (!col_right_vec && !col_right_const)
|
|
|
|
|
return false;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
typedef typename NumberTraits::ResultOfIf<T0, T1>::Type ResultType;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
if (col_right_vec)
|
|
|
|
|
NumIfImpl<T0, T1, ResultType>::vector_vector(cond_col->getData(), col_left->getData(), col_right_vec->getData(), block, result);
|
2012-09-24 02:12:59 +00:00
|
|
|
|
else
|
2013-10-08 12:30:43 +00:00
|
|
|
|
NumIfImpl<T0, T1, ResultType>::vector_constant(cond_col->getData(), col_left->getData(), col_right_const->getData(), block, result);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
return true;
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
template <typename T0, typename T1>
|
|
|
|
|
bool executeConstRightType(
|
|
|
|
|
const ColumnVector<UInt8> * cond_col,
|
|
|
|
|
Block & block,
|
|
|
|
|
const ColumnNumbers & arguments,
|
|
|
|
|
size_t result,
|
|
|
|
|
const ColumnConst<T0> * col_left)
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
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);
|
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
if (!col_right_vec && !col_right_const)
|
2012-09-24 02:12:59 +00:00
|
|
|
|
return false;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
typedef typename NumberTraits::ResultOfIf<T0, T1>::Type ResultType;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
if (col_right_vec)
|
|
|
|
|
NumIfImpl<T0, T1, ResultType>::constant_vector(cond_col->getData(), col_left->getData(), col_right_vec->getData(), block, result);
|
|
|
|
|
else
|
|
|
|
|
NumIfImpl<T0, T1, ResultType>::constant_constant(cond_col->getData(), col_left->getData(), col_right_const->getData(), block, result);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-09-24 02:12:59 +00:00
|
|
|
|
return true;
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
template <typename T0>
|
|
|
|
|
bool executeLeftType(const ColumnVector<UInt8> * cond_col, Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (ColumnVector<T0> * col_left = typeid_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[1]).column))
|
2013-10-08 12:30:43 +00:00
|
|
|
|
{
|
|
|
|
|
if ( executeRightType<T0, UInt8>(cond_col, block, arguments, result, col_left)
|
|
|
|
|
|| executeRightType<T0, UInt16>(cond_col, block, arguments, result, col_left)
|
|
|
|
|
|| executeRightType<T0, UInt32>(cond_col, block, arguments, result, col_left)
|
|
|
|
|
|| executeRightType<T0, UInt64>(cond_col, block, arguments, result, col_left)
|
|
|
|
|
|| executeRightType<T0, Int8>(cond_col, block, arguments, result, col_left)
|
|
|
|
|
|| executeRightType<T0, Int16>(cond_col, block, arguments, result, col_left)
|
|
|
|
|
|| executeRightType<T0, Int32>(cond_col, block, arguments, result, col_left)
|
|
|
|
|
|| executeRightType<T0, Int64>(cond_col, block, arguments, result, col_left)
|
|
|
|
|
|| executeRightType<T0, Float32>(cond_col, block, arguments, result, col_left)
|
|
|
|
|
|| executeRightType<T0, Float64>(cond_col, block, arguments, result, col_left))
|
|
|
|
|
return true;
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
|
|
|
|
+ " of third argument of function " + getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (ColumnConst<T0> * col_left = typeid_cast<ColumnConst<T0> *>(&*block.getByPosition(arguments[1]).column))
|
2013-10-08 12:30:43 +00:00
|
|
|
|
{
|
|
|
|
|
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))
|
|
|
|
|
return true;
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
|
|
|
|
+ " of third argument of function " + getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
return false;
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-09-24 04:11:55 +00:00
|
|
|
|
bool executeString(const ColumnVector<UInt8> * cond_col, Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
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);
|
|
|
|
|
|
2012-09-24 04:11:55 +00:00
|
|
|
|
ColumnString * col_res = new ColumnString;
|
|
|
|
|
block.getByPosition(result).column = col_res;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-05-05 15:25:25 +00:00
|
|
|
|
ColumnString::Chars_t & res_vec = col_res->getChars();
|
|
|
|
|
ColumnString::Offsets_t & res_offsets = col_res->getOffsets();
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-09-24 04:11:55 +00:00
|
|
|
|
if (col_then && col_else)
|
|
|
|
|
StringIfImpl::vector_vector(
|
|
|
|
|
cond_col->getData(),
|
2013-10-08 12:30:43 +00:00
|
|
|
|
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;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
return true;
|
2012-09-24 04:11:55 +00:00
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-09-24 02:12:59 +00:00
|
|
|
|
public:
|
|
|
|
|
/// Получить имя функции.
|
|
|
|
|
String getName() const
|
|
|
|
|
{
|
2014-11-12 17:23:26 +00:00
|
|
|
|
return name;
|
2012-09-24 02:12:59 +00:00
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
2012-09-24 02:12:59 +00:00
|
|
|
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
|
|
|
|
{
|
|
|
|
|
if (arguments.size() != 3)
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
2013-06-21 20:34:19 +00:00
|
|
|
|
+ toString(arguments.size()) + ", should be 3.",
|
2012-09-24 02:12:59 +00:00
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
|
|
|
|
if (!typeid_cast<const DataTypeUInt8 *>(&*arguments[0]))
|
2012-09-24 02:12:59 +00:00
|
|
|
|
throw Exception("Illegal type of first argument (condition) of function if. Must be UInt8.",
|
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-10-08 12:30:43 +00:00
|
|
|
|
if (arguments[1]->behavesAsNumber() && arguments[2]->behavesAsNumber())
|
|
|
|
|
{
|
|
|
|
|
DataTypePtr type_res;
|
|
|
|
|
if (!( checkLeftType<DataTypeUInt8>(arguments, type_res)
|
|
|
|
|
|| checkLeftType<DataTypeUInt16>(arguments, type_res)
|
|
|
|
|
|| checkLeftType<DataTypeUInt32>(arguments, type_res)
|
|
|
|
|
|| checkLeftType<DataTypeUInt64>(arguments, type_res)
|
|
|
|
|
|| checkLeftType<DataTypeInt8>(arguments, type_res)
|
|
|
|
|
|| checkLeftType<DataTypeInt16>(arguments, type_res)
|
|
|
|
|
|| checkLeftType<DataTypeInt32>(arguments, type_res)
|
|
|
|
|
|| checkLeftType<DataTypeInt64>(arguments, type_res)
|
|
|
|
|
|| checkLeftType<DataTypeFloat32>(arguments, type_res)
|
|
|
|
|
|| checkLeftType<DataTypeFloat64>(arguments, type_res)))
|
|
|
|
|
throw Exception("Internal error: unexpected type " + arguments[1]->getName() + " of first argument of function " + getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
return type_res;
|
|
|
|
|
}
|
|
|
|
|
else if (arguments[1]->getName() != arguments[2]->getName())
|
|
|
|
|
{
|
|
|
|
|
throw Exception("Incompatible second and third arguments for function " + getName() + ": "
|
|
|
|
|
+ arguments[1]->getName() + " and " + arguments[2]->getName(),
|
2012-09-24 02:12:59 +00:00
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2013-10-08 12:30:43 +00:00
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-09-24 02:12:59 +00:00
|
|
|
|
return arguments[1];
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-09-24 02:12:59 +00:00
|
|
|
|
/// Выполнить функцию над блоком.
|
|
|
|
|
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
const ColumnVector<UInt8> * cond_col = typeid_cast<const ColumnVector<UInt8> *>(&*block.getByPosition(arguments[0]).column);
|
|
|
|
|
const ColumnConst<UInt8> * cond_const_col = typeid_cast<const ColumnConst<UInt8> *>(&*block.getByPosition(arguments[0]).column);
|
2013-10-08 12:30:43 +00:00
|
|
|
|
ColumnPtr materialized_cond_col;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-09-24 02:12:59 +00:00
|
|
|
|
if (cond_const_col)
|
|
|
|
|
{
|
2013-10-08 12:30:43 +00:00
|
|
|
|
if (block.getByPosition(arguments[1]).type->getName() ==
|
|
|
|
|
block.getByPosition(arguments[2]).type->getName())
|
|
|
|
|
{
|
|
|
|
|
block.getByPosition(result).column = cond_const_col->getData()
|
|
|
|
|
? block.getByPosition(arguments[1]).column
|
|
|
|
|
: block.getByPosition(arguments[2]).column;
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
materialized_cond_col = cond_const_col->convertToFullColumn();
|
2014-06-26 00:58:14 +00:00
|
|
|
|
cond_col = typeid_cast<const ColumnVector<UInt8> *>(&*materialized_cond_col);
|
2013-10-08 12:30:43 +00:00
|
|
|
|
}
|
2012-09-24 02:12:59 +00:00
|
|
|
|
}
|
2013-10-08 12:30:43 +00:00
|
|
|
|
if (cond_col)
|
2012-09-24 04:11:55 +00:00
|
|
|
|
{
|
2013-10-08 12:30:43 +00:00
|
|
|
|
if (!( executeLeftType<UInt8>(cond_col, block, arguments, result)
|
|
|
|
|
|| executeLeftType<UInt16>(cond_col, block, arguments, result)
|
|
|
|
|
|| executeLeftType<UInt32>(cond_col, block, arguments, result)
|
|
|
|
|
|| executeLeftType<UInt64>(cond_col, block, arguments, result)
|
|
|
|
|
|| executeLeftType<Int8>(cond_col, block, arguments, result)
|
|
|
|
|
|| executeLeftType<Int16>(cond_col, block, arguments, result)
|
|
|
|
|
|| executeLeftType<Int32>(cond_col, block, arguments, result)
|
|
|
|
|
|| executeLeftType<Int64>(cond_col, block, arguments, result)
|
|
|
|
|
|| executeLeftType<Float32>(cond_col, block, arguments, result)
|
|
|
|
|
|| executeLeftType<Float64>(cond_col, block, arguments, result)
|
2012-09-24 04:11:55 +00:00
|
|
|
|
|| executeString(cond_col, block, arguments, result)))
|
|
|
|
|
throw Exception("Illegal columns " + block.getByPosition(arguments[1]).column->getName()
|
2013-10-08 12:30:43 +00:00
|
|
|
|
+ " and " + block.getByPosition(arguments[2]).column->getName()
|
|
|
|
|
+ " of second (then) and third (else) arguments of function " + getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
2012-09-24 04:11:55 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
2012-09-24 02:12:59 +00:00
|
|
|
|
throw Exception("Illegal column " + cond_col->getName() + " of first argument of function " + getName() + ". Must be ColumnUInt8 or ColumnConstUInt8.",
|
2013-10-08 12:30:43 +00:00
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
2012-09-24 02:12:59 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
}
|