mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Fixed tests
This commit is contained in:
parent
496302cb1d
commit
437d70d4da
@ -12,7 +12,6 @@
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
#include <DataTypes/Native.h>
|
||||
|
||||
#include <llvm/IR/IRBuilder.h>
|
||||
#endif
|
||||
|
||||
@ -254,7 +253,8 @@ bool ColumnNullable::isComparatorCompilable() const
|
||||
return nested_column->isComparatorCompilable();
|
||||
}
|
||||
|
||||
llvm::Value * ColumnNullable::compileComparator(llvm::IRBuilderBase & builder, llvm::Value * lhs, llvm::Value * rhs, llvm::Value * nan_direction_hint) const
|
||||
llvm::Value * ColumnNullable::compileComparator(llvm::IRBuilderBase & builder, llvm::Value * lhs, llvm::Value * rhs,
|
||||
llvm::Value * nan_direction_hint) const
|
||||
{
|
||||
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
|
||||
auto * head = b.GetInsertBlock();
|
||||
@ -280,6 +280,7 @@ llvm::Value * ColumnNullable::compileComparator(llvm::IRBuilderBase & builder, l
|
||||
// else
|
||||
// return lval_is_null ? null_direction_hint : -null_direction_hint;
|
||||
// }
|
||||
|
||||
b.SetInsertPoint(lhs_or_rhs_are_null_block);
|
||||
auto * lhs_equals_rhs_result = llvm::ConstantInt::getSigned(b.getInt8Ty(), 0);
|
||||
llvm::Value * lhs_and_rhs_are_null = b.CreateAnd(lhs_is_null_value, rhs_is_null_value);
|
||||
@ -288,8 +289,10 @@ llvm::Value * ColumnNullable::compileComparator(llvm::IRBuilderBase & builder, l
|
||||
b.CreateBr(join_block);
|
||||
|
||||
// getNestedColumn().compareAt(n, m, nested_rhs, null_direction_hint);
|
||||
|
||||
b.SetInsertPoint(lhs_rhs_are_not_null_block);
|
||||
llvm::Value *lhs_rhs_are_not_null_block_result = nested_column->compileComparator(builder, lhs_unwrapped_value, rhs_unwrapped_value, nan_direction_hint);
|
||||
llvm::Value * lhs_rhs_are_not_null_block_result
|
||||
= nested_column->compileComparator(builder, lhs_unwrapped_value, rhs_unwrapped_value, nan_direction_hint);
|
||||
b.CreateBr(join_block);
|
||||
|
||||
b.SetInsertPoint(join_block);
|
||||
|
@ -27,7 +27,6 @@
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
#include <DataTypes/Native.h>
|
||||
|
||||
#include <llvm/IR/IRBuilder.h>
|
||||
#endif
|
||||
|
||||
@ -204,7 +203,7 @@ llvm::Value * ColumnVector<T>::compileComparator(llvm::IRBuilderBase & builder,
|
||||
{
|
||||
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
|
||||
|
||||
if constexpr (std::is_arithmetic_v<T>)
|
||||
if constexpr (std::is_integral_v<T>)
|
||||
{
|
||||
// a > b ? 1 : (a < b ? -1 : 0);
|
||||
|
||||
@ -223,7 +222,6 @@ llvm::Value * ColumnVector<T>::compileComparator(llvm::IRBuilderBase & builder,
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Method compileComparator is not supported for type {}", TypeName<T>);
|
||||
return nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -292,9 +292,9 @@ public:
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
virtual bool isComparatorCompilable() const { return false; }
|
||||
[[nodiscard]] virtual bool isComparatorCompilable() const { return false; }
|
||||
|
||||
virtual llvm::Value * compileComparator(llvm::IRBuilderBase & /*builder*/, llvm::Value * /*lhs*/, llvm::Value * /*rhs*/, llvm::Value * /*nan_direction_hint*/) const
|
||||
[[nodiscard]] virtual llvm::Value * compileComparator(llvm::IRBuilderBase & /*builder*/, llvm::Value * /*lhs*/, llvm::Value * /*rhs*/, llvm::Value * /*nan_direction_hint*/) const
|
||||
{
|
||||
throw Exception("Method compileComparator is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
@ -3,9 +3,12 @@
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
#include <DataTypes/Native.h>
|
||||
#include <Interpreters/JIT/compileFunction.h>
|
||||
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -63,8 +66,6 @@ public:
|
||||
CompiledSortDescriptionFunction compiled_sort_description_function;
|
||||
};
|
||||
|
||||
static std::unordered_map<std::string, CompiledSortDescriptionFunction> sort_description_to_compiled_function;
|
||||
|
||||
static std::string getSortDescriptionDump(const SortDescription & description, const DataTypes & header_types)
|
||||
{
|
||||
WriteBufferFromOwnString buffer;
|
||||
|
@ -105,8 +105,7 @@ public:
|
||||
};
|
||||
|
||||
/** Compile sort description for header_types.
|
||||
* Description is compiled for header only if compile_sort_description = true and attemps to compile sort descriptions
|
||||
* with unique combination of description and header will be greater or equal min_count_to_compile_sort_description.
|
||||
* Description is compiled only if compilation attempts to compile identical description is more than min_count_to_compile_sort_description.
|
||||
*/
|
||||
void compileSortDescriptionIfNeeded(SortDescription & description, const DataTypes & sort_description_types, bool increase_compile_attemps);
|
||||
|
||||
|
@ -111,7 +111,8 @@ static inline bool canBeNativeType(const IDataType & type)
|
||||
return canBeNativeType(*data_type_nullable.getNestedType());
|
||||
}
|
||||
|
||||
return data_type.isNativeInt() || data_type.isNativeUInt() || data_type.isFloat() || data_type.isDate() || data_type.isDateTime() || data_type.isEnum();
|
||||
return data_type.isNativeInt() || data_type.isNativeUInt() || data_type.isFloat() || data_type.isDate()
|
||||
|| data_type.isDate32() || data_type.isDateTime() || data_type.isEnum();
|
||||
}
|
||||
|
||||
static inline llvm::Type * toNativeType(llvm::IRBuilderBase & builder, const DataTypePtr & type)
|
||||
|
@ -38,7 +38,7 @@ MergingSortedAlgorithm::MergingSortedAlgorithm(
|
||||
sort_description_types.emplace_back(header.getByName(column_description.column_name).type);
|
||||
}
|
||||
|
||||
compileSortDescriptionIfNeeded(description, sort_description_types, true /*increase_unique_count*/);
|
||||
compileSortDescriptionIfNeeded(description, sort_description_types, true /*increase_compile_attemps*/);
|
||||
}
|
||||
|
||||
void MergingSortedAlgorithm::addInput()
|
||||
|
@ -197,12 +197,12 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build
|
||||
increase_sort_description_compile_attempts = false;
|
||||
|
||||
return std::make_shared<MergeSortingTransform>(
|
||||
header, result_description, max_block_size, limit, increase_sort_description_compile_attempts_current,
|
||||
max_bytes_before_remerge / pipeline.getNumStreams(),
|
||||
remerge_lowered_memory_bytes_ratio,
|
||||
max_bytes_before_external_sort,
|
||||
tmp_volume,
|
||||
min_free_disk_space);
|
||||
header, result_description, max_block_size, limit, increase_sort_description_compile_attempts_current,
|
||||
max_bytes_before_remerge / pipeline.getNumStreams(),
|
||||
remerge_lowered_memory_bytes_ratio,
|
||||
max_bytes_before_external_sort,
|
||||
tmp_volume,
|
||||
min_free_disk_space);
|
||||
});
|
||||
|
||||
/// If there are several streams, then we merge them into one
|
||||
|
@ -168,8 +168,7 @@ SortingTransform::SortingTransform(
|
||||
|
||||
if (new_pos < num_columns)
|
||||
{
|
||||
auto type = sample.safeGetByPosition(old_pos).type;
|
||||
sort_description_types.emplace_back(type);
|
||||
sort_description_types.emplace_back(sample.safeGetByPosition(old_pos).type);
|
||||
description_without_constants.push_back(column_description);
|
||||
}
|
||||
}
|
||||
|
@ -778,6 +778,9 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
|
||||
|
||||
Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns();
|
||||
SortDescription sort_description;
|
||||
sort_description.compile_sort_description = global_ctx->data->getContext()->getSettingsRef().compile_sort_description;
|
||||
sort_description.min_count_to_compile_sort_description = global_ctx->data->getContext()->getSettingsRef().min_count_to_compile_sort_description;
|
||||
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
|
||||
@ -796,7 +799,6 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
|
||||
ctx->blocks_are_granules_size = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical);
|
||||
|
||||
UInt64 merge_block_size = data_settings->merge_max_block_size;
|
||||
sort_description.compile_sort_description = global_ctx->data->getContext()->getSettingsRef().compile_sort_description;
|
||||
|
||||
switch (ctx->merging_params.mode)
|
||||
{
|
||||
|
@ -119,4 +119,4 @@
|
||||
|
||||
<drop_query>DROP TABLE IF EXISTS {table}</drop_query>
|
||||
|
||||
</test>
|
||||
</test>
|
||||
|
Loading…
Reference in New Issue
Block a user