Fixed tests

This commit is contained in:
Maksim Kita 2022-05-11 15:35:37 +02:00
parent 496302cb1d
commit 437d70d4da
11 changed files with 27 additions and 24 deletions

View File

@ -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);

View File

@ -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;
}
}

View File

@ -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);
}

View File

@ -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;

View File

@ -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);

View File

@ -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)

View File

@ -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()

View File

@ -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

View File

@ -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);
}
}

View File

@ -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)
{

View File

@ -119,4 +119,4 @@
<drop_query>DROP TABLE IF EXISTS {table}</drop_query>
</test>
</test>