ClickHouse/src/Interpreters/ExpressionJIT.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

613 lines
19 KiB
C++
Raw Normal View History

#include "config.h"
#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>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnVector.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionsComparison.h>
#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>
#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
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
2021-05-04 20:32:43 +00:00
static CHJIT & getJITInstance()
{
2021-05-04 20:32:43 +00:00
static CHJIT jit;
return jit;
}
2021-05-05 15:39:26 +00:00
static Poco::Logger * getLogger()
{
static Poco::Logger & logger = Poco::Logger::get("ExpressionJIT");
return &logger;
}
class CompiledFunctionHolder : public CompiledExpressionCacheEntry
{
public:
explicit CompiledFunctionHolder(CompiledFunction compiled_function_)
: CompiledExpressionCacheEntry(compiled_function_.compiled_module.size)
, compiled_function(compiled_function_)
{}
~CompiledFunctionHolder() override
{
getJITInstance().deleteCompiledModule(compiled_function.compiled_module);
}
CompiledFunction compiled_function;
};
2021-05-15 17:33:15 +00:00
class LLVMExecutableFunction : public IExecutableFunction
{
public:
2021-05-07 18:36:07 +00:00
explicit LLVMExecutableFunction(const std::string & name_, std::shared_ptr<CompiledFunctionHolder> compiled_function_holder_)
: name(name_)
, compiled_function_holder(compiled_function_holder_)
{
}
String getName() const override { return name; }
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
{
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)
{
2021-05-02 18:15:34 +00:00
result_column = result_column->cloneResized(input_rows_count);
std::vector<ColumnData> columns(arguments.size() + 1);
std::vector<ColumnPtr> columns_backup;
2018-05-06 10:42:35 +00:00
for (size_t i = 0; i < arguments.size(); ++i)
{
auto column = arguments[i].column->convertToFullColumnIfConst();
columns_backup.emplace_back(column);
columns[i] = getColumnData(column.get());
}
2021-05-02 18:15:34 +00:00
columns[arguments.size()] = getColumnData(result_column.get());
auto jit_compiled_function = compiled_function_holder->compiled_function.compiled_function;
jit_compiled_function(input_rows_count, columns.data());
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();
__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();
__msan_unpoison(null_map_column_raw_data.data(), null_map_column_raw_data.size());
2021-05-02 18:15:34 +00:00
}
else
{
__msan_unpoison(result_column->getRawData().data(), result_column->getRawData().size());
2021-05-02 18:15:34 +00:00
}
#endif
}
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
private:
std::string name;
std::shared_ptr<CompiledFunctionHolder> compiled_function_holder;
};
2021-05-15 17:33:15 +00:00
class LLVMFunction : public IFunctionBase
{
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);
}
}
void setCompiledFunction(std::shared_ptr<CompiledFunctionHolder> compiled_function_holder_)
{
compiled_function_holder = compiled_function_holder_;
}
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
{
2023-06-03 18:31:08 +00:00
return dag.compile(builder, arguments).value;
2021-05-04 20:32:43 +00:00
}
2021-06-22 16:21:23 +00:00
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & arguments) const override
{
for (const auto & f : nested_functions)
2021-06-22 16:21:23 +00:00
if (!f->isSuitableForShortCircuitArgumentsExecution(arguments))
return false;
2021-06-22 16:21:23 +00:00
return true;
}
2021-05-14 14:07:24 +00:00
2021-05-04 20:32:43 +00:00
String getName() const override { return name; }
2021-05-04 20:32:43 +00:00
const DataTypes & getArgumentTypes() const override { return argument_types; }
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
{
if (!compiled_function_holder)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Compiled function was not initialized {}", name);
2021-05-05 15:39:26 +00:00
return std::make_unique<LLVMExecutableFunction>(name, compiled_function_holder);
}
2021-05-04 20:32:43 +00:00
bool isDeterministic() const override
{
for (const auto & f : nested_functions)
if (!f->isDeterministic())
return false;
2021-05-04 20:32:43 +00:00
return true;
}
2021-05-04 20:32:43 +00:00
bool isDeterministicInScopeOfQuery() const override
{
2021-05-04 20:32:43 +00:00
for (const auto & f : nested_functions)
if (!f->isDeterministicInScopeOfQuery())
return false;
2021-05-04 20:32:43 +00:00
return true;
}
2021-05-04 20:32:43 +00:00
bool isSuitableForConstantFolding() const override
{
2021-05-04 20:32:43 +00:00
for (const auto & f : nested_functions)
if (!f->isSuitableForConstantFolding())
return false;
2021-05-04 20:32:43 +00:00
return true;
}
2021-05-04 20:32:43 +00:00
bool isInjective(const ColumnsWithTypeAndName & sample_block) const override
{
2021-05-04 20:32:43 +00:00
for (const auto & f : nested_functions)
if (!f->isInjective(sample_block))
return false;
2021-05-04 20:32:43 +00:00
return true;
}
2021-05-04 20:32:43 +00:00
bool hasInformationAboutMonotonicity() const override
{
for (const auto & f : nested_functions)
if (!f->hasInformationAboutMonotonicity())
return false;
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
{
2021-05-04 20:32:43 +00:00
const IDataType * type_ptr = &type;
Field left_mut = left;
Field right_mut = right;
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())
{
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();
}
}
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-04 20:32:43 +00:00
private:
std::string name;
CompileDAG dag;
DataTypes argument_types;
std::vector<FunctionBasePtr> nested_functions;
std::shared_ptr<CompiledFunctionHolder> compiled_function_holder;
2021-05-04 20:32:43 +00:00
};
2021-05-04 20:32:43 +00:00
static FunctionBasePtr compile(
const CompileDAG & dag,
size_t min_count_to_compile_expression)
{
2021-05-04 20:32:43 +00:00
static std::unordered_map<UInt128, UInt64, UInt128Hash> counter;
static std::mutex mutex;
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;
}
auto llvm_function = std::make_shared<LLVMFunction>(dag);
2021-05-04 20:32:43 +00:00
if (auto * compilation_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
{
auto [compiled_function_cache_entry, _] = compilation_cache->getOrSet(hash_key, [&] ()
{
LOG_TRACE(getLogger(), "Compile expression {}", llvm_function->getName());
auto compiled_function = compileFunction(getJITInstance(), *llvm_function);
return std::make_shared<CompiledFunctionHolder>(compiled_function);
2021-05-04 20:32:43 +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
{
auto compiled_function = compileFunction(getJITInstance(), *llvm_function);
auto compiled_function_holder = std::make_shared<CompiledFunctionHolder>(compiled_function);
llvm_function->setCompiledFunction(std::move(compiled_function_holder));
}
return llvm_function;
2021-05-04 20:32:43 +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
{
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)
{
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);
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,
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);
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
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;
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
}
}
}
}
#endif