diff --git a/.asf.yaml b/.asf.yaml index e928b3900268d2..be6a58bf506f99 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -35,4 +35,10 @@ github: dismiss_stale_reviews: true required_approving_review_count: 1 notifications: - pullrequests_status: dev@doris.apache.org + pullrequests_status: commits@doris.apache.org + collaborators: + - jackwener + - qidaye + - tinkerrrr + - luzhijing + - spaces-X diff --git a/.github/workflows/build-extension.yml b/.github/workflows/build-extension.yml index 261d6c4519b1ee..2270746989a880 100644 --- a/.github/workflows/build-extension.yml +++ b/.github/workflows/build-extension.yml @@ -55,27 +55,6 @@ jobs: run: | cd fs_brokers/apache_hdfs_broker/ && /bin/bash build.sh - - name: Build spark connector v2 - run: | - thrift --version - cd extension/spark-doris-connector/ && /bin/bash build.sh 2.3.4 2.11 - - - name: Build spark connector v3 - run: | - cd extension/spark-doris-connector/ && /bin/bash build.sh 3.1.2 2.12 - - - name: Build flink connector 1.11 - run: | - cd extension/flink-doris-connector/ && /bin/bash build.sh 1.11.6 2.12 - - - name: Build flink connector 1.12 - run: | - cd extension/flink-doris-connector/ && /bin/bash build.sh 1.12.7 2.12 - - - name: Build flink connector 1.13 - run: | - cd extension/flink-doris-connector/ && /bin/bash build.sh 1.13.5 2.12 - - name: Build docs run: | cd docs && npm install && npm run build diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index c594d0a8242cfc..669c2d2f91247b 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -25,7 +25,7 @@ Your suggestions, comments and comments on Doris can be made directly through Gi There are many ways to participate in and contribute to Doris projects: code implementation, test writing, process tool improvement, document improvement, and so on. Any contribution will be welcomed and you will be added to the list of contributors. Further, with sufficient contributions, you will have the opportunity to become a Commiter of Apache with Apache mailbox and be included in the list of [Apache Commiters] (http://people.apache.org/committer-index.html). -Any questions, you can contact us to get timely answers, including Wechat, Gitter (GitHub instant messaging tool), e-mail and so on. +Any questions, you can contact us to get timely answers, including dev mail list or Slack. ## Initial contact @@ -33,8 +33,7 @@ For the first time in Doris community, you can: * Follow [Doris Github](https://github.com/apache/incubator-doris) * Subscribe to our [mailing list] (./subscribe-mail-list.md); -* Join Doris Wechat Group (add micro-signal: morningman-cmy, note: join Doris Group) and ask questions at any time. -* Enter Doris's [Gitter] (./gitter.md) chat room; +* Join Doris [Slack](https://join.slack.com/t/apachedoriscommunity/shared_invite/zt-11jb8gesh-7IukzSrdea6mqoG0HB4gZg) Learn the development trends of Doris project in time and give your opinions on the topics you are concerned about. diff --git a/CONTRIBUTING_CN.md b/CONTRIBUTING_CN.md index 72f0839df65a82..7de688f06ecc45 100644 --- a/CONTRIBUTING_CN.md +++ b/CONTRIBUTING_CN.md @@ -25,7 +25,7 @@ under the License. 参与 Doris 项目并为其作出贡献的方法有很多:代码实现、测试编写、流程工具改进、文档完善等等。任何贡献我们都会非常欢迎,并将您加入贡献者列表,进一步,有了足够的贡献后,您还可以有机会成为 Apache 的 Commiter,拥有 Apache 邮箱,并被收录到 [Apache Commiter 列表中](http://people.apache.org/committer-index.html)。 -任何问题,您都可以联系我们得到及时解答,联系方式包括微信、Gitter(GitHub提供的即时聊天工具)、邮件等等。 +任何问题,您都可以联系我们得到及时解答,联系方式包括 dev 邮件组,Slack 等。 ## 初次接触 @@ -33,8 +33,7 @@ under the License. * 关注 Doris [Github 代码库](https://github.com/apache/incubator-doris) * 订阅我们的 [邮件列表](./subscribe-mail-list.md); -* 加入 Doris 微信群(加微信号:morningman-cmy, 备注:加入Doris群) 随时提问; -* 进入 Doris 的 [Gitter](./gitter.md) 聊天室; +* 加入 Doris 的 [Slack](https://join.slack.com/t/apachedoriscommunity/shared_invite/zt-11jb8gesh-7IukzSrdea6mqoG0HB4gZg) 通过以上方式及时了解 Doris 项目的开发动态并为您关注的话题发表意见。 diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt index 6bf98132f1d4a6..377ddd6e30c208 100644 --- a/be/CMakeLists.txt +++ b/be/CMakeLists.txt @@ -586,7 +586,6 @@ set(DORIS_LINK_LIBS Webserver Geo Vec - Plugin ${WL_END_GROUP} ) if (${MAKE_TEST} STREQUAL "ON") @@ -674,7 +673,6 @@ else() ${DORIS_DEPENDENCIES} ${WL_START_GROUP} ${X86_DEPENDENCIES} - ${WL_END_GROUP} ) endif() @@ -690,6 +688,8 @@ if (WITH_MYSQL) ) endif() +set(DORIS_DEPENDENCIES ${DORIS_DEPENDENCIES} ${WL_END_GROUP}) + message(STATUS "DORIS_DEPENDENCIES is ${DORIS_DEPENDENCIES}") # Add all external dependencies. They should come after the palo libs. @@ -774,7 +774,6 @@ if (BUILD_META_TOOL AND BUILD_META_TOOL STREQUAL "ON") endif() add_subdirectory(${SRC_DIR}/util) -add_subdirectory(${SRC_DIR}/plugin) add_subdirectory(${SRC_DIR}/vec) # Utility CMake function to make specifying tests and benchmarks less verbose @@ -796,24 +795,6 @@ FUNCTION(ADD_BE_TEST TEST_NAME) ADD_TEST(${TEST_FILE_NAME} "${BUILD_OUTPUT_ROOT_DIRECTORY}/${TEST_NAME}") ENDFUNCTION() -FUNCTION(ADD_BE_PLUGIN PLUGIN_NAME) - set(BUILD_OUTPUT_ROOT_DIRECTORY "${CMAKE_CURRENT_BINARY_DIR}/") - - get_filename_component(DIR_NAME ${CMAKE_CURRENT_SOURCE_DIR} NAME) - get_filename_component(PLUGIN_DIR_NAME ${PLUGIN_NAME} PATH) - get_filename_component(PLUGIN_FILE_NAME ${PLUGIN_NAME} NAME) - - ADD_LIBRARY(${PLUGIN_FILE_NAME} SHARED ${PLUGIN_NAME}.cpp) - - TARGET_LINK_LIBRARIES(${PLUGIN_FILE_NAME} ${DORIS_LINK_LIBS}) - SET_TARGET_PROPERTIES(${PLUGIN_FILE_NAME} PROPERTIES COMPILE_FLAGS "-fno-access-control") - - if (NOT "${PLUGIN_DIR_NAME}" STREQUAL "") - SET_TARGET_PROPERTIES(${PLUGIN_FILE_NAME} PROPERTIES RUNTIME_OUTPUT_DIRECTORY "${BUILD_OUTPUT_ROOT_DIRECTORY}/${PLUGIN_DIR_NAME}") - endif () - -ENDFUNCTION() - if (${MAKE_TEST} STREQUAL "ON") add_subdirectory(${TEST_DIR}/test_util) add_subdirectory(${TEST_DIR}/agent) @@ -834,8 +815,6 @@ if (${MAKE_TEST} STREQUAL "ON") add_subdirectory(${TEST_DIR}/vec/function) add_subdirectory(${TEST_DIR}/vec/runtime) add_subdirectory(${TEST_DIR}/vec/aggregate_functions) - add_subdirectory(${TEST_DIR}/plugin) - add_subdirectory(${TEST_DIR}/plugin/example) add_subdirectory(${TEST_DIR}/tools) endif () diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index 777c20dc1d4085..a0eaf570714ebd 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -383,7 +383,8 @@ void TaskWorkerPool::_create_tablet_worker_thread_callback() { tablet_info.tablet_id = tablet->table_id(); tablet_info.schema_hash = tablet->schema_hash(); tablet_info.version = create_tablet_req.version; - tablet_info.version_hash = create_tablet_req.version_hash; + // Useless but it is a required field in TTabletInfo + tablet_info.version_hash = 0; tablet_info.row_count = 0; tablet_info.data_size = 0; tablet_info.__set_path_hash(tablet->data_dir()->path_hash()); @@ -662,7 +663,6 @@ void TaskWorkerPool::_push_worker_thread_callback() { finish_task_request.__set_signature(agent_task_req.signature); if (push_req.push_type == TPushType::DELETE) { finish_task_request.__set_request_version(push_req.version); - finish_task_request.__set_request_version_hash(push_req.version_hash); } if (status == DORIS_SUCCESS) { @@ -1074,7 +1074,7 @@ void TaskWorkerPool::_check_consistency_worker_thread_callback() { uint32_t checksum = 0; EngineChecksumTask engine_task( check_consistency_req.tablet_id, check_consistency_req.schema_hash, - check_consistency_req.version, check_consistency_req.version_hash, &checksum); + check_consistency_req.version, &checksum); OLAPStatus res = _env->storage_engine()->execute_task(&engine_task); if (res != OLAP_SUCCESS) { LOG(WARNING) << "check consistency failed. status: " << res @@ -1095,7 +1095,6 @@ void TaskWorkerPool::_check_consistency_worker_thread_callback() { finish_task_request.__set_task_status(task_status); finish_task_request.__set_tablet_checksum(static_cast(checksum)); finish_task_request.__set_request_version(check_consistency_req.version); - finish_task_request.__set_request_version_hash(check_consistency_req.version_hash); _finish_task(finish_task_request); _remove_task_info(agent_task_req.task_type, agent_task_req.signature); @@ -1404,7 +1403,6 @@ void TaskWorkerPool::_make_snapshot_thread_callback() { LOG(WARNING) << "make_snapshot failed. tablet_id:" << snapshot_request.tablet_id << ", schema_hash:" << snapshot_request.schema_hash << ", version:" << snapshot_request.version - << ", version_hash:" << snapshot_request.version_hash << ", status: " << make_snapshot_status; error_msgs.push_back("make_snapshot failed. status: " + boost::lexical_cast(make_snapshot_status)); @@ -1412,7 +1410,6 @@ void TaskWorkerPool::_make_snapshot_thread_callback() { LOG(INFO) << "make_snapshot success. tablet_id:" << snapshot_request.tablet_id << ", schema_hash:" << snapshot_request.schema_hash << ", version:" << snapshot_request.version - << ", version_hash:" << snapshot_request.version_hash << ", snapshot_path:" << snapshot_path; if (snapshot_request.__isset.list_files) { // list and save all snapshot files @@ -1427,7 +1424,6 @@ void TaskWorkerPool::_make_snapshot_thread_callback() { LOG(WARNING) << "make_snapshot failed. tablet_id:" << snapshot_request.tablet_id << ", schema_hash:" << snapshot_request.schema_hash << ", version:" << snapshot_request.version - << ", version_hash:" << snapshot_request.version_hash << ",list file failed: " << st.get_error_msg(); error_msgs.push_back("make_snapshot failed. list file failed: " + st.get_error_msg()); 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/config.h b/be/src/common/config.h index 85c1f6d5a018ea..cc3448928da584 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -204,7 +204,8 @@ CONF_mInt32(tablet_rowset_stale_sweep_time_sec, "1800"); CONF_Int32(max_garbage_sweep_interval, "3600"); CONF_Int32(min_garbage_sweep_interval, "180"); CONF_mInt32(snapshot_expire_time_sec, "172800"); -// 仅仅是建议值,当磁盘空间不足时,trash下的文件保存期可不遵守这个参数 +// It is only a recommended value. When the disk space is insufficient, +// the file storage period under trash dose not have to comply with this parameter. CONF_mInt32(trash_file_expire_time_sec, "259200"); // check row nums for BE/CE and schema change. true is open, false is closed. CONF_mBool(row_nums_check, "true"); @@ -543,8 +544,6 @@ CONF_mInt64(max_runnings_transactions_per_txn_map, "100"); // this is a an enhancement for better performance to manage tablet CONF_Int32(tablet_map_shard_size, "1"); -CONF_String(plugin_path, "${DORIS_HOME}/plugin"); - // txn_map_lock shard size, the value is 2^n, n=0,1,2,3,4 // this is a an enhancement for better performance to manage txn CONF_Int32(txn_map_shard_size, "128"); 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/logconfig.cpp b/be/src/common/logconfig.cpp index cb2d748e4886b2..2264f673f5d40f 100644 --- a/be/src/common/logconfig.cpp +++ b/be/src/common/logconfig.cpp @@ -54,6 +54,10 @@ bool init_glog(const char* basename, bool install_signal_handler) { return true; } + if (getenv("DORIS_LOG_TO_STDERR") != nullptr) { + FLAGS_alsologtostderr = true; + } + if (install_signal_handler) { google::InstallFailureSignalHandler(); } 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/data_sink.cpp b/be/src/exec/data_sink.cpp index 2a259482251a45..286ff7fe772c24 100644 --- a/be/src/exec/data_sink.cpp +++ b/be/src/exec/data_sink.cpp @@ -37,6 +37,7 @@ #include "vec/sink/result_sink.h" #include "vec/sink/vdata_stream_sender.h" +#include "vec/sink/vmysql_table_writer.h" #include "vec/sink/vtablet_sink.h" namespace doris { @@ -113,10 +114,14 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink if (!thrift_sink.__isset.mysql_table_sink) { return Status::InternalError("Missing data buffer sink."); } - - // TODO: figure out good buffer size based on size of output row - MysqlTableSink* mysql_tbl_sink = new MysqlTableSink(pool, row_desc, output_exprs); - sink->reset(mysql_tbl_sink); + if (is_vec) { + doris::vectorized::VMysqlTableSink* vmysql_tbl_sink = new doris::vectorized::VMysqlTableSink(pool, row_desc, output_exprs); + sink->reset(vmysql_tbl_sink); + } else { + // TODO: figure out good buffer size based on size of output row + MysqlTableSink* mysql_tbl_sink = new MysqlTableSink(pool, row_desc, output_exprs); + sink->reset(mysql_tbl_sink); + } break; #else return Status::InternalError( 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/olap_scan_node.cpp b/be/src/exec/olap_scan_node.cpp index 48f0ab0e258a5f..4b1f775bcdb72c 100644 --- a/be/src/exec/olap_scan_node.cpp +++ b/be/src/exec/olap_scan_node.cpp @@ -425,7 +425,6 @@ Status OlapScanNode::close(RuntimeState* state) { // 1: required list hosts // 2: required string schema_hash // 3: required string version -// 4: required string version_hash // 5: required Types.TTabletId tablet_id // 6: required string db_name // 7: optional list partition_column_ranges 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/tablet_sink.cpp b/be/src/exec/tablet_sink.cpp index 2fe349294dd044..590e1136d921dc 100644 --- a/be/src/exec/tablet_sink.cpp +++ b/be/src/exec/tablet_sink.cpp @@ -183,12 +183,16 @@ Status NodeChannel::open_wait() { // add batch closure _add_batch_closure = ReusableClosure::create(); - _add_batch_closure->addFailedHandler([this]() { + _add_batch_closure->addFailedHandler([this](bool is_last_rpc) { // If rpc failed, mark all tablets on this node channel as failed - _index_channel->mark_as_failed(this, _add_batch_closure->cntl.ErrorText(), -1); + _index_channel->mark_as_failed(this->node_id(), this->host(), _add_batch_closure->cntl.ErrorText(), -1); Status st = _index_channel->check_intolerable_failure(); if (!st.ok()) { _cancel_with_msg(fmt::format("{}, err: {}", channel_info(), st.get_error_msg())); + } else if (is_last_rpc) { + // if this is last rpc, will must set _add_batches_finished. otherwise, node channel's close_wait + // will be blocked. + _add_batches_finished = true; } }); @@ -198,7 +202,7 @@ Status NodeChannel::open_wait() { if (status.ok()) { // if has error tablet, handle them first for (auto& error : result.tablet_errors()) { - _index_channel->mark_as_failed(this, error.msg(), error.tablet_id()); + _index_channel->mark_as_failed(this->node_id(), this->host(), error.msg(), error.tablet_id()); } Status st = _index_channel->check_intolerable_failure(); @@ -493,7 +497,7 @@ void NodeChannel::try_send_batch() { // eos request must be the last request _add_batch_closure->end_mark(); _send_finished = true; - DCHECK(_pending_batches_num == 0); + CHECK(_pending_batches_num == 0) << _pending_batches_num; } if (_parent->_transfer_data_by_brpc_attachment && request.has_row_batch()) { @@ -541,13 +545,15 @@ Status IndexChannel::init(RuntimeState* state, const std::vector channels; + std::vector> channels; for (auto& node_id : location->node_ids) { - NodeChannel* channel = nullptr; + std::shared_ptr channel; auto it = _node_channels.find(node_id); if (it == _node_channels.end()) { - channel = - _parent->_pool->add(new NodeChannel(_parent, this, node_id, _schema_hash)); + // NodeChannel is not added to the _parent->_pool. + // Because the deconstruction of NodeChannel may take a long time to wait rpc finish. + // but the ObjectPool will hold a spin lock to delete objects. + channel = std::make_shared(_parent, this, node_id, _schema_hash); _node_channels.emplace(node_id, channel); } else { channel = it->second; @@ -571,7 +577,7 @@ void IndexChannel::add_row(Tuple* tuple, int64_t tablet_id) { // if this node channel is already failed, this add_row will be skipped auto st = channel->add_row(tuple, tablet_id); if (!st.ok()) { - mark_as_failed(channel, st.get_error_msg(), tablet_id); + mark_as_failed(channel->node_id(), channel->host(), st.get_error_msg(), tablet_id); // continue add row to other node, the error will be checked for every batch outside } } @@ -586,14 +592,14 @@ void IndexChannel::add_row(BlockRow& block_row, int64_t tablet_id) { // if this node channel is already failed, this add_row will be skipped auto st = channel->add_row(block_row, tablet_id); if (!st.ok()) { - mark_as_failed(channel, st.get_error_msg(), tablet_id); + mark_as_failed(channel->node_id(), channel->host(), st.get_error_msg(), tablet_id); } } } -void IndexChannel::mark_as_failed(const NodeChannel* ch, const std::string& err, +void IndexChannel::mark_as_failed(int64_t node_id, const std::string& host, const std::string& err, int64_t tablet_id) { - const auto& it = _tablets_by_channel.find(ch->node_id()); + const auto& it = _tablets_by_channel.find(node_id); if (it == _tablets_by_channel.end()) { return; } @@ -602,16 +608,16 @@ void IndexChannel::mark_as_failed(const NodeChannel* ch, const std::string& err, std::lock_guard l(_fail_lock); if (tablet_id == -1) { for (const auto the_tablet_id : it->second) { - _failed_channels[the_tablet_id].insert(ch->node_id()); - _failed_channels_msgs.emplace(the_tablet_id, err + ", host: " + ch->host()); + _failed_channels[the_tablet_id].insert(node_id); + _failed_channels_msgs.emplace(the_tablet_id, err + ", host: " + host); if (_failed_channels[the_tablet_id].size() >= ((_parent->_num_replicas + 1) / 2)) { _intolerable_failure_status = Status::InternalError(_failed_channels_msgs[the_tablet_id]); } } } else { - _failed_channels[tablet_id].insert(ch->node_id()); - _failed_channels_msgs.emplace(tablet_id, err + ", host: " + ch->host()); + _failed_channels[tablet_id].insert(node_id); + _failed_channels_msgs.emplace(tablet_id, err + ", host: " + host); if (_failed_channels[tablet_id].size() >= ((_parent->_num_replicas + 1) / 2)) { _intolerable_failure_status = Status::InternalError(_failed_channels_msgs[tablet_id]); @@ -655,7 +661,7 @@ OlapTableSink::~OlapTableSink() { // OlapTableSink::_mem_tracker and its parents. // But their destructions are after OlapTableSink's. for (auto index_channel : _channels) { - index_channel->for_each_node_channel([](NodeChannel* ch) { ch->clear_all_batches(); }); + index_channel->for_each_node_channel([](const std::shared_ptr& ch) { ch->clear_all_batches(); }); } } @@ -809,17 +815,17 @@ Status OlapTableSink::open(RuntimeState* state) { RETURN_IF_ERROR(Expr::open(_output_expr_ctxs, state)); for (auto index_channel : _channels) { - index_channel->for_each_node_channel([](NodeChannel* ch) { ch->open(); }); + index_channel->for_each_node_channel([](const std::shared_ptr& ch) { ch->open(); }); } for (auto index_channel : _channels) { - index_channel->for_each_node_channel([&index_channel](NodeChannel* ch) { + index_channel->for_each_node_channel([&index_channel](const std::shared_ptr& ch) { auto st = ch->open_wait(); if (!st.ok()) { // The open() phase is mainly to generate DeltaWriter instances on the nodes corresponding to each node channel. // This phase will not fail due to a single tablet. // Therefore, if the open() phase fails, all tablets corresponding to the node need to be marked as failed. - index_channel->mark_as_failed(ch, + index_channel->mark_as_failed(ch->node_id(), ch->host(), fmt::format("{}, open failed, err: {}", ch->channel_info(), st.get_error_msg()), -1); @@ -935,7 +941,7 @@ Status OlapTableSink::close(RuntimeState* state, Status close_status) { { SCOPED_TIMER(_close_timer); for (auto index_channel : _channels) { - index_channel->for_each_node_channel([](NodeChannel* ch) { ch->mark_close(); }); + index_channel->for_each_node_channel([](const std::shared_ptr& ch) { ch->mark_close(); }); num_node_channels += index_channel->num_node_channels(); } @@ -945,10 +951,10 @@ Status OlapTableSink::close(RuntimeState* state, Status close_status) { [&index_channel, &state, &node_add_batch_counter_map, &serialize_batch_ns, &mem_exceeded_block_ns, &queue_push_lock_ns, &actual_consume_ns, &total_add_batch_exec_time_ns, &add_batch_exec_time, - &total_add_batch_num](NodeChannel* ch) { + &total_add_batch_num](const std::shared_ptr& ch) { auto s = ch->close_wait(state); if (!s.ok()) { - index_channel->mark_as_failed(ch, s.get_error_msg(), -1); + index_channel->mark_as_failed(ch->node_id(), ch->host(), s.get_error_msg(), -1); LOG(WARNING) << ch->channel_info() << ", close channel failed, err: " << s.get_error_msg(); @@ -1007,7 +1013,7 @@ Status OlapTableSink::close(RuntimeState* state, Status close_status) { } else { for (auto channel : _channels) { channel->for_each_node_channel( - [&status](NodeChannel* ch) { ch->cancel(status.get_error_msg()); }); + [&status](const std::shared_ptr& ch) { ch->cancel(status.get_error_msg()); }); } } @@ -1208,7 +1214,7 @@ void OlapTableSink::_send_batch_process() { do { int running_channels_num = 0; for (auto index_channel : _channels) { - index_channel->for_each_node_channel([&running_channels_num, this](NodeChannel* ch) { + index_channel->for_each_node_channel([&running_channels_num, this](const std::shared_ptr& ch) { running_channels_num += ch->try_send_and_fetch_status(this->_send_batch_thread_pool_token); }); @@ -1216,7 +1222,7 @@ void OlapTableSink::_send_batch_process() { if (running_channels_num == 0) { LOG(INFO) << "all node channels are stopped(maybe finished/offending/cancelled), " - "consumer thread exit."; + "sender thread exit. " << print_id(_load_id); return; } } while (!_stop_background_threads_latch.wait_for( diff --git a/be/src/exec/tablet_sink.h b/be/src/exec/tablet_sink.h index 10655882a2e6e2..b31f3841c337a7 100644 --- a/be/src/exec/tablet_sink.h +++ b/be/src/exec/tablet_sink.h @@ -92,7 +92,7 @@ class ReusableClosure : public google::protobuf::Closure { static ReusableClosure* create() { return new ReusableClosure(); } - void addFailedHandler(std::function fn) { failed_handler = fn; } + void addFailedHandler(std::function fn) { failed_handler = fn; } void addSuccessHandler(std::function fn) { success_handler = fn; } void join() { @@ -126,7 +126,7 @@ class ReusableClosure : public google::protobuf::Closure { if (cntl.Failed()) { LOG(WARNING) << "failed to send brpc batch, error=" << berror(cntl.ErrorCode()) << ", error_text=" << cntl.ErrorText(); - failed_handler(); + failed_handler(_is_last_rpc); } else { success_handler(result, _is_last_rpc); } @@ -140,7 +140,7 @@ class ReusableClosure : public google::protobuf::Closure { brpc::CallId cid; std::atomic _packet_in_flight {false}; std::atomic _is_last_rpc {false}; - std::function failed_handler; + std::function failed_handler; std::function success_handler; }; @@ -288,13 +288,13 @@ class IndexChannel { void add_row(BlockRow& block_row, int64_t tablet_id); - void for_each_node_channel(const std::function& func) { + void for_each_node_channel(const std::function&)>& func) { for (auto& it : _node_channels) { func(it.second); } } - void mark_as_failed(const NodeChannel* ch, const std::string& err, int64_t tablet_id = -1); + void mark_as_failed(int64_t node_id, const std::string& host, const std::string& err, int64_t tablet_id = -1); Status check_intolerable_failure(); // set error tablet info in runtime state, so that it can be returned to FE. @@ -310,9 +310,9 @@ class IndexChannel { int32_t _schema_hash; // BeId -> channel - std::unordered_map _node_channels; + std::unordered_map> _node_channels; // from tablet_id to backend channel - std::unordered_map> _channels_by_tablet; + std::unordered_map>> _channels_by_tablet; // from backend channel to tablet_id std::unordered_map> _tablets_by_channel; diff --git a/be/src/exec/text_converter.hpp b/be/src/exec/text_converter.hpp index 44e807ced02875..3b885348234591 100644 --- a/be/src/exec/text_converter.hpp +++ b/be/src/exec/text_converter.hpp @@ -41,7 +41,7 @@ namespace doris { inline bool TextConverter::write_slot(const SlotDescriptor* slot_desc, Tuple* tuple, const char* data, int len, bool copy_string, bool need_escape, MemPool* pool) { - //小批量导入只有\N被认为是NULL,没有批量导入的replace_value函数 + //Small batch import only \N is considered to be NULL, there is no replace_value function for batch import if (true == slot_desc->is_nullable()) { if (len == 2 && data[0] == '\\' && data[1] == 'N') { tuple->set_null(slot_desc->null_indicator_offset()); 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/exprs/topn_function.cpp b/be/src/exprs/topn_function.cpp index e956e5e627f06b..544f98086783ff 100644 --- a/be/src/exprs/topn_function.cpp +++ b/be/src/exprs/topn_function.cpp @@ -91,50 +91,10 @@ StringVal TopNFunctions::topn_finalize(FunctionContext* ctx, const StringVal& sr return result; } -template void TopNFunctions::topn_update(FunctionContext*, const BooleanVal&, const IntVal&, - StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const TinyIntVal&, const IntVal&, - StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const SmallIntVal&, const IntVal&, - StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const IntVal&, const IntVal&, - StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const BigIntVal&, const IntVal&, - StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const FloatVal&, const IntVal&, - StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const DoubleVal&, const IntVal&, - StringVal*); template void TopNFunctions::topn_update(FunctionContext*, const StringVal&, const IntVal&, StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const DateTimeVal&, const IntVal&, - StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const LargeIntVal&, const IntVal&, - StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const DecimalV2Val&, const IntVal&, - StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const BooleanVal&, const IntVal&, - const IntVal&, StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const TinyIntVal&, const IntVal&, - const IntVal&, StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const SmallIntVal&, const IntVal&, - const IntVal&, StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const IntVal&, const IntVal&, - const IntVal&, StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const BigIntVal&, const IntVal&, - const IntVal&, StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const FloatVal&, const IntVal&, - const IntVal&, StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const DoubleVal&, const IntVal&, - const IntVal&, StringVal*); template void TopNFunctions::topn_update(FunctionContext*, const StringVal&, const IntVal&, const IntVal&, StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const DateTimeVal&, const IntVal&, - const IntVal&, StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const LargeIntVal&, const IntVal&, - const IntVal&, StringVal*); -template void TopNFunctions::topn_update(FunctionContext*, const DecimalV2Val&, const IntVal&, - const IntVal&, StringVal*); } // namespace doris \ No newline at end of file diff --git a/be/src/exprs/v_string_functions.h b/be/src/exprs/v_string_functions.h deleted file mode 100644 index 3fd9845535eda1..00000000000000 --- a/be/src/exprs/v_string_functions.h +++ /dev/null @@ -1,219 +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. - -#pragma once - -#ifndef BE_V_STRING_FUNCTIONS_H -#define BE_V_STRING_FUNCTIONS_H - -#include -#include -#include "runtime/string_value.hpp" - -#ifdef __SSE2__ -#include -#endif - -namespace doris { -class VStringFunctions { -public: -#ifdef __SSE2__ - /// n equals to 16 chars length - static constexpr auto REGISTER_SIZE = sizeof(__m128i); -#endif -public: - static StringVal rtrim(const StringVal& str) { - if (str.is_null || str.len == 0) { - return str; - } - auto begin = 0; - auto end = str.len - 1; -#ifdef __SSE2__ - char blank = ' '; - const auto pattern = _mm_set1_epi8(blank); - while (end - begin + 1 >= REGISTER_SIZE) { - const auto v_haystack = _mm_loadu_si128(reinterpret_cast(str.ptr + end + 1 - REGISTER_SIZE)); - const auto v_against_pattern = _mm_cmpeq_epi8(v_haystack, pattern); - const auto mask = _mm_movemask_epi8(v_against_pattern); - int offset = __builtin_clz(~(mask << REGISTER_SIZE)); - /// means not found - if (offset == 0) - { - return StringVal(str.ptr + begin, end - begin + 1); - } else { - end -= offset; - } - } -#endif - while (end >= begin && str.ptr[end] == ' ') { - --end; - } - if (end < 0) { - return StringVal(""); - } - return StringVal(str.ptr + begin, end - begin + 1); - } - - static StringVal ltrim(const StringVal& str) { - if (str.is_null || str.len == 0) { - return str; - } - auto begin = 0; - auto end = str.len - 1; -#ifdef __SSE2__ - char blank = ' '; - const auto pattern = _mm_set1_epi8(blank); - while (end - begin + 1 >= REGISTER_SIZE) { - const auto v_haystack = _mm_loadu_si128(reinterpret_cast(str.ptr + begin)); - const auto v_against_pattern = _mm_cmpeq_epi8(v_haystack, pattern); - const auto mask = _mm_movemask_epi8(v_against_pattern); - const auto offset = __builtin_ctz(mask ^ 0xffff); - /// means not found - if (offset == 0) - { - return StringVal(str.ptr + begin, end - begin + 1); - } else if (offset > REGISTER_SIZE) { - begin += REGISTER_SIZE; - } else { - begin += offset; - return StringVal(str.ptr + begin, end - begin + 1); - } - } -#endif - while (begin <= end && str.ptr[begin] == ' ') { - ++begin; - } - return StringVal(str.ptr + begin, end - begin + 1); - } - - static StringVal trim(const StringVal& str) { - if (str.is_null || str.len == 0) { - return str; - } - return rtrim(ltrim(str)); - } - - static bool is_ascii(StringVal str) { - #ifdef __SSE2__ - size_t i = 0; - __m128i binary_code = _mm_setzero_si128(); - if (str.len >= REGISTER_SIZE) { - for (; i <= str.len - REGISTER_SIZE; i += REGISTER_SIZE) { - __m128i chars = _mm_loadu_si128((const __m128i*)(str.ptr + i)); - binary_code = _mm_or_si128(binary_code, chars); - } - } - int mask = _mm_movemask_epi8(binary_code); - - char or_code = 0; - for (; i < str.len; i++) { - or_code |= str.ptr[i]; - } - mask |= (or_code & 0x80); - - return !mask; - #else - char or_code = 0; - for (size_t i = 0; i < str.len; i++) { - or_code |= str.ptr[i]; - } - return !(or_code & 0x80); - #endif - } - - static void reverse(const StringVal& str, StringVal dst) { - if (str.is_null) { - dst.ptr = NULL; - return; - } - const bool is_ascii = VStringFunctions::is_ascii(str); - if (is_ascii) { - int64_t begin = 0; - int64_t end = str.len; - int64_t result_end = dst.len; - #if defined(__SSE2__) - const auto shuffle_array = _mm_set_epi64((__m64)0x00'01'02'03'04'05'06'07ull, (__m64)0x08'09'0a'0b'0c'0d'0e'0full); - for (; (begin + REGISTER_SIZE) < end; begin += REGISTER_SIZE) { - result_end -= REGISTER_SIZE; - _mm_storeu_si128((__m128i*)(dst.ptr + result_end), - _mm_shuffle_epi8(_mm_loadu_si128((__m128i*)(str.ptr + begin)), shuffle_array)); - } - #endif - for (; begin < end; ++begin) { - --result_end; - dst.ptr[result_end] = str.ptr[begin]; - } - } else { - for (size_t i = 0, char_size = 0; i < str.len; i += char_size) { - char_size = get_utf8_byte_length((unsigned)(str.ptr)[i]); - std::copy(str.ptr + i, str.ptr + i + char_size, dst.ptr + str.len - i - char_size); - } - } - } - - static size_t get_utf8_byte_length(unsigned char byte) { - size_t char_size = 0; - if (byte >= 0xFC) { - char_size = 6; - } else if (byte >= 0xF8) { - char_size = 5; - } else if (byte >= 0xF0) { - char_size = 4; - } else if (byte >= 0xE0) { - char_size = 3; - } else if (byte >= 0xC0) { - char_size = 2; - } else { - char_size = 1; - } - return char_size; - } - - static void hex_encode(const unsigned char* src_str, size_t length, char* dst_str) { - static constexpr auto hex_table = "0123456789ABCDEF"; - auto src_str_end = src_str + length; - -#if defined(__SSE2__) - constexpr auto step = sizeof(uint64); - if (src_str + step < src_str_end) { - const auto hex_map = _mm_loadu_si128(reinterpret_cast(hex_table)); - const auto mask_map = _mm_set1_epi8(0x0F); - - do { - auto data = _mm_loadu_si64(src_str); - auto hex_loc = _mm_and_si128(_mm_unpacklo_epi8(_mm_srli_epi64(data, 4), data), mask_map); - _mm_storeu_si128(reinterpret_cast<__m128i *>(dst_str), _mm_shuffle_epi8(hex_map, hex_loc)); - - src_str += step; - dst_str += step * 2; - } while (src_str + step < src_str_end); - } -#endif - char res[2]; - // hex(str) str length is n, result must be 2 * n length - for (; src_str < src_str_end; src_str += 1, dst_str += 2) { - // low 4 bits - *(res + 1) = hex_table[src_str[0] & 0x0F]; - // high 4 bits - *res = hex_table[(src_str[0] >> 4)]; - std::copy(res, res + 2, dst_str); - } - } -}; -} - -#endif //BE_V_STRING_FUNCTIONS_H \ No newline at end of file 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/checksum_action.cpp b/be/src/http/action/checksum_action.cpp index f65a86a36d54d7..d1757d84e259c7 100644 --- a/be/src/http/action/checksum_action.cpp +++ b/be/src/http/action/checksum_action.cpp @@ -39,7 +39,6 @@ const std::string TABLET_ID = "tablet_id"; // do not use name "VERSION", // or will be conflict with "VERSION" in thrift/config.h const std::string TABLET_VERSION = "version"; -const std::string VERSION_HASH = "version_hash"; const std::string SCHEMA_HASH = "schema_hash"; ChecksumAction::ChecksumAction(ExecEnv* exec_env) : _exec_env(exec_env) {} @@ -67,14 +66,6 @@ void ChecksumAction::handle(HttpRequest* req) { return; } - // Get version hash - const std::string& version_hash_str = req->param(VERSION_HASH); - if (version_hash_str.empty()) { - std::string error_msg = std::string("parameter " + VERSION_HASH + " not specified in url."); - HttpChannel::send_reply(req, HttpStatus::BAD_REQUEST, error_msg); - return; - } - // Get schema hash const std::string& schema_hash_str = req->param(SCHEMA_HASH); if (schema_hash_str.empty()) { @@ -86,12 +77,10 @@ void ChecksumAction::handle(HttpRequest* req) { // valid str format int64_t tablet_id; int64_t version; - int64_t version_hash; int32_t schema_hash; try { tablet_id = boost::lexical_cast(tablet_id_str); version = boost::lexical_cast(version_str); - version_hash = boost::lexical_cast(version_hash_str); schema_hash = boost::lexical_cast(schema_hash_str); } catch (boost::bad_lexical_cast& e) { std::string error_msg = std::string("param format is invalid: ") + std::string(e.what()); @@ -99,10 +88,9 @@ void ChecksumAction::handle(HttpRequest* req) { return; } - VLOG_ROW << "get checksum tablet info: " << tablet_id << "-" << version << "-" << version_hash - << "-" << schema_hash; + VLOG_ROW << "get checksum tablet info: " << tablet_id << "-" << version << "-" << schema_hash; - int64_t checksum = do_checksum(tablet_id, version, version_hash, schema_hash, req); + int64_t checksum = do_checksum(tablet_id, version, schema_hash, req); if (checksum == -1L) { std::string error_msg = std::string("checksum failed"); HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, error_msg); @@ -117,11 +105,11 @@ void ChecksumAction::handle(HttpRequest* req) { LOG(INFO) << "deal with checksum request finished! tablet id: " << tablet_id; } -int64_t ChecksumAction::do_checksum(int64_t tablet_id, int64_t version, int64_t version_hash, +int64_t ChecksumAction::do_checksum(int64_t tablet_id, int64_t version, int32_t schema_hash, HttpRequest* req) { OLAPStatus res = OLAP_SUCCESS; uint32_t checksum; - EngineChecksumTask engine_task(tablet_id, schema_hash, version, version_hash, &checksum); + EngineChecksumTask engine_task(tablet_id, schema_hash, version, &checksum); res = engine_task.execute(); if (res != OLAP_SUCCESS) { LOG(WARNING) << "checksum failed. status: " << res << ", signature: " << tablet_id; diff --git a/be/src/http/action/checksum_action.h b/be/src/http/action/checksum_action.h index 302a83bbe65ed4..9b85429222351b 100644 --- a/be/src/http/action/checksum_action.h +++ b/be/src/http/action/checksum_action.h @@ -35,7 +35,7 @@ class ChecksumAction : public HttpHandler { void handle(HttpRequest* req) override; private: - int64_t do_checksum(int64_t tablet_id, int64_t version, int64_t version_hash, + int64_t do_checksum(int64_t tablet_id, int64_t version, int32_t schema_hash, HttpRequest* req); ExecEnv* _exec_env; 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/CMakeLists.txt b/be/src/olap/CMakeLists.txt index 36f451592174e2..ced9307ee860b9 100644 --- a/be/src/olap/CMakeLists.txt +++ b/be/src/olap/CMakeLists.txt @@ -120,7 +120,6 @@ add_library(Olap STATIC task/engine_storage_migration_task.cpp task/engine_publish_version_task.cpp task/engine_alter_tablet_task.cpp - olap_snapshot_converter.cpp column_vector.cpp segment_loader.cpp ) diff --git a/be/src/olap/column_vector.cpp b/be/src/olap/column_vector.cpp index 86723391b27f76..949c8022cd01ad 100644 --- a/be/src/olap/column_vector.cpp +++ b/be/src/olap/column_vector.cpp @@ -210,14 +210,19 @@ void ArrayColumnVectorBatch::prepare_for_read(size_t start_idx, size_t size, boo DCHECK(start_idx + size <= capacity()); for (size_t i = 0; i < size; ++i) { if (!is_null_at(start_idx + i)) { - _data[start_idx + i] = CollectionValue( - _elements->mutable_cell_ptr(*(_offsets->scalar_cell_ptr(start_idx + i))), - *(_offsets->scalar_cell_ptr(start_idx + i + 1)) - - *(_offsets->scalar_cell_ptr(start_idx + i)), - item_has_null, - _elements->is_nullable() ? const_cast(&_elements->null_signs()[*( - _offsets->scalar_cell_ptr(start_idx + i))]) - : nullptr); + auto next_offset = *(_offsets->scalar_cell_ptr(start_idx + i + 1)); + auto offset = *(_offsets->scalar_cell_ptr(start_idx + i)); + uint32_t length = next_offset - offset; + if (length == 0) { + _data[start_idx + i] = CollectionValue(length); + } else { + _data[start_idx + i] = CollectionValue( + _elements->mutable_cell_ptr(offset), + length, + item_has_null, + _elements->is_nullable() ? const_cast(&_elements->null_signs()[offset]) + : nullptr); + } } } } diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index d6fc2e96c7dddd..8576292a807c81 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -80,7 +80,6 @@ OLAPStatus Compaction::do_compaction_impl(int64_t permits) { _output_version = Version(_input_rowsets.front()->start_version(), _input_rowsets.back()->end_version()); - _tablet->compute_version_hash_from_rowsets(_input_rowsets, &_output_version_hash); LOG(INFO) << "start " << compaction_name() << ". tablet=" << _tablet->full_name() << ", output_version=" << _output_version << ", permits: " << permits; @@ -163,7 +162,6 @@ OLAPStatus Compaction::construct_output_rowset_writer() { context.tablet_schema = &(_tablet->tablet_schema()); context.rowset_state = VISIBLE; context.version = _output_version; - context.version_hash = _output_version_hash; context.segments_overlap = NONOVERLAPPING; context.parent_mem_tracker = _writer_tracker; // The test results show that one rs writer is low-memory-footprint, there is no need to tracker its mem pool diff --git a/be/src/olap/compaction.h b/be/src/olap/compaction.h index f27944f23e1b59..6e7985a4d1c7c6 100644 --- a/be/src/olap/compaction.h +++ b/be/src/olap/compaction.h @@ -103,7 +103,6 @@ class Compaction { CompactionState _state; Version _output_version; - VersionHash _output_version_hash; DISALLOW_COPY_AND_ASSIGN(Compaction); }; diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp index a3ddab201b157b..5bb86dfe408ee2 100644 --- a/be/src/olap/data_dir.cpp +++ b/be/src/olap/data_dir.cpp @@ -35,7 +35,6 @@ #include "gutil/strings/substitute.h" #include "olap/file_helper.h" #include "olap/olap_define.h" -#include "olap/olap_snapshot_converter.h" #include "olap/rowset/alpha_rowset_meta.h" #include "olap/rowset/rowset_factory.h" #include "olap/rowset/rowset_meta_manager.h" diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index 623528d343c1ab..470efee1f8530c 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -44,7 +44,6 @@ namespace doris { static const int64_t MAX_ROWSET_ID = 1L << 56; typedef int32_t SchemaHash; -typedef int64_t VersionHash; typedef __int128 int128_t; typedef unsigned __int128 uint128_t; diff --git a/be/src/olap/olap_define.h b/be/src/olap/olap_define.h index c72017785d7c5d..6a3f4d6636c4b4 100644 --- a/be/src/olap/olap_define.h +++ b/be/src/olap/olap_define.h @@ -240,7 +240,6 @@ enum OLAPStatus { OLAP_ERR_BE_VERSION_NOT_MATCH = -800, OLAP_ERR_BE_REPLACE_VERSIONS_ERROR = -801, OLAP_ERR_BE_MERGE_ERROR = -802, - OLAP_ERR_BE_COMPUTE_VERSION_HASH_ERROR = -803, OLAP_ERR_CAPTURE_ROWSET_ERROR = -804, OLAP_ERR_BE_SAVE_HEADER_ERROR = -805, OLAP_ERR_BE_INIT_OLAP_DATA = -806, diff --git a/be/src/olap/olap_snapshot_converter.cpp b/be/src/olap/olap_snapshot_converter.cpp deleted file mode 100644 index 334af43dab67dd..00000000000000 --- a/be/src/olap/olap_snapshot_converter.cpp +++ /dev/null @@ -1,306 +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 "olap/olap_snapshot_converter.h" - -#include "olap/rowset/alpha_rowset.h" -#include "olap/rowset/alpha_rowset_meta.h" -#include "olap/rowset/rowset_id_generator.h" -#include "olap/storage_engine.h" - -namespace doris { - -OLAPStatus OlapSnapshotConverter::convert_to_pdelta(const RowsetMetaPB& rowset_meta_pb, - PDelta* delta) { - if (!rowset_meta_pb.has_start_version()) { - LOG(FATAL) << "rowset does not have start_version." - << " rowset id = " << rowset_meta_pb.rowset_id(); - } - delta->set_start_version(rowset_meta_pb.start_version()); - if (!rowset_meta_pb.has_end_version()) { - LOG(FATAL) << "rowset does not have end_version." - << " rowset id = " << rowset_meta_pb.rowset_id(); - } - delta->set_end_version(rowset_meta_pb.end_version()); - if (!rowset_meta_pb.has_version_hash()) { - LOG(FATAL) << "rowset does not have version_hash." - << " rowset id = " << rowset_meta_pb.rowset_id(); - } - delta->set_version_hash(rowset_meta_pb.version_hash()); - if (!rowset_meta_pb.has_creation_time()) { - LOG(FATAL) << "rowset does not have creation_time." - << " rowset id = " << rowset_meta_pb.rowset_id(); - } - delta->set_creation_time(rowset_meta_pb.creation_time()); - AlphaRowsetExtraMetaPB extra_meta_pb = rowset_meta_pb.alpha_rowset_extra_meta_pb(); - - for (auto& segment_group : extra_meta_pb.segment_groups()) { - SegmentGroupPB* new_segment_group = delta->add_segment_group(); - *new_segment_group = segment_group; - } - if (rowset_meta_pb.has_delete_predicate()) { - DeletePredicatePB* delete_condition = delta->mutable_delete_condition(); - *delete_condition = rowset_meta_pb.delete_predicate(); - } - return OLAP_SUCCESS; -} - -OLAPStatus OlapSnapshotConverter::convert_to_rowset_meta(const PDelta& delta, - const RowsetId& rowset_id, - int64_t tablet_id, int32_t schema_hash, - RowsetMetaPB* rowset_meta_pb) { - rowset_meta_pb->set_rowset_id(0); - rowset_meta_pb->set_rowset_id_v2(rowset_id.to_string()); - rowset_meta_pb->set_tablet_id(tablet_id); - rowset_meta_pb->set_tablet_schema_hash(schema_hash); - rowset_meta_pb->set_rowset_type(RowsetTypePB::ALPHA_ROWSET); - rowset_meta_pb->set_rowset_state(RowsetStatePB::VISIBLE); - rowset_meta_pb->set_start_version(delta.start_version()); - rowset_meta_pb->set_end_version(delta.end_version()); - rowset_meta_pb->set_version_hash(delta.version_hash()); - - bool empty = true; - int64_t num_rows = 0; - int64_t index_size = 0; - int64_t data_size = 0; - AlphaRowsetExtraMetaPB* extra_meta_pb = rowset_meta_pb->mutable_alpha_rowset_extra_meta_pb(); - for (auto& segment_group : delta.segment_group()) { - SegmentGroupPB* new_segment_group = extra_meta_pb->add_segment_groups(); - *new_segment_group = segment_group; - // if segment group does not has empty property, then it is not empty - // if segment group's empty == false, then it is not empty - if (!segment_group.has_empty() || !segment_group.empty()) { - empty = false; - } - num_rows += segment_group.num_rows(); - index_size += segment_group.index_size(); - data_size += segment_group.data_size(); - } - - rowset_meta_pb->set_empty(empty); - rowset_meta_pb->set_num_rows(num_rows); - rowset_meta_pb->set_data_disk_size(data_size); - rowset_meta_pb->set_index_disk_size(index_size); - rowset_meta_pb->set_total_disk_size(data_size + index_size); - if (delta.has_delete_condition()) { - DeletePredicatePB* delete_condition = rowset_meta_pb->mutable_delete_predicate(); - *delete_condition = delta.delete_condition(); - } - rowset_meta_pb->set_creation_time(delta.creation_time()); - LOG(INFO) << "convert visible delta start_version = " << delta.start_version() - << " end_version = " << delta.end_version() - << " version_hash = " << delta.version_hash() << " to rowset id = " << rowset_id - << " tablet_id = " << tablet_id; - return OLAP_SUCCESS; -} - -OLAPStatus OlapSnapshotConverter::convert_to_rowset_meta(const PPendingDelta& pending_delta, - const RowsetId& rowset_id, - int64_t tablet_id, int32_t schema_hash, - RowsetMetaPB* rowset_meta_pb) { - rowset_meta_pb->set_rowset_id(0); - rowset_meta_pb->set_rowset_id_v2(rowset_id.to_string()); - rowset_meta_pb->set_tablet_id(tablet_id); - rowset_meta_pb->set_tablet_schema_hash(schema_hash); - rowset_meta_pb->set_rowset_type(RowsetTypePB::ALPHA_ROWSET); - rowset_meta_pb->set_rowset_state(RowsetStatePB::COMMITTED); - rowset_meta_pb->set_partition_id(pending_delta.partition_id()); - rowset_meta_pb->set_txn_id(pending_delta.transaction_id()); - rowset_meta_pb->set_creation_time(pending_delta.creation_time()); - - bool empty = true; - int64_t num_rows = 0; - int64_t index_size = 0; - int64_t data_size = 0; - AlphaRowsetExtraMetaPB* extra_meta_pb = rowset_meta_pb->mutable_alpha_rowset_extra_meta_pb(); - for (auto& pending_segment_group : pending_delta.pending_segment_group()) { - SegmentGroupPB* new_segment_group = extra_meta_pb->add_segment_groups(); - new_segment_group->set_segment_group_id(pending_segment_group.pending_segment_group_id()); - new_segment_group->set_num_segments(pending_segment_group.num_segments()); - new_segment_group->set_index_size(0); - new_segment_group->set_data_size(0); - new_segment_group->set_num_rows(0); - for (auto& pending_zone_map : pending_segment_group.zone_maps()) { - ZoneMap* zone_map = new_segment_group->add_zone_maps(); - *zone_map = pending_zone_map; - } - new_segment_group->set_empty(pending_segment_group.empty()); - PUniqueId* load_id = new_segment_group->mutable_load_id(); - *load_id = pending_segment_group.load_id(); - - if (!pending_segment_group.empty()) { - empty = false; - } - } - - rowset_meta_pb->set_empty(empty); - rowset_meta_pb->set_num_rows(num_rows); - rowset_meta_pb->set_data_disk_size(data_size); - rowset_meta_pb->set_index_disk_size(index_size); - rowset_meta_pb->set_total_disk_size(data_size + index_size); - if (pending_delta.has_delete_condition()) { - DeletePredicatePB* delete_condition = rowset_meta_pb->mutable_delete_predicate(); - *delete_condition = pending_delta.delete_condition(); - } - rowset_meta_pb->set_creation_time(pending_delta.creation_time()); - LOG(INFO) << "convert pending delta txn id = " << pending_delta.transaction_id() - << " tablet_id = " << tablet_id << " schema_hash = " << schema_hash - << " to rowset id = " << rowset_id; - return OLAP_SUCCESS; -} - -OLAPStatus OlapSnapshotConverter::to_column_pb(const ColumnMessage& column_msg, - ColumnPB* column_pb) { - if (column_msg.has_unique_id()) { - column_pb->set_unique_id(column_msg.unique_id()); - } - column_pb->set_name(column_msg.name()); - column_pb->set_type(column_msg.type()); - column_pb->set_is_key(column_msg.is_key()); - column_pb->set_aggregation(column_msg.aggregation()); - if (column_msg.has_is_allow_null()) { - column_pb->set_is_nullable(column_msg.is_allow_null()); - } - if (column_msg.has_default_value()) { - column_pb->set_default_value(column_msg.default_value()); - } - if (column_msg.has_precision()) { - column_pb->set_precision(column_msg.precision()); - } - if (column_msg.has_frac()) { - column_pb->set_frac(column_msg.frac()); - } - column_pb->set_length(column_msg.length()); - if (column_msg.has_index_length()) { - column_pb->set_index_length(column_msg.index_length()); - } - if (column_msg.has_is_bf_column()) { - column_pb->set_is_bf_column(column_msg.is_bf_column()); - } - if (column_msg.has_has_bitmap_index()) { - column_pb->set_has_bitmap_index(column_msg.has_bitmap_index()); - } - // TODO(ygl) calculate column id from column list - // column_pb->set_referenced_column_id(column_msg.()); - - if (column_msg.has_referenced_column()) { - column_pb->set_referenced_column(column_msg.referenced_column()); - } - return OLAP_SUCCESS; -} - -OLAPStatus OlapSnapshotConverter::to_column_msg(const ColumnPB& column_pb, - ColumnMessage* column_msg) { - if (!column_pb.has_name()) { - LOG(FATAL) << "column pb does not have name" - << " column id " << column_pb.unique_id(); - } - column_msg->set_name(column_pb.name()); - column_msg->set_type(column_pb.type()); - if (!column_pb.has_aggregation()) { - LOG(FATAL) << "column pb does not have aggregation" - << " column id " << column_pb.unique_id(); - } - column_msg->set_aggregation(column_pb.aggregation()); - if (!column_pb.has_length()) { - LOG(FATAL) << "column pb does not have length" - << " column id " << column_pb.unique_id(); - } - column_msg->set_length(column_pb.length()); - if (!column_pb.has_is_key()) { - LOG(FATAL) << "column pb does not have is_key" - << " column id " << column_pb.unique_id(); - } - column_msg->set_is_key(column_pb.is_key()); - if (column_pb.has_default_value()) { - column_msg->set_default_value(column_pb.default_value()); - } - if (column_pb.has_referenced_column()) { - column_msg->set_referenced_column(column_pb.referenced_column()); - } - if (column_pb.has_index_length()) { - column_msg->set_index_length(column_pb.index_length()); - } - if (column_pb.has_precision()) { - column_msg->set_precision(column_pb.precision()); - } - if (column_pb.has_frac()) { - column_msg->set_frac(column_pb.frac()); - } - if (column_pb.has_is_nullable()) { - column_msg->set_is_allow_null(column_pb.is_nullable()); - } - column_msg->set_unique_id(column_pb.unique_id()); - if (column_pb.has_is_bf_column()) { - column_msg->set_is_bf_column(column_pb.is_bf_column()); - } - if (column_pb.has_has_bitmap_index()) { - column_msg->set_has_bitmap_index(column_pb.has_bitmap_index()); - } - column_msg->set_is_root_column(true); - return OLAP_SUCCESS; -} - -OLAPStatus OlapSnapshotConverter::save(const string& file_path, - const OLAPHeaderMessage& olap_header) { - DCHECK(!file_path.empty()); - - FileHeader file_header; - FileHandler file_handler; - - if (file_handler.open_with_mode(file_path.c_str(), O_CREAT | O_WRONLY | O_TRUNC, - S_IRUSR | S_IWUSR) != OLAP_SUCCESS) { - LOG(WARNING) << "fail to open header file. file='" << file_path; - return OLAP_ERR_IO_ERROR; - } - - try { - file_header.mutable_message()->CopyFrom(olap_header); - } catch (...) { - LOG(WARNING) << "fail to copy protocol buffer object. file='" << file_path; - return OLAP_ERR_OTHER_ERROR; - } - - if (file_header.prepare(&file_handler) != OLAP_SUCCESS || - file_header.serialize(&file_handler) != OLAP_SUCCESS) { - LOG(WARNING) << "fail to serialize to file header. file='" << file_path; - return OLAP_ERR_SERIALIZE_PROTOBUF_ERROR; - } - - return OLAP_SUCCESS; -} - -void OlapSnapshotConverter::_modify_old_segment_group_id(RowsetMetaPB& rowset_meta) { - if (!rowset_meta.has_alpha_rowset_extra_meta_pb()) { - return; - } - AlphaRowsetExtraMetaPB* alpha_rowset_extra_meta_pb = - rowset_meta.mutable_alpha_rowset_extra_meta_pb(); - for (auto& segment_group_pb : alpha_rowset_extra_meta_pb->segment_groups()) { - if (segment_group_pb.segment_group_id() == -1) { - // check if segment groups size == 1 - if (alpha_rowset_extra_meta_pb->segment_groups().size() != 1) { - LOG(FATAL) << "the rowset has a segment group's id == -1 but it contains more than " - "one segment group" - << " it should not happen"; - } - (const_cast(segment_group_pb)).set_segment_group_id(0); - } - } -} - -} // namespace doris diff --git a/be/src/olap/olap_snapshot_converter.h b/be/src/olap/olap_snapshot_converter.h deleted file mode 100644 index 05a8233e4c01f3..00000000000000 --- a/be/src/olap/olap_snapshot_converter.h +++ /dev/null @@ -1,65 +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_OLAP_OLAP_SNAPSHOT_CONVERTER_H -#define DORIS_BE_SRC_OLAP_OLAP_SNAPSHOT_CONVERTER_H - -#include -#include -#include - -#include "gen_cpp/olap_file.pb.h" -#include "olap/data_dir.h" -#include "olap/delete_handler.h" -#include "olap/olap_common.h" -#include "olap/olap_define.h" -#include "olap/rowset/rowset.h" -#include "olap/rowset/rowset_meta.h" -#include "olap/tablet_schema.h" - -using std::ifstream; -using std::string; -using std::vector; - -namespace doris { - -class OlapSnapshotConverter { -public: - - OLAPStatus convert_to_pdelta(const RowsetMetaPB& rowset_meta_pb, PDelta* delta); - - OLAPStatus convert_to_rowset_meta(const PDelta& delta, const RowsetId& rowset_id, - int64_t tablet_id, int32_t schema_hash, - RowsetMetaPB* rowset_meta_pb); - - OLAPStatus convert_to_rowset_meta(const PPendingDelta& pending_delta, const RowsetId& rowset_id, - int64_t tablet_id, int32_t schema_hash, - RowsetMetaPB* rowset_meta_pb); - - OLAPStatus to_column_pb(const ColumnMessage& column_msg, ColumnPB* column_pb); - - OLAPStatus to_column_msg(const ColumnPB& column_pb, ColumnMessage* column_msg); - - OLAPStatus save(const string& file_path, const OLAPHeaderMessage& olap_header); - -private: - void _modify_old_segment_group_id(RowsetMetaPB& rowset_meta); -}; - -} // namespace doris - -#endif // DORIS_BE_SRC_OLAP_OLAP_SNAPSHOT_CONVERTER_H diff --git a/be/src/olap/row_block2.h b/be/src/olap/row_block2.h index b98ab954c3be49..7f2b79d638e90c 100644 --- a/be/src/olap/row_block2.h +++ b/be/src/olap/row_block2.h @@ -111,7 +111,7 @@ class RowBlockV2 { private: Status _copy_data_to_column(int cid, vectorized::MutableColumnPtr& mutable_column_ptr); - Schema _schema; + const Schema& _schema; size_t _capacity; // _column_vector_batches[cid] == null if cid is not in `_schema`. // memory are not allocated from `_pool` because we don't wan't to reallocate them in clear() diff --git a/be/src/olap/rowset/alpha_rowset.cpp b/be/src/olap/rowset/alpha_rowset.cpp index 62ae5c933f491a..53c695f4d65056 100644 --- a/be/src/olap/rowset/alpha_rowset.cpp +++ b/be/src/olap/rowset/alpha_rowset.cpp @@ -35,15 +35,14 @@ OLAPStatus AlphaRowset::do_load(bool use_cache) { // validate segment group if (segment_group->validate() != OLAP_SUCCESS) { LOG(WARNING) << "fail to validate segment_group. [version=" << start_version() << "-" - << end_version() << " version_hash=" << version_hash(); + << end_version(); // if load segment group failed, rowset init failed return OLAP_ERR_TABLE_INDEX_VALIDATE_ERROR; } OLAPStatus res = segment_group->load(use_cache); if (res != OLAP_SUCCESS) { LOG(WARNING) << "fail to load segment_group. res=" << res << ", " - << "version=" << start_version() << "-" << end_version() << ", " - << "version_hash=" << version_hash(); + << "version=" << start_version() << "-" << end_version(); return res; } } @@ -79,7 +78,7 @@ OLAPStatus AlphaRowset::remove() { return OLAP_SUCCESS; } -void AlphaRowset::make_visible_extra(Version version, VersionHash version_hash) { +void AlphaRowset::make_visible_extra(Version version) { AlphaRowsetMetaSharedPtr alpha_rowset_meta = std::dynamic_pointer_cast(_rowset_meta); std::vector published_segment_groups; @@ -87,7 +86,6 @@ void AlphaRowset::make_visible_extra(Version version, VersionHash version_hash) int32_t segment_group_idx = 0; for (auto& segment_group : _segment_groups) { segment_group->set_version(version); - segment_group->set_version_hash(version_hash); segment_group->set_pending_finished(); published_segment_groups.at(segment_group_idx).clear_load_id(); ++segment_group_idx; @@ -322,7 +320,7 @@ OLAPStatus AlphaRowset::init() { } else { segment_group.reset(new SegmentGroup( _rowset_meta->tablet_id(), _rowset_meta->rowset_id(), _schema, _rowset_path_desc.filepath, - _rowset_meta->version(), _rowset_meta->version_hash(), false, + _rowset_meta->version(), false, segment_group_meta.segment_group_id(), segment_group_meta.num_segments())); } if (segment_group == nullptr) { diff --git a/be/src/olap/rowset/alpha_rowset.h b/be/src/olap/rowset/alpha_rowset.h index 84b17ad669ea09..364d59dc78fef6 100644 --- a/be/src/olap/rowset/alpha_rowset.h +++ b/be/src/olap/rowset/alpha_rowset.h @@ -33,7 +33,6 @@ class AlphaRowset; using AlphaRowsetSharedPtr = std::shared_ptr; class AlphaRowsetWriter; class AlphaRowsetReader; -class OlapSnapshotConverter; class RowsetFactory; class AlphaRowset : public Rowset { @@ -81,7 +80,7 @@ class AlphaRowset : public Rowset { void do_close() override {} // add custom logic when rowset is published - void make_visible_extra(Version version, VersionHash version_hash) override; + void make_visible_extra(Version version) override; private: std::shared_ptr _segment_group_with_largest_size(); @@ -89,7 +88,6 @@ class AlphaRowset : public Rowset { private: friend class AlphaRowsetWriter; friend class AlphaRowsetReader; - friend class OlapSnapshotConverter; std::vector> _segment_groups; }; diff --git a/be/src/olap/rowset/alpha_rowset_reader.cpp b/be/src/olap/rowset/alpha_rowset_reader.cpp index b86cefa9dadeb4..b22bfa1fb59eb0 100644 --- a/be/src/olap/rowset/alpha_rowset_reader.cpp +++ b/be/src/olap/rowset/alpha_rowset_reader.cpp @@ -120,10 +120,6 @@ Version AlphaRowsetReader::version() { return _alpha_rowset_meta->version(); } -VersionHash AlphaRowsetReader::version_hash() { - return _alpha_rowset_meta->version_hash(); -} - int64_t AlphaRowsetReader::filtered_rows() { return _stats->rows_del_filtered; } diff --git a/be/src/olap/rowset/alpha_rowset_reader.h b/be/src/olap/rowset/alpha_rowset_reader.h index 26e8d46429a153..e76bb9465d44f5 100644 --- a/be/src/olap/rowset/alpha_rowset_reader.h +++ b/be/src/olap/rowset/alpha_rowset_reader.h @@ -73,8 +73,6 @@ class AlphaRowsetReader : public RowsetReader { Version version() override; - VersionHash version_hash() override; - RowsetSharedPtr rowset() override; int64_t filtered_rows() override; diff --git a/be/src/olap/rowset/alpha_rowset_writer.cpp b/be/src/olap/rowset/alpha_rowset_writer.cpp index 58630a852b8b4c..d3d087bf81979a 100644 --- a/be/src/olap/rowset/alpha_rowset_writer.cpp +++ b/be/src/olap/rowset/alpha_rowset_writer.cpp @@ -68,7 +68,6 @@ OLAPStatus AlphaRowsetWriter::init(const RowsetWriterContext& rowset_writer_cont _current_rowset_meta->set_load_id(_rowset_writer_context.load_id); } else { _current_rowset_meta->set_version(_rowset_writer_context.version); - _current_rowset_meta->set_version_hash(_rowset_writer_context.version_hash); } RETURN_NOT_OK(_init()); return OLAP_SUCCESS; @@ -259,7 +258,7 @@ OLAPStatus AlphaRowsetWriter::_init() { _cur_segment_group = new (std::nothrow) SegmentGroup( _rowset_writer_context.tablet_id, _rowset_writer_context.rowset_id, _rowset_writer_context.tablet_schema, _rowset_writer_context.path_desc.filepath, - _rowset_writer_context.version, _rowset_writer_context.version_hash, false, + _rowset_writer_context.version, false, _segment_group_id, 0); } DCHECK(_cur_segment_group != nullptr) << "failed to malloc SegmentGroup"; diff --git a/be/src/olap/rowset/beta_rowset_reader.cpp b/be/src/olap/rowset/beta_rowset_reader.cpp index 263a4cc774fa5a..3aed8eb3c37124 100644 --- a/be/src/olap/rowset/beta_rowset_reader.cpp +++ b/be/src/olap/rowset/beta_rowset_reader.cpp @@ -55,7 +55,7 @@ OLAPStatus BetaRowsetReader::init(RowsetReaderContext* read_context) { _stats = _context->stats; } // SegmentIterator will load seek columns on demand - Schema schema(_context->tablet_schema->columns(), *(_context->return_columns)); + _schema = std::make_unique(_context->tablet_schema->columns(), *(_context->return_columns)); // convert RowsetReaderContext to StorageReadOptions StorageReadOptions read_options; @@ -102,7 +102,7 @@ OLAPStatus BetaRowsetReader::init(RowsetReaderContext* read_context) { std::vector> seg_iterators; for (auto& seg_ptr : _segment_cache_handle.get_segments()) { std::unique_ptr iter; - auto s = seg_ptr->new_iterator(schema, read_options, _parent_tracker, &iter); + auto s = seg_ptr->new_iterator(*_schema, read_options, _parent_tracker, &iter); if (!s.ok()) { LOG(WARNING) << "failed to create iterator[" << seg_ptr->id() << "]: " << s.to_string(); return OLAP_ERR_ROWSET_READER_INIT; @@ -131,7 +131,7 @@ OLAPStatus BetaRowsetReader::init(RowsetReaderContext* read_context) { _iterator.reset(final_iterator); // init input block - _input_block.reset(new RowBlockV2(schema, + _input_block.reset(new RowBlockV2(*_schema, std::min(1024, read_context->batch_size), _parent_tracker)); if (!read_context->is_vec) { diff --git a/be/src/olap/rowset/beta_rowset_reader.h b/be/src/olap/rowset/beta_rowset_reader.h index add0c31f297de7..55a8938dbfb7f5 100644 --- a/be/src/olap/rowset/beta_rowset_reader.h +++ b/be/src/olap/rowset/beta_rowset_reader.h @@ -46,8 +46,6 @@ class BetaRowsetReader : public RowsetReader { Version version() override { return _rowset->version(); } - VersionHash version_hash() override { return _rowset->version_hash(); } - RowsetSharedPtr rowset() override { return std::dynamic_pointer_cast(_rowset); } // Return the total number of filtered rows, will be used for validation of schema change @@ -58,6 +56,7 @@ class BetaRowsetReader : public RowsetReader { RowsetTypePB type() const override { return RowsetTypePB::BETA_ROWSET; } private: + std::unique_ptr _schema; RowsetReaderContext* _context; BetaRowsetSharedPtr _rowset; diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp b/be/src/olap/rowset/beta_rowset_writer.cpp index d2d317fd175955..be8713137a3dfa 100644 --- a/be/src/olap/rowset/beta_rowset_writer.cpp +++ b/be/src/olap/rowset/beta_rowset_writer.cpp @@ -82,7 +82,6 @@ OLAPStatus BetaRowsetWriter::init(const RowsetWriterContext& rowset_writer_conte _rowset_meta->set_load_id(_context.load_id); } else { _rowset_meta->set_version(_context.version); - _rowset_meta->set_version_hash(_context.version_hash); } _rowset_meta->set_tablet_uid(_context.tablet_uid); 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_data.h b/be/src/olap/rowset/column_data.h index 93351a2cd005f4..e2565b4c4b84a8 100644 --- a/be/src/olap/rowset/column_data.h +++ b/be/src/olap/rowset/column_data.h @@ -47,7 +47,6 @@ class ColumnData { // 为了与之前兼容, 暴露部分index的接口 Version version() const { return _segment_group->version(); } - VersionHash version_hash() const { return _segment_group->version_hash(); } bool delete_flag() const { return _segment_group->delete_flag(); } uint32_t num_segments() const { return _segment_group->num_segments(); } 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/rowset.cpp b/be/src/olap/rowset/rowset.cpp index 79f7371e6eb210..1e2dea00b072dd 100644 --- a/be/src/olap/rowset/rowset.cpp +++ b/be/src/olap/rowset/rowset.cpp @@ -59,10 +59,9 @@ OLAPStatus Rowset::load(bool use_cache) { return OLAP_SUCCESS; } -void Rowset::make_visible(Version version, VersionHash version_hash) { +void Rowset::make_visible(Version version) { _is_pending = false; _rowset_meta->set_version(version); - _rowset_meta->set_version_hash(version_hash); _rowset_meta->set_rowset_state(VISIBLE); // update create time to the visible time, // it's used to skip recently published version during compaction @@ -72,7 +71,7 @@ void Rowset::make_visible(Version version, VersionHash version_hash) { _rowset_meta->mutable_delete_predicate()->set_version(version.first); return; } - make_visible_extra(version, version_hash); + make_visible_extra(version); } } // namespace doris diff --git a/be/src/olap/rowset/rowset.h b/be/src/olap/rowset/rowset.h index 1983a3621e2b60..8e952d48a6ab63 100644 --- a/be/src/olap/rowset/rowset.h +++ b/be/src/olap/rowset/rowset.h @@ -140,12 +140,11 @@ class Rowset : public std::enable_shared_from_this { bool is_pending() const { return _is_pending; } // publish rowset to make it visible to read - void make_visible(Version version, VersionHash version_hash); + void make_visible(Version version); // helper class to access RowsetMeta int64_t start_version() const { return rowset_meta()->version().first; } int64_t end_version() const { return rowset_meta()->version().second; } - VersionHash version_hash() const { return rowset_meta()->version_hash(); } size_t index_disk_size() const { return rowset_meta()->index_disk_size(); } size_t data_disk_size() const { return rowset_meta()->total_disk_size(); } bool empty() const { return rowset_meta()->empty(); } @@ -269,7 +268,7 @@ class Rowset : public std::enable_shared_from_this { virtual void do_close() = 0; // allow subclass to add custom logic when rowset is being published - virtual void make_visible_extra(Version version, VersionHash version_hash) {} + virtual void make_visible_extra(Version version) {} const TabletSchema* _schema; FilePathDesc _rowset_path_desc; diff --git a/be/src/olap/rowset/rowset_converter.cpp b/be/src/olap/rowset/rowset_converter.cpp index d6d3a563541dad..a8bae5ebb4b550 100644 --- a/be/src/olap/rowset/rowset_converter.cpp +++ b/be/src/olap/rowset/rowset_converter.cpp @@ -52,7 +52,6 @@ OLAPStatus RowsetConverter::_convert_rowset(const RowsetMetaSharedPtr& src_rowse context.segments_overlap = src_rowset_meta->segments_overlap(); if (context.rowset_state == VISIBLE) { context.version = src_rowset_meta->version(); - context.version_hash = src_rowset_meta->version_hash(); } else { context.txn_id = src_rowset_meta->txn_id(); context.load_id = src_rowset_meta->load_id(); diff --git a/be/src/olap/rowset/rowset_meta.h b/be/src/olap/rowset/rowset_meta.h index a31f14e2eff69c..b55fc9402183a4 100644 --- a/be/src/olap/rowset/rowset_meta.h +++ b/be/src/olap/rowset/rowset_meta.h @@ -133,12 +133,6 @@ class RowsetMeta { void set_end_version(int64_t end_version) { _rowset_meta_pb.set_end_version(end_version); } - VersionHash version_hash() const { return _rowset_meta_pb.version_hash(); } - - void set_version_hash(VersionHash version_hash) { - _rowset_meta_pb.set_version_hash(version_hash); - } - int64_t num_rows() const { return _rowset_meta_pb.num_rows(); } void set_num_rows(int64_t num_rows) { _rowset_meta_pb.set_num_rows(num_rows); } diff --git a/be/src/olap/rowset/rowset_reader.h b/be/src/olap/rowset/rowset_reader.h index d0c0bf26ef1d5c..542e705f6d69b8 100644 --- a/be/src/olap/rowset/rowset_reader.h +++ b/be/src/olap/rowset/rowset_reader.h @@ -56,8 +56,6 @@ class RowsetReader { virtual Version version() = 0; - virtual VersionHash version_hash() = 0; - virtual RowsetSharedPtr rowset() = 0; virtual int64_t filtered_rows() = 0; diff --git a/be/src/olap/rowset/rowset_writer_context.h b/be/src/olap/rowset/rowset_writer_context.h index a0ad2736283760..8c314f5dba3cd6 100644 --- a/be/src/olap/rowset/rowset_writer_context.h +++ b/be/src/olap/rowset/rowset_writer_context.h @@ -36,7 +36,6 @@ struct RowsetWriterContext { tablet_schema(nullptr), rowset_state(PREPARED), version(Version(0, 0)), - version_hash(0), txn_id(0), tablet_uid(0, 0), segments_overlap(OVERLAP_UNKNOWN) { @@ -55,7 +54,6 @@ struct RowsetWriterContext { RowsetStatePB rowset_state; // properties for non-pending rowset Version version; - VersionHash version_hash; // properties for pending rowset int64_t txn_id; diff --git a/be/src/olap/rowset/segment_group.cpp b/be/src/olap/rowset/segment_group.cpp index d0e35884e018d4..0dc2f51c38f19c 100644 --- a/be/src/olap/rowset/segment_group.cpp +++ b/be/src/olap/rowset/segment_group.cpp @@ -68,14 +68,13 @@ namespace doris { SegmentGroup::SegmentGroup(int64_t tablet_id, const RowsetId& rowset_id, const TabletSchema* schema, const std::string& rowset_path_prefix, Version version, - VersionHash version_hash, bool delete_flag, int32_t segment_group_id, + bool delete_flag, int32_t segment_group_id, int32_t num_segments) : _tablet_id(tablet_id), _rowset_id(rowset_id), _schema(schema), _rowset_path_prefix(rowset_path_prefix), _version(version), - _version_hash(version_hash), _delete_flag(delete_flag), _segment_group_id(segment_group_id), _num_segments(num_segments) { @@ -118,7 +117,6 @@ SegmentGroup::SegmentGroup(int64_t tablet_id, const RowsetId& rowset_id, const T _partition_id(partition_id), _txn_id(transaction_id) { _version = {-1, -1}; - _version_hash = 0; _load_id.set_hi(0); _load_id.set_lo(0); _index_loaded = false; @@ -584,7 +582,8 @@ OLAPStatus SegmentGroup::add_segment() { index_header = _file_header.mutable_message(); index_header->set_start_version(_version.first); index_header->set_end_version(_version.second); - index_header->set_cumulative_version_hash(_version_hash); + // Version hash is useless but it is a required field in header message pb + index_header->set_cumulative_version_hash(0); index_header->set_segment(_num_segments - 1); index_header->set_num_rows_per_block(_schema->num_rows_per_row_block()); index_header->set_delete_flag(_delete_flag); @@ -1007,12 +1006,12 @@ std::string SegmentGroup::_construct_old_file_path(const std::string& path_prefi const std::string& suffix) const { char file_path[OLAP_MAX_PATH_LEN]; if (_segment_group_id == -1) { - snprintf(file_path, sizeof(file_path), "%s/%ld_%ld_%ld_%ld_%d%s", path_prefix.c_str(), - _tablet_id, _version.first, _version.second, _version_hash, segment_id, + snprintf(file_path, sizeof(file_path), "%s/%ld_%ld_%ld_%d%s", path_prefix.c_str(), + _tablet_id, _version.first, _version.second, segment_id, suffix.c_str()); } else { - snprintf(file_path, sizeof(file_path), "%s/%ld_%ld_%ld_%ld_%d_%d%s", path_prefix.c_str(), - _tablet_id, _version.first, _version.second, _version_hash, _segment_group_id, + snprintf(file_path, sizeof(file_path), "%s/%ld_%ld_%ld_%d_%d%s", path_prefix.c_str(), + _tablet_id, _version.first, _version.second, _segment_group_id, segment_id, suffix.c_str()); } @@ -1024,8 +1023,8 @@ std::string SegmentGroup::_construct_err_sg_file_path(const std::string& path_pr int32_t segment_id, const std::string& suffix) const { char file_path[OLAP_MAX_PATH_LEN]; - snprintf(file_path, sizeof(file_path), "%s/%ld_%ld_%ld_%ld_%d%s", path_prefix.c_str(), - _tablet_id, _version.first, _version.second, _version_hash, segment_id, + snprintf(file_path, sizeof(file_path), "%s/%ld_%ld_%ld_%d%s", path_prefix.c_str(), + _tablet_id, _version.first, _version.second, segment_id, suffix.c_str()); return file_path; diff --git a/be/src/olap/rowset/segment_group.h b/be/src/olap/rowset/segment_group.h index e5a033f5e3cb64..09302feee5b00c 100644 --- a/be/src/olap/rowset/segment_group.h +++ b/be/src/olap/rowset/segment_group.h @@ -49,7 +49,7 @@ class SegmentGroup { public: SegmentGroup(int64_t tablet_id, const RowsetId& rowset_id, const TabletSchema* tablet_schema, - const std::string& rowset_path_prefix, Version version, VersionHash version_hash, + const std::string& rowset_path_prefix, Version version, bool delete_flag, int segment_group_id, int32_t num_segments); SegmentGroup(int64_t tablet_id, const RowsetId& rowset_id, const TabletSchema* tablet_schema, @@ -142,8 +142,6 @@ class SegmentGroup { inline Version version() const { return _version; } inline void set_version(Version version) { _version = version; } - inline VersionHash version_hash() const { return _version_hash; } - inline void set_version_hash(VersionHash version_hash) { _version_hash = version_hash; } inline bool is_pending() const { return _is_pending; } inline void set_pending_finished() { _is_pending = false; } @@ -271,7 +269,6 @@ class SegmentGroup { const TabletSchema* _schema; std::string _rowset_path_prefix; // path of rowset Version _version; // version of associated data file - VersionHash _version_hash; // version hash for this segment group bool _delete_flag; int32_t _segment_group_id; // segment group id of segment group PUniqueId _load_id; // load id for segment group diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp index 9b14ff4201d4f7..3d1bf80182c541 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.cpp +++ b/be/src/olap/rowset/segment_v2/column_reader.cpp @@ -127,7 +127,7 @@ Status ColumnReader::init() { "Bad file $0: invalid column index type $1", _path_desc.filepath, index_meta.type())); } } - if (_ordinal_index_meta == nullptr) { + if (!is_empty() && _ordinal_index_meta == nullptr) { return Status::Corruption(strings::Substitute( "Bad file $0: missing ordinal index for column $1", _path_desc.filepath, _meta.column_id())); } @@ -339,6 +339,10 @@ Status ColumnReader::seek_at_or_before(ordinal_t ordinal, OrdinalPageIndexIterat } Status ColumnReader::new_iterator(ColumnIterator** iterator) { + if (is_empty()) { + *iterator = new EmptyFileColumnIterator(); + return Status::OK(); + } if (is_scalar_type((FieldType)_meta.type())) { *iterator = new FileColumnIterator(this); return Status::OK(); @@ -427,7 +431,7 @@ Status ArrayFileColumnIterator::next_batch(size_t* n, ColumnBlockView* dst, bool // read item size_t item_size = array_batch->get_item_size(dst->current_offset(), *n); - if (item_size > 0) { + if (item_size >= 0) { bool item_has_null = false; ColumnVectorBatch* item_vector_batch = array_batch->elements(); diff --git a/be/src/olap/rowset/segment_v2/column_reader.h b/be/src/olap/rowset/segment_v2/column_reader.h index e2e6c35ee42582..28189eb147e763 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.h +++ b/be/src/olap/rowset/segment_v2/column_reader.h @@ -123,13 +123,16 @@ 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); PagePointer get_dict_page_pointer() const { return _meta.dict_page(); } + inline bool is_empty() const { return _num_rows == 0; } + private: ColumnReader(const ColumnReaderOptions& opts, const ColumnMetaPB& meta, uint64_t num_rows, FilePathDesc path_desc); @@ -159,7 +162,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 +219,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 +229,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 +244,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 +263,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; } @@ -324,12 +310,21 @@ class FileColumnIterator final : public ColumnIterator { std::unique_ptr _dict_len_array; }; +class EmptyFileColumnIterator final : public ColumnIterator { +public: + Status seek_to_first() override { return Status::OK(); } + Status seek_to_ordinal(ordinal_t ord) override { return Status::OK(); } + Status next_batch(size_t* n, ColumnBlockView* dst, bool* has_null) override { + *n = 0; + return Status::OK(); + } + ordinal_t get_current_ordinal() const override { return 0; } +}; + 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 +349,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 +414,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/olap/rowset/segment_v2/column_writer.cpp b/be/src/olap/rowset/segment_v2/column_writer.cpp index ee26ce3f859ff1..5f982083f96300 100644 --- a/be/src/olap/rowset/segment_v2/column_writer.cpp +++ b/be/src/olap/rowset/segment_v2/column_writer.cpp @@ -524,8 +524,8 @@ Status ArrayColumnWriter::append_data(const uint8_t** ptr, size_t num_rows) { if (num_written < 1) { // page is full, write first item offset and update current length page's start ordinal RETURN_IF_ERROR(_length_writer->finish_current_page()); - _current_length_page_first_ordinal += _lengh_sum_in_cur_page; - _lengh_sum_in_cur_page = 0; + _current_length_page_first_ordinal += _length_sum_in_cur_page; + _length_sum_in_cur_page = 0; } else { // write child item. if (_item_writer->is_nullable()) { @@ -539,7 +539,7 @@ Status ArrayColumnWriter::append_data(const uint8_t** ptr, size_t num_rows) { RETURN_IF_ERROR(_item_writer->append_data(reinterpret_cast(&data), col_cursor->length())); } - _lengh_sum_in_cur_page += col_cursor->length(); + _length_sum_in_cur_page += col_cursor->length(); } remaining -= num_written; col_cursor += num_written; @@ -579,7 +579,9 @@ Status ArrayColumnWriter::write_ordinal_index() { if (is_nullable()) { RETURN_IF_ERROR(_null_writer->write_ordinal_index()); } - RETURN_IF_ERROR(_item_writer->write_ordinal_index()); + if (!has_empty_items()) { + RETURN_IF_ERROR(_item_writer->write_ordinal_index()); + } return Status::OK(); } diff --git a/be/src/olap/rowset/segment_v2/column_writer.h b/be/src/olap/rowset/segment_v2/column_writer.h index b98f4883ca4e09..26c973108e6465 100644 --- a/be/src/olap/rowset/segment_v2/column_writer.h +++ b/be/src/olap/rowset/segment_v2/column_writer.h @@ -307,6 +307,9 @@ class ArrayColumnWriter final : public ColumnWriter, public FlushPageCallback { private: Status put_extra_info_in_page(DataPageFooterPB* header) override; inline Status write_null_column(size_t num_rows, bool is_null); // 写入num_rows个null标记 + inline bool has_empty_items() const { + return _item_writer->get_next_rowid() == 0; + } private: std::unique_ptr _length_writer; @@ -314,7 +317,7 @@ class ArrayColumnWriter final : public ColumnWriter, public FlushPageCallback { std::unique_ptr _item_writer; ColumnWriterOptions _opts; ordinal_t _current_length_page_first_ordinal = 0; - ordinal_t _lengh_sum_in_cur_page = 0; + ordinal_t _length_sum_in_cur_page = 0; }; } // namespace segment_v2 diff --git a/be/src/olap/rowset/segment_v2/empty_segment_iterator.h b/be/src/olap/rowset/segment_v2/empty_segment_iterator.h index 3e1a4f9bf4980c..0c186ed861cf38 100644 --- a/be/src/olap/rowset/segment_v2/empty_segment_iterator.h +++ b/be/src/olap/rowset/segment_v2/empty_segment_iterator.h @@ -35,7 +35,7 @@ class EmptySegmentIterator : public RowwiseIterator { Status next_batch(vectorized::Block* block) override; private: - Schema _schema; + const Schema& _schema; }; } // namespace segment_v2 diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h b/be/src/olap/rowset/segment_v2/segment_iterator.h index 0577526a02f616..2eae13eb3422d0 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.h +++ b/be/src/olap/rowset/segment_v2/segment_iterator.h @@ -103,8 +103,7 @@ class SegmentIterator : public RowwiseIterator { class BitmapRangeIterator; std::shared_ptr _segment; - // TODO(zc): rethink if we need copy it - Schema _schema; + const Schema& _schema; // _column_iterators.size() == _schema.num_columns() // _column_iterators[cid] == nullptr if cid is not in _schema std::vector _column_iterators; diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index f8a6af29441f65..e3c7d8b4bf8feb 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -1123,9 +1123,7 @@ SchemaChangeWithSorting::SchemaChangeWithSorting(const RowBlockChanger& row_bloc _memory_limitation(memory_limitation), _row_block_allocator(nullptr) { // Every time SchemaChange is used for external rowing, some temporary versions (such as 999, 1000, 1001) will be written, in order to avoid Cache conflicts, temporary - // The version performs 2 processes: - // 1. Random value as VersionHash - // 2. The version number takes a BIG NUMBER plus the version number of the current SchemaChange + // The version number takes a BIG NUMBER plus the version number of the current SchemaChange _temp_delta_versions.first = (1 << 28); _temp_delta_versions.second = (1 << 28); // TODO(zyh): remove the magic number @@ -1238,7 +1236,7 @@ OLAPStatus SchemaChangeWithSorting::process(RowsetReaderSharedPtr rowset_reader, if (!_internal_sorting( row_block_arr, Version(_temp_delta_versions.second, _temp_delta_versions.second), - rowset_reader->version_hash(), new_tablet, new_rowset_type, + new_tablet, new_rowset_type, segments_overlap, &rowset)) { LOG(WARNING) << "failed to sorting internally."; return OLAP_ERR_ALTER_STATUS_ERR; @@ -1294,7 +1292,7 @@ OLAPStatus SchemaChangeWithSorting::process(RowsetReaderSharedPtr rowset_reader, } if (!_internal_sorting(row_block_arr, Version(_temp_delta_versions.second, _temp_delta_versions.second), - rowset_reader->version_hash(), new_tablet, new_rowset_type, + new_tablet, new_rowset_type, segments_overlap, &rowset)) { LOG(WARNING) << "failed to sorting internally."; return OLAP_ERR_ALTER_STATUS_ERR; @@ -1350,7 +1348,7 @@ OLAPStatus SchemaChangeWithSorting::process(RowsetReaderSharedPtr rowset_reader, } bool SchemaChangeWithSorting::_internal_sorting(const std::vector& row_block_arr, - const Version& version, VersionHash version_hash, + const Version& version, TabletSharedPtr new_tablet, RowsetTypePB new_rowset_type, SegmentsOverlapPB segments_overlap, @@ -1369,7 +1367,6 @@ bool SchemaChangeWithSorting::_internal_sorting(const std::vector& ro context.tablet_schema = &(new_tablet->tablet_schema()); context.rowset_state = VISIBLE; context.version = version; - context.version_hash = version_hash; context.segments_overlap = segments_overlap; context.parent_mem_tracker = _mem_tracker; @@ -1437,8 +1434,7 @@ OLAPStatus SchemaChangeHandler::process_alter_tablet_v2(const TAlterTabletReqV2& << ", base_schema_hash=" << request.base_schema_hash << ", new_tablet_id=" << request.new_tablet_id << ", new_schema_hash=" << request.new_schema_hash - << ", alter_version=" << request.alter_version - << ", alter_version_hash=" << request.alter_version_hash; + << ", alter_version=" << request.alter_version; // Lock schema_change_lock util schema change info is stored in tablet header if (!StorageEngine::instance()->tablet_manager()->try_schema_change_lock( @@ -1922,7 +1918,6 @@ OLAPStatus SchemaChangeHandler::_convert_historical_rowsets(const SchemaChangePa writer_context.tablet_schema = &(new_tablet->tablet_schema()); writer_context.rowset_state = VISIBLE; writer_context.version = rs_reader->version(); - writer_context.version_hash = rs_reader->version_hash(); writer_context.segments_overlap = rs_reader->rowset()->rowset_meta()->segments_overlap(); writer_context.parent_mem_tracker = _mem_tracker; @@ -2197,9 +2192,7 @@ OLAPStatus SchemaChangeHandler::_init_column_mapping(ColumnMapping* column_mappi OLAPStatus SchemaChangeHandler::_validate_alter_result(TabletSharedPtr new_tablet, const TAlterTabletReqV2& request) { Version max_continuous_version = {-1, 0}; - VersionHash max_continuous_version_hash = 0; - new_tablet->max_continuous_version_from_beginning(&max_continuous_version, - &max_continuous_version_hash); + new_tablet->max_continuous_version_from_beginning(&max_continuous_version); LOG(INFO) << "find max continuous version of tablet=" << new_tablet->full_name() << ", start_version=" << max_continuous_version.first << ", end_version=" << max_continuous_version.second; diff --git a/be/src/olap/schema_change.h b/be/src/olap/schema_change.h index e0d42e8fbec98d..53aa34934a5aec 100644 --- a/be/src/olap/schema_change.h +++ b/be/src/olap/schema_change.h @@ -166,7 +166,7 @@ class SchemaChangeWithSorting : public SchemaChange { private: bool _internal_sorting(const std::vector& row_block_arr, - const Version& temp_delta_versions, const VersionHash version_hash, + const Version& temp_delta_versions, TabletSharedPtr new_tablet, RowsetTypePB new_rowset_type, SegmentsOverlapPB segments_overlap, RowsetSharedPtr* rowset); diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp index 3789a1e33f588c..80816d71c7be2e 100644 --- a/be/src/olap/snapshot_manager.cpp +++ b/be/src/olap/snapshot_manager.cpp @@ -29,7 +29,6 @@ #include "env/env.h" #include "gen_cpp/Types_constants.h" -#include "olap/olap_snapshot_converter.h" #include "olap/rowset/alpha_rowset_meta.h" #include "olap/rowset/rowset.h" #include "olap/rowset/rowset_converter.h" @@ -220,7 +219,6 @@ OLAPStatus SnapshotManager::_rename_rowset_id(const RowsetMetaPB& rs_meta_pb, context.tablet_schema = &tablet_schema; context.rowset_state = org_rowset_meta->rowset_state(); context.version = org_rowset_meta->version(); - context.version_hash = org_rowset_meta->version_hash(); // keep segments_overlap same as origin rowset context.segments_overlap = alpha_rowset_meta->segments_overlap(); diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index aeac35022a8d5e..201bffb6649bb2 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -42,7 +42,6 @@ #include "olap/fs/file_block_manager.h" #include "olap/lru_cache.h" #include "olap/memtable_flush_executor.h" -#include "olap/olap_snapshot_converter.h" #include "olap/push_handler.h" #include "olap/reader.h" #include "olap/rowset/alpha_rowset.h" diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 6508b1980c997c..b96290db74c2fb 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -395,7 +395,6 @@ void Tablet::_delete_stale_rowset_by_version(const Version& version) { void Tablet::delete_expired_stale_rowset() { int64_t now = UnixSeconds(); - std::vector> expired_versions; WriteLock wrlock(&_meta_lock); // Compute the end time to delete rowsets, when a expired rowset createtime less then this time, it will be deleted. double expired_stale_sweep_endtime = @@ -767,17 +766,6 @@ const uint32_t Tablet::_calc_base_compaction_score() const { return base_rowset_exist ? score : 0; } -void Tablet::compute_version_hash_from_rowsets(const std::vector& rowsets, - VersionHash* version_hash) { - DCHECK(version_hash != nullptr) << "invalid parameter, version_hash is nullptr"; - int64_t v_hash = 0; - // version hash is useless since Doris version 0.11 - // but for compatibility, we set version hash as the last rowset's version hash. - // this can also enable us to do the compaction for last one rowset. - v_hash = rowsets.back()->version_hash(); - *version_hash = v_hash; -} - void Tablet::calc_missed_versions(int64_t spec_version, std::vector* missed_versions) { ReadLock rdlock(&_meta_lock); calc_missed_versions_unlocked(spec_version, missed_versions); @@ -818,36 +806,32 @@ void Tablet::calc_missed_versions_unlocked(int64_t spec_version, } } -void Tablet::max_continuous_version_from_beginning(Version* version, VersionHash* v_hash) { +void Tablet::max_continuous_version_from_beginning(Version* version) { ReadLock rdlock(&_meta_lock); - _max_continuous_version_from_beginning_unlocked(version, v_hash); + _max_continuous_version_from_beginning_unlocked(version); } -void Tablet::_max_continuous_version_from_beginning_unlocked(Version* version, - VersionHash* v_hash) const { - std::vector> existing_versions; +void Tablet::_max_continuous_version_from_beginning_unlocked(Version* version) const { + std::vector existing_versions; for (auto& rs : _tablet_meta->all_rs_metas()) { - existing_versions.emplace_back(rs->version(), rs->version_hash()); + existing_versions.emplace_back(rs->version()); } // sort the existing versions in ascending order std::sort(existing_versions.begin(), existing_versions.end(), - [](const pair& left, const pair& right) { + [](const Version& left, const Version& right) { // simple because 2 versions are certainly not overlapping - return left.first.first < right.first.first; + return left.first < right.first; }); Version max_continuous_version = {-1, 0}; - VersionHash max_continuous_version_hash = 0; for (int i = 0; i < existing_versions.size(); ++i) { - if (existing_versions[i].first.first > max_continuous_version.second + 1) { + if (existing_versions[i].first > max_continuous_version.second + 1) { break; } - max_continuous_version = existing_versions[i].first; - max_continuous_version_hash = existing_versions[i].second; + max_continuous_version = existing_versions[i]; } *version = max_continuous_version; - *v_hash = max_continuous_version_hash; } void Tablet::calculate_cumulative_point() { @@ -1264,8 +1248,7 @@ void Tablet::build_tablet_report_info(TTabletInfo* tablet_info) { tablet_info->row_count = _tablet_meta->num_rows(); tablet_info->data_size = _tablet_meta->tablet_footprint(); Version version = {-1, 0}; - VersionHash v_hash = 0; - _max_continuous_version_from_beginning_unlocked(&version, &v_hash); + _max_continuous_version_from_beginning_unlocked(&version); auto max_rowset = rowset_with_max_version(); if (max_rowset != nullptr) { if (max_rowset->version() != version) { @@ -1284,7 +1267,8 @@ void Tablet::build_tablet_report_info(TTabletInfo* tablet_info) { // and perform state modification operations. } tablet_info->version = version.second; - tablet_info->version_hash = v_hash; + // Useless but it is a required filed in TTabletInfo + tablet_info->version_hash = 0; tablet_info->__set_partition_id(_tablet_meta->partition_id()); tablet_info->__set_storage_medium(_data_dir->storage_medium()); tablet_info->__set_version_count(_tablet_meta->version_count()); diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index 42e703af3864c3..03a86f820a56ec 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -164,8 +164,6 @@ class Tablet : public BaseTablet { uint32_t calc_compaction_score( CompactionType compaction_type, std::shared_ptr cumulative_compaction_policy); - static void compute_version_hash_from_rowsets(const std::vector& rowsets, - VersionHash* version_hash); // operation for clone void calc_missed_versions(int64_t spec_version, std::vector* missed_versions); @@ -174,7 +172,7 @@ class Tablet : public BaseTablet { // This function to find max continuous version from the beginning. // For example: If there are 1, 2, 3, 5, 6, 7 versions belongs tablet, then 3 is target. - void max_continuous_version_from_beginning(Version* version, VersionHash* v_hash); + void max_continuous_version_from_beginning(Version* version); // operation for query OLAPStatus split_range(const OlapTuple& start_key_strings, const OlapTuple& end_key_strings, @@ -269,8 +267,7 @@ class Tablet : public BaseTablet { // Returns: // version: the max continuous version from beginning - void _max_continuous_version_from_beginning_unlocked(Version* version, - VersionHash* v_hash) const; + void _max_continuous_version_from_beginning_unlocked(Version* version) const; RowsetSharedPtr _rowset_with_largest_size(); /// Delete stale rowset by version. This method not only delete the version in expired rowset map, /// but also delete the version in rowset meta vector. diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index 3cb085922f6581..2e7cd8a8a45c52 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -1243,7 +1243,6 @@ OLAPStatus TabletManager::_create_initial_rowset_unlocked(const TCreateTabletReq context.tablet_schema = &(tablet->tablet_schema()); context.rowset_state = VISIBLE; context.version = version; - context.version_hash = request.version_hash; // there is no data in init rowset, so overlapping info is unknown. context.segments_overlap = OVERLAP_UNKNOWN; diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index 249e554fd7a06c..6e6195d951e23b 100644 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -601,8 +601,8 @@ void TabletMeta::remove_delete_predicate_by_version(const Version& version) { for (const auto& it : temp.sub_predicates()) { del_cond_str += it + ";"; } - LOG(INFO) << "remove one del_pred. version=" << temp.version() - << ", condition=" << del_cond_str; + VLOG_NOTICE << "remove one del_pred. version=" << temp.version() + << ", condition=" << del_cond_str; // remove delete condition from PB _del_pred_array.SwapElements(ordinal, _del_pred_array.size() - 1); diff --git a/be/src/olap/task/engine_checksum_task.cpp b/be/src/olap/task/engine_checksum_task.cpp index a5c6091b9f0d9e..b795efb834ab27 100644 --- a/be/src/olap/task/engine_checksum_task.cpp +++ b/be/src/olap/task/engine_checksum_task.cpp @@ -23,12 +23,10 @@ namespace doris { EngineChecksumTask::EngineChecksumTask(TTabletId tablet_id, TSchemaHash schema_hash, - TVersion version, TVersionHash version_hash, - uint32_t* checksum) + TVersion version, uint32_t* checksum) : _tablet_id(tablet_id), _schema_hash(schema_hash), _version(version), - _version_hash(version_hash), _checksum(checksum) {} OLAPStatus EngineChecksumTask::execute() { diff --git a/be/src/olap/task/engine_checksum_task.h b/be/src/olap/task/engine_checksum_task.h index 5fb0d1c68d946f..7f0cdb6e6bdcc8 100644 --- a/be/src/olap/task/engine_checksum_task.h +++ b/be/src/olap/task/engine_checksum_task.h @@ -32,7 +32,7 @@ class EngineChecksumTask : public EngineTask { public: EngineChecksumTask(TTabletId tablet_id, TSchemaHash schema_hash, TVersion version, - TVersionHash version_hash, uint32_t* checksum); + uint32_t* checksum); ~EngineChecksumTask() {} @@ -43,7 +43,6 @@ class EngineChecksumTask : public EngineTask { TTabletId _tablet_id; TSchemaHash _schema_hash; TVersion _version; - TVersionHash _version_hash; uint32_t* _checksum; }; // EngineTask diff --git a/be/src/olap/task/engine_clone_task.cpp b/be/src/olap/task/engine_clone_task.cpp index 25a863c575252b..71e73a3b642ab9 100644 --- a/be/src/olap/task/engine_clone_task.cpp +++ b/be/src/olap/task/engine_clone_task.cpp @@ -26,7 +26,6 @@ #include "gutil/strings/stringpiece.h" #include "gutil/strings/substitute.h" #include "http/http_client.h" -#include "olap/olap_snapshot_converter.h" #include "olap/rowset/rowset.h" #include "olap/rowset/rowset_factory.h" #include "olap/snapshot_manager.h" diff --git a/be/src/olap/task/engine_publish_version_task.cpp b/be/src/olap/task/engine_publish_version_task.cpp index 51ad517ca5dd6a..fead3f03a1eba7 100644 --- a/be/src/olap/task/engine_publish_version_task.cpp +++ b/be/src/olap/task/engine_publish_version_task.cpp @@ -54,7 +54,6 @@ OLAPStatus EnginePublishVersionTask::finish() { transaction_id, partition_id, &tablet_related_rs); Version version(par_ver_info.version, par_ver_info.version); - VersionHash version_hash = par_ver_info.version_hash; // each tablet for (auto& tablet_rs : tablet_related_rs) { @@ -64,7 +63,7 @@ OLAPStatus EnginePublishVersionTask::finish() { VLOG_CRITICAL << "begin to publish version on tablet. " << "tablet_id=" << tablet_info.tablet_id << ", schema_hash=" << tablet_info.schema_hash - << ", version=" << version.first << ", version_hash=" << version_hash + << ", version=" << version.first << ", transaction_id=" << transaction_id; // if rowset is null, it means this be received write task, but failed during write // and receive fe's publish version task @@ -87,7 +86,7 @@ OLAPStatus EnginePublishVersionTask::finish() { } publish_status = StorageEngine::instance()->txn_manager()->publish_txn( - partition_id, tablet, transaction_id, version, version_hash); + partition_id, tablet, transaction_id, version); if (publish_status != OLAP_SUCCESS) { LOG(WARNING) << "failed to publish version. rowset_id=" << rowset->rowset_id() << ", tablet_id=" << tablet_info.tablet_id diff --git a/be/src/olap/txn_manager.cpp b/be/src/olap/txn_manager.cpp index 11437e2b5b91a1..1825edae3ab8e8 100644 --- a/be/src/olap/txn_manager.cpp +++ b/be/src/olap/txn_manager.cpp @@ -97,11 +97,9 @@ OLAPStatus TxnManager::commit_txn(TPartitionId partition_id, const TabletSharedP } OLAPStatus TxnManager::publish_txn(TPartitionId partition_id, const TabletSharedPtr& tablet, - TTransactionId transaction_id, const Version& version, - VersionHash version_hash) { + TTransactionId transaction_id, const Version& version) { return publish_txn(tablet->data_dir()->get_meta(), partition_id, transaction_id, - tablet->tablet_id(), tablet->schema_hash(), tablet->tablet_uid(), version, - version_hash); + tablet->tablet_id(), tablet->schema_hash(), tablet->tablet_uid(), version); } // delete the txn from manager if it is not committed(not have a valid rowset) @@ -259,7 +257,7 @@ OLAPStatus TxnManager::commit_txn(OlapMeta* meta, TPartitionId partition_id, OLAPStatus TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id, TTransactionId transaction_id, TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid, - const Version& version, VersionHash version_hash) { + const Version& version) { pair key(partition_id, transaction_id); TabletInfo tablet_info(tablet_id, schema_hash, tablet_uid); RowsetSharedPtr rowset_ptr = nullptr; @@ -283,7 +281,7 @@ OLAPStatus TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id, if (rowset_ptr != nullptr) { // TODO(ygl): rowset is already set version here, memory is changed, if save failed // it maybe a fatal error - rowset_ptr->make_visible(version, version_hash); + rowset_ptr->make_visible(version); OLAPStatus save_status = RowsetMetaManager::save(meta, tablet_uid, rowset_ptr->rowset_id(), rowset_ptr->rowset_meta()->get_rowset_pb()); diff --git a/be/src/olap/txn_manager.h b/be/src/olap/txn_manager.h index 2c3c3f3e159be0..a4050d8c795a2f 100644 --- a/be/src/olap/txn_manager.h +++ b/be/src/olap/txn_manager.h @@ -80,8 +80,7 @@ class TxnManager { const RowsetSharedPtr& rowset_ptr, bool is_recovery); OLAPStatus publish_txn(TPartitionId partition_id, const TabletSharedPtr& tablet, - TTransactionId transaction_id, const Version& version, - VersionHash version_hash); + TTransactionId transaction_id, const Version& version); // delete the txn from manager if it is not committed(not have a valid rowset) OLAPStatus rollback_txn(TPartitionId partition_id, const TabletSharedPtr& tablet, @@ -105,7 +104,7 @@ class TxnManager { // not persist rowset meta because OLAPStatus publish_txn(OlapMeta* meta, TPartitionId partition_id, TTransactionId transaction_id, TTabletId tablet_id, SchemaHash schema_hash, TabletUid tablet_uid, - const Version& version, VersionHash version_hash); + const Version& version); // delete the txn from manager if it is not committed(not have a valid rowset) OLAPStatus rollback_txn(TPartitionId partition_id, TTransactionId transaction_id, diff --git a/be/src/olap/types.h b/be/src/olap/types.h index ec59a92d4bdae6..457cf602578f43 100644 --- a/be/src/olap/types.h +++ b/be/src/olap/types.h @@ -248,7 +248,7 @@ class ArrayTypeInfo : public TypeInfo { dest_value->shallow_copy(src_value); } - inline void deep_copy(void* dest, const void* src, MemPool* mem_pool) const { + inline void deep_copy(void* dest, const void* src, MemPool* mem_pool) const override { auto dest_value = reinterpret_cast(dest); auto src_value = reinterpret_cast(src); diff --git a/be/src/plugin/CMakeLists.txt b/be/src/plugin/CMakeLists.txt deleted file mode 100644 index ecc5dac55f695b..00000000000000 --- a/be/src/plugin/CMakeLists.txt +++ /dev/null @@ -1,28 +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. - -# where to put generated libraries -set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/plugin") - -# where to put generated binaries -set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/plugin") - -add_library(Plugin STATIC - plugin_loader.cpp - plugin_mgr.cpp - plugin_zip.cpp - ) diff --git a/be/src/plugin/plugin.h b/be/src/plugin/plugin.h deleted file mode 100644 index 9a31f46aefa67d..00000000000000 --- a/be/src/plugin/plugin.h +++ /dev/null @@ -1,82 +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_PLUGIN_PLUGIN_H -#define DORIS_BE_PLUGIN_PLUGIN_H - -namespace doris { - -#define PLUGIN_TYPE_AUDIT 1 -#define PLUGIN_TYPE_IMPORT 2 -#define PLUGIN_TYPE_STORAGE 3 -#define PLUGIN_TYPE_MAX 4 - -#define PLUGIN_DEFAULT_FLAG 0UL -#define PLUGIN_INSTALL_EARLY 1UL -#define PLUGIN_NOT_DYNAMIC_UNINSTALL 2UL - -#define DORIS_PLUGIN_VERSION 001100UL - -/** - * define a plugin: - * - * declare_plugin(PLUGIN_NAME) { - * xx_handler, - * init_method, - * close_method, - * PLUGIN_NOT_DYNAMIC_INSTALL | PLUGIN_NOT_DYNAMIC_UNINSTALL, - * nullptr, - * nullptr - * } declare_plugin_end - * - */ -struct Plugin { - // support by type-specific plugin - void* handler; - - // invoke when plugin install - int (*init)(void*); - - // invoke when plugin uninstall - int (*close)(void*); - - // flag for plugin - uint64_t flags; - - // use to set/get variables - void* variable; - - // return the plugin's status - void* status; -}; - -#define __DECLARE_PLUGIN(VERSION, PSIZE, DECLS) \ - int VERSION = DORIS_PLUGIN_VERSION; \ - int PSIZE = sizeof(struct Plugin); \ - Plugin DECLS[] = { -// Plugin Name must be same with plugin's description file -#define declare_plugin(NAME) \ - __DECLARE_PLUGIN(NAME##_plugin_interface_version, NAME##_sizeof_plugin, NAME##_plugin) - -#define declare_plugin_end \ - , { 0, 0, 0, 0, 0, 0 } \ - } \ - ; - -} // namespace doris - -#endif //DORIS_BE_PLUGIN_PLUGIN_H diff --git a/be/src/plugin/plugin_loader.cpp b/be/src/plugin/plugin_loader.cpp deleted file mode 100644 index a0d0674022442b..00000000000000 --- a/be/src/plugin/plugin_loader.cpp +++ /dev/null @@ -1,197 +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 "plugin/plugin_loader.h" - -#include -#include - -#include "env/env.h" -#include "gutil/strings/substitute.h" -#include "gutil/strings/util.h" -#include "http/http_client.h" -#include "plugin/plugin_zip.h" -#include "util/dynamic_util.h" -#include "util/file_utils.h" -#include "util/md5.h" -#include "util/time.h" - -namespace doris { - -using namespace strings; - -static const std::string PLUGIN_VERSION_SYMBOL = "_plugin_interface_version"; -static const std::string PLUGIN_SIZE_SYMBOL = "_sizeof_plugin"; -static const std::string PLUGIN_STRUCT_SYMBOL = "_plugin"; - -Status PluginLoader::open_valid() { - return Status::OK(); -} - -Status PluginLoader::close_valid() { - if (_plugin.get() != nullptr && (_plugin->flags & PLUGIN_NOT_DYNAMIC_UNINSTALL)) { - return Status::InternalError( - strings::Substitute("plugin $0 not allow dynamic uninstall", _name)); - } - - return Status::OK(); -} - -Status DynamicPluginLoader::install() { - // check already install - std::string so_path = _install_path + "/" + _name + "/" + _so_name; - if (!FileUtils::check_exist(so_path)) { - // no, need download zip install - PluginZip zip(_source); - - RETURN_NOT_OK_STATUS_WITH_WARN(zip.extract(_install_path, _name), "plugin install failed"); - } - - // open plugin - RETURN_NOT_OK_STATUS_WITH_WARN(open_plugin(), "plugin install failed"); - - RETURN_NOT_OK_STATUS_WITH_WARN(open_valid(), "plugin install failed"); - - // plugin init - // todo: what should be send? - if (_plugin->init != nullptr) { - _plugin->init(&_plugin->handler); - } - - return Status::OK(); -} - -/** - * open & valid Plugin: - * 1. check .so file exists - * 2. check .so version symbol - * 3. check .so plugin symbol - */ -Status DynamicPluginLoader::open_plugin() { - // check .so file - std::string so_path = _install_path + "/" + _name + "/" + _so_name; - if (!FileUtils::check_exist(so_path)) { - return Status::InternalError("plugin install not found " + _so_name); - } - - RETURN_IF_ERROR(dynamic_open(so_path.c_str(), &_plugin_handler)); - - void* symbol; - // check version symbol - RETURN_IF_ERROR( - dynamic_lookup(_plugin_handler, (_name + PLUGIN_VERSION_SYMBOL).c_str(), &symbol)); - - if (DORIS_PLUGIN_VERSION > *(int*)symbol) { - return Status::InternalError("plugin compile version too old"); - } - - RETURN_IF_ERROR(dynamic_lookup(_plugin_handler, (_name + PLUGIN_SIZE_SYMBOL).c_str(), &symbol)); - - int plugin_size = *(int*)symbol; - if (plugin_size != sizeof(Plugin)) { - return Status::InternalError("plugin struct error"); - } - - // check Plugin declaration - RETURN_IF_ERROR( - dynamic_lookup(_plugin_handler, (_name + PLUGIN_STRUCT_SYMBOL).c_str(), &symbol)); - - Plugin* end_plugin = (Plugin*)((char*)symbol + plugin_size); - - if (end_plugin->handler != nullptr || end_plugin->init != nullptr || - end_plugin->close != nullptr) { - return Status::InternalError("plugin struct error"); - } - - _plugin = std::make_shared(); - std::memcpy(_plugin.get(), symbol, plugin_size); - - return Status::OK(); -} - -Status DynamicPluginLoader::uninstall() { - // close plugin - RETURN_IF_ERROR(close_plugin()); - - // remove plugin install path - RETURN_IF_ERROR(FileUtils::remove_all(_install_path + "/" + _name)); - - return Status::OK(); -} - -Status DynamicPluginLoader::close_plugin() { - if (_close) { - return Status::OK(); - } - - if (_plugin.get() != nullptr) { - RETURN_IF_ERROR(close_valid()); - - if (_plugin->close != nullptr) { - // todo: what should be send? - _plugin->close(&_plugin->handler); - } - } - - // builtin plugin don't need dynamic uninstall - if (_plugin_handler != nullptr) { - dynamic_close(_plugin_handler); - } - - _close = true; - return Status::OK(); -} - -BuiltinPluginLoader::BuiltinPluginLoader(const std::string& name, int type, - const doris::Plugin* plugin) - : PluginLoader(name, type) { - _plugin = std::make_shared(); - std::memcpy(_plugin.get(), plugin, sizeof(Plugin)); -} - -Status BuiltinPluginLoader::install() { - RETURN_IF_ERROR(open_valid()); - LOG(INFO) << "plugin: " << _plugin.get(); - - if (_plugin->init != nullptr) { - _plugin->init(&_plugin->handler); - } - - return Status::OK(); -} - -Status BuiltinPluginLoader::uninstall() { - if (_close) { - return Status::OK(); - } - - if (_plugin.get() != nullptr) { - RETURN_IF_ERROR(close_valid()); - - if (_plugin->close != nullptr) { - // todo: what should be send? - _plugin->close(&_plugin->handler); - } - - _plugin.reset(); - } - - _close = true; - return Status::OK(); -} - -} // namespace doris diff --git a/be/src/plugin/plugin_loader.h b/be/src/plugin/plugin_loader.h deleted file mode 100644 index 3a37f45dd4d496..00000000000000 --- a/be/src/plugin/plugin_loader.h +++ /dev/null @@ -1,108 +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_PLUGIN_PLUGIN_LOADER_H -#define DORIS_BE_PLUGIN_PLUGIN_LOADER_H - -#include -#include -#include - -#include "common/status.h" -#include "gen_cpp/Types_types.h" -#include "plugin/plugin.h" - -namespace doris { - -class PluginLoader { -public: - PluginLoader(const std::string& name, int type) : _name(name), _type(type), _close(false) {} - - virtual ~PluginLoader(){}; - - virtual Status install() = 0; - - virtual Status uninstall() = 0; - - virtual std::shared_ptr& plugin() { return _plugin; }; - - const std::string& name() { return _name; } - - int type() { return _type; } - -protected: - virtual Status open_valid(); - - virtual Status close_valid(); - -protected: - std::string _name; - - int _type; - - std::shared_ptr _plugin; - - bool _close; -}; - -class DynamicPluginLoader : public PluginLoader { -public: - DynamicPluginLoader(const std::string& name, int type, const std::string& source, - const std::string& so_name, const std::string& install_path) - : PluginLoader(name, type), - _source(source), - _so_name(so_name), - _install_path(install_path), - _plugin_handler(nullptr){}; - - virtual ~DynamicPluginLoader() { - // just close plugin, but don't clean install path (maybe other plugin has used) - WARN_IF_ERROR(close_plugin(), "close plugin failed."); - }; - - virtual Status install(); - - virtual Status uninstall(); - -private: - Status open_plugin(); - - Status close_plugin(); - -private: - std::string _source; - - std::string _so_name; - - std::string _install_path; - - void* _plugin_handler; -}; - -class BuiltinPluginLoader : public PluginLoader { -public: - BuiltinPluginLoader(const std::string& name, int type, const Plugin* plugin); - - virtual ~BuiltinPluginLoader() { WARN_IF_ERROR(uninstall(), "close plugin failed."); } - - virtual Status install(); - - virtual Status uninstall(); -}; - -} // namespace doris -#endif //DORIS_BE_PLUGIN_PLUGIN_LOADER_H diff --git a/be/src/plugin/plugin_mgr.cpp b/be/src/plugin/plugin_mgr.cpp deleted file mode 100644 index 5cfee90626754b..00000000000000 --- a/be/src/plugin/plugin_mgr.cpp +++ /dev/null @@ -1,165 +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 "plugin/plugin_mgr.h" - -#include "common/config.h" -#include "gutil/strings/substitute.h" - -namespace doris { - -using namespace strings; - -#define PLUGIN_TYPE_CHECK(_type) \ - { \ - if (_type >= PLUGIN_TYPE_MAX) { \ - return Status::InvalidArgument(strings::Substitute("error plugin type: $0", _type)); \ - } \ - } - -Status PluginMgr::install_plugin(const TPluginMetaInfo& info) { - { - std::lock_guard l(_lock); - auto iter = _plugins[info.type].find(info.name); - - if (iter != _plugins[info.type].end()) { - return Status::AlreadyExist("plugin " + info.name + " is already installed"); - } - } - - DCHECK(info.__isset.so_name); - DCHECK(info.__isset.source); - - std::unique_ptr loader = std::unique_ptr(new DynamicPluginLoader( - info.name, info.type, info.source, info.so_name, config::plugin_path)); - - Status st = loader->install(); - - if (!st.ok() && !st.is_already_exist()) { - RETURN_IF_ERROR(loader->uninstall()); - return st; - } - - { - std::lock_guard l(_lock); - auto iter = _plugins[info.type].find(info.name); - - if (iter != _plugins[info.type].end()) { - return Status::AlreadyExist("plugin " + info.name + " is already installed"); - } else { - _plugins[info.type][info.name] = std::move(loader); - }; - } - - return Status::OK(); -} - -Status PluginMgr::uninstall_plugin(const TPluginMetaInfo& info) { - std::lock_guard l(_lock); - - auto iter = _plugins[info.type].find(info.name); - - if (iter != _plugins[info.type].end()) { - _plugins[info.type].erase(iter); - } - - return Status::OK(); -} - -Status PluginMgr::get_plugin(const std::string& name, int type, std::shared_ptr* plugin) { - PLUGIN_TYPE_CHECK(type); - - std::lock_guard l(_lock); - - auto iter = _plugins[type].find(name); - - if (iter != _plugins[type].end()) { - *plugin = iter->second->plugin(); - return Status::OK(); - } - - return Status::NotFound(strings::Substitute("not found type $0 plugin $1", type, name)); -} - -Status PluginMgr::get_plugin(const std::string& name, std::shared_ptr* plugin) { - for (int i = 0; i < PLUGIN_TYPE_MAX; ++i) { - std::lock_guard l(_lock); - - auto iter = _plugins[i].find(name); - - if (iter != _plugins[i].end()) { - *plugin = iter->second->plugin(); - return Status::OK(); - } - } - - return Status::NotFound(strings::Substitute("not found plugin $0", name)); -} - -Status PluginMgr::get_plugin_list(int type, std::vector>* plugin_list) { - PLUGIN_TYPE_CHECK(type); - - std::lock_guard l(_lock); - - for (const PluginLoaderMap::value_type& iter : _plugins[type]) { - plugin_list->push_back(iter.second->plugin()); - } - - return Status::OK(); -} - -Status PluginMgr::register_builtin_plugin(const std::string& name, int type, - const doris::Plugin* plugin) { - PLUGIN_TYPE_CHECK(type); - - std::lock_guard l(_lock); - - auto iter = _plugins[type].find(name); - if (iter != _plugins[type].end()) { - return Status::AlreadyExist( - strings::Substitute("the type $0 plugin $1 already register", type, name)); - } - - std::unique_ptr loader = - std::unique_ptr(new BuiltinPluginLoader(name, type, plugin)); - - Status st = loader->install(); - if (!st.ok()) { - RETURN_IF_ERROR(loader->uninstall()); - return st; - } - - _plugins[type][name] = std::move(loader); - - return Status::OK(); -} - -Status PluginMgr::get_all_plugin_info(std::vector* plugin_info_list) { - for (int i = 0; i < PLUGIN_TYPE_MAX; ++i) { - for (const PluginLoaderMap::value_type& iter : _plugins[i]) { - TPluginInfo info; - info.__set_plugin_name(iter.second->name()); - info.__set_type(iter.second->type()); - - plugin_info_list->push_back(info); - } - } - - return Status::OK(); -} - -} // namespace doris diff --git a/be/src/plugin/plugin_mgr.h b/be/src/plugin/plugin_mgr.h deleted file mode 100644 index 44c5eb756c0b3b..00000000000000 --- a/be/src/plugin/plugin_mgr.h +++ /dev/null @@ -1,64 +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_PLUGIN_PLUGIN_MGR_H -#define DORIS_BE_PLUGIN_PLUGIN_MGR_H - -#include -#include -#include -#include - -#include "common/status.h" -#include "gen_cpp/AgentService_types.h" -#include "gen_cpp/MasterService_types.h" -#include "plugin/plugin.h" -#include "plugin/plugin_loader.h" - -namespace doris { - -typedef std::unordered_map> PluginLoaderMap; - -class PluginMgr { -public: - PluginMgr() {} - - ~PluginMgr() {} - - Status install_plugin(const TPluginMetaInfo& info); - - Status uninstall_plugin(const TPluginMetaInfo& info); - - Status register_builtin_plugin(const std::string& name, int type, const Plugin* plugin); - - Status get_plugin(const std::string& name, int type, std::shared_ptr* plugin); - - Status get_plugin(const std::string& name, std::shared_ptr* plugin); - - Status get_plugin_list(int type, std::vector>* plugin_list); - - Status get_all_plugin_info(std::vector* plugin_info_list); - -private: - PluginLoaderMap _plugins[PLUGIN_TYPE_MAX]; - - std::mutex _lock; -}; - -} // namespace doris - -#endif // DORIS_BE_PLUGIN_PLUGIN_LOADER_H diff --git a/be/src/plugin/plugin_zip.cpp b/be/src/plugin/plugin_zip.cpp deleted file mode 100644 index 84345812897def..00000000000000 --- a/be/src/plugin/plugin_zip.cpp +++ /dev/null @@ -1,132 +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 "plugin/plugin_zip.h" - -#include - -#include - -#include "env/env.h" -#include "gutil/strings/substitute.h" -#include "gutil/strings/util.h" -#include "http/http_client.h" -#include "util/file_utils.h" -#include "util/md5.h" -#include "util/slice.h" -#include "util/time.h" -#include "util/zip_util.h" - -namespace doris { - -using namespace strings; - -bool is_local_source(const std::string& source) { - if (HasPrefixString(source, "http") || HasPrefixString(source, "https")) { - return false; - } - - return true; -} - -PluginZip::~PluginZip() { - for (auto& p : _clean_paths) { - WARN_IF_ERROR(FileUtils::remove_all(p), "clean plugin_zip temp path failed: " + p); - } -} - -Status PluginZip::extract(const std::string& target_dir, const std::string& plugin_name) { - // check plugin install path - std::string plugin_install_path = strings::Substitute("$0/$1", target_dir, plugin_name); - - if (FileUtils::check_exist(plugin_install_path)) { - return Status::AlreadyExist(strings::Substitute("plugin $0 already install!", plugin_name)); - } - - if (!FileUtils::check_exist(target_dir)) { - RETURN_IF_ERROR(FileUtils::create_dir(target_dir)); - } - - std::string zip_path = _source; - if (!is_local_source(_source)) { - zip_path = strings::Substitute("$0/.temp_$1_$2.zip", target_dir, GetCurrentTimeMicros(), - plugin_name); - _clean_paths.push_back(zip_path); - - RETURN_IF_ERROR(download(zip_path)); - } - - // zip extract - ZipFile zip_file(zip_path); - RETURN_IF_ERROR(zip_file.extract(target_dir, plugin_name)); - - return Status::OK(); -} - -Status PluginZip::download(const std::string& zip_path) { - // download .zip - Status status; - HttpClient client; - Md5Digest digest; - - std::unique_ptr file; - - RETURN_IF_ERROR(Env::Default()->new_writable_file(zip_path, &file)); - RETURN_IF_ERROR(client.init(_source)); - - auto download_cb = [&status, &digest, &file](const void* data, size_t length) { - digest.update(data, length); - - Slice slice((const char*)data, length); - status = file->append(slice); - if (!status.ok()) { - LOG(WARNING) << "fail to download data, file: " << file->filename() - << ", error: " << status.to_string(); - return false; - } - - return true; - }; - - RETURN_IF_ERROR(client.execute(download_cb)); - RETURN_IF_ERROR(status); - RETURN_IF_ERROR(file->flush(WritableFile::FLUSH_ASYNC)); - RETURN_IF_ERROR(file->sync()); - RETURN_IF_ERROR(file->close()); - - // md5 check - HttpClient md5_client; - RETURN_IF_ERROR(md5_client.init(_source + ".md5")); - - std::string expect; - auto download_md5_cb = [&status, &expect](const void* data, size_t length) { - expect = std::string((const char*)data, length); - return true; - }; - - RETURN_IF_ERROR(md5_client.execute(download_md5_cb)); - - digest.digest(); - if (0 != strncmp(digest.hex().c_str(), expect.c_str(), 32)) { - return Status::InternalError(strings::Substitute( - "plugin install checksum failed. expect: $0, actual:$1", expect, digest.hex())); - } - - return Status::OK(); -} - -} // namespace doris \ No newline at end of file diff --git a/be/src/plugin/plugin_zip.h b/be/src/plugin/plugin_zip.h deleted file mode 100644 index ea5af8497c0b8c..00000000000000 --- a/be/src/plugin/plugin_zip.h +++ /dev/null @@ -1,43 +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_PLUGIN_PLUGIN_ZIP_H -#define DORIS_BE_PLUGIN_PLUGIN_ZIP_H - -#include "common/status.h" - -namespace doris { - -class PluginZip { -public: - PluginZip(std::string source) : _source(source) {} - - ~PluginZip(); - - Status extract(const std::string& target_path, const std::string& plugin_name); - -private: - Status download(const std::string& zip_path); - -private: - std::string _source; - - std::vector _clean_paths; -}; - -} // namespace doris -#endif //DORIS_BE_PLUGIN_PLUGIN_ZIP_H 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/collection_value.cpp b/be/src/runtime/collection_value.cpp index 0729646e1350d9..0daa1903b5de14 100644 --- a/be/src/runtime/collection_value.cpp +++ b/be/src/runtime/collection_value.cpp @@ -23,6 +23,10 @@ namespace doris { int sizeof_type(PrimitiveType type) { switch (type) { + case TYPE_TINYINT: + return sizeof(int8_t); + case TYPE_SMALLINT: + return sizeof(int16_t); case TYPE_INT: return sizeof(int32_t); case TYPE_CHAR: @@ -40,6 +44,8 @@ int sizeof_type(PrimitiveType type) { Status type_check(PrimitiveType type) { switch (type) { + case TYPE_TINYINT: + case TYPE_SMALLINT: case TYPE_INT: case TYPE_CHAR: case TYPE_VARCHAR: @@ -160,6 +166,12 @@ Status CollectionValue::set(uint32_t i, PrimitiveType type, const AnyVal* value) } switch (type) { + case TYPE_TINYINT: + *reinterpret_cast(iter.value()) = reinterpret_cast(value)->val; + break; + case TYPE_SMALLINT: + *reinterpret_cast(iter.value()) = reinterpret_cast(value)->val; + break; case TYPE_INT: *reinterpret_cast(iter.value()) = reinterpret_cast(value)->val; break; @@ -214,7 +226,7 @@ void ArrayIterator::value(AnyVal* dest) { break; case TYPE_SMALLINT: - reinterpret_cast(dest)->val = *reinterpret_cast(value()); + reinterpret_cast(dest)->val = *reinterpret_cast(value()); break; case TYPE_INT: 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/exec_env.h b/be/src/runtime/exec_env.h index 8c8a9fbc2bbd70..39808bf8704201 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -58,7 +58,6 @@ class WebPageHandler; class StreamLoadExecutor; class RoutineLoadTaskExecutor; class SmallFileMgr; -class PluginMgr; class BackendServiceClient; class FrontendServiceClient; @@ -154,8 +153,6 @@ class ExecEnv { RoutineLoadTaskExecutor* routine_load_task_executor() { return _routine_load_task_executor; } HeartbeatFlags* heartbeat_flags() { return _heartbeat_flags; } - PluginMgr* plugin_mgr() { return _plugin_mgr; } - // The root tracker should be set before calling ExecEnv::init(); void set_root_mem_tracker(std::shared_ptr root_tracker); @@ -228,7 +225,6 @@ class ExecEnv { SmallFileMgr* _small_file_mgr = nullptr; HeartbeatFlags* _heartbeat_flags = nullptr; - PluginMgr* _plugin_mgr = nullptr; }; template <> diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 128f52e6ab93f5..eb29e5eaae3048 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -28,7 +28,6 @@ #include "olap/page_cache.h" #include "olap/segment_loader.h" #include "olap/storage_engine.h" -#include "plugin/plugin_mgr.h" #include "runtime/broker_mgr.h" #include "runtime/bufferpool/buffer_pool.h" #include "runtime/bufferpool/reservation_tracker.h" @@ -130,7 +129,6 @@ Status ExecEnv::_init(const std::vector& store_paths) { _stream_load_executor = new StreamLoadExecutor(this); _routine_load_task_executor = new RoutineLoadTaskExecutor(this); _small_file_mgr = new SmallFileMgr(this, config::small_file_dir); - _plugin_mgr = new PluginMgr(); _backend_client_cache->init_metrics("backend"); _frontend_client_cache->init_metrics("frontend"); diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index b70f5d69174b14..dcff85c2d5a2df 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -790,7 +790,8 @@ Status FragmentMgr::exec_external_plan_fragment(const TScanOpenParams& params, TTabletVersionInfo info = iter->second; scan_range.tablet_id = tablet_id; scan_range.version = std::to_string(info.version); - scan_range.version_hash = std::to_string(info.version_hash); + // Useless but it is required field in TPaloScanRange + scan_range.version_hash = "0"; scan_range.schema_hash = std::to_string(info.schema_hash); scan_range.hosts.push_back(address); } else { diff --git a/be/src/runtime/mysql_table_writer.cpp b/be/src/runtime/mysql_table_writer.cpp index a2ae2af0605ac5..90ec03fed646e6 100644 --- a/be/src/runtime/mysql_table_writer.cpp +++ b/be/src/runtime/mysql_table_writer.cpp @@ -26,6 +26,12 @@ #include "runtime/row_batch.h" #include "runtime/tuple_row.h" #include "util/types.h" +#include "vec/columns/column_nullable.h" +#include "vec/core/block.h" +#include "vec/data_types/data_type.h" +#include "vec/exprs/vexpr.h" +#include "vec/exprs/vexpr_context.h" +#include "vec/core/materialize_block.h" namespace doris { @@ -38,7 +44,10 @@ std::string MysqlConnInfo::debug_string() const { } MysqlTableWriter::MysqlTableWriter(const std::vector& output_expr_ctxs) - : _output_expr_ctxs(output_expr_ctxs) {} + : _output_expr_ctxs(output_expr_ctxs), _vec_output_expr_ctxs {} {} + +MysqlTableWriter::MysqlTableWriter(const std::vector& output_expr_ctxs) + : _output_expr_ctxs {}, _vec_output_expr_ctxs(output_expr_ctxs) {} MysqlTableWriter::~MysqlTableWriter() { if (_mysql_conn) { @@ -182,4 +191,141 @@ Status MysqlTableWriter::append(RowBatch* batch) { return Status::OK(); } +Status MysqlTableWriter::append(vectorized::Block* block) { + Status status = Status::OK(); + if (block == nullptr || block->rows() == 0) { + return status; + } + + auto output_block = vectorized::VExprContext::get_output_block_after_execute_exprs( + _vec_output_expr_ctxs, *block, status); + + auto num_rows = output_block.rows(); + if (UNLIKELY(num_rows == 0)) { + return status; + } + materialize_block_inplace(output_block); + for (int i = 0; i < num_rows; ++i) { + RETURN_IF_ERROR(insert_row(output_block, i)); + } + return Status::OK(); +} + +Status MysqlTableWriter::insert_row(vectorized::Block& block, size_t row) { + _insert_stmt_buffer.clear(); + fmt::format_to(_insert_stmt_buffer, "INSERT INTO {} VALUES (", _mysql_tbl); + int num_columns = _vec_output_expr_ctxs.size(); + + for (int i = 0; i < num_columns; ++i) { + auto& column_ptr = block.get_by_position(i).column; + auto& type_ptr = block.get_by_position(i).type; + + if (i != 0) { + fmt::format_to(_insert_stmt_buffer, "{}", ", "); + } + + vectorized::ColumnPtr column; + if (type_ptr->is_nullable()) { + column = assert_cast(*column_ptr) + .get_nested_column_ptr(); + if (column_ptr->is_null_at(row)) { + fmt::format_to(_insert_stmt_buffer, "{}", "NULL"); + continue; + } + } else { + column = column_ptr; + } + + switch (_vec_output_expr_ctxs[i]->root()->result_type()) { + case TYPE_BOOLEAN: { + auto& data = assert_cast(*column).get_data(); + fmt::format_to(_insert_stmt_buffer, "{}", data[row]); + break; + } + case TYPE_TINYINT: { + auto& data = assert_cast(*column).get_data(); + fmt::format_to(_insert_stmt_buffer, "{}", data[row]); + break; + } + case TYPE_SMALLINT: { + auto& data = assert_cast(*column).get_data(); + fmt::format_to(_insert_stmt_buffer, "{}", data[row]); + break; + } + case TYPE_INT: { + auto& data = assert_cast(*column).get_data(); + fmt::format_to(_insert_stmt_buffer, "{}", data[row]); + break; + } + case TYPE_BIGINT: { + auto& data = assert_cast(*column).get_data(); + fmt::format_to(_insert_stmt_buffer, "{}", data[row]); + break; + } + case TYPE_FLOAT: { + auto& data = assert_cast(*column).get_data(); + fmt::format_to(_insert_stmt_buffer, "{}", data[row]); + break; + } + case TYPE_DOUBLE: { + auto& data = assert_cast(*column).get_data(); + fmt::format_to(_insert_stmt_buffer, "{}", data[row]); + break; + } + + case TYPE_STRING: + case TYPE_CHAR: + case TYPE_VARCHAR: { + const auto& string_val = + assert_cast(*column).get_data_at(row); + DCHECK(string_val.data != nullptr); + std::unique_ptr buf(new char[2 * string_val.size + 1]); + mysql_real_escape_string(_mysql_conn, buf.get(), string_val.data, string_val.size); + fmt::format_to(_insert_stmt_buffer, "'{}'", buf.get()); + + break; + } + case TYPE_DECIMALV2: { + DecimalV2Value value = + (DecimalV2Value) + assert_cast&>( + *column) + .get_data()[row]; + fmt::format_to(_insert_stmt_buffer, "{}", value.to_string()); + break; + } + case TYPE_DATE: + case TYPE_DATETIME: { + int64_t int_val = assert_cast(*column).get_data()[row]; + vectorized::VecDateTimeValue value = + binary_cast(int_val); + + char buf[64]; + char* pos = value.to_string(buf); + std::string str(buf, pos - buf - 1); + fmt::format_to(_insert_stmt_buffer, "'{}'", str); + break; + } + default: { + fmt::memory_buffer err_out; + fmt::format_to(err_out, "can't convert this type to mysql type. type = {}", + _vec_output_expr_ctxs[i]->root()->type().type); + return Status::InternalError(err_out.data()); + } + } + } + + fmt::format_to(_insert_stmt_buffer, "{}", ")"); + + // Insert this to MySQL server + if (mysql_real_query(_mysql_conn, _insert_stmt_buffer.data(), _insert_stmt_buffer.size())) { + fmt::memory_buffer err_ss; + fmt::format_to(err_ss, "Insert to mysql server({}) failed, because: {}.", + mysql_get_host_info(_mysql_conn), mysql_error(_mysql_conn)); + return Status::InternalError(err_ss.data()); + } + + return Status::OK(); +} + } // namespace doris diff --git a/be/src/runtime/mysql_table_writer.h b/be/src/runtime/mysql_table_writer.h index 8a1d04595d7233..770910cc45daf5 100644 --- a/be/src/runtime/mysql_table_writer.h +++ b/be/src/runtime/mysql_table_writer.h @@ -18,11 +18,11 @@ #ifndef DORIS_BE_RUNTIME_MYSQL_TABLE_WRITER_H #define DORIS_BE_RUNTIME_MYSQL_TABLE_WRITER_H +#include #include #include #include "common/status.h" - #ifndef __DorisMysql #define __DorisMysql void #endif @@ -42,10 +42,15 @@ struct MysqlConnInfo { class RowBatch; class TupleRow; class ExprContext; +namespace vectorized { +class VExprContext; +class Block; +} class MysqlTableWriter { public: MysqlTableWriter(const std::vector& output_exprs); + MysqlTableWriter(const std::vector& output_exprs); ~MysqlTableWriter(); // connect to mysql server @@ -55,6 +60,8 @@ class MysqlTableWriter { Status append(RowBatch* batch); + Status append(vectorized::Block* block); + Status abort_tarns() { return Status::OK(); } Status finish_tarns() { return Status::OK(); } @@ -63,6 +70,10 @@ class MysqlTableWriter { Status insert_row(TupleRow* row); const std::vector& _output_expr_ctxs; + //vectorized mode insert_row + Status insert_row(vectorized::Block& block, size_t row); + const std::vector& _vec_output_expr_ctxs; + fmt::memory_buffer _insert_stmt_buffer; std::string _mysql_tbl; __DorisMysql* _mysql_conn; }; 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.cpp b/be/src/runtime/row_batch.cpp index 139740b9c3786e..e491d27f19286f 100644 --- a/be/src/runtime/row_batch.cpp +++ b/be/src/runtime/row_batch.cpp @@ -188,8 +188,10 @@ RowBatch::RowBatch(const RowDescriptor& row_desc, const PRowBatch& input_batch, // assgin data and null_sign pointer position in tuple_data int data_offset = convert_to(array_val->data()); array_val->set_data(tuple_data + data_offset); - int null_offset = convert_to(array_val->null_signs()); - array_val->set_null_signs(convert_to(tuple_data + null_offset)); + if (array_val->has_null()) { + int null_offset = convert_to(array_val->null_signs()); + array_val->set_null_signs(convert_to(tuple_data + null_offset)); + } const TypeDescriptor& item_type = slot_collection->type().children.at(0); if (!item_type.is_string_type()) { @@ -254,7 +256,10 @@ Status RowBatch::serialize(PRowBatch* output_batch, size_t* uncompressed_size, s // row_tuples _row_desc.to_protobuf(output_batch->mutable_row_tuples()); // tuple_offsets: must clear before reserve + // TODO(cmy): the tuple_offsets should be removed after v1.1.0, use new_tuple_offsets instead. + // keep tuple_offsets here is just for compatibility. output_batch->clear_tuple_offsets(); + output_batch->mutable_tuple_offsets()->Reserve(_num_rows * _num_tuples_per_row); output_batch->clear_new_tuple_offsets(); output_batch->mutable_new_tuple_offsets()->Reserve(_num_rows * _num_tuples_per_row); // is_compressed @@ -279,7 +284,8 @@ Status RowBatch::serialize(PRowBatch* output_batch, size_t* uncompressed_size, s int64_t offset = 0; // current offset into output_batch->tuple_data char* tuple_data = mutable_tuple_data->data(); const auto& tuple_descs = _row_desc.tuple_descriptors(); - const auto& mutable_tuple_offsets = output_batch->mutable_new_tuple_offsets(); + const auto& mutable_tuple_offsets = output_batch->mutable_tuple_offsets(); + const auto& mutable_new_tuple_offsets = output_batch->mutable_new_tuple_offsets(); for (int i = 0; i < _num_rows; ++i) { TupleRow* row = get_row(i); @@ -288,10 +294,12 @@ Status RowBatch::serialize(PRowBatch* output_batch, size_t* uncompressed_size, s if (row->get_tuple(j) == nullptr) { // NULLs are encoded as -1 mutable_tuple_offsets->Add(-1); + mutable_new_tuple_offsets->Add(-1); continue; } // Record offset before creating copy (which increments offset and tuple_data) - mutable_tuple_offsets->Add(offset); + mutable_tuple_offsets->Add((int32_t) offset); + mutable_new_tuple_offsets->Add(offset); row->get_tuple(j)->deep_copy(*desc, &tuple_data, &offset, /* convert_ptrs */ true); CHECK_LE(offset, size); } @@ -519,7 +527,9 @@ vectorized::Block RowBatch::convert_to_vec_block() const { size_t RowBatch::get_batch_size(const PRowBatch& batch) { size_t result = batch.tuple_data().size(); result += batch.row_tuples().size() * sizeof(int32_t); + // TODO(cmy): remove batch.tuple_offsets result += batch.tuple_offsets().size() * sizeof(int32_t); + result += batch.new_tuple_offsets().size() * sizeof(int64_t); return result; } @@ -607,7 +617,9 @@ size_t RowBatch::total_byte_size() const { // compute data null_signs size CollectionValue* array_val = tuple->get_collection_slot(slot_collection->tuple_offset()); - result += array_val->length() * sizeof(bool); + if (array_val->has_null()) { + result += array_val->length() * sizeof(bool); + } const TypeDescriptor& item_type = slot_collection->type().children.at(0); result += array_val->length() * item_type.get_slot_size(); 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/snapshot_loader.cpp b/be/src/runtime/snapshot_loader.cpp index 2309352a62914a..eea8b399de3f2f 100644 --- a/be/src/runtime/snapshot_loader.cpp +++ b/be/src/runtime/snapshot_loader.cpp @@ -556,25 +556,6 @@ Status SnapshotLoader::_get_existing_files_from_local(const std::string& local_p return Status::OK(); } -void SnapshotLoader::_assemble_file_name(const std::string& snapshot_path, - const std::string& tablet_path, int64_t tablet_id, - int64_t start_version, int64_t end_version, - int64_t vesion_hash, int32_t seg_num, - const std::string suffix, std::string* snapshot_file, - std::string* tablet_file) { - std::stringstream ss1; - ss1 << snapshot_path << "/" << tablet_id << "_" << start_version << "_" << end_version << "_" - << vesion_hash << "_" << seg_num << suffix; - *snapshot_file = ss1.str(); - - std::stringstream ss2; - ss2 << tablet_path << "/" << tablet_id << "_" << start_version << "_" << end_version << "_" - << vesion_hash << "_" << seg_num << suffix; - *tablet_file = ss2.str(); - - VLOG_CRITICAL << "assemble file name: " << *snapshot_file << ", " << *tablet_file; -} - Status SnapshotLoader::_replace_tablet_id(const std::string& file_name, int64_t tablet_id, std::string* new_file_name) { // eg: diff --git a/be/src/runtime/snapshot_loader.h b/be/src/runtime/snapshot_loader.h index 12acc66542b5cb..0d2386b6e976c8 100644 --- a/be/src/runtime/snapshot_loader.h +++ b/be/src/runtime/snapshot_loader.h @@ -86,11 +86,6 @@ class SnapshotLoader { bool _end_with(const std::string& str, const std::string& match); - void _assemble_file_name(const std::string& snapshot_path, const std::string& tablet_path, - int64_t tablet_id, int64_t start_version, int64_t end_version, - int64_t version_hash, int32_t seg_num, const std::string suffix, - std::string* snapshot_file, std::string* tablet_file); - Status _replace_tablet_id(const std::string& file_name, int64_t tablet_id, std::string* new_file_name); diff --git a/be/src/runtime/string_search.hpp b/be/src/runtime/string_search.hpp index 2525321d55c96e..e35905b3fed30c 100644 --- a/be/src/runtime/string_search.hpp +++ b/be/src/runtime/string_search.hpp @@ -51,10 +51,10 @@ // agrees to be bound by the terms and conditions of this License // Agreement. -#ifndef DORIS_BE_SRC_QUERY_BE_RUNTIME_STRING_SEARCH_H -#define DORIS_BE_SRC_QUERY_BE_RUNTIME_STRING_SEARCH_H +#pragma once #include +#include #include #include "common/logging.h" @@ -62,109 +62,32 @@ namespace doris { -// TODO: This can be sped up with SIDD_CMP_EQUAL_ORDERED or at the very least rewritten -// from published algorithms. class StringSearch { public: virtual ~StringSearch() {} - StringSearch() : _pattern(NULL), _mask(0) {} + StringSearch() : _pattern(nullptr) {} - // Initialize/Precompute a StringSearch object from the pattern - StringSearch(const StringValue* pattern) : _pattern(pattern), _mask(0), _skip(0) { - // Special cases - if (_pattern->len <= 1) { - return; - } - - // Build compressed lookup table - int mlast = _pattern->len - 1; - _skip = mlast - 1; - - for (int i = 0; i < mlast; ++i) { - bloom_add(_pattern->ptr[i]); - - if (_pattern->ptr[i] == _pattern->ptr[mlast]) { - _skip = mlast - i - 1; - } - } - - bloom_add(_pattern->ptr[mlast]); - } + StringSearch(const StringValue* pattern) : _pattern(pattern) {} // search for this pattern in str. // Returns the offset into str if the pattern exists // Returns -1 if the pattern is not found int search(const StringValue* str) const { - // Special cases if (!str || !_pattern || _pattern->len == 0) { return -1; } - int mlast = _pattern->len - 1; - int w = str->len - _pattern->len; - int n = str->len; - int m = _pattern->len; - const char* s = str->ptr; - const char* p = _pattern->ptr; - - // Special case if pattern->len == 1 - if (m == 1) { - const char* result = reinterpret_cast(memchr(s, p[0], n)); - - if (result != NULL) { - return result - s; - } - + auto it = std::search(str->ptr, str->ptr + str->len, + std::default_searcher(_pattern->ptr, _pattern->ptr + _pattern->len)); + if (it == str->ptr + str->len) { return -1; + } else { + return it - str->ptr; } - - // General case. - int j; - // TODO: the original code seems to have an off by one error. It is possible - // to index at w + m which is the length of the input string. Checks have - // been added to make sure that w + m < str->len. - for (int i = 0; i <= w; i++) { - // note: using mlast in the skip path slows things down on x86 - if (s[i + m - 1] == p[m - 1]) { - // candidate match - for (j = 0; j < mlast; j++) - if (s[i + j] != p[j]) { - break; - } - - if (j == mlast) { - return i; - } - - // miss: check if next character is part of pattern - if (i + m < n && !bloom_query(s[i + m])) { - i = i + m; - } else { - i = i + _skip; - } - } else { - // skip: check if next character is part of pattern - if (i + m < n && !bloom_query(s[i + m])) { - i = i + m; - } - } - } - - return -1; } private: - static const int BLOOM_WIDTH = 64; - - void bloom_add(char c) { _mask |= (1UL << (c & (BLOOM_WIDTH - 1))); } - - bool bloom_query(char c) const { return _mask & (1UL << (c & (BLOOM_WIDTH - 1))); } - const StringValue* _pattern; - int64_t _mask; - int64_t _skip; }; } // namespace doris - -#endif 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/tuple.cpp b/be/src/runtime/tuple.cpp index af552611b9bd48..75ae9ceb8b2140 100644 --- a/be/src/runtime/tuple.cpp +++ b/be/src/runtime/tuple.cpp @@ -99,25 +99,20 @@ void Tuple::deep_copy(Tuple* dst, const TupleDescriptor& desc, MemPool* pool, bo const TypeDescriptor& item_type = slot_desc->type().children.at(0); int coll_byte_size = cv->length() * item_type.get_slot_size(); - int nulls_size = cv->length() * sizeof(bool); + int nulls_size = cv->has_null() ? cv->length() * sizeof(bool) : 0; int64_t offset = pool->total_allocated_bytes(); char* coll_data = (char*)(pool->allocate(coll_byte_size + nulls_size)); // copy data and null_signs - if (nulls_size > 0) { - cv->set_has_null(true); - cv->set_null_signs(convert_to(coll_data) + coll_byte_size); - memory_copy(coll_data, cv->null_signs(), nulls_size); - } else { - cv->set_has_null(false); - } + memory_copy(convert_to(coll_data), cv->null_signs(), nulls_size); memory_copy(coll_data + nulls_size, cv->data(), coll_byte_size); // assgin new null_sign and data location - cv->set_null_signs(convert_ptrs ? convert_to(offset) : convert_to(coll_data)); - cv->set_data(convert_ptrs ? convert_to(offset + nulls_size) - : coll_data + nulls_size); + if (cv->has_null()) { + cv->set_null_signs(convert_ptrs ? convert_to(offset) : convert_to(coll_data)); + } + cv->set_data(convert_ptrs ? convert_to(offset + nulls_size) : coll_data + nulls_size); if (!item_type.is_string_type()) { continue; @@ -212,7 +207,7 @@ void Tuple::deep_copy(const TupleDescriptor& desc, char** data, int64_t* offset, const TypeDescriptor& item_type = slot_desc->type().children.at(0); int coll_byte_size = cv->length() * item_type.get_slot_size(); - int nulls_size = cv->length() * sizeof(bool); + int nulls_size = cv->has_null() ? cv->length() * sizeof(bool) : 0; // copy null_sign memory_copy(*data, cv->null_signs(), nulls_size); @@ -220,8 +215,9 @@ void Tuple::deep_copy(const TupleDescriptor& desc, char** data, int64_t* offset, memory_copy(*data + nulls_size, cv->data(), coll_byte_size); if (!item_type.is_string_type()) { - cv->set_null_signs(convert_ptrs ? convert_to(*offset) - : convert_to(*data)); + if (cv->has_null()) { + cv->set_null_signs(convert_ptrs ? convert_to(*offset) : convert_to(*data)); + } cv->set_data(convert_ptrs ? convert_to(*offset + nulls_size) : *data + nulls_size); *data += coll_byte_size + nulls_size; @@ -250,8 +246,9 @@ void Tuple::deep_copy(const TupleDescriptor& desc, char** data, int64_t* offset, } } // assgin new null_sign and data location - cv->set_null_signs(convert_ptrs ? convert_to(base_offset) - : convert_to(base_data)); + if (cv->has_null()) { + cv->set_null_signs(convert_ptrs ? convert_to(base_offset) : convert_to(base_data)); + } cv->set_data(convert_ptrs ? convert_to(base_offset + nulls_size) : base_data + nulls_size); } 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/CMakeLists.txt b/be/src/vec/CMakeLists.txt index 6201c67a4425b2..c526cf221450ef 100644 --- a/be/src/vec/CMakeLists.txt +++ b/be/src/vec/CMakeLists.txt @@ -32,6 +32,7 @@ set(VEC_FILES aggregate_functions/aggregate_function_reader.cpp aggregate_functions/aggregate_function_window.cpp aggregate_functions/aggregate_function_stddev.cpp + aggregate_functions/aggregate_function_topn.cpp aggregate_functions/aggregate_function_simple_factory.cpp columns/collator.cpp columns/column.cpp @@ -149,6 +150,7 @@ set(VEC_FILES sink/result_sink.cpp sink/vdata_stream_sender.cpp sink/vtablet_sink.cpp + sink/vmysql_table_writer.cpp runtime/vdatetime_value.cpp runtime/vdata_stream_recvr.cpp runtime/vdata_stream_mgr.cpp diff --git a/be/src/vec/aggregate_functions/aggregate_function.h b/be/src/vec/aggregate_functions/aggregate_function.h index 4c2ef36d7bb72c..c3b507245cbdfd 100644 --- a/be/src/vec/aggregate_functions/aggregate_function.h +++ b/be/src/vec/aggregate_functions/aggregate_function.h @@ -20,13 +20,6 @@ #pragma once -#include -#include -#include -#include -#include -#include - #include "vec/common/exception.h" #include "vec/core/block.h" #include "vec/core/column_numbers.h" @@ -95,13 +88,15 @@ class IAggregateFunction { Arena* arena) const = 0; /// Merges state (on which place points to) with other state of current aggregation function. - virtual void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena* arena) const = 0; + virtual void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, + Arena* arena) const = 0; /// Serializes state (to transmit it over the network, for example). virtual void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const = 0; /// Deserializes state. This function is called only for empty (just created) states. - virtual void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena* arena) const = 0; + virtual void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, + Arena* arena) const = 0; /// Returns true if a function requires Arena to handle own states (see add(), merge(), deserialize()). virtual bool allocates_memory_in_arena() const { return false; } @@ -114,21 +109,11 @@ class IAggregateFunction { */ virtual bool is_state() const { return false; } - /// if return false, during insert_result_into function, you colud get nullable result column, + /// if return false, during insert_result_into function, you colud get nullable result column, /// so could insert to null value by yourself, rather than by AggregateFunctionNullBase; /// because you maybe be calculate a invalid value, but want to use null replace it; virtual bool insert_to_null_default() const { return true; } - /** The inner loop that uses the function pointer is better than using the virtual function. - * The reason is that in the case of virtual functions GCC 5.1.2 generates code, - * which, at each iteration of the loop, reloads the function address (the offset value in the virtual function table) from memory to the register. - * This gives a performance drop on simple queries around 12%. - * After the appearance of better compilers, the code can be removed. - */ - using AddFunc = void (*)(const IAggregateFunction*, AggregateDataPtr, const IColumn**, size_t, - Arena*); - virtual AddFunc get_address_of_add_function() const = 0; - /** Contains a loop with calls to "add" function. You can collect arguments into array "places" * and do a single call to "add_batch" for devirtualization and inlining. */ @@ -150,12 +135,6 @@ class IAggregateFunction { AggregateDataPtr place, const IColumn** columns, Arena* arena) const = 0; - /** This is used for runtime code generation to determine, which header files to include in generated source. - * Always implement it as - * const char * get_header_file_path() const override { return __FILE__; } - */ - virtual const char* get_header_file_path() const = 0; - const DataTypes& get_argument_types() const { return argument_types; } const Array& get_parameters() const { return parameters; } @@ -167,18 +146,10 @@ class IAggregateFunction { /// Implement method to obtain an address of 'add' function. template class IAggregateFunctionHelper : public IAggregateFunction { -private: - static void add_free(const IAggregateFunction* that, AggregateDataPtr place, - const IColumn** columns, size_t row_num, Arena* arena) { - static_cast(*that).add(place, columns, row_num, arena); - } - public: IAggregateFunctionHelper(const DataTypes& argument_types_, const Array& parameters_) : IAggregateFunction(argument_types_, parameters_) {} - AddFunc get_address_of_add_function() const override { return &add_free; } - void add_batch(size_t batch_size, AggregateDataPtr* places, size_t place_offset, const IColumn** columns, Arena* arena) const override { for (size_t i = 0; i < batch_size; ++i) diff --git a/be/src/vec/aggregate_functions/aggregate_function_avg.h b/be/src/vec/aggregate_functions/aggregate_function_avg.h index 18584ee91a3c21..7b40f9552a91b9 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_avg.h +++ b/be/src/vec/aggregate_functions/aggregate_function_avg.h @@ -101,7 +101,8 @@ class AggregateFunctionAvg final this->data(place).count = 0; } - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, + Arena*) const override { this->data(place).sum += this->data(rhs).sum; this->data(place).count += this->data(rhs).count; } @@ -110,7 +111,8 @@ class AggregateFunctionAvg final this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, + Arena*) const override { this->data(place).read(buf); } @@ -119,8 +121,6 @@ class AggregateFunctionAvg final column.get_data().push_back(this->data(place).template result()); } - const char* get_header_file_path() const override { return __FILE__; } - private: UInt32 scale; }; diff --git a/be/src/vec/aggregate_functions/aggregate_function_bitmap.h b/be/src/vec/aggregate_functions/aggregate_function_bitmap.h index a2e43e5392caf2..4d72f070bbf574 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_bitmap.h +++ b/be/src/vec/aggregate_functions/aggregate_function_bitmap.h @@ -16,8 +16,6 @@ // under the License. #pragma once -#include -#include #include "vec/aggregate_functions/aggregate_function.h" #include "vec/columns/column_complex.h" @@ -91,7 +89,8 @@ class AggregateFunctionBitmapOp final this->data(place).add(column.get_data()[row_num]); } - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, + Arena*) const override { this->data(place).merge( const_cast&>(this->data(rhs)).get()); } @@ -100,7 +99,8 @@ class AggregateFunctionBitmapOp final this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, + Arena*) const override { this->data(place).read(buf); } @@ -109,8 +109,6 @@ class AggregateFunctionBitmapOp final column.get_data().push_back( const_cast&>(this->data(place)).get()); } - - const char* get_header_file_path() const override { return __FILE__; } }; template @@ -146,7 +144,8 @@ class AggregateFunctionBitmapCount final } } - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, + Arena*) const override { this->data(place).merge(const_cast(this->data(rhs)).get()); } @@ -154,7 +153,8 @@ class AggregateFunctionBitmapCount final this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, + Arena*) const override { this->data(place).read(buf); } @@ -163,8 +163,6 @@ class AggregateFunctionBitmapCount final auto& column = static_cast(to); column.get_data().push_back(value_data.cardinality()); } - - const char* get_header_file_path() const override { return __FILE__; } }; AggregateFunctionPtr create_aggregate_function_bitmap_union(const std::string& name, diff --git a/be/src/vec/aggregate_functions/aggregate_function_count.h b/be/src/vec/aggregate_functions/aggregate_function_count.h index fd096dc6bed435..3d8ab796e19ba7 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_count.h +++ b/be/src/vec/aggregate_functions/aggregate_function_count.h @@ -50,11 +50,10 @@ class AggregateFunctionCount final ++data(place).count; } - void reset(AggregateDataPtr place) const override { - this->data(place).count = 0; - } + void reset(AggregateDataPtr place) const override { this->data(place).count = 0; } - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, + Arena*) const override { data(place).count += data(rhs).count; } @@ -62,15 +61,14 @@ class AggregateFunctionCount final write_var_uint(data(place).count, buf); } - void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, + Arena*) const override { read_var_uint(data(place).count, buf); } void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { assert_cast(to).get_data().push_back(data(place).count); } - - const char* get_header_file_path() const override { return __FILE__; } }; /// Simply count number of not-NULL values. @@ -90,11 +88,10 @@ class AggregateFunctionCountNotNullUnary final data(place).count += !assert_cast(*columns[0]).is_null_at(row_num); } - void reset(AggregateDataPtr place) const override { - data(place).count = 0; - } + void reset(AggregateDataPtr place) const override { data(place).count = 0; } - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, + Arena*) const override { data(place).count += data(rhs).count; } @@ -102,21 +99,22 @@ class AggregateFunctionCountNotNullUnary final write_var_uint(data(place).count, buf); } - void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, + Arena*) const override { read_var_uint(data(place).count, buf); } void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { if (to.is_nullable()) { - auto& null_column = assert_cast(to); + auto& null_column = assert_cast(to); null_column.get_null_map_data().push_back(0); - assert_cast(null_column.get_nested_column()).get_data().push_back(data(place).count); + assert_cast(null_column.get_nested_column()) + .get_data() + .push_back(data(place).count); } else { - assert_cast(to).get_data().push_back(data(place).count); + assert_cast(to).get_data().push_back(data(place).count); } } - - const char* get_header_file_path() const override { return __FILE__; } }; } // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_distinct.h b/be/src/vec/aggregate_functions/aggregate_function_distinct.h index e100cf3ca5577f..81f33bddbc71ea 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_distinct.h +++ b/be/src/vec/aggregate_functions/aggregate_function_distinct.h @@ -69,13 +69,12 @@ struct AggregateFunctionDistinctGenericData { Set::LookupResult it; bool inserted; for (const auto& elem : rhs.set) - set.emplace(ArenaKeyHolder{elem.get_value(), *arena}, it, inserted); + set.emplace(ArenaKeyHolder {elem.get_value(), *arena}, it, inserted); } void serialize(BufferWritable& buf) const { write_var_uint(set.size(), buf); - for (const auto& elem : set) - write_string_binary(elem.get_value(), buf); + for (const auto& elem : set) write_string_binary(elem.get_value(), buf); } void deserialize(BufferReadable& buf, Arena* arena) { @@ -121,7 +120,7 @@ struct AggregateFunctionDistinctMultipleGenericData : public AggregateFunctionDi Set::LookupResult it; bool inserted; - auto key_holder = SerializedKeyHolder{value, *arena}; + auto key_holder = SerializedKeyHolder {value, *arena}; set.emplace(key_holder, it, inserted); } @@ -180,7 +179,8 @@ class AggregateFunctionDistinct this->data(place).serialize(buf); } - void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena* arena) const override { + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, + Arena* arena) const override { this->data(place).deserialize(buf, arena); } @@ -219,8 +219,6 @@ class AggregateFunctionDistinct DataTypePtr get_return_type() const override { return nested_func->get_return_type(); } bool allocates_memory_in_arena() const override { return true; } - - const char* get_header_file_path() const override { return __FILE__; } }; } // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h b/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h index f71a1f5596fceb..612b552946e064 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h +++ b/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h @@ -17,10 +17,6 @@ #pragma once -#include -#include -#include - #include "exprs/hll_function.h" #include "olap/hll.h" #include "util/slice.h" @@ -86,7 +82,8 @@ class AggregateFunctionHLLUnionAgg this->data(place).add(column.get_data_at(row_num)); } - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, + Arena*) const override { this->data(place).merge(this->data(rhs)); } @@ -94,16 +91,16 @@ class AggregateFunctionHLLUnionAgg this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, + Arena*) const override { this->data(place).read(buf); } - virtual void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { + virtual void insert_result_into(ConstAggregateDataPtr __restrict place, + IColumn& to) const override { auto& column = static_cast&>(to); column.get_data().push_back(this->data(place).get_cardinality()); } - - const char* get_header_file_path() const override { return __FILE__; } }; class AggregateFunctionHLLUnion final : public AggregateFunctionHLLUnionAgg { diff --git a/be/src/vec/aggregate_functions/aggregate_function_min_max.h b/be/src/vec/aggregate_functions/aggregate_function_min_max.h index 17d682339a2f71..9c2f0970df2c55 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_min_max.h +++ b/be/src/vec/aggregate_functions/aggregate_function_min_max.h @@ -35,7 +35,8 @@ struct SingleValueDataFixed { private: using Self = SingleValueDataFixed; - bool has_value = false; /// We need to remember if at least one value has been passed. This is necessary for AggregateFunctionIf. + bool has_value = + false; /// We need to remember if at least one value has been passed. This is necessary for AggregateFunctionIf. T value; public: @@ -50,7 +51,7 @@ struct SingleValueDataFixed { void reset() { if (has()) { - has_value = false; + has_value = false; } } @@ -166,10 +167,10 @@ struct SingleValueDataFixed { void reset() { if (has()) { - has_value = false; + has_value = false; } } - + void write(BufferWritable& buf) const { write_binary(has(), buf); if (has()) write_binary(value, buf); @@ -297,13 +298,13 @@ struct SingleValueDataString { void reset() { if (size != -1) { - size = -1; - capacity = 0; + size = -1; + capacity = 0; delete large_data; large_data = nullptr; } } - + void write(BufferWritable& buf) const { write_binary(size, buf); if (has()) buf.write(get_data(), size); @@ -497,11 +498,10 @@ class AggregateFunctionsSingleValue final this->data(place).change_if_better(*columns[0], row_num, arena); } - void reset(AggregateDataPtr place) const override { - this->data(place).reset(); - } + void reset(AggregateDataPtr place) const override { this->data(place).reset(); } - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena* arena) const override { + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, + Arena* arena) const override { this->data(place).change_if_better(this->data(rhs), arena); } @@ -509,7 +509,8 @@ class AggregateFunctionsSingleValue final this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, + Arena*) const override { this->data(place).read(buf); } @@ -518,8 +519,6 @@ class AggregateFunctionsSingleValue final void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { this->data(place).insert_result_into(to); } - - const char* get_header_file_path() const override { return __FILE__; } }; AggregateFunctionPtr create_aggregate_function_max(const std::string& name, diff --git a/be/src/vec/aggregate_functions/aggregate_function_nothing.h b/be/src/vec/aggregate_functions/aggregate_function_nothing.h index 4c7b1933e6bd0f..c0ae740be4eeb9 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_nothing.h +++ b/be/src/vec/aggregate_functions/aggregate_function_nothing.h @@ -54,7 +54,7 @@ class AggregateFunctionNothing final : public IAggregateFunctionHelper { } static void init_flag(AggregateDataPtr __restrict place) noexcept { - if constexpr (result_is_nullable) - place[0] = 0; + if constexpr (result_is_nullable) place[0] = 0; } static void set_flag(AggregateDataPtr __restrict place) noexcept { - if constexpr (result_is_nullable) - place[0] = 1; + if constexpr (result_is_nullable) place[0] = 1; } static bool get_flag(ConstAggregateDataPtr __restrict place) noexcept { @@ -117,7 +115,8 @@ class AggregateFunctionNullBase : public IAggregateFunctionHelper { size_t align_of_data() const override { return nested_function->align_of_data(); } - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena* arena) const override { + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, + Arena* arena) const override { if (result_is_nullable && get_flag(rhs)) set_flag(place); nested_function->merge(nested_place(place), nested_place(rhs), arena); @@ -131,7 +130,8 @@ class AggregateFunctionNullBase : public IAggregateFunctionHelper { } } - void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena* arena) const override { + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, + Arena* arena) const override { bool flag = true; if (result_is_nullable) read_binary(flag, buf); if (flag) { @@ -145,10 +145,12 @@ class AggregateFunctionNullBase : public IAggregateFunctionHelper { ColumnNullable& to_concrete = assert_cast(to); if (get_flag(place)) { if (nested_function->insert_to_null_default()) { - nested_function->insert_result_into(nested_place(place), to_concrete.get_nested_column()); + nested_function->insert_result_into(nested_place(place), + to_concrete.get_nested_column()); to_concrete.get_null_map_data().push_back(0); } else { - nested_function->insert_result_into(nested_place(place), to); //want to insert into null value by self + nested_function->insert_result_into( + nested_place(place), to); //want to insert into null value by self } } else { to_concrete.insert_default(); @@ -163,8 +165,6 @@ class AggregateFunctionNullBase : public IAggregateFunctionHelper { } bool is_state() const override { return nested_function->is_state(); } - - const char* get_header_file_path() const override { return __FILE__; } }; /** There are two cases: for single argument and variadic. diff --git a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp index ba1b2ba98e095a..4844000b3ddf4c 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp @@ -36,6 +36,7 @@ void register_aggregate_function_bitmap(AggregateFunctionSimpleFactory& factory) void register_aggregate_function_window_rank(AggregateFunctionSimpleFactory& factory); void register_aggregate_function_window_lead_lag(AggregateFunctionSimpleFactory& factory); void register_aggregate_function_stddev_variance(AggregateFunctionSimpleFactory& factory); +void register_aggregate_function_topn(AggregateFunctionSimpleFactory& factory); AggregateFunctionSimpleFactory& AggregateFunctionSimpleFactory::instance() { static std::once_flag oc; static AggregateFunctionSimpleFactory instance; @@ -51,10 +52,11 @@ AggregateFunctionSimpleFactory& AggregateFunctionSimpleFactory::instance() { register_aggregate_function_reader(instance); // register aggregate function for agg reader register_aggregate_function_window_rank(instance); register_aggregate_function_stddev_variance(instance); - + register_aggregate_function_topn(instance); + // if you only register function with no nullable, and wants to add nullable automatically, you should place function above this line register_aggregate_function_combinator_null(instance); - + register_aggregate_function_reader_no_spread(instance); register_aggregate_function_window_lead_lag(instance); }); diff --git a/be/src/vec/aggregate_functions/aggregate_function_stddev.h b/be/src/vec/aggregate_functions/aggregate_function_stddev.h index 6cdba20032e8fb..82e8718fcd7c8a 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_stddev.h +++ b/be/src/vec/aggregate_functions/aggregate_function_stddev.h @@ -188,7 +188,8 @@ struct BaseDatadecimal { template struct PopData : Data { - using ColVecResult = std::conditional_t, ColumnDecimal, ColumnVector>; + using ColVecResult = std::conditional_t, ColumnDecimal, + ColumnVector>; void insert_result_into(IColumn& to) const { ColumnNullable& nullable_column = assert_cast(to); auto& col = static_cast(nullable_column.get_nested_column()); @@ -203,7 +204,8 @@ struct PopData : Data { template struct SampData : Data { - using ColVecResult = std::conditional_t, ColumnDecimal, ColumnVector>; + using ColVecResult = std::conditional_t, ColumnDecimal, + ColumnVector>; void insert_result_into(IColumn& to) const { ColumnNullable& nullable_column = assert_cast(to); if (this->count == 1) { @@ -278,8 +280,6 @@ class AggregateFunctionStddevSamp final void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { this->data(place).insert_result_into(to); } - - const char* get_header_file_path() const override { return __FILE__; } }; } // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_sum.h b/be/src/vec/aggregate_functions/aggregate_function_sum.h index 402af36354b589..eceaf03338e8f6 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_sum.h +++ b/be/src/vec/aggregate_functions/aggregate_function_sum.h @@ -20,10 +20,6 @@ #pragma once -#include -#include -#include - #include "vec/aggregate_functions/aggregate_function.h" #include "vec/columns/column_vector.h" #include "vec/data_types/data_type_decimal.h" @@ -82,12 +78,11 @@ class AggregateFunctionSum final const auto& column = static_cast(*columns[0]); this->data(place).add(column.get_data()[row_num]); } - - void reset(AggregateDataPtr place) const override { - this->data(place).sum = {}; - } - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + void reset(AggregateDataPtr place) const override { this->data(place).sum = {}; } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, + Arena*) const override { this->data(place).merge(this->data(rhs)); } @@ -95,7 +90,8 @@ class AggregateFunctionSum final this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, + Arena*) const override { this->data(place).read(buf); } @@ -104,15 +100,13 @@ class AggregateFunctionSum final column.get_data().push_back(this->data(place).get()); } - const char* get_header_file_path() const override { return __FILE__; } - private: UInt32 scale; }; AggregateFunctionPtr create_aggregate_function_sum_reader(const std::string& name, - const DataTypes& argument_types, - const Array& parameters, - const bool result_is_nullable); + const DataTypes& argument_types, + const Array& parameters, + const bool result_is_nullable); } // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_topn.cpp b/be/src/vec/aggregate_functions/aggregate_function_topn.cpp new file mode 100644 index 00000000000000..a8347bf02916ac --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_topn.cpp @@ -0,0 +1,63 @@ +// 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 + +namespace doris::vectorized { + +template