From ff3cc1f6f771f19e4dff37aca59532cf808b0260 Mon Sep 17 00:00:00 2001 From: Zhengguo Yang Date: Tue, 18 May 2021 17:47:27 +0800 Subject: [PATCH] support serialize and deserialize date/datetime (#42) --- be/src/runtime/types.h | 4 +- be/src/service/bench.cpp | 323 ---- be/src/vec/CMakeLists.txt | 4 +- .../aggregate_function_avg.h | 75 +- .../aggregate_function_count.h | 2 +- .../aggregate_function_hll_union_agg.h | 40 +- .../aggregate_function_sum.h | 4 +- .../aggregate_function_uniq.h | 2 +- be/src/vec/core/block.cpp | 16 +- be/src/vec/core/decimal_comparison.h | 2 +- ...ypes_decimal.cpp => data_type_decimal.cpp} | 2 +- ...ta_types_decimal.h => data_type_decimal.h} | 2 +- be/src/vec/data_types/data_type_factory.hpp | 4 +- be/src/vec/data_types/data_type_nullable.cpp | 2 +- ..._types_number.cpp => data_type_number.cpp} | 2 +- ...data_types_number.h => data_type_number.h} | 0 be/src/vec/data_types/get_least_supertype.cpp | 4 +- be/src/vec/exprs/vectorized_fn_call.cpp | 2 +- .../functions/function_binary_arithmetic.h | 1390 +++++++++-------- be/src/vec/functions/function_cast.h | 80 +- be/src/vec/functions/function_const.h | 16 +- .../functions/function_string_or_array_to_t.h | 125 +- .../vec/functions/function_string_to_string.h | 83 +- .../vec/functions/function_unary_arithmetic.h | 4 +- be/src/vec/functions/functions_comparison.h | 2 +- be/src/vec/functions/functions_logical.cpp | 6 +- be/test/vec/aggregate_functions/agg_test.cpp | 2 +- 27 files changed, 925 insertions(+), 1273 deletions(-) delete mode 100644 be/src/service/bench.cpp rename be/src/vec/data_types/{data_types_decimal.cpp => data_type_decimal.cpp} (99%) rename be/src/vec/data_types/{data_types_decimal.h => data_type_decimal.h} (99%) rename be/src/vec/data_types/{data_types_number.cpp => data_type_number.cpp} (94%) rename be/src/vec/data_types/{data_types_number.h => data_type_number.h} (100%) diff --git a/be/src/runtime/types.h b/be/src/runtime/types.h index ddd5e76012b518..c1e13f6750f205 100644 --- a/be/src/runtime/types.h +++ b/be/src/runtime/types.h @@ -29,9 +29,9 @@ #include "thrift/protocol/TDebugProtocol.h" #include "vec/data_types/data_type_date.h" #include "vec/data_types/data_type_date_time.h" +#include "vec/data_types/data_type_decimal.h" +#include "vec/data_types/data_type_number.h" #include "vec/data_types/data_type_string.h" -#include "vec/data_types/data_types_decimal.h" -#include "vec/data_types/data_types_number.h" namespace doris { diff --git a/be/src/service/bench.cpp b/be/src/service/bench.cpp deleted file mode 100644 index cb2266d150ae1a..00000000000000 --- a/be/src/service/bench.cpp +++ /dev/null @@ -1,323 +0,0 @@ -#include -#include - -#include -#include - -#include "exec/schema_scanner.h" -#include "exprs/aggregate_functions.h" -#include "exprs/expr.h" -#include "exprs/expr_context.h" -#include "exprs/math_functions.h" -#include "gen_cpp/Data_types.h" -#include "gen_cpp/Exprs_types.h" -#include "runtime/exec_env.h" -#include "runtime/memory/chunk_allocator.h" -#include "runtime/row_batch.h" -#include "runtime/runtime_state.h" -#include "runtime/tuple_row.h" -#include "testutil/desc_tbl_builder.h" -#include "udf/udf_internal.h" -#include "vec/core/block.h" -#include "vec/functions/abs.hpp" -#include "vec/aggregate_functions/aggregate_function.h" -#include "vec/aggregate_functions/aggregate_function_simple_factory.h" -#include "vec/aggregate_functions/aggregate_function_sum.h" -#include "vec/columns/column_vector.h" -#include "vec/core/block.h" -#include "vec/data_types/data_type.h" -#include "vec/data_types/data_types_number.h" -#include "vec/exprs/vexpr.h" -#include "vec/exprs/vexpr_context.h" - -static void BM_ABS_SCALAR(benchmark::State& state) { - using namespace doris; - ChunkAllocator::init_instance(4096); - - ObjectPool object_pool; - DescriptorTblBuilder builder(&object_pool); - builder.declare_tuple() << TYPE_INT << TYPE_INT << TYPE_DOUBLE; - DescriptorTbl* desc_tbl = builder.build(); - - auto tuple_desc = const_cast(desc_tbl->get_tuple_descriptor(0)); - RowDescriptor row_desc(tuple_desc, false); - auto tracker_ptr = MemTracker::CreateTracker(-1, "BlockTest", nullptr, false); - RowBatch row_batch(row_desc, 1024, tracker_ptr.get()); - - row_batch.reset(); - int32_t k1 = -100; - int32_t k2 = 100000; - double k3 = 7.7; - for (int i = 0; i < 1024; ++i, k1++, k2++, k3 += 0.1) { - auto idx = row_batch.add_row(); - TupleRow* tuple_row = row_batch.get_row(idx); - auto tuple = (Tuple*)(row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size())); - auto slot_desc = tuple_desc->slots()[0]; - memcpy(tuple->get_slot(slot_desc->tuple_offset()), &k1, slot_desc->slot_size()); - slot_desc = tuple_desc->slots()[1]; - memcpy(tuple->get_slot(slot_desc->tuple_offset()), &k2, slot_desc->slot_size()); - tuple_row->set_tuple(0, tuple); - row_batch.commit_last_row(); - } - - MathFunctions::init(); - - std::string expr_json = - R"|({"1":{"lst":["rec",2,{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"abs"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]}}},"5":{"tf":0},"7":{"str":"abs(INT)"},"9":{"rec":{"1":{"str":"_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_6IntValE"}}},"11":{"i64":0}}}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0}}},"20":{"i32":-1},"23":{"i32":-1}}]}})|"; - TExpr exprx = apache::thrift::from_json_string(expr_json); - ObjectPool pool; - RuntimeState runtime_stat(TUniqueId(), TQueryOptions(), TQueryGlobals(), nullptr); - runtime_stat.init_instance_mem_tracker(); - runtime_stat.set_desc_tbl(desc_tbl); - std::shared_ptr tracker = MemTracker::CreateTracker(); - ExprContext* ctx = nullptr; - Expr::create_expr_tree(&pool, exprx, &ctx); - ctx->prepare(&runtime_stat, row_desc, tracker); - ctx->open(&runtime_stat); - - void* res_ary[1024]; - for (auto _ : state) { - for (int i = 0; i < 1024; ++i) { - TupleRow* tuple_row = row_batch.get_row(i); - auto res = ctx->get_value(tuple_row); - res_ary[i] = res; - } - } - benchmark::DoNotOptimize(res_ary); -} -// Register the function as a benchmark -BENCHMARK(BM_ABS_SCALAR); - -// Define another benchmark -static void BM_ABS_VEC(benchmark::State& state) { - using namespace doris; - SchemaScanner::ColumnDesc column_descs[] = {{"k1", TYPE_SMALLINT, sizeof(int16_t), false}, - {"k2", TYPE_INT, sizeof(int32_t), false}, - {"k3", TYPE_DOUBLE, sizeof(double), false}}; - SchemaScanner schema_scanner(column_descs, 3); - ObjectPool object_pool; - SchemaScannerParam param; - schema_scanner.init(¶m, &object_pool); - auto tuple_desc = const_cast(schema_scanner.tuple_desc()); - RowDescriptor row_desc(tuple_desc, false); - auto tracker_ptr = MemTracker::CreateTracker(-1, "BlockTest", nullptr, false); - RowBatch row_batch(row_desc, 1024, tracker_ptr.get()); - //for (auto _ : state) std::string empty_string; - auto slot_ref = new SlotRef(tuple_desc->slots()[0], tuple_desc->slots()[0]->type()); - auto context = new ExprContext(slot_ref); - row_batch.reset(); - int16_t k1 = -100; - int32_t k2 = 100000; - double k3 = 7.7; - for (int i = 0; i < 1024; ++i, k1++, k2++, k3 += 0.1) { - auto idx = row_batch.add_row(); - TupleRow* tuple_row = row_batch.get_row(idx); - auto tuple = (Tuple*)(row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size())); - auto slot_desc = tuple_desc->slots()[0]; - memcpy(tuple->get_slot(slot_desc->tuple_offset()), &k1, column_descs[0].size); - slot_desc = tuple_desc->slots()[1]; - memcpy(tuple->get_slot(slot_desc->tuple_offset()), &k2, column_descs[1].size); - tuple_row->set_tuple(0, tuple); - row_batch.commit_last_row(); - } - auto block = row_batch.convert_to_vec_block(); - doris::vectorized::FunctionAbs function_abs; - std::shared_ptr abs_function_ptr = function_abs.create(); - doris::vectorized::ColumnNumbers arguments; - arguments.emplace_back(block.getPositionByName("k2")); - doris::vectorized::ColumnPtr column1 = block.getColumns()[0]; - size_t num_columns_without_result = block.columns(); - block.insert({nullptr, block.getByPosition(0).type, "abs(k2)"}); - abs_function_ptr->execute(block, arguments, num_columns_without_result, 1024, false); - for (auto _ : state) { - abs_function_ptr->execute(block, arguments, num_columns_without_result, 1024, false); - } - benchmark::DoNotOptimize(block); -} -BENCHMARK(BM_ABS_VEC); - -// Define another benchmark -static void BM_ABS_VECIMPL(benchmark::State& state) { - using namespace doris; - SchemaScanner::ColumnDesc column_descs[] = {{"k1", TYPE_INT, sizeof(int32_t), false}, - {"k2", TYPE_INT, sizeof(int32_t), false}, - {"k3", TYPE_DOUBLE, sizeof(double), false}}; - SchemaScanner schema_scanner(column_descs, 3); - ObjectPool object_pool; - SchemaScannerParam param; - schema_scanner.init(¶m, &object_pool); - auto tuple_desc = const_cast(schema_scanner.tuple_desc()); - RowDescriptor row_desc(tuple_desc, false); - auto tracker_ptr = MemTracker::CreateTracker(-1, "BlockTest", nullptr, false); - RowBatch row_batch(row_desc, 1024, tracker_ptr.get()); - //for (auto _ : state) std::string empty_string; - row_batch.reset(); - int32_t k1 = -100; - int32_t k2 = 100000; - double k3 = 7.7; - for (int i = 0; i < 1024; ++i, k1++, k2++, k3 += 0.1) { - auto idx = row_batch.add_row(); - TupleRow* tuple_row = row_batch.get_row(idx); - auto tuple = (Tuple*)(row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size())); - auto slot_desc = tuple_desc->slots()[0]; - memcpy(tuple->get_slot(slot_desc->tuple_offset()), &k1, column_descs[0].size); - slot_desc = tuple_desc->slots()[1]; - memcpy(tuple->get_slot(slot_desc->tuple_offset()), &k2, column_descs[1].size); - tuple_row->set_tuple(0, tuple); - row_batch.commit_last_row(); - } - auto block = row_batch.convert_to_vec_block(); - - std::string expr_json = - R"|({"1":{"lst":["rec",2,{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"abs"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]}}},"5":{"tf":0},"7":{"str":"abs(INT)"},"9":{"rec":{"1":{"str":"_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_6IntValE"}}},"11":{"i64":0}}}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0}}},"20":{"i32":-1},"23":{"i32":-1}}]}})|"; - TExpr exprx = apache::thrift::from_json_string(expr_json); - doris::vectorized::VExprContext* context = nullptr; - - doris::vectorized::VExpr::create_expr_tree(&object_pool, exprx, &context); - doris::RuntimeState runtime_stat(doris::TUniqueId(), doris::TQueryOptions(), - doris::TQueryGlobals(), nullptr); - runtime_stat.init_instance_mem_tracker(); - DescriptorTbl desc_tbl; - desc_tbl._slot_desc_map[0] = tuple_desc->slots()[0]; - runtime_stat.set_desc_tbl(&desc_tbl); - std::shared_ptr tracker = doris::MemTracker::CreateTracker(); - context->prepare(&runtime_stat, row_desc, tracker); - context->open(&runtime_stat); - - int ts = -1; - - for (auto _ : state) { - block = row_batch.convert_to_vec_block(); - context->execute(&block, &ts); - block.erase(ts); - } - benchmark::DoNotOptimize(block); -} -BENCHMARK(BM_ABS_VECIMPL); - -static void BM_AGG_COUNST_SCALAR(benchmark::State& state) { - using namespace doris; - ChunkAllocator::init_instance(4096); - - ObjectPool object_pool; - DescriptorTblBuilder builder(&object_pool); - builder.declare_tuple() << TYPE_SMALLINT << TYPE_INT << TYPE_DOUBLE; - DescriptorTbl* desc_tbl = builder.build(); - - auto tuple_desc = const_cast(desc_tbl->get_tuple_descriptor(0)); - RowDescriptor row_desc(tuple_desc, false); - auto tracker_ptr = MemTracker::CreateTracker(-1, "ScalarAGG", nullptr, false); - RowBatch row_batch(row_desc, 1024, tracker_ptr.get()); - - int16_t k1 = -100; - int32_t k2 = 100000; - double k3 = 7.7; - for (int i = 0; i < 1024; ++i, k1++, k2++, k3 += 0.1) { - auto idx = row_batch.add_row(); - TupleRow* tuple_row = row_batch.get_row(idx); - auto tuple = (Tuple*)(row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size())); - auto slot_desc = tuple_desc->slots()[0]; - memcpy(tuple->get_slot(slot_desc->tuple_offset()), &k1, slot_desc->slot_size()); - slot_desc = tuple_desc->slots()[1]; - memcpy(tuple->get_slot(slot_desc->tuple_offset()), &k2, slot_desc->slot_size()); - tuple_row->set_tuple(0, tuple); - row_batch.commit_last_row(); - } - - RuntimeState runtime_stat(TUniqueId(), TQueryOptions(), TQueryGlobals(), nullptr); - runtime_stat.init_instance_mem_tracker(); - runtime_stat.set_desc_tbl(desc_tbl); - MemPool pool(runtime_stat.instance_mem_tracker().get()); - auto context = FunctionContextImpl::create_context( - &runtime_stat, &pool, - doris_udf::FunctionContext::TypeDesc {.type = doris_udf::FunctionContext::TYPE_INT}, - doris_udf::FunctionContext::TypeDesc {.type = doris_udf::FunctionContext::TYPE_INT}, - std::vector { - doris_udf::FunctionContext::TypeDesc { - .type = doris_udf::FunctionContext::TYPE_INT}}, - 0, false); - - // AggregateFunctions::init(); - doris_udf::BigIntVal bigintval; - auto slot_ref = new SlotRef(tuple_desc->slots()[1], tuple_desc->slots()[1]->type()); - auto slot_expr = new ExprContext(slot_ref); - for (auto _ : state) { - for (int i = 0; i < 1024; ++i) { - TupleRow* tuple_row = row_batch.get_row(i); - IntVal val = slot_expr->get_int_val(tuple_row); - AggregateFunctions::count_update(context, &val, &bigintval); - } - } - benchmark::DoNotOptimize(bigintval); -} - -BENCHMARK(BM_AGG_COUNST_SCALAR); -namespace doris::vectorized { -void registerAggregateFunctionSum(vectorized::AggregateFunctionSimpleFactory& factory); -} -static void BM_AGG_COUNT_VEC(benchmark::State& state) { - using namespace doris; - - SchemaScanner::ColumnDesc column_descs[] = {{"k1", TYPE_SMALLINT, sizeof(int16_t), false}, - {"k2", TYPE_INT, sizeof(int32_t), false}, - {"k3", TYPE_DOUBLE, sizeof(double), false}}; - SchemaScanner schema_scanner(column_descs, 3); - ObjectPool object_pool; - SchemaScannerParam param; - schema_scanner.init(¶m, &object_pool); - auto tuple_desc = const_cast(schema_scanner.tuple_desc()); - RowDescriptor row_desc(tuple_desc, false); - auto tracker_ptr = MemTracker::CreateTracker(-1, "BlockTest", nullptr, false); - RowBatch row_batch(row_desc, 1024, tracker_ptr.get()); - //for (auto _ : state) std::string empty_string; - // auto slot_ref = new SlotRef(tuple_desc->slots()[0], tuple_desc->slots()[0]->type()); - // auto context = new ExprContext(slot_ref); - row_batch.reset(); - int16_t k1 = -100; - int32_t k2 = 100000; - double k3 = 7.7; - for (int i = 0; i < 1024; ++i, k1++, k2++, k3 += 0.1) { - auto idx = row_batch.add_row(); - TupleRow* tuple_row = row_batch.get_row(idx); - auto tuple = (Tuple*)(row_batch.tuple_data_pool()->allocate(tuple_desc->byte_size())); - auto slot_desc = tuple_desc->slots()[0]; - memcpy(tuple->get_slot(slot_desc->tuple_offset()), &k1, column_descs[0].size); - slot_desc = tuple_desc->slots()[1]; - memcpy(tuple->get_slot(slot_desc->tuple_offset()), &k2, column_descs[1].size); - tuple_row->set_tuple(0, tuple); - row_batch.commit_last_row(); - } - auto block = row_batch.convert_to_vec_block(); - doris::vectorized::Columns columns = block.getColumns(); - doris::vectorized::AggregateFunctionSimpleFactory factory; - registerAggregateFunctionSum(factory); - doris::vectorized::DataTypePtr data_type(std::make_shared()); - doris::vectorized::DataTypes data_types = {data_type}; - doris::vectorized::Array array; - auto agg_function = factory.get("sum", data_types, array); - doris::vectorized::AggregateDataPtr place = (char*)malloc(sizeof(uint64_t) * 4096); - agg_function->create(place); - const doris::vectorized::IColumn* column[1] = {columns[1].get()}; - - // using ResultType = NearestFieldType; - // using AggregateDataType = AggregateFunctionSumData; - // using Function = AggregateFunctionSum; - - doris::vectorized::AggregateFunctionSum>* func = - nullptr; - func = (vectorized::AggregateFunctionSum< - int32_t, int64_t, vectorized::AggregateFunctionSumData>*)agg_function.get(); - for (auto _ : state) { - agg_function->addBatchSinglePlace(4096,place,column,nullptr); - for (int i = 0; i < 4096; i++) { - // agg_function->add(place, column, i, nullptr); - func->add(place, column, i, nullptr); - } - } - - benchmark::DoNotOptimize(block); -} -BENCHMARK(BM_AGG_COUNT_VEC); - -BENCHMARK_MAIN(); diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt index 4ee9d85f02d039..56852e46f1b448 100644 --- a/be/src/vec/CMakeLists.txt +++ b/be/src/vec/CMakeLists.txt @@ -54,8 +54,8 @@ set(VEC_FILES data_types/data_type_nullable.cpp data_types/data_type_number_base.cpp data_types/data_type_string.cpp - data_types/data_types_decimal.cpp - data_types/data_types_number.cpp + data_types/data_type_decimal.cpp + data_types/data_type_number.cpp data_types/get_least_supertype.cpp data_types/nested_utils.cpp data_types/data_type_date.cpp diff --git a/be/src/vec/aggregate_functions/aggregate_function_avg.h b/be/src/vec/aggregate_functions/aggregate_function_avg.h index 711c311f1fe61a..3c85a59c1a0694 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_avg.h +++ b/be/src/vec/aggregate_functions/aggregate_function_avg.h @@ -1,28 +1,24 @@ #pragma once -#include "vec/data_types/data_types_number.h" -#include "vec/data_types/data_types_decimal.h" -#include "vec/columns/columns_number.h" #include "vec/aggregate_functions/aggregate_function.h" +#include "vec/columns/columns_number.h" +#include "vec/data_types/data_type_decimal.h" +#include "vec/data_types/data_type_number.h" #include "vec/io/io_helper.h" -namespace doris::vectorized -{ +namespace doris::vectorized { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; +namespace ErrorCodes { +extern const int LOGICAL_ERROR; } template -struct AggregateFunctionAvgData -{ +struct AggregateFunctionAvgData { T sum = 0; UInt64 count = 0; template - ResultT NO_SANITIZE_UNDEFINED result() const - { + ResultT NO_SANITIZE_UNDEFINED result() const { if constexpr (std::is_floating_point_v) if constexpr (std::numeric_limits::is_iec559) return static_cast(sum) / count; /// allow division by zero @@ -43,73 +39,68 @@ struct AggregateFunctionAvgData } }; - /// Calculates arithmetic mean of numbers. template -class AggregateFunctionAvg final : public IAggregateFunctionDataHelper> -{ +class AggregateFunctionAvg final + : public IAggregateFunctionDataHelper> { public: using ResultType = std::conditional_t, Decimal128, Float64>; - using ResultDataType = std::conditional_t, DataTypeDecimal, DataTypeNumber>; + using ResultDataType = std::conditional_t, DataTypeDecimal, + DataTypeNumber>; using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; - using ColVecResult = std::conditional_t, ColumnDecimal, ColumnVector>; + using ColVecResult = std::conditional_t, ColumnDecimal, + ColumnVector>; /// ctor for native types - AggregateFunctionAvg(const DataTypes & argument_types_) - : IAggregateFunctionDataHelper>(argument_types_, {}) - , scale(0) - {} + AggregateFunctionAvg(const DataTypes& argument_types_) + : IAggregateFunctionDataHelper>(argument_types_, + {}), + scale(0) {} /// ctor for Decimals - AggregateFunctionAvg(const IDataType & data_type, const DataTypes & argument_types_) - : IAggregateFunctionDataHelper>(argument_types_, {}) - , scale(getDecimalScale(data_type)) - {} + AggregateFunctionAvg(const IDataType& data_type, const DataTypes& argument_types_) + : IAggregateFunctionDataHelper>(argument_types_, + {}), + scale(getDecimalScale(data_type)) {} String getName() const override { return "avg"; } - DataTypePtr getReturnType() const override - { + DataTypePtr getReturnType() const override { if constexpr (IsDecimalNumber) return std::make_shared(ResultDataType::maxPrecision(), scale); else return std::make_shared(); } - void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override - { - const auto & column = static_cast(*columns[0]); + void add(AggregateDataPtr place, const IColumn** columns, size_t row_num, + Arena*) const override { + const auto& column = static_cast(*columns[0]); this->data(place).sum += column.getData()[row_num]; ++this->data(place).count; } - void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override - { + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena*) const override { this->data(place).sum += this->data(rhs).sum; this->data(place).count += this->data(rhs).count; } - void serialize(ConstAggregateDataPtr place, std::ostream& buf) const override - { + void serialize(ConstAggregateDataPtr place, std::ostream& buf) const override { this->data(place).write(buf); } - void deserialize(AggregateDataPtr place, std::istream& buf, Arena *) const override - { + void deserialize(AggregateDataPtr place, std::istream& buf, Arena*) const override { this->data(place).read(buf); } - void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override - { - auto & column = static_cast(to); + void insertResultInto(ConstAggregateDataPtr place, IColumn& to) const override { + auto& column = static_cast(to); column.getData().push_back(this->data(place).template result()); } - const char * getHeaderFilePath() const override { return __FILE__; } + const char* getHeaderFilePath() const override { return __FILE__; } private: UInt32 scale; }; - -} +} // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_count.h b/be/src/vec/aggregate_functions/aggregate_function_count.h index 1c5f61a26eb4a3..607f8eb12f71a0 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_count.h +++ b/be/src/vec/aggregate_functions/aggregate_function_count.h @@ -23,7 +23,7 @@ #include #include #include -#include +#include #include #include diff --git a/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h b/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h index 087b9fdd746a9e..9ecd3ffce2a950 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h +++ b/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h @@ -17,18 +17,17 @@ #pragma once -#include #include #include +#include #include "exprs/hll_function.h" #include "olap/hll.h" #include "util/slice.h" - #include "vec/aggregate_functions/aggregate_function.h" #include "vec/columns/column_string.h" #include "vec/columns/column_vector.h" -#include "vec/data_types/data_types_number.h" +#include "vec/data_types/data_type_number.h" #include "vec/data_types/data_type_string.h" #include "vec/io/io_helper.h" @@ -37,30 +36,24 @@ namespace doris::vectorized { struct AggregateFunctionHLLData { doris::HyperLogLog dst_hll{}; - void add(const StringRef& src) { - dst_hll.merge(HyperLogLog(Slice(src.data, src.size))); - } + void add(const StringRef& src) { dst_hll.merge(HyperLogLog(Slice(src.data, src.size))); } - void merge(const AggregateFunctionHLLData& rhs) { - dst_hll.merge(rhs.dst_hll); - } + void merge(const AggregateFunctionHLLData& rhs) { dst_hll.merge(rhs.dst_hll); } void write(std::ostream& buf) const { std::string result(dst_hll.max_serialized_size(), '0'); int size = dst_hll.serialize((uint8_t*)result.c_str()); result.resize(size); - writeBinary(result, buf); + writeBinary(result, buf); } void read(std::istream& buf) { std::string result; - readBinary(result, buf); + readBinary(result, buf); dst_hll.deserialize(Slice(result.c_str(), result.length())); } - Int64 get_cardinality() const { - return dst_hll.estimate_cardinality(); - } + Int64 get_cardinality() const { return dst_hll.estimate_cardinality(); } std::string get() const { std::string result(dst_hll.max_serialized_size(), '0'); @@ -69,24 +62,21 @@ struct AggregateFunctionHLLData { return result; } - }; class AggregateFunctionHLLUnionAgg - : public IAggregateFunctionDataHelper { + : public IAggregateFunctionDataHelper { public: virtual String getName() const override { return "hll_union_agg"; } AggregateFunctionHLLUnionAgg(const DataTypes& argument_types_) - : IAggregateFunctionDataHelper( - argument_types_, {}){} + : IAggregateFunctionDataHelper(argument_types_, {}) {} AggregateFunctionHLLUnionAgg(const IDataType& data_type, const DataTypes& argument_types_) : IAggregateFunctionDataHelper(argument_types_, {}) {} - virtual DataTypePtr getReturnType() const override { - return std::make_shared(); - } + virtual DataTypePtr getReturnType() const override { return std::make_shared(); } void add(AggregateDataPtr place, const IColumn** columns, size_t row_num, Arena*) const override { @@ -112,11 +102,9 @@ class AggregateFunctionHLLUnionAgg } const char* getHeaderFilePath() const override { return __FILE__; } - }; -class AggregateFunctionHLLUnion final - : public AggregateFunctionHLLUnionAgg { +class AggregateFunctionHLLUnion final : public AggregateFunctionHLLUnionAgg { public: String getName() const override { return "hll_union"; } @@ -126,9 +114,7 @@ class AggregateFunctionHLLUnion final AggregateFunctionHLLUnion(const IDataType& data_type, const DataTypes& argument_types_) : AggregateFunctionHLLUnionAgg(data_type, argument_types_) {} - DataTypePtr getReturnType() const override { - return std::make_shared(); - } + DataTypePtr getReturnType() const override { return std::make_shared(); } void insertResultInto(ConstAggregateDataPtr place, IColumn& to) const override { auto& column = static_cast(to); diff --git a/be/src/vec/aggregate_functions/aggregate_function_sum.h b/be/src/vec/aggregate_functions/aggregate_function_sum.h index 0aa98ac84d9671..28a37b31390cba 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_sum.h +++ b/be/src/vec/aggregate_functions/aggregate_function_sum.h @@ -26,8 +26,8 @@ #include "vec/aggregate_functions/aggregate_function.h" #include "vec/columns/column_vector.h" -#include "vec/data_types/data_types_decimal.h" -#include "vec/data_types/data_types_number.h" +#include "vec/data_types/data_type_decimal.h" +#include "vec/data_types/data_type_number.h" #include "vec/io/io_helper.h" namespace doris::vectorized { diff --git a/be/src/vec/aggregate_functions/aggregate_function_uniq.h b/be/src/vec/aggregate_functions/aggregate_function_uniq.h index 0436d95795a66d..3f88695e9c4a81 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_uniq.h +++ b/be/src/vec/aggregate_functions/aggregate_function_uniq.h @@ -13,7 +13,7 @@ #include #include "vec/common/aggregation_common.h" -#include "vec/data_types/data_types_number.h" +#include "vec/data_types/data_type_number.h" // #include // #include #include diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp index f72ef0ba447b51..2cd42d226b9b33 100644 --- a/be/src/vec/core/block.cpp +++ b/be/src/vec/core/block.cpp @@ -32,10 +32,12 @@ #include "vec/common/exception.h" #include "vec/common/field_visitors.h" #include "vec/common/typeid_cast.h" +#include "vec/data_types/data_type_date.h" +#include "vec/data_types/data_type_date_time.h" +#include "vec/data_types/data_type_decimal.h" #include "vec/data_types/data_type_nullable.h" +#include "vec/data_types/data_type_number.h" #include "vec/data_types/data_type_string.h" -#include "vec/data_types/data_types_decimal.h" -#include "vec/data_types/data_types_number.h" namespace doris::vectorized { @@ -88,6 +90,12 @@ inline DataTypePtr get_data_type(const PColumn& pcolumn) { case PColumn::STRING: { return std::make_shared(); } + case PColumn::DATE: { + return std::make_shared(); + } + case PColumn::DATETIME: { + return std::make_shared(); + } case PColumn::DECIMAL32: { return std::make_shared>(pcolumn.decimal_param().precision(), pcolumn.decimal_param().scale()); @@ -143,6 +151,10 @@ PColumn::DataType get_pdata_type(DataTypePtr data_type) { return PColumn::DECIMAL128; case TypeIndex::String: return PColumn::STRING; + case TypeIndex::Date: + return PColumn::DATE; + case TypeIndex::DateTime: + return PColumn::DATETIME; default: return PColumn::UNKNOWN; } diff --git a/be/src/vec/core/decimal_comparison.h b/be/src/vec/core/decimal_comparison.h index c43881c79fda5e..355c68ac4d2f3a 100644 --- a/be/src/vec/core/decimal_comparison.h +++ b/be/src/vec/core/decimal_comparison.h @@ -24,7 +24,7 @@ #include "vec/core/accurate_comparison.h" #include "vec/core/block.h" #include "vec/core/call_on_type_index.h" -#include "vec/data_types/data_types_decimal.h" +#include "vec/data_types/data_type_decimal.h" #include "vec/functions/function_helpers.h" /// todo core should not depend on function" namespace doris::vectorized { diff --git a/be/src/vec/data_types/data_types_decimal.cpp b/be/src/vec/data_types/data_type_decimal.cpp similarity index 99% rename from be/src/vec/data_types/data_types_decimal.cpp rename to be/src/vec/data_types/data_type_decimal.cpp index 3da3f6f39cf858..13849f4af8f954 100644 --- a/be/src/vec/data_types/data_types_decimal.cpp +++ b/be/src/vec/data_types/data_type_decimal.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "vec/data_types/data_types_decimal.h" +#include "vec/data_types/data_type_decimal.h" #include diff --git a/be/src/vec/data_types/data_types_decimal.h b/be/src/vec/data_types/data_type_decimal.h similarity index 99% rename from be/src/vec/data_types/data_types_decimal.h rename to be/src/vec/data_types/data_type_decimal.h index e3e04fc58299c2..f6181ea8fe754f 100644 --- a/be/src/vec/data_types/data_types_decimal.h +++ b/be/src/vec/data_types/data_type_decimal.h @@ -22,7 +22,7 @@ #include "vec/common/arithmetic_overflow.h" #include "vec/common/typeid_cast.h" #include "vec/data_types/data_type.h" -#include "vec/data_types/data_types_number.h" +#include "vec/data_types/data_type_number.h" namespace doris::vectorized { diff --git a/be/src/vec/data_types/data_type_factory.hpp b/be/src/vec/data_types/data_type_factory.hpp index 285458a4ad25b4..13462103d319c3 100644 --- a/be/src/vec/data_types/data_type_factory.hpp +++ b/be/src/vec/data_types/data_type_factory.hpp @@ -22,10 +22,10 @@ #include "vec/data_types/data_type.h" #include "vec/data_types/data_type_date.h" #include "vec/data_types/data_type_date_time.h" +#include "vec/data_types/data_type_decimal.h" #include "vec/data_types/data_type_nothing.h" +#include "vec/data_types/data_type_number.h" #include "vec/data_types/data_type_string.h" -#include "vec/data_types/data_types_decimal.h" -#include "vec/data_types/data_types_number.h" namespace doris::vectorized { diff --git a/be/src/vec/data_types/data_type_nullable.cpp b/be/src/vec/data_types/data_type_nullable.cpp index 740d1e6f596c77..f00d4866b9d343 100644 --- a/be/src/vec/data_types/data_type_nullable.cpp +++ b/be/src/vec/data_types/data_type_nullable.cpp @@ -23,7 +23,7 @@ #include "vec/common/typeid_cast.h" #include "vec/core/field.h" #include "vec/data_types/data_type_nothing.h" -#include "vec/data_types/data_types_number.h" +#include "vec/data_types/data_type_number.h" namespace doris::vectorized { diff --git a/be/src/vec/data_types/data_types_number.cpp b/be/src/vec/data_types/data_type_number.cpp similarity index 94% rename from be/src/vec/data_types/data_types_number.cpp rename to be/src/vec/data_types/data_type_number.cpp index 8ec44aebcc7d0f..6fcc4146419bc8 100644 --- a/be/src/vec/data_types/data_types_number.cpp +++ b/be/src/vec/data_types/data_type_number.cpp @@ -15,6 +15,6 @@ // specific language governing permissions and limitations // under the License. -#include "vec/data_types/data_types_number.h" +#include "vec/data_types/data_type_number.h" namespace doris::vectorized {} // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_types_number.h b/be/src/vec/data_types/data_type_number.h similarity index 100% rename from be/src/vec/data_types/data_types_number.h rename to be/src/vec/data_types/data_type_number.h diff --git a/be/src/vec/data_types/get_least_supertype.cpp b/be/src/vec/data_types/get_least_supertype.cpp index 2cb48ae6b34d0e..456413b0200970 100644 --- a/be/src/vec/data_types/get_least_supertype.cpp +++ b/be/src/vec/data_types/get_least_supertype.cpp @@ -22,11 +22,11 @@ #include "vec/common/typeid_cast.h" #include "vec/data_types/data_type_date.h" #include "vec/data_types/data_type_date_time.h" +#include "vec/data_types/data_type_decimal.h" #include "vec/data_types/data_type_nothing.h" #include "vec/data_types/data_type_nullable.h" +#include "vec/data_types/data_type_number.h" #include "vec/data_types/data_type_string.h" -#include "vec/data_types/data_types_decimal.h" -#include "vec/data_types/data_types_number.h" namespace doris::vectorized { diff --git a/be/src/vec/exprs/vectorized_fn_call.cpp b/be/src/vec/exprs/vectorized_fn_call.cpp index 26fe84f2eca772..957df69e2a7409 100644 --- a/be/src/vec/exprs/vectorized_fn_call.cpp +++ b/be/src/vec/exprs/vectorized_fn_call.cpp @@ -22,7 +22,7 @@ #include "fmt/format.h" #include "fmt/ranges.h" #include "vec/data_types/data_type_nullable.h" -#include "vec/data_types/data_types_number.h" +#include "vec/data_types/data_type_number.h" #include "vec/functions/simple_function_factory.h" namespace doris::vectorized { diff --git a/be/src/vec/functions/function_binary_arithmetic.h b/be/src/vec/functions/function_binary_arithmetic.h index 448699a66b4838..23b0aa34f058f0 100644 --- a/be/src/vec/functions/function_binary_arithmetic.h +++ b/be/src/vec/functions/function_binary_arithmetic.h @@ -6,25 +6,25 @@ //#include "vec/common/arena.h" #include "vec/data_types/data_type.h" -#include "vec/data_types/data_types_number.h" -#include "vec/data_types/data_types_decimal.h" +#include "vec/data_types/data_type_decimal.h" +#include "vec/data_types/data_type_number.h" //#include "vec/data_types/data_type_date.h" //#include "vec/data_types/data_type_date_time.h" //#include "vec/data_types/data_type_interval.h" //#include "vec/data_types/data_type_aggregate_function.h" //#include "vec/data_types/native.h" -#include "vec/data_types/number_traits.h" -#include "vec/columns/column_vector.h" -#include "vec/columns/column_decimal.h" #include "vec/columns/column_const.h" +#include "vec/columns/column_decimal.h" +#include "vec/columns/column_vector.h" +#include "vec/data_types/number_traits.h" //#include "vec/columns/column_aggregate_function.h" +#include "vec/functions/cast_type_to_either.h" #include "vec/functions/function.h" #include "vec/functions/function_helpers.h" #include "vec/functions/int_div.h" -#include "vec/functions/cast_type_to_either.h" //#include "vec/functions/FunctionFactory.h" -#include "vec/common/typeid_cast.h" #include "vec/common/assert_cast.h" +#include "vec/common/typeid_cast.h" //#include "vec/common/config.h" #if USE_EMBEDDED_COMPILER @@ -34,20 +34,16 @@ #pragma GCC diagnostic pop #endif +namespace doris::vectorized { -namespace doris::vectorized -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int LOGICAL_ERROR; - extern const int DECIMAL_OVERFLOW; - extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES; - extern const int ILLEGAL_DIVISION; -} - +namespace ErrorCodes { +extern const int ILLEGAL_COLUMN; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int LOGICAL_ERROR; +extern const int DECIMAL_OVERFLOW; +extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES; +extern const int ILLEGAL_DIVISION; +} // namespace ErrorCodes /** Arithmetic operations: +, -, *, /, %, * intDiv (integer division) @@ -56,78 +52,84 @@ namespace ErrorCodes */ template -struct BinaryOperationImplBase -{ +struct BinaryOperationImplBase { using ResultType = ResultType_; - static void NO_INLINE vector_vector(const PaddedPODArray & a, const PaddedPODArray & b, PaddedPODArray & c) - { + static void NO_INLINE vector_vector(const PaddedPODArray& a, const PaddedPODArray& b, + PaddedPODArray& c) { size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = Op::template apply(a[i], b[i]); + for (size_t i = 0; i < size; ++i) c[i] = Op::template apply(a[i], b[i]); } - static void NO_INLINE vector_constant(const PaddedPODArray & a, B b, PaddedPODArray & c) - { + static void NO_INLINE vector_constant(const PaddedPODArray& a, B b, + PaddedPODArray& c) { size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = Op::template apply(a[i], b); + for (size_t i = 0; i < size; ++i) c[i] = Op::template apply(a[i], b); } - static void NO_INLINE constant_vector(A a, const PaddedPODArray & b, PaddedPODArray & c) - { + static void NO_INLINE constant_vector(A a, const PaddedPODArray& b, + PaddedPODArray& c) { size_t size = b.size(); - for (size_t i = 0; i < size; ++i) - c[i] = Op::template apply(a, b[i]); + for (size_t i = 0; i < size; ++i) c[i] = Op::template apply(a, b[i]); } - static ResultType constant_constant(A a, B b) - { - return Op::template apply(a, b); - } + static ResultType constant_constant(A a, B b) { return Op::template apply(a, b); } }; template -struct BinaryOperationImpl : BinaryOperationImplBase -{ -}; - - -template struct PlusImpl; -template struct MinusImpl; -template struct MultiplyImpl; -template struct DivideFloatingImpl; -template struct DivideIntegralImpl; -template struct DivideIntegralOrZeroImpl; -template struct LeastBaseImpl; -template struct GreatestBaseImpl; -template struct ModuloImpl; - +struct BinaryOperationImpl : BinaryOperationImplBase {}; + +template +struct PlusImpl; +template +struct MinusImpl; +template +struct MultiplyImpl; +template +struct DivideFloatingImpl; +template +struct DivideIntegralImpl; +template +struct DivideIntegralOrZeroImpl; +template +struct LeastBaseImpl; +template +struct GreatestBaseImpl; +template +struct ModuloImpl; /// Binary operations for Decimals need scale args /// +|- scale one of args (which scale factor is not 1). ScaleR = oneof(Scale1, Scale2); /// * no agrs scale. ScaleR = Scale1 + Scale2; /// / first arg scale. ScaleR = Scale1 (scale_a = DecimalType::getScale()). -template typename Operation, typename ResultType_, bool _check_overflow = true> -struct DecimalBinaryOperation -{ - static constexpr bool is_plus_minus = std::is_same_v, PlusImpl> || - std::is_same_v, MinusImpl>; - static constexpr bool is_multiply = std::is_same_v, MultiplyImpl>; - static constexpr bool is_float_division = std::is_same_v, DivideFloatingImpl>; - static constexpr bool is_int_division = std::is_same_v, DivideIntegralImpl> || - std::is_same_v, DivideIntegralOrZeroImpl>; +template typename Operation, + typename ResultType_, bool _check_overflow = true> +struct DecimalBinaryOperation { + static constexpr bool is_plus_minus = + std::is_same_v, PlusImpl> || + std::is_same_v, MinusImpl>; + static constexpr bool is_multiply = + std::is_same_v, MultiplyImpl>; + static constexpr bool is_float_division = + std::is_same_v, DivideFloatingImpl>; + static constexpr bool is_int_division = + std::is_same_v, DivideIntegralImpl> || + std::is_same_v, DivideIntegralOrZeroImpl>; static constexpr bool is_division = is_float_division || is_int_division; - static constexpr bool is_compare = std::is_same_v, LeastBaseImpl> || - std::is_same_v, GreatestBaseImpl>; + static constexpr bool is_compare = + std::is_same_v, LeastBaseImpl> || + std::is_same_v, GreatestBaseImpl>; static constexpr bool is_plus_minus_compare = is_plus_minus || is_compare; static constexpr bool can_overflow = is_plus_minus || is_multiply; using ResultType = ResultType_; using NativeResultType = typename NativeType::Type; - using Op = std::conditional_t, /// substitute divide by intDiv (throw on division by zero) - Operation>; + using Op = std::conditional_t< + is_float_division, + DivideIntegralImpl< + NativeResultType, + NativeResultType>, /// substitute divide by intDiv (throw on division by zero) + Operation>; using ColVecA = std::conditional_t, ColumnDecimal, ColumnVector>; using ColVecB = std::conditional_t, ColumnDecimal, ColumnVector>; using ArrayA = typename ColVecA::Container; @@ -135,169 +137,133 @@ struct DecimalBinaryOperation using ArrayC = typename ColumnDecimal::Container; using SelfNoOverflow = DecimalBinaryOperation; - static void vector_vector(const ArrayA & a, const ArrayB & b, ArrayC & c, ResultType scale_a, ResultType scale_b, bool check_overflow) - { + static void vector_vector(const ArrayA& a, const ArrayB& b, ArrayC& c, ResultType scale_a, + ResultType scale_b, bool check_overflow) { if (check_overflow) vector_vector(a, b, c, scale_a, scale_b); else SelfNoOverflow::vector_vector(a, b, c, scale_a, scale_b); } - static void vector_constant(const ArrayA & a, B b, ArrayC & c, ResultType scale_a, ResultType scale_b, bool check_overflow) - { + static void vector_constant(const ArrayA& a, B b, ArrayC& c, ResultType scale_a, + ResultType scale_b, bool check_overflow) { if (check_overflow) vector_constant(a, b, c, scale_a, scale_b); else SelfNoOverflow::vector_constant(a, b, c, scale_a, scale_b); } - static void constant_vector(A a, const ArrayB & b, ArrayC & c, ResultType scale_a, ResultType scale_b, bool check_overflow) - { + static void constant_vector(A a, const ArrayB& b, ArrayC& c, ResultType scale_a, + ResultType scale_b, bool check_overflow) { if (check_overflow) constant_vector(a, b, c, scale_a, scale_b); else SelfNoOverflow::constant_vector(a, b, c, scale_a, scale_b); } - static ResultType constant_constant(A a, B b, ResultType scale_a, ResultType scale_b, bool check_overflow) - { + static ResultType constant_constant(A a, B b, ResultType scale_a, ResultType scale_b, + bool check_overflow) { if (check_overflow) return constant_constant(a, b, scale_a, scale_b); else return SelfNoOverflow::constant_constant(a, b, scale_a, scale_b); } - static void NO_INLINE vector_vector(const ArrayA & a, const ArrayB & b, ArrayC & c, - ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]]) - { + static void NO_INLINE vector_vector(const ArrayA& a, const ArrayB& b, ArrayC& c, + ResultType scale_a [[maybe_unused]], + ResultType scale_b [[maybe_unused]]) { size_t size = a.size(); - if constexpr (is_plus_minus_compare) - { - if (scale_a != 1) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaled(a[i], b[i], scale_a); + if constexpr (is_plus_minus_compare) { + if (scale_a != 1) { + for (size_t i = 0; i < size; ++i) c[i] = applyScaled(a[i], b[i], scale_a); return; - } - else if (scale_b != 1) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaled(a[i], b[i], scale_b); + } else if (scale_b != 1) { + for (size_t i = 0; i < size; ++i) c[i] = applyScaled(a[i], b[i], scale_b); return; } - } - else if constexpr (is_division && IsDecimalNumber) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaledDiv(a[i], b[i], scale_a); + } else if constexpr (is_division && IsDecimalNumber) { + for (size_t i = 0; i < size; ++i) c[i] = applyScaledDiv(a[i], b[i], scale_a); return; } /// default: use it if no return before - for (size_t i = 0; i < size; ++i) - c[i] = apply(a[i], b[i]); + for (size_t i = 0; i < size; ++i) c[i] = apply(a[i], b[i]); } - static void NO_INLINE vector_constant(const ArrayA & a, B b, ArrayC & c, - ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]]) - { + static void NO_INLINE vector_constant(const ArrayA& a, B b, ArrayC& c, + ResultType scale_a [[maybe_unused]], + ResultType scale_b [[maybe_unused]]) { size_t size = a.size(); - if constexpr (is_plus_minus_compare) - { - if (scale_a != 1) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaled(a[i], b, scale_a); + if constexpr (is_plus_minus_compare) { + if (scale_a != 1) { + for (size_t i = 0; i < size; ++i) c[i] = applyScaled(a[i], b, scale_a); return; - } - else if (scale_b != 1) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaled(a[i], b, scale_b); + } else if (scale_b != 1) { + for (size_t i = 0; i < size; ++i) c[i] = applyScaled(a[i], b, scale_b); return; } - } - else if constexpr (is_division && IsDecimalNumber) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaledDiv(a[i], b, scale_a); + } else if constexpr (is_division && IsDecimalNumber) { + for (size_t i = 0; i < size; ++i) c[i] = applyScaledDiv(a[i], b, scale_a); return; } /// default: use it if no return before - for (size_t i = 0; i < size; ++i) - c[i] = apply(a[i], b); + for (size_t i = 0; i < size; ++i) c[i] = apply(a[i], b); } - static void NO_INLINE constant_vector(A a, const ArrayB & b, ArrayC & c, - ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]]) - { + static void NO_INLINE constant_vector(A a, const ArrayB& b, ArrayC& c, + ResultType scale_a [[maybe_unused]], + ResultType scale_b [[maybe_unused]]) { size_t size = b.size(); - if constexpr (is_plus_minus_compare) - { - if (scale_a != 1) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaled(a, b[i], scale_a); + if constexpr (is_plus_minus_compare) { + if (scale_a != 1) { + for (size_t i = 0; i < size; ++i) c[i] = applyScaled(a, b[i], scale_a); return; - } - else if (scale_b != 1) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaled(a, b[i], scale_b); + } else if (scale_b != 1) { + for (size_t i = 0; i < size; ++i) c[i] = applyScaled(a, b[i], scale_b); return; } - } - else if constexpr (is_division && IsDecimalNumber) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaledDiv(a, b[i], scale_a); + } else if constexpr (is_division && IsDecimalNumber) { + for (size_t i = 0; i < size; ++i) c[i] = applyScaledDiv(a, b[i], scale_a); return; } /// default: use it if no return before - for (size_t i = 0; i < size; ++i) - c[i] = apply(a, b[i]); + for (size_t i = 0; i < size; ++i) c[i] = apply(a, b[i]); } - static ResultType constant_constant(A a, B b, ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]]) - { - if constexpr (is_plus_minus_compare) - { + static ResultType constant_constant(A a, B b, ResultType scale_a [[maybe_unused]], + ResultType scale_b [[maybe_unused]]) { + if constexpr (is_plus_minus_compare) { if (scale_a != 1) return applyScaled(a, b, scale_a); else if (scale_b != 1) return applyScaled(a, b, scale_b); - } - else if constexpr (is_division && IsDecimalNumber) + } else if constexpr (is_division && IsDecimalNumber) return applyScaledDiv(a, b, scale_a); return apply(a, b); } private: /// there's implicit type convertion here - static NativeResultType apply(NativeResultType a, NativeResultType b) - { - if constexpr (can_overflow && _check_overflow) - { + static NativeResultType apply(NativeResultType a, NativeResultType b) { + if constexpr (can_overflow && _check_overflow) { NativeResultType res; if (Op::template apply(a, b, res)) throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); return res; - } - else + } else return Op::template apply(a, b); } template - static NO_SANITIZE_UNDEFINED NativeResultType applyScaled(NativeResultType a, NativeResultType b, NativeResultType scale) - { - if constexpr (is_plus_minus_compare) - { + static NO_SANITIZE_UNDEFINED NativeResultType applyScaled(NativeResultType a, + NativeResultType b, + NativeResultType scale) { + if constexpr (is_plus_minus_compare) { NativeResultType res; - if constexpr (_check_overflow) - { + if constexpr (_check_overflow) { bool overflow = false; if constexpr (scale_left) overflow |= common::mulOverflow(a, scale, a); @@ -311,9 +277,7 @@ struct DecimalBinaryOperation if (overflow) throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); - } - else - { + } else { if constexpr (scale_left) a *= scale; else @@ -325,23 +289,19 @@ struct DecimalBinaryOperation } } - static NO_SANITIZE_UNDEFINED NativeResultType applyScaledDiv(NativeResultType a, NativeResultType b, NativeResultType scale) - { - if constexpr (is_division) - { - if constexpr (_check_overflow) - { + static NO_SANITIZE_UNDEFINED NativeResultType applyScaledDiv(NativeResultType a, + NativeResultType b, + NativeResultType scale) { + if constexpr (is_division) { + if constexpr (_check_overflow) { bool overflow = false; if constexpr (!IsDecimalNumber) overflow |= common::mulOverflow(scale, scale, scale); overflow |= common::mulOverflow(a, scale, a); if (overflow) throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); - } - else - { - if constexpr (!IsDecimalNumber) - scale *= scale; + } else { + if constexpr (!IsDecimalNumber) scale *= scale; a *= scale; } @@ -350,189 +310,211 @@ struct DecimalBinaryOperation } }; - /// Used to indicate undefined operation struct InvalidType; -template struct Case : std::bool_constant { using type = T; }; +template +struct Case : std::bool_constant { + using type = T; +}; /// Switch, ...> -- select the first Ti for which Ci is true; InvalidType if none. -template using Switch = typename std::disjunction>::type; - -template constexpr bool IsIntegral = false; -template <> inline constexpr bool IsIntegral = true; -template <> inline constexpr bool IsIntegral = true; -template <> inline constexpr bool IsIntegral = true; -template <> inline constexpr bool IsIntegral = true; -template <> inline constexpr bool IsIntegral = true; -template <> inline constexpr bool IsIntegral = true; -template <> inline constexpr bool IsIntegral = true; -template <> inline constexpr bool IsIntegral = true; - -template constexpr bool IsFloatingPoint = false; -template <> inline constexpr bool IsFloatingPoint = true; -template <> inline constexpr bool IsFloatingPoint = true; +template +using Switch = typename std::disjunction>::type; + +template +constexpr bool IsIntegral = false; +template <> +inline constexpr bool IsIntegral = true; +template <> +inline constexpr bool IsIntegral = true; +template <> +inline constexpr bool IsIntegral = true; +template <> +inline constexpr bool IsIntegral = true; +template <> +inline constexpr bool IsIntegral = true; +template <> +inline constexpr bool IsIntegral = true; +template <> +inline constexpr bool IsIntegral = true; +template <> +inline constexpr bool IsIntegral = true; + +template +constexpr bool IsFloatingPoint = false; +template <> +inline constexpr bool IsFloatingPoint = true; +template <> +inline constexpr bool IsFloatingPoint = true; //template constexpr bool IsDateOrDateTime = false; //template <> inline constexpr bool IsDateOrDateTime = true; //template <> inline constexpr bool IsDateOrDateTime = true; -template constexpr bool UseLeftDecimal = false; -template <> inline constexpr bool UseLeftDecimal, DataTypeDecimal> = true; -template <> inline constexpr bool UseLeftDecimal, DataTypeDecimal> = true; -template <> inline constexpr bool UseLeftDecimal, DataTypeDecimal> = true; - -template using DataTypeFromFieldType = std::conditional_t, InvalidType, DataTypeNumber>; - -template