This commit is contained in:
李扬 2024-11-21 14:14:32 +08:00 committed by GitHub
commit ef45faff05
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
23 changed files with 884 additions and 111 deletions

View File

@ -209,12 +209,14 @@ template <typename A, typename B> struct EqualsOp
using SymmetricOp = EqualsOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::equalsOp(a, b); }
static constexpr bool compilable = true;
};
template <typename A, typename B> struct NotEqualsOp
{
using SymmetricOp = NotEqualsOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::notEqualsOp(a, b); }
static constexpr bool compilable = true;
};
template <typename A, typename B> struct GreaterOp;
@ -223,12 +225,14 @@ template <typename A, typename B> struct LessOp
{
using SymmetricOp = GreaterOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::lessOp(a, b); }
static constexpr bool compilable = true;
};
template <typename A, typename B> struct GreaterOp
{
using SymmetricOp = LessOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::greaterOp(a, b); }
static constexpr bool compilable = true;
};
template <typename A, typename B> struct GreaterOrEqualsOp;
@ -237,12 +241,14 @@ template <typename A, typename B> struct LessOrEqualsOp
{
using SymmetricOp = GreaterOrEqualsOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::lessOrEqualsOp(a, b); }
static constexpr bool compilable = true;
};
template <typename A, typename B> struct GreaterOrEqualsOp
{
using SymmetricOp = LessOrEqualsOp<B, A>;
static UInt8 apply(A a, B b) { return accurate::greaterOrEqualsOp(a, b); }
static constexpr bool compilable = true;
};
}

View File

@ -555,6 +555,7 @@ inline bool isNullableOrLowCardinalityNullable(const DataTypePtr & data_type)
template <typename DataType> constexpr bool IsDataTypeDecimal = false;
template <typename DataType> constexpr bool IsDataTypeNumber = false;
template <typename DataType> constexpr bool IsDataTypeNativeNumber = false;
template <typename DataType> constexpr bool IsDataTypeDateOrDateTime = false;
template <typename DataType> constexpr bool IsDataTypeDate = false;
template <typename DataType> constexpr bool IsDataTypeEnum = false;
@ -581,6 +582,9 @@ template <is_decimal T> constexpr bool IsDataTypeDecimal<DataTypeDecimal<T>> = t
template <> inline constexpr bool IsDataTypeDecimal<DataTypeDateTime64> = true;
template <typename T> constexpr bool IsDataTypeNumber<DataTypeNumber<T>> = true;
template <typename T>
requires std::is_arithmetic_v<T>
constexpr bool IsDataTypeNativeNumber<DataTypeNumber<T>> = true;
template <> inline constexpr bool IsDataTypeDate<DataTypeDate> = true;
template <> inline constexpr bool IsDataTypeDate<DataTypeDate32> = true;

View File

