From db07e51cd39fadfe6990d18ad7b9ec3ad25df66e Mon Sep 17 00:00:00 2001 From: plat1ko Date: Mon, 29 Aug 2022 12:06:01 +0800 Subject: [PATCH] [refactor](status) Refactor status handling in agent task (#11940) Refactor TaggableLogger Refactor status handling in agent task: Unify log format in TaskWorkerPool Pass Status to the top caller, and replace some OLAPInternalError with more detailed error message Status Premature return with the opposite condition to reduce indention --- be/src/agent/agent_server.cpp | 38 +- be/src/agent/task_worker_pool.cpp | 608 ++++++++---------- be/src/common/daemon.cpp | 2 +- be/src/common/logconfig.cpp | 30 +- be/src/common/logging.h | 53 ++ be/src/olap/olap_common.h | 4 +- be/src/olap/storage_engine.cpp | 13 +- be/src/olap/task/engine_alter_tablet_task.cpp | 12 - be/src/olap/task/engine_batch_load_task.cpp | 134 ++-- be/src/olap/task/engine_batch_load_task.h | 6 +- be/src/olap/task/engine_checksum_task.cpp | 42 +- be/src/olap/task/engine_clone_task.cpp | 486 ++++++-------- be/src/olap/task/engine_clone_task.h | 10 +- .../task/engine_storage_migration_task.cpp | 112 +--- be/src/runtime/data_stream_recvr.cc | 1 - be/src/runtime/fragment_mgr.cpp | 9 +- be/src/runtime/memory/mem_tracker_limiter.cpp | 1 + be/src/runtime/plan_fragment_executor.cpp | 36 +- be/src/runtime/query_statistics.cpp | 2 + be/src/runtime/result_buffer_mgr.h | 2 +- be/src/runtime/runtime_state.h | 1 - be/src/service/doris_main.cpp | 1 - be/src/util/cpu_info.h | 4 +- be/src/util/hash_util.hpp | 3 +- be/src/util/hdfs_util.cpp | 2 +- be/src/util/internal_queue.h | 1 + be/src/util/logging.h | 178 ----- be/src/util/s3_uri.cpp | 2 +- be/src/util/s3_util.cpp | 50 +- be/src/util/sm3.cpp | 2 +- be/src/util/spinlock.h | 4 +- be/src/util/trace.cpp | 13 +- be/src/util/uid_util.cpp | 13 + be/src/util/uid_util.h | 14 +- be/test/agent/agent_server_test.cpp | 1 - be/test/agent/cgroups_mgr_test.cpp | 1 - be/test/agent/heartbeat_server_test.cpp | 1 - be/test/agent/utils_test.cpp | 1 - be/test/common/resource_tls_test.cpp | 1 - be/test/common/status_test.cpp | 1 - be/test/exec/csv_scan_bench_test.cpp | 1 - be/test/exec/csv_scan_node_test.cpp | 1 - be/test/exec/csv_scanner_test.cpp | 2 - be/test/exec/olap_common_test.cpp | 1 - be/test/exprs/bitmap_function_test.cpp | 1 - be/test/exprs/encryption_functions_test.cpp | 1 - be/test/exprs/hll_function_test.cpp | 1 - be/test/exprs/hybrid_set_test.cpp | 1 - be/test/exprs/json_function_test.cpp | 1 - be/test/olap/block_column_predicate_test.cpp | 1 - .../bloom_filter_column_predicate_test.cpp | 1 - be/test/olap/bloom_filter_test.cpp | 1 - be/test/olap/byte_buffer_test.cpp | 1 - be/test/olap/comparison_predicate_test.cpp | 1 - be/test/olap/delete_handler_test.cpp | 1 - be/test/olap/delta_writer_test.cpp | 1 - .../engine_storage_migration_task_test.cpp | 1 - be/test/olap/file_helper_test.cpp | 1 - be/test/olap/file_utils_test.cpp | 1 - be/test/olap/in_list_predicate_test.cpp | 1 - be/test/olap/lru_cache_test.cpp | 1 - be/test/olap/memtable_flush_executor_test.cpp | 1 - be/test/olap/null_predicate_test.cpp | 1 - be/test/olap/row_block_test.cpp | 1 - be/test/olap/row_cursor_test.cpp | 1 - .../segment_v2/bitshuffle_page_test.cpp | 1 - .../segment_v2/bloom_filter_page_test.cpp | 1 - .../olap/rowset/segment_v2/rle_page_test.cpp | 1 - be/test/runtime/buffered_block_mgr2_test.cpp | 1 - .../runtime/buffered_tuple_stream2_test.cpp | 1 - .../runtime/cache/partition_cache_test.cpp | 1 - be/test/runtime/data_stream_test.cpp | 1 - be/test/runtime/datetime_value_test.cpp | 1 - be/test/runtime/decimal_value_test.cpp | 1 - be/test/runtime/decimalv2_value_test.cpp | 2 - be/test/runtime/disk_io_mgr_test.cpp | 1 - be/test/runtime/mem_limit_test.cpp | 1 - be/test/runtime/mem_pool_test.cpp | 2 - be/test/runtime/memory_scratch_sink_test.cpp | 1 - be/test/runtime/result_sink_test.cpp | 1 - .../routine_load_task_executor_test.cpp | 1 - be/test/runtime/tmp_file_mgr_test.cpp | 1 - be/test/testutil/run_all_tests.cpp | 1 - be/test/udf/uda_test.cpp | 1 - be/test/udf/udf_test.cpp | 1 - be/test/util/arrow/arrow_work_flow_test.cpp | 1 - be/test/util/bit_util_test.cpp | 1 - be/test/util/cidr_test.cpp | 1 - be/test/util/doris_metrics_test.cpp | 1 - be/test/util/filesystem_util_test.cpp | 1 - be/test/util/http_channel_test.cpp | 1 - be/test/util/internal_queue_test.cpp | 1 - be/test/util/lru_cache_util_test.cpp | 1 - be/test/util/metrics_test.cpp | 1 - be/test/util/mysql_row_buffer_test.cpp | 1 - be/test/util/path_trie_test.cpp | 1 - be/test/util/path_util_test.cpp | 1 - be/test/util/s3_uri_test.cpp | 2 - be/test/util/string_parser_test.cpp | 2 - be/test/util/system_metrics_test.cpp | 1 - be/test/util/url_coding_test.cpp | 2 - be/test/util/zip_util_test.cpp | 1 - 102 files changed, 710 insertions(+), 1253 deletions(-) delete mode 100644 be/src/util/logging.h diff --git a/be/src/agent/agent_server.cpp b/be/src/agent/agent_server.cpp index 8bac7811c8..e6ec8029a4 100644 --- a/be/src/agent/agent_server.cpp +++ b/be/src/agent/agent_server.cpp @@ -111,7 +111,7 @@ void AgentServer::submit_tasks(TAgentResult& agent_result, Status ret_st; // TODO check master_info here if it is the same with that of heartbeat rpc - if (_master_info.network_address.hostname == "" || _master_info.network_address.port == 0) { + if (_master_info.network_address.hostname.empty() || _master_info.network_address.port == 0) { Status ret_st = Status::Cancelled("Have not get FE Master heartbeat yet"); ret_st.to_thrift(&agent_result.status); return; @@ -191,8 +191,7 @@ void AgentServer::submit_tasks(TAgentResult& agent_result, #undef HANDLE_TYPE if (!ret_st.ok()) { - LOG(WARNING) << "fail to submit task. reason: " << ret_st.get_error_msg() - << ", task: " << task; + LOG_WARNING("failed to submit task").tag("task", task).error(ret_st.get_error_msg()); // For now, all tasks in the batch share one status, so if any task // was failed to submit, we can only return error to FE(even when some // tasks have already been successfully submitted). @@ -213,33 +212,34 @@ void AgentServer::make_snapshot(TAgentResult& t_agent_result, const TSnapshotRequest& snapshot_request) { string snapshot_path; bool allow_incremental_clone = false; - Status err_code = SnapshotManager::instance()->make_snapshot(snapshot_request, &snapshot_path, - &allow_incremental_clone); - if (!err_code) { - LOG(WARNING) << "fail to make_snapshot. tablet_id=" << snapshot_request.tablet_id - << ", schema_hash=" << snapshot_request.schema_hash - << ", error_code=" << err_code.to_string(); + Status status = SnapshotManager::instance()->make_snapshot(snapshot_request, &snapshot_path, + &allow_incremental_clone); + if (!status) { + LOG_WARNING("failed to make snapshot") + .tag("tablet_id", snapshot_request.tablet_id) + .tag("schema_hash", snapshot_request.schema_hash) + .error(status); } else { - LOG(INFO) << "success to make_snapshot. tablet_id=" << snapshot_request.tablet_id - << ", schema_hash=" << snapshot_request.schema_hash - << ", snapshot_path: " << snapshot_path; + LOG_INFO("successfully make snapshot") + .tag("tablet_id", snapshot_request.tablet_id) + .tag("schema_hash", snapshot_request.schema_hash) + .tag("snapshot_path", snapshot_path); t_agent_result.__set_snapshot_path(snapshot_path); t_agent_result.__set_allow_incremental_clone(allow_incremental_clone); } - err_code.to_thrift(&t_agent_result.status); + status.to_thrift(&t_agent_result.status); t_agent_result.__set_snapshot_version(snapshot_request.preferred_snapshot_version); } void AgentServer::release_snapshot(TAgentResult& t_agent_result, const std::string& snapshot_path) { - Status err_code = SnapshotManager::instance()->release_snapshot(snapshot_path); - if (!err_code) { - LOG(WARNING) << "failed to release_snapshot. snapshot_path: " << snapshot_path - << ", err_code: " << err_code.to_string(); + Status status = SnapshotManager::instance()->release_snapshot(snapshot_path); + if (!status) { + LOG_WARNING("failed to release snapshot").tag("snapshot_path", snapshot_path).error(status); } else { - LOG(INFO) << "success to release_snapshot. snapshot_path=" << snapshot_path; + LOG_INFO("successfully release snapshot").tag("snapshot_path", snapshot_path); } - err_code.to_thrift(&t_agent_result.status); + status.to_thrift(&t_agent_result.status); } void AgentServer::publish_cluster_state(TAgentResult& t_agent_result, diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index f63e9abb25..b561491600 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -248,10 +248,12 @@ void TaskWorkerPool::submit_task(const TAgentTaskRequest& task) { task_count_in_queue = _tasks.size(); _worker_thread_condition_variable.notify_one(); } - LOG(INFO) << "success to submit task. type=" << type_str << ", signature=" << signature - << ", queue size=" << task_count_in_queue; + LOG_INFO("successfully submit task") + .tag("type", type_str) + .tag("signature", signature) + .tag("queue_size", task_count_in_queue); } else { - LOG(INFO) << "fail to register task. type=" << type_str << ", signature=" << signature; + LOG_WARNING("failed to register task").tag("type", type_str).tag("signature", signature); } } @@ -295,9 +297,10 @@ void TaskWorkerPool::_finish_task(const TFinishTaskRequest& finish_task_request) break; } else { DorisMetrics::instance()->finish_task_requests_failed->increment(1); - LOG(WARNING) << "finish task failed. type=" << to_string(finish_task_request.task_type) - << ", signature=" << finish_task_request.signature - << ", status_code=" << result.status.status_code; + LOG_WARNING("failed to finish task") + .tag("type", finish_task_request.task_type) + .tag("signature", finish_task_request.signature) + .error(result.status); try_time += 1; } sleep(config::sleep_one_second); @@ -358,19 +361,19 @@ void TaskWorkerPool::_create_tablet_worker_thread_callback() { } }); ADOPT_TRACE(trace.get()); - TRACE("start to create tablet $0", create_tablet_req.tablet_id); - TStatusCode::type status_code = TStatusCode::OK; - std::vector error_msgs; - TStatus task_status; + DorisMetrics::instance()->create_tablet_requests_total->increment(1); + TRACE("start to create tablet $0", create_tablet_req.tablet_id); std::vector finish_tablet_infos; VLOG_NOTICE << "create tablet: " << create_tablet_req; - Status create_status = _env->storage_engine()->create_tablet(create_tablet_req); - if (!create_status.ok()) { - LOG(WARNING) << "create table failed. status: " << create_status - << ", signature: " << agent_task_req.signature; - status_code = TStatusCode::RUNTIME_ERROR; + Status status = _env->storage_engine()->create_tablet(create_tablet_req); + if (!status.ok()) { + DorisMetrics::instance()->create_tablet_requests_failed->increment(1); + LOG_WARNING("failed to create tablet") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", create_tablet_req.tablet_id) + .error(status); } else { ++_s_report_version; // get path hash of the created tablet @@ -388,18 +391,18 @@ void TaskWorkerPool::_create_tablet_worker_thread_callback() { tablet_info.__set_path_hash(tablet->data_dir()->path_hash()); tablet_info.__set_replica_id(tablet->replica_id()); finish_tablet_infos.push_back(tablet_info); + LOG_INFO("successfully create tablet") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", create_tablet_req.tablet_id); } - task_status.__set_status_code(status_code); - task_status.__set_error_msgs(error_msgs); - TFinishTaskRequest finish_task_request; finish_task_request.__set_finish_tablet_infos(finish_tablet_infos); finish_task_request.__set_backend(_backend); finish_task_request.__set_report_version(_s_report_version); finish_task_request.__set_task_type(agent_task_req.task_type); finish_task_request.__set_signature(agent_task_req.signature); - finish_task_request.__set_task_status(task_status); + finish_task_request.__set_task_status(status.to_thrift()); _finish_task(finish_task_request); _remove_task_info(agent_task_req.task_type, agent_task_req.signature); @@ -424,38 +427,36 @@ void TaskWorkerPool::_drop_tablet_worker_thread_callback() { _tasks.pop_front(); } - TStatusCode::type status_code = TStatusCode::OK; - std::vector error_msgs; - TStatus task_status; - string err; + Status status; TabletSharedPtr dropped_tablet = StorageEngine::instance()->tablet_manager()->get_tablet( - drop_tablet_req.tablet_id, false, &err); + drop_tablet_req.tablet_id, false); if (dropped_tablet != nullptr) { - Status drop_status = StorageEngine::instance()->tablet_manager()->drop_tablet( + status = StorageEngine::instance()->tablet_manager()->drop_tablet( drop_tablet_req.tablet_id, drop_tablet_req.replica_id, drop_tablet_req.is_drop_table_or_partition); - if (!drop_status.ok()) { - LOG(WARNING) << "drop table failed! signature: " << agent_task_req.signature; - error_msgs.push_back("drop table failed!"); - status_code = TStatusCode::RUNTIME_ERROR; - } else { - // if tablet is dropped by fe, then the related txn should also be removed - StorageEngine::instance()->txn_manager()->force_rollback_tablet_related_txns( - dropped_tablet->data_dir()->get_meta(), drop_tablet_req.tablet_id, - drop_tablet_req.schema_hash, dropped_tablet->tablet_uid()); - } } else { - status_code = TStatusCode::NOT_FOUND; - error_msgs.push_back(err); + status = Status::NotFound("could not find tablet {}", drop_tablet_req.tablet_id); + } + if (status.ok()) { + // if tablet is dropped by fe, then the related txn should also be removed + StorageEngine::instance()->txn_manager()->force_rollback_tablet_related_txns( + dropped_tablet->data_dir()->get_meta(), drop_tablet_req.tablet_id, + drop_tablet_req.schema_hash, dropped_tablet->tablet_uid()); + LOG_INFO("successfully drop tablet") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", drop_tablet_req.tablet_id); + } else { + LOG_WARNING("failed to drop tablet") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", drop_tablet_req.tablet_id) + .error(status); } - task_status.__set_status_code(status_code); - task_status.__set_error_msgs(error_msgs); TFinishTaskRequest finish_task_request; finish_task_request.__set_backend(_backend); finish_task_request.__set_task_type(agent_task_req.task_type); finish_task_request.__set_signature(agent_task_req.signature); - finish_task_request.__set_task_status(task_status); + finish_task_request.__set_task_status(status.to_thrift()); _finish_task(finish_task_request); _remove_task_info(agent_task_req.task_type, agent_task_req.signature); @@ -508,14 +509,12 @@ void TaskWorkerPool::_alter_tablet_worker_thread_callback() { void TaskWorkerPool::_alter_tablet(const TAgentTaskRequest& agent_task_req, int64_t signature, const TTaskType::type task_type, TFinishTaskRequest* finish_task_request) { - Status status = Status::OK(); - TStatus task_status; - std::vector error_msgs; + Status status; string process_name; switch (task_type) { case TTaskType::ALTER: - process_name = "AlterTablet"; + process_name = "alter tablet"; break; default: std::string task_name; @@ -535,20 +534,11 @@ void TaskWorkerPool::_alter_tablet(const TAgentTaskRequest& agent_task_req, int6 new_tablet_id = agent_task_req.alter_tablet_req_v2.new_tablet_id; new_schema_hash = agent_task_req.alter_tablet_req_v2.new_schema_hash; EngineAlterTabletTask engine_task(agent_task_req.alter_tablet_req_v2); - Status sc_status = _env->storage_engine()->execute_task(&engine_task); - if (!sc_status.ok()) { - if (sc_status.precise_code() == OLAP_ERR_DATA_QUALITY_ERR) { - error_msgs.push_back("The data quality does not satisfy, please check your data. "); - } - status = sc_status; - } else { - status = Status::OK(); - } + status = _env->storage_engine()->execute_task(&engine_task); } if (status.ok()) { ++_s_report_version; - LOG(INFO) << process_name << " finished. signature: " << signature; } // Return result to fe @@ -561,28 +551,25 @@ void TaskWorkerPool::_alter_tablet(const TAgentTaskRequest& agent_task_req, int6 if (status.ok()) { TTabletInfo tablet_info; status = _get_tablet_info(new_tablet_id, new_schema_hash, signature, &tablet_info); - - if (!status.ok()) { - LOG(WARNING) << process_name << " success, but get new tablet info failed." - << "tablet_id: " << new_tablet_id << ", schema_hash: " << new_schema_hash - << ", signature: " << signature; - } else { + if (status.ok()) { finish_tablet_infos.push_back(tablet_info); } } - if (status.ok()) { - finish_task_request->__set_finish_tablet_infos(finish_tablet_infos); - LOG(INFO) << process_name << " success. signature: " << signature; - error_msgs.push_back(process_name + " success"); + if (!status.ok() && status.code() != TStatusCode::NOT_IMPLEMENTED_ERROR) { + LOG_WARNING("failed to {}", process_name) + .tag("signature", agent_task_req.signature) + .tag("base_tablet_id", agent_task_req.alter_tablet_req_v2.base_tablet_id) + .tag("new_tablet_id", new_tablet_id) + .error(status); } else { - LOG(WARNING) << process_name << " failed. signature: " << signature; - error_msgs.push_back(process_name + " failed"); - error_msgs.push_back("status: " + status.to_string()); + finish_task_request->__set_finish_tablet_infos(finish_tablet_infos); + LOG_INFO("successfully {}", process_name) + .tag("signature", agent_task_req.signature) + .tag("base_tablet_id", agent_task_req.alter_tablet_req_v2.base_tablet_id) + .tag("new_tablet_id", new_tablet_id); } - task_status.__set_status_code(status.code()); - task_status.__set_error_msgs(error_msgs); - finish_task_request->__set_task_status(task_status); + finish_task_request->__set_task_status(status.to_thrift()); } void TaskWorkerPool::_push_worker_thread_callback() { @@ -599,7 +586,6 @@ void TaskWorkerPool::_push_worker_thread_callback() { } while (_is_work) { - Status status = Status::OK(); TAgentTaskRequest agent_task_req; TPushReq push_req; int32_t index = 0; @@ -633,16 +619,14 @@ void TaskWorkerPool::_push_worker_thread_callback() { continue; } - LOG(INFO) << "get push task. signature: " << agent_task_req.signature - << " priority: " << priority << " push_type: " << push_req.push_type; + LOG(INFO) << "get push task. signature=" << agent_task_req.signature + << ", priority=" << priority << " push_type=" << push_req.push_type; std::vector tablet_infos; - EngineBatchLoadTask engine_task(push_req, &tablet_infos, agent_task_req.signature, &status); - _env->storage_engine()->execute_task(&engine_task); - // Return result to fe - std::vector error_msgs; - TStatus task_status; + EngineBatchLoadTask engine_task(push_req, &tablet_infos); + auto status = _env->storage_engine()->execute_task(&engine_task); + // Return result to fe TFinishTaskRequest finish_task_request; finish_task_request.__set_backend(_backend); finish_task_request.__set_task_type(agent_task_req.task_type); @@ -652,22 +636,20 @@ void TaskWorkerPool::_push_worker_thread_callback() { } if (status.ok()) { - VLOG_NOTICE << "push ok. signature: " << agent_task_req.signature - << ", push_type: " << push_req.push_type; - error_msgs.push_back("push success"); - + LOG_INFO("successfully execute push task") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", push_req.tablet_id) + .tag("push_type", push_req.push_type); ++_s_report_version; - - task_status.__set_status_code(TStatusCode::OK); finish_task_request.__set_finish_tablet_infos(tablet_infos); } else { - LOG(WARNING) << "push failed, error_code: " << status - << ", signature: " << agent_task_req.signature; - error_msgs.push_back("push failed"); - task_status.__set_status_code(TStatusCode::RUNTIME_ERROR); + LOG_WARNING("failed to execute push task") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", push_req.tablet_id) + .tag("push_type", push_req.push_type) + .error(status); } - task_status.__set_error_msgs(error_msgs); - finish_task_request.__set_task_status(task_status); + finish_task_request.__set_task_status(status.to_thrift()); finish_task_request.__set_report_version(_s_report_version); _finish_task(finish_task_request); @@ -694,20 +676,20 @@ void TaskWorkerPool::_publish_version_worker_thread_callback() { } DorisMetrics::instance()->publish_task_request_total->increment(1); - VLOG_NOTICE << "get publish version task, signature:" << agent_task_req.signature; + VLOG_NOTICE << "get publish version task. signature=" << agent_task_req.signature; std::vector error_tablet_ids; std::vector succ_tablet_ids; uint32_t retry_time = 0; - Status res = Status::OK(); + Status status; while (retry_time < PUBLISH_VERSION_MAX_RETRY) { error_tablet_ids.clear(); EnginePublishVersionTask engine_task(publish_version_req, &error_tablet_ids, &succ_tablet_ids); - res = _env->storage_engine()->execute_task(&engine_task); - if (res.ok()) { + status = _env->storage_engine()->execute_task(&engine_task); + if (status.ok()) { break; - } else if (res.precise_code() == OLAP_ERR_PUBLISH_VERSION_NOT_CONTINUOUS) { + } else if (status.precise_code() == OLAP_ERR_PUBLISH_VERSION_NOT_CONTINUOUS) { // version not continuous, put to queue and wait pre version publish // task execute std::unique_lock worker_thread_lock(_worker_thread_lock); @@ -715,24 +697,29 @@ void TaskWorkerPool::_publish_version_worker_thread_callback() { _worker_thread_condition_variable.notify_one(); break; } else { - LOG(WARNING) << "publish version error, retry. [transaction_id=" - << publish_version_req.transaction_id - << ", error_tablets_size=" << error_tablet_ids.size() << "]"; + LOG_WARNING("failed to publish version") + .tag("transaction_id", publish_version_req.transaction_id) + .tag("error_tablets_num", error_tablet_ids.size()) + .tag("retry_time", retry_time) + .error(status); ++retry_time; std::this_thread::sleep_for(std::chrono::seconds(1)); } } - if (res.precise_code() == OLAP_ERR_PUBLISH_VERSION_NOT_CONTINUOUS) { + if (status.precise_code() == OLAP_ERR_PUBLISH_VERSION_NOT_CONTINUOUS) { continue; } TFinishTaskRequest finish_task_request; - if (!res) { + if (!status) { DorisMetrics::instance()->publish_task_failed_total->increment(1); // if publish failed, return failed, FE will ignore this error and // check error tablet ids and FE will also republish this task - LOG(WARNING) << "publish version failed. signature:" << agent_task_req.signature - << ", error_code=" << res; + LOG_WARNING("failed to publish version") + .tag("signature", agent_task_req.signature) + .tag("transaction_id", publish_version_req.transaction_id) + .tag("error_tablets_num", error_tablet_ids.size()) + .error(status); finish_task_request.__set_error_tablet_ids(error_tablet_ids); } else { int submit_tablets = 0; @@ -755,12 +742,14 @@ void TaskWorkerPool::_publish_version_worker_thread_callback() { << succ_tablet_ids[i]; } } - LOG(INFO) << "publish_version success. signature:" << agent_task_req.signature - << ", size:" << succ_tablet_ids.size(); + LOG_INFO("successfully publish version") + .tag("signature", agent_task_req.signature) + .tag("transaction_id", publish_version_req.transaction_id) + .tag("tablets_num", succ_tablet_ids.size()); } } - res.to_thrift(&finish_task_request.task_status); + status.to_thrift(&finish_task_request.task_status); finish_task_request.__set_backend(_backend); finish_task_request.__set_task_type(agent_task_req.task_type); finish_task_request.__set_signature(agent_task_req.signature); @@ -788,13 +777,11 @@ void TaskWorkerPool::_clear_transaction_task_worker_thread_callback() { clear_transaction_task_req = agent_task_req.clear_transaction_task_req; _tasks.pop_front(); } - LOG(INFO) << "get clear transaction task task, signature:" << agent_task_req.signature - << ", transaction_id: " << clear_transaction_task_req.transaction_id - << ", partition id size: " << clear_transaction_task_req.partition_id.size(); + LOG(INFO) << "get clear transaction task. signature=" << agent_task_req.signature + << ", transaction_id=" << clear_transaction_task_req.transaction_id + << ", partition_id_size=" << clear_transaction_task_req.partition_id.size(); - TStatusCode::type status_code = TStatusCode::OK; - std::vector error_msgs; - TStatus task_status; + Status status; if (clear_transaction_task_req.transaction_id > 0) { // transaction_id should be greater than zero. @@ -808,18 +795,15 @@ void TaskWorkerPool::_clear_transaction_task_worker_thread_callback() { _env->storage_engine()->clear_transaction_task( clear_transaction_task_req.transaction_id); } - LOG(INFO) << "finish to clear transaction task. signature:" << agent_task_req.signature - << ", transaction_id: " << clear_transaction_task_req.transaction_id; + LOG(INFO) << "finish to clear transaction task. signature=" << agent_task_req.signature + << ", transaction_id=" << clear_transaction_task_req.transaction_id; } else { - LOG(WARNING) << "invalid transaction id: " << clear_transaction_task_req.transaction_id - << ", signature: " << agent_task_req.signature; + LOG(WARNING) << "invalid transaction id " << clear_transaction_task_req.transaction_id + << ". signature= " << agent_task_req.signature; } - task_status.__set_status_code(status_code); - task_status.__set_error_msgs(error_msgs); - TFinishTaskRequest finish_task_request; - finish_task_request.__set_task_status(task_status); + finish_task_request.__set_task_status(status.to_thrift()); finish_task_request.__set_backend(_backend); finish_task_request.__set_task_type(agent_task_req.task_type); finish_task_request.__set_signature(agent_task_req.signature); @@ -846,19 +830,17 @@ void TaskWorkerPool::_update_tablet_meta_worker_thread_callback() { update_tablet_meta_req = agent_task_req.update_tablet_meta_info_req; _tasks.pop_front(); } - LOG(INFO) << "get update tablet meta task, signature:" << agent_task_req.signature; + LOG(INFO) << "get update tablet meta task. signature=" << agent_task_req.signature; - TStatusCode::type status_code = TStatusCode::OK; - std::vector error_msgs; - TStatus task_status; + Status status; - for (auto tablet_meta_info : update_tablet_meta_req.tabletMetaInfos) { + for (auto& tablet_meta_info : update_tablet_meta_req.tabletMetaInfos) { TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet( tablet_meta_info.tablet_id); if (tablet == nullptr) { - LOG(WARNING) << "could not find tablet when update partition id" - << " tablet_id=" << tablet_meta_info.tablet_id - << " schema_hash=" << tablet_meta_info.schema_hash; + LOG(WARNING) << "could not find tablet when update partition id. tablet_id=" + << tablet_meta_info.tablet_id + << ", schema_hash=" << tablet_meta_info.schema_hash; continue; } std::lock_guard wrlock(tablet->get_header_lock()); @@ -885,13 +867,10 @@ void TaskWorkerPool::_update_tablet_meta_worker_thread_callback() { tablet->save_meta(); } - LOG(INFO) << "finish update tablet meta task. signature:" << agent_task_req.signature; - - task_status.__set_status_code(status_code); - task_status.__set_error_msgs(error_msgs); + LOG(INFO) << "finish update tablet meta task. signature=" << agent_task_req.signature; TFinishTaskRequest finish_task_request; - finish_task_request.__set_task_status(task_status); + finish_task_request.__set_task_status(status.to_thrift()); finish_task_request.__set_backend(_backend); finish_task_request.__set_task_type(agent_task_req.task_type); finish_task_request.__set_signature(agent_task_req.signature); @@ -903,7 +882,6 @@ void TaskWorkerPool::_update_tablet_meta_worker_thread_callback() { void TaskWorkerPool::_clone_worker_thread_callback() { while (_is_work) { - Status status = Status::OK(); TAgentTaskRequest agent_task_req; TCloneReq clone_req; @@ -922,34 +900,31 @@ void TaskWorkerPool::_clone_worker_thread_callback() { } DorisMetrics::instance()->clone_requests_total->increment(1); - LOG(INFO) << "get clone task. signature:" << agent_task_req.signature; + LOG(INFO) << "get clone task. signature=" << agent_task_req.signature; - std::vector error_msgs; std::vector tablet_infos; - EngineCloneTask engine_task(clone_req, _master_info, agent_task_req.signature, &error_msgs, - &tablet_infos, &status); - _env->storage_engine()->execute_task(&engine_task); + EngineCloneTask engine_task(clone_req, _master_info, agent_task_req.signature, + &tablet_infos); + auto status = _env->storage_engine()->execute_task(&engine_task); // Return result to fe - TStatus task_status; TFinishTaskRequest finish_task_request; finish_task_request.__set_backend(_backend); finish_task_request.__set_task_type(agent_task_req.task_type); finish_task_request.__set_signature(agent_task_req.signature); + finish_task_request.__set_task_status(status.to_thrift()); - TStatusCode::type status_code = TStatusCode::OK; if (!status.ok()) { DorisMetrics::instance()->clone_requests_failed->increment(1); - status_code = TStatusCode::RUNTIME_ERROR; - LOG(WARNING) << "clone failed. signature: " << agent_task_req.signature; - error_msgs.push_back("clone failed."); + LOG_WARNING("failed to clone tablet") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", clone_req.tablet_id) + .error(status); } else { - LOG(INFO) << "clone success, set tablet infos." - << "signature:" << agent_task_req.signature; + LOG_INFO("successfully clone tablet") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", clone_req.tablet_id); finish_task_request.__set_finish_tablet_infos(tablet_infos); } - task_status.__set_status_code(status_code); - task_status.__set_error_msgs(error_msgs); - finish_task_request.__set_task_status(task_status); _finish_task(finish_task_request); _remove_task_info(agent_task_req.task_type, agent_task_req.signature); @@ -974,35 +949,31 @@ void TaskWorkerPool::_storage_medium_migrate_worker_thread_callback() { _tasks.pop_front(); } - TStatusCode::type status_code = TStatusCode::OK; // check request and get info TabletSharedPtr tablet; DataDir* dest_store = nullptr; - if (!_check_migrate_request(storage_medium_migrate_req, tablet, &dest_store)) { - status_code = TStatusCode::RUNTIME_ERROR; - } else { - EngineStorageMigrationTask engine_task(tablet, dest_store); - Status res = _env->storage_engine()->execute_task(&engine_task); - if (!res.ok()) { - LOG(WARNING) << "storage media migrate failed. status: " << res - << ", signature: " << agent_task_req.signature; - status_code = TStatusCode::RUNTIME_ERROR; - } else { - LOG(INFO) << "storage media migrate success. status:" << res - << ", signature:" << agent_task_req.signature; - } - } - TStatus task_status; - std::vector error_msgs; - task_status.__set_status_code(status_code); - task_status.__set_error_msgs(error_msgs); + auto status = _check_migrate_request(storage_medium_migrate_req, tablet, &dest_store); + if (status.ok()) { + EngineStorageMigrationTask engine_task(tablet, dest_store); + status = _env->storage_engine()->execute_task(&engine_task); + } + if (!status.ok()) { + LOG_WARNING("failed to migrate storage medium") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", storage_medium_migrate_req.tablet_id) + .error(status); + } else { + LOG_INFO("successfully migrate storage medium") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", storage_medium_migrate_req.tablet_id); + } TFinishTaskRequest finish_task_request; finish_task_request.__set_backend(_backend); finish_task_request.__set_task_type(agent_task_req.task_type); finish_task_request.__set_signature(agent_task_req.signature); - finish_task_request.__set_task_status(task_status); + finish_task_request.__set_task_status(status.to_thrift()); _finish_task(finish_task_request); _remove_task_info(agent_task_req.task_type, agent_task_req.signature); @@ -1014,16 +985,14 @@ Status TaskWorkerPool::_check_migrate_request(const TStorageMediumMigrateReq& re int64_t tablet_id = req.tablet_id; tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id); if (tablet == nullptr) { - LOG(WARNING) << "can't find tablet. tablet_id= " << tablet_id; - return Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND); + return Status::InternalError("could not find tablet {}", tablet_id); } if (req.__isset.data_dir) { // request specify the data dir *dest_store = StorageEngine::instance()->get_store(req.data_dir); if (*dest_store == nullptr) { - LOG(WARNING) << "data dir not found: " << req.data_dir; - return Status::OLAPInternalError(OLAP_ERR_DIR_NOT_EXIST); + return Status::InternalError("could not find data dir {}", req.data_dir); } } else { // this is a storage medium @@ -1032,39 +1001,33 @@ Status TaskWorkerPool::_check_migrate_request(const TStorageMediumMigrateReq& re // judge case when no need to migrate uint32_t count = StorageEngine::instance()->available_storage_medium_type_count(); if (count <= 1) { - LOG(INFO) << "available storage medium type count is less than 1, " - << "no need to migrate. count=" << count; - return Status::OLAPInternalError(OLAP_REQUEST_FAILED); + return Status::InternalError("available storage medium type count is less than 1"); } // check current tablet storage medium TStorageMedium::type storage_medium = req.storage_medium; TStorageMedium::type src_storage_medium = tablet->data_dir()->storage_medium(); if (src_storage_medium == storage_medium) { - LOG(INFO) << "tablet is already on specified storage medium. " - << "storage_medium=" << storage_medium; - return Status::OLAPInternalError(OLAP_REQUEST_FAILED); + return Status::InternalError("tablet is already on specified storage medium {}", + storage_medium); } // get a random store of specified storage medium auto stores = StorageEngine::instance()->get_stores_for_create_tablet(storage_medium); if (stores.empty()) { - LOG(WARNING) << "fail to get root path for create tablet."; - return Status::OLAPInternalError(OLAP_ERR_INVALID_ROOT_PATH); + return Status::InternalError("failed to get root path for create tablet"); } *dest_store = stores[0]; } if (tablet->data_dir()->path() == (*dest_store)->path()) { - LOG(INFO) << "tablet is already on specified path. " - << "path=" << tablet->data_dir()->path(); - return Status::OLAPInternalError(OLAP_REQUEST_FAILED); + return Status::InternalError("tablet is already on specified path {}", + tablet->data_dir()->path()); } // check local disk capacity int64_t tablet_size = tablet->tablet_local_size(); if ((*dest_store)->reach_capacity_limit(tablet_size)) { - LOG(WARNING) << "reach the capacity limit of path: " << (*dest_store)->path() - << ", tablet size: " << tablet_size; - return Status::OLAPInternalError(OLAP_ERR_DISK_REACH_CAPACITY_LIMIT); + return Status::InternalError("reach the capacity limit of path {}, tablet_size={}", + (*dest_store)->path(), tablet_size); } return Status::OK(); @@ -1088,32 +1051,28 @@ void TaskWorkerPool::_check_consistency_worker_thread_callback() { _tasks.pop_front(); } - TStatusCode::type status_code = TStatusCode::OK; - std::vector error_msgs; - TStatus task_status; - uint32_t checksum = 0; EngineChecksumTask engine_task(check_consistency_req.tablet_id, check_consistency_req.schema_hash, check_consistency_req.version, &checksum); - Status res = _env->storage_engine()->execute_task(&engine_task); - if (!res.ok()) { - LOG(WARNING) << "check consistency failed. status: " << res - << ", signature: " << agent_task_req.signature; - status_code = TStatusCode::RUNTIME_ERROR; + Status status = _env->storage_engine()->execute_task(&engine_task); + if (!status.ok()) { + LOG_WARNING("failed to check consistency") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", check_consistency_req.tablet_id) + .error(status); } else { - LOG(INFO) << "check consistency success. status: " << res - << ", signature:" << agent_task_req.signature << ", checksum:" << checksum; + LOG_INFO("successfully check consistency") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", check_consistency_req.tablet_id) + .tag("checksum", checksum); } - task_status.__set_status_code(status_code); - task_status.__set_error_msgs(error_msgs); - TFinishTaskRequest finish_task_request; finish_task_request.__set_backend(_backend); finish_task_request.__set_task_type(agent_task_req.task_type); finish_task_request.__set_signature(agent_task_req.signature); - finish_task_request.__set_task_status(task_status); + finish_task_request.__set_task_status(status.to_thrift()); finish_task_request.__set_tablet_checksum(static_cast(checksum)); finish_task_request.__set_request_version(check_consistency_req.version); @@ -1246,9 +1205,8 @@ void TaskWorkerPool::_report_tablet_worker_thread_callback() { _random_sleep(5); request.tablets.clear(); uint64_t report_version = _s_report_version; - Status build_all_report_tablets_info_status = - StorageEngine::instance()->tablet_manager()->build_all_report_tablets_info( - &request.tablets); + StorageEngine::instance()->tablet_manager()->build_all_report_tablets_info( + &request.tablets); if (report_version < _s_report_version) { // TODO llj This can only reduce the possibility for report error, but can't avoid it. // If FE create a tablet in FE meta and send CREATE task to this BE, the tablet may not be included in this @@ -1259,11 +1217,6 @@ void TaskWorkerPool::_report_tablet_worker_thread_callback() { DorisMetrics::instance()->report_all_tablets_requests_skip->increment(1); continue; } - if (!build_all_report_tablets_info_status.ok()) { - LOG(WARNING) << "build all report tablets info failed. status: " - << build_all_report_tablets_info_status; - continue; - } int64_t max_compaction_score = std::max(DorisMetrics::instance()->tablet_cumulative_max_compaction_score->value(), DorisMetrics::instance()->tablet_base_max_compaction_score->value()); @@ -1292,8 +1245,8 @@ void TaskWorkerPool::_upload_worker_thread_callback() { _tasks.pop_front(); } - LOG(INFO) << "get upload task, signature:" << agent_task_req.signature - << ", job id:" << upload_request.job_id; + LOG(INFO) << "get upload task. signature=" << agent_task_req.signature + << ", job_id=" << upload_request.job_id; std::map> tablet_files; std::unique_ptr loader = std::make_unique( @@ -1303,31 +1256,26 @@ void TaskWorkerPool::_upload_worker_thread_callback() { : TStorageBackendType::type::BROKER); Status status = loader->upload(upload_request.src_dest_map, &tablet_files); - TStatusCode::type status_code = TStatusCode::OK; - std::vector error_msgs; if (!status.ok()) { - status_code = TStatusCode::RUNTIME_ERROR; - LOG(WARNING) << "upload failed. job id: " << upload_request.job_id - << ", msg: " << status.get_error_msg(); - error_msgs.push_back(status.get_error_msg()); + LOG_WARNING("failed to upload") + .tag("signature", agent_task_req.signature) + .tag("job_id", upload_request.job_id) + .error(status); + } else { + LOG_INFO("successfully upload") + .tag("signature", agent_task_req.signature) + .tag("job_id", upload_request.job_id); } - TStatus task_status; - task_status.__set_status_code(status_code); - task_status.__set_error_msgs(error_msgs); - TFinishTaskRequest finish_task_request; finish_task_request.__set_backend(_backend); finish_task_request.__set_task_type(agent_task_req.task_type); finish_task_request.__set_signature(agent_task_req.signature); - finish_task_request.__set_task_status(task_status); + finish_task_request.__set_task_status(status.to_thrift()); finish_task_request.__set_tablet_files(tablet_files); _finish_task(finish_task_request); _remove_task_info(agent_task_req.task_type, agent_task_req.signature); - - LOG(INFO) << "finished upload task, signature: " << agent_task_req.signature - << ", job id:" << upload_request.job_id; } } @@ -1348,12 +1296,8 @@ void TaskWorkerPool::_download_worker_thread_callback() { download_request = agent_task_req.download_req; _tasks.pop_front(); } - LOG(INFO) << "get download task, signature: " << agent_task_req.signature - << ", job id:" << download_request.job_id; - - TStatusCode::type status_code = TStatusCode::OK; - std::vector error_msgs; - TStatus task_status; + LOG(INFO) << "get download task. signature=" << agent_task_req.signature + << ", job_id=" << download_request.job_id; // TODO: download std::vector downloaded_tablet_ids; @@ -1366,27 +1310,25 @@ void TaskWorkerPool::_download_worker_thread_callback() { Status status = loader->download(download_request.src_dest_map, &downloaded_tablet_ids); if (!status.ok()) { - status_code = TStatusCode::RUNTIME_ERROR; - LOG(WARNING) << "download failed. job id: " << download_request.job_id - << ", msg: " << status.get_error_msg(); - error_msgs.push_back(status.get_error_msg()); + LOG_WARNING("failed to download") + .tag("signature", agent_task_req.signature) + .tag("job_id", download_request.job_id) + .error(status); + } else { + LOG_INFO("successfully download") + .tag("signature", agent_task_req.signature) + .tag("job_id", download_request.job_id); } - task_status.__set_status_code(status_code); - task_status.__set_error_msgs(error_msgs); - TFinishTaskRequest finish_task_request; finish_task_request.__set_backend(_backend); finish_task_request.__set_task_type(agent_task_req.task_type); finish_task_request.__set_signature(agent_task_req.signature); - finish_task_request.__set_task_status(task_status); + finish_task_request.__set_task_status(status.to_thrift()); finish_task_request.__set_downloaded_tablet_ids(downloaded_tablet_ids); _finish_task(finish_task_request); _remove_task_info(agent_task_req.task_type, agent_task_req.signature); - - LOG(INFO) << "finished download task, signature: " << agent_task_req.signature - << ", job id:" << download_request.job_id; } } @@ -1407,52 +1349,35 @@ void TaskWorkerPool::_make_snapshot_thread_callback() { snapshot_request = agent_task_req.snapshot_req; _tasks.pop_front(); } - LOG(INFO) << "get snapshot task, signature:" << agent_task_req.signature; - - TStatusCode::type status_code = TStatusCode::OK; - std::vector error_msgs; - TStatus task_status; + LOG(INFO) << "get snapshot task. signature=" << agent_task_req.signature; string snapshot_path; bool allow_incremental_clone = false; // not used std::vector snapshot_files; - Status make_snapshot_status = SnapshotManager::instance()->make_snapshot( - snapshot_request, &snapshot_path, &allow_incremental_clone); - if (!make_snapshot_status.ok()) { - status_code = make_snapshot_status.code(); - LOG(WARNING) << "make_snapshot failed. tablet_id:" << snapshot_request.tablet_id - << ", schema_hash:" << snapshot_request.schema_hash - << ", version:" << snapshot_request.version - << ", status: " << make_snapshot_status.to_string(); - error_msgs.push_back("make_snapshot failed. status: " + - make_snapshot_status.get_error_msg()); - } else { - LOG(INFO) << "make_snapshot success. tablet_id:" << snapshot_request.tablet_id - << ", schema_hash:" << snapshot_request.schema_hash - << ", version:" << snapshot_request.version - << ", snapshot_path:" << snapshot_path; - if (snapshot_request.__isset.list_files) { - // list and save all snapshot files - // snapshot_path like: data/snapshot/20180417205230.1.86400 - // we need to add subdir: tablet_id/schema_hash/ - std::stringstream ss; - ss << snapshot_path << "/" << snapshot_request.tablet_id << "/" - << snapshot_request.schema_hash << "/"; - Status st = FileUtils::list_files(Env::Default(), ss.str(), &snapshot_files); - if (!st.ok()) { - status_code = TStatusCode::RUNTIME_ERROR; - LOG(WARNING) << "make_snapshot failed. tablet_id:" << snapshot_request.tablet_id - << ", schema_hash:" << snapshot_request.schema_hash - << ", version:" << snapshot_request.version - << ",list file failed: " << st.to_string(); - error_msgs.push_back("make_snapshot failed. list file failed: " + - st.get_error_msg()); - } - } + Status status = SnapshotManager::instance()->make_snapshot(snapshot_request, &snapshot_path, + &allow_incremental_clone); + if (status.ok() && snapshot_request.__isset.list_files) { + // list and save all snapshot files + // snapshot_path like: data/snapshot/20180417205230.1.86400 + // we need to add subdir: tablet_id/schema_hash/ + std::stringstream ss; + ss << snapshot_path << "/" << snapshot_request.tablet_id << "/" + << snapshot_request.schema_hash << "/"; + status = FileUtils::list_files(Env::Default(), ss.str(), &snapshot_files); + } + if (!status.ok()) { + LOG_WARNING("failed to make snapshot") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", snapshot_request.tablet_id) + .tag("version", snapshot_request.version) + .error(status); + } else { + LOG_INFO("successfully make snapshot") + .tag("signature", agent_task_req.signature) + .tag("tablet_id", snapshot_request.tablet_id) + .tag("version", snapshot_request.version) + .tag("snapshot_path", snapshot_path); } - - task_status.__set_status_code(status_code); - task_status.__set_error_msgs(error_msgs); TFinishTaskRequest finish_task_request; finish_task_request.__set_backend(_backend); @@ -1460,7 +1385,7 @@ void TaskWorkerPool::_make_snapshot_thread_callback() { finish_task_request.__set_signature(agent_task_req.signature); finish_task_request.__set_snapshot_path(snapshot_path); finish_task_request.__set_snapshot_files(snapshot_files); - finish_task_request.__set_task_status(task_status); + finish_task_request.__set_task_status(status.to_thrift()); _finish_task(finish_task_request); _remove_task_info(agent_task_req.task_type, agent_task_req.signature); @@ -1484,34 +1409,26 @@ void TaskWorkerPool::_release_snapshot_thread_callback() { release_snapshot_request = agent_task_req.release_snapshot_req; _tasks.pop_front(); } - LOG(INFO) << "get release snapshot task, signature:" << agent_task_req.signature; - - TStatusCode::type status_code = TStatusCode::OK; - std::vector error_msgs; - TStatus task_status; + LOG(INFO) << "get release snapshot task. signature=" << agent_task_req.signature; string& snapshot_path = release_snapshot_request.snapshot_path; - Status release_snapshot_status = - SnapshotManager::instance()->release_snapshot(snapshot_path); - if (!release_snapshot_status.ok()) { - status_code = TStatusCode::RUNTIME_ERROR; - LOG(WARNING) << "release_snapshot failed. snapshot_path: " << snapshot_path - << ". status: " << release_snapshot_status; - error_msgs.push_back("release_snapshot failed. status: " + - boost::lexical_cast(release_snapshot_status)); + Status status = SnapshotManager::instance()->release_snapshot(snapshot_path); + if (!status.ok()) { + LOG_WARNING("failed to release snapshot") + .tag("signature", agent_task_req.signature) + .tag("snapshot_path", snapshot_path) + .error(status); } else { - LOG(INFO) << "release_snapshot success. snapshot_path: " << snapshot_path - << ". status: " << release_snapshot_status; + LOG_INFO("successfully release snapshot") + .tag("signature", agent_task_req.signature) + .tag("snapshot_path", snapshot_path); } - task_status.__set_status_code(status_code); - task_status.__set_error_msgs(error_msgs); - TFinishTaskRequest finish_task_request; finish_task_request.__set_backend(_backend); finish_task_request.__set_task_type(agent_task_req.task_type); finish_task_request.__set_signature(agent_task_req.signature); - finish_task_request.__set_task_status(task_status); + finish_task_request.__set_task_status(status.to_thrift()); _finish_task(finish_task_request); _remove_task_info(agent_task_req.task_type, agent_task_req.signature); @@ -1520,17 +1437,9 @@ void TaskWorkerPool::_release_snapshot_thread_callback() { Status TaskWorkerPool::_get_tablet_info(const TTabletId tablet_id, const TSchemaHash schema_hash, int64_t signature, TTabletInfo* tablet_info) { - Status status = Status::OK(); tablet_info->__set_tablet_id(tablet_id); tablet_info->__set_schema_hash(schema_hash); - Status olap_status = - StorageEngine::instance()->tablet_manager()->report_tablet_info(tablet_info); - if (!olap_status.ok()) { - LOG(WARNING) << "get tablet info failed. status: " << olap_status - << ", signature: " << signature; - status = Status::InternalError("Get tablet info failed"); - } - return status; + return StorageEngine::instance()->tablet_manager()->report_tablet_info(tablet_info); } void TaskWorkerPool::_move_dir_thread_callback() { @@ -1550,21 +1459,24 @@ void TaskWorkerPool::_move_dir_thread_callback() { move_dir_req = agent_task_req.move_dir_req; _tasks.pop_front(); } - LOG(INFO) << "get move dir task, signature:" << agent_task_req.signature - << ", job id:" << move_dir_req.job_id; + LOG(INFO) << "get move dir task. signature=" << agent_task_req.signature + << ", job_id=" << move_dir_req.job_id; Status status = _move_dir(move_dir_req.tablet_id, move_dir_req.src, move_dir_req.job_id, true /* TODO */); if (!status.ok()) { - LOG(WARNING) << "failed to move dir: " << move_dir_req.src - << ", tablet id: " << move_dir_req.tablet_id - << ", signature: " << agent_task_req.signature - << ", job id: " << move_dir_req.job_id; + LOG_WARNING("failed to move dir") + .tag("signature", agent_task_req.signature) + .tag("job_id", move_dir_req.job_id) + .tag("tablet_id", move_dir_req.tablet_id) + .tag("src", move_dir_req.src) + .error(status); } else { - LOG(INFO) << "finished to move dir:" << move_dir_req.src - << ", tablet_id:" << move_dir_req.tablet_id - << ", signature:" << agent_task_req.signature - << ", job id:" << move_dir_req.job_id; + LOG_INFO("successfully move dir") + .tag("signature", agent_task_req.signature) + .tag("job_id", move_dir_req.job_id) + .tag("tablet_id", move_dir_req.tablet_id) + .tag("src", move_dir_req.src); } TFinishTaskRequest finish_task_request; @@ -1582,19 +1494,10 @@ Status TaskWorkerPool::_move_dir(const TTabletId tablet_id, const std::string& s bool overwrite) { TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id); if (tablet == nullptr) { - LOG(INFO) << "failed to get tablet. tablet_id:" << tablet_id; return Status::InvalidArgument("Could not find tablet"); } - SnapshotLoader loader(_env, job_id, tablet_id); - Status status = loader.move(src, tablet, overwrite); - - if (!status.ok()) { - LOG(WARNING) << "move failed. job id: " << job_id << ", msg: " << status.get_error_msg(); - return status; - } - - return Status::OK(); + return loader.move(src, tablet, overwrite); } void TaskWorkerPool::_handle_report(TReportRequest& request, ReportType type) { @@ -1602,25 +1505,20 @@ void TaskWorkerPool::_handle_report(TReportRequest& request, ReportType type) { Status status = _master_client->report(request, &result); bool is_report_success = false; if (!status.ok()) { - LOG(WARNING) << "report " << TYPE_STRING(type) << " failed. status: " << status - << ", master host: " << _master_info.network_address.hostname - << ", port:" << _master_info.network_address.port; + LOG_WARNING("failed to report {}", TYPE_STRING(type)) + .tag("host", _master_info.network_address.hostname) + .tag("port", _master_info.network_address.port) + .error(status); } else if (result.status.status_code != TStatusCode::OK) { - std::stringstream ss; - if (!result.status.error_msgs.empty()) { - ss << result.status.error_msgs[0]; - for (int i = 1; i < result.status.error_msgs.size(); i++) { - ss << "," << result.status.error_msgs[i]; - } - } - LOG(WARNING) << "finish report " << TYPE_STRING(type) - << " failed. status:" << result.status.status_code - << ", error msg:" << ss.str(); + LOG_WARNING("failed to report {}", TYPE_STRING(type)) + .tag("host", _master_info.network_address.hostname) + .tag("port", _master_info.network_address.port) + .error(result.status); } else { is_report_success = true; - LOG(INFO) << "finish report " << TYPE_STRING(type) - << ". master host: " << _master_info.network_address.hostname - << ", port: " << _master_info.network_address.port; + LOG_INFO("successfully report {}", TYPE_STRING(type)) + .tag("host", _master_info.network_address.hostname) + .tag("port", _master_info.network_address.port); } switch (type) { case TASK: @@ -1670,8 +1568,8 @@ void TaskWorkerPool::_submit_table_compaction_worker_thread_callback() { _tasks.pop_front(); } - LOG(INFO) << "get compaction task. signature:" << agent_task_req.signature - << ", compaction type:" << compaction_req.type; + LOG(INFO) << "get compaction task. signature=" << agent_task_req.signature + << ", compaction_type=" << compaction_req.type; CompactionType compaction_type; if (compaction_req.type == "base") { @@ -1685,8 +1583,8 @@ void TaskWorkerPool::_submit_table_compaction_worker_thread_callback() { if (tablet_ptr != nullptr) { auto data_dir = tablet_ptr->data_dir(); if (!tablet_ptr->can_do_compaction(data_dir->path_hash(), compaction_type)) { - LOG(WARNING) << "can not do compaction: " << tablet_ptr->tablet_id() - << ", compaction type: " << compaction_type; + LOG(WARNING) << "could not do compaction. tablet_id=" << tablet_ptr->tablet_id() + << ", compaction_type=" << compaction_type; _remove_task_info(agent_task_req.task_type, agent_task_req.signature); continue; } @@ -1694,7 +1592,7 @@ void TaskWorkerPool::_submit_table_compaction_worker_thread_callback() { Status status = StorageEngine::instance()->submit_compaction_task(tablet_ptr, compaction_type); if (!status.ok()) { - LOG(WARNING) << "failed to submit table compaction task. " << status.to_string(); + LOG(WARNING) << "failed to submit table compaction task. error=" << status; } _remove_task_info(agent_task_req.task_type, agent_task_req.signature); } @@ -1747,7 +1645,7 @@ void TaskWorkerPool::_storage_refresh_storage_policy_worker_thread_callback() { policy_ptr->s3_request_timeout_ms = iter.s3_storage_param.s3_request_timeout_ms; policy_ptr->md5_sum = iter.md5_checksum; - LOG_EVERY_N(INFO, 12) << "refresh storage policy task, policy " << *policy_ptr; + LOG_EVERY_N(INFO, 12) << "refresh storage policy task. policy=" << *policy_ptr; spm->periodic_put(iter.policy_name, policy_ptr); } } @@ -1789,7 +1687,7 @@ void TaskWorkerPool::_storage_update_storage_policy_worker_thread_callback() { get_storage_policy_req.s3_storage_param.s3_request_timeout_ms; policy_ptr->md5_sum = get_storage_policy_req.md5_checksum; - LOG(INFO) << "get storage update policy task, update policy " << *policy_ptr; + LOG(INFO) << "get storage update policy task. policy=" << *policy_ptr; spm->update(get_storage_policy_req.policy_name, policy_ptr); _remove_task_info(agent_task_req.task_type, agent_task_req.signature); diff --git a/be/src/common/daemon.cpp b/be/src/common/daemon.cpp index e5525d264c..4fb1abe628 100644 --- a/be/src/common/daemon.cpp +++ b/be/src/common/daemon.cpp @@ -22,6 +22,7 @@ #include #include "common/config.h" +#include "common/logging.h" #include "exprs/array_functions.h" #include "exprs/bitmap_function.h" #include "exprs/cast_functions.h" @@ -56,7 +57,6 @@ #include "util/debug_util.h" #include "util/disk_info.h" #include "util/doris_metrics.h" -#include "util/logging.h" #include "util/mem_info.h" #include "util/network_util.h" #include "util/system_metrics.h" diff --git a/be/src/common/logconfig.cpp b/be/src/common/logconfig.cpp index b176e8ca0f..829be25dfb 100644 --- a/be/src/common/logconfig.cpp +++ b/be/src/common/logconfig.cpp @@ -25,8 +25,7 @@ #include #include "common/config.h" -#include "gutil/stringprintf.h" -#include "util/logging.h" +#include "common/logging.h" namespace doris { @@ -150,31 +149,4 @@ void shutdown_logging() { google::ShutdownGoogleLogging(); } -std::string FormatTimestampForLog(MicrosecondsInt64 micros_since_epoch) { - time_t secs_since_epoch = micros_since_epoch / 1000000; - int usecs = micros_since_epoch % 1000000; - struct tm tm_time; - localtime_r(&secs_since_epoch, &tm_time); - - return StringPrintf("%02d%02d %02d:%02d:%02d.%06d", 1 + tm_time.tm_mon, tm_time.tm_mday, - tm_time.tm_hour, tm_time.tm_min, tm_time.tm_sec, usecs); -} - -/// Custom your log format here -void TaggableLogger::flush() { - _stream << _message; - Tags* head = _tags; - Tags* next; - while (head) { - next = head->next; - _stream << "|" << head->key << "=" << head->value; - delete head; - head = next; - } -} - -/// Modify these tag names to suit your log format and collector. -const std::string TaggableLogger::QUERY_ID = "query_id"; -const std::string TaggableLogger::INSTANCE_ID = "instance_id"; - } // namespace doris diff --git a/be/src/common/logging.h b/be/src/common/logging.h index 573af24eef..7c025b7e89 100644 --- a/be/src/common/logging.h +++ b/be/src/common/logging.h @@ -57,3 +57,56 @@ << static_cast(b) << " ]" #include + +#include "util/uid_util.h" + +namespace doris { + +// glog doesn't allow multiple invocations of InitGoogleLogging. This method conditionally +// calls InitGoogleLogging only if it hasn't been called before. +bool init_glog(const char* basename); + +// Shuts down the google logging library. Call before exit to ensure that log files are +// flushed. May only be called once. +void shutdown_logging(); + +/// Wrap a glog stream and tag on the log. usage: +/// LOG_INFO("here is an info for a {} query", query_type).tag("query_id", queryId); +#define LOG_INFO(...) doris::TaggableLogger(LOG(INFO), ##__VA_ARGS__) +#define LOG_WARNING(...) doris::TaggableLogger(LOG(WARNING), ##__VA_ARGS__) +#define LOG_ERROR(...) doris::TaggableLogger(LOG(ERROR), ##__VA_ARGS__) +#define LOG_FATAL(...) doris::TaggableLogger(LOG(FATAL), ##__VA_ARGS__) + +class TaggableLogger { +public: + template + TaggableLogger(std::ostream& stream, std::string_view fmt, Args&&... args) : _stream(stream) { + if constexpr (sizeof...(args) == 0) { + _stream << fmt; + } else { + _stream << fmt::format(fmt, std::forward(args)...); + } + }; + + template + TaggableLogger& tag(std::string_view key, const V& value) { + _stream << '|' << key << '='; + if constexpr (std::is_same_v || std::is_same_v) { + _stream << print_id(value); + } else { + _stream << value; + } + return *this; + } + + template + TaggableLogger& error(const E& error) { + _stream << "|error=" << error; + return *this; + } + +private: + std::ostream& _stream; +}; + +} // namespace doris diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index 160cf4121c..5ebfc72e8d 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -222,12 +222,14 @@ struct Version { bool contains(const Version& other) const { return first <= other.first && second >= other.second; } + + std::string to_string() const { return fmt::format("[{}-{}]", first, second); } }; using Versions = std::vector; inline std::ostream& operator<<(std::ostream& os, const Version& version) { - return os << "[" << version.first << "-" << version.second << "]"; + return os << version.to_string(); } // used for hash-struct of hash_map. diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index c0e001dbae..4e594c8a62 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -1048,22 +1048,15 @@ Status StorageEngine::execute_task(EngineTask* task) { { auto wrlocks = lock_related_tablets(); - Status prepare_status = task->prepare(); - if (prepare_status != Status::OK()) { - return prepare_status; - } + RETURN_IF_ERROR(task->prepare()); } // do execute work without lock - Status exec_status = task->execute(); - if (exec_status != Status::OK()) { - return exec_status; - } + RETURN_IF_ERROR(task->execute()); { auto wrlocks = lock_related_tablets(); - Status fin_status = task->finish(); - return fin_status; + return task->finish(); } } diff --git a/be/src/olap/task/engine_alter_tablet_task.cpp b/be/src/olap/task/engine_alter_tablet_task.cpp index 7689df96b3..8164049296 100644 --- a/be/src/olap/task/engine_alter_tablet_task.cpp +++ b/be/src/olap/task/engine_alter_tablet_task.cpp @@ -38,22 +38,10 @@ Status EngineAlterTabletTask::execute() { DorisMetrics::instance()->create_rollup_requests_total->increment(1); Status res = SchemaChangeHandler::process_alter_tablet_v2(_alter_tablet_req); - if (!res.ok()) { - LOG(WARNING) << "failed to do alter task. res=" << res - << " base_tablet_id=" << _alter_tablet_req.base_tablet_id - << ", base_schema_hash=" << _alter_tablet_req.base_schema_hash - << ", new_tablet_id=" << _alter_tablet_req.new_tablet_id - << ", new_schema_hash=" << _alter_tablet_req.new_schema_hash; DorisMetrics::instance()->create_rollup_requests_failed->increment(1); return res; } - - LOG(INFO) << "success to create new alter tablet. res=" << res - << " base_tablet_id=" << _alter_tablet_req.base_tablet_id - << ", base_schema_hash=" << _alter_tablet_req.base_schema_hash - << ", new_tablet_id=" << _alter_tablet_req.new_tablet_id - << ", new_schema_hash=" << _alter_tablet_req.new_schema_hash; return res; } // execute diff --git a/be/src/olap/task/engine_batch_load_task.cpp b/be/src/olap/task/engine_batch_load_task.cpp index c478adcd8c..9b56055339 100644 --- a/be/src/olap/task/engine_batch_load_task.cpp +++ b/be/src/olap/task/engine_batch_load_task.cpp @@ -46,13 +46,8 @@ using std::vector; namespace doris { -EngineBatchLoadTask::EngineBatchLoadTask(TPushReq& push_req, std::vector* tablet_infos, - int64_t signature, Status* res_status) - : _push_req(push_req), - _tablet_infos(tablet_infos), - _signature(signature), - _res_status(res_status) { - _download_status = Status::OK(); +EngineBatchLoadTask::EngineBatchLoadTask(TPushReq& push_req, std::vector* tablet_infos) + : _push_req(push_req), _tablet_infos(tablet_infos) { _mem_tracker = std::make_shared( -1, fmt::format("EngineBatchLoadTask#pushType={}:tabletId={}", _push_req.push_type, @@ -64,34 +59,24 @@ EngineBatchLoadTask::~EngineBatchLoadTask() {} Status EngineBatchLoadTask::execute() { SCOPED_ATTACH_TASK(_mem_tracker, ThreadContext::TaskType::STORAGE); - Status status = Status::OK(); + Status status; if (_push_req.push_type == TPushType::LOAD || _push_req.push_type == TPushType::LOAD_V2) { - status = _init(); - if (status.ok()) { - uint32_t retry_time = 0; - while (retry_time < PUSH_MAX_RETRY) { - status = _process(); - // Internal error, need retry - if (!status.ok()) { - LOG(WARNING) << "push internal error, need retry.signature: " << _signature; - retry_time += 1; - } else { - break; - } + RETURN_IF_ERROR(_init()); + uint32_t retry_time = 0; + while (retry_time < PUSH_MAX_RETRY) { + status = _process(); + // Internal error, need retry + if (status.ok()) { + break; } + retry_time += 1; } } else if (_push_req.push_type == TPushType::DELETE) { - Status delete_data_status = _delete_data(_push_req, _tablet_infos); - if (delete_data_status != Status::OK()) { - LOG(WARNING) << "delete data failed. status:" << delete_data_status - << " signature:" << _signature; - status = delete_data_status; - } + status = _delete_data(_push_req, _tablet_infos); } else { - status = Status::InvalidArgument("Not support task type"); + return Status::InvalidArgument("Not support task type"); } - *_res_status = status; - return Status::OK(); + return status; } Status EngineBatchLoadTask::_init() { @@ -106,9 +91,6 @@ Status EngineBatchLoadTask::_init() { TabletSharedPtr tablet; tablet = StorageEngine::instance()->tablet_manager()->get_tablet(_push_req.tablet_id); if (tablet == nullptr) { - LOG(WARNING) << "get tables failed. " - << "tablet_id: " << _push_req.tablet_id - << ", schema_hash: " << _push_req.schema_hash; return Status::InvalidArgument("Could not find tablet {}", _push_req.tablet_id); } @@ -132,9 +114,7 @@ Status EngineBatchLoadTask::_init() { string tmp_file_dir; string root_path = tablet->data_dir()->path(); status = _get_tmp_file_dir(root_path, &tmp_file_dir); - if (!status.ok()) { - LOG(WARNING) << "get local path failed. tmp file dir: " << tmp_file_dir; return status; } string tmp_file_name; @@ -146,7 +126,6 @@ Status EngineBatchLoadTask::_init() { // Get replica root path Status EngineBatchLoadTask::_get_tmp_file_dir(const string& root_path, string* download_path) { - Status status = Status::OK(); *download_path = root_path + "/" + DPP_PREFIX; // Check path exist @@ -158,13 +137,10 @@ Status EngineBatchLoadTask::_get_tmp_file_dir(const string& root_path, string* d std::filesystem::create_directories(*download_path, ec); if (ec) { - status = Status::IOError("Create download dir failed {}", *download_path); - LOG(WARNING) << "create download dir failed.path: " << *download_path - << ", error code: " << ec; + return Status::IOError("Create download dir failed {}", *download_path); } } - - return status; + return Status::OK(); } void EngineBatchLoadTask::_get_file_name_from_path(const string& file_path, string* file_name) { @@ -176,7 +152,6 @@ void EngineBatchLoadTask::_get_file_name_from_path(const string& file_path, stri Status EngineBatchLoadTask::_process() { Status status = Status::OK(); if (!_is_init) { - LOG(WARNING) << "has not init yet. tablet_id: " << _push_req.tablet_id; return Status::InternalError("Tablet has not init yet"); } // Remote file not empty, need to download @@ -219,9 +194,9 @@ Status EngineBatchLoadTask::_process() { // Check file size uint64_t local_file_size = std::filesystem::file_size(_local_file_path); if (file_size != local_file_size) { - LOG(WARNING) << "download_file size error. file_size=" << file_size - << ", local_file_size=" << local_file_size; - return Status::InternalError("downloaded file's size isn't right"); + return Status::InternalError( + "download_file size error. file_size={}, local_file_size={}", file_size, + local_file_size); } } // NOTE: change http_file_path is not good design @@ -231,38 +206,35 @@ Status EngineBatchLoadTask::_process() { MonotonicStopWatch stopwatch; stopwatch.start(); - auto st = HttpClient::execute_with_retry(MAX_RETRY, 1, download_cb); + status = HttpClient::execute_with_retry(MAX_RETRY, 1, download_cb); auto cost = stopwatch.elapsed_time(); if (cost <= 0) { cost = 1; } - if (st.ok() && !is_timeout) { + if (status.ok() && !is_timeout) { double rate = -1.0; if (_push_req.__isset.http_file_size) { rate = (double)_push_req.http_file_size / (cost / 1000 / 1000 / 1000) / 1024; } - LOG(INFO) << "down load file success. local_file=" << _local_file_path + LOG(INFO) << "succeed to download file. local_file=" << _local_file_path << ", remote_file=" << _remote_file_path << ", tablet_id" << _push_req.tablet_id << ", cost=" << cost / 1000 << "us, file_size" << _push_req.http_file_size << ", download rage:" << rate << "KB/s"; } else { - LOG(WARNING) << "down load file failed. remote_file=" << _remote_file_path + LOG(WARNING) << "download file failed. remote_file=" << _remote_file_path << ", tablet=" << _push_req.tablet_id << ", cost=" << cost / 1000 - << "us, errmsg=" << st.get_error_msg() << ", is_timeout=" << is_timeout; - status = Status::InternalError("Download file failed"); + << "us, is_timeout=" << is_timeout; } } if (status.ok()) { // Load delta file time_t push_begin = time(nullptr); - Status push_status = _push(_push_req, _tablet_infos); + status = _push(_push_req, _tablet_infos); time_t push_finish = time(nullptr); LOG(INFO) << "Push finish, cost time: " << (push_finish - push_begin); - if (push_status.precise_code() == OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST) { + if (status.precise_code() == OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST) { status = Status::OK(); - } else if (push_status != Status::OK()) { - status = push_status; } } @@ -284,18 +256,15 @@ Status EngineBatchLoadTask::_push(const TPushReq& request, << ", version=" << request.version; if (tablet_info_vec == nullptr) { - LOG(WARNING) << "invalid output parameter which is nullptr pointer."; DorisMetrics::instance()->push_requests_fail_total->increment(1); - return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR); + return Status::InvalidArgument("invalid tablet_info_vec which is nullptr"); } TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet(request.tablet_id); if (tablet == nullptr) { - LOG(WARNING) << "false to find tablet. tablet=" << request.tablet_id - << ", schema_hash=" << request.schema_hash; DorisMetrics::instance()->push_requests_fail_total->increment(1); - return Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND); + return Status::InternalError("could not find tablet {}", request.tablet_id); } PushType type = PUSH_NORMAL; @@ -305,28 +274,22 @@ Status EngineBatchLoadTask::_push(const TPushReq& request, int64_t duration_ns = 0; PushHandler push_handler; - if (request.__isset.transaction_id) { - { - SCOPED_RAW_TIMER(&duration_ns); - res = push_handler.process_streaming_ingestion(tablet, request, type, tablet_info_vec); - } - } else { - { - SCOPED_RAW_TIMER(&duration_ns); - res = Status::OLAPInternalError(OLAP_ERR_PUSH_BATCH_PROCESS_REMOVED); - } + if (!request.__isset.transaction_id) { + return Status::InvalidArgument("transaction_id is not set"); + } + { + SCOPED_RAW_TIMER(&duration_ns); + res = push_handler.process_streaming_ingestion(tablet, request, type, tablet_info_vec); } if (!res.ok()) { - LOG(WARNING) << "fail to push delta, " - << "transaction_id=" << request.transaction_id - << " tablet=" << tablet->full_name() + LOG(WARNING) << "failed to push delta, transaction_id=" << request.transaction_id + << ", tablet=" << tablet->full_name() << ", cost=" << PrettyPrinter::print(duration_ns, TUnit::TIME_NS); DorisMetrics::instance()->push_requests_fail_total->increment(1); } else { - LOG(INFO) << "success to push delta, " - << "transaction_id=" << request.transaction_id - << " tablet=" << tablet->full_name() + LOG(INFO) << "succeed to push delta, transaction_id=" << request.transaction_id + << ", tablet=" << tablet->full_name() << ", cost=" << PrettyPrinter::print(duration_ns, TUnit::TIME_NS); DorisMetrics::instance()->push_requests_success_total->increment(1); DorisMetrics::instance()->push_request_duration_us->increment(duration_ns / 1000); @@ -344,35 +307,26 @@ Status EngineBatchLoadTask::_delete_data(const TPushReq& request, Status res = Status::OK(); if (tablet_info_vec == nullptr) { - LOG(WARNING) << "invalid tablet info parameter which is nullptr pointer."; - return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR); + return Status::InvalidArgument("invalid tablet_info_vec which is nullptr"); } // 1. Get all tablets with same tablet_id TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet(request.tablet_id); if (tablet == nullptr) { - LOG(WARNING) << "can't find tablet. tablet=" << request.tablet_id; - return Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND); + return Status::InternalError("could not find tablet {}", request.tablet_id); } // 2. Process delete data by push interface PushHandler push_handler; - if (request.__isset.transaction_id) { - res = push_handler.process_streaming_ingestion(tablet, request, PUSH_FOR_DELETE, - tablet_info_vec); - } else { - res = Status::OLAPInternalError(OLAP_ERR_PUSH_BATCH_PROCESS_REMOVED); + if (!request.__isset.transaction_id) { + return Status::InvalidArgument("transaction_id is not set"); } - + res = push_handler.process_streaming_ingestion(tablet, request, PUSH_FOR_DELETE, + tablet_info_vec); if (!res.ok()) { - LOG(WARNING) << "fail to push empty version for delete data. " - << "res=" << res << "tablet=" << tablet->full_name(); DorisMetrics::instance()->delete_requests_failed->increment(1); - return res; } - - LOG(INFO) << "finish to process delete data. res=" << res; return res; } diff --git a/be/src/olap/task/engine_batch_load_task.h b/be/src/olap/task/engine_batch_load_task.h index e2aba71f00..a37d6033b2 100644 --- a/be/src/olap/task/engine_batch_load_task.h +++ b/be/src/olap/task/engine_batch_load_task.h @@ -39,8 +39,7 @@ class StorageEngine; class EngineBatchLoadTask : public EngineTask { public: - EngineBatchLoadTask(TPushReq& push_req, std::vector* tablet_infos, - int64_t signature, Status* res_status); + EngineBatchLoadTask(TPushReq& push_req, std::vector* tablet_infos); virtual ~EngineBatchLoadTask(); virtual Status execute(); @@ -71,9 +70,6 @@ private: bool _is_init = false; TPushReq& _push_req; std::vector* _tablet_infos; - int64_t _signature; - Status _download_status; - Status* _res_status; std::string _remote_file_path; std::string _local_file_path; std::shared_ptr _mem_tracker; diff --git a/be/src/olap/task/engine_checksum_task.cpp b/be/src/olap/task/engine_checksum_task.cpp index 3add201854..b6f25bb2a4 100644 --- a/be/src/olap/task/engine_checksum_task.cpp +++ b/be/src/olap/task/engine_checksum_task.cpp @@ -40,18 +40,14 @@ Status EngineChecksumTask::_compute_checksum() { LOG(INFO) << "begin to process compute checksum." << "tablet_id=" << _tablet_id << ", schema_hash=" << _schema_hash << ", version=" << _version; - Status res = Status::OK(); if (_checksum == nullptr) { - LOG(WARNING) << "invalid output parameter which is null pointer."; - return Status::OLAPInternalError(OLAP_ERR_CE_CMD_PARAMS_ERROR); + return Status::InvalidArgument("invalid checksum which is nullptr"); } TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet(_tablet_id); - if (nullptr == tablet.get()) { - LOG(WARNING) << "can't find tablet. [tablet_id=" << _tablet_id - << " schema_hash=" << _schema_hash << "]"; - return Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND); + if (nullptr == tablet) { + return Status::InternalError("could not find tablet {}", _tablet_id); } TupleReader reader; @@ -65,49 +61,33 @@ Status EngineChecksumTask::_compute_checksum() { const RowsetSharedPtr message = tablet->rowset_with_max_version(); if (message == nullptr) { LOG(FATAL) << "fail to get latest version. tablet_id=" << _tablet_id; - return Status::OLAPInternalError(OLAP_ERR_WRITE_PROTOBUF_ERROR); } - Status acquire_reader_st = - tablet->capture_rs_readers(reader_params.version, &reader_params.rs_readers); - if (acquire_reader_st != Status::OK()) { - LOG(WARNING) << "fail to init reader. tablet=" << tablet->full_name() - << "res=" << acquire_reader_st; - return acquire_reader_st; - } + RETURN_IF_ERROR( + tablet->capture_rs_readers(reader_params.version, &reader_params.rs_readers)); } for (size_t i = 0; i < tablet->tablet_schema()->num_columns(); ++i) { reader_params.return_columns.push_back(i); } - res = reader.init(reader_params); - if (!res.ok()) { - LOG(WARNING) << "initiate reader fail. res = " << res; - return res; - } + RETURN_IF_ERROR(reader.init(reader_params)); RowCursor row; std::unique_ptr mem_pool(new MemPool()); std::unique_ptr agg_object_pool(new ObjectPool()); - res = row.init(tablet->tablet_schema(), reader_params.return_columns); - if (!res.ok()) { - LOG(WARNING) << "failed to init row cursor. res = " << res; - return res; - } + RETURN_IF_ERROR(row.init(tablet->tablet_schema(), reader_params.return_columns)); + row.allocate_memory_for_string_type(tablet->tablet_schema()); bool eof = false; uint32_t row_checksum = 0; while (true) { - Status res = - reader.next_row_with_aggregation(&row, mem_pool.get(), agg_object_pool.get(), &eof); - if (res.ok() && eof) { + RETURN_IF_ERROR(reader.next_row_with_aggregation(&row, mem_pool.get(), + agg_object_pool.get(), &eof)); + if (eof) { VLOG_NOTICE << "reader reads to the end."; break; - } else if (!res.ok()) { - LOG(WARNING) << "fail to read in reader. res = " << res; - return res; } // The value of checksum is independent of the sorting of data rows. row_checksum ^= hash_row(row, 0); diff --git a/be/src/olap/task/engine_clone_task.cpp b/be/src/olap/task/engine_clone_task.cpp index becae02264..7130c97e9c 100644 --- a/be/src/olap/task/engine_clone_task.cpp +++ b/be/src/olap/task/engine_clone_task.cpp @@ -24,18 +24,18 @@ #include "gen_cpp/Types_constants.h" #include "gutil/strings/split.h" #include "gutil/strings/stringpiece.h" -#include "gutil/strings/substitute.h" #include "http/http_client.h" +#include "io/fs/local_file_system.h" #include "olap/rowset/rowset.h" #include "olap/rowset/rowset_factory.h" #include "olap/snapshot_manager.h" #include "runtime/client_cache.h" #include "runtime/thread_context.h" +#include "util/defer_op.h" #include "util/thrift_rpc_helper.h" using std::set; using std::stringstream; -using strings::Substitute; using strings::Split; using strings::SkipWhitespace; @@ -49,12 +49,9 @@ const uint32_t LIST_REMOTE_FILE_TIMEOUT = 15; const uint32_t GET_LENGTH_TIMEOUT = 10; EngineCloneTask::EngineCloneTask(const TCloneReq& clone_req, const TMasterInfo& master_info, - int64_t signature, std::vector* error_msgs, - std::vector* tablet_infos, Status* res_status) + int64_t signature, std::vector* tablet_infos) : _clone_req(clone_req), - _error_msgs(error_msgs), _tablet_infos(tablet_infos), - _res_status(res_status), _signature(signature), _master_info(master_info) { _mem_tracker = std::make_shared( @@ -96,176 +93,108 @@ Status EngineCloneTask::_do_clone() { // if missed version size is 0, then it is useless to clone from remote be, it means local data is // completed. Or remote be will just return header not the rowset files. clone will failed. - if (missed_versions.size() == 0) { - LOG(INFO) << "missed version size = 0, skip clone and return success. tablet id=" + if (missed_versions.empty()) { + LOG(INFO) << "missed version size = 0, skip clone and return success. tablet_id=" << _clone_req.tablet_id; - _set_tablet_info(Status::OK(), is_new_tablet); + _set_tablet_info(is_new_tablet); return Status::OK(); } + LOG(INFO) << "clone to existed tablet. missed_versions_size=" << missed_versions.size() + << ", allow_incremental_clone=" << allow_incremental_clone + << ", signature=" << _signature << ", tablet_id=" << _clone_req.tablet_id + << ", committed_version=" << _clone_req.committed_version; + // try to download missing version from src backend. // if tablet on src backend does not contains missing version, it will download all versions, // and set allow_incremental_clone to false - status = _make_and_download_snapshots(*(tablet->data_dir()), local_data_path, &src_host, - &src_file_path, _error_msgs, &missed_versions, - &allow_incremental_clone); - - LOG(INFO) << "tablet exist with number of missing version: " << missed_versions.size() - << ", try to incremental clone succeed: " << allow_incremental_clone - << ", signature: " << _signature << ", tablet id: " << _clone_req.tablet_id - << ", schema hash: " << _clone_req.schema_hash - << ", clone version: " << _clone_req.committed_version - << ", download snapshot: " << status; - - if (status.ok()) { - Status olap_status = - _finish_clone(tablet.get(), local_data_path, _clone_req.committed_version, - allow_incremental_clone); - if (!olap_status.ok()) { - LOG(WARNING) << "failed to finish clone. [table=" << tablet->full_name() - << " res=" << olap_status << "]"; - _error_msgs->push_back("clone error."); - status = Status::InternalError("Failed to finish clone"); - } - } + RETURN_IF_ERROR(_make_and_download_snapshots(*(tablet->data_dir()), local_data_path, + &src_host, &src_file_path, &missed_versions, + &allow_incremental_clone)); + RETURN_IF_ERROR(_finish_clone(tablet.get(), local_data_path, _clone_req.committed_version, + allow_incremental_clone)); } else { LOG(INFO) << "clone tablet not exist, begin clone a new tablet from remote be. " - << "signature:" << _signature << ", tablet_id:" << _clone_req.tablet_id - << ", schema_hash:" << _clone_req.schema_hash - << ", committed_version:" << _clone_req.committed_version; + << "signature=" << _signature << ", tablet_id=" << _clone_req.tablet_id + << ", committed_version=" << _clone_req.committed_version; // create a new tablet in this be // Get local disk from olap string local_shard_root_path; DataDir* store = nullptr; - Status olap_status = StorageEngine::instance()->obtain_shard_path( - _clone_req.storage_medium, &local_shard_root_path, &store); - if (!olap_status.ok()) { - LOG(WARNING) << "clone get local root path failed. signature: " << _signature; - _error_msgs->push_back("clone get local root path failed."); - status = Status::InternalError("Clone to get local root path failed"); - } - std::stringstream tablet_dir_stream; - tablet_dir_stream << local_shard_root_path << "/" << _clone_req.tablet_id << "/" - << _clone_req.schema_hash; + RETURN_IF_ERROR(StorageEngine::instance()->obtain_shard_path( + _clone_req.storage_medium, &local_shard_root_path, &store)); + auto tablet_dir = fmt::format("{}/{}/{}", local_shard_root_path, _clone_req.tablet_id, + _clone_req.schema_hash); - if (status.ok()) { - bool allow_incremental_clone = false; - status = _make_and_download_snapshots(*store, tablet_dir_stream.str(), &src_host, - &src_file_path, _error_msgs, nullptr, - &allow_incremental_clone); - } - - if (status.ok()) { - LOG(INFO) << "clone copy done. src_host: " << src_host.host - << " src_file_path: " << src_file_path; - std::stringstream schema_hash_path_stream; - schema_hash_path_stream << local_shard_root_path << "/" << _clone_req.tablet_id << "/" - << _clone_req.schema_hash; - string header_path = TabletMeta::construct_header_file_path( - schema_hash_path_stream.str(), _clone_req.tablet_id); - Status reset_id_status = TabletMeta::reset_tablet_uid(header_path); - if (reset_id_status != Status::OK()) { - LOG(WARNING) << "errors while set tablet uid: '" << header_path; - _error_msgs->push_back("errors while set tablet uid."); - status = Status::InternalError("Errors while set tablet uid"); - } else { - Status load_header_status = - StorageEngine::instance()->tablet_manager()->load_tablet_from_dir( - store, _clone_req.tablet_id, _clone_req.schema_hash, - schema_hash_path_stream.str(), false); - if (load_header_status != Status::OK()) { - LOG(WARNING) << "load header failed. local_shard_root_path: '" - << local_shard_root_path - << "' schema_hash: " << _clone_req.schema_hash - << ". status: " << load_header_status - << ". signature: " << _signature; - _error_msgs->push_back("load header failed."); - status = Status::InternalError("Load tablet header failed"); - } + Defer remove_useless_dir {[&] { + if (status.ok()) { + return; } - // clone success, delete .hdr file because tablet meta is stored in rocksdb - string cloned_meta_file = - tablet_dir_stream.str() + "/" + std::to_string(_clone_req.tablet_id) + ".hdr"; - FileUtils::remove(cloned_meta_file); - } - // Clean useless dir, if failed, ignore it. - if (!status.ok()) { - std::stringstream local_data_path_stream; - local_data_path_stream << local_shard_root_path << "/" << _clone_req.tablet_id; - string local_data_path = local_data_path_stream.str(); - LOG(INFO) << "clone failed. want to delete local dir: " << local_data_path + LOG(INFO) << "clone failed. want to delete local dir: " << tablet_dir << ". signature: " << _signature; - try { - std::filesystem::path local_path(local_data_path); - if (std::filesystem::exists(local_path)) { - std::filesystem::remove_all(local_path); - } - } catch (std::filesystem::filesystem_error& e) { - // Ignore the error, OLAP will delete it - LOG(WARNING) << "clone delete useless dir failed. " - << " error: " << e.what() << " local dir: " << local_data_path.c_str() - << " signature: " << _signature; - } + WARN_IF_ERROR(io::global_local_filesystem()->delete_directory(tablet_dir), + "failed to delete useless clone dir "); + }}; + + bool allow_incremental_clone = false; + status = _make_and_download_snapshots(*store, tablet_dir, &src_host, &src_file_path, + nullptr, &allow_incremental_clone); + if (!status.ok()) { + return status; } + + LOG(INFO) << "clone copy done. src_host: " << src_host.host + << " src_file_path: " << src_file_path; + string header_path = + TabletMeta::construct_header_file_path(tablet_dir, _clone_req.tablet_id); + status = TabletMeta::reset_tablet_uid(header_path); + if (!status.ok()) { + return status; + } + status = StorageEngine::instance()->tablet_manager()->load_tablet_from_dir( + store, _clone_req.tablet_id, _clone_req.schema_hash, tablet_dir, false); + if (!status.ok()) { + return status; + } + // clone success, delete .hdr file because tablet meta is stored in rocksdb + FileUtils::remove(header_path); } - _set_tablet_info(status, is_new_tablet); - return Status::OK(); + return _set_tablet_info(is_new_tablet); } -void EngineCloneTask::_set_tablet_info(Status status, bool is_new_tablet) { +Status EngineCloneTask::_set_tablet_info(bool is_new_tablet) { // Get clone tablet info - if (status.ok()) { - TTabletInfo tablet_info; - tablet_info.__set_tablet_id(_clone_req.tablet_id); - tablet_info.__set_replica_id(_clone_req.replica_id); - tablet_info.__set_schema_hash(_clone_req.schema_hash); - Status get_tablet_info_status = - StorageEngine::instance()->tablet_manager()->report_tablet_info(&tablet_info); - if (get_tablet_info_status != Status::OK()) { - LOG(WARNING) << "clone success, but get tablet info failed." - << " tablet id: " << _clone_req.tablet_id - << ", replica_id:" << _clone_req.replica_id - << " schema hash: " << _clone_req.schema_hash - << " signature: " << _signature; - _error_msgs->push_back("clone success, but get tablet info failed."); - status = Status::InternalError("Clone success but get tablet info failed"); - } else if (_clone_req.__isset.committed_version && - tablet_info.version < _clone_req.committed_version) { - LOG(WARNING) << "failed to clone tablet. tablet_id:" << _clone_req.tablet_id + TTabletInfo tablet_info; + tablet_info.__set_tablet_id(_clone_req.tablet_id); + tablet_info.__set_replica_id(_clone_req.replica_id); + tablet_info.__set_schema_hash(_clone_req.schema_hash); + RETURN_IF_ERROR(StorageEngine::instance()->tablet_manager()->report_tablet_info(&tablet_info)); + if (_clone_req.__isset.committed_version && + tablet_info.version < _clone_req.committed_version) { + // if it is a new tablet and clone failed, then remove the tablet + // if it is incremental clone, then must not drop the tablet + if (is_new_tablet) { + // we need to check if this cloned table's version is what we expect. + // if not, maybe this is a stale remaining table which is waiting for drop. + // we drop it. + LOG(WARNING) << "begin to drop the stale tablet. tablet_id:" << _clone_req.tablet_id << ", replica_id:" << _clone_req.replica_id << ", schema_hash:" << _clone_req.schema_hash << ", signature:" << _signature << ", version:" << tablet_info.version << ", expected_version: " << _clone_req.committed_version; - // if it is a new tablet and clone failed, then remove the tablet - // if it is incremental clone, then must not drop the tablet - if (is_new_tablet) { - // we need to check if this cloned table's version is what we expect. - // if not, maybe this is a stale remaining table which is waiting for drop. - // we drop it. - LOG(WARNING) << "begin to drop the stale tablet. tablet_id:" << _clone_req.tablet_id - << ", replica_id:" << _clone_req.replica_id - << ", schema_hash:" << _clone_req.schema_hash - << ", signature:" << _signature << ", version:" << tablet_info.version - << ", expected_version: " << _clone_req.committed_version; - Status drop_status = StorageEngine::instance()->tablet_manager()->drop_tablet( - _clone_req.tablet_id, _clone_req.replica_id, false); - if (drop_status != Status::OK() && - drop_status.precise_code() != OLAP_ERR_TABLE_NOT_FOUND) { - // just log - LOG(WARNING) << "drop stale cloned table failed! tablet id: " - << _clone_req.tablet_id; - } - } - status = Status::InternalError("Failed to clone tablet"); - } else { - LOG(INFO) << "clone get tablet info success. tablet_id:" << _clone_req.tablet_id - << ", schema_hash:" << _clone_req.schema_hash << ", signature:" << _signature - << ", version:" << tablet_info.version; - _tablet_infos->push_back(tablet_info); + WARN_IF_ERROR(StorageEngine::instance()->tablet_manager()->drop_tablet( + _clone_req.tablet_id, _clone_req.replica_id, false), + "drop stale cloned table failed"); } + return Status::InternalError("unexpected version. tablet version: {}, expected version: {}", + tablet_info.version, _clone_req.committed_version); } - *_res_status = status; + LOG(INFO) << "clone get tablet info success. tablet_id:" << _clone_req.tablet_id + << ", schema_hash:" << _clone_req.schema_hash << ", signature:" << _signature + << ", version:" << tablet_info.version; + _tablet_infos->push_back(tablet_info); + return Status::OK(); } /// This method will do following things: @@ -276,7 +205,6 @@ void EngineCloneTask::_set_tablet_info(Status status, bool is_new_tablet) { Status EngineCloneTask::_make_and_download_snapshots(DataDir& data_dir, const std::string& local_data_path, TBackend* src_host, string* snapshot_path, - std::vector* error_msgs, const std::vector* missed_versions, bool* allow_incremental_clone) { Status status = Status::OK(); @@ -292,23 +220,22 @@ Status EngineCloneTask::_make_and_download_snapshots(DataDir& data_dir, // Make snapshot in remote olap engine *src_host = src; // make snapshot - auto st = - _make_snapshot(src.host, src.be_port, _clone_req.tablet_id, _clone_req.schema_hash, - timeout_s, missed_versions, snapshot_path, allow_incremental_clone); - if (st.ok()) { - LOG(INFO) << "success to make snapshot. ip=" << src.host << ", port=" << src.be_port - << ", tablet=" << _clone_req.tablet_id - << ", schema_hash=" << _clone_req.schema_hash - << ", snapshot_path=" << *snapshot_path << ", signature=" << _signature; - status = Status::OK(); + status = _make_snapshot(src.host, src.be_port, _clone_req.tablet_id, _clone_req.schema_hash, + timeout_s, missed_versions, snapshot_path, allow_incremental_clone); + if (status.ok()) { + LOG_INFO("successfully make snapshot in remote BE") + .tag("host", src.host) + .tag("port", src.be_port) + .tag("tablet", _clone_req.tablet_id) + .tag("snapshot_path", *snapshot_path) + .tag("signature", _signature); } else { - LOG(WARNING) << "fail to make snapshot, ip=" << src.host << ", port=" << src.be_port - << ", tablet=" << _clone_req.tablet_id - << ", schema_hash=" << _clone_req.schema_hash - << ", signature=" << _signature << ", error=" << st.to_string(); - error_msgs->push_back("make snapshot failed. backend_ip: " + src_host->host); - - status = Status::InternalError("Failed to make snapshot"); + LOG_WARNING("failed to make snapshot in remote BE") + .tag("host", src.host) + .tag("port", src.be_port) + .tag("tablet", _clone_req.tablet_id) + .tag("signature", _signature) + .error(status); continue; } @@ -324,36 +251,28 @@ Status EngineCloneTask::_make_and_download_snapshots(DataDir& data_dir, remote_url_prefix = ss.str(); } - st = _download_files(&data_dir, remote_url_prefix, local_data_path); - if (!st.ok()) { - LOG(WARNING) << "fail to download and convert tablet, remote=" << remote_url_prefix - << ", error=" << st.to_string(); - status = Status::InternalError("Fail to download and convert tablet"); - // when there is an error, keep this program executing to release snapshot - } + status = _download_files(&data_dir, remote_url_prefix, local_data_path); + // when there is an error, keep this program executing to release snapshot if (status.ok()) { // change all rowset ids because they maybe its id same with local rowset - auto olap_st = SnapshotManager::instance()->convert_rowset_ids( + status = SnapshotManager::instance()->convert_rowset_ids( local_data_path, _clone_req.tablet_id, _clone_req.replica_id, _clone_req.schema_hash); - if (olap_st != Status::OK()) { - LOG(WARNING) << "fail to convert rowset ids, path=" << local_data_path - << ", tablet_id=" << _clone_req.tablet_id - << ", replica_id=" << _clone_req.replica_id - << ", schema_hash=" << _clone_req.schema_hash << ", error=" << olap_st; - status = Status::InternalError("Failed to convert rowset ids"); - } + } else { + LOG_WARNING("failed to download snapshot from remote BE") + .tag("url", remote_url_prefix) + .error(status); } // Release snapshot, if failed, ignore it. OLAP engine will drop useless snapshot - st = _release_snapshot(src.host, src.be_port, *snapshot_path); - if (st.ok()) { - LOG(INFO) << "success to release snapshot, ip=" << src.host << ", port=" << src.be_port - << ", snapshot_path=" << *snapshot_path; - } else { - LOG(WARNING) << "fail to release snapshot, ip=" << src.host << ", port=" << src.be_port - << ", snapshot_path=" << *snapshot_path << ", error=" << st.to_string(); + auto st = _release_snapshot(src.host, src.be_port, *snapshot_path); + if (!st.ok()) { + LOG_WARNING("failed to release snapshot in remote BE") + .tag("host", src.host) + .tag("port", src.be_port) + .tag("snapshot_path", *snapshot_path) + .error(status); // DON'T change the status } if (status.ok()) { @@ -392,14 +311,14 @@ Status EngineCloneTask::_make_snapshot(const std::string& ip, int port, TTableId return Status(result.status); } - if (result.__isset.snapshot_path) { - *snapshot_path = result.snapshot_path; - if (snapshot_path->at(snapshot_path->length() - 1) != '/') { - snapshot_path->append("/"); - } - } else { - return Status::InternalError("success snapshot without snapshot path"); + if (!result.__isset.snapshot_path) { + return Status::InternalError("success snapshot request without snapshot path"); } + *snapshot_path = result.snapshot_path; + if (snapshot_path->at(snapshot_path->length() - 1) != '/') { + snapshot_path->append("/"); + } + if (result.__isset.allow_incremental_clone) { // During upgrading, some BE nodes still be installed an old previous old. // which incremental clone is not ready in those nodes. @@ -529,111 +448,77 @@ Status EngineCloneTask::_download_files(DataDir* data_dir, const std::string& re /// 2. Call _finish_xx_clone() to revise the tablet meta. Status EngineCloneTask::_finish_clone(Tablet* tablet, const std::string& clone_dir, int64_t committed_version, bool is_incremental_clone) { - Status res = Status::OK(); - std::vector linked_success_files; + Defer remove_clone_dir {[&]() { std::filesystem::remove_all(clone_dir); }}; + // clone and compaction operation should be performed sequentially - { - std::lock_guard base_compaction_lock(tablet->get_base_compaction_lock()); - std::lock_guard cumulative_compaction_lock( - tablet->get_cumulative_compaction_lock()); - tablet->set_clone_occurred(true); - std::lock_guard push_lock(tablet->get_push_lock()); - std::lock_guard wrlock(tablet->get_header_lock()); - do { - // check clone dir existed - if (!FileUtils::check_exist(clone_dir)) { - res = Status::OLAPInternalError(OLAP_ERR_DIR_NOT_EXIST); - LOG(WARNING) << "clone dir not existed when clone. clone_dir=" << clone_dir; - break; - } + std::lock_guard base_compaction_lock(tablet->get_base_compaction_lock()); + std::lock_guard cumulative_compaction_lock( + tablet->get_cumulative_compaction_lock()); + tablet->set_clone_occurred(true); + std::lock_guard push_lock(tablet->get_push_lock()); + std::lock_guard wrlock(tablet->get_header_lock()); + // check clone dir existed + if (!FileUtils::check_exist(clone_dir)) { + return Status::InternalError("clone dir not existed. clone_dir={}", clone_dir); + } - // Load src header. - // The tablet meta info is downloaded from source BE as .hdr file. - // So we load it and generate cloned_tablet_meta. - auto cloned_tablet_meta_file = fmt::format("{}/{}.hdr", clone_dir, tablet->tablet_id()); - TabletMeta cloned_tablet_meta; - res = cloned_tablet_meta.create_from_file(cloned_tablet_meta_file); - if (!res.ok()) { - LOG(WARNING) << "fail to load src header when clone. " - << ", cloned_tablet_meta_file=" << cloned_tablet_meta_file; - break; - } - // remove the cloned meta file - FileUtils::remove(cloned_tablet_meta_file); + // Load src header. + // The tablet meta info is downloaded from source BE as .hdr file. + // So we load it and generate cloned_tablet_meta. + auto cloned_tablet_meta_file = fmt::format("{}/{}.hdr", clone_dir, tablet->tablet_id()); + TabletMeta cloned_tablet_meta; + RETURN_IF_ERROR(cloned_tablet_meta.create_from_file(cloned_tablet_meta_file)); - // check all files in /clone and /tablet - set clone_files; - Status ret = - FileUtils::list_dirs_files(clone_dir, nullptr, &clone_files, Env::Default()); - if (!ret.ok()) { - LOG(WARNING) << "failed to list clone dir when clone. [clone_dir=" << clone_dir - << "]" - << " error: " << ret.to_string(); - res = Status::OLAPInternalError(OLAP_ERR_DISK_FAILURE); - break; - } + // remove the cloned meta file + FileUtils::remove(cloned_tablet_meta_file); - set local_files; - const auto& tablet_dir = tablet->tablet_path(); - ret = FileUtils::list_dirs_files(tablet_dir, nullptr, &local_files, Env::Default()); - if (!ret.ok()) { - LOG(WARNING) << "failed to list local tablet dir when clone. [tablet_dir=" - << tablet_dir << "]" - << " error: " << ret.to_string(); - res = Status::OLAPInternalError(OLAP_ERR_DISK_FAILURE); - break; - } + // check all files in /clone and /tablet + set clone_files; + RETURN_IF_ERROR(FileUtils::list_dirs_files(clone_dir, nullptr, &clone_files, Env::Default())); - /// Traverse all downloaded clone files in CLONE dir. - /// If it does not exist in local tablet dir, link the file to local tablet dir - /// And save all linked files in linked_success_files. - for (const string& clone_file : clone_files) { - if (local_files.find(clone_file) != local_files.end()) { - VLOG_NOTICE << "find same file when clone, skip it. " - << "tablet=" << tablet->full_name() - << ", clone_file=" << clone_file; - continue; - } + set local_files; + const auto& tablet_dir = tablet->tablet_path(); + RETURN_IF_ERROR(FileUtils::list_dirs_files(tablet_dir, nullptr, &local_files, Env::Default())); - auto from = fmt::format("{}/{}", clone_dir, clone_file); - auto to = fmt::format("{}/{}", tablet_dir, clone_file); - LOG(INFO) << "src file:" << from << " dest file:" << to; - if (link(from.c_str(), to.c_str()) != 0) { - LOG(WARNING) << "fail to create hard link when clone. " - << " from=" << from << " to=" << to; - res = Status::OLAPInternalError(OLAP_ERR_OS_ERROR); - break; - } - linked_success_files.emplace_back(std::move(to)); - } - - if (!res.ok()) { - break; - } - - if (is_incremental_clone) { - res = _finish_incremental_clone(tablet, cloned_tablet_meta, committed_version); - } else { - res = _finish_full_clone(tablet, const_cast(&cloned_tablet_meta)); - } - - // if full clone success, need to update cumulative layer point - if (!is_incremental_clone && res.ok()) { - tablet->set_cumulative_layer_point(Tablet::K_INVALID_CUMULATIVE_POINT); - } - - } while (0); - - // clear linked files if errors happen - if (!res.ok()) { + Status status; + std::vector linked_success_files; + Defer remove_linked_files {[&]() { // clear linked files if errors happen + if (!status.ok()) { FileUtils::remove_paths(linked_success_files); } + }}; + /// Traverse all downloaded clone files in CLONE dir. + /// If it does not exist in local tablet dir, link the file to local tablet dir + /// And save all linked files in linked_success_files. + for (const string& clone_file : clone_files) { + if (local_files.find(clone_file) != local_files.end()) { + VLOG_NOTICE << "find same file when clone, skip it. " + << "tablet=" << tablet->full_name() << ", clone_file=" << clone_file; + continue; + } + + auto from = fmt::format("{}/{}", clone_dir, clone_file); + auto to = fmt::format("{}/{}", tablet_dir, clone_file); + if (link(from.c_str(), to.c_str()) != 0) { + status = Status::InternalError("failed to create hard link. from={}, to={}", from, to); + return status; + } + linked_success_files.emplace_back(std::move(to)); } + + if (is_incremental_clone) { + status = _finish_incremental_clone(tablet, cloned_tablet_meta, committed_version); + } else { + status = _finish_full_clone(tablet, const_cast(&cloned_tablet_meta)); + } + + // if full clone success, need to update cumulative layer point + if (!is_incremental_clone && status.ok()) { + tablet->set_cumulative_layer_point(Tablet::K_INVALID_CUMULATIVE_POINT); + } + // clear clone dir - std::filesystem::remove_all(clone_dir); - LOG(INFO) << "finish to clone data, clear downloaded data. res=" << res - << ", tablet=" << tablet->full_name() << ", clone_dir=" << clone_dir; - return res; + return status; } /// This method will do: @@ -643,7 +528,7 @@ Status EngineCloneTask::_finish_incremental_clone(Tablet* tablet, const TabletMeta& cloned_tablet_meta, int64_t committed_version) { LOG(INFO) << "begin to finish incremental clone. tablet=" << tablet->full_name() - << ", clone version=" << committed_version; + << ", committed_version=" << committed_version; /// Get missing versions again from local tablet. /// We got it before outside the lock, so it has to be got again. @@ -658,11 +543,9 @@ Status EngineCloneTask::_finish_incremental_clone(Tablet* tablet, for (Version version : missed_versions) { RowsetMetaSharedPtr rs_meta = cloned_tablet_meta.acquire_rs_meta_by_version(version); if (rs_meta == nullptr) { - LOG(WARNING) << "missed version is not found in cloned tablet meta." - << ", missed_version=" << version.first << "-" << version.second; - return Status::OLAPInternalError(OLAP_ERR_WRITE_PROTOBUF_ERROR); + return Status::InternalError("missed version {} is not found in cloned tablet meta", + version.to_string()); } - rowsets_to_clone.push_back(rs_meta); } @@ -670,10 +553,7 @@ Status EngineCloneTask::_finish_incremental_clone(Tablet* tablet, /// For incremental clone, nothing will be deleted. /// So versions_to_delete is empty. std::vector versions_to_delete; - Status clone_res = tablet->revise_tablet_meta(rowsets_to_clone, versions_to_delete); - LOG(INFO) << "finish to incremental clone. [tablet=" << tablet->full_name() - << " res=" << clone_res << "]"; - return clone_res; + return tablet->revise_tablet_meta(rowsets_to_clone, versions_to_delete); } /// This method will do: @@ -713,10 +593,9 @@ Status EngineCloneTask::_finish_full_clone(Tablet* tablet, TabletMeta* cloned_ta // do compaction. if (local_version.first <= cloned_max_version.second && local_version.second > cloned_max_version.second) { - LOG(WARNING) << "stop to full clone, version cross src latest." - << "tablet=" << tablet->full_name() << ", local_version=" << local_version; - return Status::OLAPInternalError(OLAP_ERR_TABLE_VERSION_DUPLICATE_ERROR); - + return Status::InternalError( + "version cross src latest. cloned_max_version={}, local_version={}", + cloned_max_version.to_string(), local_version.to_string()); } else if (local_version.second <= cloned_max_version.second) { // if local version smaller than src, check if existed in src, will not clone it bool existed_in_src = false; @@ -761,7 +640,6 @@ Status EngineCloneTask::_finish_full_clone(Tablet* tablet, TabletMeta* cloned_ta // 3. local tablet cloned rowset from other nodes // 4. if cleared alter task info, then push will not write to new tablet, the report info is error Status clone_res = tablet->revise_tablet_meta(rowsets_to_clone, versions_to_delete); - LOG(INFO) << "finish to full clone. tablet=" << tablet->full_name() << ", res=" << clone_res; // in previous step, copy all files from CLONE_DIR to tablet dir // but some rowset is useless, so that remove them here for (auto& rs_meta_ptr : rs_metas_found_in_src) { @@ -777,7 +655,7 @@ Status EngineCloneTask::_finish_full_clone(Tablet* tablet, TabletMeta* cloned_ta s = rowset_to_remove->remove(); if (!s.ok()) { LOG(WARNING) << "failed to remove rowset " << rs_meta_ptr->rowset_id().to_string() - << ", res=" << s; + << ": " << s; } } return clone_res; diff --git a/be/src/olap/task/engine_clone_task.h b/be/src/olap/task/engine_clone_task.h index 90c4b43596..eae7bf0fb6 100644 --- a/be/src/olap/task/engine_clone_task.h +++ b/be/src/olap/task/engine_clone_task.h @@ -35,9 +35,8 @@ public: virtual Status execute(); public: - EngineCloneTask(const TCloneReq& _clone_req, const TMasterInfo& _master_info, - int64_t _signature, vector* error_msgs, - vector* tablet_infos, Status* _res_status); + EngineCloneTask(const TCloneReq& clone_req, const TMasterInfo& master_info, int64_t signature, + vector* tablet_infos); ~EngineCloneTask() {} private: @@ -53,11 +52,10 @@ private: Status _make_and_download_snapshots(DataDir& data_dir, const std::string& local_data_path, TBackend* src_host, string* src_file_path, - vector* error_msgs, const vector* missing_versions, bool* allow_incremental_clone); - void _set_tablet_info(Status status, bool is_new_tablet); + Status _set_tablet_info(bool is_new_tablet); // Download tablet files from Status _download_files(DataDir* data_dir, const std::string& remote_url_prefix, @@ -72,9 +70,7 @@ private: private: const TCloneReq& _clone_req; - vector* _error_msgs; vector* _tablet_infos; - Status* _res_status; int64_t _signature; const TMasterInfo& _master_info; int64_t _copy_size; diff --git a/be/src/olap/task/engine_storage_migration_task.cpp b/be/src/olap/task/engine_storage_migration_task.cpp index e3737a62e4..6ecefe4722 100644 --- a/be/src/olap/task/engine_storage_migration_task.cpp +++ b/be/src/olap/task/engine_storage_migration_task.cpp @@ -43,8 +43,8 @@ Status EngineStorageMigrationTask::_get_versions(int32_t start_version, int32_t* std::shared_lock rdlock(_tablet->get_header_lock()); const RowsetSharedPtr last_version = _tablet->rowset_with_max_version(); if (last_version == nullptr) { - LOG(WARNING) << "failed to get rowset with max version, tablet=" << _tablet->full_name(); - return Status::OLAPInternalError(OLAP_ERR_WRITE_PROTOBUF_ERROR); + return Status::InternalError("failed to get rowset with max version, tablet={}", + _tablet->tablet_id()); } *end_version = last_version->end_version(); @@ -54,13 +54,8 @@ Status EngineStorageMigrationTask::_get_versions(int32_t start_version, int32_t* << ", start_version=" << start_version << ", end_version=" << *end_version; return Status::OK(); } - _tablet->capture_consistent_rowsets(Version(start_version, *end_version), consistent_rowsets); - if (consistent_rowsets->empty()) { - LOG(WARNING) << "fail to capture consistent rowsets. tablet=" << _tablet->full_name() - << ", version=" << *end_version; - return Status::OLAPInternalError(OLAP_ERR_WRITE_PROTOBUF_ERROR); - } - return Status::OK(); + return _tablet->capture_consistent_rowsets(Version(start_version, *end_version), + consistent_rowsets); } bool EngineStorageMigrationTask::_is_timeout() { @@ -82,7 +77,7 @@ Status EngineStorageMigrationTask::_check_running_txns() { _tablet->tablet_id(), _tablet->schema_hash(), _tablet->tablet_uid(), &partition_id, &transaction_ids); if (transaction_ids.size() > 0) { - return Status::OLAPInternalError(OLAP_ERR_HEADER_HAS_PENDING_DATA); + return Status::InternalError("tablet {} has unfinished txns", _tablet->tablet_id()); } return Status::OK(); } @@ -103,9 +98,6 @@ Status EngineStorageMigrationTask::_check_running_txns_until_timeout( *migration_wlock = std::move(wlock); return res; } - LOG(INFO) << "check running txns fail, try again until timeout." - << " tablet=" << _tablet->full_name() << ", try times=" << try_times - << ", res=" << res; // unlock and sleep for a while, try again wlock.unlock(); sleep(std::min(config::sleep_one_second * try_times, CHECK_TXNS_MAX_WAIT_TIME_SECS)); @@ -118,7 +110,6 @@ Status EngineStorageMigrationTask::_gen_and_write_header_to_hdr_file( uint64_t shard, const std::string& full_path, const std::vector& consistent_rowsets, int64_t end_version) { // need hold migration lock and push lock outside - Status res = Status::OK(); int64_t tablet_id = _tablet->tablet_id(); int32_t schema_hash = _tablet->schema_hash(); TabletMetaSharedPtr new_tablet_meta(new (std::nothrow) TabletMeta()); @@ -127,51 +118,31 @@ Status EngineStorageMigrationTask::_gen_and_write_header_to_hdr_file( _generate_new_header(shard, consistent_rowsets, new_tablet_meta, end_version); } std::string new_meta_file = full_path + "/" + std::to_string(tablet_id) + ".hdr"; - res = new_tablet_meta->save(new_meta_file); - if (!res.ok()) { - LOG(WARNING) << "failed to save meta to path: " << new_meta_file; - return res; - } + RETURN_IF_ERROR(new_tablet_meta->save(new_meta_file)); // reset tablet id and rowset id - res = TabletMeta::reset_tablet_uid(new_meta_file); - if (!res.ok()) { - LOG(WARNING) << "errors while set tablet uid: '" << new_meta_file; - return res; - } + RETURN_IF_ERROR(TabletMeta::reset_tablet_uid(new_meta_file)); + // it will change rowset id and its create time // rowset create time is useful when load tablet from meta to check which tablet is the tablet to load - res = SnapshotManager::instance()->convert_rowset_ids(full_path, tablet_id, - _tablet->replica_id(), schema_hash); - if (!res.ok()) { - LOG(WARNING) << "failed to convert rowset id when do storage migration" - << " path = " << full_path; - return res; - } - return res; + return SnapshotManager::instance()->convert_rowset_ids(full_path, tablet_id, + _tablet->replica_id(), schema_hash); } Status EngineStorageMigrationTask::_reload_tablet(const std::string& full_path) { // need hold migration lock and push lock outside - Status res = Status::OK(); int64_t tablet_id = _tablet->tablet_id(); int32_t schema_hash = _tablet->schema_hash(); - res = StorageEngine::instance()->tablet_manager()->load_tablet_from_dir( - _dest_store, tablet_id, schema_hash, full_path, false); - if (!res.ok()) { - LOG(WARNING) << "failed to load tablet from new path. tablet_id=" << tablet_id - << " schema_hash=" << schema_hash << " path = " << full_path; - return res; - } + RETURN_IF_ERROR(StorageEngine::instance()->tablet_manager()->load_tablet_from_dir( + _dest_store, tablet_id, schema_hash, full_path, false)); // if old tablet finished schema change, then the schema change status of the new tablet is DONE // else the schema change status of the new tablet is FAILED TabletSharedPtr new_tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id); if (new_tablet == nullptr) { - LOG(WARNING) << "tablet not found. tablet_id=" << tablet_id; - return Status::OLAPInternalError(OLAP_ERR_TABLE_NOT_FOUND); + return Status::NotFound("could not find tablet {}", tablet_id); } - return res; + return Status::OK(); } // if the size less than threshold, return true @@ -198,54 +169,36 @@ Status EngineStorageMigrationTask::_migrate() { std::vector consistent_rowsets; // try hold migration lock first - Status res = Status::OK(); + Status res; uint64_t shard = 0; std::string full_path; { std::unique_lock migration_wlock(_tablet->get_migration_lock(), std::try_to_lock); if (!migration_wlock.owns_lock()) { - return Status::OLAPInternalError(OLAP_ERR_RWLOCK_ERROR); + return Status::InternalError("could not own migration_wlock"); } // check if this tablet has related running txns. if yes, can not do migration. - res = _check_running_txns(); - if (!res.ok()) { - LOG(WARNING) << "could not migration because has unfinished txns, " - << " tablet=" << _tablet->full_name(); - return res; - } + RETURN_IF_ERROR(_check_running_txns()); std::lock_guard lock(_tablet->get_push_lock()); // get versions to be migrate - res = _get_versions(start_version, &end_version, &consistent_rowsets); - if (!res.ok()) { - return res; - } + RETURN_IF_ERROR(_get_versions(start_version, &end_version, &consistent_rowsets)); // TODO(ygl): the tablet should not under schema change or rollup or load - res = _dest_store->get_shard(&shard); - if (!res.ok()) { - LOG(WARNING) << "fail to get shard from store: " << _dest_store->path(); - return res; - } + RETURN_IF_ERROR(_dest_store->get_shard(&shard)); + auto shard_path = fmt::format("{}/{}/{}", _dest_store->path(), DATA_PREFIX, shard); full_path = SnapshotManager::get_schema_hash_full_path(_tablet, shard_path); // if dir already exist then return err, it should not happen. // should not remove the dir directly, for safety reason. if (FileUtils::check_exist(full_path)) { - LOG(INFO) << "schema hash path already exist, skip this path. " - << "full_path=" << full_path; - return Status::OLAPInternalError(OLAP_ERR_FILE_ALREADY_EXIST); + return Status::AlreadyExist("schema hash path {} already exist, skip this path", + full_path); } - Status st = FileUtils::create_dir(full_path); - if (!st.ok()) { - res = Status::OLAPInternalError(OLAP_ERR_CANNOT_CREATE_DIR); - LOG(WARNING) << "fail to create path. path=" << full_path - << ", error:" << st.to_string(); - return res; - } + RETURN_IF_ERROR(FileUtils::create_dir(full_path)); } std::vector temp_consistent_rowsets(consistent_rowsets); @@ -253,7 +206,6 @@ Status EngineStorageMigrationTask::_migrate() { // migrate all index and data files but header file res = _copy_index_and_data_files(full_path, temp_consistent_rowsets); if (!res.ok()) { - LOG(WARNING) << "fail to copy index and data files when migrate. res=" << res; break; } std::unique_lock migration_wlock; @@ -282,14 +234,11 @@ Status EngineStorageMigrationTask::_migrate() { // force to copy the remaining data and index res = _copy_index_and_data_files(full_path, temp_consistent_rowsets); if (!res.ok()) { - LOG(WARNING) - << "fail to copy the remaining index and data files when migrate. res=" - << res; break; } } else { if (_is_timeout()) { - res = Status::OLAPInternalError(OLAP_ERR_HEADER_HAS_PENDING_DATA); + res = Status::TimedOut("failed to migrate due to timeout"); break; } // there is too much remaining data here. @@ -341,19 +290,10 @@ void EngineStorageMigrationTask::_generate_new_header( Status EngineStorageMigrationTask::_copy_index_and_data_files( const string& full_path, const std::vector& consistent_rowsets) const { - Status status = Status::OK(); for (const auto& rs : consistent_rowsets) { - status = rs->copy_files_to(full_path, rs->rowset_id()); - if (!status.ok()) { - Status ret = FileUtils::remove_all(full_path); - if (!ret.ok()) { - LOG(FATAL) << "remove storage migration path failed. " - << "full_path:" << full_path << " error: " << ret.to_string(); - } - break; - } + RETURN_IF_ERROR(rs->copy_files_to(full_path, rs->rowset_id())); } - return status; + return Status::OK(); } } // namespace doris diff --git a/be/src/runtime/data_stream_recvr.cc b/be/src/runtime/data_stream_recvr.cc index b7988178fb..26bb917a5d 100644 --- a/be/src/runtime/data_stream_recvr.cc +++ b/be/src/runtime/data_stream_recvr.cc @@ -33,7 +33,6 @@ #include "runtime/sorted_run_merger.h" #include "runtime/thread_context.h" #include "util/debug_util.h" -#include "util/logging.h" #include "util/runtime_profile.h" using std::list; diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index bc5c7bb721..969900e995 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -491,11 +491,10 @@ void FragmentMgr::_exec_actual(std::shared_ptr exec_state, Fi doris::signal::query_id_hi = exec_state->query_id().hi; doris::signal::query_id_lo = exec_state->query_id().lo; - TAG(LOG(INFO)) - .log(std::move(func_name)) - .query_id(exec_state->query_id()) - .instance_id(exec_state->fragment_instance_id()) - .tag("pthread_id", std::to_string((uintptr_t)pthread_self())); + LOG_INFO(func_name) + .tag("query_id", exec_state->query_id()) + .tag("instance_id", exec_state->fragment_instance_id()) + .tag("pthread_id", (uintptr_t)pthread_self()); exec_state->execute(); diff --git a/be/src/runtime/memory/mem_tracker_limiter.cpp b/be/src/runtime/memory/mem_tracker_limiter.cpp index 32061c8ad9..097092c58b 100644 --- a/be/src/runtime/memory/mem_tracker_limiter.cpp +++ b/be/src/runtime/memory/mem_tracker_limiter.cpp @@ -22,6 +22,7 @@ #include #include "gutil/once.h" +#include "gutil/walltime.h" #include "runtime/runtime_state.h" #include "runtime/thread_context.h" #include "service/backend_options.h" diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp index 90870b6a4e..ff3a0feed9 100644 --- a/be/src/runtime/plan_fragment_executor.cpp +++ b/be/src/runtime/plan_fragment_executor.cpp @@ -40,7 +40,6 @@ #include "runtime/thread_context.h" #include "util/container_util.hpp" #include "util/defer_op.h" -#include "util/logging.h" #include "util/mem_info.h" #include "util/parse_util.h" #include "util/pretty_printer.h" @@ -85,12 +84,11 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request, const TPlanFragmentExecParams& params = request.params; _query_id = params.query_id; - TAG(LOG(INFO)) - .log("PlanFragmentExecutor::prepare") - .query_id(_query_id) - .instance_id(params.fragment_instance_id) - .tag("backend_num", std::to_string(request.backend_num)) - .tag("pthread_id", std::to_string((uintptr_t)pthread_self())); + LOG_INFO("PlanFragmentExecutor::prepare") + .tag("query_id", _query_id) + .tag("instance_id", params.fragment_instance_id) + .tag("backend_num", request.backend_num) + .tag("pthread_id", (uintptr_t)pthread_self()); // VLOG_CRITICAL << "request:\n" << apache::thrift::ThriftDebugString(request); const TQueryGlobals& query_globals = @@ -226,12 +224,10 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request, Status PlanFragmentExecutor::open() { int64_t mem_limit = _runtime_state->instance_mem_tracker()->limit(); - TAG(LOG(INFO)) - .log("PlanFragmentExecutor::open, using query memory limit: " + - PrettyPrinter::print(mem_limit, TUnit::BYTES)) - .query_id(_query_id) - .instance_id(_runtime_state->fragment_instance_id()) - .tag("mem_limit", std::to_string(mem_limit)); + LOG_INFO("PlanFragmentExecutor::open") + .tag("query_id", _query_id) + .tag("instance_id", _runtime_state->fragment_instance_id()) + .tag("mem_limit", PrettyPrinter::print(mem_limit, TUnit::BYTES)); // we need to start the profile-reporting thread before calling Open(), since it // may block @@ -562,10 +558,9 @@ Status PlanFragmentExecutor::get_next(RowBatch** batch) { update_status(status); if (_done) { - TAG(LOG(INFO)) - .log("PlanFragmentExecutor::get_next finished") - .query_id(_query_id) - .instance_id(_runtime_state->fragment_instance_id()); + LOG_INFO("PlanFragmentExecutor::get_next finished") + .tag("query_id", _query_id) + .tag("instance_id", _runtime_state->fragment_instance_id()); // Query is done, return the thread token stop_report_thread(); send_report(true); @@ -623,10 +618,9 @@ void PlanFragmentExecutor::update_status(const Status& new_status) { } void PlanFragmentExecutor::cancel(const PPlanFragmentCancelReason& reason, const std::string& msg) { - TAG(LOG(INFO)) - .log("PlanFragmentExecutor::cancel") - .query_id(_query_id) - .instance_id(_runtime_state->fragment_instance_id()); + LOG_INFO("PlanFragmentExecutor::cancel") + .tag("query_id", _query_id) + .tag("instance_id", _runtime_state->fragment_instance_id()); DCHECK(_prepared); _cancel_reason = reason; _cancel_msg = msg; diff --git a/be/src/runtime/query_statistics.cpp b/be/src/runtime/query_statistics.cpp index a3961b3d8c..4ef8e4a7ce 100644 --- a/be/src/runtime/query_statistics.cpp +++ b/be/src/runtime/query_statistics.cpp @@ -17,6 +17,8 @@ #include "runtime/query_statistics.h" +#include + namespace doris { void NodeStatistics::merge(const NodeStatistics& other) { diff --git a/be/src/runtime/result_buffer_mgr.h b/be/src/runtime/result_buffer_mgr.h index 919fdfb031..26a07bd90c 100644 --- a/be/src/runtime/result_buffer_mgr.h +++ b/be/src/runtime/result_buffer_mgr.h @@ -26,8 +26,8 @@ #include "common/status.h" #include "gen_cpp/Types_types.h" #include "gutil/ref_counted.h" +#include "util/hash_util.hpp" #include "util/thread.h" -#include "util/uid_util.h" namespace doris { diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index 46838af5cb..484e7d971b 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -36,7 +36,6 @@ #include "runtime/mem_pool.h" #include "runtime/query_fragments_ctx.h" #include "runtime/thread_resource_mgr.h" -#include "util/logging.h" #include "util/runtime_profile.h" #include "util/telemetry/telemetry.h" diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp index 68582b418c..f6fa80e52c 100644 --- a/be/src/service/doris_main.cpp +++ b/be/src/service/doris_main.cpp @@ -59,7 +59,6 @@ #include "service/single_replica_load_download_service.h" #include "util/debug_util.h" #include "util/doris_metrics.h" -#include "util/logging.h" #include "util/perf_counters.h" #include "util/telemetry/telemetry.h" #include "util/thrift_rpc_helper.h" diff --git a/be/src/util/cpu_info.h b/be/src/util/cpu_info.h index 80d719e4fa..d09c89775f 100644 --- a/be/src/util/cpu_info.h +++ b/be/src/util/cpu_info.h @@ -20,12 +20,12 @@ #pragma once +#include + #include #include #include -#include "common/logging.h" - namespace doris { /// CpuInfo is an interface to query for cpu information at runtime. The caller can diff --git a/be/src/util/hash_util.hpp b/be/src/util/hash_util.hpp index 685472b84f..3a7ac11704 100644 --- a/be/src/util/hash_util.hpp +++ b/be/src/util/hash_util.hpp @@ -20,8 +20,9 @@ #pragma once +#include + #include "common/compiler_util.h" -#include "common/logging.h" // For cross compiling with clang, we need to be able to generate an IR file with // no sse instructions. Attempting to load a precompiled IR file that contains diff --git a/be/src/util/hdfs_util.cpp b/be/src/util/hdfs_util.cpp index e20f750157..5ffa09b966 100644 --- a/be/src/util/hdfs_util.cpp +++ b/be/src/util/hdfs_util.cpp @@ -20,7 +20,7 @@ #include #include "common/config.h" -#include "util/logging.h" +#include "common/logging.h" namespace doris { diff --git a/be/src/util/internal_queue.h b/be/src/util/internal_queue.h index 55faf006bf..a20944893a 100644 --- a/be/src/util/internal_queue.h +++ b/be/src/util/internal_queue.h @@ -23,6 +23,7 @@ #include #include +#include "common/logging.h" #include "util/fake_lock.h" #include "util/spinlock.h" diff --git a/be/src/util/logging.h b/be/src/util/logging.h deleted file mode 100644 index 1ae2230b65..0000000000 --- a/be/src/util/logging.h +++ /dev/null @@ -1,178 +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 - -#include -#include - -#include -#include - -#include "common/logging.h" -#include "gutil/walltime.h" -#include "util/uid_util.h" - -namespace doris { - -// glog doesn't allow multiple invocations of InitGoogleLogging. This method conditionally -// calls InitGoogleLogging only if it hasn't been called before. -bool init_glog(const char* basename); - -// Shuts down the google logging library. Call before exit to ensure that log files are -// flushed. May only be called once. -void shutdown_logging(); - -// Format a timestamp in the same format as used by GLog. -std::string FormatTimestampForLog(MicrosecondsInt64 micros_since_epoch); - -class DorisAWSLogger final : public Aws::Utils::Logging::LogSystemInterface { -public: - DorisAWSLogger() : _log_level(Aws::Utils::Logging::LogLevel::Info) {} - DorisAWSLogger(Aws::Utils::Logging::LogLevel log_level) : _log_level(log_level) {} - ~DorisAWSLogger() final = default; - Aws::Utils::Logging::LogLevel GetLogLevel() const final { return _log_level; } - void Log(Aws::Utils::Logging::LogLevel log_level, const char* tag, const char* format_str, - ...) final { - _log_impl(log_level, tag, format_str); - } - void LogStream(Aws::Utils::Logging::LogLevel log_level, const char* tag, - const Aws::OStringStream& message_stream) final { - _log_impl(log_level, tag, message_stream.str().c_str()); - } - - void Flush() final {} - -private: - void _log_impl(Aws::Utils::Logging::LogLevel log_level, const char* tag, const char* message) { - switch (log_level) { - case Aws::Utils::Logging::LogLevel::Off: - break; - case Aws::Utils::Logging::LogLevel::Fatal: - LOG(FATAL) << "[" << tag << "] " << message; - break; - case Aws::Utils::Logging::LogLevel::Error: - LOG(ERROR) << "[" << tag << "] " << message; - break; - case Aws::Utils::Logging::LogLevel::Warn: - LOG(WARNING) << "[" << tag << "] " << message; - break; - case Aws::Utils::Logging::LogLevel::Info: - LOG(INFO) << "[" << tag << "] " << message; - break; - case Aws::Utils::Logging::LogLevel::Debug: - VLOG_ROW << "[" << tag << "] " << message; - break; - case Aws::Utils::Logging::LogLevel::Trace: - VLOG_ROW << "[" << tag << "] " << message; - break; - default: - break; - } - } - - std::atomic _log_level; -}; - -/// Wrap a glog stream and tag on the log. usage: -/// TAG(LOG(INFO)).tag("query_id", queryId).log("here is an info for a query"); -/// -/// TAG is the macro to TaggableLogger, which use method tag(key, value) to add tags -/// and log(fmt, ...) to flush and emit the log. Usually the tag key is determined, -/// like "query_id", so we use specified tag methods more often, like query_id(id). -/// You can add a new tag method if needed. -/// -/// You can custom your tagged logging format in logconfig.cpp, the default is like -/// "#message#|k1=v1|k2=v2". You can also custom all the tag names. For example, if -/// you wish to use camelCase style, just set tag name constants like QUERY_ID to -/// "queryId". The constants is also listed in logconfig.cpp. -/// -/// The transfer from the variable of tag method to string is immediate. If a tagged -/// logging has time-consuming to-string procedure and is logged to VLOG(10) which -/// may not be processed, check VLOG_IS_ON(n) first. -#define TAG doris::TaggableLogger - -class TaggableLogger { -public: - TaggableLogger(std::ostream& _stream) : _stream(_stream), _tags(nullptr) {}; - - ~TaggableLogger() { flush(); } - - void flush(); - - TaggableLogger& log(std::string&& message) { - _message = std::move(message); - return *this; - } - - TaggableLogger& tag(const std::string& key, const std::string& value) { - _tags = new Tags(key, value, _tags); - return *this; - } - - TaggableLogger& tag(const std::string& key, std::string&& value) { - _tags = new Tags(key, std::move(value), _tags); - return *this; - } - -private: - std::ostream& _stream; - std::string _message; - - struct Tags { - const std::string key; - const std::string value; - Tags* next; - - Tags(const std::string& key, const std::string& value, Tags* next) - : key(key), value(value), next(next) {} - Tags(const std::string& key, std::string&& value, Tags* next) - : key(key), value(std::move(value)), next(next) {} - }; - - Tags* _tags; - -public: - // add tag method here - const static std::string QUERY_ID; - - TaggableLogger& query_id(const std::string& query_id) { return tag(QUERY_ID, query_id); } - - TaggableLogger& query_id(const TUniqueId& query_id) { - return tag(QUERY_ID, print_id(query_id)); - } - - TaggableLogger& query_id(const PUniqueId& query_id) { - return tag(QUERY_ID, print_id(query_id)); - } - - const static std::string INSTANCE_ID; - - TaggableLogger& instance_id(const std::string& instance_id) { - return tag(INSTANCE_ID, instance_id); - } - - TaggableLogger& instance_id(const TUniqueId& instance_id) { - return tag(INSTANCE_ID, print_id(instance_id)); - } - - TaggableLogger& instance_id(const PUniqueId& instance_id) { - return tag(INSTANCE_ID, print_id(instance_id)); - } -}; - -} // namespace doris diff --git a/be/src/util/s3_uri.cpp b/be/src/util/s3_uri.cpp index 177df0e1c6..882ca021a7 100644 --- a/be/src/util/s3_uri.cpp +++ b/be/src/util/s3_uri.cpp @@ -19,9 +19,9 @@ #include +#include "common/logging.h" #include "gutil/strings/split.h" #include "gutil/strings/strip.h" -#include "util/logging.h" namespace doris { diff --git a/be/src/util/s3_util.cpp b/be/src/util/s3_util.cpp index 8a42e9f6ba..f9b67e57d7 100644 --- a/be/src/util/s3_util.cpp +++ b/be/src/util/s3_util.cpp @@ -22,10 +22,58 @@ #include #include "common/config.h" -#include "util/logging.h" +#include "common/logging.h" namespace doris { +class DorisAWSLogger final : public Aws::Utils::Logging::LogSystemInterface { +public: + DorisAWSLogger() : _log_level(Aws::Utils::Logging::LogLevel::Info) {} + DorisAWSLogger(Aws::Utils::Logging::LogLevel log_level) : _log_level(log_level) {} + ~DorisAWSLogger() final = default; + Aws::Utils::Logging::LogLevel GetLogLevel() const final { return _log_level; } + void Log(Aws::Utils::Logging::LogLevel log_level, const char* tag, const char* format_str, + ...) final { + _log_impl(log_level, tag, format_str); + } + void LogStream(Aws::Utils::Logging::LogLevel log_level, const char* tag, + const Aws::OStringStream& message_stream) final { + _log_impl(log_level, tag, message_stream.str().c_str()); + } + + void Flush() final {} + +private: + void _log_impl(Aws::Utils::Logging::LogLevel log_level, const char* tag, const char* message) { + switch (log_level) { + case Aws::Utils::Logging::LogLevel::Off: + break; + case Aws::Utils::Logging::LogLevel::Fatal: + LOG(FATAL) << "[" << tag << "] " << message; + break; + case Aws::Utils::Logging::LogLevel::Error: + LOG(ERROR) << "[" << tag << "] " << message; + break; + case Aws::Utils::Logging::LogLevel::Warn: + LOG(WARNING) << "[" << tag << "] " << message; + break; + case Aws::Utils::Logging::LogLevel::Info: + LOG(INFO) << "[" << tag << "] " << message; + break; + case Aws::Utils::Logging::LogLevel::Debug: + VLOG_ROW << "[" << tag << "] " << message; + break; + case Aws::Utils::Logging::LogLevel::Trace: + VLOG_ROW << "[" << tag << "] " << message; + break; + default: + break; + } + } + + std::atomic _log_level; +}; + const static std::string USE_PATH_STYLE = "use_path_style"; ClientFactory::ClientFactory() { diff --git a/be/src/util/sm3.cpp b/be/src/util/sm3.cpp index c124a99a5f..25f59c1f6b 100644 --- a/be/src/util/sm3.cpp +++ b/be/src/util/sm3.cpp @@ -17,7 +17,7 @@ #include "util/sm3.h" -#include "util/logging.h" +#include "common/logging.h" namespace doris { diff --git a/be/src/util/spinlock.h b/be/src/util/spinlock.h index 3c63b50bc1..c0712875fe 100644 --- a/be/src/util/spinlock.h +++ b/be/src/util/spinlock.h @@ -20,9 +20,9 @@ #pragma once -#include +#include /* For sched_yield() */ -#include "common/logging.h" +#include namespace doris { diff --git a/be/src/util/trace.cpp b/be/src/util/trace.cpp index 4779e5a7c3..f8cae91f9f 100644 --- a/be/src/util/trace.cpp +++ b/be/src/util/trace.cpp @@ -31,8 +31,6 @@ #include #include "gutil/strings/substitute.h" -#include "gutil/walltime.h" -#include "util/logging.h" //#include "util/memory/arena.h" using std::pair; @@ -42,6 +40,17 @@ using strings::internal::SubstituteArg; namespace doris { +// Format a timestamp in the same format as used by GLog. +static std::string FormatTimestampForLog(MicrosecondsInt64 micros_since_epoch) { + time_t secs_since_epoch = micros_since_epoch / 1000000; + int usecs = micros_since_epoch % 1000000; + struct tm tm_time; + localtime_r(&secs_since_epoch, &tm_time); + + return StringPrintf("%02d%02d %02d:%02d:%02d.%06d", 1 + tm_time.tm_mon, tm_time.tm_mday, + tm_time.tm_hour, tm_time.tm_min, tm_time.tm_sec, usecs); +} + __thread Trace* Trace::threadlocal_trace_; Trace::Trace() diff --git a/be/src/util/uid_util.cpp b/be/src/util/uid_util.cpp index 550a5057ad..1fc6c4d457 100644 --- a/be/src/util/uid_util.cpp +++ b/be/src/util/uid_util.cpp @@ -17,8 +17,21 @@ #include "util/uid_util.h" +#include "util/hash_util.hpp" + namespace doris { +size_t UniqueId::hash(size_t seed) const { + return doris::HashUtil::hash(this, sizeof(*this), seed); +} + +std::size_t hash_value(const doris::TUniqueId& id) { + std::size_t seed = 0; + HashUtil::hash_combine(seed, id.lo); + HashUtil::hash_combine(seed, id.hi); + return seed; +} + std::ostream& operator<<(std::ostream& os, const UniqueId& uid) { os << uid.to_string(); return os; diff --git a/be/src/util/uid_util.h b/be/src/util/uid_util.h index c5d2e6222c..d23258f982 100644 --- a/be/src/util/uid_util.h +++ b/be/src/util/uid_util.h @@ -22,7 +22,6 @@ #include "gen_cpp/Types_types.h" // for TUniqueId #include "gen_cpp/types.pb.h" // for PUniqueId -#include "util/hash_util.hpp" #include "util/uuid_generator.h" namespace doris { @@ -42,9 +41,9 @@ inline void from_hex(T* ret, const std::string& buf) { T val = 0; for (int i = 0; i < buf.length(); ++i) { int buf_val = 0; - if (buf.c_str()[i] >= '0' && buf.c_str()[i] <= '9') + if (buf.c_str()[i] >= '0' && buf.c_str()[i] <= '9') { buf_val = buf.c_str()[i] - '0'; - else { + } else { buf_val = buf.c_str()[i] - 'a' + 10; } val <<= 4; @@ -117,7 +116,7 @@ struct UniqueId { } // std::unordered_map need this api - size_t hash(size_t seed = 0) const { return doris::HashUtil::hash(this, sizeof(*this), seed); } + size_t hash(size_t seed = 0) const; // std::unordered_map need this api bool operator==(const UniqueId& rhs) const { return hi == rhs.hi && lo == rhs.lo; } @@ -140,12 +139,7 @@ struct UniqueId { }; // This function must be called 'hash_value' to be picked up by boost. -inline std::size_t hash_value(const doris::TUniqueId& id) { - std::size_t seed = 0; - HashUtil::hash_combine(seed, id.lo); - HashUtil::hash_combine(seed, id.hi); - return seed; -} +std::size_t hash_value(const doris::TUniqueId& id); /// generates a 16 byte UUID inline std::string generate_uuid_string() { diff --git a/be/test/agent/agent_server_test.cpp b/be/test/agent/agent_server_test.cpp index 5ffdb7b84a..6340737b31 100644 --- a/be/test/agent/agent_server_test.cpp +++ b/be/test/agent/agent_server_test.cpp @@ -23,7 +23,6 @@ #include "gmock/gmock.h" #include "gtest/gtest.h" #include "olap/mock_command_executor.h" -#include "util/logging.h" using ::testing::_; using ::testing::Return; diff --git a/be/test/agent/cgroups_mgr_test.cpp b/be/test/agent/cgroups_mgr_test.cpp index 9e5e002d29..5e5ed62906 100644 --- a/be/test/agent/cgroups_mgr_test.cpp +++ b/be/test/agent/cgroups_mgr_test.cpp @@ -23,7 +23,6 @@ #include "gmock/gmock.h" #include "gtest/gtest.h" -#include "util/logging.h" #ifndef BE_TEST #define BE_TEST diff --git a/be/test/agent/heartbeat_server_test.cpp b/be/test/agent/heartbeat_server_test.cpp index 4adeddf6f2..db3a7a2eb7 100644 --- a/be/test/agent/heartbeat_server_test.cpp +++ b/be/test/agent/heartbeat_server_test.cpp @@ -23,7 +23,6 @@ #include "gen_cpp/Types_types.h" #include "gmock/gmock.h" #include "gtest/gtest.h" -#include "util/logging.h" using ::testing::_; using ::testing::Return; diff --git a/be/test/agent/utils_test.cpp b/be/test/agent/utils_test.cpp index 700b0fd5cf..7521a842fb 100644 --- a/be/test/agent/utils_test.cpp +++ b/be/test/agent/utils_test.cpp @@ -22,7 +22,6 @@ #include "gmock/gmock.h" #include "gtest/gtest.h" #include "service/backend_options.h" -#include "util/logging.h" using ::testing::_; using ::testing::Return; diff --git a/be/test/common/resource_tls_test.cpp b/be/test/common/resource_tls_test.cpp index 8e9eeeeae7..717f5068dc 100644 --- a/be/test/common/resource_tls_test.cpp +++ b/be/test/common/resource_tls_test.cpp @@ -21,7 +21,6 @@ #include "common/configbase.h" #include "gen_cpp/Types_types.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/common/status_test.cpp b/be/test/common/status_test.cpp index f8d67666a6..dbeb4ad9b3 100644 --- a/be/test/common/status_test.cpp +++ b/be/test/common/status_test.cpp @@ -20,7 +20,6 @@ #include #include "gen_cpp/Types_types.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/exec/csv_scan_bench_test.cpp b/be/test/exec/csv_scan_bench_test.cpp index d7c6e2200d..0fa1a5b4b9 100644 --- a/be/test/exec/csv_scan_bench_test.cpp +++ b/be/test/exec/csv_scan_bench_test.cpp @@ -26,7 +26,6 @@ #include "runtime/row_batch.h" #include "runtime/runtime_state.h" #include "util/debug_util.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/exec/csv_scan_node_test.cpp b/be/test/exec/csv_scan_node_test.cpp index 929b5f355f..f8d9251f99 100644 --- a/be/test/exec/csv_scan_node_test.cpp +++ b/be/test/exec/csv_scan_node_test.cpp @@ -29,7 +29,6 @@ #include "util/cpu_info.h" #include "util/debug_util.h" #include "util/disk_info.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/exec/csv_scanner_test.cpp b/be/test/exec/csv_scanner_test.cpp index 846618518a..cc879eb69e 100644 --- a/be/test/exec/csv_scanner_test.cpp +++ b/be/test/exec/csv_scanner_test.cpp @@ -19,8 +19,6 @@ #include -#include "util/logging.h" - namespace doris { class CsvScannerTest : public testing::Test { diff --git a/be/test/exec/olap_common_test.cpp b/be/test/exec/olap_common_test.cpp index 68324e06f6..3e47e350c7 100644 --- a/be/test/exec/olap_common_test.cpp +++ b/be/test/exec/olap_common_test.cpp @@ -29,7 +29,6 @@ #include "gen_cpp/Types_types.h" #include "runtime/descriptors.h" #include "util/cpu_info.h" -#include "util/logging.h" #include "util/runtime_profile.h" namespace doris { diff --git a/be/test/exprs/bitmap_function_test.cpp b/be/test/exprs/bitmap_function_test.cpp index 78116bcbed..d76e30f333 100644 --- a/be/test/exprs/bitmap_function_test.cpp +++ b/be/test/exprs/bitmap_function_test.cpp @@ -29,7 +29,6 @@ #include "testutil/function_utils.h" #include "util/bitmap_intersect.h" #include "util/bitmap_value.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/exprs/encryption_functions_test.cpp b/be/test/exprs/encryption_functions_test.cpp index 26dc7c2fa4..3c48b38b45 100644 --- a/be/test/exprs/encryption_functions_test.cpp +++ b/be/test/exprs/encryption_functions_test.cpp @@ -24,7 +24,6 @@ #include "exprs/anyval_util.h" #include "testutil/function_utils.h" -#include "util/logging.h" namespace doris { class EncryptionFunctionsTest : public testing::Test { diff --git a/be/test/exprs/hll_function_test.cpp b/be/test/exprs/hll_function_test.cpp index d7948d4d2a..23e7a0dbc9 100644 --- a/be/test/exprs/hll_function_test.cpp +++ b/be/test/exprs/hll_function_test.cpp @@ -26,7 +26,6 @@ #include "exprs/anyval_util.h" #include "olap/hll.h" #include "testutil/function_utils.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/exprs/hybrid_set_test.cpp b/be/test/exprs/hybrid_set_test.cpp index 681c79af1f..83e05c6ce3 100644 --- a/be/test/exprs/hybrid_set_test.cpp +++ b/be/test/exprs/hybrid_set_test.cpp @@ -23,7 +23,6 @@ #include "common/configbase.h" #include "exprs/create_predicate_function.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/exprs/json_function_test.cpp b/be/test/exprs/json_function_test.cpp index 36a87e27be..b371799836 100644 --- a/be/test/exprs/json_function_test.cpp +++ b/be/test/exprs/json_function_test.cpp @@ -29,7 +29,6 @@ #include "exprs/anyval_util.h" #include "exprs/json_functions.h" #include "runtime/runtime_state.h" -#include "util/logging.h" #include "util/stopwatch.hpp" namespace doris { diff --git a/be/test/olap/block_column_predicate_test.cpp b/be/test/olap/block_column_predicate_test.cpp index 7de5c4a279..56a47da0e0 100644 --- a/be/test/olap/block_column_predicate_test.cpp +++ b/be/test/olap/block_column_predicate_test.cpp @@ -27,7 +27,6 @@ #include "olap/wrapper_field.h" #include "runtime/mem_pool.h" #include "runtime/string_value.hpp" -#include "util/logging.h" #include "vec/columns/predicate_column.h" namespace doris { diff --git a/be/test/olap/bloom_filter_column_predicate_test.cpp b/be/test/olap/bloom_filter_column_predicate_test.cpp index 6092fab5c8..79ba5af940 100644 --- a/be/test/olap/bloom_filter_column_predicate_test.cpp +++ b/be/test/olap/bloom_filter_column_predicate_test.cpp @@ -26,7 +26,6 @@ #include "olap/row_block2.h" #include "runtime/mem_pool.h" #include "runtime/string_value.hpp" -#include "util/logging.h" #include "vec/columns/column_nullable.h" #include "vec/columns/predicate_column.h" #include "vec/core/block.h" diff --git a/be/test/olap/bloom_filter_test.cpp b/be/test/olap/bloom_filter_test.cpp index 8d25b9eabf..f545e9223d 100644 --- a/be/test/olap/bloom_filter_test.cpp +++ b/be/test/olap/bloom_filter_test.cpp @@ -22,7 +22,6 @@ #include #include "common/configbase.h" -#include "util/logging.h" using std::string; diff --git a/be/test/olap/byte_buffer_test.cpp b/be/test/olap/byte_buffer_test.cpp index 8cccb1707c..18c6cbc190 100644 --- a/be/test/olap/byte_buffer_test.cpp +++ b/be/test/olap/byte_buffer_test.cpp @@ -24,7 +24,6 @@ #include "common/configbase.h" #include "olap/file_helper.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/olap/comparison_predicate_test.cpp b/be/test/olap/comparison_predicate_test.cpp index ab61b59fa7..9f6ef32b55 100644 --- a/be/test/olap/comparison_predicate_test.cpp +++ b/be/test/olap/comparison_predicate_test.cpp @@ -28,7 +28,6 @@ #include "runtime/mem_pool.h" #include "runtime/primitive_type.h" #include "runtime/string_value.hpp" -#include "util/logging.h" namespace doris { diff --git a/be/test/olap/delete_handler_test.cpp b/be/test/olap/delete_handler_test.cpp index c0582bdf06..74f6572923 100644 --- a/be/test/olap/delete_handler_test.cpp +++ b/be/test/olap/delete_handler_test.cpp @@ -35,7 +35,6 @@ #include "olap/utils.h" #include "util/cpu_info.h" #include "util/file_utils.h" -#include "util/logging.h" using namespace std; using namespace doris; diff --git a/be/test/olap/delta_writer_test.cpp b/be/test/olap/delta_writer_test.cpp index 08a3aacc93..3c1e7f9924 100644 --- a/be/test/olap/delta_writer_test.cpp +++ b/be/test/olap/delta_writer_test.cpp @@ -37,7 +37,6 @@ #include "runtime/mem_pool.h" #include "runtime/tuple.h" #include "util/file_utils.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/olap/engine_storage_migration_task_test.cpp b/be/test/olap/engine_storage_migration_task_test.cpp index c6edb83360..8f415ec8aa 100644 --- a/be/test/olap/engine_storage_migration_task_test.cpp +++ b/be/test/olap/engine_storage_migration_task_test.cpp @@ -38,7 +38,6 @@ #include "runtime/mem_pool.h" #include "runtime/tuple.h" #include "util/file_utils.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/olap/file_helper_test.cpp b/be/test/olap/file_helper_test.cpp index 259bce5c00..7a7a238817 100644 --- a/be/test/olap/file_helper_test.cpp +++ b/be/test/olap/file_helper_test.cpp @@ -27,7 +27,6 @@ #include "gtest/gtest.h" #include "olap/olap_define.h" #include "testutil/test_util.h" -#include "util/logging.h" #ifndef BE_TEST #define BE_TEST diff --git a/be/test/olap/file_utils_test.cpp b/be/test/olap/file_utils_test.cpp index 038d7c4cfb..51dcd489c4 100644 --- a/be/test/olap/file_utils_test.cpp +++ b/be/test/olap/file_utils_test.cpp @@ -30,7 +30,6 @@ #include "gtest/gtest.h" #include "olap/file_helper.h" #include "olap/olap_define.h" -#include "util/logging.h" #ifndef BE_TEST #define BE_TEST diff --git a/be/test/olap/in_list_predicate_test.cpp b/be/test/olap/in_list_predicate_test.cpp index 5d80f7ec49..eccac83e97 100644 --- a/be/test/olap/in_list_predicate_test.cpp +++ b/be/test/olap/in_list_predicate_test.cpp @@ -27,7 +27,6 @@ #include "runtime/mem_pool.h" #include "runtime/primitive_type.h" #include "runtime/string_value.hpp" -#include "util/logging.h" namespace doris { diff --git a/be/test/olap/lru_cache_test.cpp b/be/test/olap/lru_cache_test.cpp index 0ec5b98a63..c9fb0cbd93 100644 --- a/be/test/olap/lru_cache_test.cpp +++ b/be/test/olap/lru_cache_test.cpp @@ -22,7 +22,6 @@ #include #include "testutil/test_util.h" -#include "util/logging.h" using namespace doris; using namespace std; diff --git a/be/test/olap/memtable_flush_executor_test.cpp b/be/test/olap/memtable_flush_executor_test.cpp index f571ba5044..df2e42e03a 100644 --- a/be/test/olap/memtable_flush_executor_test.cpp +++ b/be/test/olap/memtable_flush_executor_test.cpp @@ -38,7 +38,6 @@ #include "runtime/exec_env.h" #include "runtime/tuple.h" #include "util/file_utils.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/olap/null_predicate_test.cpp b/be/test/olap/null_predicate_test.cpp index 0b51598617..fc879bb241 100644 --- a/be/test/olap/null_predicate_test.cpp +++ b/be/test/olap/null_predicate_test.cpp @@ -26,7 +26,6 @@ #include "olap/row_block2.h" #include "runtime/mem_pool.h" #include "runtime/string_value.hpp" -#include "util/logging.h" #include "vec/columns/column_nullable.h" #include "vec/core/block.h" diff --git a/be/test/olap/row_block_test.cpp b/be/test/olap/row_block_test.cpp index 0c0b146c11..823b104a13 100644 --- a/be/test/olap/row_block_test.cpp +++ b/be/test/olap/row_block_test.cpp @@ -27,7 +27,6 @@ #include "olap/tablet.h" #include "runtime/runtime_state.h" #include "util/debug_util.h" -#include "util/logging.h" using std::vector; using std::string; diff --git a/be/test/olap/row_cursor_test.cpp b/be/test/olap/row_cursor_test.cpp index 9412dfbea7..3d82e58511 100644 --- a/be/test/olap/row_cursor_test.cpp +++ b/be/test/olap/row_cursor_test.cpp @@ -24,7 +24,6 @@ #include "olap/schema.h" #include "olap/tablet_schema.h" #include "runtime/mem_pool.h" -#include "util/logging.h" #include "util/types.h" namespace doris { diff --git a/be/test/olap/rowset/segment_v2/bitshuffle_page_test.cpp b/be/test/olap/rowset/segment_v2/bitshuffle_page_test.cpp index 48b5dc7ec3..e7b3d4adc8 100644 --- a/be/test/olap/rowset/segment_v2/bitshuffle_page_test.cpp +++ b/be/test/olap/rowset/segment_v2/bitshuffle_page_test.cpp @@ -26,7 +26,6 @@ #include "olap/rowset/segment_v2/page_builder.h" #include "olap/rowset/segment_v2/page_decoder.h" #include "runtime/mem_pool.h" -#include "util/logging.h" using doris::segment_v2::PageBuilderOptions; using doris::OlapReaderStatistics; diff --git a/be/test/olap/rowset/segment_v2/bloom_filter_page_test.cpp b/be/test/olap/rowset/segment_v2/bloom_filter_page_test.cpp index 7e78fab5cc..bdae2dfb8b 100644 --- a/be/test/olap/rowset/segment_v2/bloom_filter_page_test.cpp +++ b/be/test/olap/rowset/segment_v2/bloom_filter_page_test.cpp @@ -26,7 +26,6 @@ #include "olap/rowset/segment_v2/page_builder.h" #include "olap/rowset/segment_v2/page_decoder.h" #include "runtime/mem_pool.h" -#include "util/logging.h" using doris::segment_v2::PageBuilderOptions; using doris::segment_v2::PageDecoderOptions; diff --git a/be/test/olap/rowset/segment_v2/rle_page_test.cpp b/be/test/olap/rowset/segment_v2/rle_page_test.cpp index c2970534db..7539e6183c 100644 --- a/be/test/olap/rowset/segment_v2/rle_page_test.cpp +++ b/be/test/olap/rowset/segment_v2/rle_page_test.cpp @@ -25,7 +25,6 @@ #include "olap/rowset/segment_v2/page_builder.h" #include "olap/rowset/segment_v2/page_decoder.h" #include "runtime/mem_pool.h" -#include "util/logging.h" using doris::segment_v2::PageBuilderOptions; using doris::segment_v2::PageDecoderOptions; diff --git a/be/test/runtime/buffered_block_mgr2_test.cpp b/be/test/runtime/buffered_block_mgr2_test.cpp index 9aaadeeea3..09994c0c42 100644 --- a/be/test/runtime/buffered_block_mgr2_test.cpp +++ b/be/test/runtime/buffered_block_mgr2_test.cpp @@ -32,7 +32,6 @@ #include "util/cpu_info.h" #include "util/disk_info.h" #include "util/filesystem_util.h" -#include "util/logging.h" #include "util/monotime.h" #include "util/thread_group.h" diff --git a/be/test/runtime/buffered_tuple_stream2_test.cpp b/be/test/runtime/buffered_tuple_stream2_test.cpp index 5ef517f407..d247325638 100644 --- a/be/test/runtime/buffered_tuple_stream2_test.cpp +++ b/be/test/runtime/buffered_tuple_stream2_test.cpp @@ -33,7 +33,6 @@ #include "util/cpu_info.h" #include "util/debug_util.h" #include "util/disk_info.h" -#include "util/logging.h" using std::vector; diff --git a/be/test/runtime/cache/partition_cache_test.cpp b/be/test/runtime/cache/partition_cache_test.cpp index 7a66c75912..ecbf8c1344 100644 --- a/be/test/runtime/cache/partition_cache_test.cpp +++ b/be/test/runtime/cache/partition_cache_test.cpp @@ -23,7 +23,6 @@ #include "runtime/cache/result_cache.h" #include "testutil/test_util.h" #include "util/cpu_info.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/runtime/data_stream_test.cpp b/be/test/runtime/data_stream_test.cpp index 00ebc6636f..3273a65fd2 100644 --- a/be/test/runtime/data_stream_test.cpp +++ b/be/test/runtime/data_stream_test.cpp @@ -36,7 +36,6 @@ #include "util/cpu_info.h" #include "util/debug_util.h" #include "util/disk_info.h" -#include "util/logging.h" #include "util/mem_info.h" #include "util/thrift_server.h" diff --git a/be/test/runtime/datetime_value_test.cpp b/be/test/runtime/datetime_value_test.cpp index f6bea60c81..b1f8c51d46 100644 --- a/be/test/runtime/datetime_value_test.cpp +++ b/be/test/runtime/datetime_value_test.cpp @@ -22,7 +22,6 @@ #include #include "common/logging.h" -#include "util/logging.h" #include "util/timezone_utils.h" namespace doris { diff --git a/be/test/runtime/decimal_value_test.cpp b/be/test/runtime/decimal_value_test.cpp index b6bbc3631a..e65300c4a7 100644 --- a/be/test/runtime/decimal_value_test.cpp +++ b/be/test/runtime/decimal_value_test.cpp @@ -21,7 +21,6 @@ #include #include "runtime/decimalv2_value.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/runtime/decimalv2_value_test.cpp b/be/test/runtime/decimalv2_value_test.cpp index 100f9e82d4..5ebf69974e 100644 --- a/be/test/runtime/decimalv2_value_test.cpp +++ b/be/test/runtime/decimalv2_value_test.cpp @@ -22,8 +22,6 @@ #include #include -#include "util/logging.h" - namespace doris { class DecimalV2ValueTest : public testing::Test { diff --git a/be/test/runtime/disk_io_mgr_test.cpp b/be/test/runtime/disk_io_mgr_test.cpp index a1ae91711c..f25d8a4e25 100644 --- a/be/test/runtime/disk_io_mgr_test.cpp +++ b/be/test/runtime/disk_io_mgr_test.cpp @@ -26,7 +26,6 @@ #include "util/cpu_info.h" #include "util/disk_info.h" -#include "util/logging.h" #include "util/thread_group.h" using std::string; diff --git a/be/test/runtime/mem_limit_test.cpp b/be/test/runtime/mem_limit_test.cpp index eeadb41b28..470074d0eb 100644 --- a/be/test/runtime/mem_limit_test.cpp +++ b/be/test/runtime/mem_limit_test.cpp @@ -19,7 +19,6 @@ #include "runtime/memory/mem_tracker.h" #include "runtime/memory/mem_tracker_limiter.h" -#include "util/logging.h" #include "util/metrics.h" namespace doris { diff --git a/be/test/runtime/mem_pool_test.cpp b/be/test/runtime/mem_pool_test.cpp index 6cff27dce5..15fe9999b5 100644 --- a/be/test/runtime/mem_pool_test.cpp +++ b/be/test/runtime/mem_pool_test.cpp @@ -21,8 +21,6 @@ #include -#include "util/logging.h" - namespace doris { TEST(MemPoolTest, Basic) { diff --git a/be/test/runtime/memory_scratch_sink_test.cpp b/be/test/runtime/memory_scratch_sink_test.cpp index ef566559f9..bf9c4a2954 100644 --- a/be/test/runtime/memory_scratch_sink_test.cpp +++ b/be/test/runtime/memory_scratch_sink_test.cpp @@ -40,7 +40,6 @@ #include "runtime/tuple_row.h" #include "testutil/desc_tbl_builder.h" #include "util/blocking_queue.hpp" -#include "util/logging.h" namespace doris { diff --git a/be/test/runtime/result_sink_test.cpp b/be/test/runtime/result_sink_test.cpp index fa6f3a5686..05a3caee03 100644 --- a/be/test/runtime/result_sink_test.cpp +++ b/be/test/runtime/result_sink_test.cpp @@ -39,7 +39,6 @@ #include "runtime/runtime_state.h" #include "runtime/tuple_row.h" #include "util/cpu_info.h" -#include "util/logging.h" #include "util/mysql_row_buffer.h" namespace doris { diff --git a/be/test/runtime/routine_load_task_executor_test.cpp b/be/test/runtime/routine_load_task_executor_test.cpp index 01fc299c67..2e5a1e93c6 100644 --- a/be/test/runtime/routine_load_task_executor_test.cpp +++ b/be/test/runtime/routine_load_task_executor_test.cpp @@ -26,7 +26,6 @@ #include "runtime/stream_load/load_stream_mgr.h" #include "runtime/stream_load/stream_load_executor.h" #include "util/cpu_info.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/runtime/tmp_file_mgr_test.cpp b/be/test/runtime/tmp_file_mgr_test.cpp index 3338819dcb..07837bb636 100644 --- a/be/test/runtime/tmp_file_mgr_test.cpp +++ b/be/test/runtime/tmp_file_mgr_test.cpp @@ -25,7 +25,6 @@ #include "gen_cpp/Types_types.h" // for TUniqueId #include "util/disk_info.h" #include "util/filesystem_util.h" -#include "util/logging.h" #include "util/metrics.h" using std::filesystem::path; diff --git a/be/test/testutil/run_all_tests.cpp b/be/test/testutil/run_all_tests.cpp index 037cf91891..77352db010 100644 --- a/be/test/testutil/run_all_tests.cpp +++ b/be/test/testutil/run_all_tests.cpp @@ -25,7 +25,6 @@ #include "service/backend_options.h" #include "util/cpu_info.h" #include "util/disk_info.h" -#include "util/logging.h" #include "util/mem_info.h" int main(int argc, char** argv) { diff --git a/be/test/udf/uda_test.cpp b/be/test/udf/uda_test.cpp index 79fc599762..9be9d830eb 100644 --- a/be/test/udf/uda_test.cpp +++ b/be/test/udf/uda_test.cpp @@ -21,7 +21,6 @@ #include "common/logging.h" #include "udf/uda_test_harness.h" -#include "util/logging.h" namespace doris_udf { diff --git a/be/test/udf/udf_test.cpp b/be/test/udf/udf_test.cpp index 33ea42d2e5..6b4d886b0f 100644 --- a/be/test/udf/udf_test.cpp +++ b/be/test/udf/udf_test.cpp @@ -21,7 +21,6 @@ #include "common/logging.h" #include "udf/udf_test_harness.hpp" -#include "util/logging.h" namespace doris_udf { diff --git a/be/test/util/arrow/arrow_work_flow_test.cpp b/be/test/util/arrow/arrow_work_flow_test.cpp index 240a9e7ea2..0475d479a4 100644 --- a/be/test/util/arrow/arrow_work_flow_test.cpp +++ b/be/test/util/arrow/arrow_work_flow_test.cpp @@ -39,7 +39,6 @@ #include "util/cpu_info.h" #include "util/debug_util.h" #include "util/disk_info.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/util/bit_util_test.cpp b/be/test/util/bit_util_test.cpp index cdfc2c1a39..008b0b71f0 100644 --- a/be/test/util/bit_util_test.cpp +++ b/be/test/util/bit_util_test.cpp @@ -26,7 +26,6 @@ #include "common/config.h" #include "util/cpu_info.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/util/cidr_test.cpp b/be/test/util/cidr_test.cpp index 33e4dd88f0..1a35b27587 100644 --- a/be/test/util/cidr_test.cpp +++ b/be/test/util/cidr_test.cpp @@ -25,7 +25,6 @@ #include "common/configbase.h" #include "util/cpu_info.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/util/doris_metrics_test.cpp b/be/test/util/doris_metrics_test.cpp index 2dcea035b2..9e94572884 100644 --- a/be/test/util/doris_metrics_test.cpp +++ b/be/test/util/doris_metrics_test.cpp @@ -20,7 +20,6 @@ #include #include "common/config.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/util/filesystem_util_test.cpp b/be/test/util/filesystem_util_test.cpp index e69163ab56..67f3891cb7 100644 --- a/be/test/util/filesystem_util_test.cpp +++ b/be/test/util/filesystem_util_test.cpp @@ -23,7 +23,6 @@ #include #include "common/configbase.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/util/http_channel_test.cpp b/be/test/util/http_channel_test.cpp index 3c3ae12edd..c8c6fdffa0 100644 --- a/be/test/util/http_channel_test.cpp +++ b/be/test/util/http_channel_test.cpp @@ -19,7 +19,6 @@ #include -#include "util/logging.h" #include "util/zlib.h" namespace doris { diff --git a/be/test/util/internal_queue_test.cpp b/be/test/util/internal_queue_test.cpp index 5949560ac9..fe11c9059e 100644 --- a/be/test/util/internal_queue_test.cpp +++ b/be/test/util/internal_queue_test.cpp @@ -25,7 +25,6 @@ #include "common/configbase.h" #include "testutil/test_util.h" -#include "util/logging.h" #include "util/thread_group.h" using std::vector; diff --git a/be/test/util/lru_cache_util_test.cpp b/be/test/util/lru_cache_util_test.cpp index 98d7246b33..59ae10adcf 100644 --- a/be/test/util/lru_cache_util_test.cpp +++ b/be/test/util/lru_cache_util_test.cpp @@ -20,7 +20,6 @@ #include #include "common/config.h" -#include "util/logging.h" #include "util/lru_cache.hpp" namespace doris { diff --git a/be/test/util/metrics_test.cpp b/be/test/util/metrics_test.cpp index e71801e9bd..9722e660ea 100644 --- a/be/test/util/metrics_test.cpp +++ b/be/test/util/metrics_test.cpp @@ -24,7 +24,6 @@ #include "common/config.h" #include "testutil/test_util.h" -#include "util/logging.h" #include "util/stopwatch.hpp" namespace doris { diff --git a/be/test/util/mysql_row_buffer_test.cpp b/be/test/util/mysql_row_buffer_test.cpp index a575e7653a..fdfc11f186 100644 --- a/be/test/util/mysql_row_buffer_test.cpp +++ b/be/test/util/mysql_row_buffer_test.cpp @@ -25,7 +25,6 @@ #include "env/env.h" #include "gutil/strings/util.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/util/path_trie_test.cpp b/be/test/util/path_trie_test.cpp index bd599e1bab..3e590b9194 100644 --- a/be/test/util/path_trie_test.cpp +++ b/be/test/util/path_trie_test.cpp @@ -20,7 +20,6 @@ #include #include "common/config.h" -#include "util/logging.h" namespace doris { diff --git a/be/test/util/path_util_test.cpp b/be/test/util/path_util_test.cpp index a6c5ba54b3..7be3e543bc 100644 --- a/be/test/util/path_util_test.cpp +++ b/be/test/util/path_util_test.cpp @@ -23,7 +23,6 @@ #include #include "common/config.h" -#include "util/logging.h" using std::string; using std::vector; diff --git a/be/test/util/s3_uri_test.cpp b/be/test/util/s3_uri_test.cpp index 3263521e52..f166cf0e95 100644 --- a/be/test/util/s3_uri_test.cpp +++ b/be/test/util/s3_uri_test.cpp @@ -21,8 +21,6 @@ #include -#include "util/logging.h" - namespace doris { class S3URITest : public testing::Test { diff --git a/be/test/util/string_parser_test.cpp b/be/test/util/string_parser_test.cpp index c3428a8068..a85e5b97ed 100644 --- a/be/test/util/string_parser_test.cpp +++ b/be/test/util/string_parser_test.cpp @@ -24,8 +24,6 @@ #include #include -#include "util/logging.h" - namespace doris { std::string space[] = {"", " ", "\t\t\t", "\n\n\n", "\v\v\v", "\f\f\f", "\r\r\r"}; diff --git a/be/test/util/system_metrics_test.cpp b/be/test/util/system_metrics_test.cpp index 23ac0e1a46..b83da94d3a 100644 --- a/be/test/util/system_metrics_test.cpp +++ b/be/test/util/system_metrics_test.cpp @@ -21,7 +21,6 @@ #include "common/config.h" #include "testutil/test_util.h" -#include "util/logging.h" #include "util/metrics.h" #include "util/stopwatch.hpp" diff --git a/be/test/util/url_coding_test.cpp b/be/test/util/url_coding_test.cpp index e62f13018c..c9d0471abd 100644 --- a/be/test/util/url_coding_test.cpp +++ b/be/test/util/url_coding_test.cpp @@ -23,8 +23,6 @@ #include -#include "util/logging.h" - namespace doris { // Tests encoding/decoding of input. If expected_encoded is non-empty, the diff --git a/be/test/util/zip_util_test.cpp b/be/test/util/zip_util_test.cpp index cf4aecac97..1850b3241c 100644 --- a/be/test/util/zip_util_test.cpp +++ b/be/test/util/zip_util_test.cpp @@ -26,7 +26,6 @@ #include "gutil/strings/util.h" #include "testutil/test_util.h" #include "util/file_utils.h" -#include "util/logging.h" namespace doris {