mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
Refactored interfaces
This commit is contained in:
parent
24798ef07c
commit
c79d7eae21
@ -879,7 +879,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
|||||||
global_context->setMMappedFileCache(mmap_cache_size);
|
global_context->setMMappedFileCache(mmap_cache_size);
|
||||||
|
|
||||||
#if USE_EMBEDDED_COMPILER
|
#if USE_EMBEDDED_COMPILER
|
||||||
size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", 500);
|
constexpr size_t compiled_expression_cache_size_default = 1024 * 1024 * 1024;
|
||||||
|
size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", compiled_expression_cache_size_default);
|
||||||
CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_size);
|
CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_size);
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
@ -10,6 +10,8 @@
|
|||||||
# include <DataTypes/IDataType.h>
|
# include <DataTypes/IDataType.h>
|
||||||
# include <DataTypes/DataTypeNullable.h>
|
# include <DataTypes/DataTypeNullable.h>
|
||||||
# include <DataTypes/DataTypeFixedString.h>
|
# include <DataTypes/DataTypeFixedString.h>
|
||||||
|
# include <Columns/ColumnConst.h>
|
||||||
|
# include <Columns/ColumnNullable.h>
|
||||||
# pragma GCC diagnostic push
|
# pragma GCC diagnostic push
|
||||||
# pragma GCC diagnostic ignored "-Wunused-parameter"
|
# pragma GCC diagnostic ignored "-Wunused-parameter"
|
||||||
|
|
||||||
@ -142,6 +144,74 @@ static inline llvm::Value * nativeCast(llvm::IRBuilder<> & b, const DataTypePtr
|
|||||||
return nativeCast(b, from, value, n_to);
|
return nativeCast(b, from, value, n_to);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static inline llvm::Constant * getColumnNativeConstant(llvm::Type * native_type, WhichDataType column_data_type, const IColumn & column)
|
||||||
|
{
|
||||||
|
llvm::Constant * result = nullptr;
|
||||||
|
|
||||||
|
if (column_data_type.isFloat32())
|
||||||
|
{
|
||||||
|
result = llvm::ConstantFP::get(native_type, column.getFloat32(0));
|
||||||
|
}
|
||||||
|
else if (column_data_type.isFloat64())
|
||||||
|
{
|
||||||
|
result = llvm::ConstantFP::get(native_type, column.getFloat64(0));
|
||||||
|
}
|
||||||
|
else if (column_data_type.isNativeInt())
|
||||||
|
{
|
||||||
|
result = llvm::ConstantInt::get(native_type, column.getInt(0));
|
||||||
|
}
|
||||||
|
else if (column_data_type.isNativeUInt())
|
||||||
|
{
|
||||||
|
result = llvm::ConstantInt::get(native_type, column.getUInt(0));
|
||||||
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
static inline llvm::Constant * getColumnNativeValue(llvm::IRBuilderBase & builder, const DataTypePtr & column_type, const IColumn & column, size_t index)
|
||||||
|
{
|
||||||
|
WhichDataType column_data_type(column_type);
|
||||||
|
auto * type = toNativeType(builder, column_type);
|
||||||
|
|
||||||
|
if (!type || column.size() <= index)
|
||||||
|
return nullptr;
|
||||||
|
|
||||||
|
if (const auto * constant = typeid_cast<const ColumnConst *>(&column))
|
||||||
|
{
|
||||||
|
return getColumnNativeValue(builder, column_type, constant->getDataColumn(), 0);
|
||||||
|
}
|
||||||
|
else 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())
|
||||||
|
{
|
||||||
|
return llvm::ConstantInt::get(type, column.getUInt(index));
|
||||||
|
}
|
||||||
|
else if (column_data_type.isNativeInt())
|
||||||
|
{
|
||||||
|
return llvm::ConstantInt::get(type, column.getInt(index));
|
||||||
|
}
|
||||||
|
|
||||||
|
return nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
@ -557,65 +557,6 @@ llvm::Value * IFunction::compile(llvm::IRBuilderBase & builder, const DataTypes
|
|||||||
nullable_structure_result_null = b.CreateOr(nullable_structure_result_null, is_null_value);
|
nullable_structure_result_null = b.CreateOr(nullable_structure_result_null, is_null_value);
|
||||||
|
|
||||||
return b.CreateInsertValue(nullable_structure_with_result_value, nullable_structure_result_null, {1});
|
return b.CreateInsertValue(nullable_structure_with_result_value, nullable_structure_result_null, {1});
|
||||||
// DataTypes non_null_arguments;
|
|
||||||
// non_null_arguments.reserve(arguments.size());
|
|
||||||
|
|
||||||
// for (size_t i = 0; i < arguments.size(); ++i)
|
|
||||||
// {
|
|
||||||
// WhichDataType data_type(arguments[i]);
|
|
||||||
// if (data_type.isNullable())
|
|
||||||
// {
|
|
||||||
|
|
||||||
// }
|
|
||||||
// else
|
|
||||||
// {
|
|
||||||
|
|
||||||
// }
|
|
||||||
|
|
||||||
|
|
||||||
// non_null_arguments.emplace_back(removeNullable(arguments[i]));
|
|
||||||
// auto * value = values[i]();
|
|
||||||
|
|
||||||
// }
|
|
||||||
|
|
||||||
// if (auto denulled = removeNullables(arguments))
|
|
||||||
// {
|
|
||||||
// DataTypes denulled_types = *denulled;
|
|
||||||
// std::cerr << "IFunction::denulled types " << std::endl;
|
|
||||||
// for (size_t i = 0; i < denulled_types.size(); ++i)
|
|
||||||
// {
|
|
||||||
// std::cerr << "Index " << i << " name " << denulled_types[i]->getName() << std::endl;
|
|
||||||
// }
|
|
||||||
// /// FIXME: when only one column is nullable, this can actually be slower than the non-jitted version
|
|
||||||
// /// because this involves copying the null map while `wrapInNullable` reuses it.
|
|
||||||
// auto & b = static_cast<llvm::IRBuilder<> &>(builder);
|
|
||||||
// auto * fail = llvm::BasicBlock::Create(b.GetInsertBlock()->getContext(), "", b.GetInsertBlock()->getParent());
|
|
||||||
// auto * join = llvm::BasicBlock::Create(b.GetInsertBlock()->getContext(), "", b.GetInsertBlock()->getParent());
|
|
||||||
// auto * zero = llvm::Constant::getNullValue(toNativeType(b, makeNullable(getReturnTypeImpl(*denulled))));
|
|
||||||
// for (size_t i = 0; i < arguments.size(); i++)
|
|
||||||
// {
|
|
||||||
// if (!arguments[i]->isNullable())
|
|
||||||
// continue;
|
|
||||||
// /// Would be nice to evaluate all this lazily, but that'd change semantics: if only unevaluated
|
|
||||||
// /// arguments happen to contain NULLs, the return value would not be NULL, though it should be.
|
|
||||||
// auto * value = values[i]();
|
|
||||||
// auto * ok = llvm::BasicBlock::Create(b.GetInsertBlock()->getContext(), "", b.GetInsertBlock()->getParent());
|
|
||||||
// b.CreateCondBr(b.CreateExtractValue(value, {1}), fail, ok);
|
|
||||||
// b.SetInsertPoint(ok);
|
|
||||||
// values[i] = [value = b.CreateExtractValue(value, {0})]() { return value; };
|
|
||||||
// }
|
|
||||||
// auto * result = b.CreateInsertValue(zero, compileImpl(builder, *denulled, std::move(values)), {0});
|
|
||||||
// auto * result_columns = b.GetInsertBlock();
|
|
||||||
// b.CreateBr(join);
|
|
||||||
// b.SetInsertPoint(fail);
|
|
||||||
// auto * null = b.CreateInsertValue(zero, b.getTrue(), {1});
|
|
||||||
// b.CreateBr(join);
|
|
||||||
// b.SetInsertPoint(join);
|
|
||||||
// auto * phi = b.CreatePHI(result->getType(), 2);
|
|
||||||
// phi->addIncoming(result, result_columns);
|
|
||||||
// phi->addIncoming(null, fail);
|
|
||||||
// return phi;
|
|
||||||
// }
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return compileImpl(builder, arguments, std::move(values));
|
return compileImpl(builder, arguments, std::move(values));
|
||||||
|
@ -450,6 +450,12 @@ struct ContextSharedPart
|
|||||||
/// TODO: Get rid of this.
|
/// TODO: Get rid of this.
|
||||||
|
|
||||||
delete_system_logs = std::move(system_logs);
|
delete_system_logs = std::move(system_logs);
|
||||||
|
|
||||||
|
#if USE_EMBEDDED_COMPILER
|
||||||
|
if (auto * cache = CompiledExpressionCacheFactory::instance().tryGetCache())
|
||||||
|
cache->reset();
|
||||||
|
#endif
|
||||||
|
|
||||||
embedded_dictionaries.reset();
|
embedded_dictionaries.reset();
|
||||||
external_dictionaries_loader.reset();
|
external_dictionaries_loader.reset();
|
||||||
models_repository_guard.reset();
|
models_repository_guard.reset();
|
||||||
|
@ -116,10 +116,6 @@ struct BackgroundTaskSchedulingSettings;
|
|||||||
class ZooKeeperMetadataTransaction;
|
class ZooKeeperMetadataTransaction;
|
||||||
using ZooKeeperMetadataTransactionPtr = std::shared_ptr<ZooKeeperMetadataTransaction>;
|
using ZooKeeperMetadataTransactionPtr = std::shared_ptr<ZooKeeperMetadataTransaction>;
|
||||||
|
|
||||||
#if USE_EMBEDDED_COMPILER
|
|
||||||
class CompiledExpressionCache;
|
|
||||||
#endif
|
|
||||||
|
|
||||||
/// Callback for external tables initializer
|
/// Callback for external tables initializer
|
||||||
using ExternalTablesInitializer = std::function<void(ContextPtr)>;
|
using ExternalTablesInitializer = std::function<void(ContextPtr)>;
|
||||||
|
|
||||||
|
@ -10,27 +10,15 @@
|
|||||||
#include <Columns/ColumnVector.h>
|
#include <Columns/ColumnVector.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Common/assert_cast.h>
|
#include <Common/assert_cast.h>
|
||||||
#include <Common/ProfileEvents.h>
|
|
||||||
#include <Common/Stopwatch.h>
|
|
||||||
#include <DataTypes/DataTypeNullable.h>
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <DataTypes/Native.h>
|
#include <DataTypes/Native.h>
|
||||||
#include <Functions/IFunctionAdaptors.h>
|
#include <Functions/IFunctionAdaptors.h>
|
||||||
#include <IO/WriteBufferFromString.h>
|
|
||||||
#include <IO/Operators.h>
|
|
||||||
|
|
||||||
#include <llvm/IR/BasicBlock.h>
|
|
||||||
#include <llvm/IR/Function.h>
|
|
||||||
#include <llvm/IR/IRBuilder.h>
|
|
||||||
|
|
||||||
#include <Interpreters/JIT/CHJIT.h>
|
#include <Interpreters/JIT/CHJIT.h>
|
||||||
|
#include <Interpreters/JIT/CompileDAG.h>
|
||||||
namespace ProfileEvents
|
#include <Interpreters/JIT/compileFunction.h>
|
||||||
{
|
#include <Interpreters/ActionsDAG.h>
|
||||||
extern const Event CompileFunction;
|
|
||||||
extern const Event CompileExpressionsMicroseconds;
|
|
||||||
extern const Event CompileExpressionsBytes;
|
|
||||||
}
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -40,52 +28,12 @@ namespace ErrorCodes
|
|||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
}
|
}
|
||||||
|
|
||||||
namespace
|
static CHJIT & getJITInstance()
|
||||||
{
|
{
|
||||||
struct ColumnData
|
static CHJIT jit;
|
||||||
{
|
return jit;
|
||||||
const char * data = nullptr;
|
|
||||||
const char * null = nullptr;
|
|
||||||
};
|
|
||||||
|
|
||||||
struct ColumnDataPlaceholder
|
|
||||||
{
|
|
||||||
llvm::Value * data_init; /// first row
|
|
||||||
llvm::Value * null_init;
|
|
||||||
llvm::PHINode * data; /// current row
|
|
||||||
llvm::PHINode * null;
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
|
|
||||||
static ColumnData getColumnData(const IColumn * column)
|
|
||||||
{
|
|
||||||
ColumnData result;
|
|
||||||
const bool is_const = isColumnConst(*column);
|
|
||||||
|
|
||||||
if (is_const)
|
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Input columns should not be constant");
|
|
||||||
|
|
||||||
if (const auto * nullable = typeid_cast<const ColumnNullable *>(column))
|
|
||||||
{
|
|
||||||
result.null = nullable->getNullMapColumn().getRawData().data;
|
|
||||||
column = &nullable->getNestedColumn();
|
|
||||||
}
|
|
||||||
|
|
||||||
result.data = column->getRawData().data;
|
|
||||||
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
static void applyFunction(IFunctionBase & function, Field & value)
|
|
||||||
{
|
|
||||||
const auto & type = function.getArgumentTypes().at(0);
|
|
||||||
ColumnsWithTypeAndName args{{type->createColumnConst(1, value), type, "x" }};
|
|
||||||
auto col = function.execute(args, function.getResultType(), 1);
|
|
||||||
col->get(0, value);
|
|
||||||
}
|
|
||||||
|
|
||||||
static CHJIT jit;
|
|
||||||
|
|
||||||
class LLVMExecutableFunction : public IExecutableFunctionImpl
|
class LLVMExecutableFunction : public IExecutableFunctionImpl
|
||||||
{
|
{
|
||||||
std::string name;
|
std::string name;
|
||||||
@ -94,10 +42,10 @@ public:
|
|||||||
explicit LLVMExecutableFunction(const std::string & name_)
|
explicit LLVMExecutableFunction(const std::string & name_)
|
||||||
: name(name_)
|
: name(name_)
|
||||||
{
|
{
|
||||||
function = jit.findCompiledFunction(name_);
|
function = getJITInstance().findCompiledFunction(name);
|
||||||
|
|
||||||
if (!function)
|
if (!function)
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find compiled function {}", name_);
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find compiled function {}", name);
|
||||||
}
|
}
|
||||||
|
|
||||||
String getName() const override { return name; }
|
String getName() const override { return name; }
|
||||||
@ -122,13 +70,14 @@ public:
|
|||||||
{
|
{
|
||||||
const auto * column = arguments[i].column.get();
|
const auto * column = arguments[i].column.get();
|
||||||
if (!column)
|
if (!column)
|
||||||
throw Exception("Column " + arguments[i].name + " is missing", ErrorCodes::LOGICAL_ERROR);
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Column {} is missing", arguments[i].name);
|
||||||
|
|
||||||
columns[i] = getColumnData(column);
|
columns[i] = getColumnData(column);
|
||||||
}
|
}
|
||||||
|
|
||||||
columns[arguments.size()] = getColumnData(result_column.get());
|
columns[arguments.size()] = getColumnData(result_column.get());
|
||||||
reinterpret_cast<void (*) (size_t, ColumnData *)>(function)(input_rows_count, columns.data());
|
auto * function_typed = reinterpret_cast<void (*) (size_t, ColumnData *)>(function);
|
||||||
|
function_typed(input_rows_count, columns.data());
|
||||||
|
|
||||||
#if defined(MEMORY_SANITIZER)
|
#if defined(MEMORY_SANITIZER)
|
||||||
/// Memory sanitizer don't know about stores from JIT-ed code.
|
/// Memory sanitizer don't know about stores from JIT-ed code.
|
||||||
@ -158,299 +107,184 @@ public:
|
|||||||
|
|
||||||
};
|
};
|
||||||
|
|
||||||
static void compileFunction(llvm::Module & module, const IFunctionBaseImpl & f)
|
class LLVMFunction : public IFunctionBaseImpl
|
||||||
{
|
{
|
||||||
ProfileEvents::increment(ProfileEvents::CompileFunction);
|
public:
|
||||||
|
|
||||||
const auto & arg_types = f.getArgumentTypes();
|
explicit LLVMFunction(const CompileDAG & dag_)
|
||||||
|
: name(dag_.dump())
|
||||||
llvm::IRBuilder<> b(module.getContext());
|
, dag(dag_)
|
||||||
auto * size_type = b.getIntNTy(sizeof(size_t) * 8);
|
|
||||||
auto * data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy(), size_type);
|
|
||||||
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { size_type, data_type->getPointerTo() }, /*isVarArg=*/false);
|
|
||||||
|
|
||||||
auto * func = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, f.getName(), module);
|
|
||||||
auto * args = func->args().begin();
|
|
||||||
llvm::Value * counter_arg = &*args++;
|
|
||||||
llvm::Value * columns_arg = &*args++;
|
|
||||||
|
|
||||||
auto * entry = llvm::BasicBlock::Create(b.getContext(), "entry", func);
|
|
||||||
b.SetInsertPoint(entry);
|
|
||||||
std::vector<ColumnDataPlaceholder> columns(arg_types.size() + 1);
|
|
||||||
for (size_t i = 0; i <= arg_types.size(); ++i)
|
|
||||||
{
|
{
|
||||||
const auto & type = i == arg_types.size() ? f.getResultType() : arg_types[i];
|
for (size_t i = 0; i < dag.getNodesCount(); ++i)
|
||||||
auto * data = b.CreateLoad(b.CreateConstInBoundsGEP1_32(data_type, columns_arg, i));
|
|
||||||
columns[i].data_init = b.CreatePointerCast(b.CreateExtractValue(data, {0}), toNativeType(b, removeNullable(type))->getPointerTo());
|
|
||||||
columns[i].null_init = type->isNullable() ? b.CreateExtractValue(data, {1}) : nullptr;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// assume nonzero initial value in `counter_arg`
|
|
||||||
auto * loop = llvm::BasicBlock::Create(b.getContext(), "loop", func);
|
|
||||||
b.CreateBr(loop);
|
|
||||||
b.SetInsertPoint(loop);
|
|
||||||
auto * counter_phi = b.CreatePHI(counter_arg->getType(), 2);
|
|
||||||
counter_phi->addIncoming(counter_arg, entry);
|
|
||||||
for (auto & col : columns)
|
|
||||||
{
|
|
||||||
col.data = b.CreatePHI(col.data_init->getType(), 2);
|
|
||||||
col.data->addIncoming(col.data_init, entry);
|
|
||||||
if (col.null_init)
|
|
||||||
{
|
{
|
||||||
col.null = b.CreatePHI(col.null_init->getType(), 2);
|
const auto & node = dag[i];
|
||||||
col.null->addIncoming(col.null_init, entry);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
Values arguments;
|
if (node.type == CompileDAG::CompileType::FUNCTION)
|
||||||
arguments.reserve(arg_types.size());
|
nested_functions.emplace_back(node.function);
|
||||||
|
else if (node.type == CompileDAG::CompileType::INPUT)
|
||||||
for (size_t i = 0; i < arg_types.size(); ++i) // NOLINT
|
argument_types.emplace_back(node.result_type);
|
||||||
{
|
|
||||||
auto & column = columns[i];
|
|
||||||
auto type = arg_types[i];
|
|
||||||
|
|
||||||
auto * value = b.CreateLoad(column.data);
|
|
||||||
if (!column.null)
|
|
||||||
{
|
|
||||||
arguments.emplace_back(value);
|
|
||||||
continue;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
auto * is_null = b.CreateICmpNE(b.CreateLoad(column.null), b.getInt8(0));
|
module_info = compileFunction(getJITInstance(), *this);
|
||||||
auto * nullable_unitilized = llvm::Constant::getNullValue(toNativeType(b, type));
|
|
||||||
auto * nullable_value = b.CreateInsertValue(b.CreateInsertValue(nullable_unitilized, value, {0}), is_null, {1});
|
|
||||||
arguments.emplace_back(nullable_value);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
auto * result = f.compile(b, std::move(arguments));
|
~LLVMFunction() override
|
||||||
if (columns.back().null)
|
|
||||||
{
|
{
|
||||||
b.CreateStore(b.CreateExtractValue(result, {0}), columns.back().data);
|
getJITInstance().deleteCompiledModule(module_info);
|
||||||
b.CreateStore(b.CreateSelect(b.CreateExtractValue(result, {1}), b.getInt8(1), b.getInt8(0)), columns.back().null);
|
}
|
||||||
|
|
||||||
|
size_t getCompiledSize() const { return module_info.size; }
|
||||||
|
|
||||||
|
bool isCompilable() const override { return true; }
|
||||||
|
|
||||||
|
llvm::Value * compile(llvm::IRBuilderBase & builder, Values values) const override
|
||||||
|
{
|
||||||
|
return dag.compile(builder, values);
|
||||||
|
}
|
||||||
|
|
||||||
|
String getName() const override { return name; }
|
||||||
|
|
||||||
|
const DataTypes & getArgumentTypes() const override { return argument_types; }
|
||||||
|
|
||||||
|
const DataTypePtr & getResultType() const override { return dag.back().result_type; }
|
||||||
|
|
||||||
|
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName &) const override
|
||||||
|
{
|
||||||
|
return std::make_unique<LLVMExecutableFunction>(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
bool isDeterministic() const override
|
||||||
|
{
|
||||||
|
for (const auto & f : nested_functions)
|
||||||
|
if (!f->isDeterministic())
|
||||||
|
return false;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
bool isDeterministicInScopeOfQuery() const override
|
||||||
|
{
|
||||||
|
for (const auto & f : nested_functions)
|
||||||
|
if (!f->isDeterministicInScopeOfQuery())
|
||||||
|
return false;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
bool isSuitableForConstantFolding() const override
|
||||||
|
{
|
||||||
|
for (const auto & f : nested_functions)
|
||||||
|
if (!f->isSuitableForConstantFolding())
|
||||||
|
return false;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
bool isInjective(const ColumnsWithTypeAndName & sample_block) const override
|
||||||
|
{
|
||||||
|
for (const auto & f : nested_functions)
|
||||||
|
if (!f->isInjective(sample_block))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
bool hasInformationAboutMonotonicity() const override
|
||||||
|
{
|
||||||
|
for (const auto & f : nested_functions)
|
||||||
|
if (!f->hasInformationAboutMonotonicity())
|
||||||
|
return false;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
|
||||||
|
{
|
||||||
|
const IDataType * type_ptr = &type;
|
||||||
|
Field left_mut = left;
|
||||||
|
Field right_mut = right;
|
||||||
|
Monotonicity result(true, true, true);
|
||||||
|
/// monotonicity is only defined for unary functions, so the chain must describe a sequence of nested calls
|
||||||
|
for (size_t i = 0; i < nested_functions.size(); ++i)
|
||||||
|
{
|
||||||
|
Monotonicity m = nested_functions[i]->getMonotonicityForRange(*type_ptr, left_mut, right_mut);
|
||||||
|
if (!m.is_monotonic)
|
||||||
|
return m;
|
||||||
|
result.is_positive ^= !m.is_positive;
|
||||||
|
result.is_always_monotonic &= m.is_always_monotonic;
|
||||||
|
if (i + 1 < nested_functions.size())
|
||||||
|
{
|
||||||
|
if (left_mut != Field())
|
||||||
|
applyFunction(*nested_functions[i], left_mut);
|
||||||
|
if (right_mut != Field())
|
||||||
|
applyFunction(*nested_functions[i], right_mut);
|
||||||
|
if (!m.is_positive)
|
||||||
|
std::swap(left_mut, right_mut);
|
||||||
|
type_ptr = nested_functions[i]->getResultType().get();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
static void applyFunction(IFunctionBase & function, Field & value)
|
||||||
|
{
|
||||||
|
const auto & type = function.getArgumentTypes().at(0);
|
||||||
|
ColumnsWithTypeAndName args{{type->createColumnConst(1, value), type, "x" }};
|
||||||
|
auto col = function.execute(args, function.getResultType(), 1);
|
||||||
|
col->get(0, value);
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
std::string name;
|
||||||
|
CompileDAG dag;
|
||||||
|
DataTypes argument_types;
|
||||||
|
std::vector<FunctionBasePtr> nested_functions;
|
||||||
|
CHJIT::CompiledModuleInfo module_info;
|
||||||
|
};
|
||||||
|
|
||||||
|
static FunctionBasePtr compile(
|
||||||
|
const CompileDAG & dag,
|
||||||
|
size_t min_count_to_compile_expression)
|
||||||
|
{
|
||||||
|
static std::unordered_map<UInt128, UInt64, UInt128Hash> counter;
|
||||||
|
static std::mutex mutex;
|
||||||
|
|
||||||
|
auto hash_key = dag.hash();
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
if (counter[hash_key]++ < min_count_to_compile_expression)
|
||||||
|
return nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
FunctionBasePtr fn;
|
||||||
|
|
||||||
|
if (auto * compilation_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
|
||||||
|
{
|
||||||
|
auto [compiled_function, was_inserted] = compilation_cache->getOrSet(hash_key, [&dag] ()
|
||||||
|
{
|
||||||
|
auto llvm_function = std::make_unique<LLVMFunction>(dag);
|
||||||
|
size_t compiled_size = llvm_function->getCompiledSize();
|
||||||
|
|
||||||
|
FunctionBasePtr llvm_function_wrapper = std::make_shared<FunctionBaseAdaptor>(std::move(llvm_function));
|
||||||
|
CompiledFunction compiled_function
|
||||||
|
{
|
||||||
|
.function = llvm_function_wrapper,
|
||||||
|
.compiled_size = compiled_size
|
||||||
|
};
|
||||||
|
|
||||||
|
return std::make_shared<CompiledFunction>(compiled_function);
|
||||||
|
});
|
||||||
|
|
||||||
|
fn = compiled_function->function;
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
b.CreateStore(result, columns.back().data);
|
fn = std::make_shared<FunctionBaseAdaptor>(std::make_unique<LLVMFunction>(dag));
|
||||||
}
|
}
|
||||||
|
|
||||||
auto * cur_block = b.GetInsertBlock();
|
return fn;
|
||||||
for (auto & col : columns)
|
|
||||||
{
|
|
||||||
col.data->addIncoming(b.CreateConstInBoundsGEP1_32(nullptr, col.data, 1), cur_block);
|
|
||||||
if (col.null)
|
|
||||||
col.null->addIncoming(b.CreateConstInBoundsGEP1_32(nullptr, col.null, 1), cur_block);
|
|
||||||
}
|
|
||||||
|
|
||||||
counter_phi->addIncoming(b.CreateSub(counter_phi, llvm::ConstantInt::get(size_type, 1)), cur_block);
|
|
||||||
|
|
||||||
auto * end = llvm::BasicBlock::Create(b.getContext(), "end", func);
|
|
||||||
b.CreateCondBr(b.CreateICmpNE(counter_phi, llvm::ConstantInt::get(size_type, 1)), loop, end);
|
|
||||||
b.SetInsertPoint(end);
|
|
||||||
b.CreateRetVoid();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
static llvm::Constant * getNativeValue(llvm::Type * type, const IColumn & column, size_t i)
|
|
||||||
{
|
|
||||||
/// TODO: Change name this is just for constants
|
|
||||||
if (!type || column.size() <= i)
|
|
||||||
return nullptr;
|
|
||||||
if (const auto * constant = typeid_cast<const ColumnConst *>(&column))
|
|
||||||
return getNativeValue(type, constant->getDataColumn(), 0);
|
|
||||||
if (const auto * nullable = typeid_cast<const ColumnNullable *>(&column))
|
|
||||||
{
|
|
||||||
auto * value = getNativeValue(type->getContainedType(0), nullable->getNestedColumn(), i);
|
|
||||||
auto * is_null = llvm::ConstantInt::get(type->getContainedType(1), nullable->isNullAt(i));
|
|
||||||
return value ? llvm::ConstantStruct::get(static_cast<llvm::StructType *>(type), value, is_null) : nullptr;
|
|
||||||
}
|
|
||||||
if (type->isFloatTy())
|
|
||||||
return llvm::ConstantFP::get(type, assert_cast<const ColumnVector<Float32> &>(column).getElement(i));
|
|
||||||
if (type->isDoubleTy())
|
|
||||||
return llvm::ConstantFP::get(type, assert_cast<const ColumnVector<Float64> &>(column).getElement(i));
|
|
||||||
if (type->isIntegerTy())
|
|
||||||
return llvm::ConstantInt::get(type, column.getUInt(i));
|
|
||||||
/// TODO: if (type->isVectorTy())
|
|
||||||
return nullptr;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Same as IFunctionBase::compile, but also for constants and input columns.
|
|
||||||
|
|
||||||
static CompilableExpression subexpression(ColumnPtr c, DataTypePtr type)
|
|
||||||
{
|
|
||||||
return [=](llvm::IRBuilderBase & b, const Values &)
|
|
||||||
{
|
|
||||||
auto * native_value = getNativeValue(toNativeType(b, type), *c, 0);
|
|
||||||
return native_value;
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
static CompilableExpression subexpression(size_t i)
|
|
||||||
{
|
|
||||||
return [=](llvm::IRBuilderBase &, const Values & inputs)
|
|
||||||
{
|
|
||||||
auto * column = inputs[i];
|
|
||||||
return column;
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
static CompilableExpression subexpression(const IFunctionBase & f, std::vector<CompilableExpression> args)
|
|
||||||
{
|
|
||||||
return [&, args = std::move(args)](llvm::IRBuilderBase & builder, const Values & inputs)
|
|
||||||
{
|
|
||||||
Values input;
|
|
||||||
for (const auto & arg : args)
|
|
||||||
input.push_back(arg(builder, inputs));
|
|
||||||
|
|
||||||
auto * result = f.compile(builder, input);
|
|
||||||
if (result->getType() != toNativeType(builder, f.getResultType()))
|
|
||||||
throw Exception("Function " + f.getName() + " generated an llvm::Value of invalid type", ErrorCodes::LOGICAL_ERROR);
|
|
||||||
|
|
||||||
return result;
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
LLVMFunction::LLVMFunction(const CompileDAG & dag)
|
|
||||||
: name(dag.dump())
|
|
||||||
{
|
|
||||||
std::vector<CompilableExpression> expressions;
|
|
||||||
expressions.reserve(dag.size());
|
|
||||||
|
|
||||||
jit.compileModule([&](llvm::Module & module)
|
|
||||||
{
|
|
||||||
auto & context = module.getContext();
|
|
||||||
llvm::IRBuilder<> builder(context);
|
|
||||||
|
|
||||||
for (const auto & node : dag)
|
|
||||||
{
|
|
||||||
switch (node.type)
|
|
||||||
{
|
|
||||||
case CompileNode::NodeType::CONSTANT:
|
|
||||||
{
|
|
||||||
const auto * col = typeid_cast<const ColumnConst *>(node.column.get());
|
|
||||||
|
|
||||||
/// TODO: implement `getNativeValue` for all types & replace the check with `c.column && toNativeType(...)`
|
|
||||||
if (!getNativeValue(toNativeType(builder, node.result_type), col->getDataColumn(), 0))
|
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
|
||||||
"Cannot compile constant of type {} = {}",
|
|
||||||
node.result_type->getName(),
|
|
||||||
applyVisitor(FieldVisitorToString(), col->getDataColumn()[0]));
|
|
||||||
|
|
||||||
expressions.emplace_back(subexpression(col->getDataColumnPtr(), node.result_type));
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case CompileNode::NodeType::FUNCTION:
|
|
||||||
{
|
|
||||||
std::vector<CompilableExpression> args;
|
|
||||||
args.reserve(node.arguments.size());
|
|
||||||
|
|
||||||
for (auto arg : node.arguments)
|
|
||||||
{
|
|
||||||
// std::cerr << "CompileNode::Function emplace expression " << arg << std::endl;
|
|
||||||
args.emplace_back(expressions[arg]);
|
|
||||||
}
|
|
||||||
|
|
||||||
originals.push_back(node.function);
|
|
||||||
expressions.emplace_back(subexpression(*node.function, std::move(args)));
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case CompileNode::NodeType::INPUT:
|
|
||||||
{
|
|
||||||
expressions.emplace_back(subexpression(arg_types.size()));
|
|
||||||
arg_types.push_back(node.result_type);
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
expression = std::move(expressions.back());
|
|
||||||
|
|
||||||
compileFunction(module, *this);
|
|
||||||
|
|
||||||
// for (auto & func : module)
|
|
||||||
// {
|
|
||||||
// std::cerr << "Func name " << std::string(func.getName()) << std::endl;
|
|
||||||
// }
|
|
||||||
|
|
||||||
// module.print(llvm::errs(), nullptr);
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
llvm::Value * LLVMFunction::compile(llvm::IRBuilderBase & builder, Values values) const
|
|
||||||
{
|
|
||||||
return expression(builder, values);
|
|
||||||
}
|
|
||||||
|
|
||||||
ExecutableFunctionImplPtr LLVMFunction::prepare(const ColumnsWithTypeAndName &) const { return std::make_unique<LLVMExecutableFunction>(name); }
|
|
||||||
|
|
||||||
bool LLVMFunction::isDeterministic() const
|
|
||||||
{
|
|
||||||
for (const auto & f : originals)
|
|
||||||
if (!f->isDeterministic())
|
|
||||||
return false;
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool LLVMFunction::isDeterministicInScopeOfQuery() const
|
|
||||||
{
|
|
||||||
for (const auto & f : originals)
|
|
||||||
if (!f->isDeterministicInScopeOfQuery())
|
|
||||||
return false;
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool LLVMFunction::isSuitableForConstantFolding() const
|
|
||||||
{
|
|
||||||
for (const auto & f : originals)
|
|
||||||
if (!f->isSuitableForConstantFolding())
|
|
||||||
return false;
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool LLVMFunction::isInjective(const ColumnsWithTypeAndName & sample_block) const
|
|
||||||
{
|
|
||||||
for (const auto & f : originals)
|
|
||||||
if (!f->isInjective(sample_block))
|
|
||||||
return false;
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool LLVMFunction::hasInformationAboutMonotonicity() const
|
|
||||||
{
|
|
||||||
for (const auto & f : originals)
|
|
||||||
if (!f->hasInformationAboutMonotonicity())
|
|
||||||
return false;
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
LLVMFunction::Monotonicity LLVMFunction::getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const
|
|
||||||
{
|
|
||||||
const IDataType * type_ptr = &type;
|
|
||||||
Field left_mut = left;
|
|
||||||
Field right_mut = right;
|
|
||||||
Monotonicity result(true, true, true);
|
|
||||||
/// monotonicity is only defined for unary functions, so the chain must describe a sequence of nested calls
|
|
||||||
for (size_t i = 0; i < originals.size(); ++i)
|
|
||||||
{
|
|
||||||
Monotonicity m = originals[i]->getMonotonicityForRange(*type_ptr, left_mut, right_mut);
|
|
||||||
if (!m.is_monotonic)
|
|
||||||
return m;
|
|
||||||
result.is_positive ^= !m.is_positive;
|
|
||||||
result.is_always_monotonic &= m.is_always_monotonic;
|
|
||||||
if (i + 1 < originals.size())
|
|
||||||
{
|
|
||||||
if (left_mut != Field())
|
|
||||||
applyFunction(*originals[i], left_mut);
|
|
||||||
if (right_mut != Field())
|
|
||||||
applyFunction(*originals[i], right_mut);
|
|
||||||
if (!m.is_positive)
|
|
||||||
std::swap(left_mut, right_mut);
|
|
||||||
type_ptr = originals[i]->getResultType().get();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
static bool isCompilable(const IFunctionBase & function)
|
static bool isCompilable(const IFunctionBase & function)
|
||||||
{
|
{
|
||||||
if (!canBeNativeType(*function.getResultType()))
|
if (!canBeNativeType(*function.getResultType()))
|
||||||
@ -475,12 +309,12 @@ static bool isCompilableFunction(const ActionsDAG::Node & node)
|
|||||||
return node.type == ActionsDAG::ActionType::FUNCTION && isCompilable(*node.function_base);
|
return node.type == ActionsDAG::ActionType::FUNCTION && isCompilable(*node.function_base);
|
||||||
}
|
}
|
||||||
|
|
||||||
static LLVMFunction::CompileDAG getCompilableDAG(
|
static CompileDAG getCompilableDAG(
|
||||||
const ActionsDAG::Node * root,
|
const ActionsDAG::Node * root,
|
||||||
ActionsDAG::NodeRawConstPtrs & children,
|
ActionsDAG::NodeRawConstPtrs & children,
|
||||||
const std::unordered_set<const ActionsDAG::Node *> & used_in_result)
|
const std::unordered_set<const ActionsDAG::Node *> & used_in_result)
|
||||||
{
|
{
|
||||||
LLVMFunction::CompileDAG dag;
|
CompileDAG dag;
|
||||||
|
|
||||||
std::unordered_map<const ActionsDAG::Node *, size_t> positions;
|
std::unordered_map<const ActionsDAG::Node *, size_t> positions;
|
||||||
struct Frame
|
struct Frame
|
||||||
@ -514,25 +348,30 @@ static LLVMFunction::CompileDAG getCompilableDAG(
|
|||||||
|
|
||||||
if (!is_compilable_function || frame.next_child_to_visit == frame.node->children.size())
|
if (!is_compilable_function || frame.next_child_to_visit == frame.node->children.size())
|
||||||
{
|
{
|
||||||
LLVMFunction::CompileNode node;
|
CompileDAG::Node node;
|
||||||
node.function = frame.node->function_base;
|
node.function = frame.node->function_base;
|
||||||
node.result_type = frame.node->result_type;
|
node.result_type = frame.node->result_type;
|
||||||
node.type = is_const ? LLVMFunction::CompileNode::NodeType::CONSTANT
|
|
||||||
: (is_compilable_function ? LLVMFunction::CompileNode::NodeType::FUNCTION
|
|
||||||
: LLVMFunction::CompileNode::NodeType::INPUT);
|
|
||||||
|
|
||||||
if (node.type == LLVMFunction::CompileNode::NodeType::FUNCTION)
|
if (is_compilable_function)
|
||||||
|
{
|
||||||
|
node.type = CompileDAG::CompileType::FUNCTION;
|
||||||
for (const auto * child : frame.node->children)
|
for (const auto * child : frame.node->children)
|
||||||
node.arguments.push_back(positions[child]);
|
node.arguments.push_back(positions[child]);
|
||||||
|
}
|
||||||
if (node.type == LLVMFunction::CompileNode::NodeType::CONSTANT)
|
else if (is_const)
|
||||||
|
{
|
||||||
|
node.type = CompileDAG::CompileType::CONSTANT;
|
||||||
node.column = frame.node->column;
|
node.column = frame.node->column;
|
||||||
|
}
|
||||||
if (node.type == LLVMFunction::CompileNode::NodeType::INPUT)
|
else
|
||||||
|
{
|
||||||
|
node.type = CompileDAG::CompileType::INPUT;
|
||||||
children.emplace_back(frame.node);
|
children.emplace_back(frame.node);
|
||||||
|
|
||||||
positions[frame.node] = dag.size();
|
}
|
||||||
dag.push_back(std::move(node));
|
|
||||||
|
positions[frame.node] = dag.getNodesCount();
|
||||||
|
dag.addNode(std::move(node));
|
||||||
stack.pop();
|
stack.pop();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -540,127 +379,8 @@ static LLVMFunction::CompileDAG getCompilableDAG(
|
|||||||
return dag;
|
return dag;
|
||||||
}
|
}
|
||||||
|
|
||||||
std::string LLVMFunction::CompileDAG::dump() const
|
|
||||||
{
|
|
||||||
WriteBufferFromOwnString out;
|
|
||||||
bool first = true;
|
|
||||||
for (const auto & node : *this)
|
|
||||||
{
|
|
||||||
if (!first)
|
|
||||||
out << " ; ";
|
|
||||||
first = false;
|
|
||||||
|
|
||||||
switch (node.type)
|
|
||||||
{
|
|
||||||
case CompileNode::NodeType::CONSTANT:
|
|
||||||
{
|
|
||||||
const auto * column = typeid_cast<const ColumnConst *>(node.column.get());
|
|
||||||
const auto & data = column->getDataColumn();
|
|
||||||
out << node.result_type->getName() << " = " << applyVisitor(FieldVisitorToString(), data[0]);
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case CompileNode::NodeType::FUNCTION:
|
|
||||||
{
|
|
||||||
out << node.result_type->getName() << " = ";
|
|
||||||
out << node.function->getName() << "(";
|
|
||||||
|
|
||||||
for (size_t i = 0; i < node.arguments.size(); ++i)
|
|
||||||
{
|
|
||||||
if (i)
|
|
||||||
out << ", ";
|
|
||||||
|
|
||||||
out << node.arguments[i];
|
|
||||||
}
|
|
||||||
|
|
||||||
out << ")";
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case CompileNode::NodeType::INPUT:
|
|
||||||
{
|
|
||||||
out << node.result_type->getName();
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return out.str();
|
|
||||||
}
|
|
||||||
|
|
||||||
UInt128 LLVMFunction::CompileDAG::hash() const
|
|
||||||
{
|
|
||||||
SipHash hash;
|
|
||||||
for (const auto & node : *this)
|
|
||||||
{
|
|
||||||
hash.update(node.type);
|
|
||||||
hash.update(node.result_type->getName());
|
|
||||||
|
|
||||||
switch (node.type)
|
|
||||||
{
|
|
||||||
case CompileNode::NodeType::CONSTANT:
|
|
||||||
{
|
|
||||||
typeid_cast<const ColumnConst *>(node.column.get())->getDataColumn().updateHashWithValue(0, hash);
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case CompileNode::NodeType::FUNCTION:
|
|
||||||
{
|
|
||||||
hash.update(node.function->getName());
|
|
||||||
for (size_t arg : node.arguments)
|
|
||||||
hash.update(arg);
|
|
||||||
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case CompileNode::NodeType::INPUT:
|
|
||||||
{
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
UInt128 result;
|
|
||||||
hash.get128(result.low, result.high);
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
static FunctionBasePtr compile(
|
|
||||||
const LLVMFunction::CompileDAG & dag,
|
|
||||||
size_t min_count_to_compile_expression)
|
|
||||||
{
|
|
||||||
static std::unordered_map<UInt128, UInt32, UInt128Hash> counter;
|
|
||||||
static std::mutex mutex;
|
|
||||||
|
|
||||||
auto hash_key = dag.hash();
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
if (counter[hash_key]++ < min_count_to_compile_expression)
|
|
||||||
return nullptr;
|
|
||||||
}
|
|
||||||
|
|
||||||
FunctionBasePtr fn;
|
|
||||||
if (auto * compilation_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
|
|
||||||
{
|
|
||||||
std::tie(fn, std::ignore) = compilation_cache->getOrSet(hash_key, [&dag] ()
|
|
||||||
{
|
|
||||||
Stopwatch watch;
|
|
||||||
FunctionBasePtr result_fn;
|
|
||||||
result_fn = std::make_shared<FunctionBaseAdaptor>(std::make_unique<LLVMFunction>(dag));
|
|
||||||
ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds());
|
|
||||||
return result_fn;
|
|
||||||
});
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
Stopwatch watch;
|
|
||||||
fn = std::make_shared<FunctionBaseAdaptor>(std::make_unique<LLVMFunction>(dag));
|
|
||||||
ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds());
|
|
||||||
}
|
|
||||||
|
|
||||||
return fn;
|
|
||||||
}
|
|
||||||
|
|
||||||
void ActionsDAG::compileFunctions(size_t min_count_to_compile_expression)
|
void ActionsDAG::compileFunctions(size_t min_count_to_compile_expression)
|
||||||
{
|
{
|
||||||
/// TODO: Rewrite
|
|
||||||
|
|
||||||
struct Data
|
struct Data
|
||||||
{
|
{
|
||||||
bool is_compilable = false;
|
bool is_compilable = false;
|
||||||
@ -743,18 +463,7 @@ void ActionsDAG::compileFunctions(size_t min_count_to_compile_expression)
|
|||||||
NodeRawConstPtrs new_children;
|
NodeRawConstPtrs new_children;
|
||||||
auto dag = getCompilableDAG(frame.node, new_children, used_in_result);
|
auto dag = getCompilableDAG(frame.node, new_children, used_in_result);
|
||||||
|
|
||||||
bool all_constants = true;
|
if (dag.getInputNodesCount() > 0)
|
||||||
|
|
||||||
for (const auto & compiled_node : dag)
|
|
||||||
{
|
|
||||||
if (compiled_node.type == LLVMFunction::CompileNode::NodeType::INPUT)
|
|
||||||
{
|
|
||||||
all_constants = false;
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!all_constants)
|
|
||||||
{
|
{
|
||||||
if (auto fn = compile(dag, min_count_to_compile_expression))
|
if (auto fn = compile(dag, min_count_to_compile_expression))
|
||||||
{
|
{
|
||||||
|
@ -5,93 +5,34 @@
|
|||||||
#endif
|
#endif
|
||||||
|
|
||||||
#if USE_EMBEDDED_COMPILER
|
#if USE_EMBEDDED_COMPILER
|
||||||
# include <set>
|
|
||||||
# include <Functions/IFunctionImpl.h>
|
# include <Functions/IFunctionImpl.h>
|
||||||
# include <Interpreters/Context.h>
|
|
||||||
# include <Interpreters/ExpressionActions.h>
|
|
||||||
# include <Common/LRUCache.h>
|
# include <Common/LRUCache.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
using CompilableExpression = std::function<llvm::Value * (llvm::IRBuilderBase &, const Values &)>;
|
struct CompiledFunction
|
||||||
|
|
||||||
class LLVMFunction : public IFunctionBaseImpl
|
|
||||||
{
|
{
|
||||||
std::string name;
|
FunctionBasePtr function;
|
||||||
DataTypes arg_types;
|
size_t compiled_size;
|
||||||
|
};
|
||||||
|
|
||||||
std::vector<FunctionBasePtr> originals;
|
struct CompiledFunctionWeightFunction
|
||||||
CompilableExpression expression;
|
{
|
||||||
|
size_t operator()(const CompiledFunction & compiled_function) const
|
||||||
public:
|
|
||||||
|
|
||||||
/// LLVMFunction is a compiled part of ActionsDAG.
|
|
||||||
/// We store this part as independent DAG with minial required information to compile it.
|
|
||||||
struct CompileNode
|
|
||||||
{
|
{
|
||||||
enum class NodeType
|
return compiled_function.compiled_size;
|
||||||
{
|
}
|
||||||
INPUT = 0,
|
|
||||||
CONSTANT = 1,
|
|
||||||
FUNCTION = 2,
|
|
||||||
};
|
|
||||||
|
|
||||||
NodeType type;
|
|
||||||
DataTypePtr result_type;
|
|
||||||
|
|
||||||
/// For CONSTANT
|
|
||||||
ColumnPtr column;
|
|
||||||
|
|
||||||
/// For FUNCTION
|
|
||||||
FunctionBasePtr function;
|
|
||||||
std::vector<size_t> arguments;
|
|
||||||
};
|
|
||||||
|
|
||||||
/// DAG is represented as list of nodes stored in in-order traverse order.
|
|
||||||
/// Expression (a + 1) + (b + 1) will be represented like chain: a, 1, a + 1, b, b + 1, (a + 1) + (b + 1).
|
|
||||||
struct CompileDAG : public std::vector<CompileNode>
|
|
||||||
{
|
|
||||||
std::string dump() const;
|
|
||||||
UInt128 hash() const;
|
|
||||||
};
|
|
||||||
|
|
||||||
explicit LLVMFunction(const CompileDAG & dag);
|
|
||||||
|
|
||||||
bool isCompilable() const override { return true; }
|
|
||||||
|
|
||||||
llvm::Value * compile(llvm::IRBuilderBase & builder, Values values) const override;
|
|
||||||
|
|
||||||
String getName() const override { return name; }
|
|
||||||
|
|
||||||
const DataTypes & getArgumentTypes() const override { return arg_types; }
|
|
||||||
|
|
||||||
const DataTypePtr & getResultType() const override { return originals.back()->getResultType(); }
|
|
||||||
|
|
||||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName &) const override;
|
|
||||||
|
|
||||||
bool isDeterministic() const override;
|
|
||||||
|
|
||||||
bool isDeterministicInScopeOfQuery() const override;
|
|
||||||
|
|
||||||
bool isSuitableForConstantFolding() const override;
|
|
||||||
|
|
||||||
bool isInjective(const ColumnsWithTypeAndName & sample_block) const override;
|
|
||||||
|
|
||||||
bool hasInformationAboutMonotonicity() const override;
|
|
||||||
|
|
||||||
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override;
|
|
||||||
|
|
||||||
};
|
};
|
||||||
|
|
||||||
/** This child of LRUCache breaks one of it's invariants: total weight may be changed after insertion.
|
/** This child of LRUCache breaks one of it's invariants: total weight may be changed after insertion.
|
||||||
* We have to do so, because we don't known real memory consumption of generated LLVM code for every function.
|
* We have to do so, because we don't known real memory consumption of generated LLVM code for every function.
|
||||||
*/
|
*/
|
||||||
class CompiledExpressionCache : public LRUCache<UInt128, IFunctionBase, UInt128Hash>
|
class CompiledExpressionCache : public LRUCache<UInt128, CompiledFunction, UInt128Hash, CompiledFunctionWeightFunction>
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
using Base = LRUCache<UInt128, IFunctionBase, UInt128Hash>;
|
using Base = LRUCache<UInt128, CompiledFunction, UInt128Hash, CompiledFunctionWeightFunction>;
|
||||||
using Base::Base;
|
using Base::Base;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -13,6 +13,7 @@
|
|||||||
#include <llvm/IR/LegacyPassManager.h>
|
#include <llvm/IR/LegacyPassManager.h>
|
||||||
#include <llvm/ExecutionEngine/JITSymbol.h>
|
#include <llvm/ExecutionEngine/JITSymbol.h>
|
||||||
#include <llvm/ExecutionEngine/SectionMemoryManager.h>
|
#include <llvm/ExecutionEngine/SectionMemoryManager.h>
|
||||||
|
#include <llvm/ExecutionEngine/JITEventListener.h>
|
||||||
#include <llvm/MC/SubtargetFeature.h>
|
#include <llvm/MC/SubtargetFeature.h>
|
||||||
#include <llvm/Support/DynamicLibrary.h>
|
#include <llvm/Support/DynamicLibrary.h>
|
||||||
#include <llvm/Support/Host.h>
|
#include <llvm/Support/Host.h>
|
||||||
@ -140,6 +141,30 @@ private:
|
|||||||
std::unordered_map<std::string, void *> symbol_name_to_symbol_address;
|
std::unordered_map<std::string, void *> symbol_name_to_symbol_address;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
// class JITEventListener
|
||||||
|
// {
|
||||||
|
// public:
|
||||||
|
// JITEventListener()
|
||||||
|
// : gdb_listener(llvm::JITEventListener::createGDBRegistrationListener())
|
||||||
|
// {}
|
||||||
|
|
||||||
|
// void notifyObjectLoaded(
|
||||||
|
// llvm::JITEventListener::ObjectKey object_key,
|
||||||
|
// const llvm::object::ObjectFile & object_file,
|
||||||
|
// const llvm::RuntimeDyld::LoadedObjectInfo & loaded_object_Info)
|
||||||
|
// {
|
||||||
|
// gdb_listener->notifyObjectLoaded(object_key, object_file, loaded_object_Info);
|
||||||
|
// }
|
||||||
|
|
||||||
|
// void notifyFreeingObject(llvm::JITEventListener::ObjectKey object_key)
|
||||||
|
// {
|
||||||
|
// gdb_listener->notifyFreeingObject(object_key);
|
||||||
|
// }
|
||||||
|
|
||||||
|
// private:
|
||||||
|
// llvm::JITEventListener * gdb_listener = nullptr;
|
||||||
|
// };
|
||||||
|
|
||||||
CHJIT::CHJIT()
|
CHJIT::CHJIT()
|
||||||
: machine(getTargetMachine())
|
: machine(getTargetMachine())
|
||||||
, layout(machine->createDataLayout())
|
, layout(machine->createDataLayout())
|
||||||
@ -156,20 +181,21 @@ CHJIT::~CHJIT() = default;
|
|||||||
CHJIT::CompiledModuleInfo CHJIT::compileModule(std::function<void (llvm::Module &)> compile_function)
|
CHJIT::CompiledModuleInfo CHJIT::compileModule(std::function<void (llvm::Module &)> compile_function)
|
||||||
{
|
{
|
||||||
std::lock_guard<std::mutex> lock(jit_lock);
|
std::lock_guard<std::mutex> lock(jit_lock);
|
||||||
|
|
||||||
auto module = createModuleForCompilation();
|
auto module = createModuleForCompilation();
|
||||||
compile_function(*module);
|
compile_function(*module);
|
||||||
auto module_info = compileModule(std::move(module));
|
auto module_info = compileModule(std::move(module));
|
||||||
|
|
||||||
|
++current_module_key;
|
||||||
return module_info;
|
return module_info;
|
||||||
}
|
}
|
||||||
|
|
||||||
std::unique_ptr<llvm::Module> CHJIT::createModuleForCompilation()
|
std::unique_ptr<llvm::Module> CHJIT::createModuleForCompilation()
|
||||||
{
|
{
|
||||||
std::unique_ptr<llvm::Module> module = std::make_unique<llvm::Module>("jit " + std::to_string(current_module_key), context);
|
std::unique_ptr<llvm::Module> module = std::make_unique<llvm::Module>("jit" + std::to_string(current_module_key), context);
|
||||||
module->setDataLayout(layout);
|
module->setDataLayout(layout);
|
||||||
module->setTargetTriple(machine->getTargetTriple().getTriple());
|
module->setTargetTriple(machine->getTargetTriple().getTriple());
|
||||||
|
|
||||||
++current_module_key;
|
|
||||||
|
|
||||||
return module;
|
return module;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -194,9 +220,6 @@ CHJIT::CompiledModuleInfo CHJIT::compileModule(std::unique_ptr<llvm::Module> mod
|
|||||||
|
|
||||||
std::unique_ptr<llvm::RuntimeDyld::LoadedObjectInfo> linked_object = dynamic_linker.loadObject(*object.get());
|
std::unique_ptr<llvm::RuntimeDyld::LoadedObjectInfo> linked_object = dynamic_linker.loadObject(*object.get());
|
||||||
|
|
||||||
if (dynamic_linker.hasError())
|
|
||||||
throw Exception(ErrorCodes::CANNOT_COMPILE_CODE, "RuntimeDyld error {}", std::string(dynamic_linker.getErrorString()));
|
|
||||||
|
|
||||||
dynamic_linker.resolveRelocations();
|
dynamic_linker.resolveRelocations();
|
||||||
module_memory_manager->getManager().finalizeMemory();
|
module_memory_manager->getManager().finalizeMemory();
|
||||||
|
|
||||||
@ -217,14 +240,15 @@ CHJIT::CompiledModuleInfo CHJIT::compileModule(std::unique_ptr<llvm::Module> mod
|
|||||||
|
|
||||||
auto * jit_symbol_address = reinterpret_cast<void *>(jit_symbol.getAddress());
|
auto * jit_symbol_address = reinterpret_cast<void *>(jit_symbol.getAddress());
|
||||||
name_to_symbol[function_name] = jit_symbol_address;
|
name_to_symbol[function_name] = jit_symbol_address;
|
||||||
|
module_info.compiled_functions.emplace_back(std::move(function_name));
|
||||||
}
|
}
|
||||||
|
|
||||||
auto module_identifier = module->getModuleIdentifier();
|
auto module_identifier = module->getModuleIdentifier();
|
||||||
|
|
||||||
module_info.size = module_memory_manager->getAllocatedSize();
|
module_info.size = module_memory_manager->getAllocatedSize();
|
||||||
module_info.module_identifier = module_identifier;
|
module_info.module_identifier = current_module_key;
|
||||||
|
|
||||||
module_identifier_to_memory_manager[module_identifier] = std::move(module_memory_manager);
|
module_identifier_to_memory_manager[current_module_key] = std::move(module_memory_manager);
|
||||||
|
|
||||||
compiled_code_size.fetch_add(module_info.size, std::memory_order_relaxed);
|
compiled_code_size.fetch_add(module_info.size, std::memory_order_relaxed);
|
||||||
|
|
||||||
|
@ -20,7 +20,6 @@ class JITModuleMemoryManager;
|
|||||||
class JITSymbolResolver;
|
class JITSymbolResolver;
|
||||||
class JITCompiler;
|
class JITCompiler;
|
||||||
|
|
||||||
/// TODO: Add profile events
|
|
||||||
/// TODO: Add documentation
|
/// TODO: Add documentation
|
||||||
class CHJIT
|
class CHJIT
|
||||||
{
|
{
|
||||||
@ -32,7 +31,7 @@ public:
|
|||||||
struct CompiledModuleInfo
|
struct CompiledModuleInfo
|
||||||
{
|
{
|
||||||
size_t size;
|
size_t size;
|
||||||
std::string module_identifier;
|
uint64_t module_identifier;
|
||||||
std::vector<std::string> compiled_functions;
|
std::vector<std::string> compiled_functions;
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -65,10 +64,11 @@ private:
|
|||||||
std::unique_ptr<JITSymbolResolver> symbol_resolver;
|
std::unique_ptr<JITSymbolResolver> symbol_resolver;
|
||||||
|
|
||||||
std::unordered_map<std::string, void *> name_to_symbol;
|
std::unordered_map<std::string, void *> name_to_symbol;
|
||||||
std::unordered_map<std::string, std::unique_ptr<JITModuleMemoryManager>> module_identifier_to_memory_manager;
|
std::unordered_map<uint64_t, std::unique_ptr<JITModuleMemoryManager>> module_identifier_to_memory_manager;
|
||||||
size_t current_module_key = 0;
|
uint64_t current_module_key = 0;
|
||||||
std::atomic<size_t> compiled_code_size = 0;
|
std::atomic<size_t> compiled_code_size = 0;
|
||||||
mutable std::mutex jit_lock;
|
mutable std::mutex jit_lock;
|
||||||
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
155
src/Interpreters/JIT/CompileDAG.cpp
Normal file
155
src/Interpreters/JIT/CompileDAG.cpp
Normal file
@ -0,0 +1,155 @@
|
|||||||
|
#include "CompileDAG.h"
|
||||||
|
|
||||||
|
#if USE_EMBEDDED_COMPILER
|
||||||
|
|
||||||
|
#include <llvm/IR/BasicBlock.h>
|
||||||
|
#include <llvm/IR/Function.h>
|
||||||
|
#include <llvm/IR/IRBuilder.h>
|
||||||
|
|
||||||
|
#include <Common/SipHash.h>
|
||||||
|
#include <Common/FieldVisitors.h>
|
||||||
|
#include <Columns/ColumnConst.h>
|
||||||
|
#include <DataTypes/Native.h>
|
||||||
|
#include <IO/WriteBufferFromString.h>
|
||||||
|
#include <IO/Operators.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
llvm::Value * CompileDAG::compile(llvm::IRBuilderBase & builder, Values 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());
|
||||||
|
|
||||||
|
size_t input_nodes_values_index = 0;
|
||||||
|
size_t compiled_values_index = 0;
|
||||||
|
|
||||||
|
size_t dag_size = nodes.size();
|
||||||
|
|
||||||
|
for (size_t i = 0; i < dag_size; ++i)
|
||||||
|
{
|
||||||
|
const auto & node = nodes[i];
|
||||||
|
|
||||||
|
switch (node.type)
|
||||||
|
{
|
||||||
|
case CompileType::CONSTANT:
|
||||||
|
{
|
||||||
|
compiled_values[compiled_values_index] = getColumnNativeValue(b, node.result_type, *node.column, 0);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case CompileType::FUNCTION:
|
||||||
|
{
|
||||||
|
Values temporary_values;
|
||||||
|
temporary_values.reserve(node.arguments.size());
|
||||||
|
|
||||||
|
for (auto argument_index : node.arguments)
|
||||||
|
temporary_values.emplace_back(compiled_values[argument_index]);
|
||||||
|
|
||||||
|
compiled_values[compiled_values_index] = node.function->compile(builder, temporary_values);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case CompileType::INPUT:
|
||||||
|
{
|
||||||
|
compiled_values[compiled_values_index] = input_nodes_values[input_nodes_values_index];
|
||||||
|
++input_nodes_values_index;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
++compiled_values_index;
|
||||||
|
}
|
||||||
|
|
||||||
|
return compiled_values.back();
|
||||||
|
}
|
||||||
|
|
||||||
|
std::string CompileDAG::dump() const
|
||||||
|
{
|
||||||
|
std::vector<std::string> dumped_values;
|
||||||
|
dumped_values.resize(nodes.size());
|
||||||
|
|
||||||
|
size_t input_index = 0;
|
||||||
|
size_t dag_size = nodes.size();
|
||||||
|
for (size_t i = 0; i < dag_size; ++i)
|
||||||
|
{
|
||||||
|
const auto & node = nodes[i];
|
||||||
|
|
||||||
|
switch (node.type)
|
||||||
|
{
|
||||||
|
case CompileType::CONSTANT:
|
||||||
|
{
|
||||||
|
const auto * column = typeid_cast<const ColumnConst *>(node.column.get());
|
||||||
|
const auto & data = column->getDataColumn();
|
||||||
|
|
||||||
|
dumped_values[i] = applyVisitor(FieldVisitorToString(), data[0]) + " : " + node.result_type->getName();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case CompileType::FUNCTION:
|
||||||
|
{
|
||||||
|
std::string function_dump = node.function->getName();
|
||||||
|
function_dump += '(';
|
||||||
|
|
||||||
|
for (auto argument_index : node.arguments)
|
||||||
|
function_dump += dumped_values[argument_index] += ", ";
|
||||||
|
|
||||||
|
function_dump.pop_back();
|
||||||
|
function_dump.pop_back();
|
||||||
|
|
||||||
|
function_dump += ')';
|
||||||
|
|
||||||
|
dumped_values[i] = function_dump;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case CompileType::INPUT:
|
||||||
|
{
|
||||||
|
dumped_values[i] = node.result_type->getName();
|
||||||
|
++input_index;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return dumped_values.back();
|
||||||
|
}
|
||||||
|
|
||||||
|
UInt128 CompileDAG::hash() const
|
||||||
|
{
|
||||||
|
SipHash hash;
|
||||||
|
for (const auto & node : nodes)
|
||||||
|
{
|
||||||
|
hash.update(node.type);
|
||||||
|
hash.update(node.result_type->getName());
|
||||||
|
|
||||||
|
switch (node.type)
|
||||||
|
{
|
||||||
|
case CompileType::CONSTANT:
|
||||||
|
{
|
||||||
|
assert_cast<const ColumnConst *>(node.column.get())->getDataColumn().updateHashWithValue(0, hash);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case CompileType::FUNCTION:
|
||||||
|
{
|
||||||
|
hash.update(node.function->getName());
|
||||||
|
for (size_t arg : node.arguments)
|
||||||
|
hash.update(arg);
|
||||||
|
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case CompileType::INPUT:
|
||||||
|
{
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
UInt128 result;
|
||||||
|
hash.get128(result.low, result.high);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
#endif
|
86
src/Interpreters/JIT/CompileDAG.h
Normal file
86
src/Interpreters/JIT/CompileDAG.h
Normal file
@ -0,0 +1,86 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
|
#if USE_EMBEDDED_COMPILER
|
||||||
|
|
||||||
|
#include <vector>
|
||||||
|
|
||||||
|
#include <Core/Types.h>
|
||||||
|
#include <Common/UInt128.h>
|
||||||
|
|
||||||
|
#include <Columns/IColumn.h>
|
||||||
|
#include <DataTypes/IDataType.h>
|
||||||
|
#include <Functions/IFunctionImpl.h>
|
||||||
|
|
||||||
|
namespace llvm
|
||||||
|
{
|
||||||
|
class Value;
|
||||||
|
class IRBuilderBase;
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// DAG is represented as list of nodes stored in in-order traverse order.
|
||||||
|
/// Expression (a + 1) + (b + 1) will be represented like chain: a, 1, a + 1, b, b + 1, (a + 1) + (b + 1).
|
||||||
|
/// TODO: Consider to rename in CompileStack
|
||||||
|
class CompileDAG
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
|
||||||
|
enum class CompileType
|
||||||
|
{
|
||||||
|
INPUT = 0,
|
||||||
|
CONSTANT = 1,
|
||||||
|
FUNCTION = 2,
|
||||||
|
};
|
||||||
|
|
||||||
|
struct Node
|
||||||
|
{
|
||||||
|
CompileType type;
|
||||||
|
DataTypePtr result_type;
|
||||||
|
|
||||||
|
/// For CONSTANT
|
||||||
|
ColumnPtr column;
|
||||||
|
|
||||||
|
/// For FUNCTION
|
||||||
|
FunctionBasePtr function;
|
||||||
|
std::vector<size_t> arguments;
|
||||||
|
};
|
||||||
|
|
||||||
|
llvm::Value * compile(llvm::IRBuilderBase & builder, Values input_nodes_values) const;
|
||||||
|
|
||||||
|
std::string dump() const;
|
||||||
|
|
||||||
|
UInt128 hash() const;
|
||||||
|
|
||||||
|
void addNode(Node node)
|
||||||
|
{
|
||||||
|
input_nodes_count += (node.type == CompileType::INPUT);
|
||||||
|
nodes.emplace_back(std::move(node));
|
||||||
|
}
|
||||||
|
|
||||||
|
inline size_t getNodesCount() const { return nodes.size(); }
|
||||||
|
inline size_t getInputNodesCount() const { return input_nodes_count; }
|
||||||
|
|
||||||
|
inline Node & operator[](size_t index) { return nodes[index]; }
|
||||||
|
inline const Node & operator[](size_t index) const { return nodes[index]; }
|
||||||
|
|
||||||
|
|
||||||
|
inline Node & front() { return nodes.front(); }
|
||||||
|
inline const Node & front() const { return nodes.front(); }
|
||||||
|
|
||||||
|
inline Node & back() { return nodes.back(); }
|
||||||
|
inline const Node & back() const { return nodes.back(); }
|
||||||
|
|
||||||
|
private:
|
||||||
|
std::vector<Node> nodes;
|
||||||
|
size_t input_nodes_count = 0;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
#endif
|
160
src/Interpreters/JIT/compileFunction.cpp
Normal file
160
src/Interpreters/JIT/compileFunction.cpp
Normal file
@ -0,0 +1,160 @@
|
|||||||
|
#include "compileFunction.h"
|
||||||
|
|
||||||
|
#include <llvm/IR/BasicBlock.h>
|
||||||
|
#include <llvm/IR/Function.h>
|
||||||
|
#include <llvm/IR/IRBuilder.h>
|
||||||
|
|
||||||
|
#include <Common/Stopwatch.h>
|
||||||
|
#include <Common/ProfileEvents.h>
|
||||||
|
#include <DataTypes/Native.h>
|
||||||
|
#include <Interpreters/JIT/CHJIT.h>
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
struct ColumnDataPlaceholder
|
||||||
|
{
|
||||||
|
llvm::Value * data_init; /// first row
|
||||||
|
llvm::Value * null_init;
|
||||||
|
llvm::PHINode * data; /// current row
|
||||||
|
llvm::PHINode * null;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace ProfileEvents
|
||||||
|
{
|
||||||
|
extern const Event CompileFunction;
|
||||||
|
extern const Event CompileExpressionsMicroseconds;
|
||||||
|
extern const Event CompileExpressionsBytes;
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
ColumnData getColumnData(const IColumn * column)
|
||||||
|
{
|
||||||
|
ColumnData result;
|
||||||
|
const bool is_const = isColumnConst(*column);
|
||||||
|
|
||||||
|
if (is_const)
|
||||||
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Input columns should not be constant");
|
||||||
|
|
||||||
|
if (const auto * nullable = typeid_cast<const ColumnNullable *>(column))
|
||||||
|
{
|
||||||
|
result.null = nullable->getNullMapColumn().getRawData().data;
|
||||||
|
column = & nullable->getNestedColumn();
|
||||||
|
}
|
||||||
|
|
||||||
|
result.data = column->getRawData().data;
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
static void compileFunction(llvm::Module & module, const IFunctionBaseImpl & f)
|
||||||
|
{
|
||||||
|
ProfileEvents::increment(ProfileEvents::CompileFunction);
|
||||||
|
|
||||||
|
const auto & arg_types = f.getArgumentTypes();
|
||||||
|
|
||||||
|
llvm::IRBuilder<> b(module.getContext());
|
||||||
|
auto * size_type = b.getIntNTy(sizeof(size_t) * 8);
|
||||||
|
auto * data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy());
|
||||||
|
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { size_type, data_type->getPointerTo() }, /*isVarArg=*/false);
|
||||||
|
|
||||||
|
auto * func = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, f.getName(), module);
|
||||||
|
auto * args = func->args().begin();
|
||||||
|
llvm::Value * counter_arg = &*args++;
|
||||||
|
llvm::Value * columns_arg = &*args++;
|
||||||
|
|
||||||
|
auto * entry = llvm::BasicBlock::Create(b.getContext(), "entry", func);
|
||||||
|
b.SetInsertPoint(entry);
|
||||||
|
std::vector<ColumnDataPlaceholder> columns(arg_types.size() + 1);
|
||||||
|
for (size_t i = 0; i <= arg_types.size(); ++i)
|
||||||
|
{
|
||||||
|
const auto & type = i == arg_types.size() ? f.getResultType() : arg_types[i];
|
||||||
|
auto * data = b.CreateLoad(b.CreateConstInBoundsGEP1_32(data_type, columns_arg, i));
|
||||||
|
columns[i].data_init = b.CreatePointerCast(b.CreateExtractValue(data, {0}), toNativeType(b, removeNullable(type))->getPointerTo());
|
||||||
|
columns[i].null_init = type->isNullable() ? b.CreateExtractValue(data, {1}) : nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// assume nonzero initial value in `counter_arg`
|
||||||
|
auto * loop = llvm::BasicBlock::Create(b.getContext(), "loop", func);
|
||||||
|
b.CreateBr(loop);
|
||||||
|
b.SetInsertPoint(loop);
|
||||||
|
auto * counter_phi = b.CreatePHI(counter_arg->getType(), 2);
|
||||||
|
counter_phi->addIncoming(counter_arg, entry);
|
||||||
|
for (auto & col : columns)
|
||||||
|
{
|
||||||
|
col.data = b.CreatePHI(col.data_init->getType(), 2);
|
||||||
|
col.data->addIncoming(col.data_init, entry);
|
||||||
|
if (col.null_init)
|
||||||
|
{
|
||||||
|
col.null = b.CreatePHI(col.null_init->getType(), 2);
|
||||||
|
col.null->addIncoming(col.null_init, entry);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Values arguments;
|
||||||
|
arguments.reserve(arg_types.size());
|
||||||
|
|
||||||
|
for (size_t i = 0; i < arg_types.size(); ++i) // NOLINT
|
||||||
|
{
|
||||||
|
auto & column = columns[i];
|
||||||
|
auto type = arg_types[i];
|
||||||
|
|
||||||
|
auto * value = b.CreateLoad(column.data);
|
||||||
|
if (!column.null)
|
||||||
|
{
|
||||||
|
arguments.emplace_back(value);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
auto * is_null = b.CreateICmpNE(b.CreateLoad(column.null), b.getInt8(0));
|
||||||
|
auto * nullable_unitilized = llvm::Constant::getNullValue(toNativeType(b, type));
|
||||||
|
auto * nullable_value = b.CreateInsertValue(b.CreateInsertValue(nullable_unitilized, value, {0}), is_null, {1});
|
||||||
|
arguments.emplace_back(nullable_value);
|
||||||
|
}
|
||||||
|
|
||||||
|
auto * result = f.compile(b, std::move(arguments));
|
||||||
|
if (columns.back().null)
|
||||||
|
{
|
||||||
|
b.CreateStore(b.CreateExtractValue(result, {0}), columns.back().data);
|
||||||
|
b.CreateStore(b.CreateSelect(b.CreateExtractValue(result, {1}), b.getInt8(1), b.getInt8(0)), columns.back().null);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
b.CreateStore(result, columns.back().data);
|
||||||
|
}
|
||||||
|
|
||||||
|
auto * cur_block = b.GetInsertBlock();
|
||||||
|
for (auto & col : columns)
|
||||||
|
{
|
||||||
|
col.data->addIncoming(b.CreateConstInBoundsGEP1_32(nullptr, col.data, 1), cur_block);
|
||||||
|
if (col.null)
|
||||||
|
col.null->addIncoming(b.CreateConstInBoundsGEP1_32(nullptr, col.null, 1), cur_block);
|
||||||
|
}
|
||||||
|
|
||||||
|
counter_phi->addIncoming(b.CreateSub(counter_phi, llvm::ConstantInt::get(size_type, 1)), cur_block);
|
||||||
|
|
||||||
|
auto * end = llvm::BasicBlock::Create(b.getContext(), "end", func);
|
||||||
|
b.CreateCondBr(b.CreateICmpNE(counter_phi, llvm::ConstantInt::get(size_type, 1)), loop, end);
|
||||||
|
b.SetInsertPoint(end);
|
||||||
|
b.CreateRetVoid();
|
||||||
|
}
|
||||||
|
|
||||||
|
CHJIT::CompiledModuleInfo compileFunction(CHJIT & jit, const IFunctionBaseImpl & f)
|
||||||
|
{
|
||||||
|
Stopwatch watch;
|
||||||
|
|
||||||
|
auto compiled_module_info = jit.compileModule([&](llvm::Module & module)
|
||||||
|
{
|
||||||
|
compileFunction(module, f);
|
||||||
|
});
|
||||||
|
|
||||||
|
ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds());
|
||||||
|
ProfileEvents::increment(ProfileEvents::CompileExpressionsBytes, compiled_module_info.size);
|
||||||
|
ProfileEvents::increment(ProfileEvents::CompileFunction);
|
||||||
|
|
||||||
|
return compiled_module_info;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
25
src/Interpreters/JIT/compileFunction.h
Normal file
25
src/Interpreters/JIT/compileFunction.h
Normal file
@ -0,0 +1,25 @@
|
|||||||
|
#if !defined(ARCADIA_BUILD)
|
||||||
|
# include "config_core.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
|
#if USE_EMBEDDED_COMPILER
|
||||||
|
|
||||||
|
#include <Functions/IFunctionImpl.h>
|
||||||
|
#include <Interpreters/JIT/CHJIT.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct ColumnData
|
||||||
|
{
|
||||||
|
const char * data = nullptr;
|
||||||
|
const char * null = nullptr;
|
||||||
|
};
|
||||||
|
|
||||||
|
ColumnData getColumnData(const IColumn * column);
|
||||||
|
|
||||||
|
CHJIT::CompiledModuleInfo compileFunction(CHJIT & jit, const IFunctionBaseImpl & f);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
#endif
|
@ -26,7 +26,10 @@ int main(int argc, char **argv)
|
|||||||
auto * func_declaration_type = llvm::FunctionType::get(b.getVoidTy(), { }, /*isVarArg=*/false);
|
auto * func_declaration_type = llvm::FunctionType::get(b.getVoidTy(), { }, /*isVarArg=*/false);
|
||||||
auto * func_declaration = llvm::Function::Create(func_declaration_type, llvm::Function::ExternalLinkage, "test_function", module);
|
auto * func_declaration = llvm::Function::Create(func_declaration_type, llvm::Function::ExternalLinkage, "test_function", module);
|
||||||
|
|
||||||
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { b.getInt64Ty() }, /*isVarArg=*/false);
|
auto * value_type = b.getInt64Ty();
|
||||||
|
auto * pointer_type = value_type->getPointerTo();
|
||||||
|
|
||||||
|
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { pointer_type }, /*isVarArg=*/false);
|
||||||
auto * function = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, "test_name", module);
|
auto * function = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, "test_name", module);
|
||||||
auto * entry = llvm::BasicBlock::Create(context, "entry", function);
|
auto * entry = llvm::BasicBlock::Create(context, "entry", function);
|
||||||
|
|
||||||
@ -35,18 +38,19 @@ int main(int argc, char **argv)
|
|||||||
|
|
||||||
b.CreateCall(func_declaration);
|
b.CreateCall(func_declaration);
|
||||||
|
|
||||||
auto * value = b.CreateAdd(argument, argument);
|
auto * load_argument = b.CreateLoad(argument);
|
||||||
|
auto * value = b.CreateAdd(load_argument, load_argument);
|
||||||
b.CreateRet(value);
|
b.CreateRet(value);
|
||||||
});
|
});
|
||||||
|
|
||||||
std::cerr << "Compile module info " << compiled_module_info.module_identifier << " size " << compiled_module_info.size << std::endl;
|
|
||||||
for (const auto & compiled_function_name : compiled_module_info.compiled_functions)
|
for (const auto & compiled_function_name : compiled_module_info.compiled_functions)
|
||||||
{
|
{
|
||||||
std::cerr << compiled_function_name << std::endl;
|
std::cerr << compiled_function_name << std::endl;
|
||||||
}
|
}
|
||||||
|
|
||||||
auto * test_name_function = reinterpret_cast<int64_t (*)(int64_t)>(jit.findCompiledFunction("test_name"));
|
int64_t value = 5;
|
||||||
auto result = test_name_function(5);
|
auto * test_name_function = reinterpret_cast<int64_t (*)(int64_t *)>(jit.findCompiledFunction("test_name"));
|
||||||
|
auto result = test_name_function(&value);
|
||||||
std::cerr << "Result " << result << std::endl;
|
std::cerr << "Result " << result << std::endl;
|
||||||
|
|
||||||
return 0;
|
return 0;
|
||||||
|
Loading…
Reference in New Issue
Block a user