@ -205,6 +205,39 @@ struct ResultOfIf
ConstructedType, Error>>>;
};
/** Type casting for `modulo` function:
* UInt<x>, UInt<y> -> UInt<max(x,y)>
* Int<x>, Int<y> -> Int<max(x,y)>
* UInt<x>, Int<y> -> Int<max(x*2, y)>
* UInt64, Int<x> -> Error
* Float<x>, Float<y> -> Float64
* Float<x>, [U]Int<y> -> Float64
*/
template <typename A, typename B>
struct ResultOfModuloNativePromotion
{
static_assert(is_arithmetic_v<A> && is_arithmetic_v<B>);
static constexpr bool has_float = std::is_floating_point_v<A> || std::is_floating_point_v<B>;
static constexpr bool has_integer = is_integer<A> || is_integer<B>;
static constexpr bool has_signed = is_signed_v<A> || is_signed_v<B>;
static constexpr bool has_unsigned = !is_signed_v<A> || !is_signed_v<B>;
static constexpr size_t max_size_of_unsigned_integer = max(is_signed_v<A> ? 0 : sizeof(A), is_signed_v<B> ? 0 : sizeof(B));
static constexpr size_t max_size_of_signed_integer = max(is_signed_v<A> ? sizeof(A) : 0, is_signed_v<B> ? sizeof(B) : 0);
static constexpr size_t max_size_of_integer = max(is_integer<A> ? sizeof(A) : 0, is_integer<B> ? sizeof(B) : 0);
using ConstructedType = typename Construct<
has_signed,
false,
(has_signed ^ has_unsigned) ? max(max_size_of_unsigned_integer * 2, max_size_of_signed_integer) : max(sizeof(A), sizeof(B))>::Type;
using Type = std::conditional_t<
std::is_same_v<A, B>,
A,
std::conditional_t<has_float, Float64, std::conditional_t<sizeof(ConstructedType) <= 8, ConstructedType, Error>>>;
};
/** Before applying operator `%` and bitwise operations, operands are casted to whole numbers. */
template <typename A> struct ToInteger
{

View File

@ -6,8 +6,14 @@
#include <Common/NaNUtils.h>
#include <DataTypes/NumberTraits.h>
#include "DataTypes/Native.h"
#include "config.h"
#if USE_EMBEDDED_COMPILER
# include <Core/ValuesWithType.h>
# include <llvm/IR/IRBuilder.h>
#endif
namespace DB
{
@ -15,8 +21,42 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_DIVISION;
extern const int LOGICAL_ERROR;
}
#if USE_EMBEDDED_COMPILER
template <typename F>
static llvm::Value * compileWithNullableValues(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed, F && compile_func)
{
auto * left_type = left->getType();
auto * right_type = right->getType();
if (!left_type->isStructTy() && !right_type->isStructTy())
{
// Both arguments are not nullable.
return compile_func(b, left, right, is_signed);
}
auto * denull_left = left_type->isStructTy() ? b.CreateExtractValue(left, {1}) : left;
auto * denull_right = right_type->isStructTy() ? b.CreateExtractValue(right, {1}) : right;
auto * denull_result = compile_func(b, denull_left, denull_right, is_signed);
auto * nullable_result_type = toNullableType(b, denull_result->getType());
llvm::Value * nullable_result = llvm::Constant::getNullValue(nullable_result_type);
nullable_result = b.CreateInsertValue(nullable_result, denull_result, {0});
auto * result_is_null = b.CreateExtractValue(nullable_result, {1});
if (left_type->isStructTy())
result_is_null = b.CreateOr(result_is_null, b.CreateExtractValue(left, {1}));
if (right_type->isStructTy())
result_is_null = b.CreateOr(result_is_null, b.CreateExtractValue(right, {1}));
return b.CreateInsertValue(nullable_result, result_is_null, {1});
}
#endif
template <typename A, typename B>
inline void throwIfDivisionLeadsToFPE(A a, B b)
{
@ -158,14 +198,39 @@ struct ModuloImpl
}
#if USE_EMBEDDED_COMPILER
static constexpr bool compilable = false; /// don't know how to throw from LLVM IR
#endif
static constexpr bool compilable = true; /// Ignore exceptions in LLVM IR
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed)
{
return compileWithNullableValues(
b,
left,
right,
is_signed,
[](auto & b_, auto * left_, auto * right_, auto is_signed_) { return compileImpl(b_, left_, right_, is_signed_); });
}
static llvm::Value * compileImpl(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed)
{
if (left->getType()->isFloatingPointTy())
return b.CreateFRem(left, right);
else if (left->getType()->isIntegerTy())
return is_signed ? b.CreateSRem(left, right) : b.CreateURem(left, right);
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "ModuloImpl compilation expected native integer or floating point type");
}
#endif
};
template <typename A, typename B>
struct ModuloLegacyImpl : ModuloImpl<A, B>
{
using ResultType = typename NumberTraits::ResultOfModuloLegacy<A, B>::Type;
#if USE_EMBEDDED_COMPILER
static constexpr bool compilable = false; /// moduloLegacy is only used in partition key expression
#endif
};
template <typename A, typename B>
@ -194,6 +259,36 @@ struct PositiveModuloImpl : ModuloImpl<A, B>
}
return static_cast<ResultType>(res);
}
#if USE_EMBEDDED_COMPILER
static constexpr bool compilable = true; /// Ignore exceptions in LLVM IR
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed)
{
return compileWithNullableValues(
b,
left,
right,
is_signed,
[](auto & b_, auto * left_, auto * right_, auto is_signed_) { return compileImpl(b_, left_, right_, is_signed_); });
}
static llvm::Value * compileImpl(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed)
{
auto * result = ModuloImpl<A, B>::compileImpl(b, left, right, is_signed);
if (is_signed)
{
/// If result is negative, result += abs(right).
auto * zero = llvm::Constant::getNullValue(result->getType());
auto * is_negative = b.CreateICmpSLT(result, zero);
auto * abs_right = b.CreateSelect(b.CreateICmpSLT(right, zero), b.CreateNeg(right), right);
return b.CreateSelect(is_negative, b.CreateAdd(result, abs_right), result);
}
else
return result;
}
#endif
};
}

View File

