2022-09-28 13:29:29 +00:00
|
|
|
#include "config.h"
|
2018-04-25 15:19:22 +00:00
|
|
|
|
|
|
|
#if USE_EMBEDDED_COMPILER
|
|
|
|
|
2018-05-06 10:42:35 +00:00
|
|
|
#include <optional>
|
2021-05-01 07:06:30 +00:00
|
|
|
#include <stack>
|
2018-05-06 10:42:35 +00:00
|
|
|
|
2022-04-27 15:05:45 +00:00
|
|
|
#include <Common/logger_useful.h>
|
2022-01-30 19:49:48 +00:00
|
|
|
#include <base/sort.h>
|
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>
|
|
|
|
#include <Common/typeid_cast.h>
|
2019-08-21 02:28:04 +00:00
|
|
|
#include <Common/assert_cast.h>
|
2018-04-24 19:42:06 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2021-05-08 18:04:21 +00:00
|
|
|
#include <Functions/FunctionsComparison.h>
|
2018-04-27 21:30:38 +00:00
|
|
|
#include <DataTypes/Native.h>
|
2019-12-12 16:59:17 +00:00
|
|
|
#include <Functions/IFunctionAdaptors.h>
|
2019-12-18 00:37:35 +00:00
|
|
|
|
2021-04-29 11:15:35 +00:00
|
|
|
#include <Interpreters/JIT/CHJIT.h>
|
2021-05-04 20:32:43 +00:00
|
|
|
#include <Interpreters/JIT/CompileDAG.h>
|
|
|
|
#include <Interpreters/JIT/compileFunction.h>
|
2021-06-13 12:38:57 +00:00
|
|
|
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
2021-05-04 20:32:43 +00:00
|
|
|
#include <Interpreters/ActionsDAG.h>
|
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;
|
|
|
|
}
|
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
static CHJIT & getJITInstance()
|
2018-04-27 15:44:38 +00:00
|
|
|
{
|
2021-05-04 20:32:43 +00:00
|
|
|
static CHJIT jit;
|
|
|
|
return jit;
|
2018-04-27 15:44:38 +00:00
|
|
|
}
|
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
static Poco::Logger * getLogger()
|
|
|
|
{
|
|
|
|
static Poco::Logger & logger = Poco::Logger::get("ExpressionJIT");
|
|
|
|
return &logger;
|
|
|
|
}
|
|
|
|
|
2021-06-13 12:38:57 +00:00
|
|
|
class CompiledFunctionHolder : public CompiledExpressionCacheEntry
|
2021-05-24 22:22:30 +00:00
|
|
|
{
|
|
|
|
public:
|
|
|
|
|
2021-06-03 19:20:53 +00:00
|
|
|
explicit CompiledFunctionHolder(CompiledFunction compiled_function_)
|
2021-06-13 12:38:57 +00:00
|
|
|
: CompiledExpressionCacheEntry(compiled_function_.compiled_module.size)
|
|
|
|
, compiled_function(compiled_function_)
|
2021-05-24 22:22:30 +00:00
|
|
|
{}
|
|
|
|
|
2021-06-13 12:38:57 +00:00
|
|
|
~CompiledFunctionHolder() override
|
2021-05-24 22:22:30 +00:00
|
|
|
{
|
2021-06-03 19:20:53 +00:00
|
|
|
getJITInstance().deleteCompiledModule(compiled_function.compiled_module);
|
2021-05-24 22:22:30 +00:00
|
|
|
}
|
|
|
|
|
2021-06-03 19:20:53 +00:00
|
|
|
CompiledFunction compiled_function;
|
2021-05-24 22:22:30 +00:00
|
|
|
};
|
|
|
|
|
2021-05-15 17:33:15 +00:00
|
|
|
class LLVMExecutableFunction : public IExecutableFunction
|
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
|
|
|
public:
|
2021-05-07 18:36:07 +00:00
|
|
|
|
2021-06-03 19:20:53 +00:00
|
|
|
explicit LLVMExecutableFunction(const std::string & name_, std::shared_ptr<CompiledFunctionHolder> compiled_function_holder_)
|
2019-01-11 21:16:49 +00:00
|
|
|
: name(name_)
|
2021-06-03 19:20:53 +00:00
|
|
|
, compiled_function_holder(compiled_function_holder_)
|
2018-10-11 20:31:41 +00:00
|
|
|
{
|
|
|
|
}
|
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; }
|
|
|
|
|
2021-05-15 17:33:15 +00:00
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const 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
|
|
|
{
|
2021-05-03 21:14:47 +00:00
|
|
|
if (!canBeNativeType(*result_type))
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "LLVMExecutableFunction unexpected result type in: {}", result_type->getName());
|
|
|
|
|
2021-05-02 18:15:34 +00:00
|
|
|
auto result_column = result_type->createColumn();
|
2019-12-18 05:11:26 +00:00
|
|
|
|
2021-04-30 11:18:27 +00:00
|
|
|
if (input_rows_count)
|
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
|
|
|
{
|
2021-05-02 18:15:34 +00:00
|
|
|
result_column = result_column->cloneResized(input_rows_count);
|
|
|
|
|
2018-04-29 01:00:26 +00:00
|
|
|
std::vector<ColumnData> columns(arguments.size() + 1);
|
2021-05-09 22:02:54 +00:00
|
|
|
std::vector<ColumnPtr> columns_backup;
|
|
|
|
|
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
|
|
|
{
|
2021-05-09 22:02:54 +00:00
|
|
|
auto column = arguments[i].column->convertToFullColumnIfConst();
|
|
|
|
columns_backup.emplace_back(column);
|
|
|
|
columns[i] = getColumnData(column.get());
|
2018-04-29 01:00:26 +00:00
|
|
|
}
|
2021-05-02 18:15:34 +00:00
|
|
|
|
|
|
|
columns[arguments.size()] = getColumnData(result_column.get());
|
2021-05-24 22:22:30 +00:00
|
|
|
|
2021-06-03 19:20:53 +00:00
|
|
|
auto jit_compiled_function = compiled_function_holder->compiled_function.compiled_function;
|
|
|
|
jit_compiled_function(input_rows_count, columns.data());
|
2019-12-31 01:50:01 +00:00
|
|
|
|
2021-05-02 18:15:34 +00:00
|
|
|
#if defined(MEMORY_SANITIZER)
|
2019-12-31 01:51:09 +00:00
|
|
|
/// Memory sanitizer don't know about stores from JIT-ed code.
|
|
|
|
/// But maybe we can generate this code with MSan instrumentation?
|
2021-05-02 18:15:34 +00:00
|
|
|
|
|
|
|
if (const auto * nullable_column = typeid_cast<const ColumnNullable *>(result_column.get()))
|
|
|
|
{
|
|
|
|
const auto & nested_column = nullable_column->getNestedColumn();
|
|
|
|
const auto & null_map_column = nullable_column->getNullMapColumn();
|
|
|
|
|
|
|
|
auto nested_column_raw_data = nested_column.getRawData();
|
2022-08-21 18:10:32 +00:00
|
|
|
__msan_unpoison(nested_column_raw_data.data(), nested_column_raw_data.size());
|
2021-05-02 18:15:34 +00:00
|
|
|
|
|
|
|
auto null_map_column_raw_data = null_map_column.getRawData();
|
2022-08-21 18:10:32 +00:00
|
|
|
__msan_unpoison(null_map_column_raw_data.data(), null_map_column_raw_data.size());
|
2021-05-02 18:15:34 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2022-08-21 18:10:32 +00:00
|
|
|
__msan_unpoison(result_column->getRawData().data(), result_column->getRawData().size());
|
2021-05-02 18:15:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
#endif
|
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-12-18 05:11:26 +00:00
|
|
|
|
2021-05-02 18:15:34 +00:00
|
|
|
return result_column;
|
2018-09-03 10:14:05 +00:00
|
|
|
}
|
2021-05-02 18:15:34 +00:00
|
|
|
|
2021-05-09 22:02:54 +00:00
|
|
|
private:
|
|
|
|
std::string name;
|
2021-06-03 19:20:53 +00:00
|
|
|
std::shared_ptr<CompiledFunctionHolder> compiled_function_holder;
|
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
|
|
|
|
2021-05-15 17:33:15 +00:00
|
|
|
class LLVMFunction : public IFunctionBase
|
2018-04-29 18:03:58 +00:00
|
|
|
{
|
2021-05-04 20:32:43 +00:00
|
|
|
public:
|
2021-04-29 11:15:35 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
explicit LLVMFunction(const CompileDAG & dag_)
|
|
|
|
: name(dag_.dump())
|
|
|
|
, dag(dag_)
|
|
|
|
{
|
|
|
|
for (size_t i = 0; i < dag.getNodesCount(); ++i)
|
|
|
|
{
|
|
|
|
const auto & node = dag[i];
|
2021-04-27 11:10:38 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
if (node.type == CompileDAG::CompileType::FUNCTION)
|
|
|
|
nested_functions.emplace_back(node.function);
|
|
|
|
else if (node.type == CompileDAG::CompileType::INPUT)
|
|
|
|
argument_types.emplace_back(node.result_type);
|
|
|
|
}
|
2018-04-29 18:03:58 +00:00
|
|
|
}
|
|
|
|
|
2021-06-03 19:20:53 +00:00
|
|
|
void setCompiledFunction(std::shared_ptr<CompiledFunctionHolder> compiled_function_holder_)
|
2018-04-29 18:03:58 +00:00
|
|
|
{
|
2021-06-03 19:20:53 +00:00
|
|
|
compiled_function_holder = compiled_function_holder_;
|
2018-04-29 18:03:58 +00:00
|
|
|
}
|
2021-05-03 21:14:47 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
bool isCompilable() const override { return true; }
|
2021-05-03 22:34:40 +00:00
|
|
|
|
2023-06-03 18:31:08 +00:00
|
|
|
llvm::Value * compile(llvm::IRBuilderBase & builder, const ValuesWithType & arguments) const override
|
2018-04-29 18:03:58 +00:00
|
|
|
{
|
2023-06-03 18:31:08 +00:00
|
|
|
return dag.compile(builder, arguments).value;
|
2021-05-04 20:32:43 +00:00
|
|
|
}
|
2021-05-03 21:14:47 +00:00
|
|
|
|
2021-06-22 16:21:23 +00:00
|
|
|
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & arguments) const override
|
2021-05-17 13:06:11 +00:00
|
|
|
{
|
|
|
|
for (const auto & f : nested_functions)
|
2021-06-22 16:21:23 +00:00
|
|
|
if (!f->isSuitableForShortCircuitArgumentsExecution(arguments))
|
|
|
|
return false;
|
2021-05-17 13:06:11 +00:00
|
|
|
|
2021-06-22 16:21:23 +00:00
|
|
|
return true;
|
2021-05-17 13:06:11 +00:00
|
|
|
}
|
2021-05-14 14:07:24 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
String getName() const override { return name; }
|
2021-05-03 21:14:47 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
const DataTypes & getArgumentTypes() const override { return argument_types; }
|
2021-05-03 21:14:47 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
const DataTypePtr & getResultType() const override { return dag.back().result_type; }
|
2021-04-27 11:10:38 +00:00
|
|
|
|
2021-05-15 17:33:15 +00:00
|
|
|
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override
|
2018-04-29 18:03:58 +00:00
|
|
|
{
|
2021-06-03 19:20:53 +00:00
|
|
|
if (!compiled_function_holder)
|
2021-05-24 22:22:30 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Compiled function was not initialized {}", name);
|
2021-05-05 15:39:26 +00:00
|
|
|
|
2021-06-03 19:20:53 +00:00
|
|
|
return std::make_unique<LLVMExecutableFunction>(name, compiled_function_holder);
|
2018-04-29 18:03:58 +00:00
|
|
|
}
|
2021-05-03 21:14:47 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
bool isDeterministic() const override
|
|
|
|
{
|
|
|
|
for (const auto & f : nested_functions)
|
|
|
|
if (!f->isDeterministic())
|
|
|
|
return false;
|
2018-04-29 18:03:58 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
return true;
|
|
|
|
}
|
2018-04-29 18:03:58 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
bool isDeterministicInScopeOfQuery() const override
|
2018-04-29 18:03:58 +00:00
|
|
|
{
|
2021-05-04 20:32:43 +00:00
|
|
|
for (const auto & f : nested_functions)
|
|
|
|
if (!f->isDeterministicInScopeOfQuery())
|
|
|
|
return false;
|
2018-04-29 18:03:58 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
return true;
|
|
|
|
}
|
2018-04-29 18:03:58 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
bool isSuitableForConstantFolding() const override
|
2021-05-03 21:14:47 +00:00
|
|
|
{
|
2021-05-04 20:32:43 +00:00
|
|
|
for (const auto & f : nested_functions)
|
|
|
|
if (!f->isSuitableForConstantFolding())
|
|
|
|
return false;
|
2018-04-29 18:03:58 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
return true;
|
|
|
|
}
|
2018-04-29 18:03:58 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
bool isInjective(const ColumnsWithTypeAndName & sample_block) const override
|
2018-04-29 18:03:58 +00:00
|
|
|
{
|
2021-05-04 20:32:43 +00:00
|
|
|
for (const auto & f : nested_functions)
|
|
|
|
if (!f->isInjective(sample_block))
|
|
|
|
return false;
|
2021-05-03 21:14:47 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
return true;
|
|
|
|
}
|
2021-05-03 21:14:47 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
bool hasInformationAboutMonotonicity() const override
|
|
|
|
{
|
|
|
|
for (const auto & f : nested_functions)
|
|
|
|
if (!f->hasInformationAboutMonotonicity())
|
|
|
|
return false;
|
2018-04-29 18:03:58 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
return true;
|
|
|
|
}
|
2020-11-05 16:11:14 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
|
2018-04-27 21:30:38 +00:00
|
|
|
{
|
2021-05-04 20:32:43 +00:00
|
|
|
const IDataType * type_ptr = &type;
|
|
|
|
Field left_mut = left;
|
|
|
|
Field right_mut = right;
|
2021-09-29 16:01:26 +00:00
|
|
|
|
|
|
|
Monotonicity result = { .is_monotonic = true, .is_positive = true, .is_always_monotonic = true };
|
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
/// 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)
|
2021-05-01 13:31:06 +00:00
|
|
|
{
|
2021-05-04 20:32:43 +00:00
|
|
|
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())
|
2018-04-29 01:00:26 +00:00
|
|
|
{
|
2021-05-04 20:32:43 +00:00
|
|
|
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();
|
2018-04-29 01:00:26 +00:00
|
|
|
}
|
|
|
|
}
|
2021-05-04 20:32:43 +00:00
|
|
|
return result;
|
|
|
|
}
|
2020-11-05 16:11:14 +00:00
|
|
|
|
2022-11-30 18:48:09 +00:00
|
|
|
static void applyFunction(const IFunctionBase & function, Field & value)
|
2021-05-04 20:32:43 +00:00
|
|
|
{
|
|
|
|
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);
|
|
|
|
}
|
2021-05-03 21:14:47 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
private:
|
|
|
|
std::string name;
|
|
|
|
CompileDAG dag;
|
|
|
|
DataTypes argument_types;
|
|
|
|
std::vector<FunctionBasePtr> nested_functions;
|
2021-06-03 19:20:53 +00:00
|
|
|
std::shared_ptr<CompiledFunctionHolder> compiled_function_holder;
|
2021-05-04 20:32:43 +00:00
|
|
|
};
|
2018-04-29 01:00:26 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
static FunctionBasePtr compile(
|
|
|
|
const CompileDAG & dag,
|
|
|
|
size_t min_count_to_compile_expression)
|
2018-10-11 20:31:41 +00:00
|
|
|
{
|
2021-05-04 20:32:43 +00:00
|
|
|
static std::unordered_map<UInt128, UInt64, UInt128Hash> counter;
|
|
|
|
static std::mutex mutex;
|
2018-04-29 01:00:26 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
auto hash_key = dag.hash();
|
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
if (counter[hash_key]++ < min_count_to_compile_expression)
|
|
|
|
return nullptr;
|
|
|
|
}
|
2018-04-29 01:00:26 +00:00
|
|
|
|
2021-05-25 20:53:46 +00:00
|
|
|
auto llvm_function = std::make_shared<LLVMFunction>(dag);
|
2018-04-24 13:21:42 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
if (auto * compilation_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
|
|
|
|
{
|
2021-05-31 20:17:40 +00:00
|
|
|
auto [compiled_function_cache_entry, _] = compilation_cache->getOrSet(hash_key, [&] ()
|
2021-05-24 22:22:30 +00:00
|
|
|
{
|
2021-05-31 20:17:40 +00:00
|
|
|
LOG_TRACE(getLogger(), "Compile expression {}", llvm_function->getName());
|
2021-06-03 19:20:53 +00:00
|
|
|
auto compiled_function = compileFunction(getJITInstance(), *llvm_function);
|
2021-06-13 12:38:57 +00:00
|
|
|
return std::make_shared<CompiledFunctionHolder>(compiled_function);
|
2021-05-04 20:32:43 +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
|
|
|
|
2021-06-13 12:38:57 +00:00
|
|
|
std::shared_ptr<CompiledFunctionHolder> compiled_function_holder = std::static_pointer_cast<CompiledFunctionHolder>(compiled_function_cache_entry);
|
|
|
|
llvm_function->setCompiledFunction(std::move(compiled_function_holder));
|
2021-05-04 20:32:43 +00:00
|
|
|
}
|
|
|
|
else
|
2018-04-27 15:44:38 +00:00
|
|
|
{
|
2021-06-03 19:20:53 +00:00
|
|
|
auto compiled_function = compileFunction(getJITInstance(), *llvm_function);
|
2021-06-13 12:38:57 +00:00
|
|
|
auto compiled_function_holder = std::make_shared<CompiledFunctionHolder>(compiled_function);
|
2021-06-03 19:20:53 +00:00
|
|
|
|
2021-06-13 12:38:57 +00:00
|
|
|
llvm_function->setCompiledFunction(std::move(compiled_function_holder));
|
2018-04-27 15:44:38 +00:00
|
|
|
}
|
2018-08-30 16:31:20 +00:00
|
|
|
|
2021-05-25 20:53:46 +00:00
|
|
|
return llvm_function;
|
2021-05-04 20:32:43 +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
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
static bool isCompilableConstant(const ActionsDAG::Node & node)
|
2020-11-05 16:11:14 +00:00
|
|
|
{
|
2021-05-22 12:49:21 +00:00
|
|
|
return node.column && isColumnConst(*node.column) && canBeNativeType(*node.result_type);
|
2021-05-05 15:39:26 +00:00
|
|
|
}
|
|
|
|
|
2021-10-01 09:09:28 +00:00
|
|
|
static const ActionsDAG::Node * removeAliasIfNecessary(const ActionsDAG::Node * node)
|
2021-09-30 16:23:15 +00:00
|
|
|
{
|
|
|
|
const ActionsDAG::Node * node_no_alias = node;
|
|
|
|
|
|
|
|
while (node_no_alias->type == ActionsDAG::ActionType::ALIAS)
|
|
|
|
node_no_alias = node_no_alias->children[0];
|
|
|
|
|
|
|
|
return node_no_alias;
|
|
|
|
}
|
|
|
|
|
2021-08-10 11:31:15 +00:00
|
|
|
static bool isCompilableFunction(const ActionsDAG::Node & node, const std::unordered_set<const ActionsDAG::Node *> & lazy_executed_nodes)
|
2021-05-05 15:39:26 +00:00
|
|
|
{
|
|
|
|
if (node.type != ActionsDAG::ActionType::FUNCTION)
|
|
|
|
return false;
|
|
|
|
|
2022-12-21 00:22:46 +00:00
|
|
|
const auto & function = *node.function_base;
|
2021-05-05 15:39:26 +00:00
|
|
|
|
2021-06-22 16:21:23 +00:00
|
|
|
IFunction::ShortCircuitSettings settings;
|
|
|
|
if (function.isShortCircuit(settings, node.children.size()))
|
|
|
|
{
|
|
|
|
for (const auto & child : node.children)
|
|
|
|
{
|
2021-10-01 09:09:28 +00:00
|
|
|
const ActionsDAG::Node * child_no_alias = removeAliasIfNecessary(child);
|
2021-09-30 16:23:15 +00:00
|
|
|
|
|
|
|
if (lazy_executed_nodes.contains(child_no_alias))
|
2021-06-22 16:21:23 +00:00
|
|
|
return false;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-11-05 16:11:14 +00:00
|
|
|
if (!canBeNativeType(*function.getResultType()))
|
|
|
|
return false;
|
2021-04-27 11:10:38 +00:00
|
|
|
|
2020-11-05 16:11:14 +00:00
|
|
|
for (const auto & type : function.getArgumentTypes())
|
2021-04-27 11:10:38 +00:00
|
|
|
{
|
2020-11-05 16:11:14 +00:00
|
|
|
if (!canBeNativeType(*type))
|
|
|
|
return false;
|
2021-04-27 11:10:38 +00:00
|
|
|
}
|
|
|
|
|
2020-11-05 16:11:14 +00:00
|
|
|
return function.isCompilable();
|
|
|
|
}
|
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
static CompileDAG getCompilableDAG(
|
2021-03-03 20:01:07 +00:00
|
|
|
const ActionsDAG::Node * root,
|
2021-09-08 19:23:57 +00:00
|
|
|
ActionsDAG::NodeRawConstPtrs & children,
|
|
|
|
const std::unordered_set<const ActionsDAG::Node *> & lazy_executed_nodes)
|
2020-11-05 16:11:14 +00:00
|
|
|
{
|
2021-05-24 08:59:27 +00:00
|
|
|
/// Extract CompileDAG from root actions dag node.
|
2021-05-05 15:39:26 +00:00
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
CompileDAG dag;
|
2020-11-05 16:11:14 +00:00
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
std::unordered_map<const ActionsDAG::Node *, size_t> visited_node_to_compile_dag_position;
|
|
|
|
|
2020-11-05 16:11:14 +00:00
|
|
|
struct Frame
|
|
|
|
{
|
2021-03-03 20:01:07 +00:00
|
|
|
const ActionsDAG::Node * node;
|
2020-11-05 16:11:14 +00:00
|
|
|
size_t next_child_to_visit = 0;
|
|
|
|
};
|
|
|
|
|
|
|
|
std::stack<Frame> stack;
|
2021-05-05 15:39:26 +00:00
|
|
|
stack.emplace(Frame{.node = root});
|
2020-11-05 16:11:14 +00:00
|
|
|
|
|
|
|
while (!stack.empty())
|
|
|
|
{
|
|
|
|
auto & frame = stack.top();
|
2021-05-05 15:39:26 +00:00
|
|
|
const auto * node = frame.node;
|
2020-11-05 16:11:14 +00:00
|
|
|
|
2021-05-24 08:59:27 +00:00
|
|
|
bool is_compilable_constant = isCompilableConstant(*node);
|
2021-09-08 19:23:57 +00:00
|
|
|
bool is_compilable_function = isCompilableFunction(*node, lazy_executed_nodes);
|
2021-05-24 08:59:27 +00:00
|
|
|
|
|
|
|
if (!is_compilable_function || is_compilable_constant)
|
|
|
|
{
|
2021-09-09 19:42:47 +00:00
|
|
|
CompileDAG::Node compile_node;
|
|
|
|
compile_node.function = node->function_base;
|
|
|
|
compile_node.result_type = node->result_type;
|
|
|
|
|
|
|
|
if (is_compilable_constant)
|
|
|
|
{
|
|
|
|
compile_node.type = CompileDAG::CompileType::CONSTANT;
|
|
|
|
compile_node.column = node->column;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2021-05-24 08:59:27 +00:00
|
|
|
compile_node.type = CompileDAG::CompileType::INPUT;
|
|
|
|
children.emplace_back(node);
|
2021-09-09 19:42:47 +00:00
|
|
|
}
|
2021-05-24 08:59:27 +00:00
|
|
|
|
2021-09-09 19:42:47 +00:00
|
|
|
visited_node_to_compile_dag_position[node] = dag.getNodesCount();
|
|
|
|
dag.addNode(std::move(compile_node));
|
|
|
|
stack.pop();
|
|
|
|
continue;
|
2021-05-24 08:59:27 +00:00
|
|
|
}
|
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
while (frame.next_child_to_visit < node->children.size())
|
2020-11-05 16:11:14 +00:00
|
|
|
{
|
2021-05-05 15:39:26 +00:00
|
|
|
const auto & child = node->children[frame.next_child_to_visit];
|
2020-11-05 16:11:14 +00:00
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
if (visited_node_to_compile_dag_position.contains(child))
|
2020-11-05 18:08:23 +00:00
|
|
|
{
|
2021-05-05 15:39:26 +00:00
|
|
|
++frame.next_child_to_visit;
|
|
|
|
continue;
|
2020-11-05 18:08:23 +00:00
|
|
|
}
|
2021-05-05 15:39:26 +00:00
|
|
|
|
|
|
|
stack.emplace(Frame{.node = child});
|
|
|
|
break;
|
2020-11-05 16:11:14 +00:00
|
|
|
}
|
|
|
|
|
2021-05-05 20:08:31 +00:00
|
|
|
bool all_children_visited = frame.next_child_to_visit == node->children.size();
|
2020-11-05 16:11:14 +00:00
|
|
|
|
2021-05-06 15:25:55 +00:00
|
|
|
if (!all_children_visited)
|
2021-05-05 15:39:26 +00:00
|
|
|
continue;
|
2020-11-05 16:11:14 +00:00
|
|
|
|
2021-05-25 08:50:40 +00:00
|
|
|
/// Here we process only functions that are not compiled constants
|
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
CompileDAG::Node compile_node;
|
|
|
|
compile_node.function = node->function_base;
|
|
|
|
compile_node.result_type = node->result_type;
|
2021-05-25 08:50:40 +00:00
|
|
|
compile_node.type = CompileDAG::CompileType::FUNCTION;
|
2020-11-05 16:11:14 +00:00
|
|
|
|
2021-05-25 08:50:40 +00:00
|
|
|
for (const auto * child : node->children)
|
|
|
|
compile_node.arguments.push_back(visited_node_to_compile_dag_position[child]);
|
2021-05-05 15:39:26 +00:00
|
|
|
|
|
|
|
visited_node_to_compile_dag_position[node] = dag.getNodesCount();
|
|
|
|
|
|
|
|
dag.addNode(std::move(compile_node));
|
|
|
|
stack.pop();
|
2020-11-05 16:11:14 +00:00
|
|
|
}
|
|
|
|
|
2021-05-04 20:32:43 +00:00
|
|
|
return dag;
|
2020-11-05 16:11:14 +00:00
|
|
|
}
|
|
|
|
|
2021-08-10 11:31:15 +00:00
|
|
|
void ActionsDAG::compileFunctions(size_t min_count_to_compile_expression, const std::unordered_set<const ActionsDAG::Node *> & lazy_executed_nodes)
|
2020-11-05 16:11:14 +00:00
|
|
|
{
|
|
|
|
struct Data
|
|
|
|
{
|
2021-05-05 15:39:26 +00:00
|
|
|
bool is_compilable_in_isolation = false;
|
2020-11-05 20:36:37 +00:00
|
|
|
bool all_parents_compilable = true;
|
2021-05-24 08:59:27 +00:00
|
|
|
size_t compilable_children_size = 0;
|
2021-05-05 15:39:26 +00:00
|
|
|
size_t children_size = 0;
|
2020-11-05 16:11:14 +00:00
|
|
|
};
|
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
std::unordered_map<const Node *, Data> node_to_data;
|
2020-11-05 16:11:14 +00:00
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
/// Check which nodes can be compiled in isolation
|
2020-11-05 16:11:14 +00:00
|
|
|
|
|
|
|
for (const auto & node : nodes)
|
2021-05-05 15:39:26 +00:00
|
|
|
{
|
2021-08-10 11:31:15 +00:00
|
|
|
bool node_is_compilable_in_isolation = isCompilableFunction(node, lazy_executed_nodes) && !isCompilableConstant(node);
|
2021-05-05 15:39:26 +00:00
|
|
|
node_to_data[&node].is_compilable_in_isolation = node_is_compilable_in_isolation;
|
|
|
|
}
|
2020-11-05 16:11:14 +00:00
|
|
|
|
|
|
|
struct Frame
|
|
|
|
{
|
2021-03-03 20:01:07 +00:00
|
|
|
const Node * node;
|
2020-11-05 16:11:14 +00:00
|
|
|
size_t next_child_to_visit = 0;
|
|
|
|
};
|
|
|
|
|
|
|
|
std::stack<Frame> stack;
|
2021-05-05 15:39:26 +00:00
|
|
|
std::unordered_set<const Node *> visited_nodes;
|
|
|
|
|
2021-05-25 08:50:40 +00:00
|
|
|
/** Algorithm is to iterate over each node in ActionsDAG, and update node compilable_children_size.
|
2021-05-05 20:08:31 +00:00
|
|
|
* After this procedure data for each node is initialized.
|
2021-05-05 15:39:26 +00:00
|
|
|
*/
|
2020-11-05 16:11:14 +00:00
|
|
|
|
|
|
|
for (auto & node : nodes)
|
|
|
|
{
|
2021-05-05 15:39:26 +00:00
|
|
|
if (visited_nodes.contains(&node))
|
2020-11-05 16:11:14 +00:00
|
|
|
continue;
|
|
|
|
|
|
|
|
stack.emplace(Frame{.node = &node});
|
2021-05-05 15:39:26 +00:00
|
|
|
|
2020-11-05 16:11:14 +00:00
|
|
|
while (!stack.empty())
|
|
|
|
{
|
2021-05-05 15:39:26 +00:00
|
|
|
auto & current_frame = stack.top();
|
|
|
|
auto & current_node = current_frame.node;
|
2020-11-05 16:11:14 +00:00
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
while (current_frame.next_child_to_visit < current_node->children.size())
|
2020-11-05 16:11:14 +00:00
|
|
|
{
|
2021-05-05 15:39:26 +00:00
|
|
|
const auto & child = node.children[current_frame.next_child_to_visit];
|
2020-11-05 16:11:14 +00:00
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
if (visited_nodes.contains(child))
|
2020-11-05 18:08:23 +00:00
|
|
|
{
|
2021-05-05 15:39:26 +00:00
|
|
|
++current_frame.next_child_to_visit;
|
|
|
|
continue;
|
2020-11-05 18:08:23 +00:00
|
|
|
}
|
2021-05-05 15:39:26 +00:00
|
|
|
|
|
|
|
stack.emplace(Frame{.node=child});
|
|
|
|
break;
|
2020-11-05 16:11:14 +00:00
|
|
|
}
|
|
|
|
|
2021-05-05 20:08:31 +00:00
|
|
|
bool all_children_visited = current_frame.next_child_to_visit == current_node->children.size();
|
2021-05-05 15:39:26 +00:00
|
|
|
|
2021-05-05 20:08:31 +00:00
|
|
|
if (!all_children_visited)
|
2021-05-05 15:39:26 +00:00
|
|
|
continue;
|
|
|
|
|
|
|
|
auto & current_node_data = node_to_data[current_node];
|
|
|
|
|
|
|
|
if (current_node_data.is_compilable_in_isolation)
|
2020-11-05 16:11:14 +00:00
|
|
|
{
|
2021-05-05 15:39:26 +00:00
|
|
|
for (const auto * child : current_node->children)
|
2020-11-05 16:11:14 +00:00
|
|
|
{
|
2021-05-24 08:59:27 +00:00
|
|
|
auto & child_data = node_to_data[child];
|
|
|
|
|
|
|
|
if (child_data.is_compilable_in_isolation)
|
|
|
|
{
|
|
|
|
current_node_data.compilable_children_size += child_data.compilable_children_size;
|
|
|
|
current_node_data.compilable_children_size += 1;
|
|
|
|
}
|
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
current_node_data.children_size += node_to_data[child].children_size;
|
2020-11-05 16:11:14 +00:00
|
|
|
}
|
2020-11-05 16:29:03 +00:00
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
current_node_data.children_size += current_node->children.size();
|
2020-11-05 16:11:14 +00:00
|
|
|
}
|
2021-05-05 15:39:26 +00:00
|
|
|
|
|
|
|
visited_nodes.insert(current_node);
|
|
|
|
stack.pop();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
for (const auto & node : nodes)
|
|
|
|
{
|
|
|
|
auto & node_data = node_to_data[&node];
|
2021-05-24 08:59:27 +00:00
|
|
|
bool node_is_valid_for_compilation = node_data.is_compilable_in_isolation && node_data.compilable_children_size > 0;
|
2021-05-05 15:39:26 +00:00
|
|
|
|
|
|
|
for (const auto & child : node.children)
|
2021-05-24 08:59:27 +00:00
|
|
|
node_to_data[child].all_parents_compilable &= node_is_valid_for_compilation;
|
2021-05-05 15:39:26 +00:00
|
|
|
}
|
|
|
|
|
2022-08-08 15:54:51 +00:00
|
|
|
for (const auto & output_node : outputs)
|
2021-05-05 15:39:26 +00:00
|
|
|
{
|
2022-08-08 15:54:51 +00:00
|
|
|
/// Force output nodes to compile
|
|
|
|
node_to_data[output_node].all_parents_compilable = false;
|
2021-05-05 15:39:26 +00:00
|
|
|
}
|
|
|
|
|
2021-05-07 18:36:07 +00:00
|
|
|
std::vector<Node *> nodes_to_compile;
|
|
|
|
|
2021-05-05 15:39:26 +00:00
|
|
|
for (auto & node : nodes)
|
|
|
|
{
|
|
|
|
auto & node_data = node_to_data[&node];
|
|
|
|
|
2021-05-24 08:59:27 +00:00
|
|
|
bool node_is_valid_for_compilation = node_data.is_compilable_in_isolation && node_data.compilable_children_size > 0;
|
2021-05-05 15:39:26 +00:00
|
|
|
|
|
|
|
/// If all parents are compilable then this node should not be standalone compiled
|
2021-05-24 08:59:27 +00:00
|
|
|
bool should_compile = node_is_valid_for_compilation && !node_data.all_parents_compilable;
|
2021-05-05 15:39:26 +00:00
|
|
|
|
|
|
|
if (!should_compile)
|
|
|
|
continue;
|
|
|
|
|
2021-05-07 18:36:07 +00:00
|
|
|
nodes_to_compile.emplace_back(&node);
|
|
|
|
}
|
|
|
|
|
|
|
|
/** Sort nodes before compilation using their children size to avoid compiling subexpression before compile parent expression.
|
|
|
|
* This is needed to avoid compiling expression more than once with different names because of compilation order.
|
|
|
|
*/
|
2022-01-30 19:49:48 +00:00
|
|
|
::sort(nodes_to_compile.begin(), nodes_to_compile.end(), [&](const Node * lhs, const Node * rhs)
|
|
|
|
{
|
|
|
|
return node_to_data[lhs].children_size > node_to_data[rhs].children_size;
|
|
|
|
});
|
2021-05-07 18:36:07 +00:00
|
|
|
|
|
|
|
for (auto & node : nodes_to_compile)
|
|
|
|
{
|
2021-05-05 15:39:26 +00:00
|
|
|
NodeRawConstPtrs new_children;
|
2021-09-08 19:23:57 +00:00
|
|
|
auto dag = getCompilableDAG(node, new_children, lazy_executed_nodes);
|
2021-05-05 15:39:26 +00:00
|
|
|
|
|
|
|
if (dag.getInputNodesCount() == 0)
|
|
|
|
continue;
|
|
|
|
|
|
|
|
if (auto fn = compile(dag, min_count_to_compile_expression))
|
|
|
|
{
|
|
|
|
ColumnsWithTypeAndName arguments;
|
|
|
|
arguments.reserve(new_children.size());
|
|
|
|
for (const auto * child : new_children)
|
|
|
|
arguments.emplace_back(child->column, child->result_type, child->result_name);
|
|
|
|
|
2021-05-07 18:36:07 +00:00
|
|
|
node->type = ActionsDAG::ActionType::FUNCTION;
|
|
|
|
node->function_base = fn;
|
|
|
|
node->function = fn->prepare(arguments);
|
|
|
|
node->children.swap(new_children);
|
|
|
|
node->is_function_compiled = true;
|
|
|
|
node->column = nullptr;
|
2020-11-05 16:11:14 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
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
|