mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
CLICKHOUSE-3800: Improve cache and turn on compilation by default
This commit is contained in:
parent
cc26e1ec5e
commit
e815260363
@ -66,6 +66,21 @@ static inline llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const IDa
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
static inline bool canBeNativeType(const IDataType & type)
|
||||
{
|
||||
if (auto * nullable = typeid_cast<const DataTypeNullable *>(&type))
|
||||
return canBeNativeType(*nullable->getNestedType());
|
||||
|
||||
return typeIsEither<DataTypeInt8, DataTypeUInt8>(type)
|
||||
|| typeIsEither<DataTypeInt16, DataTypeUInt16, DataTypeDate>(type)
|
||||
|| typeIsEither<DataTypeInt32, DataTypeUInt32, DataTypeDateTime>(type)
|
||||
|| typeIsEither<DataTypeInt64, DataTypeUInt64, DataTypeInterval>(type)
|
||||
|| typeIsEither<DataTypeUUID>(type)
|
||||
|| typeIsEither<DataTypeFloat32>(type)
|
||||
|| typeIsEither<DataTypeFloat64>(type)
|
||||
|| typeid_cast<const DataTypeFixedString *>(&type);
|
||||
}
|
||||
|
||||
static inline llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const DataTypePtr & type)
|
||||
{
|
||||
return toNativeType(builder, *type);
|
||||
|
@ -803,7 +803,7 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
/// This has to be done before removing redundant actions and inserting REMOVE_COLUMNs
|
||||
/// because inlining may change dependency sets.
|
||||
if (settings.compile_expressions)
|
||||
compileFunctions(actions, output_columns, sample_block, compilation_cache);
|
||||
compileFunctions(actions, output_columns, sample_block, compilation_cache, settings.min_count_to_compile);
|
||||
#endif
|
||||
|
||||
/// Which columns are needed to perform actions from the current to the last.
|
||||
@ -1116,7 +1116,7 @@ BlockInputStreamPtr ExpressionActions::createStreamWithNonJoinedDataIfFullOrRigh
|
||||
|
||||
|
||||
/// It is not important to calculate the hash of individual strings or their concatenation
|
||||
size_t ExpressionAction::ActionHash::operator()(const ExpressionAction & action) const
|
||||
UInt128 ExpressionAction::ActionHash::operator()(const ExpressionAction & action) const
|
||||
{
|
||||
SipHash hash;
|
||||
hash.update(action.type);
|
||||
@ -1169,7 +1169,9 @@ size_t ExpressionAction::ActionHash::operator()(const ExpressionAction & action)
|
||||
case ADD_ALIASES:
|
||||
break;
|
||||
}
|
||||
return hash.get64();
|
||||
UInt128 result;
|
||||
hash.get128(result.low, result.high);
|
||||
return result;
|
||||
}
|
||||
|
||||
bool ExpressionAction::operator==(const ExpressionAction & other) const
|
||||
|
@ -126,7 +126,7 @@ public:
|
||||
|
||||
struct ActionHash
|
||||
{
|
||||
size_t operator()(const ExpressionAction & action) const;
|
||||
UInt128 operator()(const ExpressionAction & action) const;
|
||||
};
|
||||
|
||||
private:
|
||||
@ -230,6 +230,20 @@ public:
|
||||
|
||||
const Settings & getSettings() const { return settings; }
|
||||
|
||||
|
||||
struct ActionsHash
|
||||
{
|
||||
UInt128 operator()(const ExpressionActions::Actions & actions) const
|
||||
{
|
||||
SipHash hash;
|
||||
for (const ExpressionAction & act : actions)
|
||||
hash.update(ExpressionAction::ActionHash{}(act));
|
||||
UInt128 result;
|
||||
hash.get128(result.low, result.high);
|
||||
return result;
|
||||
}
|
||||
};
|
||||
|
||||
private:
|
||||
NamesAndTypesList input_columns;
|
||||
Actions actions;
|
||||
@ -249,18 +263,6 @@ private:
|
||||
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
struct ActionsHash
|
||||
{
|
||||
size_t operator()(const ExpressionActions::Actions & actions) const
|
||||
{
|
||||
SipHash hash;
|
||||
for (const ExpressionAction & act : actions)
|
||||
hash.update(ExpressionAction::ActionHash{}(act));
|
||||
return hash.get64();
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
||||
/** The sequence of transformations over the block.
|
||||
* It is assumed that the result of each step is fed to the input of the next step.
|
||||
|
@ -566,12 +566,12 @@ LLVMFunction::Monotonicity LLVMFunction::getMonotonicityForRange(const IDataType
|
||||
}
|
||||
|
||||
|
||||
static bool isCompilable(llvm::IRBuilderBase & builder, const IFunctionBase & function)
|
||||
static bool isCompilable(const IFunctionBase & function)
|
||||
{
|
||||
if (!toNativeType(builder, function.getReturnType()))
|
||||
if (!canBeNativeType(*function.getReturnType()))
|
||||
return false;
|
||||
for (const auto & type : function.getArgumentTypes())
|
||||
if (!toNativeType(builder, type))
|
||||
if (!canBeNativeType(*type))
|
||||
return false;
|
||||
return function.isCompilable();
|
||||
}
|
||||
@ -598,21 +598,8 @@ size_t CompiledExpressionCache::weight() const
|
||||
#endif
|
||||
}
|
||||
|
||||
void compileFunctions(ExpressionActions::Actions & actions, const Names & output_columns, const Block & sample_block, std::shared_ptr<CompiledExpressionCache> compilation_cache)
|
||||
std::vector<std::unordered_set<std::optional<size_t>>> getActionsDependents(const ExpressionActions::Actions & actions, const Names & output_columns)
|
||||
{
|
||||
struct LLVMTargetInitializer
|
||||
{
|
||||
LLVMTargetInitializer()
|
||||
{
|
||||
llvm::InitializeNativeTarget();
|
||||
llvm::InitializeNativeTargetAsmPrinter();
|
||||
llvm::sys::DynamicLibrary::LoadLibraryPermanently(nullptr);
|
||||
}
|
||||
};
|
||||
|
||||
static LLVMTargetInitializer initializer;
|
||||
|
||||
auto context = std::make_shared<LLVMContext>();
|
||||
/// 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;
|
||||
@ -656,7 +643,7 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output
|
||||
case ExpressionAction::APPLY_FUNCTION:
|
||||
{
|
||||
dependents[i] = current_dependents[actions[i].result_name];
|
||||
const bool compilable = isCompilable(context->builder, *actions[i].function);
|
||||
const bool compilable = isCompilable(*actions[i].function);
|
||||
for (const auto & name : actions[i].argument_names)
|
||||
{
|
||||
if (compilable)
|
||||
@ -668,11 +655,34 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output
|
||||
}
|
||||
}
|
||||
}
|
||||
return dependents;
|
||||
}
|
||||
|
||||
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)
|
||||
{
|
||||
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);
|
||||
/// Initialize context as late as possible and only if needed
|
||||
std::shared_ptr<LLVMContext> context;
|
||||
std::vector<ExpressionActions::Actions> fused(actions.size());
|
||||
for (size_t i = 0; i < actions.size(); ++i)
|
||||
{
|
||||
if (actions[i].type != ExpressionAction::APPLY_FUNCTION || !isCompilable(context->builder, *actions[i].function))
|
||||
if (actions[i].type != ExpressionAction::APPLY_FUNCTION || !isCompilable(*actions[i].function))
|
||||
continue;
|
||||
|
||||
fused[i].push_back(actions[i]);
|
||||
@ -682,18 +692,31 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output
|
||||
if (fused[i].size() == 1)
|
||||
continue;
|
||||
|
||||
|
||||
auto hash_key = ExpressionActions::ActionsHash{}(fused[i]);
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (counter[hash_key]++ < min_count_to_compile)
|
||||
continue;
|
||||
}
|
||||
std::shared_ptr<LLVMFunction> fn;
|
||||
if (compilation_cache)
|
||||
{
|
||||
bool success;
|
||||
auto set_func = [&fused, i, context, &sample_block] () { return std::make_shared<LLVMFunction>(fused[i], context, sample_block); };
|
||||
Stopwatch watch;
|
||||
std::tie(fn, success) = compilation_cache->getOrSet(fused[i], set_func);
|
||||
if (success)
|
||||
fn = compilation_cache->get(hash_key);
|
||||
if (!fn)
|
||||
{
|
||||
if (!context)
|
||||
context = std::make_shared<LLVMContext>();
|
||||
Stopwatch watch;
|
||||
fn = std::make_shared<LLVMFunction>(fused[i], context, sample_block);
|
||||
ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds());
|
||||
compilation_cache->set(hash_key, fn);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!context)
|
||||
context = std::make_shared<LLVMContext>();
|
||||
Stopwatch watch;
|
||||
fn = std::make_shared<LLVMFunction>(fused[i], context, sample_block);
|
||||
ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds());
|
||||
@ -711,7 +734,8 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output
|
||||
fused[*dep].insert(fused[*dep].end(), fused[i].begin(), fused[i].end());
|
||||
}
|
||||
|
||||
context->finalize();
|
||||
if (context)
|
||||
context->finalize();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -61,10 +61,11 @@ public:
|
||||
/** This child of LRUCache breaks one of it's invariants: total weight may be changed after insertion.
|
||||
* We have to do so, because we don't known real memory consumption of generated LLVM code for every function.
|
||||
*/
|
||||
class CompiledExpressionCache : public LRUCache<std::vector<ExpressionAction>, LLVMFunction, ActionsHash>
|
||||
class CompiledExpressionCache : public LRUCache<UInt128, LLVMFunction, UInt128Hash>
|
||||
{
|
||||
private:
|
||||
using Base = LRUCache<std::vector<ExpressionAction>, LLVMFunction, ActionsHash>;
|
||||
using Base = LRUCache<UInt128, LLVMFunction, UInt128Hash>;
|
||||
|
||||
public:
|
||||
using Base::Base;
|
||||
|
||||
@ -73,7 +74,7 @@ public:
|
||||
|
||||
/// For each APPLY_FUNCTION action, try to compile the function to native code; if the only uses of a compilable
|
||||
/// function's result are as arguments to other compilable functions, inline it and leave the now-redundant action as-is.
|
||||
void compileFunctions(ExpressionActions::Actions & actions, const Names & output_columns, const Block & sample_block, std::shared_ptr<CompiledExpressionCache> compilation_cache);
|
||||
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);
|
||||
|
||||
}
|
||||
|
||||
|
@ -74,7 +74,7 @@ struct Settings
|
||||
M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.") \
|
||||
\
|
||||
M(SettingBool, compile, false, "Whether query compilation is enabled.") \
|
||||
M(SettingBool, compile_expressions, false, "Compile some scalar functions and operators to native code.") \
|
||||
M(SettingBool, compile_expressions, true, "Compile some scalar functions and operators to native code.") \
|
||||
M(SettingUInt64, min_count_to_compile, 3, "The number of structurally identical queries before they are compiled.") \
|
||||
M(SettingUInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.") \
|
||||
M(SettingUInt64, group_by_two_level_threshold_bytes, 100000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.") \
|
||||
|
@ -30,6 +30,8 @@
|
||||
x3 * 0xc4ceb9fe1a85ec53 AS x4,
|
||||
bitXor(x4, bitShiftRight(x4, 33)) AS x5
|
||||
SELECT x5, intHash64(number) FROM system.numbers LIMIT 10
|
||||
SETTINGS
|
||||
compile_expressions = 0
|
||||
</query>
|
||||
<query>
|
||||
WITH
|
||||
@ -41,7 +43,6 @@
|
||||
bitXor(x4, bitShiftRight(x4, 33)) AS x5
|
||||
SELECT x5, intHash64(number) FROM system.numbers LIMIT 10
|
||||
SETTINGS
|
||||
compile = 1,
|
||||
compile_expressions = 1
|
||||
</query>
|
||||
</test>
|
||||
|
Loading…
Reference in New Issue
Block a user