@ -810,6 +810,7 @@ class FunctionBinaryArithmetic : public IFunction
static constexpr bool is_division = IsOperation<Op>::division;
static constexpr bool is_bit_hamming_distance = IsOperation<Op>::bit_hamming_distance;
static constexpr bool is_modulo = IsOperation<Op>::modulo;
static constexpr bool is_positive_modulo = IsOperation<Op>::positive_modulo;
static constexpr bool is_int_div = IsOperation<Op>::int_div;
static constexpr bool is_int_div_or_zero = IsOperation<Op>::int_div_or_zero;
@ -2387,59 +2388,105 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A
if (!canBeNativeType(*arguments[0]) || !canBeNativeType(*arguments[1]) || !canBeNativeType(*result_type))
return false;
WhichDataType data_type_lhs(arguments[0]);
WhichDataType data_type_rhs(arguments[1]);
auto denull_left_type = removeNullable(arguments[0]);
auto denull_right_type = removeNullable(arguments[1]);
WhichDataType data_type_lhs(denull_left_type);
WhichDataType data_type_rhs(denull_right_type);
if ((data_type_lhs.isDateOrDate32() || data_type_lhs.isDateTime()) ||
(data_type_rhs.isDateOrDate32() || data_type_rhs.isDateTime()))
return false;
return castBothTypes(arguments[0].get(), arguments[1].get(), [&](const auto & left, const auto & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (!std::is_same_v<DataTypeFixedString, LeftDataType> &&
!std::is_same_v<DataTypeFixedString, RightDataType> &&
!std::is_same_v<DataTypeString, LeftDataType> &&
!std::is_same_v<DataTypeString, RightDataType>)
return castBothTypes(
denull_left_type.get(),
denull_right_type.get(),
[&](const auto & left, const auto & right)
{
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;
if constexpr (!std::is_same_v<ResultDataType, InvalidType> && !IsDataTypeDecimal<ResultDataType> && OpSpec::compilable)
return true;
}
return false;
});
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (
!std::is_same_v<DataTypeFixedString, LeftDataType> && !std::is_same_v<DataTypeFixedString, RightDataType>
&& !std::is_same_v<DataTypeString, LeftDataType> && !std::is_same_v<DataTypeString, RightDataType>)
{
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;
if constexpr (
!std::is_same_v<ResultDataType, InvalidType> && !IsDataTypeDecimal<ResultDataType>
&& !IsDataTypeDecimal<LeftDataType> && !IsDataTypeDecimal<RightDataType> && OpSpec::compilable)
{
if constexpr (is_modulo || is_positive_modulo)
{
using LeftType = typename LeftDataType::FieldType;
using RightType = typename RightDataType::FieldType;
using PromotedType = typename NumberTraits::ResultOfModuloNativePromotion<LeftType, RightType>::Type;
if constexpr (std::is_arithmetic_v<PromotedType>)
{
return true;
}
}
else
return true;
}
}
return false;
});
}
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & result_type) const override
{
assert(2 == arguments.size());
auto denull_left_type = removeNullable(arguments[0].type);
auto denull_right_type = removeNullable(arguments[1].type);
llvm::Value * result = nullptr;
castBothTypes(arguments[0].type.get(), arguments[1].type.get(), [&](const auto & left, const auto & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (!std::is_same_v<DataTypeFixedString, LeftDataType> &&
!std::is_same_v<DataTypeFixedString, RightDataType> &&
!std::is_same_v<DataTypeString, LeftDataType> &&
!std::is_same_v<DataTypeString, RightDataType>)
castBothTypes(
denull_left_type.get(),
denull_right_type.get(),
[&](const auto & left, const auto & right)
{
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;
if constexpr (!std::is_same_v<ResultDataType, InvalidType> && !IsDataTypeDecimal<ResultDataType> && OpSpec::compilable)
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (
!std::is_same_v<DataTypeFixedString, LeftDataType> && !std::is_same_v<DataTypeFixedString, RightDataType>
&& !std::is_same_v<DataTypeString, LeftDataType> && !std::is_same_v<DataTypeString, RightDataType>)
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * lval = nativeCast(b, arguments[0], result_type);
auto * rval = nativeCast(b, arguments[1], result_type);
result = OpSpec::compile(b, lval, rval, std::is_signed_v<typename ResultDataType::FieldType>);
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;
if constexpr (
!std::is_same_v<ResultDataType, InvalidType> && !IsDataTypeDecimal<ResultDataType>
&& !IsDataTypeDecimal<LeftDataType> && !IsDataTypeDecimal<RightDataType> && OpSpec::compilable)
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
if constexpr (is_modulo || is_positive_modulo)
{
using LeftType = typename LeftDataType::FieldType;
using RightType = typename RightDataType::FieldType;
using PromotedType = typename NumberTraits::ResultOfModuloNativePromotion<LeftType, RightType>::Type;
if constexpr (std::is_arithmetic_v<PromotedType>)
{
DataTypePtr promoted_type = std::make_shared<DataTypeNumber<PromotedType>>();
if (result_type->isNullable())
promoted_type = std::make_shared<DataTypeNullable>(promoted_type);
return true;
auto * lval = nativeCast(b, arguments[0], promoted_type);
auto * rval = nativeCast(b, arguments[1], promoted_type);
result = nativeCast(
b, promoted_type, OpSpec::compile(b, lval, rval, std::is_signed_v<PromotedType>), result_type);
return true;
}
}
else
{
auto * lval = nativeCast(b, arguments[0], result_type);
auto * rval = nativeCast(b, arguments[1], result_type);
result = OpSpec::compile(b, lval, rval, std::is_signed_v<typename ResultDataType::FieldType>);
return true;
}
}
}
}
return false;
});
return false;
});
return result;
}

View File

