2018-04-25 15:19:22 +00:00
|
|
|
#include <Interpreters/ExpressionJIT.h>
|
|
|
|
|
|
|
|
#if USE_EMBEDDED_COMPILER
|
|
|
|
|
2018-05-06 10:42:35 +00:00
|
|
|
#include <optional>
|
|
|
|
|
2018-04-25 11:55:54 +00:00
|
|
|
#include <Columns/ColumnConst.h>
|
2018-04-27 21:30:38 +00:00
|
|
|
#include <Columns/ColumnNullable.h>
|
2018-04-24 19:42:06 +00:00
|
|
|
#include <Columns/ColumnVector.h>
|
2018-08-28 17:06:42 +00:00
|
|
|
#include <Common/LRUCache.h>
|
2018-04-24 19:42:06 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2019-08-21 02:28:04 +00:00
|
|
|
#include <Common/assert_cast.h>
|
2018-05-07 06:23:18 +00:00
|
|
|
#include <Common/ProfileEvents.h>
|
2018-09-03 10:14:05 +00:00
|
|
|
#include <Common/Stopwatch.h>
|
2018-04-24 19:42:06 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2018-04-27 21:30:38 +00:00
|
|
|
#include <DataTypes/Native.h>
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
|
2018-05-06 09:29:57 +00:00
|
|
|
#pragma GCC diagnostic push
|
2018-05-06 09:37:16 +00:00
|
|
|
#pragma GCC diagnostic ignored "-Wunused-parameter"
|
|
|
|
#pragma GCC diagnostic ignored "-Wnon-virtual-dtor"
|
2018-05-06 09:29:57 +00:00
|
|
|
|
2019-06-05 11:52:39 +00:00
|
|
|
#include <llvm/Analysis/TargetTransformInfo.h>
|
|
|
|
#include <llvm/Config/llvm-config.h>
|
|
|
|
#include <llvm/IR/BasicBlock.h>
|
|
|
|
#include <llvm/IR/DataLayout.h>
|
|
|
|
#include <llvm/IR/DerivedTypes.h>
|
|
|
|
#include <llvm/IR/Function.h>
|
|
|
|
#include <llvm/IR/IRBuilder.h>
|
|
|
|
#include <llvm/IR/LLVMContext.h>
|
|
|
|
#include <llvm/IR/Mangler.h>
|
|
|
|
#include <llvm/IR/Module.h>
|
|
|
|
#include <llvm/IR/Type.h>
|
|
|
|
#include <llvm/ExecutionEngine/ExecutionEngine.h>
|
|
|
|
#include <llvm/ExecutionEngine/JITSymbol.h>
|
|
|
|
#include <llvm/ExecutionEngine/SectionMemoryManager.h>
|
|
|
|
#include <llvm/ExecutionEngine/Orc/CompileUtils.h>
|
|
|
|
#include <llvm/ExecutionEngine/Orc/IRCompileLayer.h>
|
|
|
|
#include <llvm/ExecutionEngine/Orc/RTDyldObjectLinkingLayer.h>
|
|
|
|
#include <llvm/Target/TargetMachine.h>
|
|
|
|
#include <llvm/MC/SubtargetFeature.h>
|
|
|
|
#include <llvm/Support/DynamicLibrary.h>
|
|
|
|
#include <llvm/Support/Host.h>
|
|
|
|
#include <llvm/Support/TargetRegistry.h>
|
|
|
|
#include <llvm/Support/TargetSelect.h>
|
|
|
|
#include <llvm/Transforms/IPO/PassManagerBuilder.h>
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
|
2018-05-06 09:29:57 +00:00
|
|
|
#pragma GCC diagnostic pop
|
|
|
|
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
|
2018-05-07 06:23:18 +00:00
|
|
|
namespace ProfileEvents
|
|
|
|
{
|
|
|
|
extern const Event CompileFunction;
|
2018-09-03 10:14:05 +00:00
|
|
|
extern const Event CompileExpressionsMicroseconds;
|
2018-09-14 10:07:58 +00:00
|
|
|
extern const Event CompileExpressionsBytes;
|
2018-05-07 06:23:18 +00:00
|
|
|
}
|
|
|
|
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
Implement JIT compilation, without a loop for now.
It actually seems to work, so long as you only have one row that is. E.g.
> select something(cast(number + 6 as Float64), cast(number + 2 as Float64)) from system.numbers limit 1';
8
with this IR:
define void @"something(CAST(plus(number, 6), 'Float64'), CAST(plus(number, 2), 'Float64'))"(void**, i8*, double*) {
entry:
%3 = load void*, void** %0
%4 = bitcast void* %3 to double*
%5 = load double, double* %4
%6 = getelementptr void*, void** %0, i32 1
%7 = load void*, void** %6
%8 = bitcast void* %7 to double*
%9 = load double, double* %8
%10 = fadd double %5, %9
store double %10, double* %2
ret void
}
2018-04-23 23:52:54 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
2018-05-07 06:49:56 +00:00
|
|
|
extern const int CANNOT_COMPILE_CODE;
|
Implement JIT compilation, without a loop for now.
It actually seems to work, so long as you only have one row that is. E.g.
> select something(cast(number + 6 as Float64), cast(number + 2 as Float64)) from system.numbers limit 1';
8
with this IR:
define void @"something(CAST(plus(number, 6), 'Float64'), CAST(plus(number, 2), 'Float64'))"(void**, i8*, double*) {
entry:
%3 = load void*, void** %0
%4 = bitcast void* %3 to double*
%5 = load double, double* %4
%6 = getelementptr void*, void** %0, i32 1
%7 = load void*, void** %6
%8 = bitcast void* %7 to double*
%9 = load double, double* %8
%10 = fadd double %5, %9
store double %10, double* %2
ret void
}
2018-04-23 23:52:54 +00:00
|
|
|
}
|
|
|
|
|
2018-04-27 15:44:38 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
struct ColumnData
|
|
|
|
{
|
2018-04-27 21:30:38 +00:00
|
|
|
const char * data = nullptr;
|
|
|
|
const char * null = nullptr;
|
2019-04-22 16:07:09 +00:00
|
|
|
size_t stride = 0;
|
2018-04-27 15:44:38 +00:00
|
|
|
};
|
2018-04-27 21:30:38 +00:00
|
|
|
|
2018-04-29 18:03:58 +00:00
|
|
|
struct ColumnDataPlaceholder
|
2018-04-27 21:30:38 +00:00
|
|
|
{
|
2018-04-29 01:00:26 +00:00
|
|
|
llvm::Value * data_init; /// first row
|
2018-04-27 21:30:38 +00:00
|
|
|
llvm::Value * null_init;
|
|
|
|
llvm::Value * stride;
|
2018-04-29 01:00:26 +00:00
|
|
|
llvm::PHINode * data; /// current row
|
|
|
|
llvm::PHINode * null;
|
2018-04-27 21:30:38 +00:00
|
|
|
};
|
2018-04-27 15:44:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
static ColumnData getColumnData(const IColumn * column)
|
|
|
|
{
|
2018-04-27 21:30:38 +00:00
|
|
|
ColumnData result;
|
2019-06-27 19:28:52 +00:00
|
|
|
const bool is_const = isColumnConst(*column);
|
2018-04-27 21:30:38 +00:00
|
|
|
if (is_const)
|
|
|
|
column = &reinterpret_cast<const ColumnConst *>(column)->getDataColumn();
|
|
|
|
if (auto * nullable = typeid_cast<const ColumnNullable *>(column))
|
|
|
|
{
|
|
|
|
result.null = nullable->getNullMapColumn().getRawData().data;
|
|
|
|
column = &nullable->getNestedColumn();
|
|
|
|
}
|
|
|
|
result.data = column->getRawData().data;
|
|
|
|
result.stride = is_const ? 0 : column->sizeOfValueIfFixed();
|
|
|
|
return result;
|
2018-04-27 15:44:38 +00:00
|
|
|
}
|
|
|
|
|
2018-04-29 01:00:26 +00:00
|
|
|
static void applyFunction(IFunctionBase & function, Field & value)
|
|
|
|
{
|
|
|
|
const auto & type = function.getArgumentTypes().at(0);
|
|
|
|
Block block = {{ type->createColumnConst(1, value), type, "x" }, { nullptr, function.getReturnType(), "y" }};
|
2018-05-01 19:52:33 +00:00
|
|
|
function.execute(block, {0}, 1, 1);
|
2018-04-29 01:00:26 +00:00
|
|
|
block.safeGetByPosition(1).column->get(0, value);
|
|
|
|
}
|
|
|
|
|
2018-05-07 16:06:13 +00:00
|
|
|
static llvm::TargetMachine * getNativeMachine()
|
|
|
|
{
|
|
|
|
std::string error;
|
|
|
|
auto cpu = llvm::sys::getHostCPUName();
|
|
|
|
auto triple = llvm::sys::getProcessTriple();
|
|
|
|
auto target = llvm::TargetRegistry::lookupTarget(triple, error);
|
|
|
|
if (!target)
|
|
|
|
throw Exception("Could not initialize native target: " + error, ErrorCodes::CANNOT_COMPILE_CODE);
|
|
|
|
llvm::SubtargetFeatures features;
|
|
|
|
llvm::StringMap<bool> feature_map;
|
|
|
|
if (llvm::sys::getHostCPUFeatures(feature_map))
|
2018-08-24 05:25:00 +00:00
|
|
|
for (auto & f : feature_map)
|
2018-05-07 16:06:13 +00:00
|
|
|
features.AddFeature(f.first(), f.second);
|
|
|
|
llvm::TargetOptions options;
|
|
|
|
return target->createTargetMachine(
|
|
|
|
triple, cpu, features.getString(), options, llvm::None,
|
|
|
|
#if LLVM_VERSION_MAJOR >= 6
|
|
|
|
llvm::None, llvm::CodeGenOpt::Default, /*jit=*/true
|
|
|
|
#else
|
|
|
|
llvm::CodeModel::Default, llvm::CodeGenOpt::Default
|
|
|
|
#endif
|
|
|
|
);
|
|
|
|
}
|
|
|
|
|
2018-05-10 17:31:24 +00:00
|
|
|
#if LLVM_VERSION_MAJOR >= 7
|
|
|
|
auto wrapJITSymbolResolver(llvm::JITSymbolResolver & jsr)
|
|
|
|
{
|
2018-11-16 12:22:51 +00:00
|
|
|
#if USE_INTERNAL_LLVM_LIBRARY && LLVM_VERSION_PATCH == 0
|
|
|
|
// REMOVE AFTER contrib/llvm upgrade
|
2018-05-10 17:31:24 +00:00
|
|
|
auto flags = [&](llvm::orc::SymbolFlagsMap & flags, const llvm::orc::SymbolNameSet & symbols)
|
|
|
|
{
|
|
|
|
llvm::orc::SymbolNameSet missing;
|
|
|
|
for (const auto & symbol : symbols)
|
|
|
|
{
|
|
|
|
auto resolved = jsr.lookupFlags({*symbol});
|
|
|
|
if (resolved && resolved->size())
|
|
|
|
flags.emplace(symbol, resolved->begin()->second);
|
|
|
|
else
|
|
|
|
missing.emplace(symbol);
|
|
|
|
}
|
|
|
|
return missing;
|
|
|
|
};
|
2018-11-16 12:22:51 +00:00
|
|
|
#else
|
|
|
|
// Actually this should work for 7.0.0 but now we have OLDER 7.0.0svn in contrib
|
|
|
|
auto flags = [&](const llvm::orc::SymbolNameSet & symbols)
|
|
|
|
{
|
2019-01-10 11:25:59 +00:00
|
|
|
llvm::orc::SymbolFlagsMap flags_map;
|
2018-11-16 12:22:51 +00:00
|
|
|
for (const auto & symbol : symbols)
|
|
|
|
{
|
|
|
|
auto resolved = jsr.lookupFlags({*symbol});
|
|
|
|
if (resolved && resolved->size())
|
2019-01-10 11:25:59 +00:00
|
|
|
flags_map.emplace(symbol, resolved->begin()->second);
|
2018-11-16 12:22:51 +00:00
|
|
|
}
|
2019-01-10 11:25:59 +00:00
|
|
|
return flags_map;
|
2018-11-16 12:22:51 +00:00
|
|
|
};
|
|
|
|
#endif
|
|
|
|
|
2019-01-10 11:25:59 +00:00
|
|
|
auto symbols = [&](std::shared_ptr<llvm::orc::AsynchronousSymbolQuery> query, llvm::orc::SymbolNameSet symbols_set)
|
2018-05-10 17:31:24 +00:00
|
|
|
{
|
|
|
|
llvm::orc::SymbolNameSet missing;
|
2019-01-10 11:25:59 +00:00
|
|
|
for (const auto & symbol : symbols_set)
|
2018-05-10 17:31:24 +00:00
|
|
|
{
|
|
|
|
auto resolved = jsr.lookup({*symbol});
|
|
|
|
if (resolved && resolved->size())
|
|
|
|
query->resolve(symbol, resolved->begin()->second);
|
|
|
|
else
|
|
|
|
missing.emplace(symbol);
|
|
|
|
}
|
|
|
|
return missing;
|
|
|
|
};
|
|
|
|
return llvm::orc::createSymbolResolver(flags, symbols);
|
|
|
|
}
|
|
|
|
#endif
|
|
|
|
|
2019-01-11 21:16:49 +00:00
|
|
|
#if LLVM_VERSION_MAJOR >= 7
|
2019-01-12 07:19:18 +00:00
|
|
|
using ModulePtr = std::unique_ptr<llvm::Module>;
|
2019-01-11 21:16:49 +00:00
|
|
|
#else
|
|
|
|
using ModulePtr = std::shared_ptr<llvm::Module>;
|
2018-09-06 09:10:40 +00:00
|
|
|
#endif
|
2018-08-31 12:39:27 +00:00
|
|
|
|
2018-04-29 01:00:26 +00:00
|
|
|
struct LLVMContext
|
2018-04-24 19:42:06 +00:00
|
|
|
{
|
2019-01-11 21:16:49 +00:00
|
|
|
std::shared_ptr<llvm::LLVMContext> context;
|
2018-05-06 11:16:38 +00:00
|
|
|
#if LLVM_VERSION_MAJOR >= 7
|
|
|
|
llvm::orc::ExecutionSession execution_session;
|
|
|
|
#endif
|
2019-01-11 21:16:49 +00:00
|
|
|
ModulePtr module;
|
2018-04-29 01:00:26 +00:00
|
|
|
std::unique_ptr<llvm::TargetMachine> machine;
|
2018-05-10 14:00:29 +00:00
|
|
|
std::shared_ptr<llvm::SectionMemoryManager> memory_manager;
|
|
|
|
llvm::orc::RTDyldObjectLinkingLayer object_layer;
|
|
|
|
llvm::orc::IRCompileLayer<decltype(object_layer), llvm::orc::SimpleCompiler> compile_layer;
|
2018-04-29 01:00:26 +00:00
|
|
|
llvm::DataLayout layout;
|
|
|
|
llvm::IRBuilder<> builder;
|
2018-05-06 10:42:35 +00:00
|
|
|
std::unordered_map<std::string, void *> symbols;
|
2018-04-29 01:00:26 +00:00
|
|
|
|
|
|
|
LLVMContext()
|
2019-01-11 21:16:49 +00:00
|
|
|
: context(std::make_shared<llvm::LLVMContext>())
|
2018-05-06 11:16:38 +00:00
|
|
|
#if LLVM_VERSION_MAJOR >= 7
|
2019-01-11 21:16:49 +00:00
|
|
|
, module(std::make_unique<llvm::Module>("jit", *context))
|
2018-05-06 11:16:38 +00:00
|
|
|
#else
|
2019-01-11 21:16:49 +00:00
|
|
|
, module(std::make_shared<llvm::Module>("jit", *context))
|
2018-05-06 11:16:38 +00:00
|
|
|
#endif
|
2018-05-07 16:06:13 +00:00
|
|
|
, machine(getNativeMachine())
|
2018-09-06 09:10:40 +00:00
|
|
|
, memory_manager(std::make_shared<llvm::SectionMemoryManager>())
|
2018-05-06 11:16:38 +00:00
|
|
|
#if LLVM_VERSION_MAJOR >= 7
|
2018-05-10 14:00:29 +00:00
|
|
|
, object_layer(execution_session, [this](llvm::orc::VModuleKey)
|
2018-05-06 11:16:38 +00:00
|
|
|
{
|
2018-05-10 17:31:24 +00:00
|
|
|
return llvm::orc::RTDyldObjectLinkingLayer::Resources{memory_manager, wrapJITSymbolResolver(*memory_manager)};
|
2018-05-06 11:16:38 +00:00
|
|
|
})
|
|
|
|
#else
|
2018-05-10 14:00:29 +00:00
|
|
|
, object_layer([this]() { return memory_manager; })
|
2018-05-06 11:16:38 +00:00
|
|
|
#endif
|
2018-05-10 14:00:29 +00:00
|
|
|
, compile_layer(object_layer, llvm::orc::SimpleCompiler(*machine))
|
2018-04-29 01:00:26 +00:00
|
|
|
, layout(machine->createDataLayout())
|
2019-01-11 21:16:49 +00:00
|
|
|
, builder(*context)
|
2018-04-25 15:16:48 +00:00
|
|
|
{
|
2018-04-29 01:00:26 +00:00
|
|
|
module->setDataLayout(layout);
|
|
|
|
module->setTargetTriple(machine->getTargetTriple().getTriple());
|
|
|
|
}
|
|
|
|
|
2018-09-14 10:07:58 +00:00
|
|
|
/// returns used memory
|
2019-01-11 21:16:49 +00:00
|
|
|
void compileAllFunctionsToNativeCode()
|
2018-04-29 01:00:26 +00:00
|
|
|
{
|
|
|
|
if (!module->size())
|
2019-01-11 21:16:49 +00:00
|
|
|
return;
|
2019-01-10 11:25:59 +00:00
|
|
|
llvm::PassManagerBuilder pass_manager_builder;
|
2018-05-07 16:06:13 +00:00
|
|
|
llvm::legacy::PassManager mpm;
|
2018-04-29 01:00:26 +00:00
|
|
|
llvm::legacy::FunctionPassManager fpm(module.get());
|
2019-01-10 11:25:59 +00:00
|
|
|
pass_manager_builder.OptLevel = 3;
|
|
|
|
pass_manager_builder.SLPVectorize = true;
|
|
|
|
pass_manager_builder.LoopVectorize = true;
|
|
|
|
pass_manager_builder.RerollLoops = true;
|
|
|
|
pass_manager_builder.VerifyInput = true;
|
|
|
|
pass_manager_builder.VerifyOutput = true;
|
|
|
|
machine->adjustPassManager(pass_manager_builder);
|
2018-05-07 16:06:13 +00:00
|
|
|
fpm.add(llvm::createTargetTransformInfoWrapperPass(machine->getTargetIRAnalysis()));
|
|
|
|
mpm.add(llvm::createTargetTransformInfoWrapperPass(machine->getTargetIRAnalysis()));
|
2019-01-10 11:25:59 +00:00
|
|
|
pass_manager_builder.populateFunctionPassManager(fpm);
|
|
|
|
pass_manager_builder.populateModulePassManager(mpm);
|
2018-05-07 16:06:13 +00:00
|
|
|
fpm.doInitialization();
|
2018-04-29 01:00:26 +00:00
|
|
|
for (auto & function : *module)
|
|
|
|
fpm.run(function);
|
2018-05-07 16:06:13 +00:00
|
|
|
fpm.doFinalization();
|
|
|
|
mpm.run(*module);
|
2018-05-06 11:16:38 +00:00
|
|
|
|
2018-05-10 18:19:41 +00:00
|
|
|
std::vector<std::string> functions;
|
|
|
|
functions.reserve(module->size());
|
|
|
|
for (const auto & function : *module)
|
|
|
|
functions.emplace_back(function.getName());
|
|
|
|
|
2018-05-06 11:16:38 +00:00
|
|
|
#if LLVM_VERSION_MAJOR >= 7
|
|
|
|
llvm::orc::VModuleKey module_key = execution_session.allocateVModule();
|
2018-05-10 14:00:29 +00:00
|
|
|
if (compile_layer.addModule(module_key, std::move(module)))
|
2018-05-07 06:49:56 +00:00
|
|
|
throw Exception("Cannot add module to compile layer", ErrorCodes::CANNOT_COMPILE_CODE);
|
2018-05-06 11:16:38 +00:00
|
|
|
#else
|
2018-05-10 14:00:29 +00:00
|
|
|
if (!compile_layer.addModule(module, memory_manager))
|
2018-05-07 06:49:56 +00:00
|
|
|
throw Exception("Cannot add module to compile layer", ErrorCodes::CANNOT_COMPILE_CODE);
|
2018-05-06 11:16:38 +00:00
|
|
|
#endif
|
|
|
|
|
2018-05-10 18:19:41 +00:00
|
|
|
for (const auto & name : functions)
|
2018-05-07 06:23:18 +00:00
|
|
|
{
|
2018-05-10 14:00:29 +00:00
|
|
|
std::string mangled_name;
|
|
|
|
llvm::raw_string_ostream mangled_name_stream(mangled_name);
|
2018-05-10 18:19:41 +00:00
|
|
|
llvm::Mangler::getNameWithPrefix(mangled_name_stream, name, layout);
|
2018-05-10 14:00:29 +00:00
|
|
|
mangled_name_stream.flush();
|
|
|
|
auto symbol = compile_layer.findSymbol(mangled_name, false);
|
|
|
|
if (!symbol)
|
|
|
|
continue; /// external function (e.g. an intrinsic that calls into libc)
|
|
|
|
auto address = symbol.getAddress();
|
|
|
|
if (!address)
|
2018-05-10 18:19:41 +00:00
|
|
|
throw Exception("Function " + name + " failed to link", ErrorCodes::CANNOT_COMPILE_CODE);
|
|
|
|
symbols[name] = reinterpret_cast<void *>(*address);
|
2018-05-07 06:23:18 +00:00
|
|
|
}
|
2018-04-25 15:16:48 +00:00
|
|
|
}
|
2018-04-24 19:42:06 +00:00
|
|
|
};
|
|
|
|
|
2018-05-03 10:22:41 +00:00
|
|
|
class LLVMPreparedFunction : public PreparedFunctionImpl
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
{
|
2018-04-29 01:00:26 +00:00
|
|
|
std::string name;
|
2018-05-06 10:42:35 +00:00
|
|
|
void * function;
|
2018-04-29 01:00:26 +00:00
|
|
|
|
|
|
|
public:
|
2019-01-11 21:16:49 +00:00
|
|
|
LLVMPreparedFunction(const std::string & name_, const std::unordered_map<std::string, void *> & symbols)
|
|
|
|
: name(name_)
|
2018-10-11 20:31:41 +00:00
|
|
|
{
|
2019-01-11 21:16:49 +00:00
|
|
|
auto it = symbols.find(name);
|
|
|
|
if (symbols.end() == it)
|
2018-10-11 20:31:41 +00:00
|
|
|
throw Exception("Cannot find symbol " + name + " in LLVMContext", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
function = it->second;
|
|
|
|
}
|
2018-04-29 01:00:26 +00:00
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
2018-05-03 10:22:41 +00:00
|
|
|
bool useDefaultImplementationForNulls() const override { return false; }
|
|
|
|
|
|
|
|
bool useDefaultImplementationForConstants() const override { return true; }
|
|
|
|
|
|
|
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t block_size) override
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
{
|
2018-04-29 01:00:26 +00:00
|
|
|
auto col_res = block.getByPosition(result).type->createColumn()->cloneResized(block_size);
|
|
|
|
if (block_size)
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
{
|
2018-04-29 01:00:26 +00:00
|
|
|
std::vector<ColumnData> columns(arguments.size() + 1);
|
2018-05-06 10:42:35 +00:00
|
|
|
for (size_t i = 0; i < arguments.size(); ++i)
|
2018-04-29 01:00:26 +00:00
|
|
|
{
|
|
|
|
auto * column = block.getByPosition(arguments[i]).column.get();
|
|
|
|
if (!column)
|
2018-05-06 10:42:35 +00:00
|
|
|
throw Exception("Column " + block.getByPosition(arguments[i]).name + " is missing", ErrorCodes::LOGICAL_ERROR);
|
2018-04-29 01:00:26 +00:00
|
|
|
columns[i] = getColumnData(column);
|
|
|
|
}
|
|
|
|
columns[arguments.size()] = getColumnData(col_res.get());
|
|
|
|
reinterpret_cast<void (*) (size_t, ColumnData *)>(function)(block_size, columns.data());
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
}
|
2018-04-29 01:00:26 +00:00
|
|
|
block.getByPosition(result).column = std::move(col_res);
|
2018-09-03 10:14:05 +00:00
|
|
|
}
|
2018-04-29 01:00:26 +00:00
|
|
|
};
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
|
2019-01-11 21:16:49 +00:00
|
|
|
static void compileFunctionToLLVMByteCode(LLVMContext & context, const IFunctionBase & f)
|
2018-04-29 18:03:58 +00:00
|
|
|
{
|
2018-05-07 06:23:18 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::CompileFunction);
|
|
|
|
|
2018-04-29 18:03:58 +00:00
|
|
|
auto & arg_types = f.getArgumentTypes();
|
2019-01-11 21:16:49 +00:00
|
|
|
auto & b = context.builder;
|
2018-04-29 18:03:58 +00:00
|
|
|
auto * size_type = b.getIntNTy(sizeof(size_t) * 8);
|
|
|
|
auto * data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy(), size_type);
|
2018-05-03 10:22:41 +00:00
|
|
|
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { size_type, data_type->getPointerTo() }, /*isVarArg=*/false);
|
2019-01-11 21:16:49 +00:00
|
|
|
auto * func = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, f.getName(), context.module.get());
|
2018-04-29 18:03:58 +00:00
|
|
|
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);
|
2018-05-06 10:42:35 +00:00
|
|
|
for (size_t i = 0; i <= arg_types.size(); ++i)
|
2018-04-29 18:03:58 +00:00
|
|
|
{
|
|
|
|
auto & type = i == arg_types.size() ? f.getReturnType() : arg_types[i];
|
2018-05-03 10:22:41 +00:00
|
|
|
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;
|
|
|
|
columns[i].stride = b.CreateExtractValue(data, {2});
|
2018-04-29 18:03:58 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// 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);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
ValuePlaceholders arguments(arg_types.size());
|
2018-05-06 10:42:35 +00:00
|
|
|
for (size_t i = 0; i < arguments.size(); ++i)
|
2018-04-29 18:03:58 +00:00
|
|
|
{
|
|
|
|
arguments[i] = [&b, &col = columns[i], &type = arg_types[i]]() -> llvm::Value *
|
|
|
|
{
|
|
|
|
auto * value = b.CreateLoad(col.data);
|
|
|
|
if (!col.null)
|
|
|
|
return value;
|
|
|
|
auto * is_null = b.CreateICmpNE(b.CreateLoad(col.null), b.getInt8(0));
|
|
|
|
auto * nullable = llvm::Constant::getNullValue(toNativeType(b, type));
|
|
|
|
return b.CreateInsertValue(b.CreateInsertValue(nullable, value, {0}), is_null, {1});
|
|
|
|
};
|
|
|
|
}
|
|
|
|
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)
|
|
|
|
{
|
2018-05-07 13:03:26 +00:00
|
|
|
/// stride is either 0 or size of native type; output column is never constant; neither is at least one input
|
|
|
|
auto * is_const = &col == &columns.back() || columns.size() <= 2 ? b.getFalse() : b.CreateICmpEQ(col.stride, llvm::ConstantInt::get(size_type, 0));
|
2018-05-03 10:22:41 +00:00
|
|
|
col.data->addIncoming(b.CreateSelect(is_const, col.data, b.CreateConstInBoundsGEP1_32(nullptr, col.data, 1)), cur_block);
|
2018-04-29 18:03:58 +00:00
|
|
|
if (col.null)
|
2018-05-03 10:22:41 +00:00
|
|
|
col.null->addIncoming(b.CreateSelect(is_const, col.null, b.CreateConstInBoundsGEP1_32(nullptr, col.null, 1)), cur_block);
|
2018-04-29 18:03:58 +00:00
|
|
|
}
|
|
|
|
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)
|
|
|
|
{
|
2018-09-10 15:48:42 +00:00
|
|
|
if (!type || column.size() <= i)
|
2018-04-29 18:03:58 +00:00
|
|
|
return nullptr;
|
|
|
|
if (auto * constant = typeid_cast<const ColumnConst *>(&column))
|
|
|
|
return getNativeValue(type, constant->getDataColumn(), 0);
|
|
|
|
if (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())
|
2019-08-21 02:28:04 +00:00
|
|
|
return llvm::ConstantFP::get(type, assert_cast<const ColumnVector<Float32> &>(column).getElement(i));
|
2018-04-29 18:03:58 +00:00
|
|
|
if (type->isDoubleTy())
|
2019-08-21 02:28:04 +00:00
|
|
|
return llvm::ConstantFP::get(type, assert_cast<const ColumnVector<Float64> &>(column).getElement(i));
|
2018-04-29 18:03:58 +00:00
|
|
|
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.
|
|
|
|
using CompilableExpression = std::function<llvm::Value * (llvm::IRBuilderBase &, const ValuePlaceholders &)>;
|
|
|
|
|
|
|
|
static CompilableExpression subexpression(ColumnPtr c, DataTypePtr type)
|
|
|
|
{
|
|
|
|
return [=](llvm::IRBuilderBase & b, const ValuePlaceholders &) { return getNativeValue(toNativeType(b, type), *c, 0); };
|
|
|
|
}
|
|
|
|
|
|
|
|
static CompilableExpression subexpression(size_t i)
|
|
|
|
{
|
|
|
|
return [=](llvm::IRBuilderBase &, const ValuePlaceholders & inputs) { return inputs[i](); };
|
|
|
|
}
|
|
|
|
|
|
|
|
static CompilableExpression subexpression(const IFunctionBase & f, std::vector<CompilableExpression> args)
|
|
|
|
{
|
|
|
|
return [&, args = std::move(args)](llvm::IRBuilderBase & builder, const ValuePlaceholders & inputs)
|
|
|
|
{
|
|
|
|
ValuePlaceholders input;
|
|
|
|
for (const auto & arg : args)
|
|
|
|
input.push_back([&]() { return arg(builder, inputs); });
|
|
|
|
auto * result = f.compile(builder, input);
|
|
|
|
if (result->getType() != toNativeType(builder, f.getReturnType()))
|
2018-05-06 10:42:35 +00:00
|
|
|
throw Exception("Function " + f.getName() + " generated an llvm::Value of invalid type", ErrorCodes::LOGICAL_ERROR);
|
2018-04-29 18:03:58 +00:00
|
|
|
return result;
|
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2019-01-11 21:16:49 +00:00
|
|
|
struct LLVMModuleState
|
2018-08-30 16:31:20 +00:00
|
|
|
{
|
2019-01-11 21:16:49 +00:00
|
|
|
std::unordered_map<std::string, void *> symbols;
|
|
|
|
std::shared_ptr<llvm::LLVMContext> major_context;
|
|
|
|
std::shared_ptr<llvm::SectionMemoryManager> memory_manager;
|
|
|
|
};
|
|
|
|
|
|
|
|
LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, const Block & sample_block)
|
|
|
|
: name(actions.back().result_name)
|
|
|
|
, module_state(std::make_unique<LLVMModuleState>())
|
|
|
|
{
|
|
|
|
LLVMContext context;
|
2018-08-30 16:31:20 +00:00
|
|
|
for (const auto & c : sample_block)
|
|
|
|
/// TODO: implement `getNativeValue` for all types & replace the check with `c.column && toNativeType(...)`
|
2019-01-11 21:16:49 +00:00
|
|
|
if (c.column && getNativeValue(toNativeType(context.builder, c.type), *c.column, 0))
|
2018-08-30 16:31:20 +00:00
|
|
|
subexpressions[c.name] = subexpression(c.column, c.type);
|
|
|
|
for (const auto & action : actions)
|
2018-04-27 21:30:38 +00:00
|
|
|
{
|
2018-08-30 16:31:20 +00:00
|
|
|
const auto & names = action.argument_names;
|
2018-09-21 14:15:21 +00:00
|
|
|
const auto & types = action.function_base->getArgumentTypes();
|
2018-08-30 16:31:20 +00:00
|
|
|
std::vector<CompilableExpression> args;
|
|
|
|
for (size_t i = 0; i < names.size(); ++i)
|
2018-04-29 01:00:26 +00:00
|
|
|
{
|
2018-08-30 16:31:20 +00:00
|
|
|
auto inserted = subexpressions.emplace(names[i], subexpression(arg_names.size()));
|
|
|
|
if (inserted.second)
|
2018-04-29 01:00:26 +00:00
|
|
|
{
|
2018-08-30 16:31:20 +00:00
|
|
|
arg_names.push_back(names[i]);
|
|
|
|
arg_types.push_back(types[i]);
|
2018-04-29 01:00:26 +00:00
|
|
|
}
|
2018-08-30 16:31:20 +00:00
|
|
|
args.push_back(inserted.first->second);
|
2018-04-29 01:00:26 +00:00
|
|
|
}
|
2018-09-21 14:15:21 +00:00
|
|
|
subexpressions[action.result_name] = subexpression(*action.function_base, std::move(args));
|
|
|
|
originals.push_back(action.function_base);
|
2018-04-29 18:03:58 +00:00
|
|
|
}
|
2018-09-14 10:07:58 +00:00
|
|
|
compileFunctionToLLVMByteCode(context, *this);
|
2019-01-11 21:16:49 +00:00
|
|
|
context.compileAllFunctionsToNativeCode();
|
|
|
|
|
|
|
|
module_state->symbols = context.symbols;
|
|
|
|
module_state->major_context = context.context;
|
|
|
|
module_state->memory_manager = context.memory_manager;
|
2018-08-30 16:31:20 +00:00
|
|
|
}
|
2018-04-29 01:00:26 +00:00
|
|
|
|
2018-10-11 20:31:41 +00:00
|
|
|
llvm::Value * LLVMFunction::compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const
|
|
|
|
{
|
|
|
|
auto it = subexpressions.find(name);
|
|
|
|
if (subexpressions.end() == it)
|
|
|
|
throw Exception("Cannot find subexpression " + name + " in LLVMFunction", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
return it->second(builder, values);
|
|
|
|
}
|
|
|
|
|
2019-01-11 21:16:49 +00:00
|
|
|
PreparedFunctionPtr LLVMFunction::prepare(const Block &, const ColumnNumbers &, size_t) const { return std::make_shared<LLVMPreparedFunction>(name, module_state->symbols); }
|
2018-04-29 01:00:26 +00:00
|
|
|
|
2018-08-30 16:31:20 +00:00
|
|
|
bool LLVMFunction::isDeterministic() const
|
|
|
|
{
|
|
|
|
for (const auto & f : originals)
|
|
|
|
if (!f->isDeterministic())
|
|
|
|
return false;
|
|
|
|
return true;
|
|
|
|
}
|
2018-04-29 01:00:26 +00:00
|
|
|
|
2018-08-30 16:31:20 +00:00
|
|
|
bool LLVMFunction::isDeterministicInScopeOfQuery() const
|
|
|
|
{
|
|
|
|
for (const auto & f : originals)
|
|
|
|
if (!f->isDeterministicInScopeOfQuery())
|
|
|
|
return false;
|
|
|
|
return true;
|
|
|
|
}
|
2018-04-24 13:21:42 +00:00
|
|
|
|
2018-08-30 16:31:20 +00:00
|
|
|
bool LLVMFunction::isSuitableForConstantFolding() const
|
|
|
|
{
|
|
|
|
for (const auto & f : originals)
|
|
|
|
if (!f->isSuitableForConstantFolding())
|
|
|
|
return false;
|
|
|
|
return true;
|
|
|
|
}
|
Implement JIT compilation, without a loop for now.
It actually seems to work, so long as you only have one row that is. E.g.
> select something(cast(number + 6 as Float64), cast(number + 2 as Float64)) from system.numbers limit 1';
8
with this IR:
define void @"something(CAST(plus(number, 6), 'Float64'), CAST(plus(number, 2), 'Float64'))"(void**, i8*, double*) {
entry:
%3 = load void*, void** %0
%4 = bitcast void* %3 to double*
%5 = load double, double* %4
%6 = getelementptr void*, void** %0, i32 1
%7 = load void*, void** %6
%8 = bitcast void* %7 to double*
%9 = load double, double* %8
%10 = fadd double %5, %9
store double %10, double* %2
ret void
}
2018-04-23 23:52:54 +00:00
|
|
|
|
2018-08-30 16:31:20 +00:00
|
|
|
bool LLVMFunction::isInjective(const Block & sample_block)
|
|
|
|
{
|
|
|
|
for (const auto & f : originals)
|
|
|
|
if (!f->isInjective(sample_block))
|
|
|
|
return false;
|
|
|
|
return true;
|
|
|
|
}
|
2018-04-29 01:00:26 +00:00
|
|
|
|
2018-08-30 16:31:20 +00:00
|
|
|
bool LLVMFunction::hasInformationAboutMonotonicity() const
|
|
|
|
{
|
|
|
|
for (const auto & f : originals)
|
|
|
|
if (!f->hasInformationAboutMonotonicity())
|
|
|
|
return false;
|
|
|
|
return true;
|
|
|
|
}
|
Implement JIT compilation, without a loop for now.
It actually seems to work, so long as you only have one row that is. E.g.
> select something(cast(number + 6 as Float64), cast(number + 2 as Float64)) from system.numbers limit 1';
8
with this IR:
define void @"something(CAST(plus(number, 6), 'Float64'), CAST(plus(number, 2), 'Float64'))"(void**, i8*, double*) {
entry:
%3 = load void*, void** %0
%4 = bitcast void* %3 to double*
%5 = load double, double* %4
%6 = getelementptr void*, void** %0, i32 1
%7 = load void*, void** %6
%8 = bitcast void* %7 to double*
%9 = load double, double* %8
%10 = fadd double %5, %9
store double %10, double* %2
ret void
}
2018-04-23 23:52:54 +00:00
|
|
|
|
2018-08-30 16:31:20 +00:00
|
|
|
LLVMFunction::Monotonicity LLVMFunction::getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const
|
|
|
|
{
|
|
|
|
const IDataType * type_ = &type;
|
|
|
|
Field left_ = left;
|
|
|
|
Field right_ = 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)
|
2018-04-27 15:44:38 +00:00
|
|
|
{
|
2018-08-30 16:31:20 +00:00
|
|
|
Monotonicity m = originals[i]->getMonotonicityForRange(*type_, left_, right_);
|
|
|
|
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())
|
2018-04-28 15:11:23 +00:00
|
|
|
{
|
2018-08-30 16:31:20 +00:00
|
|
|
if (left_ != Field())
|
|
|
|
applyFunction(*originals[i], left_);
|
|
|
|
if (right_ != Field())
|
|
|
|
applyFunction(*originals[i], right_);
|
|
|
|
if (!m.is_positive)
|
|
|
|
std::swap(left_, right_);
|
|
|
|
type_ = originals[i]->getReturnType().get();
|
2018-04-28 15:11:23 +00:00
|
|
|
}
|
2018-04-27 15:44:38 +00:00
|
|
|
}
|
2018-08-30 16:31:20 +00:00
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
|
2018-09-07 20:17:40 +00:00
|
|
|
static bool isCompilable(const IFunctionBase & function)
|
2018-04-25 11:55:54 +00:00
|
|
|
{
|
2018-09-07 20:17:40 +00:00
|
|
|
if (!canBeNativeType(*function.getReturnType()))
|
2018-04-29 01:00:26 +00:00
|
|
|
return false;
|
|
|
|
for (const auto & type : function.getArgumentTypes())
|
2018-09-07 20:17:40 +00:00
|
|
|
if (!canBeNativeType(*type))
|
2018-04-29 01:00:26 +00:00
|
|
|
return false;
|
|
|
|
return function.isCompilable();
|
2018-04-25 11:55:54 +00:00
|
|
|
}
|
|
|
|
|
2018-09-07 20:17:40 +00:00
|
|
|
std::vector<std::unordered_set<std::optional<size_t>>> getActionsDependents(const ExpressionActions::Actions & actions, const Names & output_columns)
|
2018-04-25 11:55:54 +00:00
|
|
|
{
|
2018-04-29 01:00:26 +00:00
|
|
|
/// an empty optional is a poisoned value prohibiting the column's producer from being removed
|
|
|
|
/// (which it could be, if it was inlined into every dependent function).
|
|
|
|
std::unordered_map<std::string, std::unordered_set<std::optional<size_t>>> current_dependents;
|
|
|
|
for (const auto & name : output_columns)
|
|
|
|
current_dependents[name].emplace();
|
|
|
|
/// a snapshot of each compilable function's dependents at the time of its execution.
|
|
|
|
std::vector<std::unordered_set<std::optional<size_t>>> dependents(actions.size());
|
|
|
|
for (size_t i = actions.size(); i--;)
|
|
|
|
{
|
|
|
|
switch (actions[i].type)
|
|
|
|
{
|
|
|
|
case ExpressionAction::REMOVE_COLUMN:
|
|
|
|
current_dependents.erase(actions[i].source_name);
|
|
|
|
/// poison every other column used after this point so that inlining chains do not cross it.
|
|
|
|
for (auto & dep : current_dependents)
|
|
|
|
dep.second.emplace();
|
|
|
|
break;
|
|
|
|
|
|
|
|
case ExpressionAction::PROJECT:
|
|
|
|
current_dependents.clear();
|
|
|
|
for (const auto & proj : actions[i].projection)
|
|
|
|
current_dependents[proj.first].emplace();
|
|
|
|
break;
|
|
|
|
|
2018-09-05 10:31:00 +00:00
|
|
|
case ExpressionAction::ADD_ALIASES:
|
|
|
|
for (const auto & proj : actions[i].projection)
|
|
|
|
current_dependents[proj.first].emplace();
|
|
|
|
break;
|
|
|
|
|
2018-04-29 01:00:26 +00:00
|
|
|
case ExpressionAction::ADD_COLUMN:
|
|
|
|
case ExpressionAction::COPY_COLUMN:
|
|
|
|
case ExpressionAction::ARRAY_JOIN:
|
|
|
|
case ExpressionAction::JOIN:
|
|
|
|
{
|
|
|
|
Names columns = actions[i].getNeededColumns();
|
|
|
|
for (const auto & column : columns)
|
|
|
|
current_dependents[column].emplace();
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
case ExpressionAction::APPLY_FUNCTION:
|
|
|
|
{
|
|
|
|
dependents[i] = current_dependents[actions[i].result_name];
|
2018-09-21 14:15:21 +00:00
|
|
|
const bool compilable = isCompilable(*actions[i].function_base);
|
2018-04-29 01:00:26 +00:00
|
|
|
for (const auto & name : actions[i].argument_names)
|
|
|
|
{
|
|
|
|
if (compilable)
|
|
|
|
current_dependents[name].emplace(i);
|
|
|
|
else
|
|
|
|
current_dependents[name].emplace();
|
|
|
|
}
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2018-09-07 20:17:40 +00:00
|
|
|
return dependents;
|
|
|
|
}
|
2018-04-29 01:00:26 +00:00
|
|
|
|
2019-02-11 14:36:54 +00:00
|
|
|
void compileFunctions(ExpressionActions::Actions & actions, const Names & output_columns, const Block & sample_block, std::shared_ptr<CompiledExpressionCache> compilation_cache, size_t min_count_to_compile_expression)
|
2018-09-07 20:17:40 +00:00
|
|
|
{
|
|
|
|
static std::unordered_map<UInt128, UInt32, UInt128Hash> counter;
|
|
|
|
static std::mutex mutex;
|
|
|
|
|
|
|
|
struct LLVMTargetInitializer
|
|
|
|
{
|
|
|
|
LLVMTargetInitializer()
|
|
|
|
{
|
|
|
|
llvm::InitializeNativeTarget();
|
|
|
|
llvm::InitializeNativeTargetAsmPrinter();
|
|
|
|
llvm::sys::DynamicLibrary::LoadLibraryPermanently(nullptr);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
static LLVMTargetInitializer initializer;
|
|
|
|
|
|
|
|
auto dependents = getActionsDependents(actions, output_columns);
|
2018-04-29 01:00:26 +00:00
|
|
|
std::vector<ExpressionActions::Actions> fused(actions.size());
|
2018-05-06 10:42:35 +00:00
|
|
|
for (size_t i = 0; i < actions.size(); ++i)
|
2018-04-25 11:55:54 +00:00
|
|
|
{
|
2018-09-21 14:15:21 +00:00
|
|
|
if (actions[i].type != ExpressionAction::APPLY_FUNCTION || !isCompilable(*actions[i].function_base))
|
2018-04-29 01:00:26 +00:00
|
|
|
continue;
|
2018-05-07 06:23:18 +00:00
|
|
|
|
2018-04-29 22:23:27 +00:00
|
|
|
fused[i].push_back(actions[i]);
|
2018-04-29 01:00:26 +00:00
|
|
|
if (dependents[i].find({}) != dependents[i].end())
|
2018-04-25 11:55:54 +00:00
|
|
|
{
|
2018-05-03 13:34:42 +00:00
|
|
|
/// the result of compiling one function in isolation is pretty much the same as its `execute` method.
|
|
|
|
if (fused[i].size() == 1)
|
|
|
|
continue;
|
2018-08-29 17:49:43 +00:00
|
|
|
|
2018-09-07 20:17:40 +00:00
|
|
|
auto hash_key = ExpressionActions::ActionsHash{}(fused[i]);
|
|
|
|
{
|
2018-10-12 13:03:57 +00:00
|
|
|
std::lock_guard lock(mutex);
|
2019-02-11 14:36:54 +00:00
|
|
|
if (counter[hash_key]++ < min_count_to_compile_expression)
|
2018-09-07 20:17:40 +00:00
|
|
|
continue;
|
|
|
|
}
|
2018-09-17 15:34:43 +00:00
|
|
|
|
2018-08-29 17:49:43 +00:00
|
|
|
std::shared_ptr<LLVMFunction> fn;
|
2018-08-31 10:27:08 +00:00
|
|
|
if (compilation_cache)
|
2018-08-29 17:49:43 +00:00
|
|
|
{
|
2018-10-12 13:03:57 +00:00
|
|
|
std::tie(fn, std::ignore) = compilation_cache->getOrSet(hash_key, [&inlined_func=std::as_const(fused[i]), &sample_block] ()
|
2018-09-07 20:17:40 +00:00
|
|
|
{
|
|
|
|
Stopwatch watch;
|
2019-01-11 21:16:49 +00:00
|
|
|
std::shared_ptr<LLVMFunction> result_fn;
|
|
|
|
result_fn = std::make_shared<LLVMFunction>(inlined_func, sample_block);
|
2018-09-03 10:14:05 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds());
|
2018-10-12 13:03:57 +00:00
|
|
|
return result_fn;
|
|
|
|
});
|
2018-08-29 17:49:43 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2018-09-03 10:14:05 +00:00
|
|
|
Stopwatch watch;
|
2019-01-11 21:16:49 +00:00
|
|
|
fn = std::make_shared<LLVMFunction>(fused[i], sample_block);
|
2018-09-03 10:14:05 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds());
|
2018-08-29 17:49:43 +00:00
|
|
|
}
|
2018-09-03 10:14:05 +00:00
|
|
|
|
2018-09-21 14:15:21 +00:00
|
|
|
actions[i].function_base = fn;
|
2018-08-29 17:49:43 +00:00
|
|
|
actions[i].argument_names = fn->getArgumentNames();
|
2018-08-29 20:05:31 +00:00
|
|
|
actions[i].is_function_compiled = true;
|
2018-08-29 17:49:43 +00:00
|
|
|
|
2018-04-29 01:00:26 +00:00
|
|
|
continue;
|
2018-04-25 11:55:54 +00:00
|
|
|
}
|
2018-05-07 06:23:18 +00:00
|
|
|
|
2018-04-29 01:00:26 +00:00
|
|
|
/// TODO: determine whether it's profitable to inline the function if there's more than one dependent.
|
|
|
|
for (const auto & dep : dependents[i])
|
2018-04-29 22:23:27 +00:00
|
|
|
fused[*dep].insert(fused[*dep].end(), fused[i].begin(), fused[i].end());
|
2018-04-25 11:55:54 +00:00
|
|
|
}
|
2018-05-07 06:23:18 +00:00
|
|
|
|
2018-09-21 14:15:21 +00:00
|
|
|
for (size_t i = 0; i < actions.size(); ++i)
|
|
|
|
{
|
|
|
|
if (actions[i].type == ExpressionAction::APPLY_FUNCTION && actions[i].is_function_compiled)
|
|
|
|
actions[i].function = actions[i].function_base->prepare({}, {}, 0); /// Arguments are not used for LLVMFunction.
|
|
|
|
}
|
2018-04-25 11:55:54 +00:00
|
|
|
}
|
|
|
|
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
}
|
|
|
|
|
2018-04-25 15:19:22 +00:00
|
|
|
#endif
|