Merge pull request #50531 from kitaisreal/jit-infrastructure-refactoring

JIT infrastructure refactoring
This commit is contained in:
Alexey Milovidov 2023-06-20 21:13:44 +03:00 committed by GitHub
commit c097e6e53d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
39 changed files with 574 additions and 476 deletions

View File

@ -2,6 +2,7 @@
#include <cstdint>
#include <string>
#include <array>
#if defined(__SSE2__)
#include <emmintrin.h>

View File

@ -2,6 +2,8 @@
#include <base/types.h>
#include <vector>
namespace DB
{

View File

@ -4,6 +4,7 @@
#include <bitset>
#include <cstring>
#include <vector>
#include <unordered_map>
namespace DB

View File

@ -146,8 +146,8 @@ public:
for (const auto & argument : this->argument_types)
can_be_compiled &= canBeNativeType(*argument);
auto return_type = this->getResultType();
can_be_compiled &= canBeNativeType(*return_type);
const auto & result_type = this->getResultType();
can_be_compiled &= canBeNativeType(*result_type);
return can_be_compiled;
}
@ -198,8 +198,8 @@ public:
auto * denominator_ptr = b.CreateConstGEP1_32(b.getInt8Ty(), aggregate_data_ptr, denominator_offset);
auto * denominator_value = b.CreateLoad(denominator_type, denominator_ptr);
auto * double_numerator = nativeCast<Numerator>(b, numerator_value, b.getDoubleTy());
auto * double_denominator = nativeCast<Denominator>(b, denominator_value, b.getDoubleTy());
auto * double_numerator = nativeCast<Numerator>(b, numerator_value, this->getResultType());
auto * double_denominator = nativeCast<Denominator>(b, denominator_value, this->getResultType());
return b.CreateFDiv(double_numerator, double_denominator);
}
@ -308,7 +308,7 @@ public:
#if USE_EMBEDDED_COMPILER
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes & arguments_types, const std::vector<llvm::Value *> & argument_values) const override
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
@ -316,7 +316,7 @@ public:
auto * numerator_ptr = aggregate_data_ptr;
auto * numerator_value = b.CreateLoad(numerator_type, numerator_ptr);
auto * value_cast_to_numerator = nativeCast(b, arguments_types[0], argument_values[0], numerator_type);
auto * value_cast_to_numerator = nativeCast(b, arguments[0], toNativeDataType<Numerator>());
auto * numerator_result_value = numerator_type->isIntegerTy() ? b.CreateAdd(numerator_value, value_cast_to_numerator) : b.CreateFAdd(numerator_value, value_cast_to_numerator);
b.CreateStore(numerator_result_value, numerator_ptr);

View File

@ -30,7 +30,7 @@ public:
using Numerator = typename Base::Numerator;
using Denominator = typename Base::Denominator;
using Fraction = typename Base::Fraction;
using Fraction = typename Base::Fraction;
void NO_SANITIZE_UNDEFINED add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
@ -55,7 +55,7 @@ public:
return can_be_compiled;
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes & arguments_types, const std::vector<llvm::Value *> & argument_values) const override
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
@ -63,8 +63,9 @@ public:
auto * numerator_ptr = aggregate_data_ptr;
auto * numerator_value = b.CreateLoad(numerator_type, numerator_ptr);
auto * argument = nativeCast(b, arguments_types[0], argument_values[0], numerator_type);
auto * weight = nativeCast(b, arguments_types[1], argument_values[1], numerator_type);
auto numerator_data_type = toNativeDataType<Numerator>();
auto * argument = nativeCast(b, arguments[0], numerator_data_type);
auto * weight = nativeCast(b, arguments[1], numerator_data_type);
llvm::Value * value_weight_multiplication = argument->getType()->isIntegerTy() ? b.CreateMul(argument, weight) : b.CreateFMul(argument, weight);
auto * numerator_result_value = numerator_type->isIntegerTy() ? b.CreateAdd(numerator_value, value_weight_multiplication) : b.CreateFAdd(numerator_value, value_weight_multiplication);
@ -75,7 +76,7 @@ public:
static constexpr size_t denominator_offset = offsetof(Fraction, denominator);
auto * denominator_ptr = b.CreateConstInBoundsGEP1_64(b.getInt8Ty(), aggregate_data_ptr, denominator_offset);
auto * weight_cast_to_denominator = nativeCast(b, arguments_types[1], argument_values[1], denominator_type);
auto * weight_cast_to_denominator = nativeCast(b, arguments[1], toNativeDataType<Denominator>());
auto * denominator_value = b.CreateLoad(denominator_type, denominator_ptr);
auto * denominator_value_updated = denominator_type->isIntegerTy() ? b.CreateAdd(denominator_value, weight_cast_to_denominator) : b.CreateFAdd(denominator_value, weight_cast_to_denominator);

View File

@ -148,7 +148,7 @@ public:
Data::compileCreate(builder, value_ptr);
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes &, const std::vector<llvm::Value *> & argument_values) const override
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
@ -157,8 +157,7 @@ public:
auto * value_ptr = aggregate_data_ptr;
auto * value = b.CreateLoad(return_type, value_ptr);
const auto & argument_value = argument_values[0];
auto * result_value = Data::compileUpdate(builder, value, argument_value);
auto * result_value = Data::compileUpdate(builder, value, arguments[0].value);
b.CreateStore(result_value, value_ptr);
}

View File

@ -165,7 +165,7 @@ public:
b.CreateMemSet(aggregate_data_ptr, llvm::ConstantInt::get(b.getInt8Ty(), 0), sizeof(AggregateFunctionCountData), llvm::assumeAligned(this->alignOfData()));
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes &, const std::vector<llvm::Value *> &) const override
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType &) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
@ -309,13 +309,13 @@ public:
b.CreateMemSet(aggregate_data_ptr, llvm::ConstantInt::get(b.getInt8Ty(), 0), sizeof(AggregateFunctionCountData), llvm::assumeAligned(this->alignOfData()));
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes &, const std::vector<llvm::Value *> & values) const override
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * return_type = toNativeType(b, this->getResultType());
auto * is_null_value = b.CreateExtractValue(values[0], {1});
auto * is_null_value = b.CreateExtractValue(arguments[0].value, {1});
auto * increment_value = b.CreateSelect(is_null_value, llvm::ConstantInt::get(return_type, 0), llvm::ConstantInt::get(return_type, 1));
auto * count_value_ptr = aggregate_data_ptr;

View File