@ -489,9 +489,7 @@ public:
{
using DataType = std::decay_t<decltype(type)>;
if constexpr (std::is_same_v<DataTypeFixedString, DataType> || std::is_same_v<DataTypeString, DataType>)
{
return false;
}
else
{
using T0 = typename DataType::FieldType;
@ -513,9 +511,7 @@ public:
{
using DataType = std::decay_t<decltype(type)>;
if constexpr (std::is_same_v<DataTypeFixedString, DataType> || std::is_same_v<DataTypeString, DataType>)
{
return false;
}
else
{
using T0 = typename DataType::FieldType;
@ -523,8 +519,16 @@ public:
if constexpr (!std::is_same_v<T1, InvalidType> && !IsDataTypeDecimal<DataType> && Op<T0>::compilable)
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * v = nativeCast(b, arguments[0], result_type);
result = Op<T0>::compile(b, v, is_signed_v<T1>);
if constexpr (std::is_same_v<Op<T0>, AbsImpl<T0>> || std::is_same_v<Op<T0>, BitCountImpl<T0>>)
{
/// We don't need to cast the argument to the result type if it's abs/bitcount function.
result = Op<T0>::compile(b, arguments[0].value, is_signed_v<T0>);
}
else
{
auto * v = nativeCast(b, arguments[0], result_type);
result = Op<T0>::compile(b, v, is_signed_v<T1>);
}
return true;
}

View File

@ -1,17 +1,21 @@
#pragma once
#include <Common/memcmpSmall.h>
#include <Common/assert_cast.h>
#include <Common/TargetSpecific.h>
// Include this first, because `#define _asan_poison_address` from
// llvm/Support/Compiler.h conflicts with its forward declaration in
// sanitizer/asan_interface.h
#include <memory>
#include <limits>
#include <type_traits>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <Core/AccurateComparison.h>
#include <Core/DecimalComparison.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
@ -24,22 +28,23 @@
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/getLeastSupertype.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/castColumn.h>
#include <Functions/IFunctionAdaptors.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunctionAdaptors.h>
#include <Functions/IsOperation.h>
#include <Core/AccurateComparison.h>
#include <Core/DecimalComparison.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/ReadHelpers.h>
#include <Interpreters/castColumn.h>
#include <Interpreters/convertFieldToType.h>
#include <Common/TargetSpecific.h>
#include <Common/assert_cast.h>
#include <Common/memcmpSmall.h>
#include "DataTypes/NumberTraits.h"
#include <limits>
#include <type_traits>
#if USE_EMBEDDED_COMPILER
# include <DataTypes/Native.h>
# include <Functions/castTypeToEither.h>
# include <llvm/IR/IRBuilder.h>
#endif
namespace DB
{
@ -1356,6 +1361,109 @@ public:
return executeGeneric(col_with_type_and_name_left, col_with_type_and_name_right);
}
#if USE_EMBEDDED_COMPILER
template <typename F>
static bool castType(const IDataType * type, F && f)
{
return castTypeToEither<
DataTypeUInt8,
DataTypeUInt16,
DataTypeUInt32,
DataTypeUInt64,
DataTypeInt8,
DataTypeInt16,
DataTypeInt32,
DataTypeInt64,
DataTypeFloat32,
DataTypeFloat64>(type, std::forward<F>(f));
}
template <typename F>
static bool castBothTypes(const IDataType * left, const IDataType * right, F && f)
{
return castType(left, [&](const auto & left_)
{
return castType(right, [&](const auto & right_)
{
return f(left_, right_);
});
});
}
// bool isCompilableImpl(const DataTypes &, const DataTypePtr &) const override { return false; }
bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr & result_type) const override
{
if (2 != arguments.size())
return false;
if (!canBeNativeType(*arguments[0]) || !canBeNativeType(*arguments[1]) || !canBeNativeType(*result_type))
return false;
WhichDataType data_type_lhs(arguments[0]);
WhichDataType data_type_rhs(arguments[1]);
/// TODO support date/date32
if ((data_type_lhs.isDateOrDate32() || data_type_lhs.isDateTime()) ||
(data_type_rhs.isDateOrDate32() || data_type_rhs.isDateTime()))
return false;
return castBothTypes(arguments[0].get(), arguments[1].get(), [&](const auto & left, const auto & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
using LeftType = typename LeftDataType::FieldType;
using RightType = typename RightDataType::FieldType;
using PromotedType = typename NumberTraits::ResultOfIf<LeftType, RightType>::Type;
if constexpr (
!std::is_same_v<DataTypeFixedString, LeftDataType> && !std::is_same_v<DataTypeFixedString, RightDataType>
&& !std::is_same_v<DataTypeString, LeftDataType> && !std::is_same_v<DataTypeString, RightDataType>
&& (std::is_integral_v<PromotedType> || std::is_floating_point_v<PromotedType>))
{
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;
return OpSpec::compilable;
}
return false;
});
return false;
}
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr &) const override
{
assert(2 == arguments.size());
llvm::Value * result = nullptr;
castBothTypes(arguments[0].type.get(), arguments[1].type.get(), [&](const auto & left, const auto & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
using LeftType = typename LeftDataType::FieldType;
using RightType = typename RightDataType::FieldType;
using PromotedType = typename NumberTraits::ResultOfIf<LeftType, RightType>::Type;
if constexpr (
!std::is_same_v<DataTypeFixedString, LeftDataType> && !std::is_same_v<DataTypeFixedString, RightDataType>
&& !std::is_same_v<DataTypeString, LeftDataType> && !std::is_same_v<DataTypeString, RightDataType>
&& (std::is_integral_v<PromotedType> || std::is_floating_point_v<PromotedType>))
{
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;
if constexpr (OpSpec::compilable)
{
auto promoted_type = std::make_shared<DataTypeNumber<PromotedType>>();
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * left_value = nativeCast(b, arguments[0], promoted_type);
auto * right_value = nativeCast(b, arguments[1], promoted_type);
result = b.CreateSelect(
CompileOp<Op>::compile(b, left_value, right_value, std::is_signed_v<PromotedType>), b.getInt8(1), b.getInt8(0));
return true;
}
}
return false;
});
return result;
}
#endif
};
}

View File

@ -71,6 +71,9 @@
#include <Common/assert_cast.h>
#include <Common/quoteString.h>
#if USE_EMBEDDED_COMPILER
#include <DataTypes/Native.h>
#endif
namespace DB
{
@ -115,6 +118,43 @@ namespace ErrorCodes
namespace
{
#if USE_EMBEDDED_COMPILER
bool castType(const IDataType * type, auto && f)
{
using Types = TypeList<
DataTypeUInt8,
DataTypeUInt16,
DataTypeUInt32,
DataTypeUInt64,
DataTypeUInt128,
DataTypeUInt256,
DataTypeInt8,
DataTypeInt16,
DataTypeInt32,
DataTypeInt64,
DataTypeInt128,
DataTypeInt256,
DataTypeFloat32,
DataTypeFloat64,
DataTypeDecimal32,
DataTypeDecimal64,
DataTypeDecimal128,
DataTypeDecimal256,
DataTypeDate,
DataTypeDateTime,
DataTypeFixedString,
DataTypeString,
DataTypeInterval>;
return castTypeToEither(Types{}, type, std::forward<decltype(f)>(f));
}
template <typename F>
bool castBothTypes(const IDataType * left, const IDataType * right, F && f)
{
return castType(left, [&](const auto & left_) { return castType(right, [&](const auto & right_) { return f(left_, right_); }); });
}
#endif
/** Type conversion functions.
* toType - conversion in "natural way";
*/
@ -2238,6 +2278,52 @@ public:
return Monotonic::get(type, left, right);
}
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes & types, const DataTypePtr & result_type) const override
{
if (types.size() != 1)
return false;
if (!canBeNativeType(types[0]) || !canBeNativeType(result_type))
return false;
return castBothTypes(types[0].get(), result_type.get(), [](const auto & left, const auto & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (IsDataTypeNativeNumber<LeftDataType> && IsDataTypeNativeNumber<RightDataType>)
return true;
return false;
});
}
llvm::Value *
compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & result_type) const override
{
llvm::Value * result = nullptr;
castBothTypes(
arguments[0].type.get(),
result_type.get(),
[&](const auto & left, const auto & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (IsDataTypeNativeNumber<LeftDataType> && IsDataTypeNativeNumber<RightDataType>)
{
result = nativeCast(builder, arguments[0], result_type);
return true;
}
return false;
});
return result;
}
#endif
private:
ContextPtr context;
mutable bool checked_return_type = false;
@ -3228,6 +3314,61 @@ public:
return monotonicity_for_range(type, left, right);
}
#if USE_EMBEDDED_COMPILER
bool isCompilable() const override
{
if (getName() != "CAST" || argument_types.size() != 2)
return false;
const auto & from_type = argument_types[0];
const auto & to_type = return_type;
auto denull_from_type = removeNullable(from_type);
auto denull_to_type = removeNullable(to_type);
if (!canBeNativeType(denull_from_type) || !canBeNativeType(denull_to_type))
return false;
return castBothTypes(denull_from_type.get(), denull_to_type.get(), [](const auto & left, const auto & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (IsDataTypeNativeNumber<LeftDataType> && IsDataTypeNativeNumber<RightDataType>)
return true;
return false;
});
}
llvm::Value * compile(llvm::IRBuilderBase & builder, const ValuesWithType & arguments) const override
{
llvm::Value * result = nullptr;
const auto & from_type = arguments[0].type;
const auto & to_type = return_type;
auto denull_from_type = removeNullable(from_type);
auto denull_to_type = removeNullable(to_type);
castBothTypes(
denull_from_type.get(),
denull_to_type.get(),
[&](const auto & left, const auto & right)
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (IsDataTypeNativeNumber<LeftDataType> && IsDataTypeNativeNumber<RightDataType>)
{
result = nativeCast(builder, arguments[0], return_type);
return true;
}
return false;
});
return result;
}
#endif
private:
const char * cast_name;
MonotonicityForRange monotonicity_for_range;

