ClickHouse/dbms/include/DB/Functions/FunctionsLogical.h

360 lines
12 KiB
C
Raw Normal View History

2011-08-22 01:01:01 +00:00
#pragma once
#include <Poco/NumberFormatter.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Functions/IFunction.h>
namespace DB
{
/** Функции - логические связки: and, or, not, xor.
* Принимают любые числовые типы, возвращают UInt8, содержащий 0 или 1.
*/
template<typename A, typename B>
struct AndImpl
{
static void vector_vector(const std::vector<A> & a, const std::vector<B> & b, std::vector<UInt8> & c)
2011-08-22 01:01:01 +00:00
{
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
c[i] = a[i] && b[i];
}
static void vector_constant(const std::vector<A> & a, B b, std::vector<UInt8> & c)
2011-08-22 01:01:01 +00:00
{
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
c[i] = a[i] && b;
}
static void constant_vector(A a, const std::vector<B> & b, std::vector<UInt8> & c)
2011-08-22 01:01:01 +00:00
{
size_t size = b.size();
for (size_t i = 0; i < size; ++i)
c[i] = a && b[i];
}
static void constant_constant(A a, B b, UInt8 & c)
{
c = a && b;
}
};
template<typename A, typename B>
struct OrImpl
{
static void vector_vector(const std::vector<A> & a, const std::vector<B> & b, std::vector<UInt8> & c)
2011-08-22 01:01:01 +00:00
{
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
c[i] = a[i] || b[i];
}
static void vector_constant(const std::vector<A> & a, B b, std::vector<UInt8> & c)
2011-08-22 01:01:01 +00:00
{
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
c[i] = a[i] || b;
}
static void constant_vector(A a, const std::vector<B> & b, std::vector<UInt8> & c)
2011-08-22 01:01:01 +00:00
{
size_t size = b.size();
for (size_t i = 0; i < size; ++i)
c[i] = a || b[i];
}
static void constant_constant(A a, B b, UInt8 & c)
{
c = a || b;
}
};
template<typename A, typename B>
struct XorImpl
{
static void vector_vector(const std::vector<A> & a, const std::vector<B> & b, std::vector<UInt8> & c)
2011-08-22 01:01:01 +00:00
{
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
c[i] = (a[i] && !b[i]) || (!a[i] && b[i]);
}
static void vector_constant(const std::vector<A> & a, B b, std::vector<UInt8> & c)
2011-08-22 01:01:01 +00:00
{
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
c[i] = (a[i] && !b) || (!a[i] && b);
}
static void constant_vector(A a, const std::vector<B> & b, std::vector<UInt8> & c)
2011-08-22 01:01:01 +00:00
{
size_t size = b.size();
for (size_t i = 0; i < size; ++i)
c[i] = (a && !b[i]) || (!a && b[i]);
}
static void constant_constant(A a, B b, UInt8 & c)
{
c = (a && !b) || (!a && b);
}
};
template<typename A>
struct NotImpl
{
static void vector(const std::vector<A> & a, std::vector<UInt8> & c)
2011-08-22 01:01:01 +00:00
{
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
c[i] = !a[i];
}
static void constant(A a, UInt8 & c)
{
c = !a;
}
};
template <template <typename, typename> class Impl, typename Name>
class FunctionBinaryLogical : public IFunction
{
private:
template <typename T0, typename T1>
2011-09-24 20:32:41 +00:00
bool executeRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<T0> * col_left)
2011-08-22 01:01:01 +00:00
{
if (ColumnVector<T1> * col_right = dynamic_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
{
ColumnVector<UInt8> * col_res = new ColumnVector<UInt8>;
2011-09-24 20:32:41 +00:00
block.getByPosition(result).column = col_res;
2011-08-22 01:01:01 +00:00
typename ColumnVector<UInt8>::Container_t & vec_res = col_res->getData();
vec_res.resize(col_left->getData().size());
Impl<T0, T1>::vector_vector(col_left->getData(), col_right->getData(), vec_res);
return true;
}
else if (ColumnConst<T1> * col_right = dynamic_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[1]).column))
{
ColumnVector<UInt8> * col_res = new ColumnVector<UInt8>;
2011-09-24 20:32:41 +00:00
block.getByPosition(result).column = col_res;
2011-08-22 01:01:01 +00:00
typename ColumnVector<UInt8>::Container_t & vec_res = col_res->getData();
vec_res.resize(col_left->getData().size());
Impl<T0, T1>::vector_constant(col_left->getData(), col_right->getData(), vec_res);
return true;
}
return false;
}
template <typename T0, typename T1>
2011-09-24 20:32:41 +00:00
bool executeConstRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnConst<T0> * col_left)
2011-08-22 01:01:01 +00:00
{
if (ColumnVector<T1> * col_right = dynamic_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
{
ColumnVector<UInt8> * col_res = new ColumnVector<UInt8>;
2011-09-24 20:32:41 +00:00
block.getByPosition(result).column = col_res;
2011-08-22 01:01:01 +00:00
typename ColumnVector<UInt8>::Container_t & vec_res = col_res->getData();
vec_res.resize(col_left->size());
Impl<T0, T1>::constant_vector(col_left->getData(), col_right->getData(), vec_res);
return true;
}
else if (ColumnConst<T1> * col_right = dynamic_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[1]).column))
{
UInt8 res = 0;
Impl<T0, T1>::constant_constant(col_left->getData(), col_right->getData(), res);
ColumnConst<UInt8> * col_res = new ColumnConst<UInt8>(col_left->size(), res);
2011-09-24 20:32:41 +00:00
block.getByPosition(result).column = col_res;
2011-08-22 01:01:01 +00:00
return true;
}
return false;
}
template <typename T0>
2011-09-24 20:32:41 +00:00
bool executeLeftType(Block & block, const ColumnNumbers & arguments, size_t result)
2011-08-22 01:01:01 +00:00
{
if (ColumnVector<T0> * col_left = dynamic_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
{
if ( executeRightType<T0, UInt8>(block, arguments, result, col_left)
|| executeRightType<T0, UInt16>(block, arguments, result, col_left)
|| executeRightType<T0, UInt32>(block, arguments, result, col_left)
|| executeRightType<T0, UInt64>(block, arguments, result, col_left)
|| executeRightType<T0, Int8>(block, arguments, result, col_left)
|| executeRightType<T0, Int16>(block, arguments, result, col_left)
|| executeRightType<T0, Int32>(block, arguments, result, col_left)
|| executeRightType<T0, Int64>(block, arguments, result, col_left)
|| executeRightType<T0, Float32>(block, arguments, result, col_left)
|| executeRightType<T0, Float64>(block, arguments, result, col_left))
return true;
else
2011-08-28 00:31:30 +00:00
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
+ " of second argument of function " + getName(),
2011-08-22 01:01:01 +00:00
ErrorCodes::ILLEGAL_COLUMN);
}
else if (ColumnConst<T0> * col_left = dynamic_cast<ColumnConst<T0> *>(&*block.getByPosition(arguments[0]).column))
{
if ( executeConstRightType<T0, UInt8>(block, arguments, result, col_left)
|| executeConstRightType<T0, UInt16>(block, arguments, result, col_left)
|| executeConstRightType<T0, UInt32>(block, arguments, result, col_left)
|| executeConstRightType<T0, UInt64>(block, arguments, result, col_left)
|| executeConstRightType<T0, Int8>(block, arguments, result, col_left)
|| executeConstRightType<T0, Int16>(block, arguments, result, col_left)
|| executeConstRightType<T0, Int32>(block, arguments, result, col_left)
|| executeConstRightType<T0, Int64>(block, arguments, result, col_left)
|| executeConstRightType<T0, Float32>(block, arguments, result, col_left)
|| executeConstRightType<T0, Float64>(block, arguments, result, col_left))
return true;
else
2011-08-28 00:31:30 +00:00
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
+ " of second argument of function " + getName(),
2011-08-22 01:01:01 +00:00
ErrorCodes::ILLEGAL_COLUMN);
}
return false;
}
public:
/// Получить имя функции.
String getName() const
{
return Name::get();
}
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
2011-09-24 20:32:41 +00:00
DataTypePtr getReturnType(const DataTypes & arguments) const
2011-08-22 01:01:01 +00:00
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!(arguments[0]->isNumeric() && arguments[1]->isNumeric()))
throw Exception("Illegal types ("
+ arguments[0]->getName() + ", " + arguments[1]->getName()
+ ") of arguments of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
2011-09-24 20:32:41 +00:00
return new DataTypeUInt8;
2011-08-22 01:01:01 +00:00
}
/// Выполнить функцию над блоком.
2011-09-24 20:32:41 +00:00
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
2011-08-22 01:01:01 +00:00
{
if (!( executeLeftType<UInt8>(block, arguments, result)
|| executeLeftType<UInt16>(block, arguments, result)
|| executeLeftType<UInt32>(block, arguments, result)
|| executeLeftType<UInt64>(block, arguments, result)
|| executeLeftType<Int8>(block, arguments, result)
|| executeLeftType<Int16>(block, arguments, result)
|| executeLeftType<Int32>(block, arguments, result)
|| executeLeftType<Int64>(block, arguments, result)
|| executeLeftType<Float32>(block, arguments, result)
|| executeLeftType<Float64>(block, arguments, result)))
2011-08-28 00:31:30 +00:00
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of first argument of function " + getName(),
2011-08-22 01:01:01 +00:00
ErrorCodes::ILLEGAL_COLUMN);
}
};
template <template <typename> class Impl, typename Name>
class FunctionUnaryLogical : public IFunction
{
private:
template <typename T>
2011-09-24 20:32:41 +00:00
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
2011-08-22 01:01:01 +00:00
{
if (ColumnVector<T> * col = dynamic_cast<ColumnVector<T> *>(&*block.getByPosition(arguments[0]).column))
{
ColumnVector<UInt8> * col_res = new ColumnVector<UInt8>;
2011-09-24 20:32:41 +00:00
block.getByPosition(result).column = col_res;
2011-08-22 01:01:01 +00:00
typename ColumnVector<UInt8>::Container_t & vec_res = col_res->getData();
vec_res.resize(col->getData().size());
Impl<T>::vector(col->getData(), vec_res);
return true;
}
2011-09-05 00:32:22 +00:00
else if (ColumnConst<T> * col = dynamic_cast<ColumnConst<T> *>(&*block.getByPosition(arguments[0]).column))
2011-08-22 01:01:01 +00:00
{
UInt8 res = 0;
Impl<T>::constant(col->getData(), res);
ColumnConst<UInt8> * col_res = new ColumnConst<UInt8>(col->size(), res);
2011-09-24 20:32:41 +00:00
block.getByPosition(result).column = col_res;
2011-08-22 01:01:01 +00:00
return true;
}
return false;
}
public:
/// Получить имя функции.
String getName() const
{
return Name::get();
}
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
2011-09-24 20:32:41 +00:00
DataTypePtr getReturnType(const DataTypes & arguments) const
2011-08-22 01:01:01 +00:00
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!arguments[0]->isNumeric())
throw Exception("Illegal type ("
+ arguments[0]->getName()
+ ") of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
2011-09-24 20:32:41 +00:00
return new DataTypeUInt8;
2011-08-22 01:01:01 +00:00
}
/// Выполнить функцию над блоком.
2011-09-24 20:32:41 +00:00
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
2011-08-22 01:01:01 +00:00
{
if (!( executeType<UInt8>(block, arguments, result)
|| executeType<UInt16>(block, arguments, result)
|| executeType<UInt32>(block, arguments, result)
|| executeType<UInt64>(block, arguments, result)
|| executeType<Int8>(block, arguments, result)
|| executeType<Int16>(block, arguments, result)
|| executeType<Int32>(block, arguments, result)
|| executeType<Int64>(block, arguments, result)
|| executeType<Float32>(block, arguments, result)
|| executeType<Float64>(block, arguments, result)))
2011-08-28 00:31:30 +00:00
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
2011-08-22 01:01:01 +00:00
ErrorCodes::ILLEGAL_COLUMN);
}
};
struct NameAnd { static const char * get() { return "and"; } };
struct NameOr { static const char * get() { return "or"; } };
struct NameXor { static const char * get() { return "xor"; } };
struct NameNot { static const char * get() { return "not"; } };
typedef FunctionBinaryLogical<AndImpl, NameAnd> FunctionAnd;
typedef FunctionBinaryLogical<OrImpl, NameOr> FunctionOr;
typedef FunctionBinaryLogical<XorImpl, NameXor> FunctionXor;
typedef FunctionUnaryLogical<NotImpl, NameNot> FunctionNot;
}