mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Rectification of FunctionsComparison [#CLICKHOUSE-2]
This commit is contained in:
parent
f910b5e9de
commit
e97d1bb63f
@ -415,5 +415,52 @@ inline bool_if_safe_conversion<A, B> greaterOrEqualsOp(A a, B b)
|
||||
return a >= b;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename A, typename B> struct EqualsOp
|
||||
{
|
||||
/// An operation that gives the same result, if arguments are passed in reverse order.
|
||||
using SymmetricOp = EqualsOp<B, A>;
|
||||
|
||||
static UInt8 apply(A a, B b) { return accurate::equalsOp(a, b); }
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct NotEqualsOp
|
||||
{
|
||||
using SymmetricOp = NotEqualsOp<B, A>;
|
||||
static UInt8 apply(A a, B b) { return accurate::notEqualsOp(a, b); }
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct GreaterOp;
|
||||
|
||||
template <typename A, typename B> struct LessOp
|
||||
{
|
||||
using SymmetricOp = GreaterOp<B, A>;
|
||||
static UInt8 apply(A a, B b) { return accurate::lessOp(a, b); }
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct GreaterOp
|
||||
{
|
||||
using SymmetricOp = LessOp<B, A>;
|
||||
static UInt8 apply(A a, B b) { return accurate::greaterOp(a, b); }
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct GreaterOrEqualsOp;
|
||||
|
||||
template <typename A, typename B> struct LessOrEqualsOp
|
||||
{
|
||||
using SymmetricOp = GreaterOrEqualsOp<B, A>;
|
||||
static UInt8 apply(A a, B b) { return accurate::lessOrEqualsOp(a, b); }
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct GreaterOrEqualsOp
|
||||
{
|
||||
using SymmetricOp = LessOrEqualsOp<B, A>;
|
||||
static UInt8 apply(A a, B b) { return accurate::greaterOrEqualsOp(a, b); }
|
||||
};
|
||||
|
||||
}
|
||||
|
311
dbms/src/Core/DecimalComparison.h
Normal file
311
dbms/src/Core/DecimalComparison.h
Normal file
@ -0,0 +1,311 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/arithmeticOverflow.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Functions/FunctionHelpers.h> /// TODO Core should not depend on Functions
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int DECIMAL_OVERFLOW;
|
||||
}
|
||||
|
||||
///
|
||||
inline bool allowDecimalComparison(const IDataType & left_type, const IDataType & right_type)
|
||||
{
|
||||
if (isDecimal(left_type))
|
||||
{
|
||||
if (isDecimal(right_type) || notDecimalButComparableToDecimal(right_type))
|
||||
return true;
|
||||
}
|
||||
else if (notDecimalButComparableToDecimal(left_type) && isDecimal(right_type))
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
template <size_t > struct ConstructDecInt { using Type = Int32; };
|
||||
template <> struct ConstructDecInt<8> { using Type = Int64; };
|
||||
template <> struct ConstructDecInt<16> { using Type = Int128; };
|
||||
|
||||
template <typename T, typename U>
|
||||
struct DecCompareInt
|
||||
{
|
||||
using Type = typename ConstructDecInt<(!IsDecimalNumber<U> || sizeof(T) > sizeof(U)) ? sizeof(T) : sizeof(U)>::Type;
|
||||
using TypeA = Type;
|
||||
using TypeB = Type;
|
||||
};
|
||||
|
||||
///
|
||||
template <typename A, typename B, template <typename, typename> typename Operation, bool _check_overflow = true,
|
||||
bool _actual = IsDecimalNumber<A> || IsDecimalNumber<B>>
|
||||
class DecimalComparison
|
||||
{
|
||||
public:
|
||||
using CompareInt = typename DecCompareInt<A, B>::Type;
|
||||
using Op = Operation<CompareInt, CompareInt>;
|
||||
using ColVecA = std::conditional_t<IsDecimalNumber<A>, ColumnDecimal<A>, ColumnVector<A>>;
|
||||
using ColVecB = std::conditional_t<IsDecimalNumber<B>, ColumnDecimal<B>, ColumnVector<B>>;
|
||||
using ArrayA = typename ColVecA::Container;
|
||||
using ArrayB = typename ColVecB::Container;
|
||||
|
||||
DecimalComparison(Block & block, size_t result, const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right)
|
||||
{
|
||||
if (!apply(block, result, col_left, col_right))
|
||||
throw Exception("Wrong decimal comparison with " + col_left.type->getName() + " and " + col_right.type->getName(),
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
static bool apply(Block & block, size_t result [[maybe_unused]],
|
||||
const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right)
|
||||
{
|
||||
if constexpr (_actual)
|
||||
{
|
||||
ColumnPtr c_res;
|
||||
Shift shift = getScales<A, B>(col_left.type, col_right.type);
|
||||
|
||||
c_res = applyWithScale(col_left.column, col_right.column, shift);
|
||||
if (c_res)
|
||||
block.getByPosition(result).column = std::move(c_res);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
static bool compare(A a, B b, UInt32 scale_a, UInt32 scale_b)
|
||||
{
|
||||
static const UInt32 max_scale = maxDecimalPrecision<Decimal128>();
|
||||
if (scale_a > max_scale || scale_b > max_scale)
|
||||
throw Exception("Bad scale of decimal field", ErrorCodes::DECIMAL_OVERFLOW);
|
||||
|
||||
Shift shift;
|
||||
if (scale_a < scale_b)
|
||||
shift.a = DataTypeDecimal<B>(maxDecimalPrecision<B>(), scale_b).getScaleMultiplier(scale_b - scale_a);
|
||||
if (scale_a > scale_b)
|
||||
shift.b = DataTypeDecimal<A>(maxDecimalPrecision<A>(), scale_a).getScaleMultiplier(scale_a - scale_b);
|
||||
|
||||
return applyWithScale(a, b, shift);
|
||||
}
|
||||
|
||||
private:
|
||||
struct Shift
|
||||
{
|
||||
CompareInt a = 1;
|
||||
CompareInt b = 1;
|
||||
|
||||
bool none() const { return a == 1 && b == 1; }
|
||||
bool left() const { return a != 1; }
|
||||
bool right() const { return b != 1; }
|
||||
};
|
||||
|
||||
template <typename T, typename U>
|
||||
static auto applyWithScale(T a, U b, const Shift & shift)
|
||||
{
|
||||
if (shift.left())
|
||||
return apply<true, false>(a, b, shift.a);
|
||||
else if (shift.right())
|
||||
return apply<false, true>(a, b, shift.b);
|
||||
return apply<false, false>(a, b, 1);
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
static std::enable_if_t<IsDecimalNumber<T> && IsDecimalNumber<U>, Shift>
|
||||
getScales(const DataTypePtr & left_type, const DataTypePtr & right_type)
|
||||
{
|
||||
const DataTypeDecimal<T> * decimal0 = checkDecimal<T>(*left_type);
|
||||
const DataTypeDecimal<U> * decimal1 = checkDecimal<U>(*right_type);
|
||||
|
||||
Shift shift;
|
||||
if (decimal0 && decimal1)
|
||||
{
|
||||
auto result_type = decimalResultType(*decimal0, *decimal1, false, false);
|
||||
shift.a = result_type.scaleFactorFor(*decimal0, false);
|
||||
shift.b = result_type.scaleFactorFor(*decimal1, false);
|
||||
}
|
||||
else if (decimal0)
|
||||
shift.b = decimal0->getScaleMultiplier();
|
||||
else if (decimal1)
|
||||
shift.a = decimal1->getScaleMultiplier();
|
||||
|
||||
return shift;
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
static std::enable_if_t<IsDecimalNumber<T> && !IsDecimalNumber<U>, Shift>
|
||||
getScales(const DataTypePtr & left_type, const DataTypePtr &)
|
||||
{
|
||||
Shift shift;
|
||||
const DataTypeDecimal<T> * decimal0 = checkDecimal<T>(*left_type);
|
||||
if (decimal0)
|
||||
shift.b = decimal0->getScaleMultiplier();
|
||||
return shift;
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
static std::enable_if_t<!IsDecimalNumber<T> && IsDecimalNumber<U>, Shift>
|
||||
getScales(const DataTypePtr &, const DataTypePtr & right_type)
|
||||
{
|
||||
Shift shift;
|
||||
const DataTypeDecimal<U> * decimal1 = checkDecimal<U>(*right_type);
|
||||
if (decimal1)
|
||||
shift.a = decimal1->getScaleMultiplier();
|
||||
return shift;
|
||||
}
|
||||
|
||||
template <bool scale_left, bool scale_right>
|
||||
static ColumnPtr apply(const ColumnPtr & c0, const ColumnPtr & c1, CompareInt scale)
|
||||
{
|
||||
auto c_res = ColumnUInt8::create();
|
||||
|
||||
if constexpr (_actual)
|
||||
{
|
||||
bool c0_is_const = c0->isColumnConst();
|
||||
bool c1_is_const = c1->isColumnConst();
|
||||
|
||||
if (c0_is_const && c1_is_const)
|
||||
{
|
||||
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
|
||||
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
|
||||
|
||||
A a = c0_const->template getValue<A>();
|
||||
B b = c1_const->template getValue<B>();
|
||||
UInt8 res = apply<scale_left, scale_right>(a, b, scale);
|
||||
return DataTypeUInt8().createColumnConst(c0->size(), toField(res));
|
||||
}
|
||||
|
||||
ColumnUInt8::Container & vec_res = c_res->getData();
|
||||
vec_res.resize(c0->size());
|
||||
|
||||
if (c0_is_const)
|
||||
{
|
||||
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
|
||||
A a = c0_const->template getValue<A>();
|
||||
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
|
||||
constant_vector<scale_left, scale_right>(a, c1_vec->getData(), vec_res, scale);
|
||||
else
|
||||
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
else if (c1_is_const)
|
||||
{
|
||||
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
|
||||
B b = c1_const->template getValue<B>();
|
||||
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
|
||||
vector_constant<scale_left, scale_right>(c0_vec->getData(), b, vec_res, scale);
|
||||
else
|
||||
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
|
||||
{
|
||||
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
|
||||
vector_vector<scale_left, scale_right>(c0_vec->getData(), c1_vec->getData(), vec_res, scale);
|
||||
else
|
||||
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
else
|
||||
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
return c_res;
|
||||
}
|
||||
|
||||
template <bool scale_left, bool scale_right>
|
||||
static NO_INLINE UInt8 apply(A a, B b, CompareInt scale [[maybe_unused]])
|
||||
{
|
||||
CompareInt x = a;
|
||||
CompareInt y = b;
|
||||
|
||||
if constexpr (_check_overflow)
|
||||
{
|
||||
bool overflow = false;
|
||||
|
||||
if constexpr (sizeof(A) > sizeof(CompareInt))
|
||||
overflow |= (A(x) != a);
|
||||
if constexpr (sizeof(B) > sizeof(CompareInt))
|
||||
overflow |= (B(y) != b);
|
||||
if constexpr (std::is_unsigned_v<A>)
|
||||
overflow |= (x < 0);
|
||||
if constexpr (std::is_unsigned_v<B>)
|
||||
overflow |= (y < 0);
|
||||
|
||||
if constexpr (scale_left)
|
||||
overflow |= common::mulOverflow(x, scale, x);
|
||||
if constexpr (scale_right)
|
||||
overflow |= common::mulOverflow(y, scale, y);
|
||||
|
||||
if (overflow)
|
||||
throw Exception("Can't compare", ErrorCodes::DECIMAL_OVERFLOW);
|
||||
}
|
||||
else
|
||||
{
|
||||
if constexpr (scale_left)
|
||||
x *= scale;
|
||||
if constexpr (scale_right)
|
||||
y *= scale;
|
||||
}
|
||||
|
||||
return Op::apply(x, y);
|
||||
}
|
||||
|
||||
template <bool scale_left, bool scale_right>
|
||||
static void NO_INLINE vector_vector(const ArrayA & a, const ArrayB & b, PaddedPODArray<UInt8> & c,
|
||||
CompareInt scale)
|
||||
{
|
||||
size_t size = a.size();
|
||||
const A * a_pos = &a[0];
|
||||
const B * b_pos = &b[0];
|
||||
UInt8 * c_pos = &c[0];
|
||||
const A * a_end = a_pos + size;
|
||||
|
||||
while (a_pos < a_end)
|
||||
{
|
||||
*c_pos = apply<scale_left, scale_right>(*a_pos, *b_pos, scale);
|
||||
++a_pos;
|
||||
++b_pos;
|
||||
++c_pos;
|
||||
}
|
||||
}
|
||||
|
||||
template <bool scale_left, bool scale_right>
|
||||
static void NO_INLINE vector_constant(const ArrayA & a, B b, PaddedPODArray<UInt8> & c, CompareInt scale)
|
||||
{
|
||||
size_t size = a.size();
|
||||
const A * a_pos = &a[0];
|
||||
UInt8 * c_pos = &c[0];
|
||||
const A * a_end = a_pos + size;
|
||||
|
||||
while (a_pos < a_end)
|
||||
{
|
||||
*c_pos = apply<scale_left, scale_right>(*a_pos, b, scale);
|
||||
++a_pos;
|
||||
++c_pos;
|
||||
}
|
||||
}
|
||||
|
||||
template <bool scale_left, bool scale_right>
|
||||
static void NO_INLINE constant_vector(A a, const ArrayB & b, PaddedPODArray<UInt8> & c, CompareInt scale)
|
||||
{
|
||||
size_t size = b.size();
|
||||
const B * b_pos = &b[0];
|
||||
UInt8 * c_pos = &c[0];
|
||||
const B * b_end = b_pos + size;
|
||||
|
||||
while (b_pos < b_end)
|
||||
{
|
||||
*c_pos = apply<scale_left, scale_right>(a, *b_pos, scale);
|
||||
++b_pos;
|
||||
++c_pos;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -4,8 +4,8 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Core/DecimalComparison.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Functions/FunctionsComparison.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Core/Block.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -9,7 +9,6 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
@ -27,6 +26,8 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <Core/DecimalComparison.h>
|
||||
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
@ -37,11 +38,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int DECIMAL_OVERFLOW;
|
||||
}
|
||||
|
||||
|
||||
/** Comparison functions: ==, !=, <, >, <=, >=.
|
||||
* The comparison functions always return 0 or 1 (UInt8).
|
||||
@ -55,94 +51,8 @@ namespace ErrorCodes
|
||||
* - tuples (lexicographic comparison).
|
||||
*
|
||||
* Exception: You can compare the date and datetime with a constant string. Example: EventDate = '2015-01-01'.
|
||||
*
|
||||
* TODO Arrays.
|
||||
*/
|
||||
|
||||
template <typename A, typename B> struct EqualsOp
|
||||
{
|
||||
/// An operation that gives the same result, if arguments are passed in reverse order.
|
||||
using SymmetricOp = EqualsOp<B, A>;
|
||||
|
||||
static UInt8 apply(A a, B b) { return accurate::equalsOp(a, b); }
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool /*is_signed*/)
|
||||
{
|
||||
return x->getType()->isIntegerTy() ? b.CreateICmpEQ(x, y) : b.CreateFCmpOEQ(x, y); /// qNaNs always compare false
|
||||
}
|
||||
#endif
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct NotEqualsOp
|
||||
{
|
||||
using SymmetricOp = NotEqualsOp<B, A>;
|
||||
static UInt8 apply(A a, B b) { return accurate::notEqualsOp(a, b); }
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool /*is_signed*/)
|
||||
{
|
||||
return x->getType()->isIntegerTy() ? b.CreateICmpNE(x, y) : b.CreateFCmpONE(x, y);
|
||||
}
|
||||
#endif
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct GreaterOp;
|
||||
|
||||
template <typename A, typename B> struct LessOp
|
||||
{
|
||||
using SymmetricOp = GreaterOp<B, A>;
|
||||
static UInt8 apply(A a, B b) { return accurate::lessOp(a, b); }
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
|
||||
{
|
||||
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSLT(x, y) : b.CreateICmpULT(x, y)) : b.CreateFCmpOLT(x, y);
|
||||
}
|
||||
#endif
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct GreaterOp
|
||||
{
|
||||
using SymmetricOp = LessOp<B, A>;
|
||||
static UInt8 apply(A a, B b) { return accurate::greaterOp(a, b); }
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
|
||||
{
|
||||
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSGT(x, y) : b.CreateICmpUGT(x, y)) : b.CreateFCmpOGT(x, y);
|
||||
}
|
||||
#endif
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct GreaterOrEqualsOp;
|
||||
|
||||
template <typename A, typename B> struct LessOrEqualsOp
|
||||
{
|
||||
using SymmetricOp = GreaterOrEqualsOp<B, A>;
|
||||
static UInt8 apply(A a, B b) { return accurate::lessOrEqualsOp(a, b); }
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
|
||||
{
|
||||
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSLE(x, y) : b.CreateICmpULE(x, y)) : b.CreateFCmpOLE(x, y);
|
||||
}
|
||||
#endif
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct GreaterOrEqualsOp
|
||||
{
|
||||
using SymmetricOp = LessOrEqualsOp<B, A>;
|
||||
static UInt8 apply(A a, B b) { return accurate::greaterOrEqualsOp(a, b); }
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
|
||||
{
|
||||
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSGE(x, y) : b.CreateICmpUGE(x, y)) : b.CreateFCmpOGE(x, y);
|
||||
}
|
||||
#endif
|
||||
};
|
||||
|
||||
|
||||
template <typename A, typename B, typename Op>
|
||||
struct NumComparisonImpl
|
||||
@ -196,296 +106,6 @@ struct NumComparisonImpl
|
||||
}
|
||||
};
|
||||
|
||||
///
|
||||
inline bool allowDecimalComparison(const IDataType & left_type, const IDataType & right_type)
|
||||
{
|
||||
if (isDecimal(left_type))
|
||||
{
|
||||
if (isDecimal(right_type) || notDecimalButComparableToDecimal(right_type))
|
||||
return true;
|
||||
}
|
||||
else if (notDecimalButComparableToDecimal(left_type) && isDecimal(right_type))
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
template <size_t > struct ConstructDecInt { using Type = Int32; };
|
||||
template <> struct ConstructDecInt<8> { using Type = Int64; };
|
||||
template <> struct ConstructDecInt<16> { using Type = Int128; };
|
||||
|
||||
template <typename T, typename U>
|
||||
struct DecCompareInt
|
||||
{
|
||||
using Type = typename ConstructDecInt<(!IsDecimalNumber<U> || sizeof(T) > sizeof(U)) ? sizeof(T) : sizeof(U)>::Type;
|
||||
using TypeA = Type;
|
||||
using TypeB = Type;
|
||||
};
|
||||
|
||||
///
|
||||
template <typename A, typename B, template <typename, typename> typename Operation, bool _check_overflow = true,
|
||||
bool _actual = IsDecimalNumber<A> || IsDecimalNumber<B>>
|
||||
class DecimalComparison
|
||||
{
|
||||
public:
|
||||
using CompareInt = typename DecCompareInt<A, B>::Type;
|
||||
using Op = Operation<CompareInt, CompareInt>;
|
||||
using ColVecA = std::conditional_t<IsDecimalNumber<A>, ColumnDecimal<A>, ColumnVector<A>>;
|
||||
using ColVecB = std::conditional_t<IsDecimalNumber<B>, ColumnDecimal<B>, ColumnVector<B>>;
|
||||
using ArrayA = typename ColVecA::Container;
|
||||
using ArrayB = typename ColVecB::Container;
|
||||
|
||||
DecimalComparison(Block & block, size_t result, const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right)
|
||||
{
|
||||
if (!apply(block, result, col_left, col_right))
|
||||
throw Exception("Wrong decimal comparison with " + col_left.type->getName() + " and " + col_right.type->getName(),
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
static bool apply(Block & block, size_t result [[maybe_unused]],
|
||||
const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right)
|
||||
{
|
||||
if constexpr (_actual)
|
||||
{
|
||||
ColumnPtr c_res;
|
||||
Shift shift = getScales<A, B>(col_left.type, col_right.type);
|
||||
|
||||
c_res = applyWithScale(col_left.column, col_right.column, shift);
|
||||
if (c_res)
|
||||
block.getByPosition(result).column = std::move(c_res);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
static bool compare(A a, B b, UInt32 scale_a, UInt32 scale_b)
|
||||
{
|
||||
static const UInt32 max_scale = maxDecimalPrecision<Decimal128>();
|
||||
if (scale_a > max_scale || scale_b > max_scale)
|
||||
throw Exception("Bad scale of decimal field", ErrorCodes::DECIMAL_OVERFLOW);
|
||||
|
||||
Shift shift;
|
||||
if (scale_a < scale_b)
|
||||
shift.a = DataTypeDecimal<B>(maxDecimalPrecision<B>(), scale_b).getScaleMultiplier(scale_b - scale_a);
|
||||
if (scale_a > scale_b)
|
||||
shift.b = DataTypeDecimal<A>(maxDecimalPrecision<A>(), scale_a).getScaleMultiplier(scale_a - scale_b);
|
||||
|
||||
return applyWithScale(a, b, shift);
|
||||
}
|
||||
|
||||
private:
|
||||
struct Shift
|
||||
{
|
||||
CompareInt a = 1;
|
||||
CompareInt b = 1;
|
||||
|
||||
bool none() const { return a == 1 && b == 1; }
|
||||
bool left() const { return a != 1; }
|
||||
bool right() const { return b != 1; }
|
||||
};
|
||||
|
||||
template <typename T, typename U>
|
||||
static auto applyWithScale(T a, U b, const Shift & shift)
|
||||
{
|
||||
if (shift.left())
|
||||
return apply<true, false>(a, b, shift.a);
|
||||
else if (shift.right())
|
||||
return apply<false, true>(a, b, shift.b);
|
||||
return apply<false, false>(a, b, 1);
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
static std::enable_if_t<IsDecimalNumber<T> && IsDecimalNumber<U>, Shift>
|
||||
getScales(const DataTypePtr & left_type, const DataTypePtr & right_type)
|
||||
{
|
||||
const DataTypeDecimal<T> * decimal0 = checkDecimal<T>(*left_type);
|
||||
const DataTypeDecimal<U> * decimal1 = checkDecimal<U>(*right_type);
|
||||
|
||||
Shift shift;
|
||||
if (decimal0 && decimal1)
|
||||
{
|
||||
auto result_type = decimalResultType(*decimal0, *decimal1, false, false);
|
||||
shift.a = result_type.scaleFactorFor(*decimal0, false);
|
||||
shift.b = result_type.scaleFactorFor(*decimal1, false);
|
||||
}
|
||||
else if (decimal0)
|
||||
shift.b = decimal0->getScaleMultiplier();
|
||||
else if (decimal1)
|
||||
shift.a = decimal1->getScaleMultiplier();
|
||||
|
||||
return shift;
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
static std::enable_if_t<IsDecimalNumber<T> && !IsDecimalNumber<U>, Shift>
|
||||
getScales(const DataTypePtr & left_type, const DataTypePtr &)
|
||||
{
|
||||
Shift shift;
|
||||
const DataTypeDecimal<T> * decimal0 = checkDecimal<T>(*left_type);
|
||||
if (decimal0)
|
||||
shift.b = decimal0->getScaleMultiplier();
|
||||
return shift;
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
static std::enable_if_t<!IsDecimalNumber<T> && IsDecimalNumber<U>, Shift>
|
||||
getScales(const DataTypePtr &, const DataTypePtr & right_type)
|
||||
{
|
||||
Shift shift;
|
||||
const DataTypeDecimal<U> * decimal1 = checkDecimal<U>(*right_type);
|
||||
if (decimal1)
|
||||
shift.a = decimal1->getScaleMultiplier();
|
||||
return shift;
|
||||
}
|
||||
|
||||
template <bool scale_left, bool scale_right>
|
||||
static ColumnPtr apply(const ColumnPtr & c0, const ColumnPtr & c1, CompareInt scale)
|
||||
{
|
||||
auto c_res = ColumnUInt8::create();
|
||||
|
||||
if constexpr (_actual)
|
||||
{
|
||||
bool c0_is_const = c0->isColumnConst();
|
||||
bool c1_is_const = c1->isColumnConst();
|
||||
|
||||
if (c0_is_const && c1_is_const)
|
||||
{
|
||||
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
|
||||
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
|
||||
|
||||
A a = c0_const->template getValue<A>();
|
||||
B b = c1_const->template getValue<B>();
|
||||
UInt8 res = apply<scale_left, scale_right>(a, b, scale);
|
||||
return DataTypeUInt8().createColumnConst(c0->size(), toField(res));
|
||||
}
|
||||
|
||||
ColumnUInt8::Container & vec_res = c_res->getData();
|
||||
vec_res.resize(c0->size());
|
||||
|
||||
if (c0_is_const)
|
||||
{
|
||||
const ColumnConst * c0_const = checkAndGetColumnConst<ColVecA>(c0.get());
|
||||
A a = c0_const->template getValue<A>();
|
||||
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
|
||||
constant_vector<scale_left, scale_right>(a, c1_vec->getData(), vec_res, scale);
|
||||
else
|
||||
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
else if (c1_is_const)
|
||||
{
|
||||
const ColumnConst * c1_const = checkAndGetColumnConst<ColVecB>(c1.get());
|
||||
B b = c1_const->template getValue<B>();
|
||||
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
|
||||
vector_constant<scale_left, scale_right>(c0_vec->getData(), b, vec_res, scale);
|
||||
else
|
||||
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (const ColVecA * c0_vec = checkAndGetColumn<ColVecA>(c0.get()))
|
||||
{
|
||||
if (const ColVecB * c1_vec = checkAndGetColumn<ColVecB>(c1.get()))
|
||||
vector_vector<scale_left, scale_right>(c0_vec->getData(), c1_vec->getData(), vec_res, scale);
|
||||
else
|
||||
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
else
|
||||
throw Exception("Wrong column in Decimal comparison", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
return c_res;
|
||||
}
|
||||
|
||||
template <bool scale_left, bool scale_right>
|
||||
static NO_INLINE UInt8 apply(A a, B b, CompareInt scale [[maybe_unused]])
|
||||
{
|
||||
CompareInt x = a;
|
||||
CompareInt y = b;
|
||||
|
||||
if constexpr (_check_overflow)
|
||||
{
|
||||
bool overflow = false;
|
||||
|
||||
if constexpr (sizeof(A) > sizeof(CompareInt))
|
||||
overflow |= (A(x) != a);
|
||||
if constexpr (sizeof(B) > sizeof(CompareInt))
|
||||
overflow |= (B(y) != b);
|
||||
if constexpr (std::is_unsigned_v<A>)
|
||||
overflow |= (x < 0);
|
||||
if constexpr (std::is_unsigned_v<B>)
|
||||
overflow |= (y < 0);
|
||||
|
||||
if constexpr (scale_left)
|
||||
overflow |= common::mulOverflow(x, scale, x);
|
||||
if constexpr (scale_right)
|
||||
overflow |= common::mulOverflow(y, scale, y);
|
||||
|
||||
if (overflow)
|
||||
throw Exception("Can't compare", ErrorCodes::DECIMAL_OVERFLOW);
|
||||
}
|
||||
else
|
||||
{
|
||||
if constexpr (scale_left)
|
||||
x *= scale;
|
||||
if constexpr (scale_right)
|
||||
y *= scale;
|
||||
}
|
||||
|
||||
return Op::apply(x, y);
|
||||
}
|
||||
|
||||
template <bool scale_left, bool scale_right>
|
||||
static void NO_INLINE vector_vector(const ArrayA & a, const ArrayB & b, PaddedPODArray<UInt8> & c,
|
||||
CompareInt scale)
|
||||
{
|
||||
size_t size = a.size();
|
||||
const A * a_pos = &a[0];
|
||||
const B * b_pos = &b[0];
|
||||
UInt8 * c_pos = &c[0];
|
||||
const A * a_end = a_pos + size;
|
||||
|
||||
while (a_pos < a_end)
|
||||
{
|
||||
*c_pos = apply<scale_left, scale_right>(*a_pos, *b_pos, scale);
|
||||
++a_pos;
|
||||
++b_pos;
|
||||
++c_pos;
|
||||
}
|
||||
}
|
||||
|
||||
template <bool scale_left, bool scale_right>
|
||||
static void NO_INLINE vector_constant(const ArrayA & a, B b, PaddedPODArray<UInt8> & c, CompareInt scale)
|
||||
{
|
||||
size_t size = a.size();
|
||||
const A * a_pos = &a[0];
|
||||
UInt8 * c_pos = &c[0];
|
||||
const A * a_end = a_pos + size;
|
||||
|
||||
while (a_pos < a_end)
|
||||
{
|
||||
*c_pos = apply<scale_left, scale_right>(*a_pos, b, scale);
|
||||
++a_pos;
|
||||
++c_pos;
|
||||
}
|
||||
}
|
||||
|
||||
template <bool scale_left, bool scale_right>
|
||||
static void NO_INLINE constant_vector(A a, const ArrayB & b, PaddedPODArray<UInt8> & c, CompareInt scale)
|
||||
{
|
||||
size_t size = b.size();
|
||||
const B * b_pos = &b[0];
|
||||
UInt8 * c_pos = &c[0];
|
||||
const B * b_end = b_pos + size;
|
||||
|
||||
while (b_pos < b_end)
|
||||
{
|
||||
*c_pos = apply<scale_left, scale_right>(a, *b_pos, scale);
|
||||
++b_pos;
|
||||
++c_pos;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
inline int memcmp16(const void * a, const void * b)
|
||||
{
|
||||
@ -904,6 +524,61 @@ struct GenericComparisonImpl
|
||||
};
|
||||
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
template <template <typename, typename> Op> struct CompileOp;
|
||||
|
||||
template <> struct CompileOp<EqualsOp>
|
||||
{
|
||||
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool /*is_signed*/)
|
||||
{
|
||||
return x->getType()->isIntegerTy() ? b.CreateICmpEQ(x, y) : b.CreateFCmpOEQ(x, y); /// qNaNs always compare false
|
||||
}
|
||||
};
|
||||
|
||||
template <> struct CompileOp<NotEqualsOp>
|
||||
{
|
||||
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool /*is_signed*/)
|
||||
{
|
||||
return x->getType()->isIntegerTy() ? b.CreateICmpNE(x, y) : b.CreateFCmpONE(x, y);
|
||||
}
|
||||
};
|
||||
|
||||
template <> struct CompileOp<LessOp>
|
||||
{
|
||||
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
|
||||
{
|
||||
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSLT(x, y) : b.CreateICmpULT(x, y)) : b.CreateFCmpOLT(x, y);
|
||||
}
|
||||
};
|
||||
|
||||
template <> struct CompileOp<GreaterOp>
|
||||
{
|
||||
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
|
||||
{
|
||||
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSGT(x, y) : b.CreateICmpUGT(x, y)) : b.CreateFCmpOGT(x, y);
|
||||
}
|
||||
};
|
||||
|
||||
template <> struct CompileOp<LessOrEqualsOp>
|
||||
{
|
||||
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
|
||||
{
|
||||
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSLE(x, y) : b.CreateICmpULE(x, y)) : b.CreateFCmpOLE(x, y);
|
||||
}
|
||||
};
|
||||
|
||||
template <> struct CompileOp<GreaterOrEqualsOp>
|
||||
{
|
||||
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * x, llvm::Value * y, bool is_signed)
|
||||
{
|
||||
return x->getType()->isIntegerTy() ? (is_signed ? b.CreateICmpSGE(x, y) : b.CreateICmpUGE(x, y)) : b.CreateFCmpOGE(x, y);
|
||||
}
|
||||
};
|
||||
|
||||
#endif
|
||||
|
||||
|
||||
struct NameEquals { static constexpr auto name = "equals"; };
|
||||
struct NameNotEquals { static constexpr auto name = "notEquals"; };
|
||||
struct NameLess { static constexpr auto name = "less"; };
|
||||
@ -1585,7 +1260,7 @@ public:
|
||||
x = nativeCast(b, types[0], x, common);
|
||||
y = nativeCast(b, types[1], y, common);
|
||||
}
|
||||
auto * result = Op<int, int>::compile(b, x, y, typeIsSigned(*types[0]) || typeIsSigned(*types[1]));
|
||||
auto * result = CompileOp<Op>::compile(b, x, y, typeIsSigned(*types[0]) || typeIsSigned(*types[1]));
|
||||
return b.CreateSelect(result, b.getInt8(1), b.getInt8(0));
|
||||
}
|
||||
#endif
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <Functions/FunctionsNull.h>
|
||||
#include <Functions/FunctionsLogical.h>
|
||||
#include <Functions/FunctionsComparison.h>
|
||||
#include <Functions/FunctionsConditional.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
@ -313,7 +312,11 @@ void FunctionNullIf::executeImpl(Block & block, const ColumnNumbers & arguments,
|
||||
size_t res_pos = temp_block.columns();
|
||||
temp_block.insert({nullptr, std::make_shared<DataTypeUInt8>(), ""});
|
||||
|
||||
FunctionEquals{context}.execute(temp_block, {arguments[0], arguments[1]}, res_pos, input_rows_count);
|
||||
{
|
||||
auto equals_func = FunctionFactory::instance().get("equals", context)->build(
|
||||
{block.getByPosition(arguments[0]), block.getByPosition(arguments[1])});
|
||||
equals_func->execute(temp_block, {arguments[0], arguments[1]}, res_pos, input_rows_count);
|
||||
}
|
||||
|
||||
/// Argument corresponding to the NULL value.
|
||||
size_t null_pos = temp_block.columns();
|
||||
|
Loading…
Reference in New Issue
Block a user