@ -188,18 +188,18 @@ public:
return canBeNativeType(*this->argument_types.back()) && this->nested_function->isCompilable();
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes & arguments_types, const std::vector<llvm::Value *> & argument_values) const override
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
const auto & nullable_type = arguments_types[0];
const auto & nullable_value = argument_values[0];
const auto & nullable_type = arguments[0].type;
const auto & nullable_value = arguments[0].value;
auto * wrapped_value = b.CreateExtractValue(nullable_value, {0});
auto * is_null_value = b.CreateExtractValue(nullable_value, {1});
const auto & predicate_type = arguments_types[argument_values.size() - 1];
auto * predicate_value = argument_values[argument_values.size() - 1];
const auto & predicate_type = arguments.back().type;
auto * predicate_value = arguments.back().value;
auto * is_predicate_true = nativeBoolCast(b, predicate_type, predicate_value);
auto * head = b.GetInsertBlock();
@ -219,7 +219,7 @@ public:
b.CreateStore(llvm::ConstantInt::get(b.getInt8Ty(), 1), aggregate_data_ptr);
auto * aggregate_data_ptr_with_prefix_size_offset = b.CreateConstInBoundsGEP1_64(b.getInt8Ty(), aggregate_data_ptr, this->prefix_size);
this->nested_function->compileAdd(b, aggregate_data_ptr_with_prefix_size_offset, { removeNullable(nullable_type) }, { wrapped_value });
this->nested_function->compileAdd(b, aggregate_data_ptr_with_prefix_size_offset, { ValueWithType(wrapped_value, removeNullable(nullable_type)) });
b.CreateBr(join_block);
b.SetInsertPoint(join_block);
@ -370,38 +370,31 @@ public:
return canBeNativeType(*this->argument_types.back()) && this->nested_function->isCompilable();
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes & arguments_types, const std::vector<llvm::Value *> & argument_values) const override
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
/// TODO: Check
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
size_t arguments_size = arguments_types.size();
size_t arguments_size = arguments.size();
ValuesWithType wrapped_arguments;
wrapped_arguments.reserve(arguments_size);
DataTypes non_nullable_types;
std::vector<llvm::Value * > wrapped_values;
std::vector<llvm::Value * > is_null_values;
non_nullable_types.resize(arguments_size);
wrapped_values.resize(arguments_size);
is_null_values.resize(arguments_size);
for (size_t i = 0; i < arguments_size; ++i)
{
const auto & argument_value = argument_values[i];
const auto & argument_value = arguments[i].value;
const auto & argument_type = arguments[i].type;
if (is_nullable[i])
{
auto * wrapped_value = b.CreateExtractValue(argument_value, {0});
is_null_values[i] = b.CreateExtractValue(argument_value, {1});
wrapped_values[i] = wrapped_value;
non_nullable_types[i] = removeNullable(arguments_types[i]);
is_null_values.emplace_back(b.CreateExtractValue(argument_value, {1}));
wrapped_arguments.emplace_back(wrapped_value, removeNullable(argument_type));
}
else
{
wrapped_values[i] = argument_value;
non_nullable_types[i] = arguments_types[i];
wrapped_arguments.emplace_back(argument_value, argument_type);
}
}
@ -415,9 +408,6 @@ public:
for (auto * is_null_value : is_null_values)
{
if (!is_null_value)
continue;
auto * values_have_null = b.CreateLoad(b.getInt1Ty(), values_have_null_ptr);
b.CreateStore(b.CreateOr(values_have_null, is_null_value), values_have_null_ptr);
}
@ -426,8 +416,8 @@ public:
b.SetInsertPoint(join_block_after_null_checks);
const auto & predicate_type = arguments_types[argument_values.size() - 1];
auto * predicate_value = argument_values[argument_values.size() - 1];
const auto & predicate_type = arguments.back().type;
auto * predicate_value = arguments.back().value;
auto * is_predicate_true = nativeBoolCast(b, predicate_type, predicate_value);
auto * if_true = llvm::BasicBlock::Create(head->getContext(), "if_true", head->getParent());
@ -444,7 +434,7 @@ public:
b.CreateStore(llvm::ConstantInt::get(b.getInt8Ty(), 1), aggregate_data_ptr);
auto * aggregate_data_ptr_with_prefix_size_offset = b.CreateConstInBoundsGEP1_64(b.getInt8Ty(), aggregate_data_ptr, this->prefix_size);
this->nested_function->compileAdd(b, aggregate_data_ptr_with_prefix_size_offset, non_nullable_types, wrapped_values);
this->nested_function->compileAdd(b, aggregate_data_ptr_with_prefix_size_offset, wrapped_arguments);
b.CreateBr(join_block);
b.SetInsertPoint(join_block);

View File

@ -223,12 +223,12 @@ public:
nested_func->compileCreate(builder, aggregate_data_ptr);
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes & arguments_types, const std::vector<llvm::Value *> & argument_values) const override
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
const auto & predicate_type = arguments_types[argument_values.size() - 1];
auto * predicate_value = argument_values[argument_values.size() - 1];
const auto & predicate_type = arguments.back().type;
auto * predicate_value = arguments.back().value;
auto * head = b.GetInsertBlock();
@ -242,21 +242,9 @@ public:
b.SetInsertPoint(if_true);
size_t arguments_size_without_predicate = arguments_types.size() - 1;
DataTypes argument_types_without_predicate;
std::vector<llvm::Value *> argument_values_without_predicate;
argument_types_without_predicate.resize(arguments_size_without_predicate);
argument_values_without_predicate.resize(arguments_size_without_predicate);
for (size_t i = 0; i < arguments_size_without_predicate; ++i)
{
argument_types_without_predicate[i] = arguments_types[i];
argument_values_without_predicate[i] = argument_values[i];
}
nested_func->compileAdd(builder, aggregate_data_ptr, argument_types_without_predicate, argument_values_without_predicate);
ValuesWithType arguments_without_predicate = arguments;
arguments_without_predicate.pop_back();
nested_func->compileAdd(builder, aggregate_data_ptr, arguments_without_predicate);
b.CreateBr(join_block);

View File

@ -1459,11 +1459,11 @@ public:
b.CreateMemSet(aggregate_data_ptr, llvm::ConstantInt::get(b.getInt8Ty(), 0), this->sizeOfData(), llvm::assumeAligned(this->alignOfData()));
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes &, const std::vector<llvm::Value *> & argument_values) const override
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
if constexpr (Data::is_compilable)
{
Data::compileChangeIfBetter(builder, aggregate_data_ptr, argument_values[0]);
Data::compileChangeIfBetter(builder, aggregate_data_ptr, arguments[0].value);
}
else
{

View File

@ -378,12 +378,12 @@ public:
#if USE_EMBEDDED_COMPILER
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes & arguments_types, const std::vector<llvm::Value *> & argument_values) const override
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
const auto & nullable_type = arguments_types[0];
const auto & nullable_value = argument_values[0];
const auto & nullable_type = arguments[0].type;
const auto & nullable_value = arguments[0].value;
auto * wrapped_value = b.CreateExtractValue(nullable_value, {0});
auto * is_null_value = b.CreateExtractValue(nullable_value, {1});
@ -405,7 +405,7 @@ public:
b.CreateStore(llvm::ConstantInt::get(b.getInt8Ty(), 1), aggregate_data_ptr);
auto * aggregate_data_ptr_with_prefix_size_offset = b.CreateConstInBoundsGEP1_64(b.getInt8Ty(), aggregate_data_ptr, this->prefix_size);
this->nested_function->compileAdd(b, aggregate_data_ptr_with_prefix_size_offset, { removeNullable(nullable_type) }, { wrapped_value });
this->nested_function->compileAdd(b, aggregate_data_ptr_with_prefix_size_offset, { ValueWithType(wrapped_value, removeNullable(nullable_type)) });
b.CreateBr(join_block);
b.SetInsertPoint(join_block);
@ -568,36 +568,32 @@ public:
#if USE_EMBEDDED_COMPILER
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes & arguments_types, const std::vector<llvm::Value *> & argument_values) const override
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
size_t arguments_size = arguments_types.size();
size_t arguments_size = arguments.size();
DataTypes non_nullable_types;
std::vector<llvm::Value * > wrapped_values;
std::vector<llvm::Value * > is_null_values;
ValuesWithType wrapped_arguments;
wrapped_arguments.reserve(arguments_size);
non_nullable_types.resize(arguments_size);
wrapped_values.resize(arguments_size);
is_null_values.resize(arguments_size);
std::vector<llvm::Value *> is_null_values;
is_null_values.reserve(arguments_size);
for (size_t i = 0; i < arguments_size; ++i)
{
const auto & argument_value = argument_values[i];
const auto & argument_value = arguments[i].value;
const auto & argument_type = arguments[i].type;
if (is_nullable[i])
{
auto * wrapped_value = b.CreateExtractValue(argument_value, {0});
is_null_values[i] = b.CreateExtractValue(argument_value, {1});
wrapped_values[i] = wrapped_value;
non_nullable_types[i] = removeNullable(arguments_types[i]);
is_null_values.emplace_back(b.CreateExtractValue(argument_value, {1}));
wrapped_arguments.emplace_back(wrapped_value, removeNullable(argument_type));
}
else
{
wrapped_values[i] = argument_value;
non_nullable_types[i] = arguments_types[i];
wrapped_arguments.emplace_back(argument_value, argument_type);
}
}
@ -612,9 +608,6 @@ public:
for (auto * is_null_value : is_null_values)
{
if (!is_null_value)
continue;
auto * values_have_null = b.CreateLoad(b.getInt1Ty(), values_have_null_ptr);
b.CreateStore(b.CreateOr(values_have_null, is_null_value), values_have_null_ptr);
}
@ -630,7 +623,7 @@ public:
b.CreateStore(llvm::ConstantInt::get(b.getInt8Ty(), 1), aggregate_data_ptr);
auto * aggregate_data_ptr_with_prefix_size_offset = b.CreateConstInBoundsGEP1_64(b.getInt8Ty(), aggregate_data_ptr, this->prefix_size);
this->nested_function->compileAdd(b, aggregate_data_ptr_with_prefix_size_offset, arguments_types, wrapped_values);
this->nested_function->compileAdd(b, aggregate_data_ptr_with_prefix_size_offset, wrapped_arguments);
b.CreateBr(join_block);
b.SetInsertPoint(join_block);

