From 1828a6270b966c0e7cb9c0b3473ec9460270ee7d Mon Sep 17 00:00:00 2001 From: yangzhg Date: Fri, 11 Feb 2022 17:13:09 +0800 Subject: [PATCH] [refactor] remove some unused code --- be/src/codegen/doris_ir.h | 34 --- be/src/common/hdfs.h | 31 --- be/src/common/logging.h | 32 +-- be/src/exec/CMakeLists.txt | 5 - be/src/exec/aggregation_node.cpp | 36 ++- be/src/exec/aggregation_node_ir.cpp | 50 ---- be/src/exec/hash_join_node.cpp | 153 ++++++++++- be/src/exec/hash_join_node_ir.cpp | 172 ------------ be/src/exec/hash_table.cpp | 14 +- be/src/exec/hash_table.h | 25 +- be/src/exec/parquet_scanner.cpp | 36 +-- be/src/exec/partitioned_aggregation_node.cc | 226 +++++++++++++++- be/src/exec/partitioned_aggregation_node.h | 16 +- .../exec/partitioned_aggregation_node_ir.cc | 250 ------------------ be/src/exec/partitioned_hash_table.h | 67 +++-- be/src/exec/partitioned_hash_table.inline.h | 10 +- be/src/exec/partitioned_hash_table_ir.cc | 35 --- be/src/exec/scanner_ir.cpp | 55 ---- be/src/exec/union_node.cpp | 56 +++- be/src/exec/union_node.h | 1 - be/src/exec/union_node_ir.cpp | 67 ----- be/src/exprs/CMakeLists.txt | 2 - be/src/exprs/agg_fn_evaluator.cpp | 5 - be/src/exprs/cast_functions.cpp | 25 +- be/src/exprs/cast_functions.h | 3 - be/src/exprs/compound_predicate.cpp | 6 - be/src/exprs/conditional_functions.cpp | 101 ++++++- be/src/exprs/conditional_functions_ir.cpp | 117 -------- be/src/exprs/expr.cpp | 26 -- be/src/exprs/expr_context.cpp | 15 -- be/src/exprs/expr_ir.cpp | 81 ------ be/src/exprs/info_func.cpp | 10 - be/src/exprs/math_functions.cpp | 163 ------------ be/src/exprs/new_agg_fn_evaluator.h | 5 +- be/src/exprs/operators.cpp | 45 ---- be/src/exprs/operators.h | 7 - be/src/exprs/scalar_fn_call.cpp | 15 -- be/src/exprs/string_functions.h | 16 +- be/src/geo/geo_types.cpp | 136 ---------- be/src/geo/geo_types.h | 52 ---- be/src/gutil/atomicops.h | 69 ----- .../threading/thread_collision_warner.cc | 18 -- be/src/http/action/health_action.cpp | 4 - be/src/http/action/monitor_action.cpp | 12 - be/src/olap/rowset/column_data.cpp | 10 +- be/src/olap/rowset/column_writer.cpp | 29 -- be/src/olap/rowset/segment_v2/column_reader.h | 52 ++-- be/src/runtime/CMakeLists.txt | 4 - be/src/runtime/buffered_tuple_stream2.cc | 151 +++++++++-- be/src/runtime/buffered_tuple_stream2_ir.cc | 187 ------------- be/src/runtime/buffered_tuple_stream3.cc | 1 - be/src/runtime/disk_io_mgr.h | 1 - be/src/runtime/raw_value.cpp | 90 +++++++ be/src/runtime/raw_value_ir.cpp | 114 -------- be/src/runtime/row_batch.h | 16 +- be/src/runtime/string_value_ir.cpp | 29 -- be/src/runtime/tmp_file_mgr.cc | 9 - be/src/runtime/tmp_file_mgr.h | 3 - be/src/runtime/types.cpp | 50 ++-- be/src/runtime/types.h | 13 +- be/src/udf/CMakeLists.txt | 4 +- be/src/udf/udf.cpp | 57 +++- be/src/udf/udf_ir.cpp | 83 ------ be/src/util/bfd_parser.cpp | 11 - be/src/util/disk_info.h | 16 -- be/src/util/error_util.cc | 160 +---------- be/src/util/error_util.h | 139 ---------- be/src/util/hash_util.hpp | 4 +- be/src/util/hash_util_ir.cpp | 39 --- be/src/util/sse2neon.h | 27 +- be/src/vec/exec/vset_operation_node.h | 5 +- be/src/vec/io/io_helper.h | 10 - be/test/vec/aggregate_functions/agg_test.cpp | 6 +- 73 files changed, 945 insertions(+), 2679 deletions(-) delete mode 100644 be/src/codegen/doris_ir.h delete mode 100644 be/src/common/hdfs.h delete mode 100644 be/src/exec/aggregation_node_ir.cpp delete mode 100644 be/src/exec/hash_join_node_ir.cpp delete mode 100644 be/src/exec/partitioned_aggregation_node_ir.cc delete mode 100644 be/src/exec/partitioned_hash_table_ir.cc delete mode 100644 be/src/exec/scanner_ir.cpp delete mode 100644 be/src/exec/union_node_ir.cpp delete mode 100644 be/src/exprs/conditional_functions_ir.cpp delete mode 100644 be/src/exprs/expr_ir.cpp delete mode 100644 be/src/runtime/buffered_tuple_stream2_ir.cc delete mode 100644 be/src/runtime/raw_value_ir.cpp delete mode 100644 be/src/runtime/string_value_ir.cpp delete mode 100644 be/src/udf/udf_ir.cpp delete mode 100644 be/src/util/hash_util_ir.cpp diff --git a/be/src/codegen/doris_ir.h b/be/src/codegen/doris_ir.h deleted file mode 100644 index 620beb92a1ac3e..00000000000000 --- a/be/src/codegen/doris_ir.h +++ /dev/null @@ -1,34 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#ifndef DORIS_BE_SRC_QUERY_CODGEN_DORIS_IR_H -#define DORIS_BE_SRC_QUERY_CODGEN_DORIS_IR_H -#ifdef IR_COMPILE -// For cross compiling to IR, we need functions decorated in specific ways. For -// functions that we will replace with codegen, we need them not inlined (otherwise -// we can't find the function by name. For functions where the non-codegen'd version -// is too long for the compiler to inline, we might still want to inline it since -// the codegen'd version is suitable for inling. -// In the non-ir case (g++), we will just default to whatever the compiler thought -// best at that optimization setting. -#define IR_NO_INLINE __attribute__((noinline)) -#define IR_ALWAYS_INLINE __attribute__((always_inline)) -#else -#define IR_NO_INLINE -#define IR_ALWAYS_INLINE -#endif -#endif diff --git a/be/src/common/hdfs.h b/be/src/common/hdfs.h deleted file mode 100644 index f41fc4a1d6cccd..00000000000000 --- a/be/src/common/hdfs.h +++ /dev/null @@ -1,31 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#ifndef DORIS_BE_SRC_COMMON_COMMON_HDFS_H -#define DORIS_BE_SRC_COMMON_COMMON_HDFS_H - -// This is a wrapper around the hdfs header. When we are compiling to IR, -// we don't want to pull in the hdfs headers. We only need the headers -// for the typedefs which we will replicate here -// TODO: is this the cleanest way? -#ifdef IR_COMPILE -typedef void* hdfsFS; -typedef void* hdfsFile; -#else -#endif - -#endif diff --git a/be/src/common/logging.h b/be/src/common/logging.h index 7c4d35e975adc1..573af24eefd46b 100644 --- a/be/src/common/logging.h +++ b/be/src/common/logging.h @@ -15,34 +15,8 @@ // specific language governing permissions and limitations // under the License. -#ifndef IMPALA_COMMON_LOGGING_H -#define IMPALA_COMMON_LOGGING_H +#pragma once -// This is a wrapper around the glog header. When we are compiling to IR, -// we don't want to pull in the glog headers. Pulling them in causes linking -// issues when we try to dynamically link the codegen'd functions. -#ifdef IR_COMPILE -#include -#define DCHECK(condition) \ - while (false) std::cout -#define DCHECK_EQ(a, b) \ - while (false) std::cout -#define DCHECK_NE(a, b) \ - while (false) std::cout -#define DCHECK_GT(a, b) \ - while (false) std::cout -#define DCHECK_LT(a, b) \ - while (false) std::cout -#define DCHECK_GE(a, b) \ - while (false) std::cout -#define DCHECK_LE(a, b) \ - while (false) std::cout -// Similar to how glog defines DCHECK for release. -#define LOG(level) \ - while (false) std::cout -#define VLOG(level) \ - while (false) std::cout -#else // GLOG defines this based on the system but doesn't check if it's already // been defined. undef it first to avoid warnings. // glog MUST be included before gflags. Instead of including them, @@ -52,7 +26,6 @@ // function to get the stack trace. #include #undef MutexLock -#endif // Define VLOG levels. We want display per-row info less than per-file which // is less than per-query. For now per-connection is the same as per-query. @@ -67,7 +40,6 @@ #define VLOG_NOTICE VLOG(3) #define VLOG_CRITICAL VLOG(1) - #define VLOG_CONNECTION_IS_ON VLOG_IS_ON(1) #define VLOG_RPC_IS_ON VLOG_IS_ON(8) #define VLOG_QUERY_IS_ON VLOG_IS_ON(1) @@ -78,7 +50,6 @@ #define VLOG_NOTICE_IS_ON VLOG_IS_ON(3) #define VLOG_CRITICAL_IS_ON VLOG_IS_ON(1) - /// Define a wrapper around DCHECK for strongly typed enums that print a useful error /// message on failure. #define DCHECK_ENUM_EQ(a, b) \ @@ -86,4 +57,3 @@ << static_cast(b) << " ]" #include -#endif diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt index 30ad7eb81338e5..36e2af6693f71f 100644 --- a/be/src/exec/CMakeLists.txt +++ b/be/src/exec/CMakeLists.txt @@ -23,7 +23,6 @@ set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/exec") set(EXEC_FILES aggregation_node.cpp - aggregation_node_ir.cpp analytic_eval_node.cpp blocking_join_node.cpp broker_scan_node.cpp @@ -40,7 +39,6 @@ set(EXEC_FILES exec_node.cpp exchange_node.cpp hash_join_node.cpp - hash_join_node_ir.cpp hash_table.cpp local_file_reader.cpp merge_node.cpp @@ -70,7 +68,6 @@ set(EXEC_FILES es/es_query_builder.cpp spill_sort_node.cc union_node.cpp - union_node_ir.cpp set_operation_node.cpp intersect_node.cpp except_node.cpp @@ -94,9 +91,7 @@ set(EXEC_FILES schema_scanner/schema_partitions_scanner.cpp partitioned_hash_table.cc - partitioned_hash_table_ir.cc partitioned_aggregation_node.cc - partitioned_aggregation_node_ir.cc odbc_scan_node.cpp local_file_writer.cpp broker_writer.cpp diff --git a/be/src/exec/aggregation_node.cpp b/be/src/exec/aggregation_node.cpp index 3cadea45f8f485..82c8f6c267d35f 100644 --- a/be/src/exec/aggregation_node.cpp +++ b/be/src/exec/aggregation_node.cpp @@ -389,19 +389,6 @@ void AggregationNode::update_tuple(Tuple* tuple, TupleRow* row) { DCHECK(tuple != nullptr); AggFnEvaluator::add(_aggregate_evaluators, _agg_fn_ctxs, row, tuple); -#if 0 - std::vector::const_iterator evaluator; - int i = 0; - for (evaluator = _aggregate_evaluators.begin(); - evaluator != _aggregate_evaluators.end(); ++evaluator, ++i) { - (*evaluator)->choose_update_or_merge(_agg_fn_ctxs[i], row, tuple); - //if (_is_merge) { - // (*evaluator)->merge(_agg_fn_ctxs[i], row, tuple, pool); - //} else { - // (*evaluator)->update(_agg_fn_ctxs[i], row, tuple, pool); - //} - } -#endif } Tuple* AggregationNode::finalize_tuple(Tuple* tuple, MemPool* pool) { @@ -451,4 +438,27 @@ void AggregationNode::push_down_predicate(RuntimeState* state, std::listnum_rows(); ++i) { + update_tuple(_singleton_output_tuple, batch->get_row(i)); + } +} + +void AggregationNode::process_row_batch_with_grouping(RowBatch* batch, MemPool* pool) { + for (int i = 0; i < batch->num_rows(); ++i) { + TupleRow* row = batch->get_row(i); + Tuple* agg_tuple = nullptr; + HashTable::Iterator it = _hash_tbl->find(row); + + if (it.at_end()) { + agg_tuple = construct_intermediate_tuple(); + _hash_tbl->insert(reinterpret_cast(&agg_tuple)); + } else { + agg_tuple = it.get_row()->get_tuple(0); + } + + update_tuple(agg_tuple, row); + } +} + } // namespace doris diff --git a/be/src/exec/aggregation_node_ir.cpp b/be/src/exec/aggregation_node_ir.cpp deleted file mode 100644 index c105760c2ef88a..00000000000000 --- a/be/src/exec/aggregation_node_ir.cpp +++ /dev/null @@ -1,50 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "exec/aggregation_node.h" -#include "exec/hash_table.hpp" -#include "runtime/row_batch.h" -#include "runtime/runtime_state.h" -#include "runtime/tuple.h" -#include "runtime/tuple_row.h" - -namespace doris { - -void AggregationNode::process_row_batch_no_grouping(RowBatch* batch, MemPool* pool) { - for (int i = 0; i < batch->num_rows(); ++i) { - update_tuple(_singleton_output_tuple, batch->get_row(i)); - } -} - -void AggregationNode::process_row_batch_with_grouping(RowBatch* batch, MemPool* pool) { - for (int i = 0; i < batch->num_rows(); ++i) { - TupleRow* row = batch->get_row(i); - Tuple* agg_tuple = nullptr; - HashTable::Iterator it = _hash_tbl->find(row); - - if (it.at_end()) { - agg_tuple = construct_intermediate_tuple(); - _hash_tbl->insert(reinterpret_cast(&agg_tuple)); - } else { - agg_tuple = it.get_row()->get_tuple(0); - } - - update_tuple(agg_tuple, row); - } -} - -} // namespace doris diff --git a/be/src/exec/hash_join_node.cpp b/be/src/exec/hash_join_node.cpp index 389ca98a3c1123..f7e8bbcc649858 100644 --- a/be/src/exec/hash_join_node.cpp +++ b/be/src/exec/hash_join_node.cpp @@ -19,6 +19,7 @@ #include #include +#include "common/utils.h" #include "exec/hash_table.hpp" #include "exprs/expr.h" #include "exprs/expr_context.h" @@ -85,9 +86,8 @@ Status HashJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { } for (const auto& filter_desc : _runtime_filter_descs) { - RETURN_IF_ERROR(state->runtime_filter_mgr()->regist_filter(RuntimeFilterRole::PRODUCER, - filter_desc, - state->query_options())); + RETURN_IF_ERROR(state->runtime_filter_mgr()->regist_filter( + RuntimeFilterRole::PRODUCER, filter_desc, state->query_options())); } return Status::OK(); @@ -171,11 +171,6 @@ Status HashJoinNode::close(RuntimeState* state) { Expr::close(_build_expr_ctxs, state); Expr::close(_probe_expr_ctxs, state); Expr::close(_other_join_conjunct_ctxs, state); -#if 0 - for (auto iter : _push_down_expr_ctxs) { - iter->close(state); - } -#endif return ExecNode::close(state); } @@ -646,4 +641,146 @@ void HashJoinNode::create_output_row(TupleRow* out, TupleRow* probe, TupleRow* b } } +// Wrapper around ExecNode's eval conjuncts with a different function name. +// This lets us distinguish between the join conjuncts vs. non-join conjuncts +// for codegen. +// Note: don't declare this static. LLVM will pick the fastcc calling convention and +// we will not be able to replace the functions with codegen'd versions. +// TODO: explicitly set the calling convention? +// TODO: investigate using fastcc for all codegen internal functions? +bool eval_other_join_conjuncts(ExprContext* const* ctxs, int num_ctxs, TupleRow* row) { + return ExecNode::eval_conjuncts(ctxs, num_ctxs, row); +} + +// CreateOutputRow, EvalOtherJoinConjuncts, and EvalConjuncts are replaced by +// codegen. +int HashJoinNode::process_probe_batch(RowBatch* out_batch, RowBatch* probe_batch, + int max_added_rows) { + // This path does not handle full outer or right outer joins + DCHECK(!_match_all_build); + + int row_idx = out_batch->add_rows(max_added_rows); + DCHECK(row_idx != RowBatch::INVALID_ROW_INDEX); + uint8_t* out_row_mem = reinterpret_cast(out_batch->get_row(row_idx)); + TupleRow* out_row = reinterpret_cast(out_row_mem); + + int rows_returned = 0; + int probe_rows = probe_batch->num_rows(); + + ExprContext* const* other_conjunct_ctxs = &_other_join_conjunct_ctxs[0]; + int num_other_conjunct_ctxs = _other_join_conjunct_ctxs.size(); + + ExprContext* const* conjunct_ctxs = &_conjunct_ctxs[0]; + int num_conjunct_ctxs = _conjunct_ctxs.size(); + + while (true) { + // Create output row for each matching build row + while (_hash_tbl_iterator.has_next()) { + TupleRow* matched_build_row = _hash_tbl_iterator.get_row(); + _hash_tbl_iterator.next(); + create_output_row(out_row, _current_probe_row, matched_build_row); + + if (!eval_other_join_conjuncts(other_conjunct_ctxs, num_other_conjunct_ctxs, out_row)) { + continue; + } + + _matched_probe = true; + + // left_anti_join: equal match won't return + if (_join_op == TJoinOp::LEFT_ANTI_JOIN) { + _hash_tbl_iterator = _hash_tbl->end(); + break; + } + + if (eval_conjuncts(conjunct_ctxs, num_conjunct_ctxs, out_row)) { + ++rows_returned; + + // Filled up out batch or hit limit + if (UNLIKELY(rows_returned == max_added_rows)) { + goto end; + } + + // Advance to next out row + out_row_mem += out_batch->row_byte_size(); + out_row = reinterpret_cast(out_row_mem); + } + + // Handle left semi-join + if (_match_one_build) { + _hash_tbl_iterator = _hash_tbl->end(); + break; + } + } + + // Handle left outer-join and left semi-join + if ((!_matched_probe && _match_all_probe) || + ((!_matched_probe && _join_op == TJoinOp::LEFT_ANTI_JOIN))) { + create_output_row(out_row, _current_probe_row, nullptr); + _matched_probe = true; + + if (ExecNode::eval_conjuncts(conjunct_ctxs, num_conjunct_ctxs, out_row)) { + ++rows_returned; + + if (UNLIKELY(rows_returned == max_added_rows)) { + goto end; + } + + // Advance to next out row + out_row_mem += out_batch->row_byte_size(); + out_row = reinterpret_cast(out_row_mem); + } + } + + if (!_hash_tbl_iterator.has_next()) { + // Advance to the next probe row + if (UNLIKELY(_probe_batch_pos == probe_rows)) { + goto end; + } + if (++_probe_counter % RELEASE_CONTEXT_COUNTER == 0) { + ExprContext::free_local_allocations(_probe_expr_ctxs); + ExprContext::free_local_allocations(_build_expr_ctxs); + } + _current_probe_row = probe_batch->get_row(_probe_batch_pos++); + _hash_tbl_iterator = _hash_tbl->find(_current_probe_row); + _matched_probe = false; + } + } + +end: + + if (_match_one_build && _matched_probe) { + _hash_tbl_iterator = _hash_tbl->end(); + } + + out_batch->commit_rows(rows_returned); + return rows_returned; +} + +// when build table has too many duplicated rows, the collisions will be very serious, +// so in some case will don't need to store duplicated value in hash table, we can build an unique one +Status HashJoinNode::process_build_batch(RuntimeState* state, RowBatch* build_batch) { + // insert build row into our hash table + if (_build_unique) { + for (int i = 0; i < build_batch->num_rows(); ++i) { + // _hash_tbl->insert_unique(build_batch->get_row(i)); + TupleRow* tuple_row = nullptr; + if (_hash_tbl->emplace_key(build_batch->get_row(i), &tuple_row)) { + build_batch->get_row(i)->deep_copy(tuple_row, + child(1)->row_desc().tuple_descriptors(), + _build_pool.get(), false); + } + } + RETURN_IF_LIMIT_EXCEEDED(state, "Hash join, while constructing the hash table."); + } else { + // take ownership of tuple data of build_batch + _build_pool->acquire_data(build_batch->tuple_data_pool(), false); + RETURN_IF_LIMIT_EXCEEDED(state, "Hash join, while constructing the hash table."); + + for (int i = 0; i < build_batch->num_rows(); ++i) { + _hash_tbl->insert(build_batch->get_row(i)); + } + } + return Status::OK(); +} + } // namespace doris diff --git a/be/src/exec/hash_join_node_ir.cpp b/be/src/exec/hash_join_node_ir.cpp deleted file mode 100644 index b31f7961a9add2..00000000000000 --- a/be/src/exec/hash_join_node_ir.cpp +++ /dev/null @@ -1,172 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "common/utils.h" -#include "exec/hash_join_node.h" -#include "exec/hash_table.hpp" -#include "exprs/expr_context.h" -#include "runtime/row_batch.h" -#include "runtime/runtime_state.h" -#include "runtime/tuple_row.h" - -namespace doris { - -// Functions in this file are cross compiled to IR with clang. - -// Wrapper around ExecNode's eval conjuncts with a different function name. -// This lets us distinguish between the join conjuncts vs. non-join conjuncts -// for codegen. -// Note: don't declare this static. LLVM will pick the fastcc calling convention and -// we will not be able to replace the functions with codegen'd versions. -// TODO: explicitly set the calling convention? -// TODO: investigate using fastcc for all codegen internal functions? -bool IR_NO_INLINE eval_other_join_conjuncts(ExprContext* const* ctxs, int num_ctxs, TupleRow* row) { - return ExecNode::eval_conjuncts(ctxs, num_ctxs, row); -} - -// CreateOutputRow, EvalOtherJoinConjuncts, and EvalConjuncts are replaced by -// codegen. -int HashJoinNode::process_probe_batch(RowBatch* out_batch, RowBatch* probe_batch, - int max_added_rows) { - // This path does not handle full outer or right outer joins - DCHECK(!_match_all_build); - - int row_idx = out_batch->add_rows(max_added_rows); - DCHECK(row_idx != RowBatch::INVALID_ROW_INDEX); - uint8_t* out_row_mem = reinterpret_cast(out_batch->get_row(row_idx)); - TupleRow* out_row = reinterpret_cast(out_row_mem); - - int rows_returned = 0; - int probe_rows = probe_batch->num_rows(); - - ExprContext* const* other_conjunct_ctxs = &_other_join_conjunct_ctxs[0]; - int num_other_conjunct_ctxs = _other_join_conjunct_ctxs.size(); - - ExprContext* const* conjunct_ctxs = &_conjunct_ctxs[0]; - int num_conjunct_ctxs = _conjunct_ctxs.size(); - - while (true) { - // Create output row for each matching build row - while (_hash_tbl_iterator.has_next()) { - TupleRow* matched_build_row = _hash_tbl_iterator.get_row(); - _hash_tbl_iterator.next(); - create_output_row(out_row, _current_probe_row, matched_build_row); - - if (!eval_other_join_conjuncts(other_conjunct_ctxs, num_other_conjunct_ctxs, out_row)) { - continue; - } - - _matched_probe = true; - - // left_anti_join: equal match won't return - if (_join_op == TJoinOp::LEFT_ANTI_JOIN) { - _hash_tbl_iterator = _hash_tbl->end(); - break; - } - - if (eval_conjuncts(conjunct_ctxs, num_conjunct_ctxs, out_row)) { - ++rows_returned; - - // Filled up out batch or hit limit - if (UNLIKELY(rows_returned == max_added_rows)) { - goto end; - } - - // Advance to next out row - out_row_mem += out_batch->row_byte_size(); - out_row = reinterpret_cast(out_row_mem); - } - - // Handle left semi-join - if (_match_one_build) { - _hash_tbl_iterator = _hash_tbl->end(); - break; - } - } - - // Handle left outer-join and left semi-join - if ((!_matched_probe && _match_all_probe) || - ((!_matched_probe && _join_op == TJoinOp::LEFT_ANTI_JOIN))) { - create_output_row(out_row, _current_probe_row, nullptr); - _matched_probe = true; - - if (ExecNode::eval_conjuncts(conjunct_ctxs, num_conjunct_ctxs, out_row)) { - ++rows_returned; - - if (UNLIKELY(rows_returned == max_added_rows)) { - goto end; - } - - // Advance to next out row - out_row_mem += out_batch->row_byte_size(); - out_row = reinterpret_cast(out_row_mem); - } - } - - if (!_hash_tbl_iterator.has_next()) { - // Advance to the next probe row - if (UNLIKELY(_probe_batch_pos == probe_rows)) { - goto end; - } - if (++_probe_counter % RELEASE_CONTEXT_COUNTER == 0) { - ExprContext::free_local_allocations(_probe_expr_ctxs); - ExprContext::free_local_allocations(_build_expr_ctxs); - } - _current_probe_row = probe_batch->get_row(_probe_batch_pos++); - _hash_tbl_iterator = _hash_tbl->find(_current_probe_row); - _matched_probe = false; - } - } - -end: - - if (_match_one_build && _matched_probe) { - _hash_tbl_iterator = _hash_tbl->end(); - } - - out_batch->commit_rows(rows_returned); - return rows_returned; -} - -// when build table has too many duplicated rows, the collisions will be very serious, -// so in some case will don't need to store duplicated value in hash table, we can build an unique one -Status HashJoinNode::process_build_batch(RuntimeState* state, RowBatch* build_batch) { - // insert build row into our hash table - if (_build_unique) { - for (int i = 0; i < build_batch->num_rows(); ++i) { - // _hash_tbl->insert_unique(build_batch->get_row(i)); - TupleRow* tuple_row = nullptr; - if (_hash_tbl->emplace_key(build_batch->get_row(i), &tuple_row)) { - build_batch->get_row(i)->deep_copy(tuple_row, - child(1)->row_desc().tuple_descriptors(), - _build_pool.get(), false); - } - } - RETURN_IF_LIMIT_EXCEEDED(state, "Hash join, while constructing the hash table."); - } else { - // take ownership of tuple data of build_batch - _build_pool->acquire_data(build_batch->tuple_data_pool(), false); - RETURN_IF_LIMIT_EXCEEDED(state, "Hash join, while constructing the hash table."); - - for (int i = 0; i < build_batch->num_rows(); ++i) { - _hash_tbl->insert(build_batch->get_row(i)); - } - } - return Status::OK(); -} - -} // namespace doris diff --git a/be/src/exec/hash_table.cpp b/be/src/exec/hash_table.cpp index 4aabbc58ec286d..36c3d7b76eaa9c 100644 --- a/be/src/exec/hash_table.cpp +++ b/be/src/exec/hash_table.cpp @@ -45,8 +45,7 @@ HashTable::HashTable(const std::vector& build_expr_ctxs, _current_used(0), _total_capacity(num_buckets), _exceeded_limit(false), - _mem_tracker(mem_tracker), - _mem_limit_exceeded(false) { + _mem_tracker(mem_tracker) { DCHECK(_mem_tracker); DCHECK_EQ(_build_expr_ctxs.size(), _probe_expr_ctxs.size()); @@ -176,16 +175,16 @@ bool HashTable::equals(TupleRow* build_row) { return true; } -void HashTable::resize_buckets(int64_t num_buckets) { +Status HashTable::resize_buckets(int64_t num_buckets) { DCHECK_EQ((num_buckets & (num_buckets - 1)), 0) << "num_buckets must be a power of 2"; int64_t old_num_buckets = _num_buckets; int64_t delta_bytes = (num_buckets - old_num_buckets) * sizeof(Bucket); Status st = _mem_tracker->TryConsume(delta_bytes); - WARN_IF_ERROR(st, "resize bucket failed"); if (!st) { + LOG_EVERY_N(WARNING, 100) << "resize bucket failed: " << st.to_string(); mem_limit_exceeded(delta_bytes); - return; + return st; } _buckets.resize(num_buckets); @@ -230,6 +229,7 @@ void HashTable::resize_buckets(int64_t num_buckets) { _num_buckets = num_buckets; _num_buckets_till_resize = MAX_BUCKET_OCCUPANCY_FRACTION * _num_buckets; + return Status::OK(); } void HashTable::grow_node_array() { @@ -251,11 +251,7 @@ void HashTable::grow_node_array() { } void HashTable::mem_limit_exceeded(int64_t allocation_size) { - _mem_limit_exceeded = true; _exceeded_limit = true; - // if (_state != nullptr) { - // _state->set_mem_limit_exceeded(_mem_tracker, allocation_size); - // } } std::string HashTable::debug_string(bool skip_empty, const RowDescriptor* desc) { diff --git a/be/src/exec/hash_table.h b/be/src/exec/hash_table.h index d150bb50ff0dfc..973be73c30f72d 100644 --- a/be/src/exec/hash_table.h +++ b/be/src/exec/hash_table.h @@ -20,9 +20,9 @@ #include -#include "codegen/doris_ir.h" #include "common/logging.h" #include "common/object_pool.h" +#include "common/status.h" #include "util/hash_util.hpp" namespace doris { @@ -101,7 +101,7 @@ class HashTable { // Insert row into the hash table. Row will be evaluated over _build_expr_ctxs // This will grow the hash table if necessary - void IR_ALWAYS_INLINE insert(TupleRow* row) { + void insert(TupleRow* row) { if (_num_filled_buckets > _num_buckets_till_resize) { // TODO: next prime instead of double? resize_buckets(_num_buckets * 2); @@ -111,13 +111,13 @@ class HashTable { } // Insert row into the hash table. if the row is already exist will not insert - void IR_ALWAYS_INLINE insert_unique(TupleRow* row) { + void insert_unique(TupleRow* row) { if (find(row, false) == end()) { insert(row); } } - bool IR_ALWAYS_INLINE emplace_key(TupleRow* row, TupleRow** key_addr); + bool emplace_key(TupleRow* row, TupleRow** key_addr); // Returns the start iterator for all rows that match 'probe_row'. 'probe_row' is // evaluated with _probe_expr_ctxs. The iterator can be iterated until HashTable::end() @@ -128,7 +128,7 @@ class HashTable { // Advancing the returned iterator will go to the next matching row. The matching // rows are evaluated lazily (i.e. computed as the Iterator is moved). // Returns HashTable::end() if there is no match. - Iterator IR_ALWAYS_INLINE find(TupleRow* probe_row, bool probe = true); + Iterator find(TupleRow* probe_row, bool probe = true); // Returns number of elements in the hash table int64_t size() { return _num_nodes; } @@ -194,7 +194,7 @@ class HashTable { // from a Find, this will lazily evaluate that bucket, only returning // TupleRows that match the current scan row. template - void IR_ALWAYS_INLINE next(); + void next(); // Returns the current row or nullptr if at end. TupleRow* get_row() { @@ -300,10 +300,10 @@ class HashTable { Bucket* next_bucket(int64_t* bucket_idx); // Resize the hash table to 'num_buckets' - void resize_buckets(int64_t num_buckets); + Status resize_buckets(int64_t num_buckets); // Insert row into the hash table - void IR_ALWAYS_INLINE insert_impl(TupleRow* row); + void insert_impl(TupleRow* row); // Chains the node at 'node_idx' to 'bucket'. Nodes in a bucket are chained // as a linked list; this places the new node at the beginning of the list. @@ -323,16 +323,16 @@ class HashTable { // cross compiled because we need to be able to differentiate between EvalBuildRow // and EvalProbeRow by name and the _build_expr_ctxs/_probe_expr_ctxs are baked into // the codegen'd function. - bool IR_NO_INLINE eval_build_row(TupleRow* row) { return eval_row(row, _build_expr_ctxs); } + bool eval_build_row(TupleRow* row) { return eval_row(row, _build_expr_ctxs); } // Evaluate 'row' over _probe_expr_ctxs caching the results in '_expr_values_buffer' // This will be replaced by codegen. - bool IR_NO_INLINE eval_probe_row(TupleRow* row) { return eval_row(row, _probe_expr_ctxs); } + bool eval_probe_row(TupleRow* row) { return eval_row(row, _probe_expr_ctxs); } // Compute the hash of the values in _expr_values_buffer. // This will be replaced by codegen. We don't want this inlined for replacing // with codegen'd functions so the function name does not change. - uint32_t IR_NO_INLINE hash_current_row() { + uint32_t hash_current_row() { if (_var_result_begin == -1) { // This handles NULLs implicitly since a constant seed value was put // into results buffer for nulls. @@ -391,9 +391,6 @@ class HashTable { bool _exceeded_limit; // true if any of _mem_trackers[].limit_exceeded() std::shared_ptr _mem_tracker; - // Set to true if the hash table exceeds the memory limit. If this is set, - // subsequent calls to Insert() will be ignored. - bool _mem_limit_exceeded; std::vector _buckets; diff --git a/be/src/exec/parquet_scanner.cpp b/be/src/exec/parquet_scanner.cpp index 267397f2137a44..9a85a1253ad969 100644 --- a/be/src/exec/parquet_scanner.cpp +++ b/be/src/exec/parquet_scanner.cpp @@ -20,6 +20,7 @@ #include "exec/broker_reader.h" #include "exec/buffered_reader.h" #include "exec/decompressor.h" +#include "exec/hdfs_reader_writer.h" #include "exec/local_file_reader.h" #include "exec/parquet_reader.h" #include "exec/s3_reader.h" @@ -32,17 +33,6 @@ #include "runtime/stream_load/load_stream_mgr.h" #include "runtime/stream_load/stream_load_pipe.h" #include "runtime/tuple.h" -#include "exec/parquet_reader.h" -#include "exprs/expr.h" -#include "exec/text_converter.h" -#include "exec/text_converter.hpp" -#include "exec/local_file_reader.h" -#include "exec/broker_reader.h" -#include "exec/buffered_reader.h" -#include "exec/decompressor.h" -#include "exec/parquet_reader.h" - -#include "exec/hdfs_reader_writer.h" namespace doris { @@ -50,8 +40,7 @@ ParquetScanner::ParquetScanner(RuntimeState* state, RuntimeProfile* profile, const TBrokerScanRangeParams& params, const std::vector& ranges, const std::vector& broker_addresses, - const std::vector& pre_filter_texprs, - ScannerCounter* counter) + const std::vector& pre_filter_texprs, ScannerCounter* counter) : BaseScanner(state, profile, params, pre_filter_texprs, counter), _ranges(ranges), _broker_addresses(broker_addresses), @@ -129,7 +118,8 @@ Status ParquetScanner::open_next_reader() { } case TFileType::FILE_HDFS: { FileReader* reader; - RETURN_IF_ERROR(HdfsReaderWriter::create_reader(range.hdfs_params, range.path, range.start_offset, &reader)); + RETURN_IF_ERROR(HdfsReaderWriter::create_reader(range.hdfs_params, range.path, + range.start_offset, &reader)); file_reader.reset(reader); break; } @@ -139,27 +129,17 @@ Status ParquetScanner::open_next_reader() { if (range.__isset.file_size) { file_size = range.file_size; } - file_reader.reset(new BufferedReader(_profile, + file_reader.reset(new BufferedReader( + _profile, new BrokerReader(_state->exec_env(), _broker_addresses, _params.properties, range.path, range.start_offset, file_size))); break; } case TFileType::FILE_S3: { - file_reader.reset(new BufferedReader(_profile, - new S3Reader(_params.properties, range.path, range.start_offset))); - break; - } -#if 0 - case TFileType::FILE_STREAM: - { - _stream_load_pipe = _state->exec_env()->load_stream_mgr()->get(range.load_id); - if (_stream_load_pipe == nullptr) { - return Status::InternalError("unknown stream load id"); - } - _cur_file_reader = _stream_load_pipe.get(); + file_reader.reset(new BufferedReader( + _profile, new S3Reader(_params.properties, range.path, range.start_offset))); break; } -#endif default: { std::stringstream ss; ss << "Unknown file type, type=" << range.file_type; diff --git a/be/src/exec/partitioned_aggregation_node.cc b/be/src/exec/partitioned_aggregation_node.cc index 68f536ffda2ccd..98651998b6123b 100644 --- a/be/src/exec/partitioned_aggregation_node.cc +++ b/be/src/exec/partitioned_aggregation_node.cc @@ -1048,7 +1048,7 @@ Tuple* PartitionedAggregationNode::GetOutputTuple(const vectorrows_returned() == 0); + grouping_exprs_.size() == 0 && child(0)->rows_returned() == 0); } else { NewAggFnEvaluator::Serialize(agg_fn_evals, tuple); } @@ -1473,4 +1473,228 @@ void PartitionedAggregationNode::ClosePartitions() { template Status PartitionedAggregationNode::AppendSpilledRow(Partition*, TupleRow*); template Status PartitionedAggregationNode::AppendSpilledRow(Partition*, TupleRow*); +Status PartitionedAggregationNode::ProcessBatchNoGrouping(RowBatch* batch) { + Tuple* output_tuple = singleton_output_tuple_; + FOREACH_ROW(batch, 0, batch_iter) { + UpdateTuple(agg_fn_evals_.data(), output_tuple, batch_iter.get()); + } + return Status::OK(); +} + +template +Status PartitionedAggregationNode::ProcessBatch(RowBatch* batch, PartitionedHashTableCtx* ht_ctx) { + DCHECK(!hash_partitions_.empty()); + DCHECK(!is_streaming_preagg_); + + // Make sure that no resizes will happen when inserting individual rows to the hash + // table of each partition by pessimistically assuming that all the rows in each batch + // will end up to the same partition. + // TODO: Once we have a histogram with the number of rows per partition, we will have + // accurate resize calls. + RETURN_IF_ERROR(CheckAndResizeHashPartitions(AGGREGATED_ROWS, batch->num_rows(), ht_ctx)); + + PartitionedHashTableCtx::ExprValuesCache* expr_vals_cache = ht_ctx->expr_values_cache(); + const int cache_size = expr_vals_cache->capacity(); + const int num_rows = batch->num_rows(); + for (int group_start = 0; group_start < num_rows; group_start += cache_size) { + EvalAndHashPrefetchGroup(batch, group_start, ht_ctx); + + FOREACH_ROW_LIMIT(batch, group_start, cache_size, batch_iter) { + RETURN_IF_ERROR(ProcessRow(batch_iter.get(), ht_ctx)); + expr_vals_cache->NextRow(); + } + ht_ctx->expr_results_pool_->clear(); + DCHECK(expr_vals_cache->AtEnd()); + } + return Status::OK(); +} + +template +void PartitionedAggregationNode::EvalAndHashPrefetchGroup(RowBatch* batch, int start_row_idx, + PartitionedHashTableCtx* ht_ctx) { + PartitionedHashTableCtx::ExprValuesCache* expr_vals_cache = ht_ctx->expr_values_cache(); + const int cache_size = expr_vals_cache->capacity(); + + expr_vals_cache->Reset(); + FOREACH_ROW_LIMIT(batch, start_row_idx, cache_size, batch_iter) { + TupleRow* row = batch_iter.get(); + bool is_null; + if (AGGREGATED_ROWS) { + is_null = !ht_ctx->EvalAndHashBuild(row); + } else { + is_null = !ht_ctx->EvalAndHashProbe(row); + } + // Hoist lookups out of non-null branch to speed up non-null case. + const uint32_t hash = expr_vals_cache->CurExprValuesHash(); + const uint32_t partition_idx = hash >> (32 - NUM_PARTITIONING_BITS); + PartitionedHashTable* hash_tbl = GetHashTable(partition_idx); + if (is_null) { + expr_vals_cache->SetRowNull(); + } else if (config::enable_prefetch) { + if (LIKELY(hash_tbl != nullptr)) hash_tbl->PrefetchBucket(hash); + } + expr_vals_cache->NextRow(); + } + + expr_vals_cache->ResetForRead(); +} + +template +Status PartitionedAggregationNode::ProcessRow(TupleRow* row, PartitionedHashTableCtx* ht_ctx) { + PartitionedHashTableCtx::ExprValuesCache* expr_vals_cache = ht_ctx->expr_values_cache(); + // Hoist lookups out of non-null branch to speed up non-null case. + const uint32_t hash = expr_vals_cache->CurExprValuesHash(); + const uint32_t partition_idx = hash >> (32 - NUM_PARTITIONING_BITS); + if (expr_vals_cache->IsRowNull()) return Status::OK(); + // To process this row, we first see if it can be aggregated or inserted into this + // partition's hash table. If we need to insert it and that fails, due to OOM, we + // spill the partition. The partition to spill is not necessarily dst_partition, + // so we can try again to insert the row. + PartitionedHashTable* hash_tbl = GetHashTable(partition_idx); + Partition* dst_partition = hash_partitions_[partition_idx]; + DCHECK(dst_partition != nullptr); + DCHECK_EQ(dst_partition->is_spilled(), hash_tbl == nullptr); + if (hash_tbl == nullptr) { + // This partition is already spilled, just append the row. + return AppendSpilledRow(dst_partition, row); + } + + DCHECK(dst_partition->aggregated_row_stream->is_pinned()); + bool found; + // Find the appropriate bucket in the hash table. There will always be a free + // bucket because we checked the size above. + PartitionedHashTable::Iterator it = hash_tbl->FindBuildRowBucket(ht_ctx, &found); + DCHECK(!it.AtEnd()) << "Hash table had no free buckets"; + if (AGGREGATED_ROWS) { + // If the row is already an aggregate row, it cannot match anything in the + // hash table since we process the aggregate rows first. These rows should + // have been aggregated in the initial pass. + DCHECK(!found); + } else if (found) { + // Row is already in hash table. Do the aggregation and we're done. + UpdateTuple(dst_partition->agg_fn_evals.data(), it.GetTuple(), row); + return Status::OK(); + } + + // If we are seeing this result row for the first time, we need to construct the + // result row and initialize it. + return AddIntermediateTuple(dst_partition, row, hash, it); +} + +template +Status PartitionedAggregationNode::AddIntermediateTuple(Partition* partition, TupleRow* row, + uint32_t hash, + PartitionedHashTable::Iterator insert_it) { + while (true) { + DCHECK(partition->aggregated_row_stream->is_pinned()); + Tuple* intermediate_tuple = ConstructIntermediateTuple( + partition->agg_fn_evals, partition->aggregated_row_stream.get(), + &process_batch_status_); + + if (LIKELY(intermediate_tuple != nullptr)) { + UpdateTuple(partition->agg_fn_evals.data(), intermediate_tuple, row, AGGREGATED_ROWS); + // After copying and initializing the tuple, insert it into the hash table. + insert_it.SetTuple(intermediate_tuple, hash); + return Status::OK(); + } else if (!process_batch_status_.ok()) { + return std::move(process_batch_status_); + } + + // We did not have enough memory to add intermediate_tuple to the stream. + RETURN_IF_ERROR(SpillPartition(AGGREGATED_ROWS)); + if (partition->is_spilled()) { + return AppendSpilledRow(partition, row); + } + } +} + +Status PartitionedAggregationNode::ProcessBatchStreaming(bool needs_serialize, RowBatch* in_batch, + RowBatch* out_batch, + PartitionedHashTableCtx* ht_ctx, + int remaining_capacity[PARTITION_FANOUT]) { + DCHECK(is_streaming_preagg_); + DCHECK_EQ(out_batch->num_rows(), 0); + DCHECK_LE(in_batch->num_rows(), out_batch->capacity()); + + RowBatch::Iterator out_batch_iterator(out_batch, out_batch->num_rows()); + PartitionedHashTableCtx::ExprValuesCache* expr_vals_cache = ht_ctx->expr_values_cache(); + const int num_rows = in_batch->num_rows(); + const int cache_size = expr_vals_cache->capacity(); + for (int group_start = 0; group_start < num_rows; group_start += cache_size) { + EvalAndHashPrefetchGroup(in_batch, group_start, ht_ctx); + + FOREACH_ROW_LIMIT(in_batch, group_start, cache_size, in_batch_iter) { + // Hoist lookups out of non-null branch to speed up non-null case. + TupleRow* in_row = in_batch_iter.get(); + const uint32_t hash = expr_vals_cache->CurExprValuesHash(); + const uint32_t partition_idx = hash >> (32 - NUM_PARTITIONING_BITS); + if (!expr_vals_cache->IsRowNull() && + !TryAddToHashTable(ht_ctx, hash_partitions_[partition_idx], + GetHashTable(partition_idx), in_row, hash, + &remaining_capacity[partition_idx], &process_batch_status_)) { + RETURN_IF_ERROR(std::move(process_batch_status_)); + // Tuple is not going into hash table, add it to the output batch. + Tuple* intermediate_tuple = ConstructIntermediateTuple( + agg_fn_evals_, out_batch->tuple_data_pool(), &process_batch_status_); + if (UNLIKELY(intermediate_tuple == nullptr)) { + DCHECK(!process_batch_status_.ok()); + return std::move(process_batch_status_); + } + UpdateTuple(agg_fn_evals_.data(), intermediate_tuple, in_row); + out_batch_iterator.get()->set_tuple(0, intermediate_tuple); + out_batch_iterator.next(); + out_batch->commit_last_row(); + } + DCHECK(process_batch_status_.ok()); + expr_vals_cache->NextRow(); + } + ht_ctx->expr_results_pool_->clear(); + DCHECK(expr_vals_cache->AtEnd()); + } + if (needs_serialize) { + FOREACH_ROW(out_batch, 0, out_batch_iter) { + NewAggFnEvaluator::Serialize(agg_fn_evals_, out_batch_iter.get()->get_tuple(0)); + } + } + + return Status::OK(); +} + +bool PartitionedAggregationNode::TryAddToHashTable(PartitionedHashTableCtx* ht_ctx, + Partition* partition, + PartitionedHashTable* hash_tbl, TupleRow* in_row, + uint32_t hash, int* remaining_capacity, + Status* status) { + DCHECK(remaining_capacity != nullptr); + DCHECK_EQ(hash_tbl, partition->hash_tbl.get()); + DCHECK_GE(*remaining_capacity, 0); + bool found; + // This is called from ProcessBatchStreaming() so the rows are not aggregated. + PartitionedHashTable::Iterator it = hash_tbl->FindBuildRowBucket(ht_ctx, &found); + Tuple* intermediate_tuple; + if (found) { + intermediate_tuple = it.GetTuple(); + } else if (*remaining_capacity == 0) { + return false; + } else { + intermediate_tuple = ConstructIntermediateTuple( + partition->agg_fn_evals, partition->aggregated_row_stream.get(), status); + if (LIKELY(intermediate_tuple != nullptr)) { + it.SetTuple(intermediate_tuple, hash); + --(*remaining_capacity); + } else { + // Avoid repeatedly trying to add tuples when under memory pressure. + *remaining_capacity = 0; + return false; + } + } + UpdateTuple(partition->agg_fn_evals.data(), intermediate_tuple, in_row); + return true; +} + +// Instantiate required templates. +template Status PartitionedAggregationNode::ProcessBatch(RowBatch*, + PartitionedHashTableCtx*); +template Status PartitionedAggregationNode::ProcessBatch(RowBatch*, PartitionedHashTableCtx*); + } // namespace doris diff --git a/be/src/exec/partitioned_aggregation_node.h b/be/src/exec/partitioned_aggregation_node.h index ff0d823d5a22dc..d6271624182bd6 100644 --- a/be/src/exec/partitioned_aggregation_node.h +++ b/be/src/exec/partitioned_aggregation_node.h @@ -551,7 +551,7 @@ class PartitionedAggregationNode : public ExecNode { /// This function is replaced by codegen. We pass in ht_ctx_.get() as an argument for /// performance. template - Status IR_ALWAYS_INLINE ProcessBatch(RowBatch* batch, PartitionedHashTableCtx* ht_ctx); + Status ProcessBatch(RowBatch* batch, PartitionedHashTableCtx* ht_ctx); /// Evaluates the rows in 'batch' starting at 'start_row_idx' and stores the results in /// the expression values cache in 'ht_ctx'. The number of rows evaluated depends on @@ -566,7 +566,7 @@ class PartitionedAggregationNode : public ExecNode { /// ProcessBatch for codegen to substitute function calls with codegen'd versions. /// May spill partitions if not enough memory is available. template - Status IR_ALWAYS_INLINE ProcessRow(TupleRow* row, PartitionedHashTableCtx* ht_ctx); + Status ProcessRow(TupleRow* row, PartitionedHashTableCtx* ht_ctx); /// Create a new intermediate tuple in partition, initialized with row. ht_ctx is /// the context for the partition's hash table and hash is the precomputed hash of @@ -576,14 +576,14 @@ class PartitionedAggregationNode : public ExecNode { /// to substitute function calls with codegen'd versions. insert_it is an iterator /// for insertion returned from PartitionedHashTable::FindBuildRowBucket(). template - Status IR_ALWAYS_INLINE AddIntermediateTuple(Partition* partition, TupleRow* row, uint32_t hash, - PartitionedHashTable::Iterator insert_it); + Status AddIntermediateTuple(Partition* partition, TupleRow* row, uint32_t hash, + PartitionedHashTable::Iterator insert_it); /// Append a row to a spilled partition. May spill partitions if needed to switch to /// I/O buffers. Selects the correct stream according to the argument. Inlined into /// ProcessBatch(). template - Status IR_ALWAYS_INLINE AppendSpilledRow(Partition* partition, TupleRow* row); + Status AppendSpilledRow(Partition* partition, TupleRow* row); /// Reads all the rows from input_stream and process them by calling ProcessBatch(). template @@ -630,9 +630,9 @@ class PartitionedAggregationNode : public ExecNode { /// keeps track of how many more entries can be added to the hash table so we can avoid /// retrying inserts. It is decremented if an insert succeeds and set to zero if an /// insert fails. If an error occurs, returns false and sets 'status'. - bool IR_ALWAYS_INLINE TryAddToHashTable(PartitionedHashTableCtx* ht_ctx, Partition* partition, - PartitionedHashTable* hash_tbl, TupleRow* in_row, - uint32_t hash, int* remaining_capacity, Status* status); + bool TryAddToHashTable(PartitionedHashTableCtx* ht_ctx, Partition* partition, + PartitionedHashTable* hash_tbl, TupleRow* in_row, uint32_t hash, + int* remaining_capacity, Status* status); /// Initializes hash_partitions_. 'level' is the level for the partitions to create. /// If 'single_partition_idx' is provided, it must be a number in range diff --git a/be/src/exec/partitioned_aggregation_node_ir.cc b/be/src/exec/partitioned_aggregation_node_ir.cc deleted file mode 100644 index ae62c64b6d9172..00000000000000 --- a/be/src/exec/partitioned_aggregation_node_ir.cc +++ /dev/null @@ -1,250 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. -#include "exec/partitioned_aggregation_node.h" -#include "exec/partitioned_hash_table.inline.h" -#include "exprs/expr_context.h" -#include "exprs/new_agg_fn_evaluator.h" -#include "runtime/buffered_tuple_stream3.inline.h" -#include "runtime/row_batch.h" -#include "runtime/tuple_row.h" -#include "util/runtime_profile.h" - -using namespace doris; - -Status PartitionedAggregationNode::ProcessBatchNoGrouping(RowBatch* batch) { - Tuple* output_tuple = singleton_output_tuple_; - FOREACH_ROW(batch, 0, batch_iter) { - UpdateTuple(agg_fn_evals_.data(), output_tuple, batch_iter.get()); - } - return Status::OK(); -} - -template -Status PartitionedAggregationNode::ProcessBatch(RowBatch* batch, PartitionedHashTableCtx* ht_ctx) { - DCHECK(!hash_partitions_.empty()); - DCHECK(!is_streaming_preagg_); - - // Make sure that no resizes will happen when inserting individual rows to the hash - // table of each partition by pessimistically assuming that all the rows in each batch - // will end up to the same partition. - // TODO: Once we have a histogram with the number of rows per partition, we will have - // accurate resize calls. - RETURN_IF_ERROR(CheckAndResizeHashPartitions(AGGREGATED_ROWS, batch->num_rows(), ht_ctx)); - - PartitionedHashTableCtx::ExprValuesCache* expr_vals_cache = ht_ctx->expr_values_cache(); - const int cache_size = expr_vals_cache->capacity(); - const int num_rows = batch->num_rows(); - for (int group_start = 0; group_start < num_rows; group_start += cache_size) { - EvalAndHashPrefetchGroup(batch, group_start, ht_ctx); - - FOREACH_ROW_LIMIT(batch, group_start, cache_size, batch_iter) { - RETURN_IF_ERROR(ProcessRow(batch_iter.get(), ht_ctx)); - expr_vals_cache->NextRow(); - } - ht_ctx->expr_results_pool_->clear(); - DCHECK(expr_vals_cache->AtEnd()); - } - return Status::OK(); -} - -template -void IR_ALWAYS_INLINE PartitionedAggregationNode::EvalAndHashPrefetchGroup( - RowBatch* batch, int start_row_idx, PartitionedHashTableCtx* ht_ctx) { - PartitionedHashTableCtx::ExprValuesCache* expr_vals_cache = ht_ctx->expr_values_cache(); - const int cache_size = expr_vals_cache->capacity(); - - expr_vals_cache->Reset(); - FOREACH_ROW_LIMIT(batch, start_row_idx, cache_size, batch_iter) { - TupleRow* row = batch_iter.get(); - bool is_null; - if (AGGREGATED_ROWS) { - is_null = !ht_ctx->EvalAndHashBuild(row); - } else { - is_null = !ht_ctx->EvalAndHashProbe(row); - } - // Hoist lookups out of non-null branch to speed up non-null case. - const uint32_t hash = expr_vals_cache->CurExprValuesHash(); - const uint32_t partition_idx = hash >> (32 - NUM_PARTITIONING_BITS); - PartitionedHashTable* hash_tbl = GetHashTable(partition_idx); - if (is_null) { - expr_vals_cache->SetRowNull(); - } else if (config::enable_prefetch) { - if (LIKELY(hash_tbl != nullptr)) hash_tbl->PrefetchBucket(hash); - } - expr_vals_cache->NextRow(); - } - - expr_vals_cache->ResetForRead(); -} - -template -Status PartitionedAggregationNode::ProcessRow(TupleRow* row, PartitionedHashTableCtx* ht_ctx) { - PartitionedHashTableCtx::ExprValuesCache* expr_vals_cache = ht_ctx->expr_values_cache(); - // Hoist lookups out of non-null branch to speed up non-null case. - const uint32_t hash = expr_vals_cache->CurExprValuesHash(); - const uint32_t partition_idx = hash >> (32 - NUM_PARTITIONING_BITS); - if (expr_vals_cache->IsRowNull()) return Status::OK(); - // To process this row, we first see if it can be aggregated or inserted into this - // partition's hash table. If we need to insert it and that fails, due to OOM, we - // spill the partition. The partition to spill is not necessarily dst_partition, - // so we can try again to insert the row. - PartitionedHashTable* hash_tbl = GetHashTable(partition_idx); - Partition* dst_partition = hash_partitions_[partition_idx]; - DCHECK(dst_partition != nullptr); - DCHECK_EQ(dst_partition->is_spilled(), hash_tbl == nullptr); - if (hash_tbl == nullptr) { - // This partition is already spilled, just append the row. - return AppendSpilledRow(dst_partition, row); - } - - DCHECK(dst_partition->aggregated_row_stream->is_pinned()); - bool found; - // Find the appropriate bucket in the hash table. There will always be a free - // bucket because we checked the size above. - PartitionedHashTable::Iterator it = hash_tbl->FindBuildRowBucket(ht_ctx, &found); - DCHECK(!it.AtEnd()) << "Hash table had no free buckets"; - if (AGGREGATED_ROWS) { - // If the row is already an aggregate row, it cannot match anything in the - // hash table since we process the aggregate rows first. These rows should - // have been aggregated in the initial pass. - DCHECK(!found); - } else if (found) { - // Row is already in hash table. Do the aggregation and we're done. - UpdateTuple(dst_partition->agg_fn_evals.data(), it.GetTuple(), row); - return Status::OK(); - } - - // If we are seeing this result row for the first time, we need to construct the - // result row and initialize it. - return AddIntermediateTuple(dst_partition, row, hash, it); -} - -template -Status PartitionedAggregationNode::AddIntermediateTuple(Partition* partition, TupleRow* row, - uint32_t hash, - PartitionedHashTable::Iterator insert_it) { - while (true) { - DCHECK(partition->aggregated_row_stream->is_pinned()); - Tuple* intermediate_tuple = ConstructIntermediateTuple( - partition->agg_fn_evals, partition->aggregated_row_stream.get(), - &process_batch_status_); - - if (LIKELY(intermediate_tuple != nullptr)) { - UpdateTuple(partition->agg_fn_evals.data(), intermediate_tuple, row, AGGREGATED_ROWS); - // After copying and initializing the tuple, insert it into the hash table. - insert_it.SetTuple(intermediate_tuple, hash); - return Status::OK(); - } else if (!process_batch_status_.ok()) { - return std::move(process_batch_status_); - } - - // We did not have enough memory to add intermediate_tuple to the stream. - RETURN_IF_ERROR(SpillPartition(AGGREGATED_ROWS)); - if (partition->is_spilled()) { - return AppendSpilledRow(partition, row); - } - } -} - -Status PartitionedAggregationNode::ProcessBatchStreaming(bool needs_serialize, RowBatch* in_batch, - RowBatch* out_batch, - PartitionedHashTableCtx* ht_ctx, - int remaining_capacity[PARTITION_FANOUT]) { - DCHECK(is_streaming_preagg_); - DCHECK_EQ(out_batch->num_rows(), 0); - DCHECK_LE(in_batch->num_rows(), out_batch->capacity()); - - RowBatch::Iterator out_batch_iterator(out_batch, out_batch->num_rows()); - PartitionedHashTableCtx::ExprValuesCache* expr_vals_cache = ht_ctx->expr_values_cache(); - const int num_rows = in_batch->num_rows(); - const int cache_size = expr_vals_cache->capacity(); - for (int group_start = 0; group_start < num_rows; group_start += cache_size) { - EvalAndHashPrefetchGroup(in_batch, group_start, ht_ctx); - - FOREACH_ROW_LIMIT(in_batch, group_start, cache_size, in_batch_iter) { - // Hoist lookups out of non-null branch to speed up non-null case. - TupleRow* in_row = in_batch_iter.get(); - const uint32_t hash = expr_vals_cache->CurExprValuesHash(); - const uint32_t partition_idx = hash >> (32 - NUM_PARTITIONING_BITS); - if (!expr_vals_cache->IsRowNull() && - !TryAddToHashTable(ht_ctx, hash_partitions_[partition_idx], - GetHashTable(partition_idx), in_row, hash, - &remaining_capacity[partition_idx], &process_batch_status_)) { - RETURN_IF_ERROR(std::move(process_batch_status_)); - // Tuple is not going into hash table, add it to the output batch. - Tuple* intermediate_tuple = ConstructIntermediateTuple( - agg_fn_evals_, out_batch->tuple_data_pool(), &process_batch_status_); - if (UNLIKELY(intermediate_tuple == nullptr)) { - DCHECK(!process_batch_status_.ok()); - return std::move(process_batch_status_); - } - UpdateTuple(agg_fn_evals_.data(), intermediate_tuple, in_row); - out_batch_iterator.get()->set_tuple(0, intermediate_tuple); - out_batch_iterator.next(); - out_batch->commit_last_row(); - } - DCHECK(process_batch_status_.ok()); - expr_vals_cache->NextRow(); - } - ht_ctx->expr_results_pool_->clear(); - DCHECK(expr_vals_cache->AtEnd()); - } - if (needs_serialize) { - FOREACH_ROW(out_batch, 0, out_batch_iter) { - NewAggFnEvaluator::Serialize(agg_fn_evals_, out_batch_iter.get()->get_tuple(0)); - } - } - - return Status::OK(); -} - -bool PartitionedAggregationNode::TryAddToHashTable(PartitionedHashTableCtx* ht_ctx, - Partition* partition, - PartitionedHashTable* hash_tbl, TupleRow* in_row, - uint32_t hash, int* remaining_capacity, - Status* status) { - DCHECK(remaining_capacity != nullptr); - DCHECK_EQ(hash_tbl, partition->hash_tbl.get()); - DCHECK_GE(*remaining_capacity, 0); - bool found; - // This is called from ProcessBatchStreaming() so the rows are not aggregated. - PartitionedHashTable::Iterator it = hash_tbl->FindBuildRowBucket(ht_ctx, &found); - Tuple* intermediate_tuple; - if (found) { - intermediate_tuple = it.GetTuple(); - } else if (*remaining_capacity == 0) { - return false; - } else { - intermediate_tuple = ConstructIntermediateTuple( - partition->agg_fn_evals, partition->aggregated_row_stream.get(), status); - if (LIKELY(intermediate_tuple != nullptr)) { - it.SetTuple(intermediate_tuple, hash); - --(*remaining_capacity); - } else { - // Avoid repeatedly trying to add tuples when under memory pressure. - *remaining_capacity = 0; - return false; - } - } - UpdateTuple(partition->agg_fn_evals.data(), intermediate_tuple, in_row); - return true; -} - -// Instantiate required templates. -template Status PartitionedAggregationNode::ProcessBatch(RowBatch*, - PartitionedHashTableCtx*); -template Status PartitionedAggregationNode::ProcessBatch(RowBatch*, PartitionedHashTableCtx*); diff --git a/be/src/exec/partitioned_hash_table.h b/be/src/exec/partitioned_hash_table.h index 7dd4dc4c21a23c..23a9c3aaab9ee6 100644 --- a/be/src/exec/partitioned_hash_table.h +++ b/be/src/exec/partitioned_hash_table.h @@ -21,7 +21,6 @@ #include #include -#include "codegen/doris_ir.h" #include "common/compiler_util.h" #include "common/logging.h" #include "runtime/buffered_tuple_stream3.h" @@ -158,8 +157,8 @@ class PartitionedHashTableCtx { /// row should be rejected (doesn't need to be processed further) because it contains /// nullptr. These need to be inlined in the IR module so we can find and replace the /// calls to EvalBuildRow()/EvalProbeRow(). - bool IR_ALWAYS_INLINE EvalAndHashBuild(TupleRow* row); - bool IR_ALWAYS_INLINE EvalAndHashProbe(TupleRow* row); + bool EvalAndHashBuild(TupleRow* row); + bool EvalAndHashProbe(TupleRow* row); /// Struct that returns the number of constants replaced by ReplaceConstants(). struct HashTableReplacedConstants { @@ -391,8 +390,7 @@ class PartitionedHashTableCtx { /// Compute the hash of the values in 'expr_values' with nullness 'expr_values_null'. /// This will be replaced by codegen. We don't want this inlined for replacing /// with codegen'd functions so the function name does not change. - uint32_t IR_NO_INLINE HashRow(const uint8_t* expr_values, - const uint8_t* expr_values_null) const noexcept; + uint32_t HashRow(const uint8_t* expr_values, const uint8_t* expr_values_null) const noexcept; /// Wrapper function for calling correct HashUtil function in non-codegen'd case. uint32_t Hash(const void* input, int len, uint32_t hash) const; @@ -402,15 +400,13 @@ class PartitionedHashTableCtx { /// inlined when cross compiled because we need to be able to differentiate between /// EvalBuildRow and EvalProbeRow by name and the build/probe exprs are baked into the /// codegen'd function. - bool IR_NO_INLINE EvalBuildRow(TupleRow* row, uint8_t* expr_values, - uint8_t* expr_values_null) noexcept { + bool EvalBuildRow(TupleRow* row, uint8_t* expr_values, uint8_t* expr_values_null) noexcept { return EvalRow(row, build_expr_evals_, expr_values, expr_values_null); } /// Evaluate 'row' over probe exprs, storing the values into 'expr_values' and nullness /// into 'expr_values_null'. This will be replaced by codegen. - bool IR_NO_INLINE EvalProbeRow(TupleRow* row, uint8_t* expr_values, - uint8_t* expr_values_null) noexcept { + bool EvalProbeRow(TupleRow* row, uint8_t* expr_values, uint8_t* expr_values_null) noexcept { return EvalRow(row, probe_expr_evals_, expr_values, expr_values_null); } @@ -429,8 +425,8 @@ class PartitionedHashTableCtx { /// true if all nulls should be treated as equal, regardless of the values of /// 'finds_nulls_'. This will be replaced by codegen. template - bool IR_NO_INLINE Equals(TupleRow* build_row, const uint8_t* expr_values, - const uint8_t* expr_values_null) const noexcept; + bool Equals(TupleRow* build_row, const uint8_t* expr_values, + const uint8_t* expr_values_null) const noexcept; /// Helper function that calls Equals() with the current row. Always inlined so that /// it does not appear in cross-compiled IR. @@ -441,11 +437,11 @@ class PartitionedHashTableCtx { } /// Cross-compiled function to access member variables used in CodegenHashRow(). - uint32_t IR_ALWAYS_INLINE GetHashSeed() const; + uint32_t GetHashSeed() const; /// Functions to be replaced by codegen to specialize the hash table. - bool IR_NO_INLINE stores_nulls() const { return stores_nulls_; } - bool IR_NO_INLINE finds_some_nulls() const { return finds_some_nulls_; } + bool stores_nulls() const { return stores_nulls_; } + bool finds_some_nulls() const { return finds_some_nulls_; } std::shared_ptr tracker_; @@ -583,13 +579,12 @@ class PartitionedHashTable { /// only one tuple, a pointer to that tuple is stored. Otherwise the 'flat_row' pointer /// is stored. The 'row' is not copied by the hash table and the caller must guarantee /// it stays in memory. This will not grow the hash table. - bool IR_ALWAYS_INLINE Insert(PartitionedHashTableCtx* ht_ctx, - BufferedTupleStream3::FlatRowPtr flat_row, TupleRow* row, - Status* status); + bool Insert(PartitionedHashTableCtx* ht_ctx, BufferedTupleStream3::FlatRowPtr flat_row, + TupleRow* row, Status* status); /// Prefetch the hash table bucket which the given hash value 'hash' maps to. template - void IR_ALWAYS_INLINE PrefetchBucket(uint32_t hash); + void PrefetchBucket(uint32_t hash); /// Returns an iterator to the bucket that matches the probe expression results that /// are cached at the current position of the ExprValuesCache in 'ht_ctx'. Assumes that @@ -599,13 +594,13 @@ class PartitionedHashTable { /// row. The matching rows do not need to be evaluated since all the nodes of a bucket /// are duplicates. One scan can be in progress for each 'ht_ctx'. Used in the probe /// phase of hash joins. - Iterator IR_ALWAYS_INLINE FindProbeRow(PartitionedHashTableCtx* ht_ctx); + Iterator FindProbeRow(PartitionedHashTableCtx* ht_ctx); /// If a match is found in the table, return an iterator as in FindProbeRow(). If a /// match was not present, return an iterator pointing to the empty bucket where the key /// should be inserted. Returns End() if the table is full. The caller can set the data /// in the bucket using a Set*() method on the iterator. - Iterator IR_ALWAYS_INLINE FindBuildRowBucket(PartitionedHashTableCtx* ht_ctx, bool* found); + Iterator FindBuildRowBucket(PartitionedHashTableCtx* ht_ctx, bool* found); /// Returns number of elements inserted in the hash table int64_t size() const { @@ -714,31 +709,31 @@ class PartitionedHashTable { static const int64_t BUCKET_NOT_FOUND = -1; public: - IR_ALWAYS_INLINE Iterator() + Iterator() : table_(nullptr), scratch_row_(nullptr), bucket_idx_(BUCKET_NOT_FOUND), node_(nullptr) {} /// Iterates to the next element. It should be called only if !AtEnd(). - void IR_ALWAYS_INLINE Next(); + void Next(); /// Iterates to the next duplicate node. If the bucket does not have duplicates or /// when it reaches the last duplicate node, then it moves the Iterator to AtEnd(). /// Used when we want to iterate over all the duplicate nodes bypassing the Next() /// interface (e.g. in semi/outer joins without other_join_conjuncts, in order to /// iterate over all nodes of an unmatched bucket). - void IR_ALWAYS_INLINE NextDuplicate(); + void NextDuplicate(); /// Iterates to the next element that does not have its matched flag set. Used in /// right-outer and full-outer joins. - void IR_ALWAYS_INLINE NextUnmatched(); + void NextUnmatched(); /// Return the current row or tuple. Callers must check the iterator is not AtEnd() /// before calling them. The returned row is owned by the iterator and valid until /// the next call to GetRow(). It is safe to advance the iterator. - TupleRow* IR_ALWAYS_INLINE GetRow() const; - Tuple* IR_ALWAYS_INLINE GetTuple() const; + TupleRow* GetRow() const; + Tuple* GetTuple() const; /// Set the current tuple for an empty bucket. Designed to be used with the iterator /// returned from FindBuildRowBucket() in the case when the value is not found. It is @@ -762,7 +757,7 @@ class PartitionedHashTable { /// Prefetch the hash table bucket which the iterator is pointing to now. template - void IR_ALWAYS_INLINE PrefetchBucket(); + void PrefetchBucket(); private: friend class PartitionedHashTable; @@ -819,14 +814,14 @@ class PartitionedHashTable { /// /// There are wrappers of this function that perform the Find and Insert logic. template - int64_t IR_ALWAYS_INLINE Probe(Bucket* buckets, int64_t num_buckets, - PartitionedHashTableCtx* ht_ctx, uint32_t hash, bool* found); + int64_t Probe(Bucket* buckets, int64_t num_buckets, PartitionedHashTableCtx* ht_ctx, + uint32_t hash, bool* found); /// Performs the insert logic. Returns the HtData* of the bucket or duplicate node /// where the data should be inserted. Returns nullptr if the insert was not successful /// and either sets 'status' to OK if it failed because not enough reservation was /// available or the error if an error was encountered. - HtData* IR_ALWAYS_INLINE InsertInternal(PartitionedHashTableCtx* ht_ctx, Status* status); + HtData* InsertInternal(PartitionedHashTableCtx* ht_ctx, Status* status); /// Updates 'bucket_idx' to the index of the next non-empty bucket. If the bucket has /// duplicates, 'node' will be pointing to the head of the linked list of duplicates. @@ -840,7 +835,7 @@ class PartitionedHashTable { /// Appends the DuplicateNode pointed by next_node_ to 'bucket' and moves the next_node_ /// pointer to the next DuplicateNode in the page, updating the remaining node counter. - DuplicateNode* IR_ALWAYS_INLINE AppendNextNode(Bucket* bucket); + DuplicateNode* AppendNextNode(Bucket* bucket); /// Creates a new DuplicateNode for a entry and chains it to the bucket with index /// 'bucket_idx'. The duplicate nodes of a bucket are chained as a linked list. @@ -852,11 +847,11 @@ class PartitionedHashTable { /// Returns nullptr and sets 'status' to OK if the node array could not grow, i.e. there /// was not enough memory to allocate a new DuplicateNode. Returns nullptr and sets /// 'status' to an error if another error was encountered. - DuplicateNode* IR_ALWAYS_INLINE InsertDuplicateNode(int64_t bucket_idx, Status* status); + DuplicateNode* InsertDuplicateNode(int64_t bucket_idx, Status* status); /// Resets the contents of the empty bucket with index 'bucket_idx', in preparation for /// an insert. Sets all the fields of the bucket other than 'data'. - void IR_ALWAYS_INLINE PrepareBucketForInsert(int64_t bucket_idx, uint32_t hash); + void PrepareBucketForInsert(int64_t bucket_idx, uint32_t hash); /// Return the TupleRow pointed by 'htdata'. TupleRow* GetRow(HtData& htdata, TupleRow* row) const; @@ -871,9 +866,9 @@ class PartitionedHashTable { bool GrowNodeArray(Status* status); /// Functions to be replaced by codegen to specialize the hash table. - bool IR_NO_INLINE stores_tuples() const { return stores_tuples_; } - bool IR_NO_INLINE stores_duplicates() const { return stores_duplicates_; } - bool IR_NO_INLINE quadratic_probing() const { return quadratic_probing_; } + bool stores_tuples() const { return stores_tuples_; } + bool stores_duplicates() const { return stores_duplicates_; } + bool quadratic_probing() const { return quadratic_probing_; } /// Load factor that will trigger growing the hash table on insert. This is /// defined as the number of non-empty buckets / total_buckets diff --git a/be/src/exec/partitioned_hash_table.inline.h b/be/src/exec/partitioned_hash_table.inline.h index 32a556231fb95d..b79c40be517b7a 100644 --- a/be/src/exec/partitioned_hash_table.inline.h +++ b/be/src/exec/partitioned_hash_table.inline.h @@ -249,8 +249,7 @@ inline PartitionedHashTable::DuplicateNode* PartitionedHashTable::InsertDuplicat return AppendNextNode(bucket); } -inline TupleRow* IR_ALWAYS_INLINE PartitionedHashTable::GetRow(HtData& htdata, - TupleRow* row) const { +inline TupleRow* PartitionedHashTable::GetRow(HtData& htdata, TupleRow* row) const { if (stores_tuples()) { return reinterpret_cast(&htdata.tuple); } else { @@ -260,8 +259,7 @@ inline TupleRow* IR_ALWAYS_INLINE PartitionedHashTable::GetRow(HtData& htdata, } } -inline TupleRow* IR_ALWAYS_INLINE PartitionedHashTable::GetRow(Bucket* bucket, - TupleRow* row) const { +inline TupleRow* PartitionedHashTable::GetRow(Bucket* bucket, TupleRow* row) const { DCHECK(bucket != nullptr); if (UNLIKELY(stores_duplicates() && bucket->hasDuplicates)) { DuplicateNode* duplicate = bucket->bucketData.duplicates; @@ -272,7 +270,7 @@ inline TupleRow* IR_ALWAYS_INLINE PartitionedHashTable::GetRow(Bucket* bucket, } } -inline TupleRow* IR_ALWAYS_INLINE PartitionedHashTable::Iterator::GetRow() const { +inline TupleRow* PartitionedHashTable::Iterator::GetRow() const { DCHECK(!AtEnd()); DCHECK(table_ != nullptr); DCHECK(scratch_row_ != nullptr); @@ -285,7 +283,7 @@ inline TupleRow* IR_ALWAYS_INLINE PartitionedHashTable::Iterator::GetRow() const } } -inline Tuple* IR_ALWAYS_INLINE PartitionedHashTable::Iterator::GetTuple() const { +inline Tuple* PartitionedHashTable::Iterator::GetTuple() const { DCHECK(!AtEnd()); DCHECK(table_->stores_tuples()); Bucket* bucket = &table_->buckets_[bucket_idx_]; diff --git a/be/src/exec/partitioned_hash_table_ir.cc b/be/src/exec/partitioned_hash_table_ir.cc deleted file mode 100644 index 50dc30787cc6c3..00000000000000 --- a/be/src/exec/partitioned_hash_table_ir.cc +++ /dev/null @@ -1,35 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#ifdef IR_COMPILE -#include "exec/partitioned_hash_table.h" - -using namespace doris; - -uint32_t PartitionedHashTableCtx::GetHashSeed() const { - return seeds_[level_]; -} - -ExprContext* const* PartitionedHashTableCtx::build_expr_evals() const { - return build_expr_evals_.data(); -} - -ExprContext* const* PartitionedHashTableCtx::probe_expr_evals() const { - return probe_expr_evals_.data(); -} - -#endif diff --git a/be/src/exec/scanner_ir.cpp b/be/src/exec/scanner_ir.cpp deleted file mode 100644 index e14e57458e430f..00000000000000 --- a/be/src/exec/scanner_ir.cpp +++ /dev/null @@ -1,55 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#ifdef IR_COMPILE -extern "C" bool ir_string_to_bool(const char* s, int len, StringParser::ParseResult* result) { - return StringParser::string_to_bool(s, len, result); -} - -extern "C" int8_t ir_string_to_int8(const char* s, int len, StringParser::ParseResult* result) { - return StringParser::string_to_int(s, len, result); -} - -extern "C" int16_t ir_string_to_int16(const char* s, int len, StringParser::ParseResult* result) { - return StringParser::string_to_int(s, len, result); -} - -extern "C" int32_t ir_string_to_int32(const char* s, int len, StringParser::ParseResult* result) { - return StringParser::string_to_int(s, len, result); -} - -extern "C" int64_t ir_string_to_int64(const char* s, int len, StringParser::ParseResult* result) { - return StringParser::StringToInt(s, len, result); -} - -extern "C" float ir_string_to_float(const char* s, int len, StringParser::ParseResult* result) { - return StringParser::StringToFloat(s, len, result); -} - -extern "C" double ir_string_to_double(const char* s, int len, StringParser::ParseResult* result) { - return StringParser::StringToFloat(s, len, result); -} - -extern "C" bool ir_is_null_string(const char* data, int len) { - return data == nullptr || (len == 2 && data[0] == '\\' && data[1] == 'N'); -} - -extern "C" bool ir_generic_is_null_string(const char* s, int slen, const char* n, int nlen) { - return s == nullptr || (slen == nlen && StringCompare(s, slen, n, nlen, slen) == 0); -} -#endif -} diff --git a/be/src/exec/union_node.cpp b/be/src/exec/union_node.cpp index 611561e6aca047..cbb4bc9d5d27f0 100644 --- a/be/src/exec/union_node.cpp +++ b/be/src/exec/union_node.cpp @@ -277,20 +277,6 @@ Status UnionNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) return Status::OK(); } -#if 0 -Status UnionNode::reset(RuntimeState* state) { - _child_idx = 0; - _child_batch.reset(); - _child_row_idx = 0; - _child_eos = false; - _const_expr_list_idx = 0; - // Since passthrough is disabled in subplans, verify that there is no passthrough child - // that needs to be closed. - DCHECK_EQ(_to_close_child_idx, -1); - return ExecNode::reset(state); -} -#endif - Status UnionNode::close(RuntimeState* state) { if (is_closed()) return Status::OK(); _child_batch.reset(); @@ -316,4 +302,46 @@ void UnionNode::debug_string(int indentation_level, std::stringstream* out) cons *out << ")" << std::endl; } +void UnionNode::materialize_exprs(const std::vector& exprs, TupleRow* row, + uint8_t* tuple_buf, RowBatch* dst_batch) { + DCHECK(!dst_batch->at_capacity()); + Tuple* dst_tuple = reinterpret_cast(tuple_buf); + TupleRow* dst_row = dst_batch->get_row(dst_batch->add_row()); + // dst_tuple->materialize_exprs(row, *_tuple_desc, exprs, + dst_tuple->materialize_exprs(row, *_tuple_desc, exprs, dst_batch->tuple_data_pool(), + nullptr, nullptr); + dst_row->set_tuple(0, dst_tuple); + dst_batch->commit_last_row(); +} + +void UnionNode::materialize_batch(RowBatch* dst_batch, uint8_t** tuple_buf) { + // Take all references to member variables out of the loop to reduce the number of + // loads and stores. + RowBatch* child_batch = _child_batch.get(); + int tuple_byte_size = _tuple_desc->byte_size(); + uint8_t* cur_tuple = *tuple_buf; + const std::vector& child_exprs = _child_expr_lists[_child_idx]; + + int num_rows_to_process = std::min(child_batch->num_rows() - _child_row_idx, + dst_batch->capacity() - dst_batch->num_rows()); + FOREACH_ROW_LIMIT(child_batch, _child_row_idx, num_rows_to_process, batch_iter) { + TupleRow* child_row = batch_iter.get(); + materialize_exprs(child_exprs, child_row, cur_tuple, dst_batch); + cur_tuple += tuple_byte_size; + } + + _child_row_idx += num_rows_to_process; + *tuple_buf = cur_tuple; +} + +Status UnionNode::get_error_msg(const std::vector& exprs) { + for (auto expr_ctx : exprs) { + std::string expr_error = expr_ctx->get_error_msg(); + if (!expr_error.empty()) { + return Status::RuntimeError(expr_error); + } + } + return Status::OK(); +} + } // namespace doris diff --git a/be/src/exec/union_node.h b/be/src/exec/union_node.h index 5b49f716e482b8..a8238ab7645bd7 100644 --- a/be/src/exec/union_node.h +++ b/be/src/exec/union_node.h @@ -17,7 +17,6 @@ #pragma once -#include "codegen/doris_ir.h" #include "exec/exec_node.h" #include "runtime/row_batch.h" #include "runtime/runtime_state.h" diff --git a/be/src/exec/union_node_ir.cpp b/be/src/exec/union_node_ir.cpp deleted file mode 100644 index f29d534aae1434..00000000000000 --- a/be/src/exec/union_node_ir.cpp +++ /dev/null @@ -1,67 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "exec/union_node.h" -#include "exprs/expr_context.h" -#include "runtime/tuple_row.h" - -namespace doris { - -void IR_ALWAYS_INLINE UnionNode::materialize_exprs(const std::vector& exprs, - TupleRow* row, uint8_t* tuple_buf, - RowBatch* dst_batch) { - DCHECK(!dst_batch->at_capacity()); - Tuple* dst_tuple = reinterpret_cast(tuple_buf); - TupleRow* dst_row = dst_batch->get_row(dst_batch->add_row()); - // dst_tuple->materialize_exprs(row, *_tuple_desc, exprs, - dst_tuple->materialize_exprs(row, *_tuple_desc, exprs, dst_batch->tuple_data_pool(), - nullptr, nullptr); - dst_row->set_tuple(0, dst_tuple); - dst_batch->commit_last_row(); -} - -void UnionNode::materialize_batch(RowBatch* dst_batch, uint8_t** tuple_buf) { - // Take all references to member variables out of the loop to reduce the number of - // loads and stores. - RowBatch* child_batch = _child_batch.get(); - int tuple_byte_size = _tuple_desc->byte_size(); - uint8_t* cur_tuple = *tuple_buf; - const std::vector& child_exprs = _child_expr_lists[_child_idx]; - - int num_rows_to_process = std::min(child_batch->num_rows() - _child_row_idx, - dst_batch->capacity() - dst_batch->num_rows()); - FOREACH_ROW_LIMIT(child_batch, _child_row_idx, num_rows_to_process, batch_iter) { - TupleRow* child_row = batch_iter.get(); - materialize_exprs(child_exprs, child_row, cur_tuple, dst_batch); - cur_tuple += tuple_byte_size; - } - - _child_row_idx += num_rows_to_process; - *tuple_buf = cur_tuple; -} - -Status UnionNode::get_error_msg(const std::vector& exprs) { - for (auto expr_ctx : exprs) { - std::string expr_error = expr_ctx->get_error_msg(); - if (!expr_error.empty()) { - return Status::RuntimeError(expr_error); - } - } - return Status::OK(); -} - -} // namespace doris diff --git a/be/src/exprs/CMakeLists.txt b/be/src/exprs/CMakeLists.txt index 3b4b86cb97b3e1..c71f23ec32dc5d 100644 --- a/be/src/exprs/CMakeLists.txt +++ b/be/src/exprs/CMakeLists.txt @@ -33,14 +33,12 @@ add_library(Exprs cast_functions.cpp compound_predicate.cpp conditional_functions.cpp - conditional_functions_ir.cpp decimalv2_operators.cpp time_operators.cpp es_functions.cpp hash_functions.cpp literal.cpp expr.cpp - expr_ir.cpp expr_context.cpp in_predicate.cpp new_in_predicate.cpp diff --git a/be/src/exprs/agg_fn_evaluator.cpp b/be/src/exprs/agg_fn_evaluator.cpp index bcb7e35af16afc..d83920c43bb20e 100644 --- a/be/src/exprs/agg_fn_evaluator.cpp +++ b/be/src/exprs/agg_fn_evaluator.cpp @@ -944,11 +944,6 @@ std::string AggFnEvaluator::debug_string(const std::vector& exp std::string AggFnEvaluator::debug_string() const { std::stringstream out; out << "AggFnEvaluator(op=" << _agg_op; -#if 0 - for (int i = 0; i < _input_exprs_ctxs.size(); ++i) { - out << " " << _input_exprs[i]->debug_string() << ")"; - } -#endif out << ")"; return out.str(); diff --git a/be/src/exprs/cast_functions.cpp b/be/src/exprs/cast_functions.cpp index 925b3fe48fbf06..baadd09ce2767a 100644 --- a/be/src/exprs/cast_functions.cpp +++ b/be/src/exprs/cast_functions.cpp @@ -17,9 +17,10 @@ #include "exprs/cast_functions.h" -#include #include +#include + #include "exprs/anyval_util.h" #include "gutil/strings/numbers.h" #include "runtime/datetime_value.h" @@ -242,28 +243,6 @@ BooleanVal CastFunctions::cast_to_boolean_val(FunctionContext* ctx, const String return ret; } -#if 0 -StringVal CastFunctions::CastToChar(FunctionContext* ctx, const StringVal& val) { - if (val.is_null) return StringVal::null(); - - const FunctionContext::TypeDesc& type = ctx->GetReturnType(); - DCHECK(type.type == FunctionContext::TYPE_FIXED_BUFFER); - DCHECK_GE(type.len, 1); - char* cptr; - if (type.len > val.len) { - cptr = reinterpret_cast(ctx->impl()->AllocateLocal(type.len)); - memcpy(cptr, val.ptr, min(type.len, val.len)); - StringValue::PadWithSpaces(cptr, type.len, val.len); - } else { - cptr = reinterpret_cast(val.ptr); - } - StringVal sv; - sv.ptr = reinterpret_cast(cptr); - sv.len = type.len; - return sv; -} -#endif - #define CAST_FROM_DATETIME(to_type, type_name) \ to_type CastFunctions::cast_to_##type_name(FunctionContext* ctx, const DateTimeVal& val) { \ if (val.is_null) return to_type::null(); \ diff --git a/be/src/exprs/cast_functions.h b/be/src/exprs/cast_functions.h index b92b45fac8ab9c..0160275d235794 100644 --- a/be/src/exprs/cast_functions.h +++ b/be/src/exprs/cast_functions.h @@ -116,9 +116,6 @@ class CastFunctions { static StringVal cast_to_string_val(FunctionContext* context, const DoubleVal& val); static StringVal cast_to_string_val(FunctionContext* context, const DateTimeVal& val); static StringVal cast_to_string_val(FunctionContext* context, const StringVal& val); -#if 0 - static StringVal CastToChar(FunctionContext* context, const StringVal& val); -#endif static DateTimeVal cast_to_datetime_val(FunctionContext* context, const TinyIntVal& val); static DateTimeVal cast_to_datetime_val(FunctionContext* context, const SmallIntVal& val); diff --git a/be/src/exprs/compound_predicate.cpp b/be/src/exprs/compound_predicate.cpp index 88af421902916d..01387f58297219 100644 --- a/be/src/exprs/compound_predicate.cpp +++ b/be/src/exprs/compound_predicate.cpp @@ -25,12 +25,6 @@ namespace doris { CompoundPredicate::CompoundPredicate(const TExprNode& node) : Predicate(node) {} -#if 0 -Status CompoundPredicate::prepare(RuntimeState* state, const RowDescriptor& desc) { - DCHECK_LE(_children.size(), 2); - return Expr::prepare(state, desc); -} -#endif void CompoundPredicate::init() {} diff --git a/be/src/exprs/conditional_functions.cpp b/be/src/exprs/conditional_functions.cpp index ce1a08cc22985f..386f61d68a31da 100644 --- a/be/src/exprs/conditional_functions.cpp +++ b/be/src/exprs/conditional_functions.cpp @@ -25,13 +25,106 @@ namespace doris { -#define CTOR_DCTOR_FUN(expr_class) \ - expr_class::expr_class(const TExprNode& node) : Expr(node){}; \ - \ - expr_class::~expr_class(){}; +#define CTOR_DCTOR_FUN(expr_class) \ + expr_class::expr_class(const TExprNode& node) : Expr(node) {}; \ + \ + expr_class::~expr_class() {}; CTOR_DCTOR_FUN(IfNullExpr); CTOR_DCTOR_FUN(NullIfExpr); CTOR_DCTOR_FUN(IfExpr); CTOR_DCTOR_FUN(CoalesceExpr); + +#define IF_NULL_COMPUTE_FUNCTION(type, type_name) \ + type IfNullExpr::get_##type_name(ExprContext* context, TupleRow* row) { \ + DCHECK_EQ(_children.size(), 2); \ + type val = _children[0]->get_##type_name(context, row); \ + if (!val.is_null) return val; /* short-circuit */ \ + return _children[1]->get_##type_name(context, row); \ + } + +IF_NULL_COMPUTE_FUNCTION(BooleanVal, boolean_val); +IF_NULL_COMPUTE_FUNCTION(TinyIntVal, tiny_int_val); +IF_NULL_COMPUTE_FUNCTION(SmallIntVal, small_int_val); +IF_NULL_COMPUTE_FUNCTION(IntVal, int_val); +IF_NULL_COMPUTE_FUNCTION(BigIntVal, big_int_val); +IF_NULL_COMPUTE_FUNCTION(FloatVal, float_val); +IF_NULL_COMPUTE_FUNCTION(DoubleVal, double_val); +IF_NULL_COMPUTE_FUNCTION(StringVal, string_val); +IF_NULL_COMPUTE_FUNCTION(DateTimeVal, datetime_val); +IF_NULL_COMPUTE_FUNCTION(DecimalV2Val, decimalv2_val); +IF_NULL_COMPUTE_FUNCTION(LargeIntVal, large_int_val); + +#define NULL_IF_COMPUTE_FUNCTION(TYPE, type_name) \ + TYPE NullIfExpr::get_##type_name(ExprContext* ctx, TupleRow* row) { \ + DCHECK_EQ(_children.size(), 2); \ + TYPE lhs_val = _children[0]->get_##type_name(ctx, row); \ + /* Short-circuit in case lhs_val is nullptr. Can never be equal to RHS. */ \ + if (lhs_val.is_null) return TYPE::null(); \ + /* Get rhs and return nullptr if lhs == rhs, lhs otherwise */ \ + TYPE rhs_val = _children[1]->get_##type_name(ctx, row); \ + if (!rhs_val.is_null && AnyValUtil::equals(_children[0]->type(), lhs_val, rhs_val)) { \ + return TYPE::null(); \ + } \ + return lhs_val; \ + } + +// Just for code check..... +#define NULL_IF_COMPUTE_FUNCTION_WRAPPER(TYPE, type_name) NULL_IF_COMPUTE_FUNCTION(TYPE, type_name) + +NULL_IF_COMPUTE_FUNCTION_WRAPPER(BooleanVal, boolean_val); +NULL_IF_COMPUTE_FUNCTION_WRAPPER(TinyIntVal, tiny_int_val); +NULL_IF_COMPUTE_FUNCTION_WRAPPER(SmallIntVal, small_int_val); +NULL_IF_COMPUTE_FUNCTION_WRAPPER(IntVal, int_val); +NULL_IF_COMPUTE_FUNCTION_WRAPPER(BigIntVal, big_int_val); +NULL_IF_COMPUTE_FUNCTION_WRAPPER(FloatVal, float_val); +NULL_IF_COMPUTE_FUNCTION_WRAPPER(DoubleVal, double_val); +NULL_IF_COMPUTE_FUNCTION_WRAPPER(StringVal, string_val); +NULL_IF_COMPUTE_FUNCTION_WRAPPER(DateTimeVal, datetime_val); +NULL_IF_COMPUTE_FUNCTION_WRAPPER(DecimalV2Val, decimalv2_val); +NULL_IF_COMPUTE_FUNCTION_WRAPPER(LargeIntVal, large_int_val); + +#define IF_COMPUTE_FUNCTION(type, type_name) \ + type IfExpr::get_##type_name(ExprContext* context, TupleRow* row) { \ + DCHECK_EQ(_children.size(), 3); \ + BooleanVal cond = _children[0]->get_boolean_val(context, row); \ + if (cond.is_null || !cond.val) { \ + return _children[2]->get_##type_name(context, row); \ + } \ + return _children[1]->get_##type_name(context, row); \ + } + +IF_COMPUTE_FUNCTION(BooleanVal, boolean_val); +IF_COMPUTE_FUNCTION(TinyIntVal, tiny_int_val); +IF_COMPUTE_FUNCTION(SmallIntVal, small_int_val); +IF_COMPUTE_FUNCTION(IntVal, int_val); +IF_COMPUTE_FUNCTION(BigIntVal, big_int_val); +IF_COMPUTE_FUNCTION(FloatVal, float_val); +IF_COMPUTE_FUNCTION(DoubleVal, double_val); +IF_COMPUTE_FUNCTION(StringVal, string_val); +IF_COMPUTE_FUNCTION(DateTimeVal, datetime_val); +IF_COMPUTE_FUNCTION(DecimalV2Val, decimalv2_val); +IF_COMPUTE_FUNCTION(LargeIntVal, large_int_val); + +#define COALESCE_COMPUTE_FUNCTION(type, type_name) \ + type CoalesceExpr::get_##type_name(ExprContext* context, TupleRow* row) { \ + DCHECK_GE(_children.size(), 1); \ + for (int i = 0; i < _children.size(); ++i) { \ + type val = _children[i]->get_##type_name(context, row); \ + if (!val.is_null) return val; \ + } \ + return type::null(); \ + } + +COALESCE_COMPUTE_FUNCTION(BooleanVal, boolean_val); +COALESCE_COMPUTE_FUNCTION(TinyIntVal, tiny_int_val); +COALESCE_COMPUTE_FUNCTION(SmallIntVal, small_int_val); +COALESCE_COMPUTE_FUNCTION(IntVal, int_val); +COALESCE_COMPUTE_FUNCTION(BigIntVal, big_int_val); +COALESCE_COMPUTE_FUNCTION(FloatVal, float_val); +COALESCE_COMPUTE_FUNCTION(DoubleVal, double_val); +COALESCE_COMPUTE_FUNCTION(StringVal, string_val); +COALESCE_COMPUTE_FUNCTION(DateTimeVal, datetime_val); +COALESCE_COMPUTE_FUNCTION(DecimalV2Val, decimalv2_val); +COALESCE_COMPUTE_FUNCTION(LargeIntVal, large_int_val); } // namespace doris diff --git a/be/src/exprs/conditional_functions_ir.cpp b/be/src/exprs/conditional_functions_ir.cpp deleted file mode 100644 index f36ac2d3e3c5de..00000000000000 --- a/be/src/exprs/conditional_functions_ir.cpp +++ /dev/null @@ -1,117 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "exprs/anyval_util.h" -#include "exprs/conditional_functions.h" -#include "udf/udf.h" - -namespace doris { - -#define IF_NULL_COMPUTE_FUNCTION(type, type_name) \ - type IfNullExpr::get_##type_name(ExprContext* context, TupleRow* row) { \ - DCHECK_EQ(_children.size(), 2); \ - type val = _children[0]->get_##type_name(context, row); \ - if (!val.is_null) return val; /* short-circuit */ \ - return _children[1]->get_##type_name(context, row); \ - } - -IF_NULL_COMPUTE_FUNCTION(BooleanVal, boolean_val); -IF_NULL_COMPUTE_FUNCTION(TinyIntVal, tiny_int_val); -IF_NULL_COMPUTE_FUNCTION(SmallIntVal, small_int_val); -IF_NULL_COMPUTE_FUNCTION(IntVal, int_val); -IF_NULL_COMPUTE_FUNCTION(BigIntVal, big_int_val); -IF_NULL_COMPUTE_FUNCTION(FloatVal, float_val); -IF_NULL_COMPUTE_FUNCTION(DoubleVal, double_val); -IF_NULL_COMPUTE_FUNCTION(StringVal, string_val); -IF_NULL_COMPUTE_FUNCTION(DateTimeVal, datetime_val); -IF_NULL_COMPUTE_FUNCTION(DecimalV2Val, decimalv2_val); -IF_NULL_COMPUTE_FUNCTION(LargeIntVal, large_int_val); - -#define NULL_IF_COMPUTE_FUNCTION(TYPE, type_name) \ - TYPE NullIfExpr::get_##type_name(ExprContext* ctx, TupleRow* row) { \ - DCHECK_EQ(_children.size(), 2); \ - TYPE lhs_val = _children[0]->get_##type_name(ctx, row); \ - /* Short-circuit in case lhs_val is nullptr. Can never be equal to RHS. */ \ - if (lhs_val.is_null) return TYPE::null(); \ - /* Get rhs and return nullptr if lhs == rhs, lhs otherwise */ \ - TYPE rhs_val = _children[1]->get_##type_name(ctx, row); \ - if (!rhs_val.is_null && AnyValUtil::equals(_children[0]->type(), lhs_val, rhs_val)) { \ - return TYPE::null(); \ - } \ - return lhs_val; \ - } - -// Just for code check..... -#define NULL_IF_COMPUTE_FUNCTION_WRAPPER(TYPE, type_name) NULL_IF_COMPUTE_FUNCTION(TYPE, type_name) - -NULL_IF_COMPUTE_FUNCTION_WRAPPER(BooleanVal, boolean_val); -NULL_IF_COMPUTE_FUNCTION_WRAPPER(TinyIntVal, tiny_int_val); -NULL_IF_COMPUTE_FUNCTION_WRAPPER(SmallIntVal, small_int_val); -NULL_IF_COMPUTE_FUNCTION_WRAPPER(IntVal, int_val); -NULL_IF_COMPUTE_FUNCTION_WRAPPER(BigIntVal, big_int_val); -NULL_IF_COMPUTE_FUNCTION_WRAPPER(FloatVal, float_val); -NULL_IF_COMPUTE_FUNCTION_WRAPPER(DoubleVal, double_val); -NULL_IF_COMPUTE_FUNCTION_WRAPPER(StringVal, string_val); -NULL_IF_COMPUTE_FUNCTION_WRAPPER(DateTimeVal, datetime_val); -NULL_IF_COMPUTE_FUNCTION_WRAPPER(DecimalV2Val, decimalv2_val); -NULL_IF_COMPUTE_FUNCTION_WRAPPER(LargeIntVal, large_int_val); - -#define IF_COMPUTE_FUNCTION(type, type_name) \ - type IfExpr::get_##type_name(ExprContext* context, TupleRow* row) { \ - DCHECK_EQ(_children.size(), 3); \ - BooleanVal cond = _children[0]->get_boolean_val(context, row); \ - if (cond.is_null || !cond.val) { \ - return _children[2]->get_##type_name(context, row); \ - } \ - return _children[1]->get_##type_name(context, row); \ - } - -IF_COMPUTE_FUNCTION(BooleanVal, boolean_val); -IF_COMPUTE_FUNCTION(TinyIntVal, tiny_int_val); -IF_COMPUTE_FUNCTION(SmallIntVal, small_int_val); -IF_COMPUTE_FUNCTION(IntVal, int_val); -IF_COMPUTE_FUNCTION(BigIntVal, big_int_val); -IF_COMPUTE_FUNCTION(FloatVal, float_val); -IF_COMPUTE_FUNCTION(DoubleVal, double_val); -IF_COMPUTE_FUNCTION(StringVal, string_val); -IF_COMPUTE_FUNCTION(DateTimeVal, datetime_val); -IF_COMPUTE_FUNCTION(DecimalV2Val, decimalv2_val); -IF_COMPUTE_FUNCTION(LargeIntVal, large_int_val); - -#define COALESCE_COMPUTE_FUNCTION(type, type_name) \ - type CoalesceExpr::get_##type_name(ExprContext* context, TupleRow* row) { \ - DCHECK_GE(_children.size(), 1); \ - for (int i = 0; i < _children.size(); ++i) { \ - type val = _children[i]->get_##type_name(context, row); \ - if (!val.is_null) return val; \ - } \ - return type::null(); \ - } - -COALESCE_COMPUTE_FUNCTION(BooleanVal, boolean_val); -COALESCE_COMPUTE_FUNCTION(TinyIntVal, tiny_int_val); -COALESCE_COMPUTE_FUNCTION(SmallIntVal, small_int_val); -COALESCE_COMPUTE_FUNCTION(IntVal, int_val); -COALESCE_COMPUTE_FUNCTION(BigIntVal, big_int_val); -COALESCE_COMPUTE_FUNCTION(FloatVal, float_val); -COALESCE_COMPUTE_FUNCTION(DoubleVal, double_val); -COALESCE_COMPUTE_FUNCTION(StringVal, string_val); -COALESCE_COMPUTE_FUNCTION(DateTimeVal, datetime_val); -COALESCE_COMPUTE_FUNCTION(DecimalV2Val, decimalv2_val); -COALESCE_COMPUTE_FUNCTION(LargeIntVal, large_int_val); - -} // namespace doris diff --git a/be/src/exprs/expr.cpp b/be/src/exprs/expr.cpp index 1c29d5d840eafb..73f3775247f291 100644 --- a/be/src/exprs/expr.cpp +++ b/be/src/exprs/expr.cpp @@ -404,22 +404,6 @@ Status Expr::create_expr(ObjectPool* pool, const TExprNode& texpr_node, Expr** e *expr = pool->add(new InfoFunc(texpr_node)); return Status::OK(); } -#if 0 - case TExprNodeType::FUNCTION_CALL: { - if (!texpr_node.__isset.fn_call_expr) { - return Status::InternalError("Udf call not set in thrift node"); - } - - if (texpr_node.fn_call_expr.fn.binary_type == TFunctionBinaryType::HIVE) { - DCHECK(false); //temp add, can't get here - //*expr = pool->Add(new HiveUdfCall(texpr_node)); - } else { - *expr = pool->add(new NativeUdfExpr(texpr_node)); - } - - return Status::OK(); - } -#endif default: std::stringstream os; @@ -563,16 +547,6 @@ void Expr::close(RuntimeState* state, ExprContext* context, for (int i = 0; i < _children.size(); ++i) { _children[i]->close(state, context, scope); } - // TODO(zc) -#if 0 - if (scope == FunctionContext::FRAGMENT_LOCAL) { - // This is the final, non-cloned context to close. Clean up the whole Expr. - if (cache_entry_ != nullptr) { - LibCache::instance()->DecrementUseCount(cache_entry_); - cache_entry_ = nullptr; - } - } -#endif } Status Expr::clone_if_not_exists(const std::vector& ctxs, RuntimeState* state, diff --git a/be/src/exprs/expr_context.cpp b/be/src/exprs/expr_context.cpp index ac2694ab9430a5..40e93ee66a14fc 100644 --- a/be/src/exprs/expr_context.cpp +++ b/be/src/exprs/expr_context.cpp @@ -255,21 +255,6 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) { _result.string_val.len = v.len; return &_result.string_val; } -#if 0 - case TYPE_CHAR: { - doris_udf::StringVal v = e->get_string_val(this, row); - if (v.is_null) { - return nullptr; - } - _result.string_val.ptr = reinterpret_cast(v.ptr); - _result.string_val.len = v.len; - if (e->_type.IsVarLenStringType()) { - return &_result.string_val; - } else { - return _result.string_val.ptr; - } - } -#endif case TYPE_DATE: case TYPE_DATETIME: { doris_udf::DateTimeVal v = e->get_datetime_val(this, row); diff --git a/be/src/exprs/expr_ir.cpp b/be/src/exprs/expr_ir.cpp deleted file mode 100644 index 7be579846e10a0..00000000000000 --- a/be/src/exprs/expr_ir.cpp +++ /dev/null @@ -1,81 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "exprs/expr.h" -#include "udf/udf.h" - -#ifdef IR_COMPILE - -// Compile ExprContext declaration to IR so we can use it in codegen'd functions -#include "exprs/expr_context.h" - -// Dummy function to force compilation of UDF types. -// The arguments are pointers to prevent Clang from lowering the struct types -// (e.g. IntVal={bool, i32} can be coerced to i64). -void dummy(doris_udf::FunctionContext*, doris_udf::BooleanVal*, doris_udf::TinyIntVal*, - doris_udf::SmallIntVal*, doris_udf::IntVal*, doris_udf::BigIntVal*, doris_udf::FloatVal*, - doris_udf::DoubleVal*, doris_udf::StringVal*, doris_udf::DateTimeVal*, - doris_udf::DecimalV2Val*, doris::ExprContext*) {} -#endif - -// The following are compute functions that are cross-compiled to both native and IR -// libraries. In the interpreted path, these functions are executed as-is from the native -// code. In the codegen'd path, we load the IR functions and replace the Get*Val() calls -// with the appropriate child's codegen'd compute function. - -namespace doris { -// Static wrappers around Get*Val() functions. We'd like to be able to call these from -// directly from native code as well as from generated IR functions. - -BooleanVal Expr::get_boolean_val(Expr* expr, ExprContext* context, TupleRow* row) { - return expr->get_boolean_val(context, row); -} -TinyIntVal Expr::get_tiny_int_val(Expr* expr, ExprContext* context, TupleRow* row) { - return expr->get_tiny_int_val(context, row); -} -SmallIntVal Expr::get_small_int_val(Expr* expr, ExprContext* context, TupleRow* row) { - return expr->get_small_int_val(context, row); -} -IntVal Expr::get_int_val(Expr* expr, ExprContext* context, TupleRow* row) { - return expr->get_int_val(context, row); -} -BigIntVal Expr::get_big_int_val(Expr* expr, ExprContext* context, TupleRow* row) { - return expr->get_big_int_val(context, row); -} -LargeIntVal Expr::get_large_int_val(Expr* expr, ExprContext* context, TupleRow* row) { - return expr->get_large_int_val(context, row); -} -FloatVal Expr::get_float_val(Expr* expr, ExprContext* context, TupleRow* row) { - return expr->get_float_val(context, row); -} -DoubleVal Expr::get_double_val(Expr* expr, ExprContext* context, TupleRow* row) { - return expr->get_double_val(context, row); -} -StringVal Expr::get_string_val(Expr* expr, ExprContext* context, TupleRow* row) { - return expr->get_string_val(context, row); -} -DateTimeVal Expr::get_datetime_val(Expr* expr, ExprContext* context, TupleRow* row) { - return expr->get_datetime_val(context, row); -} - -DecimalV2Val Expr::get_decimalv2_val(Expr* expr, ExprContext* context, TupleRow* row) { - return expr->get_decimalv2_val(context, row); -} -CollectionVal Expr::get_array_val(Expr* expr, ExprContext* context, TupleRow* row) { - return expr->get_array_val(context, row); -} -} // namespace doris diff --git a/be/src/exprs/info_func.cpp b/be/src/exprs/info_func.cpp index 5f4d1ae29b428b..8096bf8fc2c680 100644 --- a/be/src/exprs/info_func.cpp +++ b/be/src/exprs/info_func.cpp @@ -46,16 +46,6 @@ std::string InfoFunc::debug_string() const { } void* InfoFunc::compute_fn(Expr* e, TupleRow* row) { -#if 0 - if (e->type() == TYPE_CHAR || e->type() == TYPE_VARCHAR) { - e->_result.set_string_val(((InfoFunc*)e)->_str_value); - return &e->_result.string_val; - } else { - e->_result.int_val = ((InfoFunc*)e)->_int_value; - return &e->_result.int_val; - } -#endif - return nullptr; } diff --git a/be/src/exprs/math_functions.cpp b/be/src/exprs/math_functions.cpp index 345fb65182b361..708fc425d9ca7c 100644 --- a/be/src/exprs/math_functions.cpp +++ b/be/src/exprs/math_functions.cpp @@ -696,167 +696,4 @@ GREATEST_FNS(); GREATEST_NONNUMERIC_FN(decimal_val, DecimalV2Val, DecimalV2Value); GREATEST_NONNUMERIC_FNS(); - -#if 0 -void* MathFunctions::greatest_bigint(Expr* e, TupleRow* row) { - DCHECK_GE(e->get_num_children(), 1); - int32_t num_args = e->get_num_children(); - - int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. - for (int i = 0; i < num_args; ++i) { - int64_t* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == nullptr) { - return nullptr; - } - - if (*arg > *reinterpret_cast(e->children()[result_idx]->get_value(row))) { - result_idx = i; - } - } - return &e->children()[result_idx]->_result.bigint_val; -} - -void* MathFunctions::greatest_double(Expr* e, TupleRow* row) { - DCHECK_GE(e->get_num_children(), 1); - int32_t num_args = e->get_num_children(); - int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. - for (int i = 0; i < num_args; ++i) { - double* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == nullptr) { - return nullptr; - } - - if (*arg > *reinterpret_cast(e->children()[result_idx]->get_value(row))) { - result_idx = i; - } - } - return &e->children()[result_idx]->_result.double_val; -} - -void* MathFunctions::greatest_string(Expr* e, TupleRow* row) { - DCHECK_GE(e->get_num_children(), 1); - int32_t num_args = e->get_num_children(); - int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. - for (int i = 0; i < num_args; ++i) { - StringValue* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == nullptr) { - return nullptr; - } - if (*arg > *reinterpret_cast(e->children()[result_idx]->get_value(row))) { - result_idx = i; - } - } - return &e->children()[result_idx]->_result.string_val; -} - -void* MathFunctions::greatest_timestamp(Expr* e, TupleRow* row) { - DCHECK_GE(e->get_num_children(), 1); - int32_t num_args = e->get_num_children(); - int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. - for (int i = 0; i < num_args; ++i) { - DateTimeValue* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == nullptr) { - return nullptr; - } - if (*arg > *reinterpret_cast(e->children()[result_idx]->get_value(row))) { - result_idx = i; - } - } - return &e->children()[result_idx]->_result.datetime_val; -} -void* MathFunctions::least_bigint(Expr* e, TupleRow* row) { - DCHECK_GE(e->get_num_children(), 1); - int32_t num_args = e->get_num_children(); - - int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. - for (int i = 0; i < num_args; ++i) { - int64_t* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == nullptr) { - return nullptr; - } - - if (*arg < *reinterpret_cast(e->children()[result_idx]->get_value(row))) { - result_idx = i; - } - } - return &e->children()[result_idx]->_result.bigint_val; - -} - -void* MathFunctions::least_double(Expr* e, TupleRow* row) { - DCHECK_GE(e->get_num_children(), 1); - int32_t num_args = e->get_num_children(); - int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. - for (int i = 0; i < num_args; ++i) { - double* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == nullptr) { - return nullptr; - } - - if (*arg < *reinterpret_cast(e->children()[result_idx]->get_value(row))) { - result_idx = i; - } - } - return &e->children()[result_idx]->_result.double_val; -} - -void* MathFunctions::least_decimalv2(Expr* e, TupleRow* row) { - DCHECK_GE(e->get_num_children(), 1); - int32_t num_args = e->get_num_children(); - int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. - for (int i = 0; i < num_args; ++i) { - DecimalV2Value* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == nullptr) { - return nullptr; - } - if (*arg < *reinterpret_cast(e->children()[result_idx]->get_value(row))) { - result_idx = i; - } - } - return &e->children()[result_idx]->_result.decimalv2_val; -} - - -void* MathFunctions::least_string(Expr* e, TupleRow* row) { - DCHECK_GE(e->get_num_children(), 1); - int32_t num_args = e->get_num_children(); - int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. - for (int i = 0; i < num_args; ++i) { - StringValue* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == nullptr) { - return nullptr; - } - if (*arg < *reinterpret_cast(e->children()[result_idx]->get_value(row))) { - result_idx = i; - } - } - return &e->children()[result_idx]->_result.string_val; -} - -void* MathFunctions::least_timestamp(Expr* e, TupleRow* row) { - DCHECK_GE(e->get_num_children(), 1); - int32_t num_args = e->get_num_children(); - int result_idx = 0; - // NOTE: loop index starts at 0, so If frist arg is nullptr, we can return early.. - for (int i = 0; i < num_args; ++i) { - DateTimeValue* arg = reinterpret_cast(e->children()[i]->get_value(row)); - if (arg == nullptr) { - return nullptr; - } - if (*arg < *reinterpret_cast(e->children()[result_idx]->get_value(row))) { - result_idx = i; - } - } - return &e->children()[result_idx]->_result.datetime_val; -} - -#endif } // namespace doris diff --git a/be/src/exprs/new_agg_fn_evaluator.h b/be/src/exprs/new_agg_fn_evaluator.h index 822e8fc77d1bd6..36bdc2f21c4dbc 100644 --- a/be/src/exprs/new_agg_fn_evaluator.h +++ b/be/src/exprs/new_agg_fn_evaluator.h @@ -20,7 +20,6 @@ #include -#include "codegen/doris_ir.h" #include "common/compiler_util.h" #include "common/status.h" #include "exprs/agg_fn.h" @@ -108,9 +107,9 @@ class NewAggFnEvaluator { const AggFn& agg_fn() const { return agg_fn_; } - FunctionContext* IR_ALWAYS_INLINE agg_fn_ctx() const; + FunctionContext* agg_fn_ctx() const; - ExprContext* const* IR_ALWAYS_INLINE input_evals() const; + ExprContext* const* input_evals() const; /// Call the initialization function of the AggFn. May update 'dst'. void Init(Tuple* dst); diff --git a/be/src/exprs/operators.cpp b/be/src/exprs/operators.cpp index aaffc0b57608b7..b0a684a97a5c12 100644 --- a/be/src/exprs/operators.cpp +++ b/be/src/exprs/operators.cpp @@ -128,51 +128,6 @@ BITNOT_FN(IntVal, int_val); BITNOT_FN(BigIntVal, big_int_val); BITNOT_FN(LargeIntVal, large_int_val); -#if 0 -static const int64_t FACTORIAL_MAX = 20; -static const int64_t FACTORIAL_LOOKUP[] = { - 1LL, // 0! - 1LL, // 1! - 2LL, // 2! - 6LL, // 3! - 24LL, // 4! - 120LL, // 5! - 720LL, // 6! - 5040LL, // 7! - 40320LL, // 8! - 362880LL, // 9! - 3628800LL, // 10! - 39916800LL, // 11! - 479001600LL, // 12! - 6227020800LL, // 13! - 87178291200LL, // 14! - 1307674368000LL, // 15! - 20922789888000LL, // 16! - 355687428096000LL, // 17! - 6402373705728000LL, // 18! - 121645100408832000LL, // 19! - 2432902008176640000LL, // 20! -}; - -// Compute factorial - return -1 if out of range -// Factorial of any number <= 1 returns 1 -static int64_t ComputeFactorial(int64_t n) { - // Check range based on arg: 20! < 2^63 -1 < 21! - if (n > FACTORIAL_MAX) { - return -1; - } else if (n < 0) { - return 1; - } - - return FACTORIAL_LOOKUP[n]; -} - -FACTORIAL_FN(TinyIntVal); -FACTORIAL_FN(SmallIntVal); -FACTORIAL_FN(IntVal); -FACTORIAL_FN(BigIntVal); -#endif - BINARY_PREDICATE_ALL_TYPES(eq, ==); BINARY_PREDICATE_ALL_TYPES(ne, !=); BINARY_PREDICATE_ALL_TYPES(gt, >); diff --git a/be/src/exprs/operators.h b/be/src/exprs/operators.h index 54acb8143c63b1..b5383bd576d21c 100644 --- a/be/src/exprs/operators.h +++ b/be/src/exprs/operators.h @@ -28,13 +28,6 @@ class Operators { // Do nothing, just get its symbols static void init(); -#if 0 - static BigIntVal factorial_tiny_int_val(FunctionContext*, const TinyIntVal&); - static BigIntVal factorial_small_int_val(FunctionContext*, const SmallIntVal&); - static BigIntVal factorial_int_val(FunctionContext*, const IntVal&); - static BigIntVal factorial_big_int_val(FunctionContext*, const BigIntVal&); -#endif - // Bit operator static TinyIntVal bitnot_tiny_int_val(FunctionContext*, const TinyIntVal&); static SmallIntVal bitnot_small_int_val(FunctionContext*, const SmallIntVal&); diff --git a/be/src/exprs/scalar_fn_call.cpp b/be/src/exprs/scalar_fn_call.cpp index 5e1d081f5705b7..27e715160063e3 100644 --- a/be/src/exprs/scalar_fn_call.cpp +++ b/be/src/exprs/scalar_fn_call.cpp @@ -187,21 +187,6 @@ Status ScalarFnCall::get_function(RuntimeState* state, const std::string& symbol _fn.binary_type == TFunctionBinaryType::HIVE) { return UserFunctionCache::instance()->get_function_ptr(_fn.id, symbol, _fn.hdfs_location, _fn.checksum, fn, &_cache_entry); - } else { -#if 0 - DCHECK_EQ(_fn.binary_type, TFunctionBinaryType::IR); - LlvmCodeGen* codegen; - RETURN_IF_ERROR(state->GetCodegen(&codegen)); - Function* ir_fn = codegen->module()->getFunction(symbol); - if (ir_fn == nullptr) { - std::stringstream ss; - ss << "Unable to locate function " << symbol - << " from LLVM module " << _fn.hdfs_location; - return Status::InternalError(ss.str()); - } - codegen->AddFunctionToJit(ir_fn, fn); - return Status::OK()(); -#endif } return Status::OK(); } diff --git a/be/src/exprs/string_functions.h b/be/src/exprs/string_functions.h index 65b5418b38d04b..af13f4fbf8a13d 100644 --- a/be/src/exprs/string_functions.h +++ b/be/src/exprs/string_functions.h @@ -112,15 +112,6 @@ class StringFunctions { const doris_udf::StringVal& replace); static void regexp_close(doris_udf::FunctionContext*, doris_udf::FunctionContext::FunctionStateScope); -#if 0 - static void RegexpMatchCountPrepare(FunctionContext* context, - FunctionContext::FunctionStateScope scope); - static IntVal RegexpMatchCount2Args(FunctionContext* context, const StringVal& str, - const StringVal& pattern); - static IntVal RegexpMatchCount4Args(FunctionContext* context, const StringVal& str, - const StringVal& pattern, const IntVal& start_pos, - const StringVal& match_parameter); -#endif static StringVal concat(doris_udf::FunctionContext*, int num_children, const StringVal* strs); static StringVal concat_ws(doris_udf::FunctionContext*, const doris_udf::StringVal& sep, int num_children, const doris_udf::StringVal* strs); @@ -149,8 +140,9 @@ class StringFunctions { static doris_udf::StringVal money_format(doris_udf::FunctionContext* context, const doris_udf::LargeIntVal& v); - template static StringVal do_money_format(FunctionContext* context, const T int_value, - const int32_t frac_value = 0) { + template + static StringVal do_money_format(FunctionContext* context, const T int_value, + const int32_t frac_value = 0) { char local[N]; char* p = SimpleItoaWithCommas(int_value, local, sizeof(local)); int32_t string_val_len = local + sizeof(local) - p + 3; @@ -191,7 +183,7 @@ class StringFunctions { const StringVal& oldStr, const StringVal& newStr); static doris_udf::IntVal bit_length(doris_udf::FunctionContext* context, - const doris_udf::StringVal& str); + const doris_udf::StringVal& str); }; } // namespace doris diff --git a/be/src/geo/geo_types.cpp b/be/src/geo/geo_types.cpp index 936bb0408848d8..6c61f963e27c0b 100644 --- a/be/src/geo/geo_types.cpp +++ b/be/src/geo/geo_types.cpp @@ -319,12 +319,6 @@ bool GeoPolygon::contains(const GeoShape* rhs) const { case GEO_SHAPE_POINT: { const GeoPoint* point = (const GeoPoint*)rhs; return _polygon->Contains(point->point()); -#if 0 - if (_polygon->Contains(point->point())) { - return true; - } - return _polygon->MayIntersect(S2Cell(point->point())); -#endif } case GEO_SHAPE_LINE_STRING: { const GeoLine* line = (const GeoLine*)rhs; @@ -334,35 +328,6 @@ bool GeoPolygon::contains(const GeoShape* rhs) const { const GeoPolygon* other = (const GeoPolygon*)rhs; return _polygon->Contains(other->polygon()); } -#if 0 - case GEO_SHAPE_MULTI_POINT: { - const GeoMultiPoint* multi_point = (const GeoMultiPoint*)rhs; - for (auto& point : multi_point->points()) { - if (!_polygon.Contains(point)) { - return false; - } - } - return true; - } - case GEO_SHAPE_MULTI_LINE_STRING: { - const GeoMultiLine* multi_line = (const GeoMultiLine*)rhs; - for (auto line : multi_line->lines()) { - if (!_polygon.Contains(line)) { - return false; - } - } - return true; - } - case GEO_SHAPE_MULTI_POLYGON: { - const GeoMultiPolygon* multi_polygon = (const GeoMultiPolygon*)rhs; - for (auto polygon : multi_polygon->polygons()) { - if (!_polygon.Contains(polygon)) { - return false; - } - } - return true; - } -#endif default: return false; } @@ -387,50 +352,7 @@ bool GeoCircle::contains(const GeoShape* rhs) const { case GEO_SHAPE_POINT: { const GeoPoint* point = (const GeoPoint*)rhs; return _cap->Contains(point->point()); -#if 0 - if (_polygon->Contains(point->point())) { - return true; - } - return _polygon->MayIntersect(S2Cell(point->point())); -#endif - } -#if 0 - case GEO_SHAPE_LINE_STRING: { - const GeoLine* line = (const GeoLine*)rhs; - return _polygon->Contains(*line->polyline()); - } - case GEO_SHAPE_POLYGON: { - const GeoPolygon* other = (const GeoPolygon*)rhs; - return _polygon->Contains(other->polygon()); - } - case GEO_SHAPE_MULTI_POINT: { - const GeoMultiPoint* multi_point = (const GeoMultiPoint*)rhs; - for (auto& point : multi_point->points()) { - if (!_polygon.Contains(point)) { - return false; - } - } - return true; - } - case GEO_SHAPE_MULTI_LINE_STRING: { - const GeoMultiLine* multi_line = (const GeoMultiLine*)rhs; - for (auto line : multi_line->lines()) { - if (!_polygon.Contains(line)) { - return false; - } - } - return true; - } - case GEO_SHAPE_MULTI_POLYGON: { - const GeoMultiPolygon* multi_polygon = (const GeoMultiPolygon*)rhs; - for (auto polygon : multi_polygon->polygons()) { - if (!_polygon.Contains(polygon)) { - return false; - } - } - return true; } -#endif default: return false; } @@ -456,62 +378,4 @@ std::string GeoCircle::as_wkt() const { return ss.str(); } -#if 0 - -template -bool GeoMultiPolygon::_contains(const T rhs) { - for (auto polygon : _polygons) { - if (polygon->Contains(point->point())) { - return true; - } - } - return false; -} - -bool GeoMultiPolygon::contains(const GeoShape* rhs) { - switch (rhs->type()) { - case GEO_SHAPE_POINT: { - const GeoPoint* point = (const GeoPoint*)rhs; - return _contains(point->point()); - } - case GEO_SHAPE_LINE_STRING: { - const GeoLine* line = (const GeoLine*)rhs; - return _contains(line->polyline()); - } - case GEO_SHAPE_POLYGON: { - const GeoPolygon* polygon = (const GeoPolygon*)rhs; - return _contains(line->polygon()); - } - case GEO_SHAPE_MULTI_POINT: { - const GeoMultiPoint* multi_point = (const GeoMultiPoint*)rhs; - for (auto point : multi_point->points()) { - if (!_contains(point)) { - return false; - } - } - return true; - } - case GEO_SHAPE_LINE_STRING: { - const GeoMultiLine* multi_line = (const GeoMultiLine*)rhs; - for (auto line : multi_line->lines()) { - if (!_contains(line)) { - return false; - } - } - return true; - } - case GEO_SHAPE_POLYGON: { - const GeoMultiPolygon* multi_polygon = (const GeoMultiPolygon*)rhs; - for (auto polygon : multi_polygon->polygons()) { - if (!_contains(polygon)) { - return false; - } - } - return true; - } - } - return false; -} -#endif - } // namespace doris diff --git a/be/src/geo/geo_types.h b/be/src/geo/geo_types.h index 3704f20256ee98..0a717cf403cc1f 100644 --- a/be/src/geo/geo_types.h +++ b/be/src/geo/geo_types.h @@ -144,56 +144,4 @@ class GeoCircle : public GeoShape { std::unique_ptr _cap; }; -#if 0 -class GeoMultiPoint : public GeoShape { -public: - GeoPolygon(); - ~GeoPolygon() override; - - GeoShapeType type() const override { return GEO_SHAPE_POLYGON; } - const std::vector& points() const { return _points; } - -private: - std::vector _points; -}; - -class GeoMultiLine : public GeoShape { -public: - GeoMultiLine(); - ~GeoMultiLine() override; - - GeoShapeType type() const override { return GEO_SHAPE_MULTI_LINE_STRING; } - const std::vector& polylines() const { return _polylines; } - -private: - std::vector _polylines; -}; - -class GeoMultiPolygon : public GeoShape { -public: - GeoMultiPolygon(); - ~GeoMultiPolygon() override; - - GeoShapeType type() const override { return GEO_SHAPE_MULTI_POLYGON; } - - const std::vector& polygons() const { return _polygons; } - - - bool contains(const GeoShape* rhs) override; -private: - std::vector _polygons; -}; - -#if 0 -class GeoEnvelope : public GeoShape { -public: -}; - -class GeoCircle : public GeoShape { -public: -}; -#endif - -#endif - } // namespace doris diff --git a/be/src/gutil/atomicops.h b/be/src/gutil/atomicops.h index bcf78f0552345f..1d04dd523440dc 100644 --- a/be/src/gutil/atomicops.h +++ b/be/src/gutil/atomicops.h @@ -185,75 +185,6 @@ inline AtomicWord Release_Load(volatile const AtomicWord* ptr) { // of the interface provided by this module. // ------------------------------------------------------------------------ -#if 0 - -// Signed 32-bit type that supports the atomic ops below, as well as atomic -// loads and stores. Instances must be naturally aligned. This type differs -// from AtomicWord in 64-bit binaries where AtomicWord is 64-bits. -typedef int32_t Atomic32; - -// Corresponding operations on Atomic32 -namespace base { -namespace subtle { - -// Signed 64-bit type that supports the atomic ops below, as well as atomic -// loads and stores. Instances must be naturally aligned. This type differs -// from AtomicWord in 32-bit binaries where AtomicWord is 32-bits. -typedef int64_t Atomic64; - -Atomic32 NoBarrier_CompareAndSwap(volatile Atomic32* ptr, - Atomic32 old_value, - Atomic32 new_value); -Atomic32 NoBarrier_AtomicExchange(volatile Atomic32* ptr, Atomic32 new_value); -Atomic32 Acquire_AtomicExchange(volatile Atomic32* ptr, Atomic32 new_value); -Atomic32 Release_AtomicExchange(volatile Atomic32* ptr, Atomic32 new_value); -Atomic32 NoBarrier_AtomicIncrement(volatile Atomic32* ptr, Atomic32 increment); -Atomic32 Barrier_AtomicIncrement(volatile Atomic32* ptr, - Atomic32 increment); -Atomic32 Acquire_CompareAndSwap(volatile Atomic32* ptr, - Atomic32 old_value, - Atomic32 new_value); -Atomic32 Release_CompareAndSwap(volatile Atomic32* ptr, - Atomic32 old_value, - Atomic32 new_value); -void NoBarrier_Store(volatile Atomic32* ptr, Atomic32 value); -void Acquire_Store(volatile Atomic32* ptr, Atomic32 value); -void Release_Store(volatile Atomic32* ptr, Atomic32 value); -Atomic32 NoBarrier_Load(volatile const Atomic32* ptr); -Atomic32 Acquire_Load(volatile const Atomic32* ptr); -Atomic32 Release_Load(volatile const Atomic32* ptr); - -// Corresponding operations on Atomic64 -Atomic64 NoBarrier_CompareAndSwap(volatile Atomic64* ptr, - Atomic64 old_value, - Atomic64 new_value); -Atomic64 NoBarrier_AtomicExchange(volatile Atomic64* ptr, Atomic64 new_value); -Atomic64 Acquire_AtomicExchange(volatile Atomic64* ptr, Atomic64 new_value); -Atomic64 Release_AtomicExchange(volatile Atomic64* ptr, Atomic64 new_value); -Atomic64 NoBarrier_AtomicIncrement(volatile Atomic64* ptr, Atomic64 increment); -Atomic64 Barrier_AtomicIncrement(volatile Atomic64* ptr, Atomic64 increment); - -Atomic64 Acquire_CompareAndSwap(volatile Atomic64* ptr, - Atomic64 old_value, - Atomic64 new_value); -Atomic64 Release_CompareAndSwap(volatile Atomic64* ptr, - Atomic64 old_value, - Atomic64 new_value); -void NoBarrier_Store(volatile Atomic64* ptr, Atomic64 value); -void Acquire_Store(volatile Atomic64* ptr, Atomic64 value); -void Release_Store(volatile Atomic64* ptr, Atomic64 value); -Atomic64 NoBarrier_Load(volatile const Atomic64* ptr); -Atomic64 Acquire_Load(volatile const Atomic64* ptr); -Atomic64 Release_Load(volatile const Atomic64* ptr); -} // namespace base::subtle -} // namespace base - -void MemoryBarrier(); - -void PauseCPU(); - -#endif // 0 - // ------------------------------------------------------------------------ // The following are to be deprecated when all uses have been changed to // use the base::subtle namespace. diff --git a/be/src/gutil/threading/thread_collision_warner.cc b/be/src/gutil/threading/thread_collision_warner.cc index 1a2991a14947d3..d2f1e47f8e02d9 100644 --- a/be/src/gutil/threading/thread_collision_warner.cc +++ b/be/src/gutil/threading/thread_collision_warner.cc @@ -23,22 +23,6 @@ void DCheckAsserter::warn(int64_t previous_thread_id, int64_t current_thread_id) << ", current thread id: " << current_thread_id; } -#if 0 -// Original source from Chromium -- we didn't import their threading library -// into Kudu source as of yet - -static subtle::Atomic32 CurrentThread() { - const PlatformThreadId current_thread_id = PlatformThread::CurrentId(); - // We need to get the thread id into an atomic data type. This might be a - // truncating conversion, but any loss-of-information just increases the - // chance of a fault negative, not a false positive. - const subtle::Atomic32 atomic_thread_id = - static_cast(current_thread_id); - - return atomic_thread_id; -} -#else - static subtle::Atomic64 CurrentThread() { #if defined(__APPLE__) uint64_t tid; @@ -49,8 +33,6 @@ static subtle::Atomic64 CurrentThread() { #endif } -#endif - void ThreadCollisionWarner::EnterSelf() { // If the active thread is 0 then I'll write the current thread ID // if two or more threads arrive here only one will succeed to diff --git a/be/src/http/action/health_action.cpp b/be/src/http/action/health_action.cpp index 92546954f69fff..26147403932650 100644 --- a/be/src/http/action/health_action.cpp +++ b/be/src/http/action/health_action.cpp @@ -42,10 +42,6 @@ void HealthAction::handle(HttpRequest* req) { req->add_output_header(HttpHeaders::CONTENT_TYPE, HEADER_JSON.c_str()); HttpChannel::send_reply(req, HttpStatus::OK, result); -#if 0 - HttpResponse response(HttpStatus::OK, HEADER_JSON, &result); - channel->send_response(response); -#endif } } // end namespace doris diff --git a/be/src/http/action/monitor_action.cpp b/be/src/http/action/monitor_action.cpp index 138582b75128d2..cf1604623e08b0 100644 --- a/be/src/http/action/monitor_action.cpp +++ b/be/src/http/action/monitor_action.cpp @@ -41,30 +41,18 @@ void MonitorAction::handle(HttpRequest* req) { const std::string& module = req->param(MODULE_KEY); if (module.empty()) { std::string err_msg = "No module params\n"; -#if 0 - HttpResponse response(HttpStatus::OK, &err_msg); - channel->send_response(response); -#endif HttpChannel::send_reply(req, HttpStatus::OK, err_msg); return; } if (_module_by_name.find(module) == _module_by_name.end()) { std::string err_msg = "Unknown module("; err_msg += module + ")\n"; -#if 0 - HttpResponse response(HttpStatus::OK, &err_msg); - channel->send_response(response); -#endif HttpChannel::send_reply(req, HttpStatus::OK, err_msg); return; } std::stringstream ss; _module_by_name[module]->debug(ss); std::string str = ss.str(); -#if 0 - HttpResponse response(HttpStatus::OK, &str); - channel->send_response(response); -#endif HttpChannel::send_reply(req, HttpStatus::OK, str); } diff --git a/be/src/olap/rowset/column_data.cpp b/be/src/olap/rowset/column_data.cpp index 4f18a9cbb98e04..224367480c6f7c 100644 --- a/be/src/olap/rowset/column_data.cpp +++ b/be/src/olap/rowset/column_data.cpp @@ -245,7 +245,7 @@ OLAPStatus ColumnData::_find_position_by_full_key(const RowCursor& key, bool fin it_result = std::upper_bound(it_start, it_end, key, comparator); } VLOG_NOTICE << "get result iterator. offset=" << *it_result - << ", start_pos=" << start_position.to_string(); + << ", start_pos=" << start_position.to_string(); } catch (std::exception& e) { LOG(WARNING) << "exception happens when doing seek. exception=" << e.what(); return OLAP_ERR_STL_ERROR; @@ -599,14 +599,6 @@ OLAPStatus ColumnData::_get_block_from_reader(VectorizedRowBatch** got_batch, bo } // If this is normal read do { -#if 0 - LOG(INFO) << "_current_segment is " << _current_segment - << ", _next_block:" << _next_block - << ", _end_segment::" << _end_segment - << ", _end_block:" << _end_block - << ", _end_row_index:" << _end_row_index - << ", _segment_eof:" << _segment_eof; -#endif vec_batch->clear(); if (rows_read > 0) { vec_batch->set_limit(rows_read); diff --git a/be/src/olap/rowset/column_writer.cpp b/be/src/olap/rowset/column_writer.cpp index 8c7b3b5153d29e..5ca0ce94ea07a3 100644 --- a/be/src/olap/rowset/column_writer.cpp +++ b/be/src/olap/rowset/column_writer.cpp @@ -638,35 +638,6 @@ OLAPStatus VarStringColumnWriter::_finalize_direct_encoding() { //uint32_t block_id = 0; _dict_stream->suppress(); -#if 0 - - for (uint32_t i = 0; i <= _string_id.size(); i++) { - //Unlike other types, the record position of string will write entries to _block_row_count - // Other types have no effect until the next call to create_index_row_entry. - while (block_id < _block_row_count.size() - 1 && - i == _block_row_count[block_id]) { - _data_stream->get_position(index()->mutable_entry(block_id)); - _length_writer->get_position(index()->mutable_entry(block_id)); - block_id++; - } - - if (i != _string_id.size()) { - const std::string& str = _string_keys[_string_id[i]]; - - if (OLAP_SUCCESS != (res = _data_stream->write(str.c_str(), - str.length()))) { - OLAP_LOG_WARNING("fail to write string content."); - return res; - } - - if (OLAP_SUCCESS != (res = _length_writer->write(str.length()))) { - OLAP_LOG_WARNING("fail to write string length."); - return res; - } - } - } - -#endif return OLAP_SUCCESS; } diff --git a/be/src/olap/rowset/segment_v2/column_reader.h b/be/src/olap/rowset/segment_v2/column_reader.h index e2e6c35ee42582..db77577788d004 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.h +++ b/be/src/olap/rowset/segment_v2/column_reader.h @@ -123,7 +123,8 @@ class ColumnReader { // get row ranges with zone map // - cond_column is user's query predicate // - delete_condition is a delete predicate of one version - Status get_row_ranges_by_zone_map(CondColumn* cond_column, CondColumn* delete_condition, RowRanges* row_ranges); + Status get_row_ranges_by_zone_map(CondColumn* cond_column, CondColumn* delete_condition, + RowRanges* row_ranges); // get row ranges with bloom filter index Status get_row_ranges_by_bloom_filter(CondColumn* cond_column, RowRanges* row_ranges); @@ -159,7 +160,8 @@ class ColumnReader { void _parse_zone_map(const ZoneMapPB& zone_map, WrapperField* min_value_container, WrapperField* max_value_container) const; - Status _get_filtered_pages(CondColumn* cond_column, CondColumn* delete_conditions, std::vector* page_indexes); + Status _get_filtered_pages(CondColumn* cond_column, CondColumn* delete_conditions, + std::vector* page_indexes); Status _calculate_row_ranges(const std::vector& page_indexes, RowRanges* row_ranges); @@ -215,7 +217,7 @@ class ColumnIterator { return next_batch(n, dst, &has_null); } - Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst) { + Status next_batch(size_t* n, vectorized::MutableColumnPtr& dst) { bool has_null; return next_batch(n, dst, &has_null); } @@ -225,7 +227,7 @@ class ColumnIterator { // from MemPool virtual Status next_batch(size_t* n, ColumnBlockView* dst, bool* has_null) = 0; - virtual Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst, bool* has_null) { + virtual Status next_batch(size_t* n, vectorized::MutableColumnPtr& dst, bool* has_null) { return Status::NotSupported("not implement"); } @@ -240,24 +242,6 @@ class ColumnIterator { return Status::OK(); } -#if 0 - // Call this function every time before next_batch. - // This function will preload pages from disk into memory if necessary. - Status prepare_batch(size_t n); - - // Fetch the next vector of values from the page into 'dst'. - // The output vector must have space for up to n cells. - // - // return the size of entries. - // - // In the case that the values are themselves references - // to other memory (eg Slices), the referred-to memory is - // allocated in the dst column vector's MemPool. - Status scan(size_t* n, ColumnBlock* dst, MemPool* pool); - - // release next_batch related resource - Status finish_batch(); -#endif protected: ColumnIteratorOptions _opts; }; @@ -277,7 +261,7 @@ class FileColumnIterator final : public ColumnIterator { Status next_batch(size_t* n, ColumnBlockView* dst, bool* has_null) override; - Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst, bool* has_null) override; + Status next_batch(size_t* n, vectorized::MutableColumnPtr& dst, bool* has_null) override; ordinal_t get_current_ordinal() const override { return _current_ordinal; } @@ -326,10 +310,8 @@ class FileColumnIterator final : public ColumnIterator { class ArrayFileColumnIterator final : public ColumnIterator { public: - explicit ArrayFileColumnIterator(ColumnReader* reader, - FileColumnIterator* length_reader, - ColumnIterator* item_iterator, - ColumnIterator* null_iterator); + explicit ArrayFileColumnIterator(ColumnReader* reader, FileColumnIterator* length_reader, + ColumnIterator* item_iterator, ColumnIterator* null_iterator); ~ArrayFileColumnIterator() override = default; @@ -354,15 +336,19 @@ class ArrayFileColumnIterator final : public ColumnIterator { RETURN_IF_ERROR(_length_iterator->seek_to_page_start()); if (_length_iterator->get_current_ordinal() == ord) { - RETURN_IF_ERROR(_item_iterator->seek_to_ordinal(_length_iterator->get_current_page()->first_array_item_ordinal)); + RETURN_IF_ERROR(_item_iterator->seek_to_ordinal( + _length_iterator->get_current_page()->first_array_item_ordinal)); } else { - ordinal_t start_offset_in_this_page = _length_iterator->get_current_page()->first_array_item_ordinal; + ordinal_t start_offset_in_this_page = + _length_iterator->get_current_page()->first_array_item_ordinal; ColumnBlock ordinal_block(_length_batch.get(), nullptr); ordinal_t size_to_read = ord - start_offset_in_this_page; bool has_null = false; ordinal_t item_ordinal = start_offset_in_this_page; while (size_to_read > 0) { - size_t this_read = _length_batch->capacity() < size_to_read ? _length_batch->capacity() : size_to_read; + size_t this_read = _length_batch->capacity() < size_to_read + ? _length_batch->capacity() + : size_to_read; ColumnBlockView ordinal_view(&ordinal_block); RETURN_IF_ERROR(_length_iterator->next_batch(&this_read, &ordinal_view, &has_null)); auto* ordinals = reinterpret_cast(_length_batch->data()); @@ -415,19 +401,19 @@ class DefaultValueColumnIterator : public ColumnIterator { return Status::OK(); } - Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst) { + Status next_batch(size_t* n, vectorized::MutableColumnPtr& dst) { bool has_null; return next_batch(n, dst, &has_null); } Status next_batch(size_t* n, ColumnBlockView* dst, bool* has_null) override; - Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst, bool* has_null) override; + Status next_batch(size_t* n, vectorized::MutableColumnPtr& dst, bool* has_null) override; ordinal_t get_current_ordinal() const override { return _current_rowid; } private: - void insert_default_data(vectorized::MutableColumnPtr &dst, size_t n); + void insert_default_data(vectorized::MutableColumnPtr& dst, size_t n); bool _has_default_value; std::string _default_value; diff --git a/be/src/runtime/CMakeLists.txt b/be/src/runtime/CMakeLists.txt index 686207dbd5979f..142934d5cbf602 100644 --- a/be/src/runtime/CMakeLists.txt +++ b/be/src/runtime/CMakeLists.txt @@ -38,7 +38,6 @@ set(RUNTIME_FILES plan_fragment_executor.cpp primitive_type.cpp raw_value.cpp - raw_value_ir.cpp result_sink.cpp result_file_sink.cpp result_buffer_mgr.cpp @@ -48,7 +47,6 @@ set(RUNTIME_FILES runtime_filter_mgr.cpp string_value.cpp thread_resource_mgr.cpp - # timestamp_value.cpp decimalv2_value.cpp large_int_value.cpp collection_value.cpp @@ -73,9 +71,7 @@ set(RUNTIME_FILES sorted_run_merger.cc data_stream_recvr.cc buffered_tuple_stream2.cc - buffered_tuple_stream2_ir.cc buffered_tuple_stream3.cc - # export_task_mgr.cpp export_sink.cpp load_channel_mgr.cpp load_channel.cpp diff --git a/be/src/runtime/buffered_tuple_stream2.cc b/be/src/runtime/buffered_tuple_stream2.cc index 69fc2c440e8ba6..d0c9428ffee0a7 100644 --- a/be/src/runtime/buffered_tuple_stream2.cc +++ b/be/src/runtime/buffered_tuple_stream2.cc @@ -662,34 +662,6 @@ void BufferedTupleStream2::read_strings(const vector& string_sl } } -#if 0 -void BufferedTupleStream2::ReadCollections(const vector& collection_slots, - int data_len, Tuple* tuple) { - DCHECK(tuple != nullptr); - for (int i = 0; i < collection_slots.size(); ++i) { - const SlotDescriptor* slot_desc = collection_slots[i]; - if (tuple->IsNull(slot_desc->null_indicator_offset())) continue; - - CollectionValue* cv = tuple->get_collectionslot(slot_desc->tuple_offset()); - const TupleDescriptor& item_desc = *slot_desc->collection_item_descriptor(); - int coll_byte_size = cv->num_tuples * item_desc.byte_size(); - DCHECK_LE(coll_byte_size, data_len - _read_bytes); - cv->ptr = reinterpret_cast(_read_ptr); - _read_ptr += coll_byte_size; - _read_bytes += coll_byte_size; - - if (!item_desc.HasVarlenSlots()) continue; - uint8_t* coll_data = cv->ptr; - for (int j = 0; j < cv->num_tuples; ++j) { - Tuple* item = reinterpret_cast(coll_data); - read_strings(item_desc.string_slots(), data_len, item); - ReadCollections(item_desc.collection_slots(), data_len, item); - coll_data += item_desc.byte_size(); - } - } -} -#endif - int64_t BufferedTupleStream2::compute_row_size(TupleRow* row) const { int64_t size = 0; for (int i = 0; i < _desc.tuple_descriptors().size(); ++i) { @@ -704,4 +676,127 @@ int64_t BufferedTupleStream2::compute_row_size(TupleRow* row) const { return size; } +bool BufferedTupleStream2::deep_copy(TupleRow* row) { + if (_nullable_tuple) { + return deep_copy_internal(row); + } else { + return deep_copy_internal(row); + } +} + +// TODO: this really needs codegen +// TODO: in case of duplicate tuples, this can redundantly serialize data. +template +bool BufferedTupleStream2::deep_copy_internal(TupleRow* row) { + if (UNLIKELY(_write_block == nullptr)) { + return false; + } + DCHECK_GE(_null_indicators_write_block, 0); + DCHECK(_write_block->is_pinned()) << debug_string() << std::endl + << _write_block->debug_string(); + + const uint64_t tuples_per_row = _desc.tuple_descriptors().size(); + if (UNLIKELY((_write_block->bytes_remaining() < _fixed_tuple_row_size) || + (HasNullableTuple && + (_write_tuple_idx + tuples_per_row > _null_indicators_write_block * 8)))) { + return false; + } + // Allocate the maximum possible buffer for the fixed portion of the tuple. + uint8_t* tuple_buf = _write_block->allocate(_fixed_tuple_row_size); + // Total bytes allocated in _write_block for this row. Saved so we can roll back + // if this row doesn't fit. + int bytes_allocated = _fixed_tuple_row_size; + + // Copy the not nullptr fixed len tuples. For the nullptr tuples just update the nullptr tuple + // indicator. + if (HasNullableTuple) { + DCHECK_GT(_null_indicators_write_block, 0); + uint8_t* null_word = nullptr; + uint32_t null_pos = 0; + // Calculate how much space it should return. + int to_return = 0; + for (int i = 0; i < tuples_per_row; ++i) { + null_word = _write_block->buffer() + (_write_tuple_idx >> 3); // / 8 + null_pos = _write_tuple_idx & 7; + ++_write_tuple_idx; + const int tuple_size = _desc.tuple_descriptors()[i]->byte_size(); + Tuple* t = row->get_tuple(i); + const uint8_t mask = 1 << (7 - null_pos); + if (t != nullptr) { + *null_word &= ~mask; + memcpy(tuple_buf, t, tuple_size); + tuple_buf += tuple_size; + } else { + *null_word |= mask; + to_return += tuple_size; + } + } + DCHECK_LE(_write_tuple_idx - 1, _null_indicators_write_block * 8); + _write_block->return_allocation(to_return); + bytes_allocated -= to_return; + } else { + // If we know that there are no nullable tuples no need to set the nullability flags. + DCHECK_EQ(_null_indicators_write_block, 0); + for (int i = 0; i < tuples_per_row; ++i) { + const int tuple_size = _desc.tuple_descriptors()[i]->byte_size(); + Tuple* t = row->get_tuple(i); + // TODO: Once IMPALA-1306 (Avoid passing empty tuples of non-materialized slots) + // is delivered, the check below should become DCHECK(t != nullptr). + DCHECK(t != nullptr || tuple_size == 0); + memcpy(tuple_buf, t, tuple_size); + tuple_buf += tuple_size; + } + } + + // Copy string slots. Note: we do not need to convert the string ptrs to offsets + // on the write path, only on the read. The tuple data is immediately followed + // by the string data so only the len information is necessary. + for (int i = 0; i < _string_slots.size(); ++i) { + Tuple* tuple = row->get_tuple(_string_slots[i].first); + if (HasNullableTuple && tuple == nullptr) { + continue; + } + if (UNLIKELY(!copy_strings(tuple, _string_slots[i].second, &bytes_allocated))) { + _write_block->return_allocation(bytes_allocated); + return false; + } + } + + // Copy collection slots. We copy collection data in a well-defined order so we do not + // need to convert pointers to offsets on the write path. + // for (int i = 0; i < _collection_slots.size(); ++i) { + // Tuple* tuple = row->get_tuple(_collection_slots[i].first); + // if (HasNullableTuple && tuple == nullptr) continue; + // if (UNLIKELY(!copy_collections(tuple, _collection_slots[i].second, + // &bytes_allocated))) { + // _write_block->return_allocation(bytes_allocated); + // return false; + // } + // } + + _write_block->add_row(); + ++_num_rows; + return true; +} + +bool BufferedTupleStream2::copy_strings(const Tuple* tuple, + const vector& string_slots, + int* bytes_allocated) { + for (int i = 0; i < string_slots.size(); ++i) { + const SlotDescriptor* slot_desc = string_slots[i]; + if (tuple->is_null(slot_desc->null_indicator_offset())) { + continue; + } + const StringValue* sv = tuple->get_string_slot(slot_desc->tuple_offset()); + if (LIKELY(sv->len > 0)) { + if (UNLIKELY(_write_block->bytes_remaining() < sv->len)) { + return false; + } + uint8_t* buf = _write_block->allocate(sv->len); + (*bytes_allocated) += sv->len; + memcpy(buf, sv->ptr, sv->len); + } + } + return true; +} } // end namespace doris diff --git a/be/src/runtime/buffered_tuple_stream2_ir.cc b/be/src/runtime/buffered_tuple_stream2_ir.cc deleted file mode 100644 index 07c356414f525a..00000000000000 --- a/be/src/runtime/buffered_tuple_stream2_ir.cc +++ /dev/null @@ -1,187 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "runtime/buffered_tuple_stream2.inline.h" - -// #include "runtime/collection-value.h" -#include "runtime/descriptors.h" -#include "runtime/tuple_row.h" - -using std::vector; - -namespace doris { - -bool BufferedTupleStream2::deep_copy(TupleRow* row) { - if (_nullable_tuple) { - return deep_copy_internal(row); - } else { - return deep_copy_internal(row); - } -} - -// TODO: this really needs codegen -// TODO: in case of duplicate tuples, this can redundantly serialize data. -template -bool BufferedTupleStream2::deep_copy_internal(TupleRow* row) { - if (UNLIKELY(_write_block == nullptr)) { - return false; - } - DCHECK_GE(_null_indicators_write_block, 0); - DCHECK(_write_block->is_pinned()) << debug_string() << std::endl - << _write_block->debug_string(); - - const uint64_t tuples_per_row = _desc.tuple_descriptors().size(); - if (UNLIKELY((_write_block->bytes_remaining() < _fixed_tuple_row_size) || - (HasNullableTuple && - (_write_tuple_idx + tuples_per_row > _null_indicators_write_block * 8)))) { - return false; - } - // Allocate the maximum possible buffer for the fixed portion of the tuple. - uint8_t* tuple_buf = _write_block->allocate(_fixed_tuple_row_size); - // Total bytes allocated in _write_block for this row. Saved so we can roll back - // if this row doesn't fit. - int bytes_allocated = _fixed_tuple_row_size; - - // Copy the not nullptr fixed len tuples. For the nullptr tuples just update the nullptr tuple - // indicator. - if (HasNullableTuple) { - DCHECK_GT(_null_indicators_write_block, 0); - uint8_t* null_word = nullptr; - uint32_t null_pos = 0; - // Calculate how much space it should return. - int to_return = 0; - for (int i = 0; i < tuples_per_row; ++i) { - null_word = _write_block->buffer() + (_write_tuple_idx >> 3); // / 8 - null_pos = _write_tuple_idx & 7; - ++_write_tuple_idx; - const int tuple_size = _desc.tuple_descriptors()[i]->byte_size(); - Tuple* t = row->get_tuple(i); - const uint8_t mask = 1 << (7 - null_pos); - if (t != nullptr) { - *null_word &= ~mask; - memcpy(tuple_buf, t, tuple_size); - tuple_buf += tuple_size; - } else { - *null_word |= mask; - to_return += tuple_size; - } - } - DCHECK_LE(_write_tuple_idx - 1, _null_indicators_write_block * 8); - _write_block->return_allocation(to_return); - bytes_allocated -= to_return; - } else { - // If we know that there are no nullable tuples no need to set the nullability flags. - DCHECK_EQ(_null_indicators_write_block, 0); - for (int i = 0; i < tuples_per_row; ++i) { - const int tuple_size = _desc.tuple_descriptors()[i]->byte_size(); - Tuple* t = row->get_tuple(i); - // TODO: Once IMPALA-1306 (Avoid passing empty tuples of non-materialized slots) - // is delivered, the check below should become DCHECK(t != nullptr). - DCHECK(t != nullptr || tuple_size == 0); - memcpy(tuple_buf, t, tuple_size); - tuple_buf += tuple_size; - } - } - - // Copy string slots. Note: we do not need to convert the string ptrs to offsets - // on the write path, only on the read. The tuple data is immediately followed - // by the string data so only the len information is necessary. - for (int i = 0; i < _string_slots.size(); ++i) { - Tuple* tuple = row->get_tuple(_string_slots[i].first); - if (HasNullableTuple && tuple == nullptr) { - continue; - } - if (UNLIKELY(!copy_strings(tuple, _string_slots[i].second, &bytes_allocated))) { - _write_block->return_allocation(bytes_allocated); - return false; - } - } - - // Copy collection slots. We copy collection data in a well-defined order so we do not - // need to convert pointers to offsets on the write path. - // for (int i = 0; i < _collection_slots.size(); ++i) { - // Tuple* tuple = row->get_tuple(_collection_slots[i].first); - // if (HasNullableTuple && tuple == nullptr) continue; - // if (UNLIKELY(!copy_collections(tuple, _collection_slots[i].second, - // &bytes_allocated))) { - // _write_block->return_allocation(bytes_allocated); - // return false; - // } - // } - - _write_block->add_row(); - ++_num_rows; - return true; -} - -bool BufferedTupleStream2::copy_strings(const Tuple* tuple, - const vector& string_slots, - int* bytes_allocated) { - for (int i = 0; i < string_slots.size(); ++i) { - const SlotDescriptor* slot_desc = string_slots[i]; - if (tuple->is_null(slot_desc->null_indicator_offset())) { - continue; - } - const StringValue* sv = tuple->get_string_slot(slot_desc->tuple_offset()); - if (LIKELY(sv->len > 0)) { - if (UNLIKELY(_write_block->bytes_remaining() < sv->len)) { - return false; - } - uint8_t* buf = _write_block->allocate(sv->len); - (*bytes_allocated) += sv->len; - memcpy(buf, sv->ptr, sv->len); - } - } - return true; -} - -#if 0 -bool BufferedTupleStream2::copy_collections(const Tuple* tuple, - const vector& collection_slots, int* bytes_allocated) { - for (int i = 0; i < collection_slots.size(); ++i) { - const SlotDescriptor* slot_desc = collection_slots[i]; - if (tuple->IsNull(slot_desc->null_indicator_offset())) continue; - const CollectionValue* cv = tuple->GetCollectionSlot(slot_desc->tuple_offset()); - const TupleDescriptor& item_desc = *slot_desc->collection_item_descriptor(); - if (LIKELY(cv->num_tuples > 0)) { - int coll_byte_size = cv->num_tuples * item_desc.byte_size(); - if (UNLIKELY(_write_block->BytesRemaining() < coll_byte_size)) { - return false; - } - uint8_t* coll_data = _write_block->allocate(coll_byte_size); - (*bytes_allocated) += coll_byte_size; - memcpy(coll_data, cv->ptr, coll_byte_size); - if (!item_desc.HasVarlenSlots()) continue; - // Copy variable length data when present in collection items. - for (int j = 0; j < cv->num_tuples; ++j) { - Tuple* item = reinterpret_cast(coll_data); - if (UNLIKELY(!copy_strings(item, item_desc.string_slots(), bytes_allocated))) { - return false; - } - if (UNLIKELY(!copy_collections(item, item_desc.collection_slots(), - bytes_allocated))) { - return false; - } - coll_data += item_desc.byte_size(); - } - } - } - return true; -} -#endif - -} // end namespace doris diff --git a/be/src/runtime/buffered_tuple_stream3.cc b/be/src/runtime/buffered_tuple_stream3.cc index 2c7ce31992e5d8..e5bdb9ecd0e532 100644 --- a/be/src/runtime/buffered_tuple_stream3.cc +++ b/be/src/runtime/buffered_tuple_stream3.cc @@ -19,7 +19,6 @@ #include "runtime/buffered_tuple_stream3.inline.h" #include "runtime/bufferpool/reservation_tracker.h" -//#include "runtime/collection_value.h" #include "runtime/descriptors.h" #include "runtime/exec_env.h" #include "runtime/mem_tracker.h" diff --git a/be/src/runtime/disk_io_mgr.h b/be/src/runtime/disk_io_mgr.h index 9b0d330b941595..af988fb73ee067 100644 --- a/be/src/runtime/disk_io_mgr.h +++ b/be/src/runtime/disk_io_mgr.h @@ -27,7 +27,6 @@ #include "common/atomic.h" #include "common/config.h" -#include "common/hdfs.h" #include "common/object_pool.h" #include "common/status.h" #include "runtime/mem_tracker.h" diff --git a/be/src/runtime/raw_value.cpp b/be/src/runtime/raw_value.cpp index b6647c9eadec23..f1012337d047db 100644 --- a/be/src/runtime/raw_value.cpp +++ b/be/src/runtime/raw_value.cpp @@ -408,4 +408,94 @@ void RawValue::write(const void* value, Tuple* tuple, const SlotDescriptor* slot } } +int RawValue::compare(const void* v1, const void* v2, const TypeDescriptor& type) { + const StringValue* string_value1; + const StringValue* string_value2; + const DateTimeValue* ts_value1; + const DateTimeValue* ts_value2; + float f1 = 0; + float f2 = 0; + double d1 = 0; + double d2 = 0; + int32_t i1; + int32_t i2; + int64_t b1; + int64_t b2; + + if (nullptr == v1 && nullptr == v2) { + return 0; + } else if (nullptr == v1 && nullptr != v2) { + return -1; + } else if (nullptr != v1 && nullptr == v2) { + return 1; + } + + switch (type.type) { + case TYPE_NULL: + return 0; + + case TYPE_BOOLEAN: + return *reinterpret_cast(v1) - *reinterpret_cast(v2); + + case TYPE_TINYINT: + return *reinterpret_cast(v1) - *reinterpret_cast(v2); + + case TYPE_SMALLINT: + return *reinterpret_cast(v1) - *reinterpret_cast(v2); + + case TYPE_INT: + i1 = *reinterpret_cast(v1); + i2 = *reinterpret_cast(v2); + return i1 > i2 ? 1 : (i1 < i2 ? -1 : 0); + + case TYPE_BIGINT: + b1 = *reinterpret_cast(v1); + b2 = *reinterpret_cast(v2); + return b1 > b2 ? 1 : (b1 < b2 ? -1 : 0); + + case TYPE_FLOAT: + // TODO: can this be faster? (just returning the difference has underflow problems) + f1 = *reinterpret_cast(v1); + f2 = *reinterpret_cast(v2); + return f1 > f2 ? 1 : (f1 < f2 ? -1 : 0); + + case TYPE_DOUBLE: + // TODO: can this be faster? + d1 = *reinterpret_cast(v1); + d2 = *reinterpret_cast(v2); + return d1 > d2 ? 1 : (d1 < d2 ? -1 : 0); + + case TYPE_CHAR: + case TYPE_VARCHAR: + case TYPE_HLL: + case TYPE_STRING: + string_value1 = reinterpret_cast(v1); + string_value2 = reinterpret_cast(v2); + return string_value1->compare(*string_value2); + + case TYPE_DATE: + case TYPE_DATETIME: + ts_value1 = reinterpret_cast(v1); + ts_value2 = reinterpret_cast(v2); + return *ts_value1 > *ts_value2 ? 1 : (*ts_value1 < *ts_value2 ? -1 : 0); + + case TYPE_DECIMALV2: { + DecimalV2Value decimal_value1(reinterpret_cast(v1)->value); + DecimalV2Value decimal_value2(reinterpret_cast(v2)->value); + return (decimal_value1 > decimal_value2) ? 1 : (decimal_value1 < decimal_value2 ? -1 : 0); + } + + case TYPE_LARGEINT: { + __int128 large_int_value1 = reinterpret_cast(v1)->value; + __int128 large_int_value2 = reinterpret_cast(v2)->value; + return large_int_value1 > large_int_value2 ? 1 + : (large_int_value1 < large_int_value2 ? -1 : 0); + } + + default: + DCHECK(false) << "invalid type: " << type.type; + return 0; + }; +} + } // namespace doris diff --git a/be/src/runtime/raw_value_ir.cpp b/be/src/runtime/raw_value_ir.cpp deleted file mode 100644 index d6fef33f437b02..00000000000000 --- a/be/src/runtime/raw_value_ir.cpp +++ /dev/null @@ -1,114 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "runtime/raw_value.h" -#include "runtime/string_value.hpp" -#include "util/types.h" - -namespace doris { - -int RawValue::compare(const void* v1, const void* v2, const TypeDescriptor& type) { - const StringValue* string_value1; - const StringValue* string_value2; - const DateTimeValue* ts_value1; - const DateTimeValue* ts_value2; - float f1 = 0; - float f2 = 0; - double d1 = 0; - double d2 = 0; - int32_t i1; - int32_t i2; - int64_t b1; - int64_t b2; - - if (nullptr == v1 && nullptr == v2) { - return 0; - } else if (nullptr == v1 && nullptr != v2) { - return -1; - } else if (nullptr != v1 && nullptr == v2) { - return 1; - } - - switch (type.type) { - case TYPE_NULL: - return 0; - - case TYPE_BOOLEAN: - return *reinterpret_cast(v1) - *reinterpret_cast(v2); - - case TYPE_TINYINT: - return *reinterpret_cast(v1) - *reinterpret_cast(v2); - - case TYPE_SMALLINT: - return *reinterpret_cast(v1) - *reinterpret_cast(v2); - - case TYPE_INT: - i1 = *reinterpret_cast(v1); - i2 = *reinterpret_cast(v2); - return i1 > i2 ? 1 : (i1 < i2 ? -1 : 0); - - case TYPE_BIGINT: - b1 = *reinterpret_cast(v1); - b2 = *reinterpret_cast(v2); - return b1 > b2 ? 1 : (b1 < b2 ? -1 : 0); - - case TYPE_FLOAT: - // TODO: can this be faster? (just returning the difference has underflow problems) - f1 = *reinterpret_cast(v1); - f2 = *reinterpret_cast(v2); - return f1 > f2 ? 1 : (f1 < f2 ? -1 : 0); - - case TYPE_DOUBLE: - // TODO: can this be faster? - d1 = *reinterpret_cast(v1); - d2 = *reinterpret_cast(v2); - return d1 > d2 ? 1 : (d1 < d2 ? -1 : 0); - - case TYPE_CHAR: - case TYPE_VARCHAR: - case TYPE_HLL: - case TYPE_STRING: - string_value1 = reinterpret_cast(v1); - string_value2 = reinterpret_cast(v2); - return string_value1->compare(*string_value2); - - case TYPE_DATE: - case TYPE_DATETIME: - ts_value1 = reinterpret_cast(v1); - ts_value2 = reinterpret_cast(v2); - return *ts_value1 > *ts_value2 ? 1 : (*ts_value1 < *ts_value2 ? -1 : 0); - - case TYPE_DECIMALV2: { - DecimalV2Value decimal_value1(reinterpret_cast(v1)->value); - DecimalV2Value decimal_value2(reinterpret_cast(v2)->value); - return (decimal_value1 > decimal_value2) ? 1 : (decimal_value1 < decimal_value2 ? -1 : 0); - } - - case TYPE_LARGEINT: { - __int128 large_int_value1 = reinterpret_cast(v1)->value; - __int128 large_int_value2 = reinterpret_cast(v2)->value; - return large_int_value1 > large_int_value2 ? 1 - : (large_int_value1 < large_int_value2 ? -1 : 0); - } - - default: - DCHECK(false) << "invalid type: " << type.type; - return 0; - }; -} - -} // namespace doris diff --git a/be/src/runtime/row_batch.h b/be/src/runtime/row_batch.h index 1d9c00d886f48c..070a1e578fb4e5 100644 --- a/be/src/runtime/row_batch.h +++ b/be/src/runtime/row_batch.h @@ -21,7 +21,6 @@ #include #include -#include "codegen/doris_ir.h" #include "common/logging.h" #include "runtime/buffered_block_mgr2.h" // for BufferedBlockMgr2::Block #include "runtime/bufferpool/buffer_pool.h" @@ -42,7 +41,6 @@ class TupleRow; class TupleDescriptor; class PRowBatch; - // A RowBatch encapsulates a batch of rows, each composed of a number of tuples. // The maximum number of rows is fixed at the time of construction, and the caller // can add rows up to that capacity. @@ -203,10 +201,10 @@ class RowBatch : public RowBatchInterface { } /// Return the current row pointed to by the row pointer. - TupleRow* IR_ALWAYS_INLINE get() { return reinterpret_cast(_row); } + TupleRow* get() { return reinterpret_cast(_row); } /// Increment the row pointer and return the next row. - TupleRow* IR_ALWAYS_INLINE next() { + TupleRow* next() { _row += _num_tuples_per_row; DCHECK_LE((_row - _parent->_tuple_ptrs) / _num_tuples_per_row, _parent->_capacity); return get(); @@ -215,7 +213,7 @@ class RowBatch : public RowBatchInterface { /// Returns true if the iterator is beyond the last row for read iterators. /// Useful for read iterators to determine the limit. Write iterators should use /// RowBatch::AtCapacity() instead. - bool IR_ALWAYS_INLINE at_end() const { return _row >= _row_batch_end; } + bool at_end() const { return _row >= _row_batch_end; } /// Returns the row batch which this iterator is iterating through. RowBatch* parent() const { return _parent; } @@ -403,10 +401,10 @@ class RowBatch : public RowBatchInterface { // All members need to be handled in RowBatch::swap() - bool _has_in_flight_row; // if true, last row hasn't been committed yet - int _num_rows; // # of committed rows - int _num_uncommitted_rows; // # of uncommited rows in row batch mem pool - int _capacity; // maximum # of rows + bool _has_in_flight_row; // if true, last row hasn't been committed yet + int _num_rows; // # of committed rows + int _num_uncommitted_rows; // # of uncommited rows in row batch mem pool + int _capacity; // maximum # of rows /// If FLUSH_RESOURCES, the resources attached to this batch should be freed or /// acquired by a new owner as soon as possible. See MarkFlushResources(). If diff --git a/be/src/runtime/string_value_ir.cpp b/be/src/runtime/string_value_ir.cpp deleted file mode 100644 index a5df929092dfc8..00000000000000 --- a/be/src/runtime/string_value_ir.cpp +++ /dev/null @@ -1,29 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#ifdef IR_COMPILE -#include "codegen/doris_ir.h" -#include "runtime/string_value.hpp" - -namespace doris { -int ir_string_compare(const char* s1, int64_t n1, const char* s2, int64_t n2) { - return string_compare(s1, n1, s2, n2, std::min(n1, n2)); -} -} // namespace doris -#else -#error "This file should only be used for cross compiling to IR." -#endif diff --git a/be/src/runtime/tmp_file_mgr.cc b/be/src/runtime/tmp_file_mgr.cc index b1664040fb5a29..f8cc4ad71a307a 100644 --- a/be/src/runtime/tmp_file_mgr.cc +++ b/be/src/runtime/tmp_file_mgr.cc @@ -245,15 +245,6 @@ void TmpFileMgr::File::report_io_error(const std::string& error_msg) { LOG(ERROR) << "Error for temporary file '" << _path << "': " << error_msg; } -#if 0 -void TmpFileMgr::File::report_io_error(const ErrorMsg& msg) { - LOG(ERROR) << "Error for temporary file '" << _path << "': " << msg.msg(); - // IMPALA-2305: avoid blacklisting to prevent test failures. - // blacklisted_ = true; - // mgr_->BlacklistDevice(device_id_); -} -#endif - Status TmpFileMgr::File::remove() { if (_current_size > 0) { FileSystemUtil::remove_paths(vector(1, _path)); diff --git a/be/src/runtime/tmp_file_mgr.h b/be/src/runtime/tmp_file_mgr.h index 58ba7ae1d07757..9ca97a6e0c8d58 100644 --- a/be/src/runtime/tmp_file_mgr.h +++ b/be/src/runtime/tmp_file_mgr.h @@ -59,9 +59,6 @@ class TmpFileMgr { Status allocate_space(int64_t write_size, int64_t* offset); // Called to notify TmpFileMgr that an IO error was encountered for this file -#if 0 - void report_io_error(const ErrorMsg& msg); -#endif void report_io_error(const std::string& error_msg); // Delete the physical file on disk, if one was created. diff --git a/be/src/runtime/types.cpp b/be/src/runtime/types.cpp index c39a161b5c0ea1..63ff5151210502 100644 --- a/be/src/runtime/types.cpp +++ b/be/src/runtime/types.cpp @@ -51,32 +51,30 @@ TypeDescriptor::TypeDescriptor(const std::vector& types, int* idx) children.push_back(TypeDescriptor(types, idx)); break; } -#if 0 // Don't support now - case TTypeNodeType::STRUCT: - type = TYPE_STRUCT; - for (int i = 0; i < node.struct_fields.size(); ++i) { - ++(*idx); - children.push_back(TypeDescriptor(types, idx)); - field_names.push_back(node.struct_fields[i].name); - } - break; - case TTypeNodeType::ARRAY: - DCHECK(!node.__isset.scalar_type); - DCHECK_LT(*idx, types.size() - 1); - type = TYPE_ARRAY; - ++(*idx); - children.push_back(TypeDescriptor(types, idx)); - break; - case TTypeNodeType::MAP: - DCHECK(!node.__isset.scalar_type); - DCHECK_LT(*idx, types.size() - 2); - type = TYPE_MAP; - ++(*idx); - children.push_back(TypeDescriptor(types, idx)); - ++(*idx); - children.push_back(TypeDescriptor(types, idx)); - break; -#endif + // case TTypeNodeType::STRUCT: + // type = TYPE_STRUCT; + // for (int i = 0; i < node.struct_fields.size(); ++i) { + // ++(*idx); + // children.push_back(TypeDescriptor(types, idx)); + // field_names.push_back(node.struct_fields[i].name); + // } + // break; + // case TTypeNodeType::ARRAY: + // DCHECK(!node.__isset.scalar_type); + // DCHECK_LT(*idx, types.size() - 1); + // type = TYPE_ARRAY; + // ++(*idx); + // children.push_back(TypeDescriptor(types, idx)); + // break; + // case TTypeNodeType::MAP: + // DCHECK(!node.__isset.scalar_type); + // DCHECK_LT(*idx, types.size() - 2); + // type = TYPE_MAP; + // ++(*idx); + // children.push_back(TypeDescriptor(types, idx)); + // ++(*idx); + // children.push_back(TypeDescriptor(types, idx)); + // break; default: DCHECK(false) << node.type; } diff --git a/be/src/runtime/types.h b/be/src/runtime/types.h index c2adca90fb3156..1099548f0291dc 100644 --- a/be/src/runtime/types.h +++ b/be/src/runtime/types.h @@ -72,13 +72,6 @@ struct TypeDescriptor { // explicit TypeDescriptor(PrimitiveType type) : TypeDescriptor(PrimitiveType type) : type(type), len(-1), precision(-1), scale(-1) { -#if 0 - DCHECK_NE(type, TYPE_CHAR); - DCHECK_NE(type, TYPE_VARCHAR); - DCHECK_NE(type, TYPE_STRUCT); - DCHECK_NE(type, TYPE_ARRAY); - DCHECK_NE(type, TYPE_MAP); -#endif if (type == TYPE_DECIMALV2) { precision = 27; scale = 9; @@ -170,7 +163,8 @@ struct TypeDescriptor { void to_protobuf(PTypeDesc* ptype) const; inline bool is_string_type() const { - return type == TYPE_VARCHAR || type == TYPE_CHAR || type == TYPE_HLL || type == TYPE_OBJECT || type == TYPE_STRING; + return type == TYPE_VARCHAR || type == TYPE_CHAR || type == TYPE_HLL || + type == TYPE_OBJECT || type == TYPE_STRING; } inline bool is_date_type() const { return type == TYPE_DATE || type == TYPE_DATETIME; } @@ -180,7 +174,8 @@ struct TypeDescriptor { inline bool is_datetime_type() const { return type == TYPE_DATETIME; } inline bool is_var_len_string_type() const { - return type == TYPE_VARCHAR || type == TYPE_HLL || type == TYPE_CHAR || type == TYPE_OBJECT || type == TYPE_STRING; + return type == TYPE_VARCHAR || type == TYPE_HLL || type == TYPE_CHAR || + type == TYPE_OBJECT || type == TYPE_STRING; } inline bool is_complex_type() const { diff --git a/be/src/udf/CMakeLists.txt b/be/src/udf/CMakeLists.txt index 21fcd5dbd21aa6..7f9306fdc373e9 100755 --- a/be/src/udf/CMakeLists.txt +++ b/be/src/udf/CMakeLists.txt @@ -26,8 +26,8 @@ set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/udf") # Build this library twice. Once to be linked into the main Doris. This version # can have dependencies on our other libs. The second version is shipped as part # of the UDF sdk, which can't use other libs. -add_library(Udf udf.cpp udf_ir.cpp) -add_library(DorisUdf udf.cpp udf_ir.cpp) +add_library(Udf udf.cpp) +add_library(DorisUdf udf.cpp) set_target_properties(DorisUdf PROPERTIES COMPILE_FLAGS "-DDORIS_UDF_SDK_BUILD") # We can't use the normal link list since we want to pick up libDorisUdf (the external diff --git a/be/src/udf/udf.cpp b/be/src/udf/udf.cpp index b9ec504585c670..eae0bf14b09c2b 100644 --- a/be/src/udf/udf.cpp +++ b/be/src/udf/udf.cpp @@ -132,7 +132,8 @@ void FunctionContextImpl::set_constant_args(const std::vector& constant_cols) { +void FunctionContextImpl::set_constant_cols( + const std::vector& constant_cols) { _constant_cols = constant_cols; } @@ -507,4 +508,58 @@ void HllVal::agg_merge(const HllVal& other) { } } +bool FunctionContext::is_arg_constant(int i) const { + if (i < 0 || i >= _impl->_constant_args.size()) { + return false; + } + return _impl->_constant_args[i] != nullptr; +} + +bool FunctionContext::is_col_constant(int i) const { + if (i < 0 || i >= _impl->_constant_cols.size()) { + return false; + } + return _impl->_constant_cols[i] != nullptr; +} + +AnyVal* FunctionContext::get_constant_arg(int i) const { + if (i < 0 || i >= _impl->_constant_args.size()) { + return nullptr; + } + return _impl->_constant_args[i]; +} + +doris::ColumnPtrWrapper* FunctionContext::get_constant_col(int i) const { + if (i < 0 || i >= _impl->_constant_cols.size()) { + return nullptr; + } + return _impl->_constant_cols[i]; +} + +int FunctionContext::get_num_args() const { + return _impl->_arg_types.size(); +} + +int FunctionContext::get_num_constant_args() const { + return _impl->_constant_args.size(); +} + +const FunctionContext::TypeDesc& FunctionContext::get_return_type() const { + return _impl->_return_type; +} + +void* FunctionContext::get_function_state(FunctionStateScope scope) const { + // assert(!_impl->_closed); + switch (scope) { + case THREAD_LOCAL: + return _impl->_thread_local_fn_state; + break; + case FRAGMENT_LOCAL: + return _impl->_fragment_local_fn_state; + break; + default: + // TODO: signal error somehow + return nullptr; + } +} } // namespace doris_udf diff --git a/be/src/udf/udf_ir.cpp b/be/src/udf/udf_ir.cpp deleted file mode 100644 index 416a52025a93a0..00000000000000 --- a/be/src/udf/udf_ir.cpp +++ /dev/null @@ -1,83 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include - -#include "udf/udf_internal.h" -#include "udf/udf.h" - -namespace doris { -class ColumnPtrWrapper; -} // namespace doris - -namespace doris_udf { -bool FunctionContext::is_arg_constant(int i) const { - if (i < 0 || i >= _impl->_constant_args.size()) { - return false; - } - return _impl->_constant_args[i] != nullptr; -} - -bool FunctionContext::is_col_constant(int i) const { - if (i < 0 || i >= _impl->_constant_cols.size()) { - return false; - } - return _impl->_constant_cols[i] != nullptr; -} - -AnyVal* FunctionContext::get_constant_arg(int i) const { - if (i < 0 || i >= _impl->_constant_args.size()) { - return nullptr; - } - return _impl->_constant_args[i]; -} - -doris::ColumnPtrWrapper* FunctionContext::get_constant_col(int i) const { - if (i < 0 || i >= _impl->_constant_cols.size()) { - return nullptr; - } - return _impl->_constant_cols[i]; -} - -int FunctionContext::get_num_args() const { - return _impl->_arg_types.size(); -} - -int FunctionContext::get_num_constant_args() const { - return _impl->_constant_args.size(); -} - -const FunctionContext::TypeDesc& FunctionContext::get_return_type() const { - return _impl->_return_type; -} - -void* FunctionContext::get_function_state(FunctionStateScope scope) const { - // assert(!_impl->_closed); - switch (scope) { - case THREAD_LOCAL: - return _impl->_thread_local_fn_state; - break; - case FRAGMENT_LOCAL: - return _impl->_fragment_local_fn_state; - break; - default: - // TODO: signal error somehow - return nullptr; - } -} - -} // namespace doris_udf diff --git a/be/src/util/bfd_parser.cpp b/be/src/util/bfd_parser.cpp index 696eda034a1f34..0fb5a8ecd93f5d 100644 --- a/be/src/util/bfd_parser.cpp +++ b/be/src/util/bfd_parser.cpp @@ -263,17 +263,6 @@ int BfdParser::decode_address(const char* str, const char** end, std::string* fi } *lineno = ctx.lineno; return 0; -#if 0 - bool inline_found = true; - while (inline_found) { - printf("%s\t%s:%u\n", ctx.func_name, ctx.file_name, ctx.lineno); - inline_found = bfd_find_inliner_info(_abfd, &ctx.file_name, &ctx.func_name, &ctx.lineno); - printf("inline found = %d\n", inline_found); - if (inline_found) { - printf("inline file_name=%s func_name=%s\n", ctx.file_name, ctx.func_name); - } - } -#endif } } // namespace doris diff --git a/be/src/util/disk_info.h b/be/src/util/disk_info.h index 5278a06c7d9acf..8826a732998ef4 100644 --- a/be/src/util/disk_info.h +++ b/be/src/util/disk_info.h @@ -43,22 +43,6 @@ class DiskInfo { return _s_disks.size(); } -#if 0 - // Returns the number of (logical) disks the data node is using. - // It is possible for this to be more than num_disks since the datanode - // can be configured to have multiple data directories on the same physical - // disk. - static int num_datanode_dirs() { - DCHECK(_initialized); - return _num_datanode_dirs; - } - - // Returns a 0-based disk index for the data node dirs index. - static int disk_id(int datanode_dir_idx) { - return 0; - } -#endif - // Returns the 0-based disk index for 'path' (path must be a FS path, not // hdfs path). static int disk_id(const char* path); diff --git a/be/src/util/error_util.cc b/be/src/util/error_util.cc index 031bf8c109d76f..d3f8c2df5916e5 100644 --- a/be/src/util/error_util.cc +++ b/be/src/util/error_util.cc @@ -18,6 +18,7 @@ #include "util/error_util.h" #include + #include #include #include @@ -41,163 +42,4 @@ string get_str_err_msg() { return ss.str(); } -#if 0 - -string get_tables_missing_stats_warning(const vector& tables_missing_stats) { - stringstream ss; - if (tables_missing_stats.empty()) return string(""); - ss << "WARNING: The following tables are missing relevant table and/or column " - << "statistics.\n"; - for (int i = 0; i < tables_missing_stats.size(); ++i) { - const TTableName& table_name = tables_missing_stats[i]; - if (i != 0) ss << ","; - ss << table_name.db_name << "." << table_name.table_name; - } - return ss.str(); -} - -ErrorMsg::ErrorMsg(TErrorCode::type error) : error_(error), - message_(strings::Substitute(g_ErrorCodes_constants.TErrorMessage[error_])) {} - -ErrorMsg::ErrorMsg(TErrorCode::type error, const ArgType& arg0) : error_(error), - message_(strings::Substitute(g_ErrorCodes_constants.TErrorMessage[error_], arg0)) {} - -ErrorMsg::ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1) : - error_(error), - message_(strings::Substitute(g_ErrorCodes_constants.TErrorMessage[error_], - arg0, arg1)) {} - -ErrorMsg::ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2) : - error_(error), - message_(strings::Substitute(g_ErrorCodes_constants.TErrorMessage[error_], - arg0, arg1, arg2)) {} - -ErrorMsg::ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3) : - error_(error), - message_(strings::Substitute(g_ErrorCodes_constants.TErrorMessage[error_], - arg0, arg1, arg2, arg3)) {} - -ErrorMsg::ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3, const ArgType& arg4) : - error_(error), - message_(strings::Substitute(g_ErrorCodes_constants.TErrorMessage[error_], - arg0, arg1, arg2, arg3, arg4)) {} - -ErrorMsg::ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3, const ArgType& arg4, - const ArgType& arg5) : - error_(error), - message_(strings::Substitute(g_ErrorCodes_constants.TErrorMessage[error_], - arg0, arg1, arg2, arg3, arg4, arg5)) {} - -ErrorMsg::ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3, const ArgType& arg4, - const ArgType& arg5, const ArgType& arg6) : - error_(error), - message_(strings::Substitute(g_ErrorCodes_constants.TErrorMessage[error_], - arg0, arg1, arg2, arg3, arg4, arg5, arg6)) {} - -ErrorMsg::ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3, const ArgType& arg4, - const ArgType& arg5, const ArgType& arg6, const ArgType& arg7) : - error_(error), - message_(strings::Substitute(g_ErrorCodes_constants.TErrorMessage[error_], - arg0, arg1, arg2, arg3, arg4, arg5, arg6, arg7)) {} - -ErrorMsg::ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3, const ArgType& arg4, - const ArgType& arg5, const ArgType& arg6, const ArgType& arg7, - const ArgType& arg8) : - error_(error), - message_(strings::Substitute(g_ErrorCodes_constants.TErrorMessage[error_], - arg0, arg1, arg2, arg3, arg4, arg5, arg6, arg7, arg8)) {} - -ErrorMsg::ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3, const ArgType& arg4, - const ArgType& arg5, const ArgType& arg6, const ArgType& arg7, - const ArgType& arg8, const ArgType& arg9) : - error_(error), - message_(strings::Substitute(g_ErrorCodes_constants.TErrorMessage[error_], - arg0, arg1, arg2, arg3, arg4, arg5, arg6, arg7, arg8, arg9)) {} - -ErrorMsg ErrorMsg::init(TErrorCode::type error, const ArgType& arg0, - const ArgType& arg1, const ArgType& arg2, const ArgType& arg3, - const ArgType& arg4, const ArgType& arg5, const ArgType& arg6, const ArgType& arg7, - const ArgType& arg8, const ArgType& arg9) { - - ErrorCodesConstants error_strings; - ErrorMsg m; - m.error_ = error; - m.message_ = strings::Substitute(error_strings.TErrorMessage[m.error_], - arg0, arg1, arg2, arg3, arg4, arg5, - arg6, arg7, arg8, arg9); - return m; -} - -void print_error_map(ostream* stream, const ErrorLogMap& errors) { - for (const ErrorLogMap::value_type& v : errors) { - if (v.first == TErrorCode::GENERAL) { - for (const string& s : v.second.messages) { - *stream << s << "\n"; - } - } else { - *stream << v.second.messages.front(); - if (v.second.count < 2) { - *stream << "\n"; - } else { - *stream << " (1 of " << v.second.count << " similar)\n"; - } - } - } -} - -string print_error_map_to_string(const ErrorLogMap& errors) { - stringstream stream; - PrintErrorMap(&stream, errors); - return stream.str(); -} - -void merge_error_maps(ErrorLogMap* left, const ErrorLogMap& right) { - for (const ErrorLogMap::value_type& v : right) { - // Append generic message, append specific codes or increment count if exists - if (v.first == TErrorCode::GENERAL) { - (*left)[v.first].messages.insert( - (*left)[v.first].messages.end(), v.second.messages.begin(), - v.second.messages.end()); - } else { - if ((*left).count(v.first) > 0) { - (*left)[v.first].count += v.second.count; - } else { - (*left)[v.first].messages.push_back(v.second.messages.front()); - (*left)[v.first].count = v.second.count; - } - } - } -} - -void append_error(ErrorLogMap* map, const ErrorMsg& e) { - if (e.error() == TErrorCode::GENERAL) { - (*map)[e.error()].messages.push_back(e.msg()); - } else { - ErrorLogMap::iterator it = map->find(e.error()); - if (it != map->end()) { - ++(it->second.count); - } else { - (*map)[e.error()].messages.push_back(e.msg()); - (*map)[e.error()].count = 1; - } - } -} - -size_t error_count(const ErrorLogMap& errors) { - ErrorLogMap::const_iterator cit = errors.find(TErrorCode::GENERAL); - size_t general_errors = cit != errors.end() ? - errors.find(TErrorCode::GENERAL)->second.messages.size() - 1 : 0; - return errors.size() + general_errors; -} - -#endif // end '#if 0': comment these code - } // namespace doris diff --git a/be/src/util/error_util.h b/be/src/util/error_util.h index ff1e0786d0d27e..a5c84a528aebe2 100644 --- a/be/src/util/error_util.h +++ b/be/src/util/error_util.h @@ -20,151 +20,12 @@ #include -// #include "gen-cpp/CatalogObjects_types.h" -// #include "gen-cpp/ErrorCodes_types.h" -// #include "gen-cpp/ErrorCodes_constants.h" -// #include "gen-cpp/ImpalaInternalService_types.h" -// #include "gutil/strings/substitute.h" - namespace doris { // Returns the error message for errno. We should not use strerror directly // as that is not thread safe. // Returns empty string if errno is 0. std::string get_str_err_msg(); - -#if 0 -/// Returns an error message warning that the given table names are missing relevant -/// table/and or column statistics. -std::string get_tables_missing_stats_warning(const std::vector& tables_missing_stats); - -/// Class that holds a formatted error message and potentially a set of detail -/// messages. Error messages are intended to be user facing. Error details can be attached -/// as strings to the message. These details should only be accessed internally. -class ErrorMsg { -public: - typedef strings::internal::SubstituteArg ArgType; - - /// Trivial constructor. - ErrorMsg() : _error(TErrorCode::OK) {} - - /// Below are a set of overloaded constructors taking all possible number of arguments - /// that can be passed to Substitute. The reason is to try to avoid forcing the compiler - /// putting all arguments for Substitute() on the stack whenver this is called and thus - /// polute the instruction cache. - explicit ErrorMsg(TErrorCode::type error); - ErrorMsg(TErrorCode::type error, const ArgType& arg0); - ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1); - ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2); - ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3); - ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3, const ArgType& arg4); - ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3, const ArgType& arg4, - const ArgType& arg5); - ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3, const ArgType& arg4, - const ArgType& arg5, const ArgType& arg6); - ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3, const ArgType& arg4, - const ArgType& arg5, const ArgType& arg6, const ArgType& arg7); - ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3, const ArgType& arg4, - const ArgType& arg5, const ArgType& arg6, const ArgType& arg7, - const ArgType& arg8); - ErrorMsg(TErrorCode::type error, const ArgType& arg0, const ArgType& arg1, - const ArgType& arg2, const ArgType& arg3, const ArgType& arg4, - const ArgType& arg5, const ArgType& arg6, const ArgType& arg7, - const ArgType& arg8, const ArgType& arg9); - - ErrorMsg(TErrorCode::type error, const std::vector& detail) - : _error(error), _details(detail) {} - - /// Static initializer that is needed to avoid issues with static initialization order - /// and the point in time when the string list generated via thrift becomes - /// available. This method should not be used if no static initialization is needed as - /// the cost of this method is proportional to the number of entries in the global error - /// message list. - /// WARNING: DO NOT CALL THIS METHOD IN A NON STATIC CONTEXT - static ErrorMsg init(TErrorCode::type error, const ArgType& arg0 = ArgType::NoArg, - const ArgType& arg1 = ArgType::NoArg, - const ArgType& arg2 = ArgType::NoArg, - const ArgType& arg3 = ArgType::NoArg, - const ArgType& arg4 = ArgType::NoArg, - const ArgType& arg5 = ArgType::NoArg, - const ArgType& arg6 = ArgType::NoArg, - const ArgType& arg7 = ArgType::NoArg, - const ArgType& arg8 = ArgType::NoArg, - const ArgType& arg9 = ArgType::NoArg); - - TErrorCode::type error() const { return _error; } - - /// Add detail string message. - void add_detail(const std::string& d) { - _details.push_back(d); - } - - /// Set a specific error code. - void set_error(TErrorCode::type e) { - _error = e; - } - - /// Return the formatted error string. - const std::string& msg() const { - return _message; - } - - const std::vector& details() const { - return _details; - } - - /// Produce a string representation of the error message that includes the formatted - /// message of the original error and the attached detail strings. - std::string get_full_message_details() const { - std::stringstream ss; - ss << _message << "\n"; - for (size_t i = 0, end = _details.size(); i < end; ++i) { - ss << _details[i] << "\n"; - } - return ss.str(); - } - -private: - TErrorCode::type _error; - std::string _message; - std::vector _details; -}; - -/// Track log messages per error code. -typedef std::map ErrorLogMap; - -/// Merge error maps. Merging of error maps occurs, when the errors from multiple backends -/// are merged into a single error map. General log messages are simply appended, -/// specific errors are deduplicated by either appending a new instance or incrementing -/// the count of an existing one. -void merge_error_maps(ErrorLogMap* left, const ErrorLogMap& right); - -/// Append an error to the error map. Performs the aggregation as follows: GENERAL errors -/// are appended to the list of GENERAL errors, to keep one item each in the map, while -/// for all other error codes only the count is incremented and only the first message -/// is kept as a sample. -void append_error(ErrorLogMap* map, const ErrorMsg& e); - -/// Helper method to print the contents of an ErrorMap to a stream. -void print_error_map(std::ostream* stream, const ErrorLogMap& errors); - -/// Return the number of errors within this error maps. General errors are counted -/// individually, while specific errors are counted once per distinct occurrence. -size_t error_count(const ErrorLogMap& errors); - -/// Generate a string representation of the error map. Produces the same output as -/// PrintErrorMap, but returns a string instead of using a stream. -std::string print_error_map_to_string(const ErrorLogMap& errors); - -#endif // end '#if 0': comment these code - } // end namespace doris #endif // DORIS_BE_SRC_UTIL_ERROR_UTIL_H diff --git a/be/src/util/hash_util.hpp b/be/src/util/hash_util.hpp index 050032c9092541..d03f466aff3194 100644 --- a/be/src/util/hash_util.hpp +++ b/be/src/util/hash_util.hpp @@ -345,7 +345,7 @@ class HashUtil { #endif } // hash_combine is the same with boost hash_combine, - // except replace boost::hash with std::hash + // except replace boost::hash with std::hash template static inline void hash_combine(std::size_t& seed, const T& v) { std::hash hasher; @@ -376,7 +376,7 @@ struct hash { } }; -#if !defined(IR_COMPILE) && __GNUC__ < 6 && !defined(__clang__) +#if __GNUC__ < 6 && !defined(__clang__) // Cause this is builtin function template <> struct hash<__int128> { diff --git a/be/src/util/hash_util_ir.cpp b/be/src/util/hash_util_ir.cpp deleted file mode 100644 index c059cf6d531dda..00000000000000 --- a/be/src/util/hash_util_ir.cpp +++ /dev/null @@ -1,39 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "util/hash_util.hpp" - -namespace doris { -// Define the hashing functions for llvm. They are not used by anything that is -// cross compiled and without this, would get stripped by the clang optimizer. -#ifdef IR_COMPILE -extern "C" uint32_t ir_fnv_hash(const void* data, int32_t bytes, uint32_t hash) { - return HashUtil::fnv_hash(data, bytes, hash); -} - -extern "C" uint32_t ir_crc_hash(const void* data, int32_t bytes, uint32_t hash) { -#ifdef __SSE4_2__ - return HashUtil::crc_hash(data, bytes, hash); -#else - return HashUtil::fnv_hash(data, bytes, hash); -#endif -} -#else -#error "This file should only be compiled by clang." -#endif - -} // namespace doris diff --git a/be/src/util/sse2neon.h b/be/src/util/sse2neon.h index cf4b5f8d1fad61..9d53b59edb2fe9 100644 --- a/be/src/util/sse2neon.h +++ b/be/src/util/sse2neon.h @@ -714,18 +714,6 @@ FORCE_INLINE __m128 _mm_shuffle_ps_2032(__m128 a, __m128 b) { // Selects four specific single-precision, floating-point values from a and b, // based on the mask i. // https://msdn.microsoft.com/en-us/library/vstudio/5f0858x0(v=vs.100).aspx -#if 0 /* C version */ -FORCE_INLINE __m128 _mm_shuffle_ps_default(__m128 a, - __m128 b, - __constrange(0, 255) int imm) { - __m128 ret; - ret[0] = a[imm & 0x3]; - ret[1] = a[(imm >> 2) & 0x3]; - ret[2] = b[(imm >> 4) & 0x03]; - ret[3] = b[(imm >> 6) & 0x03]; - return ret; -} -#endif #define _mm_shuffle_ps_default(a, b, imm) \ __extension__({ \ float32x4_t ret; \ @@ -903,8 +891,8 @@ FORCE_INLINE __m128i _mm_shuffle_epi8(__m128i a, __m128i b) { // respectively. __asm__(" vtbl.8 %e[ret], {%e[tbl], %f[tbl]}, %e[idx]\n" " vtbl.8 %f[ret], {%e[tbl], %f[tbl]}, %f[idx]\n" - : [ret] "=&w"(ret) - : [tbl] "w"(tbl), [idx] "w"(idx_masked)); + : [ ret ] "=&w"(ret) + : [ tbl ] "w"(tbl), [ idx ] "w"(idx_masked)); return vreinterpretq_m128i_s8(ret); #else // use this line if testing on aarch64 @@ -914,17 +902,6 @@ FORCE_INLINE __m128i _mm_shuffle_epi8(__m128i a, __m128i b) { #endif } -#if 0 /* C version */ -FORCE_INLINE __m128i _mm_shuffle_epi32_default(__m128i a, - __constrange(0, 255) int imm) { - __m128i ret; - ret[0] = a[imm & 0x3]; - ret[1] = a[(imm >> 2) & 0x3]; - ret[2] = a[(imm >> 4) & 0x03]; - ret[3] = a[(imm >> 6) & 0x03]; - return ret; -} -#endif #define _mm_shuffle_epi32_default(a, imm) \ __extension__({ \ int32x4_t ret; \ diff --git a/be/src/vec/exec/vset_operation_node.h b/be/src/vec/exec/vset_operation_node.h index be97d3c7b6c483..93a7f8b61a3825 100644 --- a/be/src/vec/exec/vset_operation_node.h +++ b/be/src/vec/exec/vset_operation_node.h @@ -17,7 +17,6 @@ #pragma once -#include "codegen/doris_ir.h" #include "exec/exec_node.h" #include "runtime/row_batch.h" #include "runtime/runtime_state.h" @@ -46,8 +45,8 @@ class VSetOperationNode : public ExecNode { virtual void debug_string(int indentation_level, std::stringstream* out) const; protected: - //Todo: In build process of hashtable, It's same as join node. - //It's time to abstract out the same methods and provide them directly to others; + //Todo: In build process of hashtable, It's same as join node. + //It's time to abstract out the same methods and provide them directly to others; void hash_table_init(); Status hash_table_build(RuntimeState* state); Status process_build_block(Block& block); diff --git a/be/src/vec/io/io_helper.h b/be/src/vec/io/io_helper.h index 42e8bd0d4f9f54..87d3683f16af90 100644 --- a/be/src/vec/io/io_helper.h +++ b/be/src/vec/io/io_helper.h @@ -218,16 +218,6 @@ inline void read_binary(Type& x, BufferReadable& buf) { read_pod_binary(x, buf); } -#if 0 -inline void read_binary(const PColumn& pcolumn, std::string* data) { - if (pcolumn.compressed()) { - snappy::Uncompress(pcolumn.binary().data(), pcolumn.binary().size(), data); - } else { - *data = pcolumn.binary(); - } -} -#endif - template bool read_float_text_fast_impl(T& x, ReadBuffer& in) { static_assert(std::is_same_v || std::is_same_v, diff --git a/be/test/vec/aggregate_functions/agg_test.cpp b/be/test/vec/aggregate_functions/agg_test.cpp index 0b00f2f514efa0..f52fc655ede7e4 100644 --- a/be/test/vec/aggregate_functions/agg_test.cpp +++ b/be/test/vec/aggregate_functions/agg_test.cpp @@ -57,7 +57,7 @@ TEST(AggTest, basic_test) { } ASSERT_EQ(ans, *(int32_t*)place); agg_function->destroy(place); - if(place) { + if (place) { free(place); } } @@ -90,7 +90,9 @@ TEST(AggTest, topn_test) { } std::string result = reinterpret_cast(place)->get(); - std::string expect_result="{\"1\":2048,\"2\":683,\"3\":341,\"4\":205,\"5\":137,\"6\":97,\"7\":73,\"8\":57,\"9\":46,\"10\":37}"; + std::string expect_result = + "{\"1\":2048,\"2\":683,\"3\":341,\"4\":205,\"5\":137,\"6\":97,\"7\":73,\"8\":57,\"9\":" + "46,\"10\":37}"; ASSERT_EQ(result, expect_result); agg_function->destroy(place); }