View File

@ -39,6 +39,7 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_COLUMN;
extern const int NOT_IMPLEMENTED;
}
namespace
@ -511,6 +512,44 @@ ColumnPtr basicExecuteImpl(ColumnRawPtrs arguments, size_t input_rows_count)
}
namespace FunctionsLogicalDetail
{
#if USE_EMBEDDED_COMPILER
/// Cast LLVM value with type to ternary
llvm::Value * nativeTernaryCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value)
{
auto * result_type = llvm::Type::getInt8Ty(b.getContext());
if (from_type->isNullable())
{
auto * ternary_null = llvm::ConstantInt::get(result_type, 1);
auto * inner = nativeTernaryCast(b, removeNullable(from_type), b.CreateExtractValue(value, {0}));
auto * is_null = b.CreateExtractValue(value, {1});
return b.CreateSelect(is_null, ternary_null, inner);
}
auto * zero = llvm::Constant::getNullValue(value->getType());
auto * ternary_true = llvm::ConstantInt::get(result_type, 2);
auto * ternary_false = llvm::ConstantInt::get(result_type, 0);
if (value->getType()->isIntegerTy())
return b.CreateSelect(b.CreateICmpNE(value, zero), ternary_true, ternary_false);
else if (value->getType()->isFloatingPointTy())
return b.CreateSelect(b.CreateFCmpONE(value, zero), ternary_true, ternary_false);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cast non-number {} to ternary", from_type->getName());
}
/// Cast LLVM value with type to ternary
llvm::Value * nativeTernaryCast(llvm::IRBuilderBase & b, const ValueWithType & value_with_type)
{
return nativeTernaryCast(b, value_with_type.type, value_with_type.value);
}
#endif
}
template <typename Impl, typename Name>
DataTypePtr FunctionAnyArityLogical<Impl, Name>::getReturnTypeImpl(const DataTypes & arguments) const
{

View File

@ -6,7 +6,6 @@
#include <DataTypes/DataTypesNumber.h>
#include <Functions/IFunction.h>
#include <IO/WriteHelpers.h>
#include <type_traits>
#include <Interpreters/Context_fwd.h>
@ -79,6 +78,15 @@ namespace Ternary
}
}
#if USE_EMBEDDED_COMPILER
/// Cast LLVM value with type to Ternary
llvm::Value * nativeTernaryCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value);
/// Cast LLVM value with type to Ternary
llvm::Value * nativeTernaryCast(llvm::IRBuilderBase & b, const ValueWithType & value_with_type);
#endif
struct AndImpl
{
@ -98,6 +106,18 @@ struct AndImpl
/// Will use three-valued logic for NULLs (see above) or default implementation (any operation with NULL returns NULL).
static constexpr bool specialImplementationForNulls() { return true; }
#if USE_EMBEDDED_COMPILER
static llvm::Value * apply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b)
{
return builder.CreateAnd(a, b);
}
static llvm::Value * ternaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b)
{
return builder.CreateSelect(builder.CreateICmpUGT(a, b), b, a);
}
#endif
};
struct OrImpl
@ -110,6 +130,19 @@ struct OrImpl
static constexpr ResultType apply(UInt8 a, UInt8 b) { return a | b; }
static constexpr ResultType ternaryApply(UInt8 a, UInt8 b) { return std::max(a, b); }
static constexpr bool specialImplementationForNulls() { return true; }
#if USE_EMBEDDED_COMPILER
static llvm::Value * apply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b)
{
return builder.CreateOr(a, b);
}
static llvm::Value * ternaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b)
{
return builder.CreateSelect(builder.CreateICmpUGT(a, b), a, b);
}
#endif
};
struct XorImpl
@ -128,6 +161,12 @@ struct XorImpl
{
return builder.CreateXor(a, b);
}
static llvm::Value * ternaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b)
{
llvm::Value * xor_result = builder.CreateXor(a, b);
return builder.CreateSelect(xor_result, builder.getInt8(Ternary::True), builder.getInt8(Ternary::False));
}
#endif
};
@ -184,47 +223,51 @@ public:
ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override;
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes &, const DataTypePtr &) const override { return useDefaultImplementationForNulls(); }
bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr &) const override
{
for (const auto & arg : arguments)
{
if (!canBeNativeType(arg))
return false;
}
return true;
}
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & values, const DataTypePtr &) const override
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & values, const DataTypePtr & result_type) const override
{
assert(!values.empty());
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
if constexpr (!Impl::isSaturable())
if (useDefaultImplementationForNulls() || !result_type->isNullable())
{
auto * result = nativeBoolCast(b, values[0]);
llvm::Value * result = nativeBoolCast(b, values[0]);
for (size_t i = 1; i < values.size(); ++i)
result = Impl::apply(b, result, nativeBoolCast(b, values[i]));
{
llvm::Value * casted_value = nativeBoolCast(b, values[i]);
result = Impl::apply(b, result, casted_value);
}
return b.CreateSelect(result, b.getInt8(1), b.getInt8(0));
}
constexpr bool break_on_true = Impl::isSaturatedValue(true);
auto * next = b.GetInsertBlock();
auto * stop = llvm::BasicBlock::Create(next->getContext(), "", next->getParent());
b.SetInsertPoint(stop);
auto * phi = b.CreatePHI(b.getInt8Ty(), static_cast<unsigned>(values.size()));
for (size_t i = 0; i < values.size(); ++i)
else
{
b.SetInsertPoint(next);
auto * value = values[i].value;
auto * truth = nativeBoolCast(b, values[i]);
if (!values[i].type->equals(DataTypeUInt8{}))
value = b.CreateSelect(truth, b.getInt8(1), b.getInt8(0));
phi->addIncoming(value, b.GetInsertBlock());
if (i + 1 < values.size())
/// First we need to cast all values to ternary logic
llvm::Value * ternary_result = nativeTernaryCast(b, values[0]);
for (size_t i = 1; i < values.size(); ++i)
{
next = llvm::BasicBlock::Create(next->getContext(), "", next->getParent());
b.CreateCondBr(truth, break_on_true ? stop : next, break_on_true ? next : stop);
llvm::Value * casted_value = nativeTernaryCast(b, values[i]);
ternary_result = Impl::ternaryApply(b, ternary_result, casted_value);
}
/// Then transform ternary logic to struct which represents nullable result
llvm::Value * is_null = b.CreateICmpEQ(ternary_result, b.getInt8(Ternary::Null));
llvm::Value * is_true = b.CreateICmpEQ(ternary_result, b.getInt8(Ternary::True));
auto * nullable_result_type = toNativeType(b, result_type);
auto * nullable_result = llvm::Constant::getNullValue(nullable_result_type);
auto * nullable_result_with_value
= b.CreateInsertValue(nullable_result, b.CreateSelect(is_true, b.getInt8(1), b.getInt8(0)), {0});
return b.CreateInsertValue(nullable_result_with_value, is_null, {1});
}
b.CreateBr(stop);
b.SetInsertPoint(stop);
return phi;
}
#endif
};