View File

@ -588,7 +588,7 @@ public:
b.CreateStore(llvm::Constant::getNullValue(return_type), aggregate_sum_ptr);
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes & arguments_types, const std::vector<llvm::Value *> & argument_values) const override
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
@ -597,10 +597,7 @@ public:
auto * sum_value_ptr = aggregate_data_ptr;
auto * sum_value = b.CreateLoad(return_type, sum_value_ptr);
const auto & argument_type = arguments_types[0];
const auto & argument_value = argument_values[0];
auto * value_cast_to_result = nativeCast(b, argument_type, argument_value, return_type);
auto * value_cast_to_result = nativeCast(b, arguments[0], this->getResultType());
auto * sum_result_value = sum_value->getType()->isIntegerTy() ? b.CreateAdd(sum_value, value_cast_to_result) : b.CreateFAdd(sum_value, value_cast_to_result);
b.CreateStore(sum_result_value, sum_value_ptr);

View File

@ -6,6 +6,7 @@
#include <Core/Block.h>
#include <Core/ColumnNumbers.h>
#include <Core/Field.h>
#include <Core/ValuesWithType.h>
#include <Interpreters/Context_fwd.h>
#include <base/types.h>
#include <Common/Exception.h>
@ -389,7 +390,7 @@ public:
}
/// compileAdd should generate code for updating aggregate function state stored in aggregate_data_ptr
virtual void compileAdd(llvm::IRBuilderBase & /*builder*/, llvm::Value * /*aggregate_data_ptr*/, const DataTypes & /*arguments_types*/, const std::vector<llvm::Value *> & /*arguments_values*/) const
virtual void compileAdd(llvm::IRBuilderBase & /*builder*/, llvm::Value * /*aggregate_data_ptr*/, const ValuesWithType & /*arguments*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is not JIT-compilable", getName());
}

View File

@ -1,5 +1,7 @@
#pragma once
#include <string>
#include <functional>
namespace DB
{

View File

@ -127,7 +127,7 @@ namespace fmt
template <typename FormatContext>
auto format(const DB::QualifiedTableName & name, FormatContext & ctx)
{
return format_to(ctx.out(), "{}.{}", DB::backQuoteIfNeed(name.database), DB::backQuoteIfNeed(name.table));
return fmt::format_to(ctx.out(), "{}.{}", DB::backQuoteIfNeed(name.database), DB::backQuoteIfNeed(name.table));
}
};
}

26
src/Core/ValueWithType.h Normal file
View File

@ -0,0 +1,26 @@
#pragma once
#include <DataTypes/IDataType.h>
namespace llvm
{
class Value;
}
namespace DB
{
/// LLVM value with its data type
struct ValueWithType
{
llvm::Value * value = nullptr;
DataTypePtr type;
ValueWithType() = default;
ValueWithType(llvm::Value * value_, DataTypePtr type_)
: value(value_)
, type(std::move(type_))
{}
};
}

13
src/Core/ValuesWithType.h Normal file
View File

@ -0,0 +1,13 @@
#pragma once
#include <vector>
#include <Core/ValueWithType.h>
namespace DB
{
using ValuesWithType = std::vector<ValueWithType>;
}

View File

@ -532,11 +532,6 @@ inline bool isNotDecimalButComparableToDecimal(const DataTypePtr & data_type)
return which.isInt() || which.isUInt() || which.isFloat();
}
inline bool isCompilableType(const DataTypePtr & data_type)
{
return data_type->isValueRepresentedByNumber() && !isDecimal(data_type);
}
inline bool isBool(const DataTypePtr & data_type)
{
return data_type->getName() == "Bool";

200
src/DataTypes/Native.cpp Normal file
View File

@ -0,0 +1,200 @@
#include <DataTypes/Native.h>
#if USE_EMBEDDED_COMPILER
# include <DataTypes/DataTypeNullable.h>
# include <Columns/ColumnConst.h>
# include <Columns/ColumnNullable.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
}
bool typeIsSigned(const IDataType & type)
{
WhichDataType data_type(type);
return data_type.isNativeInt() || data_type.isFloat() || data_type.isEnum() || data_type.isDate32();
}
llvm::Type * toNullableType(llvm::IRBuilderBase & builder, llvm::Type * type)
{
auto * is_null_type = builder.getInt1Ty();
return llvm::StructType::get(type, is_null_type);
}
bool canBeNativeType(const IDataType & type)
{
WhichDataType data_type(type);
if (data_type.isNullable())
{
const auto & data_type_nullable = static_cast<const DataTypeNullable&>(type);
return canBeNativeType(*data_type_nullable.getNestedType());
}
return data_type.isNativeInt() || data_type.isNativeUInt() || data_type.isFloat() || data_type.isDate()
|| data_type.isDate32() || data_type.isDateTime() || data_type.isEnum();
}
bool canBeNativeType(const DataTypePtr & type)
{
return canBeNativeType(*type);
}
llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const IDataType & type)
{
WhichDataType data_type(type);
if (data_type.isNullable())
{
const auto & data_type_nullable = static_cast<const DataTypeNullable&>(type);
auto * nested_type = toNativeType(builder, *data_type_nullable.getNestedType());
return toNullableType(builder, nested_type);
}
/// LLVM doesn't have unsigned types, it has unsigned instructions.
if (data_type.isInt8() || data_type.isUInt8())
return builder.getInt8Ty();
else if (data_type.isInt16() || data_type.isUInt16() || data_type.isDate())
return builder.getInt16Ty();
else if (data_type.isInt32() || data_type.isUInt32() || data_type.isDate32() || data_type.isDateTime())
return builder.getInt32Ty();
else if (data_type.isInt64() || data_type.isUInt64())
return builder.getInt64Ty();
else if (data_type.isFloat32())
return builder.getFloatTy();
else if (data_type.isFloat64())
return builder.getDoubleTy();
else if (data_type.isEnum8())
return builder.getInt8Ty();
else if (data_type.isEnum16())
return builder.getInt16Ty();
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid cast to native type");
}
llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const DataTypePtr & type)
{
return toNativeType(builder, *type);
}
llvm::Value * nativeBoolCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value)
{
if (from_type->isNullable())
{
auto * inner = nativeBoolCast(b, removeNullable(from_type), b.CreateExtractValue(value, {0}));
return b.CreateAnd(b.CreateNot(b.CreateExtractValue(value, {1})), inner);
}
auto * zero = llvm::Constant::getNullValue(value->getType());
if (value->getType()->isIntegerTy())
return b.CreateICmpNE(value, zero);
else if (value->getType()->isFloatingPointTy())
return b.CreateFCmpUNE(value, zero);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cast non-number {} to bool", from_type->getName());
}
llvm::Value * nativeBoolCast(llvm::IRBuilderBase & b, const ValueWithType & value_with_type)
{
return nativeBoolCast(b, value_with_type.type, value_with_type.value);
}
llvm::Value * nativeCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value, const DataTypePtr & to_type)
{
if (from_type->equals(*to_type))
{
return value;
}
else if (from_type->isNullable() && to_type->isNullable())
{
auto * inner = nativeCast(b, removeNullable(from_type), b.CreateExtractValue(value, {0}), to_type);
return b.CreateInsertValue(inner, b.CreateExtractValue(value, {1}), {1});
}
else if (from_type->isNullable())
{
return nativeCast(b, removeNullable(from_type), b.CreateExtractValue(value, {0}), to_type);
}
else if (to_type->isNullable())
{
auto * from_native_type = toNativeType(b, from_type);
auto * inner = nativeCast(b, from_type, value, removeNullable(to_type));
return b.CreateInsertValue(llvm::Constant::getNullValue(from_native_type), inner, {0});
}
else
{
auto * from_native_type = toNativeType(b, from_type);
auto * to_native_type = toNativeType(b, to_type);
if (from_native_type == to_native_type)
return value;
else if (from_native_type->isIntegerTy() && to_native_type->isFloatingPointTy())
return typeIsSigned(*from_type) ? b.CreateSIToFP(value, to_native_type) : b.CreateUIToFP(value, to_native_type);
else if (from_native_type->isFloatingPointTy() && to_native_type->isIntegerTy())
return typeIsSigned(*to_type) ? b.CreateFPToSI(value, to_native_type) : b.CreateFPToUI(value, to_native_type);
else if (from_native_type->isIntegerTy() && from_native_type->isIntegerTy())
return b.CreateIntCast(value, to_native_type, typeIsSigned(*from_type));
else if (to_native_type->isFloatingPointTy() && to_native_type->isFloatingPointTy())
return b.CreateFPCast(value, to_native_type);
}
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Invalid cast to native value from type {} to type {}",
from_type->getName(),
to_type->getName());
}
llvm::Value * nativeCast(llvm::IRBuilderBase & b, const ValueWithType & value, const DataTypePtr & to_type)
{
return nativeCast(b, value.type, value.value, to_type);
}
llvm::Constant * getColumnNativeValue(llvm::IRBuilderBase & builder, const DataTypePtr & column_type, const IColumn & column, size_t index)
{
if (const auto * constant = typeid_cast<const ColumnConst *>(&column))
return getColumnNativeValue(builder, column_type, constant->getDataColumn(), 0);
auto * type = toNativeType(builder, column_type);
WhichDataType column_data_type(column_type);
if (column_data_type.isNullable())
{
const auto & nullable_data_type = assert_cast<const DataTypeNullable &>(*column_type);
const auto & nullable_column = assert_cast<const ColumnNullable &>(column);
auto * value = getColumnNativeValue(builder, nullable_data_type.getNestedType(), nullable_column.getNestedColumn(), index);
auto * is_null = llvm::ConstantInt::get(type->getContainedType(1), nullable_column.isNullAt(index));
return llvm::ConstantStruct::get(static_cast<llvm::StructType *>(type), value, is_null);
}
else if (column_data_type.isFloat32())
{
return llvm::ConstantFP::get(type, assert_cast<const ColumnVector<Float32> &>(column).getElement(index));
}
else if (column_data_type.isFloat64())
{
return llvm::ConstantFP::get(type, assert_cast<const ColumnVector<Float64> &>(column).getElement(index));
}
else if (column_data_type.isNativeUInt() || column_data_type.isDate() || column_data_type.isDateTime())
{
return llvm::ConstantInt::get(type, column.getUInt(index));
}
else if (column_data_type.isNativeInt() || column_data_type.isEnum() || column_data_type.isDate32())
{
return llvm::ConstantInt::get(type, column.getInt(index));
}
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Cannot get native value for column with type {}",
column_type->getName());
}
}
#endif

View File

@ -4,65 +4,53 @@
#if USE_EMBEDDED_COMPILER
# include <Common/Exception.h>
# include <Core/ValueWithType.h>
# include <DataTypes/IDataType.h>
# include <DataTypes/DataTypeNullable.h>
# include <Columns/ColumnConst.h>
# include <Columns/ColumnNullable.h>
# include <llvm/IR/IRBuilder.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
}
static inline bool typeIsSigned(const IDataType & type)
/// Returns true if type is signed, false otherwise
bool typeIsSigned(const IDataType & type);
/// Cast LLVM type to nullable LLVM type
llvm::Type * toNullableType(llvm::IRBuilderBase & builder, llvm::Type * type);
/// Returns true if type can be native LLVM type, false otherwise
bool canBeNativeType(const IDataType & type);
/// Returns true if type can be native LLVM type, false otherwise
bool canBeNativeType(const DataTypePtr & type);
template <typename Type>
static inline bool canBeNativeType()
{
WhichDataType data_type(type);
return data_type.isNativeInt() || data_type.isFloat() || data_type.isEnum();
if constexpr (std::is_same_v<Type, Int8> || std::is_same_v<Type, UInt8>)
return true;
else if constexpr (std::is_same_v<Type, Int16> || std::is_same_v<Type, UInt16>)
return true;
else if constexpr (std::is_same_v<Type, Int32> || std::is_same_v<Type, UInt32>)
return true;
else if constexpr (std::is_same_v<Type, Int64> || std::is_same_v<Type, UInt64>)
return true;
else if constexpr (std::is_same_v<Type, Float32> || std::is_same_v<Type, Float64>)
return true;
return false;
}
static inline llvm::Type * toNullableType(llvm::IRBuilderBase & builder, llvm::Type * type)
{
auto * is_null_type = builder.getInt1Ty();
return llvm::StructType::get(type, is_null_type);
}
/// Cast type to native LLVM type
llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const IDataType & type);
static inline llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const IDataType & type)
{
WhichDataType data_type(type);
if (data_type.isNullable())
{
const auto & data_type_nullable = static_cast<const DataTypeNullable&>(type);
auto * wrapped = toNativeType(builder, *data_type_nullable.getNestedType());
auto * is_null_type = builder.getInt1Ty();
return wrapped ? llvm::StructType::get(wrapped, is_null_type) : nullptr;
}
/// LLVM doesn't have unsigned types, it has unsigned instructions.
if (data_type.isInt8() || data_type.isUInt8())
return builder.getInt8Ty();
else if (data_type.isInt16() || data_type.isUInt16() || data_type.isDate())
return builder.getInt16Ty();
else if (data_type.isInt32() || data_type.isUInt32() || data_type.isDate32() || data_type.isDateTime())
return builder.getInt32Ty();
else if (data_type.isInt64() || data_type.isUInt64())
return builder.getInt64Ty();
else if (data_type.isFloat32())
return builder.getFloatTy();
else if (data_type.isFloat64())
return builder.getDoubleTy();
else if (data_type.isEnum8())
return builder.getInt8Ty();
else if (data_type.isEnum16())
return builder.getInt16Ty();
return nullptr;
}
/// Cast type to native LLVM type
llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const DataTypePtr & type);
template <typename ToType>
static inline llvm::Type * toNativeType(llvm::IRBuilderBase & builder)
@ -80,203 +68,43 @@ static inline llvm::Type * toNativeType(llvm::IRBuilderBase & builder)
else if constexpr (std::is_same_v<ToType, Float64>)
return builder.getDoubleTy();
return nullptr;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid cast to native type");
}
template <typename Type>
static inline bool canBeNativeType()
template <typename ToType>
static inline DataTypePtr toNativeDataType()
{
if constexpr (std::is_same_v<Type, Int8> || std::is_same_v<Type, UInt8>)
return true;
else if constexpr (std::is_same_v<Type, Int16> || std::is_same_v<Type, UInt16>)
return true;
else if constexpr (std::is_same_v<Type, Int32> || std::is_same_v<Type, UInt32>)
return true;
else if constexpr (std::is_same_v<Type, Int64> || std::is_same_v<Type, UInt64>)
return true;
else if constexpr (std::is_same_v<Type, Float32>)
return true;
else if constexpr (std::is_same_v<Type, Float64>)
return true;
if constexpr (std::is_same_v<ToType, Int8> || std::is_same_v<ToType, UInt8> ||
std::is_same_v<ToType, Int16> || std::is_same_v<ToType, UInt16> ||
std::is_same_v<ToType, Int32> || std::is_same_v<ToType, UInt32> ||
std::is_same_v<ToType, Int64> || std::is_same_v<ToType, UInt64> ||
std::is_same_v<ToType, Float32> || std::is_same_v<ToType, Float64>)
return std::make_shared<DataTypeNumber<ToType>>();
return false;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid cast to native data type");
}
static inline bool canBeNativeType(const IDataType & type)
{
WhichDataType data_type(type);
/// Cast LLVM value with type to bool
llvm::Value * nativeBoolCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value);
if (data_type.isNullable())
{
const auto & data_type_nullable = static_cast<const DataTypeNullable&>(type);
return canBeNativeType(*data_type_nullable.getNestedType());
}
/// Cast LLVM value with type to bool
llvm::Value * nativeBoolCast(llvm::IRBuilderBase & b, const ValueWithType & value_with_type);
return data_type.isNativeInt() || data_type.isNativeUInt() || data_type.isFloat() || data_type.isDate()
|| data_type.isDate32() || data_type.isDateTime() || data_type.isEnum();
}
/// Cast LLVM value with type to specified type
llvm::Value * nativeCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value, const DataTypePtr & to_type);
static inline llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const DataTypePtr & type)
{
return toNativeType(builder, *type);
}
static inline llvm::Value * nativeBoolCast(llvm::IRBuilder<> & b, const DataTypePtr & from_type, llvm::Value * value)
{
if (from_type->isNullable())
{
auto * inner = nativeBoolCast(b, removeNullable(from_type), b.CreateExtractValue(value, {0}));
return b.CreateAnd(b.CreateNot(b.CreateExtractValue(value, {1})), inner);
}
auto * zero = llvm::Constant::getNullValue(value->getType());
if (value->getType()->isIntegerTy())
return b.CreateICmpNE(value, zero);
if (value->getType()->isFloatingPointTy())
return b.CreateFCmpUNE(value, zero);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cast non-number {} to bool", from_type->getName());
}
static inline llvm::Value * nativeCast(llvm::IRBuilder<> & b, const DataTypePtr & from, llvm::Value * value, llvm::Type * to_type)
{
auto * from_type = value->getType();
if (from_type == to_type)
return value;
else if (from_type->isIntegerTy() && to_type->isFloatingPointTy())
return typeIsSigned(*from) ? b.CreateSIToFP(value, to_type) : b.CreateUIToFP(value, to_type);
else if (from_type->isFloatingPointTy() && to_type->isIntegerTy())
return typeIsSigned(*from) ? b.CreateFPToSI(value, to_type) : b.CreateFPToUI(value, to_type);
else if (from_type->isIntegerTy() && to_type->isIntegerTy())
return b.CreateIntCast(value, to_type, typeIsSigned(*from));
else if (from_type->isFloatingPointTy() && to_type->isFloatingPointTy())
return b.CreateFPCast(value, to_type);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cast {} to requested type", from->getName());
}
/// Cast LLVM value with type to specified type
llvm::Value * nativeCast(llvm::IRBuilderBase & b, const ValueWithType & value, const DataTypePtr & to_type);
template <typename FromType>
static inline llvm::Value * nativeCast(llvm::IRBuilder<> & b, llvm::Value * value, llvm::Type * to_type)
static inline llvm::Value * nativeCast(llvm::IRBuilderBase & b, llvm::Value * value, const DataTypePtr & to)
{
auto * from_type = value->getType();
static constexpr bool from_type_is_signed = std::numeric_limits<FromType>::is_signed;
if (from_type == to_type)
return value;
else if (from_type->isIntegerTy() && to_type->isFloatingPointTy())
return from_type_is_signed ? b.CreateSIToFP(value, to_type) : b.CreateUIToFP(value, to_type);
else if (from_type->isFloatingPointTy() && to_type->isIntegerTy())
return from_type_is_signed ? b.CreateFPToSI(value, to_type) : b.CreateFPToUI(value, to_type);
else if (from_type->isIntegerTy() && to_type->isIntegerTy())
return b.CreateIntCast(value, to_type, from_type_is_signed);
else if (from_type->isFloatingPointTy() && to_type->isFloatingPointTy())
return b.CreateFPCast(value, to_type);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cast {} to requested type", TypeName<FromType>);
auto native_data_type = toNativeDataType<FromType>();
return nativeCast(b, native_data_type, value, to);
}
static inline llvm::Value * nativeCast(llvm::IRBuilder<> & b, const DataTypePtr & from, llvm::Value * value, const DataTypePtr & to)
{
auto * n_to = toNativeType(b, to);
if (value->getType() == n_to)
{
return value;
}
else if (from->isNullable() && to->isNullable())
{
auto * inner = nativeCast(b, removeNullable(from), b.CreateExtractValue(value, {0}), to);
return b.CreateInsertValue(inner, b.CreateExtractValue(value, {1}), {1});
}
else if (from->isNullable())
{
return nativeCast(b, removeNullable(from), b.CreateExtractValue(value, {0}), to);
}
else if (to->isNullable())
{
auto * inner = nativeCast(b, from, value, removeNullable(to));
return b.CreateInsertValue(llvm::Constant::getNullValue(n_to), inner, {0});
}
return nativeCast(b, from, value, n_to);
}
static inline std::pair<llvm::Value *, llvm::Value *> nativeCastToCommon(llvm::IRBuilder<> & b, const DataTypePtr & lhs_type, llvm::Value * lhs, const DataTypePtr & rhs_type, llvm::Value * rhs) /// NOLINT
{
llvm::Type * common;
bool lhs_is_signed = typeIsSigned(*lhs_type);
bool rhs_is_signed = typeIsSigned(*rhs_type);
if (lhs->getType()->isIntegerTy() && rhs->getType()->isIntegerTy())
{
/// if one integer has a sign bit, make sure the other does as well. llvm generates optimal code
/// (e.g. uses overflow flag on x86) for (word size + 1)-bit integer operations.
size_t lhs_bit_width = lhs->getType()->getIntegerBitWidth() + (!lhs_is_signed && rhs_is_signed);
size_t rhs_bit_width = rhs->getType()->getIntegerBitWidth() + (!rhs_is_signed && lhs_is_signed);
size_t max_bit_width = std::max(lhs_bit_width, rhs_bit_width);
common = b.getIntNTy(static_cast<unsigned>(max_bit_width));
}
else
{
/// TODO: Check
/// (double, float) or (double, int_N where N <= double's mantissa width) -> double
common = b.getDoubleTy();
}
auto * cast_lhs_to_common = nativeCast(b, lhs_type, lhs, common);
auto * cast_rhs_to_common = nativeCast(b, rhs_type, rhs, common);
return std::make_pair(cast_lhs_to_common, cast_rhs_to_common);
}
static inline llvm::Constant * getColumnNativeValue(llvm::IRBuilderBase & builder, const DataTypePtr & column_type, const IColumn & column, size_t index)
{
if (const auto * constant = typeid_cast<const ColumnConst *>(&column))
{
return getColumnNativeValue(builder, column_type, constant->getDataColumn(), 0);
}
WhichDataType column_data_type(column_type);
auto * type = toNativeType(builder, column_type);
if (!type || column.size() <= index)
return nullptr;
if (column_data_type.isNullable())
{
const auto & nullable_data_type = assert_cast<const DataTypeNullable &>(*column_type);
const auto & nullable_column = assert_cast<const ColumnNullable &>(column);
auto * value = getColumnNativeValue(builder, nullable_data_type.getNestedType(), nullable_column.getNestedColumn(), index);
auto * is_null = llvm::ConstantInt::get(type->getContainedType(1), nullable_column.isNullAt(index));
return value ? llvm::ConstantStruct::get(static_cast<llvm::StructType *>(type), value, is_null) : nullptr;
}
else if (column_data_type.isFloat32())
{
return llvm::ConstantFP::get(type, assert_cast<const ColumnVector<Float32> &>(column).getElement(index));
}
else if (column_data_type.isFloat64())
{
return llvm::ConstantFP::get(type, assert_cast<const ColumnVector<Float64> &>(column).getElement(index));
}
else if (column_data_type.isNativeUInt() || column_data_type.isDate() || column_data_type.isDateTime())
{
return llvm::ConstantInt::get(type, column.getUInt(index));
}
else if (column_data_type.isNativeInt() || column_data_type.isEnum() || column_data_type.isDate32())
{
return llvm::ConstantInt::get(type, column.getInt(index));
}
return nullptr;
}
/// Get column value for specified index as LLVM constant
llvm::Constant * getColumnNativeValue(llvm::IRBuilderBase & builder, const DataTypePtr & column_type, const IColumn & column, size_t index);
}

View File

@ -2046,51 +2046,68 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A
}
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes & arguments) const override
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]);
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 false;
else
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>;
return !std::is_same_v<ResultDataType, InvalidType> && !IsDataTypeDecimal<ResultDataType> && OpSpec::compilable;
if constexpr (!std::is_same_v<ResultDataType, InvalidType> && !IsDataTypeDecimal<ResultDataType> && OpSpec::compilable)
return true;
}
return false;
});
}
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, Values values) const override
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & result_type) const override
{
assert(2 == types.size() && 2 == values.size());
assert(2 == arguments.size());
llvm::Value * result = nullptr;
castBothTypes(types[0].get(), types[1].get(), [&](const auto & left, const auto & right)
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>)
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> && OpSpec::compilable)
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto type = std::make_shared<ResultDataType>();
auto * lval = nativeCast(b, types[0], values[0], type);
auto * rval = nativeCast(b, types[1], values[1], type);
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 result;
}
#endif

View File

@ -2,6 +2,7 @@
#include <Functions/IFunction.h>
#include <DataTypes/Native.h>
#include <DataTypes/DataTypeNullable.h>
#include "config.h"
@ -12,8 +13,11 @@ class FunctionIfBase : public IFunction
{
#if USE_EMBEDDED_COMPILER
public:
bool isCompilableImpl(const DataTypes & types) const override
bool isCompilableImpl(const DataTypes & types, const DataTypePtr & result_type) const override
{
if (!canBeNativeType(result_type))
return false;
/// It's difficult to compare Date and DateTime - cannot use JIT compilation.
bool has_date = false;
bool has_datetime = false;
@ -31,43 +35,43 @@ public:
if (has_date && has_datetime)
return false;
if (!isCompilableType(type_removed_nullable))
if (!canBeNativeType(type_removed_nullable))
return false;
}
return true;
}
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, Values values) const override
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & result_type) const override
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto return_type = getReturnTypeImpl(types);
auto * head = b.GetInsertBlock();
auto * join = llvm::BasicBlock::Create(head->getContext(), "join_block", head->getParent());
std::vector<std::pair<llvm::BasicBlock *, llvm::Value *>> returns;
for (size_t i = 0; i + 1 < types.size(); i += 2)
for (size_t i = 0; i + 1 < arguments.size(); i += 2)
{
auto * then = llvm::BasicBlock::Create(head->getContext(), "then_" + std::to_string(i), head->getParent());
auto * next = llvm::BasicBlock::Create(head->getContext(), "next_" + std::to_string(i), head->getParent());
auto * cond = values[i];
const auto & cond = arguments[i];
b.CreateCondBr(nativeBoolCast(b, types[i], cond), then, next);
b.CreateCondBr(nativeBoolCast(b, cond), then, next);
b.SetInsertPoint(then);
auto * value = nativeCast(b, types[i + 1], values[i + 1], return_type);
auto * value = nativeCast(b, arguments[i + 1], result_type);
returns.emplace_back(b.GetInsertBlock(), value);
b.CreateBr(join);
b.SetInsertPoint(next);
}
auto * else_value = nativeCast(b, types.back(), values.back(), return_type);
auto * else_value = nativeCast(b, arguments.back(), result_type);
returns.emplace_back(b.GetInsertBlock(), else_value);
b.CreateBr(join);
b.SetInsertPoint(join);
auto * phi = b.CreatePHI(toNativeType(b, return_type), static_cast<unsigned>(returns.size()));
auto * phi = b.CreatePHI(toNativeType(b, result_type), static_cast<unsigned>(returns.size()));
for (const auto & [block, value] : returns)
phi->addIncoming(value, block);

View File

@ -477,31 +477,45 @@ public:
}
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes & arguments) const override
bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr & result_type) const override
{
if (1 != arguments.size())
return false;
if (!canBeNativeType(*arguments[0]) || !canBeNativeType(*result_type))
return false;
return castType(arguments[0].get(), [&](const auto & type)
{
using DataType = std::decay_t<decltype(type)>;
if constexpr (std::is_same_v<DataTypeFixedString, DataType> || std::is_same_v<DataTypeString, DataType>)
{
return false;
}
else
return !IsDataTypeDecimal<DataType> && Op<typename DataType::FieldType>::compilable;
{
using T0 = typename DataType::FieldType;
using T1 = typename Op<T0>::ResultType;
if constexpr (!std::is_same_v<T1, InvalidType> && !IsDataTypeDecimal<DataType> && Op<T0>::compilable)
return true;
}
return false;
});
}
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, Values values) const override
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & result_type) const override
{
assert(1 == types.size() && 1 == values.size());
assert(1 == arguments.size());
llvm::Value * result = nullptr;
castType(types[0].get(), [&](const auto & type)
castType(arguments[0].type.get(), [&](const auto & type)
{
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;
@ -509,13 +523,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, types[0], values[0], std::make_shared<DataTypeNumber<T1>>());
auto * v = nativeCast(b, arguments[0], result_type);
result = Op<T0>::compile(b, v, is_signed_v<T1>);
return true;
}
}
return false;
});
return result;
}
#endif

View File

@ -1382,37 +1382,6 @@ public:
return executeGeneric(col_with_type_and_name_left, col_with_type_and_name_right);
}
}
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes & types) const override
{
if (2 != types.size())
return false;
WhichDataType data_type_lhs(types[0]);
WhichDataType data_type_rhs(types[1]);
auto is_big_integer = [](WhichDataType type) { return type.isUInt64() || type.isInt64(); };
if ((is_big_integer(data_type_lhs) && data_type_rhs.isFloat())
|| (is_big_integer(data_type_rhs) && data_type_lhs.isFloat())
|| (data_type_lhs.isDate() && data_type_rhs.isDateTime())
|| (data_type_rhs.isDate() && data_type_lhs.isDateTime()))
return false; /// TODO: implement (double, int_N where N > double's mantissa width)
return isCompilableType(types[0]) && isCompilableType(types[1]);
}
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, Values values) const override
{
assert(2 == types.size() && 2 == values.size());
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
auto [x, y] = nativeCastToCommon(b, types[0], values[0], types[1], values[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
};
}

View File

@ -184,41 +184,46 @@ public:
ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override;
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes &) const override { return useDefaultImplementationForNulls(); }
bool isCompilableImpl(const DataTypes &, const DataTypePtr &) const override { return useDefaultImplementationForNulls(); }
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, Values values) const override
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & values, const DataTypePtr &) const override
{
assert(!types.empty() && !values.empty());
assert(!values.empty());
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
if constexpr (!Impl::isSaturable())
{
auto * result = nativeBoolCast(b, types[0], values[0]);
for (size_t i = 1; i < types.size(); ++i)
result = Impl::apply(b, result, nativeBoolCast(b, types[i], values[i]));
auto * result = nativeBoolCast(b, values[0]);
for (size_t i = 1; i < values.size(); ++i)
result = Impl::apply(b, result, nativeBoolCast(b, values[i]));
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 < types.size(); ++i)
for (size_t i = 0; i < values.size(); ++i)
{
b.SetInsertPoint(next);
auto * value = values[i];
auto * truth = nativeBoolCast(b, types[i], value);
if (!types[i]->equals(DataTypeUInt8{}))
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 < types.size())
if (i + 1 < values.size())
{
next = llvm::BasicBlock::Create(next->getContext(), "", next->getParent());
b.CreateCondBr(truth, break_on_true ? stop : next, break_on_true ? next : stop);
}
}
b.CreateBr(stop);
b.SetInsertPoint(stop);
return phi;
}
#endif
@ -248,12 +253,12 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override;
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes &) const override { return true; }
bool isCompilableImpl(const DataTypes &, const DataTypePtr &) const override { return true; }
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, Values values) const override
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & values, const DataTypePtr &) const override
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
return b.CreateSelect(Impl<UInt8>::apply(b, nativeBoolCast(b, types[0], values[0])), b.getInt8(1), b.getInt8(0));
return b.CreateSelect(Impl<UInt8>::apply(b, nativeBoolCast(b, values[0])), b.getInt8(1), b.getInt8(0));
}
#endif
};

View File

@ -484,59 +484,74 @@ DataTypePtr IFunctionOverloadResolver::getReturnTypeWithoutLowCardinality(const
static std::optional<DataTypes> removeNullables(const DataTypes & types)
{
bool has_nullable = false;
for (const auto & type : types)
{
if (!typeid_cast<const DataTypeNullable *>(type.get()))
continue;
has_nullable = true;
break;
}
if (has_nullable)
{
DataTypes filtered;
filtered.reserve(types.size());
for (const auto & sub_type : types)
filtered.emplace_back(removeNullable(sub_type));
return filtered;
}
return {};
}
bool IFunction::isCompilable(const DataTypes & arguments) const
bool IFunction::isCompilable(const DataTypes & arguments, const DataTypePtr & result_type) const
{
if (useDefaultImplementationForNulls())
if (auto denulled = removeNullables(arguments))
return isCompilableImpl(*denulled);
return isCompilableImpl(arguments);
if (auto denulled_arguments = removeNullables(arguments))
return isCompilableImpl(*denulled_arguments, result_type);
return isCompilableImpl(arguments, result_type);
}
llvm::Value * IFunction::compile(llvm::IRBuilderBase & builder, const DataTypes & arguments, Values values) const
llvm::Value * IFunction::compile(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & result_type) const
{
auto denulled_arguments = removeNullables(arguments);
if (useDefaultImplementationForNulls() && denulled_arguments)
DataTypes arguments_types;
arguments_types.reserve(arguments.size());
for (const auto & argument : arguments)
arguments_types.push_back(argument.type);
auto denulled_arguments_types = removeNullables(arguments_types);
if (useDefaultImplementationForNulls() && denulled_arguments_types)
{
auto & b = static_cast<llvm::IRBuilder<> &>(builder);
std::vector<llvm::Value*> unwrapped_values;
std::vector<llvm::Value*> is_null_values;
ValuesWithType unwrapped_arguments;
unwrapped_arguments.reserve(arguments.size());
unwrapped_values.reserve(arguments.size());
is_null_values.reserve(arguments.size());
std::vector<llvm::Value*> is_null_values;
for (size_t i = 0; i < arguments.size(); ++i)
{
auto * value = values[i];
const auto & argument = arguments[i];
llvm::Value * unwrapped_value = argument.value;
WhichDataType data_type(arguments[i]);
if (data_type.isNullable())
if (argument.type->isNullable())
{
unwrapped_values.emplace_back(b.CreateExtractValue(value, {0}));
is_null_values.emplace_back(b.CreateExtractValue(value, {1}));
}
else
{
unwrapped_values.emplace_back(value);
unwrapped_value = b.CreateExtractValue(argument.value, {0});
is_null_values.emplace_back(b.CreateExtractValue(argument.value, {1}));
}
unwrapped_arguments.emplace_back(unwrapped_value, (*denulled_arguments_types)[i]);
}
auto * result = compileImpl(builder, *denulled_arguments, unwrapped_values);
auto * result = compileImpl(builder, unwrapped_arguments, removeNullable(result_type));
auto * nullable_structure_type = toNativeType(b, makeNullable(getReturnTypeImpl(*denulled_arguments)));
auto * nullable_structure_type = toNativeType(b, makeNullable(getReturnTypeImpl(*denulled_arguments_types)));
auto * nullable_structure_value = llvm::Constant::getNullValue(nullable_structure_type);
auto * nullable_structure_with_result_value = b.CreateInsertValue(nullable_structure_value, result, {0});
@ -548,7 +563,7 @@ llvm::Value * IFunction::compile(llvm::IRBuilderBase & builder, const DataTypes
return b.CreateInsertValue(nullable_structure_with_result_value, nullable_structure_result_null, {1});
}
return compileImpl(builder, arguments, std::move(values));
return compileImpl(builder, arguments, result_type);
}
#endif

View File

@ -3,6 +3,7 @@
#include <Core/ColumnNumbers.h>
#include <Core/ColumnsWithTypeAndName.h>
#include <Core/Field.h>
#include <Core/ValuesWithType.h>
#include <Core/Names.h>
#include <Core/IResolvedFunction.h>
#include <Common/Exception.h>
@ -121,8 +122,6 @@ private:
using ExecutableFunctionPtr = std::shared_ptr<IExecutableFunction>;
using Values = std::vector<llvm::Value *>;
/** Function with known arguments and return type (when the specific overload was chosen).
* It is also the point where all function-specific properties are known.
*/
@ -162,7 +161,7 @@ public:
* templates with default arguments is impossible and including LLVM in such a generic header
* as this one is a major pain.
*/
virtual llvm::Value * compile(llvm::IRBuilderBase & /*builder*/, Values /*values*/) const
virtual llvm::Value * compile(llvm::IRBuilderBase & /*builder*/, const ValuesWithType & /*arguments*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is not JIT-compilable", getName());
}
@ -530,9 +529,9 @@ public:
#if USE_EMBEDDED_COMPILER
bool isCompilable(const DataTypes & arguments) const;
bool isCompilable(const DataTypes & arguments, const DataTypePtr & result_type) const;
llvm::Value * compile(llvm::IRBuilderBase &, const DataTypes & arguments, Values values) const;
llvm::Value * compile(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & result_type) const;
#endif
@ -540,9 +539,9 @@ protected:
#if USE_EMBEDDED_COMPILER
virtual bool isCompilableImpl(const DataTypes &) const { return false; }
virtual bool isCompilableImpl(const DataTypes & /*arguments*/, const DataTypePtr & /*result_type*/) const { return false; }
virtual llvm::Value * compileImpl(llvm::IRBuilderBase &, const DataTypes &, Values) const
virtual llvm::Value * compileImpl(llvm::IRBuilderBase & /*builder*/, const ValuesWithType & /*arguments*/, const DataTypePtr & /*result_type*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is not JIT-compilable", getName());
}

View File

@ -55,11 +55,11 @@ public:
#if USE_EMBEDDED_COMPILER
bool isCompilable() const override { return function->isCompilable(getArgumentTypes()); }
bool isCompilable() const override { return function->isCompilable(getArgumentTypes(), getResultType()); }
llvm::Value * compile(llvm::IRBuilderBase & builder, Values values) const override
llvm::Value * compile(llvm::IRBuilderBase & builder, const ValuesWithType & compile_arguments) const override
{
return function->compile(builder, getArgumentTypes(), std::move(values));
return function->compile(builder, compile_arguments, getResultType());
}
#endif

View File

@ -160,9 +160,9 @@ public:
bool isCompilable() const override { return true; }
llvm::Value * compile(llvm::IRBuilderBase & builder, Values values) const override
llvm::Value * compile(llvm::IRBuilderBase & builder, const ValuesWithType & arguments) const override
{
return dag.compile(builder, values);
return dag.compile(builder, arguments).value;
}
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & arguments) const override

View File

@ -19,14 +19,14 @@ class JITModuleMemoryManager;
class JITSymbolResolver;
class JITCompiler;
/** Custom jit implementation
/** Custom JIT implementation.
* Main use cases:
* 1. Compiled functions in module.
* 2. Release memory for compiled functions.
*
* In LLVM library there are 2 main JIT stacks MCJIT and ORCv2.
*
* Main reasons for custom implementation vs MCJIT
* Main reasons for custom implementation vs MCJIT.
* MCJIT keeps llvm::Module and compiled object code before linking process after module was compiled.
* llvm::Module can be removed, but compiled object code cannot be removed. Memory for compiled code
* will be release only during MCJIT instance destruction. It is too expensive to create MCJIT

View File

@ -16,19 +16,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
llvm::Value * CompileDAG::compile(llvm::IRBuilderBase & builder, Values input_nodes_values) const
ValueWithType CompileDAG::compile(llvm::IRBuilderBase & builder, const ValuesWithType & input_nodes_values) const
{
assert(input_nodes_values.size() == getInputNodesCount());
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
PaddedPODArray<llvm::Value *> compiled_values;
compiled_values.resize_fill(nodes.size());
ValuesWithType compiled_values;
compiled_values.resize(nodes.size());
size_t input_nodes_values_index = 0;
size_t compiled_values_index = 0;
@ -44,31 +39,26 @@ llvm::Value * CompileDAG::compile(llvm::IRBuilderBase & builder, Values input_no
case CompileType::CONSTANT:
{
auto * native_value = getColumnNativeValue(b, node.result_type, *node.column, 0);
if (!native_value)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Cannot find native value for constant column with type {}",
node.result_type->getName());
compiled_values[compiled_values_index] = native_value;
compiled_values[compiled_values_index] = {native_value, node.result_type};
break;
}
case CompileType::FUNCTION:
{
Values temporary_values;
ValuesWithType temporary_values;
temporary_values.reserve(node.arguments.size());
for (auto argument_index : node.arguments)
{
assert(compiled_values[argument_index] != nullptr);
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);
compiled_values[compiled_values_index] = {node.function->compile(builder, temporary_values), node.result_type};
break;
}
case CompileType::INPUT:
{
compiled_values[compiled_values_index] = input_nodes_values[input_nodes_values_index];
compiled_values[compiled_values_index] = {input_nodes_values[input_nodes_values_index].value, node.result_type};
++input_nodes_values_index;
break;
}

View File

@ -53,7 +53,7 @@ public:
std::vector<size_t> arguments;
};
llvm::Value * compile(llvm::IRBuilderBase & builder, Values input_nodes_values) const;
ValueWithType compile(llvm::IRBuilderBase & builder, const ValuesWithType & input_nodes_values_with_type) const;
std::string dump() const;

View File

@ -9,6 +9,8 @@
#include <Common/Stopwatch.h>
#include <Common/ProfileEvents.h>
#include <DataTypes/Native.h>
#include <DataTypes/DataTypeNullable.h>
#include <Columns/ColumnNullable.h>
#include <Interpreters/JIT/CHJIT.h>
namespace
@ -107,7 +109,7 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio
/// Initialize column row values
Values arguments;
ValuesWithType arguments;
arguments.reserve(function_argument_types.size());
for (size_t i = 0; i < function_argument_types.size(); ++i)
@ -116,30 +118,30 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio
const auto & type = function_argument_types[i];
auto * column_data_ptr = column.data_ptr;
auto * column_element_value = b.CreateLoad(column.data_element_type, b.CreateGEP(column.data_element_type, column_data_ptr, counter_phi));
auto * column_element_value = b.CreateLoad(column.data_element_type, b.CreateInBoundsGEP(column.data_element_type, column_data_ptr, counter_phi));
if (!type->isNullable())
{
arguments.emplace_back(column_element_value);
arguments.emplace_back(column_element_value, type);
continue;
}
auto * column_is_null_element_value = b.CreateLoad(b.getInt8Ty(), b.CreateGEP(b.getInt8Ty(), column.null_data_ptr, counter_phi));
auto * column_is_null_element_value = b.CreateLoad(b.getInt8Ty(), b.CreateInBoundsGEP(b.getInt8Ty(), column.null_data_ptr, counter_phi));
auto * is_null = b.CreateICmpNE(column_is_null_element_value, b.getInt8(0));
auto * nullable_unitialized = llvm::Constant::getNullValue(toNullableType(b, column.data_element_type));
auto * nullable_value = b.CreateInsertValue(b.CreateInsertValue(nullable_unitialized, column_element_value, {0}), is_null, {1});
arguments.emplace_back(nullable_value);
arguments.emplace_back(nullable_value, type);
}
/// Compile values for column rows and store compiled value in result column
auto * result = function.compile(b, std::move(arguments));
auto * result_column_element_ptr = b.CreateGEP(columns.back().data_element_type, columns.back().data_ptr, counter_phi);
auto * result = function.compile(b, arguments);
auto * result_column_element_ptr = b.CreateInBoundsGEP(columns.back().data_element_type, columns.back().data_ptr, counter_phi);
if (columns.back().null_data_ptr)
{
b.CreateStore(b.CreateExtractValue(result, {0}), result_column_element_ptr);
auto * result_column_is_null_element_ptr = b.CreateGEP(b.getInt8Ty(), columns.back().null_data_ptr, counter_phi);
auto * result_column_is_null_element_ptr = b.CreateInBoundsGEP(b.getInt8Ty(), columns.back().null_data_ptr, counter_phi);
auto * is_result_column_element_null = b.CreateSelect(b.CreateExtractValue(result, {1}), b.getInt8(1), b.getInt8(0));
b.CreateStore(is_result_column_element_null, result_column_is_null_element_ptr);
}
@ -298,24 +300,24 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
else
aggregation_place = places_arg;
std::vector<llvm::Value *> function_arguments_values;
ValuesWithType function_arguments;
previous_columns_size = 0;
for (const auto & function : functions)
{
auto arguments_types = function.function->getArgumentTypes();
const auto & arguments_types = function.function->getArgumentTypes();
size_t function_arguments_size = arguments_types.size();
for (size_t column_argument_index = 0; column_argument_index < function_arguments_size; ++column_argument_index)
{
auto & column = columns[previous_columns_size + column_argument_index];
auto & argument_type = arguments_types[column_argument_index];
const auto & argument_type = arguments_types[column_argument_index];
auto * column_data_element = b.CreateLoad(column.data_element_type, b.CreateGEP(column.data_element_type, column.data_ptr, counter_phi));
if (!argument_type->isNullable())
{
function_arguments_values.push_back(column_data_element);
function_arguments.emplace_back(column_data_element, argument_type);
continue;
}
@ -324,16 +326,16 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
auto * nullable_unitialized = llvm::Constant::getNullValue(toNullableType(b, column.data_element_type));
auto * first_insert = b.CreateInsertValue(nullable_unitialized, column_data_element, {0});
auto * nullable_value = b.CreateInsertValue(first_insert, is_null, {1});
function_arguments_values.push_back(nullable_value);
function_arguments.emplace_back(nullable_value, argument_type);
}
size_t aggregate_function_offset = function.aggregate_data_offset;
auto * aggregation_place_with_offset = b.CreateConstInBoundsGEP1_64(b.getInt8Ty(), aggregation_place, aggregate_function_offset);
const auto * aggregate_function_ptr = function.function;
aggregate_function_ptr->compileAdd(b, aggregation_place_with_offset, arguments_types, function_arguments_values);
aggregate_function_ptr->compileAdd(b, aggregation_place_with_offset, function_arguments);
function_arguments_values.clear();
function_arguments.clear();
previous_columns_size += function_arguments_size;
}

View File

@ -43,7 +43,7 @@ struct fmt::formatter<DB::Part>
template <typename FormatContext>
auto format(const DB::Part & part, FormatContext & ctx)
{
return format_to(ctx.out(), "{} in replicas [{}]", part.description.describe(), fmt::join(part.replicas, ", "));
return fmt::format_to(ctx.out(), "{} in replicas [{}]", part.description.describe(), fmt::join(part.replicas, ", "));
}
};

View File

@ -15,7 +15,7 @@ struct fmt::formatter<DB::RangesInDataPartDescription>
template <typename FormatContext>
auto format(const DB::RangesInDataPartDescription & range, FormatContext & ctx)
{
return format_to(ctx.out(), "{}", range.describe());
return fmt::format_to(ctx.out(), "{}", range.describe());
}
};

View File

@ -0,0 +1,36 @@
SET compile_expressions = 1;
SET min_count_to_compile_expression = 0;
DROP TABLE IF EXISTS test_table_1;
CREATE TABLE test_table_1
(
pkey UInt32,
c8 UInt32,
c9 String,
c10 Float32,
c11 String
) ENGINE = MergeTree ORDER BY pkey;
DROP TABLE IF EXISTS test_table_2;
CREATE TABLE test_table_2
(
vkey UInt32,
pkey UInt32,
c15 UInt32
) ENGINE = MergeTree ORDER BY vkey;
WITH test_cte AS
(
SELECT
ref_10.c11 as c_2_c2350_1,
ref_9.c9 as c_2_c2351_2
FROM
test_table_1 as ref_9
RIGHT OUTER JOIN test_table_1 as ref_10 ON (ref_9.c11 = ref_10.c9)
INNER JOIN test_table_2 as ref_11 ON (ref_10.c8 = ref_11.vkey)
WHERE ((ref_10.pkey + ref_11.pkey) BETWEEN ref_11.vkey AND (CASE WHEN (-30.87 >= ref_9.c10) THEN ref_11.c15 ELSE ref_11.pkey END))
)
SELECT ref_13.c_2_c2350_1 as c_2_c2357_3 FROM test_cte as ref_13 WHERE (ref_13.c_2_c2351_2) in (select ref_14.c_2_c2351_2 as c_5_c2352_0 FROM test_cte as ref_14);
DROP TABLE test_table_1;
DROP TABLE test_table_2;

View File

@ -0,0 +1 @@
\N

View File

@ -0,0 +1,6 @@
SET compile_expressions = 1;
SET min_count_to_compile_expression = 0;
SELECT DISTINCT result FROM (SELECT toStartOfFifteenMinutes(toDateTime(toStartOfFifteenMinutes(toDateTime(1000.0001220703125) + (number * 65536))) + (number * 9223372036854775807)) AS result FROM system.numbers LIMIT 1048576) ORDER BY result DESC NULLS FIRST FORMAT Null; -- { serverError 407 }
SELECT DISTINCT result FROM (SELECT toStartOfFifteenMinutes(toDateTime(toStartOfFifteenMinutes(toDateTime(1000.0001220703125) + (number * 65536))) + toInt64(number * 9223372036854775807)) AS result FROM system.numbers LIMIT 1048576) ORDER BY result DESC NULLS FIRST FORMAT Null;
SELECT round(round(round(round(round(100)), round(round(round(round(NULL), round(65535)), toTypeName(now() + 9223372036854775807) LIKE 'DateTime%DateTime%DateTime%DateTime%', round(-2)), 255), round(NULL))));