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
|
|
|
#pragma once
|
|
|
|
|
2018-04-24 14:28:01 +00:00
|
|
|
#include <Columns/ColumnVector.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.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
|
|
|
#include <Functions/IFunction.h>
|
|
|
|
|
|
|
|
#include <Interpreters/ExpressionActions.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-04-24 14:28:01 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
|
|
|
|
2018-04-24 10:25:18 +00:00
|
|
|
class LLVMContext
|
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-24 10:25:18 +00:00
|
|
|
struct Data;
|
|
|
|
std::shared_ptr<Data> shared;
|
|
|
|
|
|
|
|
public:
|
|
|
|
LLVMContext();
|
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
|
|
|
|
|
|
|
void finalize();
|
2018-04-24 10:25:18 +00:00
|
|
|
|
|
|
|
bool isCompilable(const IFunctionBase& function) const;
|
|
|
|
|
|
|
|
Data * operator->() const {
|
|
|
|
return shared.get();
|
|
|
|
}
|
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
|
|
|
};
|
|
|
|
|
|
|
|
// second array is of `char` because `LLVMPreparedFunction::executeImpl` can't use a `std::vector<bool>` for this
|
|
|
|
using LLVMCompiledFunction = void(const void ** inputs, const char * is_constant, void * output, size_t block_size);
|
|
|
|
|
|
|
|
class LLVMPreparedFunction : public PreparedFunctionImpl
|
|
|
|
{
|
|
|
|
std::shared_ptr<const IFunctionBase> parent;
|
2018-04-24 10:25:18 +00:00
|
|
|
LLVMContext context;
|
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
|
|
|
LLVMCompiledFunction * function;
|
|
|
|
|
|
|
|
public:
|
2018-04-24 10:25:18 +00:00
|
|
|
LLVMPreparedFunction(LLVMContext context, std::shared_ptr<const IFunctionBase> parent);
|
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
|
|
|
|
|
|
|
String getName() const override { return parent->getName(); }
|
|
|
|
|
|
|
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
|
|
|
{
|
|
|
|
size_t block_size = 0;
|
|
|
|
std::vector<const void *> columns(arguments.size());
|
|
|
|
std::vector<char> is_const(arguments.size());
|
|
|
|
for (size_t i = 0; i < arguments.size(); i++)
|
|
|
|
{
|
|
|
|
auto * column = block.getByPosition(arguments[i]).column.get();
|
|
|
|
if (column->size())
|
|
|
|
// assume the column is a `ColumnVector<T>`. there's probably no good way to actually
|
|
|
|
// check that at runtime, so let's just hope it's always true for columns containing types
|
2018-04-24 18:46:30 +00:00
|
|
|
// for which `LLVMContext::Data::toNativeType` returns non-null.
|
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
|
|
|
columns[i] = column->getDataAt(0).data;
|
|
|
|
is_const[i] = column->isColumnConst();
|
|
|
|
block_size = column->size();
|
|
|
|
}
|
2018-04-24 14:28:01 +00:00
|
|
|
auto col_res = createColumn(parent->getReturnType(), 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
|
|
|
function(columns.data(), is_const.data(), const_cast<char *>(col_res->getDataAt(0).data), block_size);
|
|
|
|
block.getByPosition(result).column = std::move(col_res);
|
|
|
|
};
|
2018-04-24 14:28:01 +00:00
|
|
|
|
|
|
|
private:
|
|
|
|
static IColumn::Ptr createColumn(const DataTypePtr & type, size_t size)
|
|
|
|
{
|
|
|
|
if (type->equals(DataTypeInt8{}))
|
|
|
|
return ColumnVector<Int8>::create(size);
|
|
|
|
if (type->equals(DataTypeInt16{}))
|
|
|
|
return ColumnVector<Int16>::create(size);
|
|
|
|
if (type->equals(DataTypeInt32{}))
|
|
|
|
return ColumnVector<Int32>::create(size);
|
|
|
|
if (type->equals(DataTypeInt64{}))
|
|
|
|
return ColumnVector<Int64>::create(size);
|
|
|
|
if (type->equals(DataTypeUInt8{}))
|
|
|
|
return ColumnVector<UInt8>::create(size);
|
|
|
|
if (type->equals(DataTypeUInt16{}))
|
|
|
|
return ColumnVector<UInt16>::create(size);
|
|
|
|
if (type->equals(DataTypeUInt32{}))
|
|
|
|
return ColumnVector<UInt32>::create(size);
|
|
|
|
if (type->equals(DataTypeUInt64{}))
|
|
|
|
return ColumnVector<UInt64>::create(size);
|
|
|
|
if (type->equals(DataTypeFloat32{}))
|
|
|
|
return ColumnVector<Float32>::create(size);
|
|
|
|
if (type->equals(DataTypeFloat64{}))
|
|
|
|
return ColumnVector<Float64>::create(size);
|
|
|
|
throw Exception("LLVMPreparedFunction::createColumn received an unsupported data type; check "
|
|
|
|
"that the list is consistent with LLVMContext::Data::toNativeType", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
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
|
|
|
};
|
|
|
|
|
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
|
|
|
class LLVMFunction : public std::enable_shared_from_this<LLVMFunction>, public IFunctionBase
|
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
|
|
|
{
|
|
|
|
ExpressionActions::Actions actions; // all of them must have type APPLY_FUNCTION
|
|
|
|
Names arg_names;
|
|
|
|
DataTypes arg_types;
|
2018-04-24 10:25:18 +00:00
|
|
|
LLVMContext context;
|
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
|
|
|
|
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
|
|
|
public:
|
2018-04-24 10:25:18 +00:00
|
|
|
LLVMFunction(ExpressionActions::Actions actions, LLVMContext context);
|
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
|
|
|
|
|
|
|
String getName() const override { return actions.back().result_name; }
|
|
|
|
|
|
|
|
const Names & getArgumentNames() const { return arg_names; }
|
|
|
|
|
|
|
|
const DataTypes & getArgumentTypes() const override { return arg_types; }
|
|
|
|
|
|
|
|
const DataTypePtr & getReturnType() const override { return actions.back().function->getReturnType(); }
|
|
|
|
|
|
|
|
PreparedFunctionPtr prepare(const Block &) const override { return std::make_shared<LLVMPreparedFunction>(context, shared_from_this()); }
|
|
|
|
|
|
|
|
bool isDeterministic() override
|
|
|
|
{
|
|
|
|
for (const auto & action : actions)
|
|
|
|
if (!action.function->isDeterministic())
|
|
|
|
return false;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool isDeterministicInScopeOfQuery() override
|
|
|
|
{
|
|
|
|
for (const auto & action : actions)
|
|
|
|
if (!action.function->isDeterministicInScopeOfQuery())
|
|
|
|
return false;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
// TODO: these methods require reconstructing the call tree:
|
|
|
|
// bool isSuitableForConstantFolding() const;
|
|
|
|
// bool isInjective(const Block & sample_block);
|
|
|
|
// bool hasInformationAboutMonotonicity() const;
|
|
|
|
// Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const;
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|