View File

@ -6,6 +6,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A>
struct AbsImpl
{
@ -27,25 +32,65 @@ struct AbsImpl
}
#if USE_EMBEDDED_COMPILER
static constexpr bool compilable = false; /// special type handling, some other time
static constexpr bool compilable = true;
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * arg, bool sign)
{
const auto & type = arg->getType();
if (type->isIntegerTy())
{
if (sign)
{
auto & context = b.getContext();
auto * signed_type = arg->getType();
auto * unsigned_type = llvm::IntegerType::get(context, signed_type->getIntegerBitWidth());
auto * is_negative = b.CreateICmpSLT(arg, llvm::ConstantInt::get(signed_type, 0));
auto * neg_value = b.CreateNeg(arg);
auto * abs_value = b.CreateSelect(is_negative, neg_value, arg);
return b.CreateZExt(abs_value, unsigned_type);
}
else
{
return arg;
}
}
else if (type->isDoubleTy() || type->isFloatTy())
{
auto * func_fabs = llvm::Intrinsic::getDeclaration(b.GetInsertBlock()->getModule(), llvm::Intrinsic::fabs, {type});
return b.CreateCall(func_fabs, {arg});
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "AbsImpl compilation expected native integer or floating point type");
}
#endif
};
struct NameAbs { static constexpr auto name = "abs"; };
struct NameAbs
{
static constexpr auto name = "abs";
};
using FunctionAbs = FunctionUnaryArithmetic<AbsImpl, NameAbs, false>;
template <> struct FunctionUnaryArithmeticMonotonicity<NameAbs>
template <>
struct FunctionUnaryArithmeticMonotonicity<NameAbs>
{
static bool has() { return true; }
static IFunction::Monotonicity get(const Field & left, const Field & right)
{
Float64 left_float = left.isNull() ? -std::numeric_limits<Float64>::infinity() : applyVisitor(FieldVisitorConvertToNumber<Float64>(), left);
Float64 right_float = right.isNull() ? std::numeric_limits<Float64>::infinity() : applyVisitor(FieldVisitorConvertToNumber<Float64>(), right);
Float64 left_float
= left.isNull() ? -std::numeric_limits<Float64>::infinity() : applyVisitor(FieldVisitorConvertToNumber<Float64>(), left);
Float64 right_float
= right.isNull() ? std::numeric_limits<Float64>::infinity() : applyVisitor(FieldVisitorConvertToNumber<Float64>(), right);
if ((left_float < 0 && right_float > 0) || (left_float > 0 && right_float < 0))
return {};
return { .is_monotonic = true, .is_positive = std::min(left_float, right_float) >= 0, .is_strict = true, };
return {
.is_monotonic = true,
.is_positive = std::min(left_float, right_float) >= 0,
.is_strict = true,
};
}
};

View File

@ -4,6 +4,11 @@
#include <Core/ColumnNumbers.h>
#include <Columns/ColumnNullable.h>
#if USE_EMBEDDED_COMPILER
# include <DataTypes/Native.h>
# include <llvm/IR/IRBuilder.h>
#endif
namespace DB
{
@ -58,6 +63,22 @@ public:
return nullable_col->getNestedColumnPtr();
return col;
}
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr &) const override { return canBeNativeType(arguments[0]); }
llvm::Value *
compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & /*result_type*/) const override
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
if (arguments[0].type->isNullable())
return b.CreateExtractValue(arguments[0].value, {0});
else
return arguments[0].value;
}
#endif
};
}

View File

@ -7,6 +7,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A>
struct BitCountImpl
{
@ -38,7 +42,26 @@ struct BitCountImpl
}
#if USE_EMBEDDED_COMPILER
static constexpr bool compilable = false;
static constexpr bool compilable = true;
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * arg, bool)
{
const auto & type = arg->getType();
llvm::Value * int_value = nullptr;
if (type->isIntegerTy())
int_value = arg;
else if (type->isFloatTy())
int_value = b.CreateBitCast(arg, llvm::Type::getInt32Ty(b.getContext()));
else if (type->isDoubleTy())
int_value = b.CreateBitCast(arg, llvm::Type::getInt64Ty(b.getContext()));
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "BitCountImpl compilation expected native integer or floating-point type");
auto * func_ctpop = llvm::Intrinsic::getDeclaration(b.GetInsertBlock()->getModule(), llvm::Intrinsic::ctpop, {int_value->getType()});
llvm::Value * ctpop_value = b.CreateCall(func_ctpop, {int_value});
return b.CreateZExtOrTrunc(ctpop_value, llvm::Type::getInt8Ty(b.getContext()));
}
#endif
};

View File

@ -2,6 +2,10 @@
#include <Functions/IFunction.h>
#include <Interpreters/Context_fwd.h>
#if USE_EMBEDDED_COMPILER
# include <DataTypes/Native.h>
# include <llvm/IR/IRBuilder.h>
#endif
namespace DB
{
@ -11,6 +15,11 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
struct IdentityName
{
static constexpr auto name = "identity";
};
template<typename Name>
class FunctionIdentityBase : public IFunction
{
@ -32,12 +41,21 @@ public:
{
return arguments.front().column;
}
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes & /*types*/, const DataTypePtr & result_type) const override
{
return Name::name == IdentityName::name && canBeNativeType(result_type);
}
llvm::Value *
compileImpl(llvm::IRBuilderBase & /*builder*/, const ValuesWithType & arguments, const DataTypePtr & /*result_type*/) const override
{
return arguments[0].value;
}
#endif
};
struct IdentityName
{
static constexpr auto name = "identity";
};
struct ScalarSubqueryResultName
{

View File

@ -11,6 +11,11 @@
#include <Interpreters/Context.h>
#include <Common/assert_cast.h>
#if USE_EMBEDDED_COMPILER
# include <DataTypes/Native.h>
# include <llvm/IR/IRBuilder.h>
#endif
namespace DB
{
namespace Setting
@ -110,6 +115,23 @@ public:
return DataTypeUInt8().createColumnConst(elem.column->size(), 1u);
}
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr &) const override { return canBeNativeType(arguments[0]); }
llvm::Value *
compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & /*result_type*/) const override
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
if (arguments[0].type->isNullable())
{
auto * is_null = b.CreateExtractValue(arguments[0].value, {1});
return b.CreateNot(is_null);
}
else
return b.getInt8(1);
}
#endif
private:
MULTITARGET_FUNCTION_AVX2_SSE42(
MULTITARGET_FUNCTION_HEADER(static void NO_INLINE), vectorImpl, MULTITARGET_FUNCTION_BODY((const PaddedPODArray<UInt8> & null_map, PaddedPODArray<UInt8> & res) /// NOLINT

View File

@ -10,6 +10,11 @@
#include <Core/Settings.h>
#include <Interpreters/Context.h>
#if USE_EMBEDDED_COMPILER
# include <DataTypes/Native.h>
# include <llvm/IR/IRBuilder.h>
#endif
namespace DB
{
@ -107,6 +112,21 @@ public:
return DataTypeUInt8().createColumnConst(elem.column->size(), 0u);
}
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr &) const override { return canBeNativeType(arguments[0]); }
llvm::Value *
compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & /*result_type*/) const override
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
if (arguments[0].type->isNullable())
return b.CreateExtractValue(arguments[0].value, {1});
else
return b.getInt8(0);
}
#endif
private:
bool use_analyzer;
};

View File

@ -5,6 +5,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A>
struct SignImpl
{
@ -22,7 +28,44 @@ struct SignImpl
}
#if USE_EMBEDDED_COMPILER
static constexpr bool compilable = false;
static constexpr bool compilable = true;
static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * arg, bool sign)
{
auto * result_type = b.getInt8Ty();
auto * res_zero = llvm::ConstantInt::getSigned(result_type, 0);
auto * res_one = llvm::ConstantInt::getSigned(result_type, 1);
auto * res_minus_one = llvm::ConstantInt::getSigned(result_type, -1);
const auto & type = arg->getType();
if (type->isIntegerTy())
{
auto * zero = llvm::ConstantInt::get(type, 0, sign);
auto * is_zero = b.CreateICmpEQ(arg, zero);
if (sign)
{
auto * is_negative = b.CreateICmpSLT(arg, res_zero);
auto * select_zero = b.CreateSelect(is_zero, res_zero, res_one);
return b.CreateSelect(is_negative, res_minus_one, select_zero);
}
else
return b.CreateSelect(is_zero, res_zero, res_one);
}
else if (type->isDoubleTy() || type->isFloatTy())
{
auto * zero = llvm::ConstantFP::get(type, 0.0);
auto * is_zero = b.CreateFCmpOEQ(arg, zero);
auto * is_negative = b.CreateFCmpOLT(arg, zero);
auto * select_zero = b.CreateSelect(is_zero, res_zero, res_one);
return b.CreateSelect(is_negative, res_minus_one, select_zero);
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "SignImpl compilation expected native integer or floating point type");
}
#endif
};

View File

@ -1460,7 +1460,8 @@ ActionsDAG ActionsDAG::makeConvertingActions(
MatchColumnsMode mode,
bool ignore_constant_values,
bool add_casted_columns,
NameToNameMap * new_names)
NameToNameMap * new_names,
NameSet * columns_contain_compiled_function)
{
size_t num_input_columns = source.size();
size_t num_result_columns = result.size();
@ -1533,6 +1534,15 @@ ActionsDAG ActionsDAG::makeConvertingActions(
"Cannot convert column `{}` because it is constant but values of constants are different in source and result",
res_elem.name);
}
else if (columns_contain_compiled_function && columns_contain_compiled_function->contains(res_elem.name))
{
/// It may happen when JIT compilation is enabled that source column is constant and destination column is not constant.
/// e.g. expression "and(equals(materialize(null::Nullable(UInt64)), null::Nullable(UInt64)), equals(null::Nullable(UInt64), null::Nullable(UInt64)))"
/// compiled expression is "and(equals(input: Nullable(UInt64), null), null). Partial evaluation of the compiled expression isn't able to infer that the result column is constant.
/// It causes inconsistency between pipeline header(source column is not constant) and output header of ExpressionStep(destination column is constant).
/// So we need to convert non-constant column to constant column under this condition.
dst_node = &actions_dag.addColumn(res_elem);
}
else
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,

View File

@ -306,7 +306,8 @@ public:
MatchColumnsMode mode,
bool ignore_constant_values = false,
bool add_casted_columns = false,
NameToNameMap * new_names = nullptr);
NameToNameMap * new_names = nullptr,
NameSet * columns_contain_compiled_function = nullptr);
/// Create expression which add const column and then materialize it.
static ActionsDAG makeAddingColumnActions(ColumnWithTypeAndName column);

View File

@ -1,4 +1,5 @@
#include <Interpreters/Set.h>
#include <Common/logger_useful.h>
#include <Common/ProfileEvents.h>
#include <Interpreters/ArrayJoinAction.h>
#include <Interpreters/ExpressionActions.h>
@ -59,7 +60,15 @@ ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionAc
#if USE_EMBEDDED_COMPILER
if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes)
{
LOG_TEST(
getLogger("ExpressionActions"),
"Actions before compilation: {} with {} lazy_executed_nodes",
actions_dag.dumpDAG(),
lazy_executed_nodes.size());
actions_dag.compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes);
LOG_TEST(getLogger("ExpressionActions"), "Actions after compilation: {}", actions_dag.dumpDAG());
}
#endif
linearizeActions(lazy_executed_nodes);

View File

@ -3,6 +3,7 @@
#if USE_EMBEDDED_COMPILER
#include <sys/mman.h>
#include <cmath>
#include <boost/noncopyable.hpp>
@ -370,6 +371,9 @@ CHJIT::CHJIT()
symbol_resolver->registerSymbol("memset", reinterpret_cast<void *>(&memset));
symbol_resolver->registerSymbol("memcpy", reinterpret_cast<void *>(&memcpy));
symbol_resolver->registerSymbol("memcmp", reinterpret_cast<void *>(&memcmp));
double (*fmod_ptr)(double, double) = &fmod;
symbol_resolver->registerSymbol("fmod", reinterpret_cast<void *>(fmod_ptr));
}
CHJIT::~CHJIT() = default;

View File

@ -46,14 +46,18 @@ ValueWithType CompileDAG::compile(llvm::IRBuilderBase & builder, const ValuesWit
{
ValuesWithType temporary_values;
temporary_values.reserve(node.arguments.size());
for (auto argument_index : node.arguments)
{
assert(compiled_values[argument_index].value != nullptr);
temporary_values.emplace_back(compiled_values[argument_index]);
}
compiled_values[compiled_values_index] = {node.function->compile(builder, temporary_values), node.result_type};
ValueWithType compiled_value{node.function->compile(builder, temporary_values), node.function->getResultType()};
if (!node.result_type->equals(*node.function->getResultType()))
compiled_values[compiled_values_index] = {nativeCast(b, compiled_value, node.result_type), node.result_type};
else
compiled_values[compiled_values_index] = std::move(compiled_value);
break;
}
case CompileType::INPUT:

View File

@ -25,6 +25,34 @@ static ITransformingStep::Traits getTraits(const ActionsDAG & actions)
};
}
static bool containsCompiledFunction(const ActionsDAG::Node * node)
{
if (node->type == ActionsDAG::ActionType::FUNCTION && node->is_function_compiled)
return true;
const auto & children = node->children;
if (children.empty())
return false;
bool result = false;
for (const auto & child : children)
result |= containsCompiledFunction(child);
return result;
}
static NameSet getColumnsContainCompiledFunction(const ActionsDAG & actions_dag)
{
NameSet result;
for (const auto * node : actions_dag.getOutputs())
{
if (containsCompiledFunction(node))
{
result.insert(node->result_name);
}
}
return result;
}
ExpressionStep::ExpressionStep(const Header & input_header_, ActionsDAG actions_dag_)
: ITransformingStep(
input_header_,
@ -45,10 +73,15 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu
if (!blocksHaveEqualStructure(pipeline.getHeader(), *output_header))
{
auto columns_contain_compiled_function = getColumnsContainCompiledFunction(expression->getActionsDAG());
auto convert_actions_dag = ActionsDAG::makeConvertingActions(
pipeline.getHeader().getColumnsWithTypeAndName(),
output_header->getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Name);
pipeline.getHeader().getColumnsWithTypeAndName(),
output_header->getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Name,
false,
false,
nullptr,
&columns_contain_compiled_function);
auto convert_actions = std::make_shared<ExpressionActions>(std::move(convert_actions_dag), settings.getActionsSettings());
pipeline.addSimpleTransform([&](const Block & header)