diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt index 7cc9b2cb3c9a47..228d6c5441455a 100644 --- a/be/CMakeLists.txt +++ b/be/CMakeLists.txt @@ -534,6 +534,7 @@ set(KRB5_LIBS set(AWS_LIBS aws-sdk-s3 aws-sdk-core + aws-sdk-transfer aws-checksums aws-c-io aws-c-event-stream @@ -727,6 +728,7 @@ add_subdirectory(${SRC_DIR}/geo) add_subdirectory(${SRC_DIR}/io) add_subdirectory(${SRC_DIR}/gutil) add_subdirectory(${SRC_DIR}/http) +add_subdirectory(${SRC_DIR}/io) add_subdirectory(${SRC_DIR}/olap) add_subdirectory(${SRC_DIR}/runtime) add_subdirectory(${SRC_DIR}/service) diff --git a/be/src/agent/agent_server.cpp b/be/src/agent/agent_server.cpp index 34cdfe57314fac..f43394c4b4d8eb 100644 --- a/be/src/agent/agent_server.cpp +++ b/be/src/agent/agent_server.cpp @@ -40,7 +40,7 @@ AgentServer::AgentServer(ExecEnv* exec_env, const TMasterInfo& master_info) : _exec_env(exec_env), _master_info(master_info), _topic_subscriber(new TopicSubscriber()) { for (auto& path : exec_env->store_paths()) { try { - string dpp_download_path_str = path.path + DPP_PREFIX; + string dpp_download_path_str = path.path + "/" + DPP_PREFIX; std::filesystem::path dpp_download_path(dpp_download_path_str); if (std::filesystem::exists(dpp_download_path)) { std::filesystem::remove_all(dpp_download_path); @@ -90,7 +90,8 @@ AgentServer::AgentServer(ExecEnv* exec_env, const TMasterInfo& master_info) CREATE_AND_START_THREAD(REPORT_DISK_STATE, _report_disk_state_workers); CREATE_AND_START_THREAD(REPORT_OLAP_TABLE, _report_tablet_workers); CREATE_AND_START_POOL(SUBMIT_TABLE_COMPACTION, _submit_table_compaction_workers); - CREATE_AND_START_POOL(STORAGE_MEDIUM_MIGRATE_V2, _storage_medium_migrate_v2_workers); + CREATE_AND_START_POOL(REFRESH_STORAGE_POLICY, _storage_refresh_policy_workers); + CREATE_AND_START_POOL(UPDATE_STORAGE_POLICY, _storage_update_policy_workers); #undef CREATE_AND_START_POOL #undef CREATE_AND_START_THREAD @@ -153,8 +154,8 @@ void AgentServer::submit_tasks(TAgentResult& agent_result, HANDLE_TYPE(TTaskType::UPDATE_TABLET_META_INFO, _update_tablet_meta_info_workers, update_tablet_meta_info_req); HANDLE_TYPE(TTaskType::COMPACTION, _submit_table_compaction_workers, compaction_req); - HANDLE_TYPE(TTaskType::STORAGE_MEDIUM_MIGRATE_V2, _storage_medium_migrate_v2_workers, - storage_migration_req_v2); + HANDLE_TYPE(TTaskType::NOTIFY_UPDATE_STORAGE_POLICY, _storage_update_policy_workers, + update_policy); case TTaskType::REALTIME_PUSH: case TTaskType::PUSH: diff --git a/be/src/agent/agent_server.h b/be/src/agent/agent_server.h index 3998a6b2580f42..bc3d6e3f07443f 100644 --- a/be/src/agent/agent_server.h +++ b/be/src/agent/agent_server.h @@ -88,8 +88,8 @@ class AgentServer { std::unique_ptr _submit_table_compaction_workers; - std::unique_ptr _storage_medium_migrate_v2_workers; - + std::unique_ptr _storage_refresh_policy_workers; + std::unique_ptr _storage_update_policy_workers; std::unique_ptr _topic_subscriber; }; diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index c5fbe42c8dbacb..41e84297e3bee0 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -35,6 +35,7 @@ #include "olap/olap_common.h" #include "olap/snapshot_manager.h" #include "olap/storage_engine.h" +#include "olap/storage_policy_mgr.h" #include "olap/tablet.h" #include "olap/task/engine_alter_tablet_task.h" #include "olap/task/engine_batch_load_task.h" @@ -42,7 +43,6 @@ #include "olap/task/engine_clone_task.h" #include "olap/task/engine_publish_version_task.h" #include "olap/task/engine_storage_migration_task.h" -#include "olap/task/engine_storage_migration_task_v2.h" #include "olap/utils.h" #include "runtime/exec_env.h" #include "runtime/snapshot_loader.h" @@ -193,9 +193,14 @@ void TaskWorkerPool::start() { cb = std::bind(&TaskWorkerPool::_submit_table_compaction_worker_thread_callback, this); break; - case TaskWorkerType::STORAGE_MEDIUM_MIGRATE_V2: + case TaskWorkerType::REFRESH_STORAGE_POLICY: _worker_count = 1; - cb = std::bind(&TaskWorkerPool::_storage_medium_migrate_v2_worker_thread_callback, + cb = std::bind( + &TaskWorkerPool::_storage_refresh_storage_policy_worker_thread_callback, this); + break; + case TaskWorkerType::UPDATE_STORAGE_POLICY: + _worker_count = 1; + cb = std::bind(&TaskWorkerPool::_storage_update_storage_policy_worker_thread_callback, this); break; default: @@ -361,6 +366,7 @@ void TaskWorkerPool::_create_tablet_worker_thread_callback() { TStatus task_status; std::vector finish_tablet_infos; + LOG(INFO) << "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 @@ -437,6 +443,11 @@ void TaskWorkerPool::_drop_tablet_worker_thread_callback() { 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()); + // We remove remote rowset directly. + // TODO(cyx): do remove in background + if (drop_tablet_req.is_drop_table_or_partition) { + dropped_tablet->remove_all_remote_rowsets(); + } } else { status_code = TStatusCode::NOT_FOUND; error_msgs.push_back(err); @@ -854,8 +865,15 @@ void TaskWorkerPool::_update_tablet_meta_worker_thread_callback() { tablet->set_partition_id(tablet_meta_info.partition_id); break; case TTabletMetaType::INMEMORY: - tablet->tablet_meta()->mutable_tablet_schema()->set_is_in_memory( - tablet_meta_info.is_in_memory); + if (tablet_meta_info.storage_policy.empty()) { + tablet->tablet_meta()->mutable_tablet_schema()->set_is_in_memory( + tablet_meta_info.is_in_memory); + } else { + LOG(INFO) << "set tablet cooldown resource " + << tablet_meta_info.storage_policy; + tablet->tablet_meta()->set_cooldown_resource( + tablet_meta_info.storage_policy); + } break; } } @@ -1036,8 +1054,8 @@ Status TaskWorkerPool::_check_migrate_request(const TStorageMediumMigrateReq& re return Status::OLAPInternalError(OLAP_REQUEST_FAILED); } - // check disk capacity - int64_t tablet_size = tablet->tablet_footprint(); + // 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; @@ -1171,14 +1189,15 @@ void TaskWorkerPool::_report_disk_state_worker_thread_callback() { map disks; for (auto& root_path_info : data_dir_infos) { TDisk disk; - disk.__set_root_path(root_path_info.path_desc.filepath); + disk.__set_root_path(root_path_info.path); disk.__set_path_hash(root_path_info.path_hash); disk.__set_storage_medium(root_path_info.storage_medium); disk.__set_disk_total_capacity(root_path_info.disk_capacity); - disk.__set_data_used_capacity(root_path_info.data_used_capacity); + disk.__set_data_used_capacity(root_path_info.local_used_capacity); + disk.__set_remote_used_capacity(root_path_info.remote_used_capacity); disk.__set_disk_available_capacity(root_path_info.available); disk.__set_used(root_path_info.is_used); - disks[root_path_info.path_desc.filepath] = disk; + disks[root_path_info.path] = disk; } request.__set_disks(disks); _handle_report(request, ReportType::DISK); @@ -1565,7 +1584,6 @@ Status TaskWorkerPool::_move_dir(const TTabletId tablet_id, const std::string& s return Status::InvalidArgument("Could not find tablet"); } - std::string dest_tablet_dir = tablet->tablet_path_desc().filepath; SnapshotLoader loader(_env, job_id, tablet_id); Status status = loader.move(src, tablet, overwrite); @@ -1683,9 +1701,64 @@ void TaskWorkerPool::_submit_table_compaction_worker_thread_callback() { } } -void TaskWorkerPool::_storage_medium_migrate_v2_worker_thread_callback() { +void TaskWorkerPool::_storage_refresh_storage_policy_worker_thread_callback() { + while (_is_work) { + _is_doing_work = false; + // wait at most report_task_interval_seconds, or being notified + std::unique_lock worker_thread_lock(_worker_thread_lock); + _worker_thread_condition_variable.wait_for( + worker_thread_lock, + std::chrono::seconds(config::storage_refresh_storage_policy_task_interval_seconds)); + if (!_is_work) { + break; + } + + if (_master_info.network_address.port == 0) { + // port == 0 means not received heartbeat yet + // sleep a short time and try again + LOG(INFO) + << "waiting to receive first heartbeat from frontend before doing task report"; + continue; + } + + _is_doing_work = true; + + TGetStoragePolicyResult result; + Status status = _master_client->refresh_storage_policy(&result); + if (!status.ok()) { + LOG(WARNING) << "refresh storage policy status not ok. "; + } else if (result.status.status_code != TStatusCode::OK) { + LOG(WARNING) << "refresh storage policy result status status_code not ok. "; + } else { + // update storage policy mgr. + StoragePolicyMgr* spm = ExecEnv::GetInstance()->storage_policy_mgr(); + for (const auto& iter : result.result_entrys) { + shared_ptr policy_ptr = make_shared(); + policy_ptr->storage_policy_name = iter.policy_name; + policy_ptr->cooldown_datetime = iter.cooldown_datetime; + policy_ptr->cooldown_ttl = iter.cooldown_ttl; + policy_ptr->s3_endpoint = iter.s3_storage_param.s3_endpoint; + policy_ptr->s3_region = iter.s3_storage_param.s3_region; + policy_ptr->s3_ak = iter.s3_storage_param.s3_ak; + policy_ptr->s3_sk = iter.s3_storage_param.s3_sk; + policy_ptr->root_path = iter.s3_storage_param.root_path; + policy_ptr->bucket = iter.s3_storage_param.bucket; + policy_ptr->s3_conn_timeout_ms = iter.s3_storage_param.s3_conn_timeout_ms; + policy_ptr->s3_max_conn = iter.s3_storage_param.s3_max_conn; + policy_ptr->s3_request_timeout_ms = iter.s3_storage_param.s3_request_timeout_ms; + policy_ptr->md5_sum = iter.md5_checksum; + + LOG(INFO) << "refresh storage policy task, policy " << *policy_ptr; + spm->periodic_put(iter.policy_name, std::move(policy_ptr)); + } + } + } +} + +void TaskWorkerPool::_storage_update_storage_policy_worker_thread_callback() { while (_is_work) { TAgentTaskRequest agent_task_req; + TGetStoragePolicy get_storage_policy_req; { std::unique_lock worker_thread_lock(_worker_thread_lock); while (_is_work && _tasks.empty()) { @@ -1696,115 +1769,32 @@ void TaskWorkerPool::_storage_medium_migrate_v2_worker_thread_callback() { } agent_task_req = _tasks.front(); + get_storage_policy_req = agent_task_req.update_policy; _tasks.pop_front(); } - int64_t signature = agent_task_req.signature; - LOG(INFO) << "get migration table v2 task, signature: " << agent_task_req.signature; - bool is_task_timeout = false; - if (agent_task_req.__isset.recv_time) { - int64_t time_elapsed = time(nullptr) - agent_task_req.recv_time; - if (time_elapsed > config::report_task_interval_seconds * 20) { - LOG(INFO) << "task elapsed " << time_elapsed - << " seconds since it is inserted to queue, it is timeout"; - is_task_timeout = true; - } - } - if (!is_task_timeout) { - TFinishTaskRequest finish_task_request; - TTaskType::type task_type = agent_task_req.task_type; - switch (task_type) { - case TTaskType::STORAGE_MEDIUM_MIGRATE_V2: - _storage_medium_migrate_v2(agent_task_req, signature, task_type, - &finish_task_request); - break; - default: - // pass - break; - } - _finish_task(finish_task_request); - } - _remove_task_info(agent_task_req.task_type, agent_task_req.signature); - } -} - -void TaskWorkerPool::_storage_medium_migrate_v2(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; - - string process_name; - switch (task_type) { - case TTaskType::STORAGE_MEDIUM_MIGRATE_V2: - process_name = "StorageMediumMigrationV2"; - break; - default: - std::string task_name; - EnumToString(TTaskType, task_type, task_name); - LOG(WARNING) << "Storage medium migration v2 type invalid. type: " << task_name - << ", signature: " << signature; - status = Status::NotSupported("Storage medium migration v2 type invalid"); - break; - } - - // Check last storage medium migration v2 status, if failed delete tablet file - // Do not need to adjust delete success or not - // Because if delete failed task will failed - TTabletId new_tablet_id; - TSchemaHash new_schema_hash = 0; - if (status.ok()) { - new_tablet_id = agent_task_req.storage_migration_req_v2.new_tablet_id; - new_schema_hash = agent_task_req.storage_migration_req_v2.new_schema_hash; - EngineStorageMigrationTaskV2 engine_task(agent_task_req.storage_migration_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(); - } - } - if (status.ok()) { - ++_s_report_version; - LOG(INFO) << process_name << " finished. signature: " << signature; - } - - // Return result to fe - finish_task_request->__set_backend(_backend); - finish_task_request->__set_report_version(_s_report_version); - finish_task_request->__set_task_type(task_type); - finish_task_request->__set_signature(signature); - - std::vector finish_tablet_infos; - 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 { - 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"); - } else { - LOG(WARNING) << process_name << " failed. signature: " << signature; - error_msgs.push_back(process_name + " failed"); - error_msgs.push_back("status: " + status.to_string()); + StoragePolicyMgr* spm = ExecEnv::GetInstance()->storage_policy_mgr(); + shared_ptr policy_ptr = make_shared(); + policy_ptr->storage_policy_name = get_storage_policy_req.policy_name; + policy_ptr->cooldown_datetime = get_storage_policy_req.cooldown_datetime; + policy_ptr->cooldown_ttl = get_storage_policy_req.cooldown_ttl; + policy_ptr->s3_endpoint = get_storage_policy_req.s3_storage_param.s3_endpoint; + policy_ptr->s3_region = get_storage_policy_req.s3_storage_param.s3_region; + policy_ptr->s3_ak = get_storage_policy_req.s3_storage_param.s3_ak; + policy_ptr->s3_sk = get_storage_policy_req.s3_storage_param.s3_sk; + policy_ptr->root_path = get_storage_policy_req.s3_storage_param.root_path; + policy_ptr->bucket = get_storage_policy_req.s3_storage_param.bucket; + policy_ptr->s3_conn_timeout_ms = get_storage_policy_req.s3_storage_param.s3_conn_timeout_ms; + policy_ptr->s3_max_conn = get_storage_policy_req.s3_storage_param.s3_max_conn; + policy_ptr->s3_request_timeout_ms = + 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; + + spm->update(get_storage_policy_req.policy_name, std::move(policy_ptr)); + _remove_task_info(agent_task_req.task_type, agent_task_req.signature); } - task_status.__set_status_code(status.code()); - task_status.__set_error_msgs(error_msgs); - finish_task_request->__set_task_status(task_status); } } // namespace doris diff --git a/be/src/agent/task_worker_pool.h b/be/src/agent/task_worker_pool.h index 0e9615f15b3d55..ae8b9e81491b7a 100644 --- a/be/src/agent/task_worker_pool.h +++ b/be/src/agent/task_worker_pool.h @@ -66,7 +66,8 @@ class TaskWorkerPool { RECOVER_TABLET, UPDATE_TABLET_META_INFO, SUBMIT_TABLE_COMPACTION, - STORAGE_MEDIUM_MIGRATE_V2 + REFRESH_STORAGE_POLICY, + UPDATE_STORAGE_POLICY }; enum ReportType { TASK, DISK, TABLET }; @@ -122,8 +123,10 @@ class TaskWorkerPool { return "UPDATE_TABLET_META_INFO"; case SUBMIT_TABLE_COMPACTION: return "SUBMIT_TABLE_COMPACTION"; - case STORAGE_MEDIUM_MIGRATE_V2: - return "STORAGE_MEDIUM_MIGRATE_V2"; + case REFRESH_STORAGE_POLICY: + return "REFRESH_STORAGE_POLICY"; + case UPDATE_STORAGE_POLICY: + return "UPDATE_STORAGE_POLICY"; default: return "Unknown"; } @@ -187,7 +190,8 @@ class TaskWorkerPool { void _move_dir_thread_callback(); void _update_tablet_meta_worker_thread_callback(); void _submit_table_compaction_worker_thread_callback(); - void _storage_medium_migrate_v2_worker_thread_callback(); + void _storage_refresh_storage_policy_worker_thread_callback(); + void _storage_update_storage_policy_worker_thread_callback(); void _alter_tablet(const TAgentTaskRequest& alter_tablet_request, int64_t signature, const TTaskType::type task_type, TFinishTaskRequest* finish_task_request); @@ -205,10 +209,6 @@ class TaskWorkerPool { // random sleep 1~second seconds void _random_sleep(int second); - void _storage_medium_migrate_v2(const TAgentTaskRequest& agent_task_req, int64_t signature, - const TTaskType::type task_type, - TFinishTaskRequest* finish_task_request); - private: std::string _name; diff --git a/be/src/agent/utils.cpp b/be/src/agent/utils.cpp index ae243c2165f5d3..7a9a90a16177ca 100644 --- a/be/src/agent/utils.cpp +++ b/be/src/agent/utils.cpp @@ -131,6 +131,46 @@ Status MasterServerClient::report(const TReportRequest& request, TMasterResult* return Status::OK(); } +Status MasterServerClient::refresh_storage_policy(TGetStoragePolicyResult* result) { + Status client_status; + FrontendServiceConnection client(_client_cache, _master_info.network_address, + config::thrift_rpc_timeout_ms, &client_status); + + if (!client_status.ok()) { + LOG(WARNING) << "fail to get master client from cache. " + << "host=" << _master_info.network_address.hostname + << ", port=" << _master_info.network_address.port + << ", code=" << client_status.code(); + return Status::InternalError("Fail to get master client from cache"); + } + + try { + try { + client->refreshStoragePolicy(*result); + } catch (TTransportException& e) { + LOG(WARNING) << "master client, retry refresh_storage_policy: " << e.what(); + client_status = client.reopen(config::thrift_rpc_timeout_ms); + if (!client_status.ok()) { + LOG(WARNING) << "fail to get master client from cache. " + << "host=" << _master_info.network_address.hostname + << ", port=" << _master_info.network_address.port + << ", code=" << client_status.code(); + return Status::InternalError("Master client refresh storage policy failed"); + } + client->refreshStoragePolicy(*result); + } + } catch (TException& e) { + client.reopen(config::thrift_rpc_timeout_ms); + LOG(WARNING) << "fail to report to master. " + << "host=" << _master_info.network_address.hostname + << ", port=" << _master_info.network_address.port + << ", code=" << client_status.code(); + return Status::InternalError("Fail to refresh storage policy from master"); + } + + return Status::OK(); +} + bool AgentUtils::exec_cmd(const string& command, string* errmsg, bool redirect_stderr) { // The exit status of the command. uint32_t rc = 0; diff --git a/be/src/agent/utils.h b/be/src/agent/utils.h index 2785e562e48166..654d8093cda3c3 100644 --- a/be/src/agent/utils.h +++ b/be/src/agent/utils.h @@ -49,6 +49,15 @@ class MasterServerClient { // * result: The result of report task virtual Status report(const TReportRequest& request, TMasterResult* result); + // refreshStoragePolicy get storage policy from the master server + // + // Input parameters: + // * request: The name of storage policy + // + // Output parameters: + // * result: The result of storage policy + virtual Status refresh_storage_policy(TGetStoragePolicyResult* result); + private: DISALLOW_COPY_AND_ASSIGN(MasterServerClient); diff --git a/be/src/common/config.h b/be/src/common/config.h index 56e73823208fa8..2f2f8dd7c755e4 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -103,6 +103,8 @@ CONF_Int32(make_snapshot_worker_count, "5"); CONF_Int32(release_snapshot_worker_count, "5"); // the interval time(seconds) for agent report tasks signatrue to FE CONF_mInt32(report_task_interval_seconds, "10"); +// the interval time(seconds) for refresh storage policy from FE +CONF_mInt32(storage_refresh_storage_policy_task_interval_seconds, "5"); // the interval time(seconds) for agent report disk state to FE CONF_mInt32(report_disk_state_interval_seconds, "60"); // the interval time(seconds) for agent report olap table to FE @@ -755,6 +757,14 @@ CONF_Int32(quick_compaction_batch_size, "10"); // do compaction min rowsets CONF_Int32(quick_compaction_min_rowsets, "10"); +// cooldown task configs +CONF_Int32(cooldown_thread_num, "5"); +CONF_mInt64(generate_cooldown_task_interval_sec, "20"); +CONF_Int32(concurrency_per_dir, "2"); +CONF_mInt64(cooldown_lag_time_sec, "10800"); // 3h + +CONF_Int32(s3_transfer_executor_pool_size, "2"); + } // namespace config } // namespace doris diff --git a/be/src/env/env.h b/be/src/env/env.h index bf09d4a488048b..2456df9a325cbb 100644 --- a/be/src/env/env.h +++ b/be/src/env/env.h @@ -29,6 +29,10 @@ struct WritableFileOptions; struct RandomAccessFileOptions; struct RandomRWFileOptions; +namespace io { +class FileSystem; +} + class Env { public: // Governs if/how the file is created. @@ -190,6 +194,8 @@ struct FilePathDesc { std::string filepath; std::string remote_path; std::string storage_name; + io::FileSystem* file_system; + std::string debug_string() const { std::stringstream ss; ss << "storage_medium: " << to_string(storage_medium) << ", local_path: " << filepath; diff --git a/be/src/http/action/tablet_migration_action.cpp b/be/src/http/action/tablet_migration_action.cpp index 5f7a9e218ecd7e..d70161e64d77e7 100644 --- a/be/src/http/action/tablet_migration_action.cpp +++ b/be/src/http/action/tablet_migration_action.cpp @@ -205,8 +205,8 @@ Status TabletMigrationAction::_check_migrate_request(int64_t tablet_id, int32_t return Status::AlreadyExist("Tablet already exist in destination disk"); } - // check disk capacity - int64_t tablet_size = tablet->tablet_footprint(); + // 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; diff --git a/be/src/io/CMakeLists.txt b/be/src/io/CMakeLists.txt index e55a72188c1d74..768a33ccb1b3af 100644 --- a/be/src/io/CMakeLists.txt +++ b/be/src/io/CMakeLists.txt @@ -22,18 +22,24 @@ set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/io") set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/io") set(IO_FILES - broker_reader.cpp - broker_writer.cpp - buffered_reader.cpp - file_factory.cpp - hdfs_builder.cpp - hdfs_file_reader.cpp - hdfs_reader_writer.cpp - hdfs_writer.cpp - local_file_reader.cpp - local_file_writer.cpp - s3_reader.cpp - s3_writer.cpp + broker_reader.cpp + broker_writer.cpp + buffered_reader.cpp + file_factory.cpp + hdfs_builder.cpp + hdfs_file_reader.cpp + hdfs_reader_writer.cpp + hdfs_writer.cpp + local_file_reader.cpp + local_file_writer.cpp + s3_reader.cpp + s3_writer.cpp + fs/file_system_map.cpp + fs/local_file_reader.cpp + fs/local_file_system.cpp + fs/local_file_writer.cpp + fs/s3_file_reader.cpp + fs/s3_file_system.cpp ) add_library(IO STATIC diff --git a/be/src/io/fs/file_reader.h b/be/src/io/fs/file_reader.h new file mode 100644 index 00000000000000..b9c02f66dcd53a --- /dev/null +++ b/be/src/io/fs/file_reader.h @@ -0,0 +1,27 @@ +#pragma once + +#include "common/status.h" +#include "gutil/macros.h" +#include "io/fs/path.h" + +namespace doris { +namespace io { + +class FileReader { +public: + FileReader() = default; + virtual ~FileReader() = default; + + DISALLOW_COPY_AND_ASSIGN(FileReader); + + virtual Status close() = 0; + + virtual Status read_at(size_t offset, Slice result, size_t* bytes_read) = 0; + + virtual const Path& path() const = 0; + + virtual size_t size() const = 0; +}; + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/file_system.h b/be/src/io/fs/file_system.h new file mode 100644 index 00000000000000..bc49c34280c293 --- /dev/null +++ b/be/src/io/fs/file_system.h @@ -0,0 +1,67 @@ +#pragma once + +#include + +#include "common/status.h" +#include "gutil/macros.h" +#include "io/fs/path.h" + +namespace doris { +namespace io { + +class FileWriter; +class FileReader; + +// Currently, FE use resource name to identify a Resource. +using ResourceId = std::string; + +enum class FileSystemType : uint8_t { + LOCAL, + S3, +}; + +class FileSystem { +public: + FileSystem(Path&& root_path, ResourceId&& resource_id, FileSystemType type) + : _root_path(std::move(root_path)), _resource_id(std::move(resource_id)), _type(type) {} + + virtual ~FileSystem() = default; + + DISALLOW_COPY_AND_ASSIGN(FileSystem); + + virtual Status create_file(const Path& path, std::unique_ptr* writer) = 0; + + virtual Status open_file(const Path& path, std::unique_ptr* reader) = 0; + + virtual Status delete_file(const Path& path) = 0; + + // create directory recursively + virtual Status create_directory(const Path& path) = 0; + + // remove all under directory recursively + virtual Status delete_directory(const Path& path) = 0; + + // hard link `src` to `dest` + // FIXME(cyx): Should we move this method to LocalFileSystem? + virtual Status link_file(const Path& src, const Path& dest) = 0; + + virtual Status exists(const Path& path, bool* res) const = 0; + + virtual Status file_size(const Path& path, size_t* file_size) const = 0; + + virtual Status list(const Path& path, std::vector* files) = 0; + + const Path& root_path() const { return _root_path; } + const ResourceId& resource_id() const { return _resource_id; } + const FileSystemType type() const { return _type; } + +protected: + Path _root_path; + ResourceId _resource_id; + FileSystemType _type; +}; + +using FileSystemPtr = std::shared_ptr; + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/file_system_map.cpp b/be/src/io/fs/file_system_map.cpp new file mode 100644 index 00000000000000..303d17fadf710d --- /dev/null +++ b/be/src/io/fs/file_system_map.cpp @@ -0,0 +1,26 @@ +#include "io/fs/file_system_map.h" + +namespace doris { +namespace io { + +FileSystemMap* FileSystemMap::instance() { + static FileSystemMap map; + return ↦ +} + +void FileSystemMap::insert(ResourceId id, FileSystemPtr fs) { + std::unique_lock wlock(_mu); + _map.try_emplace(std::move(id), std::move(fs)); +} + +FileSystemPtr FileSystemMap::get(const ResourceId& id) { + std::shared_lock rlock(_mu); + auto it = _map.find(id); + if (it != _map.end()) { + return it->second; + } + return nullptr; +} + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/file_system_map.h b/be/src/io/fs/file_system_map.h new file mode 100644 index 00000000000000..bb33c80ad00bc1 --- /dev/null +++ b/be/src/io/fs/file_system_map.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include +#include + +#include "io/fs/file_system.h" + +namespace doris { +namespace io { + +class FileSystemMap { +public: + static FileSystemMap* instance(); + ~FileSystemMap() = default; + + void insert(ResourceId id, FileSystemPtr fs); + + // If `id` is not in `_map`, return nullptr. + FileSystemPtr get(const ResourceId& id); + +private: + FileSystemMap() = default; + +private: + std::shared_mutex _mu; + std::unordered_map _map; // GUARED_BY(_mu) +}; + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/file_writer.h b/be/src/io/fs/file_writer.h new file mode 100644 index 00000000000000..89f987ea1e889a --- /dev/null +++ b/be/src/io/fs/file_writer.h @@ -0,0 +1,42 @@ +#pragma once + +#include "common/status.h" +#include "gutil/macros.h" +#include "io/fs/path.h" + +namespace doris { +namespace io { + +class FileWriter { +public: + FileWriter(Path&& path) : _path(std::move(path)) {} + virtual ~FileWriter() = default; + + DISALLOW_COPY_AND_ASSIGN(FileWriter); + + // Normal close. Wait for all data to persist before returning. + virtual Status close() = 0; + + // Abnormal close and remove this file. + virtual Status abort() = 0; + + virtual Status append(const Slice& data) = 0; + + virtual Status appendv(const Slice* data, size_t data_cnt) = 0; + + virtual Status write_at(size_t offset, const Slice& data) = 0; + + // Call this method when there is no more data to write. + // FIXME(cyx): Does not seem to be an appropriate interface for file system? + virtual Status finalize() = 0; + + virtual size_t bytes_appended() const = 0; + + const Path& path() const { return _path; } + +protected: + Path _path; +}; + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/local_file_reader.cpp b/be/src/io/fs/local_file_reader.cpp new file mode 100644 index 00000000000000..7d8affb647550e --- /dev/null +++ b/be/src/io/fs/local_file_reader.cpp @@ -0,0 +1,65 @@ +#include "io/fs/local_file_reader.h" + +#include "util/doris_metrics.h" +#include "util/errno.h" + +namespace doris { +namespace io { + +LocalFileReader::LocalFileReader(Path path, size_t file_size, + std::shared_ptr> file_handle) + : _file_handle(std::move(file_handle)), + _path(std::move(path)), + _file_size(file_size), + _closed(false) { + _fd = *_file_handle->file(); + DorisMetrics::instance()->local_file_open_reading->increment(1); + DorisMetrics::instance()->local_file_reader_total->increment(1); +} + +LocalFileReader::~LocalFileReader() { + WARN_IF_ERROR(close(), fmt::format("Failed to close file {}", _path.native())); +} + +Status LocalFileReader::close() { + bool expected = false; + if (_closed.compare_exchange_strong(expected, true)) { + _file_handle.reset(); + DorisMetrics::instance()->local_file_open_reading->increment(-1); + } + return Status::OK(); +} + +Status LocalFileReader::read_at(size_t offset, Slice result, size_t* bytes_read) { + DCHECK(!_closed.load()); + if (offset > _file_size) { + return Status::IOError( + fmt::format("offset exceeds file size(offset: {), file size: {}, path: {})", offset, + _file_size, _path.native())); + } + size_t bytes_req = result.size; + char* to = result.data; + bytes_req = std::min(bytes_req, _file_size - offset); + *bytes_read = bytes_req; + + while (bytes_req != 0) { + auto res = ::pread(_fd, to, bytes_req, offset); + if (-1 == res && errno != EINTR) { + return Status::IOError( + fmt::format("cannot read from {}: {}", _path.native(), std::strerror(errno))); + } + if (res == 0) { + return Status::IOError( + fmt::format("cannot read from {}: unexpected EOF", _path.native())); + } + if (res > 0) { + to += res; + bytes_req -= res; + } + } + DorisMetrics::instance()->local_bytes_read_total->increment(*bytes_read); + return Status::OK(); +} + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/local_file_reader.h b/be/src/io/fs/local_file_reader.h new file mode 100644 index 00000000000000..ee579f03cb109b --- /dev/null +++ b/be/src/io/fs/local_file_reader.h @@ -0,0 +1,35 @@ +#pragma once + +#include "io/fs/file_reader.h" +#include "io/fs/path.h" +#include "util/file_cache.h" + +namespace doris { +namespace io { + +class LocalFileReader final : public FileReader { +public: + LocalFileReader(Path path, size_t file_size, + std::shared_ptr> file_handle); + + ~LocalFileReader() override; + + Status close() override; + + Status read_at(size_t offset, Slice result, size_t* bytes_read) override; + + const Path& path() const override { return _path; } + + size_t size() const override { return _file_size; } + +private: + std::shared_ptr> _file_handle; + int _fd; // ref + Path _path; + size_t _file_size; + + std::atomic_bool _closed; +}; + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/local_file_system.cpp b/be/src/io/fs/local_file_system.cpp new file mode 100644 index 00000000000000..5e573f7205cf83 --- /dev/null +++ b/be/src/io/fs/local_file_system.cpp @@ -0,0 +1,154 @@ +#include "io/fs/local_file_system.h" + +#include "io/fs/file_system.h" +#include "io/fs/local_file_reader.h" +#include "io/fs/local_file_writer.h" +#include "olap/storage_engine.h" + +namespace doris { +namespace io { + +LocalFileSystem::LocalFileSystem(Path root_path, ResourceId resource_id) + : FileSystem(std::move(root_path), std::move(resource_id), FileSystemType::LOCAL) { +#ifdef BE_TEST + _file_cache.reset( + new FileCache("Readable_file_cache", config::file_descriptor_cache_capacity)); +#else + _file_cache.reset(new FileCache("Readable_file_cache", + doris::StorageEngine::instance()->file_cache())); +#endif +} + +LocalFileSystem::~LocalFileSystem() = default; + +Path LocalFileSystem::absolute_path(const Path& path) const { + if (path.is_absolute()) { + return path; + } + return _root_path / path; +} + +Status LocalFileSystem::create_file(const Path& path, std::unique_ptr* writer) { + auto fs_path = absolute_path(path); + int fd = ::open(fs_path.c_str(), O_TRUNC | O_WRONLY | O_CREAT | O_CLOEXEC, 0666); + if (-1 == fd) { + return Status::IOError( + fmt::format("cannot open {}: {}", fs_path.native(), std::strerror(errno))); + } + *writer = std::make_unique(std::move(fs_path), fd); + return Status::OK(); +} + +Status LocalFileSystem::open_file(const Path& path, std::unique_ptr* reader) { + auto fs_path = absolute_path(path); + std::shared_ptr> file_handle(new OpenedFileHandle()); + bool found = _file_cache->lookup(fs_path.native(), file_handle.get()); + if (!found) { + int fd = -1; + RETRY_ON_EINTR(fd, open(fs_path.c_str(), O_RDONLY)); + if (fd < 0) { + return Status::IOError( + fmt::format("cannot open {}: {}", fs_path.native(), std::strerror(errno))); + } + int* p_fd = new int(fd); + _file_cache->insert(fs_path.native(), p_fd, file_handle.get(), + [](const CacheKey& key, void* value) { + auto fd = reinterpret_cast(value); + ::close(*fd); + delete fd; + }); + } + size_t fsize = 0; + RETURN_IF_ERROR(file_size(fs_path, &fsize)); + *reader = std::make_unique(std::move(fs_path), fsize, std::move(file_handle)); + return Status::OK(); +} + +Status LocalFileSystem::delete_file(const Path& path) { + auto fs_path = absolute_path(path); + if (!std::filesystem::is_regular_file(fs_path)) { + return Status::IOError(fmt::format("{} is not a file", fs_path.native())); + } + std::error_code ec; + std::filesystem::remove(fs_path, ec); + if (ec) { + return Status::IOError( + fmt::format("cannot delete {}: {}", fs_path.native(), std::strerror(ec.value()))); + } + return Status::OK(); +} + +Status LocalFileSystem::create_directory(const Path& path) { + auto fs_path = absolute_path(_root_path) / path; + if (std::filesystem::exists(fs_path)) { + return Status::IOError(fmt::format("{} exists", fs_path.native())); + } + std::error_code ec; + std::filesystem::create_directories(fs_path, ec); + if (ec) { + return Status::IOError( + fmt::format("cannot create {}: {}", fs_path.native(), std::strerror(ec.value()))); + } + return Status::OK(); +} + +Status LocalFileSystem::delete_directory(const Path& path) { + auto fs_path = absolute_path(path); + if (!std::filesystem::is_directory(fs_path)) { + return Status::IOError(fmt::format("{} is not a directory", fs_path.native())); + } + std::error_code ec; + std::filesystem::remove_all(fs_path, ec); + if (ec) { + return Status::IOError( + fmt::format("cannot delete {}: {}", fs_path.native(), std::strerror(ec.value()))); + } + return Status::OK(); +} + +Status LocalFileSystem::link_file(const Path& src, const Path& dest) { + if (::link(src.c_str(), dest.c_str()) != 0) { + return Status::IOError(fmt::format("fail to create hard link: {}. from {} to {}", + std::strerror(errno), src.native(), dest.native())); + } + return Status::OK(); +} + +Status LocalFileSystem::exists(const Path& path, bool* res) const { + auto fs_path = absolute_path(path); + *res = std::filesystem::exists(fs_path); + return Status::OK(); +} + +Status LocalFileSystem::file_size(const Path& path, size_t* file_size) const { + auto fs_path = absolute_path(path); + std::error_code ec; + *file_size = std::filesystem::file_size(fs_path, ec); + if (ec) { + return Status::IOError(fmt::format("cannot get file size {}: {}", fs_path.native(), + std::strerror(ec.value()))); + } + return Status::OK(); +} + +Status LocalFileSystem::list(const Path& path, std::vector* files) { + files->clear(); + auto fs_path = absolute_path(path); + std::error_code ec; + for (const auto& entry : std::filesystem::directory_iterator(fs_path, ec)) { + files->push_back(entry.path().filename()); + } + if (ec) { + return Status::IOError( + fmt::format("cannot list {}: {}", fs_path.native(), std::strerror(ec.value()))); + } + return Status::OK(); +} + +LocalFileSystem* global_local_filesystem() { + static LocalFileSystem fs(""); + return &fs; +} + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/local_file_system.h b/be/src/io/fs/local_file_system.h new file mode 100644 index 00000000000000..c0a26bde02b2d4 --- /dev/null +++ b/be/src/io/fs/local_file_system.h @@ -0,0 +1,41 @@ +#pragma once + +#include "io/fs/file_system.h" +#include "util/file_cache.h" + +namespace doris { +namespace io { + +class LocalFileSystem final : public FileSystem { +public: + LocalFileSystem(Path root_path, ResourceId resource_id = ResourceId()); + ~LocalFileSystem() override; + + Status create_file(const Path& path, std::unique_ptr* writer) override; + + Status open_file(const Path& path, std::unique_ptr* reader) override; + + Status delete_file(const Path& path) override; + + Status create_directory(const Path& path) override; + + Status delete_directory(const Path& path) override; + + Status link_file(const Path& src, const Path& dest) override; + + Status exists(const Path& path, bool* res) const override; + + Status file_size(const Path& path, size_t* file_size) const override; + + Status list(const Path& path, std::vector* files) override; + +private: + Path absolute_path(const Path& path) const; + + std::unique_ptr> _file_cache; +}; + +LocalFileSystem* global_local_filesystem(); + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/local_file_writer.cpp b/be/src/io/fs/local_file_writer.cpp new file mode 100644 index 00000000000000..e3494f0e81dd86 --- /dev/null +++ b/be/src/io/fs/local_file_writer.cpp @@ -0,0 +1,180 @@ +#include "io/fs/local_file_writer.h" + +#include +#include + +#include + +#include "common/compiler_util.h" +#include "common/status.h" +#include "gutil/macros.h" +#include "io/fs/file_writer.h" +#include "io/fs/local_file_system.h" +#include "io/fs/path.h" +#include "util/doris_metrics.h" + +namespace doris { +namespace detail { + +Status sync_dir(const io::Path& dirname) { + int fd; + RETRY_ON_EINTR(fd, ::open(dirname.c_str(), O_DIRECTORY | O_RDONLY)); + if (-1 == fd) { + return Status::IOError( + fmt::format("cannot open {}: {}", dirname.native(), std::strerror(errno))); + } + if (0 != ::fdatasync(fd)) { + return Status::IOError( + fmt::format("cannot fdatasync {}: {}", dirname.native(), std::strerror(errno))); + } + ::close(fd); + return Status::OK(); +} + +} // namespace detail + +namespace io { + +LocalFileWriter::LocalFileWriter(Path path, int fd) : FileWriter(std::move(path)), _fd(fd) { + DorisMetrics::instance()->local_file_open_writing->increment(1); + DorisMetrics::instance()->local_file_writer_total->increment(1); +} + +LocalFileWriter::~LocalFileWriter() { + if (!_closed) { + WARN_IF_ERROR(abort(), fmt::format("Cannot abort {}", _path.native())); + } +} + +Status LocalFileWriter::close() { + return _close(true); +} + +Status LocalFileWriter::abort() { + RETURN_IF_ERROR(_close(false)); + return io::global_local_filesystem()->delete_file(_path); +} + +Status LocalFileWriter::append(const Slice& data) { + Status st = appendv(&data, 1); + if (st.ok()) { + DorisMetrics::instance()->local_bytes_written_total->increment(data.size); + } + return st; +} + +Status LocalFileWriter::appendv(const Slice* data, size_t data_cnt) { + DCHECK(!_closed); + _dirty = true; + + // Convert the results into the iovec vector to request + // and calculate the total bytes requested. + size_t bytes_req = 0; + struct iovec iov[data_cnt]; + for (size_t i = 0; i < data_cnt; i++) { + const Slice& result = data[i]; + bytes_req += result.size; + iov[i] = {result.data, result.size}; + } + + size_t completed_iov = 0; + size_t n_left = bytes_req; + while (n_left > 0) { + // Never request more than IOV_MAX in one request. + size_t iov_count = std::min(data_cnt - completed_iov, static_cast(IOV_MAX)); + ssize_t res; + RETRY_ON_EINTR(res, ::writev(_fd, iov + completed_iov, iov_count)); + if (UNLIKELY(res < 0)) { + return Status::IOError( + fmt::format("cannot write to {}: {}", _path.native(), std::strerror(errno))); + } + + if (LIKELY(res == n_left)) { + // All requested bytes were read. This is almost always the case. + n_left = 0; + break; + } + // Adjust iovec vector based on bytes read for the next request. + ssize_t bytes_rem = res; + for (size_t i = completed_iov; i < data_cnt; i++) { + if (bytes_rem >= iov[i].iov_len) { + // The full length of this iovec was written. + completed_iov++; + bytes_rem -= iov[i].iov_len; + } else { + // Partially wrote this result. + // Adjust the iov_len and iov_base to write only the missing data. + iov[i].iov_base = static_cast(iov[i].iov_base) + bytes_rem; + iov[i].iov_len -= bytes_rem; + break; // Don't need to adjust remaining iovec's. + } + } + n_left -= res; + } + DCHECK_EQ(0, n_left); + _bytes_appended += bytes_req; + return Status::OK(); +} + +Status LocalFileWriter::finalize() { + DCHECK(!_closed); + if (_dirty) { +#if defined(__linux__) + int flags = SYNC_FILE_RANGE_WRITE; + if (sync_file_range(_fd, 0, 0, flags) < 0) { + return Status::IOError( + fmt::format("cannot sync {}: {}", _path.native(), std::strerror(errno))); + } +#endif + } + return Status::OK(); +} + +Status LocalFileWriter::_close(bool sync) { + if (_closed) { + return Status::OK(); + } + if (sync && _dirty) { + if (0 != ::fdatasync(_fd)) { + return Status::IOError( + fmt::format("cannot fdatasync {}: {}", _path.native(), std::strerror(errno))); + } + RETURN_IF_ERROR(detail::sync_dir(_path.parent_path())); + _dirty = false; + } + _closed = true; + + DorisMetrics::instance()->local_file_open_writing->increment(-1); + DorisMetrics::instance()->file_created_total->increment(1); + DorisMetrics::instance()->local_bytes_written_total->increment(_bytes_appended); + + if (0 != ::close(_fd)) { + return Status::IOError( + fmt::format("cannot close {}: {}", _path.native(), std::strerror(errno))); + } + return Status::OK(); +} + +Status LocalFileWriter::write_at(size_t offset, const Slice& data) { + DCHECK(!_closed); + _dirty = true; + + size_t bytes_req = data.size; + char* from = data.data; + + while (bytes_req != 0) { + auto res = ::pwrite(_fd, from, bytes_req, offset); + if (-1 == res && errno != EINTR) { + return Status::IOError( + fmt::format("cannot write to {}: {}", _path.native(), std::strerror(errno))); + } + if (res > 0) { + from += res; + bytes_req -= res; + } + } + return Status::OK(); +} + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/local_file_writer.h b/be/src/io/fs/local_file_writer.h new file mode 100644 index 00000000000000..00d06cd67c0d38 --- /dev/null +++ b/be/src/io/fs/local_file_writer.h @@ -0,0 +1,41 @@ +#pragma once + +#include + +#include "io/fs/file_writer.h" + +namespace doris { +namespace io { + +class LocalFileWriter final : public FileWriter { +public: + LocalFileWriter(Path path, int fd); + ~LocalFileWriter() override; + + Status close() override; + + Status abort() override; + + Status append(const Slice& data) override; + + Status appendv(const Slice* data, size_t data_cnt) override; + + Status write_at(size_t offset, const Slice& data) override; + + Status finalize() override; + + size_t bytes_appended() const override { return _bytes_appended; } + +private: + Status _close(bool sync); + +private: + int _fd; // owned + + size_t _bytes_appended = 0; + bool _dirty = false; + bool _closed = false; +}; + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/path.h b/be/src/io/fs/path.h new file mode 100644 index 00000000000000..cbfe49115f2798 --- /dev/null +++ b/be/src/io/fs/path.h @@ -0,0 +1,15 @@ +#pragma once + +#include + +namespace doris { +namespace io { + +using Path = std::filesystem::path; + +inline Path operator/(Path&& lhs, const Path& rhs) { + return std::move(lhs /= rhs); +} + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/remote_file_system.h b/be/src/io/fs/remote_file_system.h new file mode 100644 index 00000000000000..5939eeaeda1214 --- /dev/null +++ b/be/src/io/fs/remote_file_system.h @@ -0,0 +1,24 @@ +#pragma once + +#include "io/fs/file_system.h" + +namespace doris { +namespace io { + +class RemoteFileSystem : public FileSystem { +public: + RemoteFileSystem(Path&& root_path, ResourceId&& resource_id, FileSystemType type) + : FileSystem(std::move(root_path), std::move(resource_id), type) {} + ~RemoteFileSystem() override = default; + + // `local_path` should be an absolute path on local filesystem. + virtual Status upload(const Path& local_path, const Path& dest_path) = 0; + + virtual Status batch_upload(const std::vector& local_paths, + const std::vector& dest_paths) = 0; + + virtual Status connect() = 0; +}; + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/s3_common.h b/be/src/io/fs/s3_common.h new file mode 100644 index 00000000000000..110eee3c50133b --- /dev/null +++ b/be/src/io/fs/s3_common.h @@ -0,0 +1,26 @@ +#include +#include + +namespace doris { + +// A non-copying iostream. +// See https://stackoverflow.com/questions/35322033/aws-c-sdk-uploadpart-times-out +// https://stackoverflow.com/questions/13059091/creating-an-input-stream-from-constant-memory +class StringViewStream : Aws::Utils::Stream::PreallocatedStreamBuf, public std::iostream { +public: + StringViewStream(const void* buf, int64_t nbytes) + : Aws::Utils::Stream::PreallocatedStreamBuf( + reinterpret_cast(const_cast(buf)), + static_cast(nbytes)), + std::iostream(this) {} +}; + +// By default, the AWS SDK reads object data into an auto-growing StringStream. +// To avoid copies, read directly into our preallocated buffer instead. +// See https://github.com/aws/aws-sdk-cpp/issues/64 for an alternative but +// functionally similar recipe. +inline Aws::IOStreamFactory AwsWriteableStreamFactory(void* buf, int64_t nbytes) { + return [=]() { return Aws::New("", buf, nbytes); }; +} + +} // namespace doris diff --git a/be/src/io/fs/s3_file_reader.cpp b/be/src/io/fs/s3_file_reader.cpp new file mode 100644 index 00000000000000..56587dc6b7fee7 --- /dev/null +++ b/be/src/io/fs/s3_file_reader.cpp @@ -0,0 +1,62 @@ +#include "io/fs/s3_file_reader.h" + +#include +#include + +#include "io/fs/s3_common.h" +#include "util/doris_metrics.h" + +namespace doris { +namespace io { + +S3FileReader::S3FileReader(Path path, size_t file_size, std::string key, std::string bucket, + S3FileSystem* fs) + : _path(std::move(path)), + _file_size(file_size), + _fs(fs), + _bucket(std::move(bucket)), + _key(std::move(key)) { + DorisMetrics::instance()->s3_file_open_reading->increment(1); + DorisMetrics::instance()->s3_file_reader_total->increment(1); +} + +S3FileReader::~S3FileReader() { + DorisMetrics::instance()->s3_file_open_reading->increment(-1); +} + +Status S3FileReader::close() { + return Status::OK(); +} + +Status S3FileReader::read_at(size_t offset, Slice result, size_t* bytes_read) { + if (offset > _file_size) { + return Status::IOError( + fmt::format("offset exceeds file size(offset: {), file size: {}, path: {})", offset, + _file_size, _path.native())); + } + size_t bytes_req = result.size; + char* to = result.data; + bytes_req = std::min(bytes_req, _file_size - offset); + + Aws::S3::Model::GetObjectRequest request; + request.WithBucket(_bucket).WithKey(_key); + request.SetRange(fmt::format("bytes={}-{}", offset, offset + bytes_req - 1)); + request.SetResponseStreamFactory(AwsWriteableStreamFactory(to, bytes_req)); + + auto client = _fs->get_client(); + auto outcome = client->GetObject(request); + if (!outcome.IsSuccess()) { + return Status::IOError(fmt::format("failed to read from {}: {}", _path.native(), + outcome.GetError().GetMessage())); + } + *bytes_read = outcome.GetResult().GetContentLength(); + if (*bytes_read != bytes_req) { + return Status::IOError(fmt::format("failed to read from {}(bytes read: {}, bytes req: {})", + _path.native(), *bytes_read, bytes_req)); + } + DorisMetrics::instance()->s3_bytes_read_total->increment(*bytes_read); + return Status::OK(); +} + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/s3_file_reader.h b/be/src/io/fs/s3_file_reader.h new file mode 100644 index 00000000000000..4db4c69f0cb2bb --- /dev/null +++ b/be/src/io/fs/s3_file_reader.h @@ -0,0 +1,36 @@ +#pragma once + +#include "gutil/macros.h" +#include "io/fs/file_reader.h" +#include "io/fs/path.h" +#include "io/fs/s3_file_system.h" + +namespace doris { +namespace io { + +class S3FileReader final : public FileReader { +public: + S3FileReader(Path path, size_t file_size, std::string key, std::string bucket, + S3FileSystem* fs); + + ~S3FileReader() override; + + Status close() override; + + Status read_at(size_t offset, Slice result, size_t* bytes_read) override; + + const Path& path() const override { return _path; } + + size_t size() const override { return _file_size; } + +private: + Path _path; + size_t _file_size; + S3FileSystem* _fs; + + std::string _bucket; + std::string _key; +}; + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/s3_file_system.cpp b/be/src/io/fs/s3_file_system.cpp new file mode 100644 index 00000000000000..5911be512feeb6 --- /dev/null +++ b/be/src/io/fs/s3_file_system.cpp @@ -0,0 +1,204 @@ +#include "io/fs/s3_file_system.h" + +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include "common/config.h" +#include "common/status.h" +#include "gutil/strings/stringpiece.h" +#include "io/fs/remote_file_system.h" +#include "io/fs/s3_file_reader.h" + +namespace doris { +namespace io { + +S3FileSystem::S3FileSystem(const std::map& properties, std::string bucket, + std::string prefix, ResourceId resource_id) + : RemoteFileSystem(Path(properties.at(S3_ENDPOINT)) / bucket / prefix, + std::move(resource_id), FileSystemType::S3), + _properties(properties), + _bucket(std::move(bucket)), + _prefix(std::move(prefix)) { + _endpoint = properties.at(S3_ENDPOINT); + _executor = Aws::MakeShared( + resource_id.c_str(), config::s3_transfer_executor_pool_size); +} + +S3FileSystem::~S3FileSystem() = default; + +Status S3FileSystem::connect() { + std::lock_guard lock(_client_mu); + _client = ClientFactory::instance().create(_properties); + return Status::OK(); +} + +Status S3FileSystem::upload(const Path& local_path, const Path& dest_path) { + auto client = get_client(); + DCHECK(client); + + Aws::Transfer::TransferManagerConfiguration transfer_config(_executor.get()); + transfer_config.s3Client = client; + auto transfer_manager = Aws::Transfer::TransferManager::Create(transfer_config); + + auto start = std::chrono::steady_clock::now(); + + auto key = get_key(dest_path); + auto handle = transfer_manager->UploadFile(local_path.native(), _bucket, key, "text/plain", + Aws::Map()); + handle->WaitUntilFinished(); + + auto duration = std::chrono::duration(std::chrono::steady_clock::now() - start); + + if (handle->GetStatus() != Aws::Transfer::TransferStatus::COMPLETED) { + return Status::IOError(fmt::format("failed to upload(endpoint={}, bucket={}, key={}): {}", + _endpoint, _bucket, key, + handle->GetLastError().GetMessage())); + } + + auto file_size = std::filesystem::file_size(local_path); + LOG(INFO) << "Upload " << local_path.native() << " to s3, endpoint=" << _endpoint + << ", bucket=" << _bucket << ", key=" << key << ", duration=" << duration.count() + << ", capacity=" << file_size << ", tp=" << (file_size) / duration.count(); + + return Status::OK(); +} + +Status S3FileSystem::batch_upload(const std::vector& local_paths, + const std::vector& dest_paths) { + auto client = get_client(); + DCHECK(client); + + if (local_paths.size() != dest_paths.size()) { + return Status::InvalidArgument("local_paths.size() != dest_paths.size()"); + } + + Aws::Transfer::TransferManagerConfiguration transfer_config(_executor.get()); + transfer_config.s3Client = client; + auto transfer_manager = Aws::Transfer::TransferManager::Create(transfer_config); + + std::vector> handles; + for (int i = 0; i < local_paths.size(); ++i) { + auto key = get_key(dest_paths[i]); + LOG(INFO) << "Start to upload " << local_paths[i].native() + << " to s3, endpoint=" << _endpoint << ", bucket=" << _bucket << ", key=" << key; + auto handle = + transfer_manager->UploadFile(local_paths[i].native(), _bucket, key, "text/plain", + Aws::Map()); + handles.push_back(std::move(handle)); + } + for (auto& handle : handles) { + handle->WaitUntilFinished(); + if (handle->GetStatus() != Aws::Transfer::TransferStatus::COMPLETED) { + // TODO(cyx): Maybe we can cancel remaining handles. + return Status::IOError(handle->GetLastError().GetMessage()); + } + } + return Status::OK(); +} + +Status S3FileSystem::create_file(const Path& path, std::unique_ptr* writer) { + return Status::NotSupported("not support"); +} + +Status S3FileSystem::open_file(const Path& path, std::unique_ptr* reader) { + size_t fsize = 0; + RETURN_IF_ERROR(file_size(path, &fsize)); + auto key = get_key(path); + auto fs_path = Path(_endpoint) / _bucket / key; + *reader = std::make_unique(std::move(fs_path), fsize, std::move(key), _bucket, + this); + return Status::OK(); +} + +Status S3FileSystem::delete_file(const Path& path) { + auto client = get_client(); + DCHECK(client); + + Aws::S3::Model::DeleteObjectRequest request; + auto key = get_key(path); + request.WithBucket(_bucket).WithKey(key); + + auto outcome = client->DeleteObject(request); + if (!outcome.IsSuccess()) { + return Status::IOError( + fmt::format("failed to delete object(endpoint={}, bucket={}, key={}): {}", + _endpoint, _bucket, key, outcome.GetError().GetMessage())); + } + return Status::OK(); +} + +Status S3FileSystem::create_directory(const Path& path) { + return Status::OK(); +} + +Status S3FileSystem::delete_directory(const Path& path) { + return Status::NotSupported("not support"); +} + +Status S3FileSystem::link_file(const Path& src, const Path& dest) { + return Status::NotSupported("not support"); +} + +Status S3FileSystem::exists(const Path& path, bool* res) const { + auto client = get_client(); + DCHECK(client); + + Aws::S3::Model::HeadObjectRequest request; + auto key = get_key(path); + request.WithBucket(_bucket).WithKey(key); + + auto outcome = _client->HeadObject(request); + if (outcome.IsSuccess()) { + *res = true; + } else if (outcome.GetError().GetResponseCode() == Aws::Http::HttpResponseCode::NOT_FOUND) { + *res = false; + } else { + return Status::IOError( + fmt::format("failed to get object head(endpoint={}, bucket={}, key={}): {}", + _endpoint, _bucket, key, outcome.GetError().GetMessage())); + } + return Status::OK(); +} + +Status S3FileSystem::file_size(const Path& path, size_t* file_size) const { + auto client = get_client(); + DCHECK(client); + + Aws::S3::Model::HeadObjectRequest request; + auto key = get_key(path); + request.WithBucket(_bucket).WithKey(key); + + auto outcome = _client->HeadObject(request); + if (outcome.IsSuccess()) { + *file_size = outcome.GetResult().GetContentLength(); + } else { + return Status::IOError( + fmt::format("failed to get object size(endpoint={}, bucket={}, key={}): {}", + _endpoint, _bucket, key, outcome.GetError().GetMessage())); + } + return Status::OK(); +} + +Status S3FileSystem::list(const Path& path, std::vector* files) { + return Status::NotSupported("not support"); +} + +std::string S3FileSystem::get_key(const Path& path) const { + StringPiece str(path.native()); + if (str.starts_with(_root_path.native())) { + return fmt::format("{}/{}", _prefix, str.data() + _root_path.native().size()); + } + // We consider it as a relative path. + return fmt::format("{}/{}", _prefix, path.native()); +} + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/s3_file_system.h b/be/src/io/fs/s3_file_system.h new file mode 100644 index 00000000000000..3a2c767ee5f779 --- /dev/null +++ b/be/src/io/fs/s3_file_system.h @@ -0,0 +1,78 @@ +#pragma once + +#include + +#include "io/fs/remote_file_system.h" +#include "util/s3_util.h" + +namespace Aws::S3 { +class S3Client; +} // namespace Aws::S3 +namespace Aws::Utils::Threading { +class PooledThreadExecutor; +} // namespace Aws::Utils::Threading + +namespace doris { +namespace io { + +// This class is thread-safe.(Except `set_xxx` method) +class S3FileSystem final : public RemoteFileSystem { +public: + S3FileSystem(const std::map& properties, std::string bucket, + std::string prefix, ResourceId resource_id); + ~S3FileSystem() override; + + Status create_file(const Path& path, std::unique_ptr* writer) override; + + Status open_file(const Path& path, std::unique_ptr* reader) override; + + Status delete_file(const Path& path) override; + + Status create_directory(const Path& path) override; + + Status delete_directory(const Path& path) override; + + Status link_file(const Path& src, const Path& dest) override; + + Status exists(const Path& path, bool* res) const override; + + Status file_size(const Path& path, size_t* file_size) const override; + + Status list(const Path& path, std::vector* files) override; + + Status upload(const Path& local_path, const Path& dest_path) override; + + Status batch_upload(const std::vector& local_paths, + const std::vector& dest_paths) override; + + Status connect() override; + + std::shared_ptr get_client() const { + std::lock_guard lock(_client_mu); + return _client; + }; + + // Guarded by external lock. + void set_ak(std::string ak) { _properties[S3_AK] = std::move(ak); } + + // Guarded by external lock. + void set_sk(std::string sk) { _properties[S3_SK] = std::move(sk); } + +private: + std::string get_key(const Path& path) const; + +private: + std::map _properties; + std::string _endpoint; + std::string _bucket; + std::string _prefix; + + // FIXME(cyx): We can use std::atomic since c++20. + std::shared_ptr _client; + mutable std::mutex _client_mu; + + std::shared_ptr _executor; +}; + +} // namespace io +} // namespace doris diff --git a/be/src/olap/CMakeLists.txt b/be/src/olap/CMakeLists.txt index 615a9fff44f4e9..3456a0c3383d2c 100644 --- a/be/src/olap/CMakeLists.txt +++ b/be/src/olap/CMakeLists.txt @@ -71,7 +71,6 @@ add_library(Olap STATIC version_graph.cpp schema.cpp schema_change.cpp - storage_migration_v2.cpp serialize.cpp storage_engine.cpp data_dir.cpp @@ -117,9 +116,9 @@ add_library(Olap STATIC task/engine_checksum_task.cpp task/engine_clone_task.cpp task/engine_storage_migration_task.cpp - task/engine_storage_migration_task_v2.cpp task/engine_publish_version_task.cpp task/engine_alter_tablet_task.cpp column_vector.cpp segment_loader.cpp + storage_policy_mgr.cpp ) diff --git a/be/src/olap/base_compaction.cpp b/be/src/olap/base_compaction.cpp index 948ec425d329f4..e3d467be2a55e1 100644 --- a/be/src/olap/base_compaction.cpp +++ b/be/src/olap/base_compaction.cpp @@ -106,14 +106,11 @@ Status BaseCompaction::pick_rowsets_to_compact() { } // 2. the ratio between base rowset and all input cumulative rowsets reaches the threshold - int64_t base_size = 0; + // `_input_rowsets` has been sorted by end version, so we consider `_input_rowsets[0]` is the base rowset. + int64_t base_size = _input_rowsets.front()->data_disk_size(); int64_t cumulative_total_size = 0; - for (auto& rowset : _input_rowsets) { - if (rowset->start_version() != 0) { - cumulative_total_size += rowset->data_disk_size(); - } else { - base_size = rowset->data_disk_size(); - } + for (auto it = _input_rowsets.begin() + 1; it != _input_rowsets.end(); ++it) { + cumulative_total_size += (*it)->data_disk_size(); } double base_cumulative_delta_ratio = config::base_cumulative_delta_ratio; diff --git a/be/src/olap/base_tablet.cpp b/be/src/olap/base_tablet.cpp index 72970cfb2c447d..00b7dc30ae4104 100644 --- a/be/src/olap/base_tablet.cpp +++ b/be/src/olap/base_tablet.cpp @@ -67,26 +67,8 @@ Status BaseTablet::set_tablet_state(TabletState state) { void BaseTablet::_gen_tablet_path() { if (_data_dir != nullptr && _tablet_meta != nullptr) { - FilePathDesc root_path_desc; - root_path_desc.filepath = _data_dir->path_desc().filepath; - root_path_desc.storage_medium = - fs::fs_util::get_t_storage_medium(_storage_param.storage_medium()); - if (_data_dir->is_remote()) { - root_path_desc.storage_name = _storage_param.storage_name(); - root_path_desc.remote_path = - StorageBackendMgr::get_root_path_from_param(_storage_param); - } - FilePathDescStream desc_s; - desc_s << root_path_desc << DATA_PREFIX; - FilePathDesc path_desc = path_util::join_path_desc_segments( - desc_s.path_desc(), std::to_string(_tablet_meta->shard_id())); - path_desc = path_util::join_path_desc_segments(path_desc, - std::to_string(_tablet_meta->tablet_id())); - _tablet_path_desc = path_util::join_path_desc_segments( - path_desc, std::to_string(_tablet_meta->schema_hash())); - if (_tablet_path_desc.is_remote()) { - _tablet_path_desc.remote_path += "/" + _tablet_meta->tablet_uid().to_string(); - } + _tablet_path = fmt::format("{}/{}/{}/{}/{}", _data_dir->path(), DATA_PREFIX, shard_id(), + tablet_id(), schema_hash()); } } diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h index 4bf83324af2d47..69a092b84a5b37 100644 --- a/be/src/olap/base_tablet.h +++ b/be/src/olap/base_tablet.h @@ -38,7 +38,7 @@ class BaseTablet : public std::enable_shared_from_this { virtual ~BaseTablet(); DataDir* data_dir() const; - FilePathDesc tablet_path_desc() const; + const std::string& tablet_path() const; TabletState tablet_state() const { return _state; } Status set_tablet_state(TabletState state); @@ -59,6 +59,12 @@ class BaseTablet : public std::enable_shared_from_this { int16_t shard_id() const; bool equal(int64_t tablet_id, int32_t schema_hash); + const io::ResourceId& cooldown_resource() const { return _tablet_meta->cooldown_resource(); } + + void set_cooldown_resource(io::ResourceId resource) { + _tablet_meta->set_cooldown_resource(std::move(resource)); + } + // properties encapsulated in TabletSchema const TabletSchema& tablet_schema() const; @@ -72,7 +78,7 @@ class BaseTablet : public std::enable_shared_from_this { const TabletSchema& _schema; DataDir* _data_dir; - FilePathDesc _tablet_path_desc; + std::string _tablet_path; // metrics of this tablet std::shared_ptr _metric_entity = nullptr; @@ -92,8 +98,8 @@ inline DataDir* BaseTablet::data_dir() const { return _data_dir; } -inline FilePathDesc BaseTablet::tablet_path_desc() const { - return _tablet_path_desc; +inline const std::string& BaseTablet::tablet_path() const { + return _tablet_path; } inline const TabletMetaSharedPtr BaseTablet::tablet_meta() { diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index 3ac480194ef8c8..625b2a335acf69 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -133,7 +133,10 @@ Status Compaction::do_compaction_impl(int64_t permits) { _output_version = Version(_input_rowsets.front()->start_version(), _input_rowsets.back()->end_version()); - auto use_vectorized_compaction = config::enable_vectorized_compaction; + _oldest_write_timestamp = _input_rowsets.front()->oldest_write_timestamp(); + _newest_write_timestamp = _input_rowsets.back()->newest_write_timestamp(); + + auto use_vectorized_compaction = _should_use_vectorized_compaction(); string merge_type = use_vectorized_compaction ? "v" : ""; LOG(INFO) << "start " << merge_type << compaction_name() << ". tablet=" << _tablet->full_name() @@ -218,6 +221,7 @@ Status Compaction::do_compaction_impl(int64_t permits) { Status Compaction::construct_output_rowset_writer() { return _tablet->create_rowset_writer(_output_version, VISIBLE, NONOVERLAPPING, + _oldest_write_timestamp, _newest_write_timestamp, &_output_rs_writer); } diff --git a/be/src/olap/compaction.h b/be/src/olap/compaction.h index c70a82defaab52..bd76ded20a62c4 100644 --- a/be/src/olap/compaction.h +++ b/be/src/olap/compaction.h @@ -100,6 +100,9 @@ class Compaction { Version _output_version; + int64_t _oldest_write_timestamp; + int64_t _newest_write_timestamp; + DISALLOW_COPY_AND_ASSIGN(Compaction); }; diff --git a/be/src/olap/cumulative_compaction_policy.cpp b/be/src/olap/cumulative_compaction_policy.cpp index ad73e1fc805e3d..05da2150287ea3 100644 --- a/be/src/olap/cumulative_compaction_policy.cpp +++ b/be/src/olap/cumulative_compaction_policy.cpp @@ -465,7 +465,7 @@ void CumulativeCompactionPolicy::pick_candidate_rowsets( int64_t cumulative_point, std::vector* candidate_rowsets) { for (auto& it : rs_version_map) { // find all rowset version greater than cumulative_point and skip the create time in skip_window_sec - if (it.first.first >= cumulative_point) { + if (it.first.first >= cumulative_point && it.second->is_local()) { candidate_rowsets->push_back(it.second); } } diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp index 13f4648690d16c..12e2c37c94deee 100644 --- a/be/src/olap/data_dir.cpp +++ b/be/src/olap/data_dir.cpp @@ -29,11 +29,15 @@ #include #include #include +#include #include #include +#include #include "env/env_util.h" #include "gutil/strings/substitute.h" +#include "io/fs/local_file_system.h" +#include "io/fs/path.h" #include "olap/file_helper.h" #include "olap/olap_define.h" #include "olap/rowset/alpha_rowset_meta.h" @@ -54,17 +58,19 @@ namespace doris { DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(disks_total_capacity, MetricUnit::BYTES); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(disks_avail_capacity, MetricUnit::BYTES); -DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(disks_data_used_capacity, MetricUnit::BYTES); +DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(disks_local_used_capacity, MetricUnit::BYTES); +DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(disks_remote_used_capacity, MetricUnit::BYTES); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(disks_state, MetricUnit::BYTES); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(disks_compaction_score, MetricUnit::NOUNIT); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(disks_compaction_num, MetricUnit::NOUNIT); -static const char* const kTestFilePath = "/.testfile"; +static const char* const kTestFilePath = ".testfile"; DataDir::DataDir(const std::string& path, int64_t capacity_bytes, TStorageMedium::type storage_medium, TabletManager* tablet_manager, TxnManager* txn_manager) - : _path_desc(path), + : _path(path), + _fs(std::make_shared(path)), _capacity_bytes(capacity_bytes), _available_bytes(0), _disk_capacity_bytes(0), @@ -77,12 +83,12 @@ DataDir::DataDir(const std::string& path, int64_t capacity_bytes, _to_be_deleted(false), _current_shard(0), _meta(nullptr) { - _path_desc.storage_medium = storage_medium; _data_dir_metric_entity = DorisMetrics::instance()->metric_registry()->register_entity( std::string("data_dir.") + path, {{"path", path}}); INT_GAUGE_METRIC_REGISTER(_data_dir_metric_entity, disks_total_capacity); INT_GAUGE_METRIC_REGISTER(_data_dir_metric_entity, disks_avail_capacity); - INT_GAUGE_METRIC_REGISTER(_data_dir_metric_entity, disks_data_used_capacity); + INT_GAUGE_METRIC_REGISTER(_data_dir_metric_entity, disks_local_used_capacity); + INT_GAUGE_METRIC_REGISTER(_data_dir_metric_entity, disks_remote_used_capacity); INT_GAUGE_METRIC_REGISTER(_data_dir_metric_entity, disks_state); INT_GAUGE_METRIC_REGISTER(_data_dir_metric_entity, disks_compaction_score); INT_GAUGE_METRIC_REGISTER(_data_dir_metric_entity, disks_compaction_num); @@ -95,10 +101,10 @@ DataDir::~DataDir() { } Status DataDir::init() { - if (!Env::Default()->path_exists(_path_desc.filepath).ok()) { - RETURN_NOT_OK_STATUS_WITH_WARN(Status::IOError(strings::Substitute( - "opendir failed, path=$0", _path_desc.filepath)), - "check file exist failed"); + if (!Env::Default()->path_exists(_path).ok()) { + RETURN_NOT_OK_STATUS_WITH_WARN( + Status::IOError(strings::Substitute("opendir failed, path=$0", _path)), + "check file exist failed"); } RETURN_NOT_OK_STATUS_WITH_WARN(update_capacity(), "update_capacity failed"); @@ -117,14 +123,11 @@ void DataDir::stop_bg_worker() { } Status DataDir::_init_cluster_id() { - FilePathDescStream path_desc_s; - path_desc_s << _path_desc << CLUSTER_ID_PREFIX; - FilePathDesc cluster_id_path_desc = path_desc_s.path_desc(); - RETURN_IF_ERROR(read_cluster_id(Env::Default(), cluster_id_path_desc.filepath, &_cluster_id)); + auto cluster_id_path = fmt::format("{}/{}", _path, CLUSTER_ID_PREFIX); + RETURN_IF_ERROR(read_cluster_id(Env::Default(), cluster_id_path, &_cluster_id)); if (_cluster_id == -1) { _cluster_id_incomplete = true; } - return Status::OK(); } @@ -157,19 +160,19 @@ Status DataDir::_init_capacity() { int64_t disk_capacity = -1; int64_t available = -1; RETURN_NOT_OK_STATUS_WITH_WARN( - Env::Default()->get_space_info(_path_desc.filepath, &disk_capacity, &available), - strings::Substitute("get_space_info failed: $0", _path_desc.filepath)); + Env::Default()->get_space_info(_path, &disk_capacity, &available), + strings::Substitute("get_space_info failed: $0", _path)); if (_capacity_bytes == -1) { _capacity_bytes = disk_capacity; } else if (_capacity_bytes > disk_capacity) { RETURN_NOT_OK_STATUS_WITH_WARN( Status::InvalidArgument(strings::Substitute( - "root path $0's capacity $1 should not larger than disk capacity $2", - _path_desc.filepath, _capacity_bytes, disk_capacity)), + "root path $0's capacity $1 should not larger than disk capacity $2", _path, + _capacity_bytes, disk_capacity)), "init capacity failed"); } - std::string data_path = _path_desc.filepath + DATA_PREFIX; + auto data_path = fmt::format("{}/{}", _path, DATA_PREFIX); Status exist_status = Env::Default()->path_exists(data_path); if (!exist_status.ok() && (!exist_status.is_not_found() || !Env::Default()->create_dirs(data_path).ok())) { @@ -183,11 +186,11 @@ Status DataDir::_init_capacity() { Status DataDir::_init_meta() { // init path hash - _path_hash = hash_of_path(BackendOptions::get_localhost(), _path_desc.filepath); - LOG(INFO) << "path: " << _path_desc.filepath << ", hash: " << _path_hash; + _path_hash = hash_of_path(BackendOptions::get_localhost(), _path); + LOG(INFO) << "path: " << _path << ", hash: " << _path_hash; // init meta - _meta = new (std::nothrow) OlapMeta(_path_desc.filepath); + _meta = new (std::nothrow) OlapMeta(_path); if (_meta == nullptr) { RETURN_NOT_OK_STATUS_WITH_WARN( Status::MemoryAllocFailed("allocate memory for OlapMeta failed"), @@ -196,8 +199,7 @@ Status DataDir::_init_meta() { Status res = _meta->init(); if (!res.ok()) { RETURN_NOT_OK_STATUS_WITH_WARN( - Status::IOError( - strings::Substitute("open rocksdb failed, path=$0", _path_desc.filepath)), + Status::IOError(strings::Substitute("open rocksdb failed, path=$0", _path)), "init OlapMeta failed"); } return Status::OK(); @@ -212,18 +214,17 @@ Status DataDir::set_cluster_id(int32_t cluster_id) { if (!_cluster_id_incomplete) { return Status::OK(); } - FilePathDescStream path_desc_s; - path_desc_s << _path_desc << CLUSTER_ID_PREFIX; - return _write_cluster_id_to_path(path_desc_s.path_desc(), cluster_id); + auto cluster_id_path = fmt::format("{}/{}", _path, CLUSTER_ID_PREFIX); + return _write_cluster_id_to_path(cluster_id_path, cluster_id); } -Status DataDir::_write_cluster_id_to_path(const FilePathDesc& path_desc, int32_t cluster_id) { +Status DataDir::_write_cluster_id_to_path(const std::string& path, int32_t cluster_id) { std::stringstream cluster_id_ss; cluster_id_ss << cluster_id; std::unique_ptr wfile; - if (!Env::Default()->path_exists(path_desc.filepath).ok()) { - RETURN_IF_ERROR(env_util::write_string_to_file_sync( - Env::Default(), Slice(cluster_id_ss.str()), path_desc.filepath)); + if (!Env::Default()->path_exists(path).ok()) { + RETURN_IF_ERROR(env_util::write_string_to_file_sync(Env::Default(), + Slice(cluster_id_ss.str()), path)); } return Status::OK(); } @@ -233,8 +234,7 @@ void DataDir::health_check() { if (_is_used) { Status res = _read_and_write_test_file(); if (!res) { - LOG(WARNING) << "store read/write test file occur IO Error. path=" - << _path_desc.filepath; + LOG(WARNING) << "store read/write test file occur IO Error. path=" << _path; if (res.is_io_error()) { _is_used = false; } @@ -244,20 +244,18 @@ void DataDir::health_check() { } Status DataDir::_read_and_write_test_file() { - std::string test_file = _path_desc.filepath + kTestFilePath; + auto test_file = fmt::format("{}/{}", _path, kTestFilePath); return read_write_test_file(test_file); } Status DataDir::get_shard(uint64_t* shard) { - std::stringstream shard_path_stream; uint32_t next_shard = 0; { std::lock_guard l(_mutex); next_shard = _current_shard; _current_shard = (_current_shard + 1) % MAX_SHARD_NUM; } - shard_path_stream << _path_desc.filepath << DATA_PREFIX << "/" << next_shard; - std::string shard_path = shard_path_stream.str(); + auto shard_path = fmt::format("{}/{}/{}", _path, DATA_PREFIX, next_shard); RETURN_WITH_WARN_IF_ERROR(Env::Default()->create_dirs(shard_path), Status::OLAPInternalError(OLAP_ERR_CANNOT_CREATE_DIR), "fail to create path. path=" + shard_path); @@ -288,27 +286,26 @@ void DataDir::clear_tablets(std::vector* tablet_infos) { } std::string DataDir::get_absolute_shard_path(int64_t shard_id) { - return strings::Substitute("$0$1/$2", _path_desc.filepath, DATA_PREFIX, shard_id); + return fmt::format("{}/{}/{}", _path, DATA_PREFIX, shard_id); } std::string DataDir::get_absolute_tablet_path(int64_t shard_id, int64_t tablet_id, int32_t schema_hash) { - return strings::Substitute("$0/$1/$2", get_absolute_shard_path(shard_id), tablet_id, - schema_hash); + return fmt::format("{}/{}/{}", get_absolute_shard_path(shard_id), tablet_id, schema_hash); } void DataDir::find_tablet_in_trash(int64_t tablet_id, std::vector* paths) { // path: /root_path/trash/time_label/tablet_id/schema_hash - std::string trash_path = _path_desc.filepath + TRASH_PREFIX; + auto trash_path = fmt::format("{}/{}", _path, TRASH_PREFIX); std::vector sub_dirs; FileUtils::list_files(Env::Default(), trash_path, &sub_dirs); for (auto& sub_dir : sub_dirs) { // sub dir is time_label - std::string sub_path = trash_path + "/" + sub_dir; + auto sub_path = fmt::format("{}/{}", trash_path, sub_dir); if (!FileUtils::is_dir(sub_path, Env::Default())) { continue; } - std::string tablet_path = sub_path + "/" + std::to_string(tablet_id); + auto tablet_path = fmt::format("{}/{}", sub_path, tablet_id); Status exist_status = Env::Default()->path_exists(tablet_path); if (exist_status.ok()) { paths->emplace_back(std::move(tablet_path)); @@ -318,8 +315,8 @@ void DataDir::find_tablet_in_trash(int64_t tablet_id, std::vector* std::string DataDir::get_root_path_from_schema_hash_path_in_trash( const std::string& schema_hash_dir_in_trash) { - std::filesystem::path schema_hash_path_in_trash(schema_hash_dir_in_trash); - return schema_hash_path_in_trash.parent_path() + return io::Path(schema_hash_dir_in_trash) + .parent_path() .parent_path() .parent_path() .parent_path() @@ -349,21 +346,17 @@ Status DataDir::_check_incompatible_old_format_tablet() { _meta, check_incompatible_old_func, OLD_HEADER_PREFIX); if (!check_incompatible_old_status) { LOG(WARNING) << "check incompatible old format meta fails, it may lead to data missing!!! " - << _path_desc.filepath; + << _path; } else { - LOG(INFO) << "successfully check incompatible old format meta " << _path_desc.filepath; + LOG(INFO) << "successfully check incompatible old format meta " << _path; } return check_incompatible_old_status; } // TODO(ygl): deal with rowsets and tablets when load failed Status DataDir::load() { - LOG(INFO) << "start to load tablets from " << _path_desc.filepath; - if (is_remote()) { - RETURN_WITH_WARN_IF_ERROR( - StorageBackendMgr::instance()->init(_path_desc.filepath + STORAGE_PARAM_PREFIX), - Status::OLAPInternalError(OLAP_ERR_INIT_FAILED), "DataDir init failed."); - } + LOG(INFO) << "start to load tablets from " << _path; + // load rowset meta from meta env and create rowset // COMMITTED: add to txn manager // VISIBLE: add to tablet @@ -374,8 +367,9 @@ Status DataDir::load() { std::vector dir_rowset_metas; LOG(INFO) << "begin loading rowset from meta"; - auto load_rowset_func = [&dir_rowset_metas](TabletUid tablet_uid, RowsetId rowset_id, - const std::string& meta_str) -> bool { + auto load_rowset_func = [&dir_rowset_metas, &local_fs = fs()]( + TabletUid tablet_uid, RowsetId rowset_id, + const std::string& meta_str) -> bool { RowsetMetaSharedPtr rowset_meta(new AlphaRowsetMeta()); bool parsed = rowset_meta->init(meta_str); if (!parsed) { @@ -383,16 +377,18 @@ Status DataDir::load() { // return false will break meta iterator, return true to skip this error return true; } + if (rowset_meta->is_local()) { + rowset_meta->set_fs(local_fs); + } dir_rowset_metas.push_back(rowset_meta); return true; }; Status load_rowset_status = RowsetMetaManager::traverse_rowset_metas(_meta, load_rowset_func); if (!load_rowset_status) { - LOG(WARNING) << "errors when load rowset meta from meta env, skip this data dir:" - << _path_desc.filepath; + LOG(WARNING) << "errors when load rowset meta from meta env, skip this data dir:" << _path; } else { - LOG(INFO) << "load rowset from meta finished, data dir: " << _path_desc.filepath; + LOG(INFO) << "load rowset from meta finished, data dir: " << _path; } // load tablet @@ -433,23 +429,19 @@ Status DataDir::load() { if (failed_tablet_ids.size() != 0) { LOG(WARNING) << "load tablets from header failed" << ", loaded tablet: " << tablet_ids.size() - << ", error tablet: " << failed_tablet_ids.size() - << ", path: " << _path_desc.filepath; + << ", error tablet: " << failed_tablet_ids.size() << ", path: " << _path; if (!config::ignore_load_tablet_failure) { - LOG(FATAL) << "load tablets encounter failure. stop BE process. path: " - << _path_desc.filepath; + LOG(FATAL) << "load tablets encounter failure. stop BE process. path: " << _path; } } if (!load_tablet_status) { LOG(WARNING) << "there is failure when loading tablet headers" << ", loaded tablet: " << tablet_ids.size() - << ", error tablet: " << failed_tablet_ids.size() - << ", path: " << _path_desc.filepath; + << ", error tablet: " << failed_tablet_ids.size() << ", path: " << _path; } else { LOG(INFO) << "load tablet from meta finished" << ", loaded tablet: " << tablet_ids.size() - << ", error tablet: " << failed_tablet_ids.size() - << ", path: " << _path_desc.filepath; + << ", error tablet: " << failed_tablet_ids.size() << ", path: " << _path; } // traverse rowset @@ -467,6 +459,7 @@ Status DataDir::load() { ++invalid_rowset_counter; continue; } + RowsetSharedPtr rowset; Status create_status = tablet->create_rowset(rowset_meta, &rowset); if (!create_status) { @@ -518,7 +511,7 @@ Status DataDir::load() { // At startup, we only count these invalid rowset, but do not actually delete it. // The actual delete operation is in StorageEngine::_clean_unused_rowset_metas, // which is cleaned up uniformly by the background cleanup thread. - LOG(INFO) << "finish to load tablets from " << _path_desc.filepath + LOG(INFO) << "finish to load tablets from " << _path << ", total rowset meta: " << dir_rowset_metas.size() << ", invalid rowset num: " << invalid_rowset_counter; @@ -571,11 +564,9 @@ void DataDir::perform_path_gc_by_tablet() { // could find the tablet, then skip check it continue; } - std::filesystem::path tablet_path(path); - std::filesystem::path data_dir_path = - tablet_path.parent_path().parent_path().parent_path().parent_path(); - std::string data_dir_string = data_dir_path.string(); - DataDir* data_dir = StorageEngine::instance()->get_store(data_dir_string); + std::string data_dir_path = + io::Path(path).parent_path().parent_path().parent_path().parent_path(); + DataDir* data_dir = StorageEngine::instance()->get_store(data_dir_path); if (data_dir == nullptr) { LOG(WARNING) << "could not find data dir for tablet path " << path; continue; @@ -638,19 +629,19 @@ void DataDir::perform_path_scan() { LOG(INFO) << "_all_check_paths is not empty when path scan."; return; } - LOG(INFO) << "start to scan data dir path:" << _path_desc.filepath; + LOG(INFO) << "start to scan data dir path:" << _path; std::set shards; - std::string data_path = _path_desc.filepath + DATA_PREFIX; + auto data_path = fmt::format("{}/{}", _path, DATA_PREFIX); Status ret = FileUtils::list_dirs_files(data_path, &shards, nullptr, Env::Default()); if (!ret.ok()) { - LOG(WARNING) << "fail to walk dir. path=[" + data_path << "] error[" << ret.to_string() + LOG(WARNING) << "fail to walk dir. path=[" << data_path << "] error[" << ret.to_string() << "]"; return; } for (const auto& shard : shards) { - std::string shard_path = data_path + "/" + shard; + auto shard_path = fmt::format("{}/{}", data_path, shard); std::set tablet_ids; ret = FileUtils::list_dirs_files(shard_path, &tablet_ids, nullptr, Env::Default()); if (!ret.ok()) { @@ -659,7 +650,7 @@ void DataDir::perform_path_scan() { continue; } for (const auto& tablet_id : tablet_ids) { - std::string tablet_id_path = shard_path + "/" + tablet_id; + auto tablet_id_path = fmt::format("{}/{}", shard_path, tablet_id); std::set schema_hashes; ret = FileUtils::list_dirs_files(tablet_id_path, &schema_hashes, nullptr, Env::Default()); @@ -669,7 +660,7 @@ void DataDir::perform_path_scan() { continue; } for (const auto& schema_hash : schema_hashes) { - std::string tablet_schema_hash_path = tablet_id_path + "/" + schema_hash; + auto tablet_schema_hash_path = fmt::format("{}/{}", tablet_id_path, schema_hash); _all_tablet_schemahash_paths.insert(tablet_schema_hash_path); std::set rowset_files; @@ -681,13 +672,14 @@ void DataDir::perform_path_scan() { continue; } for (const auto& rowset_file : rowset_files) { - std::string rowset_file_path = tablet_schema_hash_path + "/" + rowset_file; + auto rowset_file_path = + fmt::format("{}/{}", tablet_schema_hash_path, rowset_file); _all_check_paths.insert(rowset_file_path); } } } } - LOG(INFO) << "scan data dir path: " << _path_desc.filepath << " finished. path size: " + LOG(INFO) << "scan data dir path: " << _path << " finished. path size: " << _all_check_paths.size() + _all_tablet_schemahash_paths.size(); _check_path_cv.notify_one(); } @@ -709,9 +701,8 @@ bool DataDir::_check_pending_ids(const std::string& id) { Status DataDir::update_capacity() { RETURN_NOT_OK_STATUS_WITH_WARN( - Env::Default()->get_space_info(_path_desc.filepath, &_disk_capacity_bytes, - &_available_bytes), - strings::Substitute("get_space_info failed: $0", _path_desc.filepath)); + Env::Default()->get_space_info(_path, &_disk_capacity_bytes, &_available_bytes), + strings::Substitute("get_space_info failed: $0", _path)); if (_disk_capacity_bytes < 0) { _disk_capacity_bytes = _capacity_bytes; } @@ -721,14 +712,18 @@ Status DataDir::update_capacity() { disks_total_capacity->set_value(_disk_capacity_bytes); disks_avail_capacity->set_value(_available_bytes); - LOG(INFO) << "path: " << _path_desc.filepath << " total capacity: " << _disk_capacity_bytes + LOG(INFO) << "path: " << _path << " total capacity: " << _disk_capacity_bytes << ", available capacity: " << _available_bytes; return Status::OK(); } -void DataDir::update_user_data_size(int64_t size) { - disks_data_used_capacity->set_value(size); +void DataDir::update_local_data_size(int64_t size) { + disks_local_used_capacity->set_value(size); +} + +void DataDir::update_remote_data_size(int64_t size) { + disks_remote_used_capacity->set_value(size); } size_t DataDir::tablet_size() const { @@ -743,7 +738,7 @@ bool DataDir::reach_capacity_limit(int64_t incoming_data_size) { if (used_pct >= config::storage_flood_stage_usage_percent / 100.0 && left_bytes <= config::storage_flood_stage_left_capacity_bytes) { LOG(WARNING) << "reach capacity limit. used pct: " << used_pct - << ", left bytes: " << left_bytes << ", path: " << _path_desc.filepath; + << ", left bytes: " << left_bytes << ", path: " << _path; return true; } return false; @@ -757,18 +752,8 @@ void DataDir::disks_compaction_num_increment(int64_t delta) { disks_compaction_num->increment(delta); } -// this is moved from src/olap/utils.h, the old move_to_trash() can only support local files, -// and it is more suitable in DataDir because one trash path is in one DataDir -Status DataDir::move_to_trash(const FilePathDesc& segment_path_desc) { +Status DataDir::move_to_trash(const std::string& tablet_path) { Status res = Status::OK(); - FilePathDesc storage_root_desc = _path_desc; - if (is_remote() && - !StorageBackendMgr::instance() - ->get_root_path(segment_path_desc.storage_name, &(storage_root_desc.remote_path)) - .ok()) { - LOG(WARNING) << "get_root_path failed for storage_name: " << segment_path_desc.storage_name; - return Status::OLAPInternalError(OLAP_ERR_OTHER_ERROR); - } // 1. get timestamp string string time_str; @@ -777,93 +762,35 @@ Status DataDir::move_to_trash(const FilePathDesc& segment_path_desc) { return res; } - // 2. generate new file path desc - static std::atomic delete_counter( - 0); // a global counter to avoid file name duplication. - // when file_path points to a schema_path, we need to save tablet info in trash_path, - // so we add file_path.parent_path().filename() in new_file_path. - // other conditions are not considered, for they are nothing serious. - FilePathDescStream trash_root_desc_s; - trash_root_desc_s << storage_root_desc << TRASH_PREFIX << "/" << time_str << "." - << delete_counter++; - std::stringstream trash_local_file_stream; - std::filesystem::path old_local_path(segment_path_desc.filepath); - trash_local_file_stream << trash_root_desc_s.path_desc().filepath << "/" - << old_local_path.parent_path().filename().string() // tablet_path - << "/" << old_local_path.filename().string(); // segment_path - FilePathDesc trash_path_desc(trash_local_file_stream.str()); - trash_path_desc.storage_medium = segment_path_desc.storage_medium; - if (is_remote()) { - std::stringstream trash_remote_file_stream; - std::filesystem::path old_remote_path(segment_path_desc.remote_path); - trash_remote_file_stream - << trash_root_desc_s.path_desc().remote_path << "/" - << old_remote_path.parent_path().parent_path().filename().string() // tablet_path - << "/" << old_remote_path.parent_path().filename().string() // segment_path - << "/" << old_remote_path.filename().string(); // tablet_uid - trash_path_desc.remote_path = trash_remote_file_stream.str(); - trash_path_desc.storage_name = segment_path_desc.storage_name; - } + // 2. generate new file path + // a global counter to avoid file name duplication. + static std::atomic delete_counter(0); + auto trash_root_path = + fmt::format("{}/{}/{}.{}", _path, TRASH_PREFIX, time_str, delete_counter++); + auto fs_tablet_path = io::Path(tablet_path); + auto trash_tablet_path = trash_root_path / + fs_tablet_path.parent_path().filename() /* tablet_id */ / + fs_tablet_path.filename() /* schema_hash */; // 3. create target dir, or the rename() function will fail. - string trash_local_file = trash_local_file_stream.str(); - std::filesystem::path trash_local_path(trash_local_file); - string trash_local_dir = trash_local_path.parent_path().string(); - if (!FileUtils::check_exist(trash_local_dir) && !FileUtils::create_dir(trash_local_dir).ok()) { - LOG(WARNING) << "delete file failed. due to mkdir failed. [file=" - << segment_path_desc.filepath.c_str() << " new_dir=" << trash_local_dir.c_str() - << "]"; + auto trash_tablet_parent = trash_tablet_path.parent_path(); + if (!FileUtils::check_exist(trash_tablet_parent) && + !FileUtils::create_dir(trash_tablet_parent).ok()) { + LOG(WARNING) << "delete file failed. due to mkdir failed. [file=" << tablet_path + << " new_dir=" << trash_tablet_parent << "]"; return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); } - // 4. move remote file to trash if needed - if (is_remote()) { - std::string trash_storage_name_path = - trash_root_desc_s.path_desc().filepath + "/" + STORAGE_NAME; - Status st = env_util::write_string_to_file( - Env::Default(), Slice(segment_path_desc.storage_name), trash_storage_name_path); - if (!st.ok()) { - LOG(WARNING) << "fail to write storage_name to trash path: " << trash_storage_name_path - << ", error:" << st.to_string(); - return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); - } - std::shared_ptr storage_backend = - StorageBackendMgr::instance()->get_storage_backend(segment_path_desc.storage_name); - if (storage_backend == nullptr) { - LOG(WARNING) << "storage_backend is invalid: " << segment_path_desc.storage_name; - return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); - } - Status status = storage_backend->exist_dir(segment_path_desc.remote_path); - if (status.ok()) { - VLOG_NOTICE << "Move remote file to trash. " << segment_path_desc.remote_path << " -> " - << trash_path_desc.remote_path; - Status rename_status = storage_backend->rename_dir(segment_path_desc.remote_path, - trash_path_desc.remote_path); - if (!rename_status.ok()) { - LOG(WARNING) << "Move remote file to trash failed. [file=" - << segment_path_desc.remote_path << " target='" - << trash_path_desc.remote_path << "']"; - return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); - } - } else if (status.is_not_found()) { - LOG(WARNING) << "File may be removed before: " << segment_path_desc.remote_path; - } else { - LOG(WARNING) << "File check exist error: " << segment_path_desc.remote_path; - return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); - } - } - - // 5. move file to trash - VLOG_NOTICE << "move file to trash. " << segment_path_desc.filepath << " -> " - << trash_local_file; - if (rename(segment_path_desc.filepath.c_str(), trash_local_file.c_str()) < 0) { - LOG(WARNING) << "move file to trash failed. [file=" << segment_path_desc.filepath - << " target='" << trash_local_file << "' err='" << Errno::str() << "']"; + // 4. move tablet to trash + VLOG_NOTICE << "move file to trash. " << tablet_path << " -> " << trash_tablet_path; + if (rename(tablet_path.c_str(), trash_tablet_path.c_str()) < 0) { + LOG(WARNING) << "move file to trash failed. [file=" << tablet_path << " target='" + << trash_tablet_path << "' err='" << Errno::str() << "']"; return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); } - // 6. check parent dir of source file, delete it when empty - string source_parent_dir = old_local_path.parent_path().string(); // tablet_id level + // 5. check parent dir of source file, delete it when empty + std::string source_parent_dir = fs_tablet_path.parent_path(); // tablet_id level std::set sub_dirs, sub_files; RETURN_WITH_WARN_IF_ERROR( diff --git a/be/src/olap/data_dir.h b/be/src/olap/data_dir.h index 1d197cb6f0813c..642058c46f846b 100644 --- a/be/src/olap/data_dir.h +++ b/be/src/olap/data_dir.h @@ -28,6 +28,7 @@ #include "env/env.h" #include "gen_cpp/Types_types.h" #include "gen_cpp/olap_file.pb.h" +#include "io/fs/file_system.h" #include "olap/olap_common.h" #include "olap/rowset/rowset_id_generator.h" #include "util/metrics.h" @@ -51,9 +52,11 @@ class DataDir { Status init(); void stop_bg_worker(); - const std::string& path() const { return _path_desc.filepath; } - const FilePathDesc& path_desc() const { return _path_desc; } + const std::string& path() const { return _path; } size_t path_hash() const { return _path_hash; } + + const io::FileSystemPtr& fs() const { return _fs; } + bool is_used() const { return _is_used; } void set_is_used(bool is_used) { _is_used = is_used; } int32_t cluster_id() const { return _cluster_id; } @@ -61,7 +64,7 @@ class DataDir { DataDirInfo get_dir_info() { DataDirInfo info; - info.path_desc = _path_desc; + info.path = _path; info.path_hash = _path_hash; info.disk_capacity = _disk_capacity_bytes; info.available = _available_bytes; @@ -123,7 +126,9 @@ class DataDir { Status update_capacity(); - void update_user_data_size(int64_t size); + void update_local_data_size(int64_t size); + + void update_remote_data_size(int64_t size); size_t tablet_size() const; @@ -131,13 +136,8 @@ class DataDir { void disks_compaction_num_increment(int64_t delta); - // Move segment_path_desc to trash, trash is in storage_root/trash, segment_path_desc can be file or dir - // Modify segment_path_desc when this operation is being done. - // filepath is replaced by: - // storage_root/trash/20150619154308.delete_counter/tablet_path/segment_path, - // remote_path is replaced by: - // storage_root/trash/20150619154308.delete_counter/tablet_path/segment_path/tablet_uid - Status move_to_trash(const FilePathDesc& segment_path_desc); + // Move tablet to trash. + Status move_to_trash(const std::string& tablet_path); private: Status _init_cluster_id(); @@ -147,7 +147,7 @@ class DataDir { Status _check_disk(); Status _read_and_write_test_file(); Status read_cluster_id(Env* env, const std::string& cluster_id_path, int32_t* cluster_id); - Status _write_cluster_id_to_path(const FilePathDesc& path_desc, int32_t cluster_id); + Status _write_cluster_id_to_path(const std::string& path, int32_t cluster_id); // Check whether has old format (hdr_ start) in olap. When doris updating to current version, // it may lead to data missing. When conf::storage_strict_check_incompatible_old_format is true, // process will log fatal. @@ -162,8 +162,10 @@ class DataDir { private: bool _stop_bg_worker = false; - FilePathDesc _path_desc; + std::string _path; size_t _path_hash; + + io::FileSystemPtr _fs; // user specified capacity int64_t _capacity_bytes; // the actual available capacity of the disk of this data dir @@ -203,7 +205,8 @@ class DataDir { std::shared_ptr _data_dir_metric_entity; IntGauge* disks_total_capacity; IntGauge* disks_avail_capacity; - IntGauge* disks_data_used_capacity; + IntGauge* disks_local_used_capacity; + IntGauge* disks_remote_used_capacity; IntGauge* disks_state; IntGauge* disks_compaction_score; IntGauge* disks_compaction_num; diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index 304157e4e4d907..441c9de89f863d 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -19,6 +19,7 @@ #include +#include #include #include #include @@ -50,11 +51,12 @@ using TabletUid = UniqueId; enum CompactionType { BASE_COMPACTION = 1, CUMULATIVE_COMPACTION = 2 }; struct DataDirInfo { - FilePathDesc path_desc; + std::string path; size_t path_hash = 0; int64_t disk_capacity = 1; // actual disk capacity int64_t available = 0; // available space, in bytes unit - int64_t data_used_capacity = 0; + int64_t local_used_capacity = 0; + int64_t remote_used_capacity = 0; bool is_used = false; // whether available mark TStorageMedium::type storage_medium = TStorageMedium::HDD; // Storage medium type: SSD|HDD }; diff --git a/be/src/olap/olap_define.h b/be/src/olap/olap_define.h index fa8b2efd5f2e72..077ed670f9359d 100644 --- a/be/src/olap/olap_define.h +++ b/be/src/olap/olap_define.h @@ -77,19 +77,17 @@ enum OLAPDataVersion { }; // storage_root_path下不同类型文件夹名称 -static const std::string MINI_PREFIX = "/mini_download"; -static const std::string CLUSTER_ID_PREFIX = "/cluster_id"; -static const std::string DATA_PREFIX = "/data"; -static const std::string STORAGE_PARAM_PREFIX = "/storage_param"; -static const std::string REMOTE_FILE_PARAM = "/remote_file_param"; -static const std::string DPP_PREFIX = "/dpp_download"; -static const std::string SNAPSHOT_PREFIX = "/snapshot"; -static const std::string TRASH_PREFIX = "/trash"; -static const std::string UNUSED_PREFIX = "/unused"; -static const std::string ERROR_LOG_PREFIX = "/error_log"; -static const std::string PENDING_DELTA_PREFIX = "/pending_delta"; -static const std::string INCREMENTAL_DELTA_PREFIX = "/incremental_delta"; -static const std::string CLONE_PREFIX = "/clone"; +static const std::string MINI_PREFIX = "mini_download"; +static const std::string CLUSTER_ID_PREFIX = "cluster_id"; +static const std::string DATA_PREFIX = "data"; +static const std::string DPP_PREFIX = "dpp_download"; +static const std::string SNAPSHOT_PREFIX = "snapshot"; +static const std::string TRASH_PREFIX = "trash"; +static const std::string UNUSED_PREFIX = "unused"; +static const std::string ERROR_LOG_PREFIX = "error_log"; +static const std::string PENDING_DELTA_PREFIX = "pending_delta"; +static const std::string INCREMENTAL_DELTA_PREFIX = "incremental_delta"; +static const std::string CLONE_PREFIX = "clone"; static const std::string TABLET_UID = "tablet_uid"; static const std::string STORAGE_NAME = "storage_name"; diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp index 5088bd87753e90..6095d2edf7f7ea 100644 --- a/be/src/olap/olap_server.cpp +++ b/be/src/olap/olap_server.cpp @@ -112,7 +112,6 @@ Status StorageEngine::start_bg_threads() { [this]() { this->_compaction_tasks_producer_callback(); }, &_compaction_tasks_producer_thread)); LOG(INFO) << "compaction tasks producer thread started"; - int32_t max_checkpoint_thread_num = config::max_meta_checkpoint_threads; if (max_checkpoint_thread_num < 0) { max_checkpoint_thread_num = data_dirs.size(); @@ -136,9 +135,6 @@ Status StorageEngine::start_bg_threads() { // path scan and gc thread if (config::path_gc_check) { for (auto data_dir : get_stores()) { - if (data_dir->is_remote()) { - continue; - } scoped_refptr path_scan_thread; RETURN_IF_ERROR(Thread::create( "StorageEngine", "path_scan_thread", @@ -156,6 +152,18 @@ Status StorageEngine::start_bg_threads() { LOG(INFO) << "path scan/gc threads started. number:" << get_stores().size(); } + ThreadPoolBuilder("CooldownTaskThreadPool") + .set_min_threads(config::cooldown_thread_num) + .set_max_threads(config::cooldown_thread_num) + .build(&_cooldown_thread_pool); + LOG(INFO) << "cooldown thread pool started"; + + RETURN_IF_ERROR(Thread::create( + "StorageEngine", "cooldown_tasks_producer_thread", + [this]() { this->_cooldown_tasks_producer_callback(); }, + &_cooldown_tasks_producer_thread)); + LOG(INFO) << "cooldown tasks producer thread started"; + LOG(INFO) << "all storage engine's background threads are started."; return Status::OK(); } @@ -564,9 +572,6 @@ std::vector StorageEngine::_generate_compaction_tasks( ? copied_cumu_map[data_dir] : copied_base_map[data_dir], &disk_max_score, _cumulative_compaction_policy); - if (data_dir->is_remote()) { - continue; - } if (tablet != nullptr) { if (need_pick_tablet) { tablets_compaction.emplace_back(tablet); @@ -721,4 +726,66 @@ Status StorageEngine::submit_quick_compaction_task(TabletSharedPtr tablet) { return Status::OK(); } +void StorageEngine::_cooldown_tasks_producer_callback() { + int64_t interval = config::generate_cooldown_task_interval_sec; + do { + if (_cooldown_thread_pool->get_queue_size() > 0) { + continue; + } + std::vector tablets; + // TODO(luwei) : a more efficient way to get cooldown tablets + _tablet_manager->get_cooldwon_tablets(&tablets); + LOG(INFO) << "cooldown producer get tablet num: " << tablets.size(); + for (const auto& tablet : tablets) { + Status st = _cooldown_thread_pool->submit_func([=]() { + { + // Cooldown tasks on the same tablet cannot be executed concurrently + std::lock_guard lock(_running_cooldown_mutex); + auto it = _running_cooldown_tablets.find(tablet->tablet_id()); + if (it != _running_cooldown_tablets.end()) { + return; + } + + // the number of concurrent cooldown tasks in each directory + // cannot exceed the configured value + auto dir_it = _running_cooldown_tasks_cnt.find(tablet->data_dir()); + if (dir_it != _running_cooldown_tasks_cnt.end() && + dir_it->second >= config::concurrency_per_dir) { + return; + } + + _running_cooldown_tablets.insert(tablet->tablet_id()); + dir_it = _running_cooldown_tasks_cnt.find(tablet->data_dir()); + if (dir_it != _running_cooldown_tasks_cnt.end()) { + _running_cooldown_tasks_cnt[tablet->data_dir()]++; + } else { + _running_cooldown_tasks_cnt[tablet->data_dir()] = 1; + } + } + + Status st = tablet->cooldown(); + if (!st.ok()) { + LOG(WARNING) << "failed to cooldown, tablet: " << tablet->tablet_id() + << " err: " << st.to_string(); + } else { + LOG(INFO) << "succeed to cooldown, tablet: " << tablet->tablet_id() + << " cooldown progress (" + << tablets.size() - _cooldown_thread_pool->get_queue_size() << "/" + << tablets.size() << ")"; + } + + { + std::lock_guard lock(_running_cooldown_mutex); + _running_cooldown_tasks_cnt[tablet->data_dir()]--; + _running_cooldown_tablets.erase(tablet->tablet_id()); + } + }); + + if (!st.ok()) { + LOG(INFO) << "failed to submit cooldown task, err msg: " << st.get_error_msg(); + } + } + } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(interval))); +} + } // namespace doris diff --git a/be/src/olap/rowset/alpha_rowset.cpp b/be/src/olap/rowset/alpha_rowset.cpp index 7f2e74c74453ef..f4ca2ebc394e89 100644 --- a/be/src/olap/rowset/alpha_rowset.cpp +++ b/be/src/olap/rowset/alpha_rowset.cpp @@ -27,9 +27,9 @@ namespace doris { -AlphaRowset::AlphaRowset(const TabletSchema* schema, const FilePathDesc& rowset_path_desc, +AlphaRowset::AlphaRowset(const TabletSchema* schema, const std::string& tablet_path, RowsetMetaSharedPtr rowset_meta) - : Rowset(schema, rowset_path_desc, std::move(rowset_meta)) {} + : Rowset(schema, tablet_path, std::move(rowset_meta)) {} Status AlphaRowset::do_load(bool use_cache) { for (auto& segment_group : _segment_groups) { @@ -90,9 +90,9 @@ void AlphaRowset::make_visible_extra(Version version) { } } -Status AlphaRowset::link_files_to(const FilePathDesc& dir_desc, RowsetId new_rowset_id) { +Status AlphaRowset::link_files_to(const std::string& dir, RowsetId new_rowset_id) { for (auto& segment_group : _segment_groups) { - auto status = segment_group->link_segments_to_path(dir_desc.filepath, new_rowset_id); + auto status = segment_group->link_segments_to_path(dir, new_rowset_id); if (!status.ok()) { LOG(WARNING) << "create hard links failed for segment group:" << segment_group->segment_group_id(); @@ -309,15 +309,14 @@ Status AlphaRowset::init() { std::shared_ptr segment_group; if (_is_pending) { segment_group.reset(new SegmentGroup( - _rowset_meta->tablet_id(), _rowset_meta->rowset_id(), _schema, - _rowset_path_desc.filepath, false, segment_group_meta.segment_group_id(), - segment_group_meta.num_segments(), true, _rowset_meta->partition_id(), - _rowset_meta->txn_id())); + _rowset_meta->tablet_id(), _rowset_meta->rowset_id(), _schema, _tablet_path, + false, segment_group_meta.segment_group_id(), segment_group_meta.num_segments(), + true, _rowset_meta->partition_id(), _rowset_meta->txn_id())); } else { segment_group.reset(new SegmentGroup( - _rowset_meta->tablet_id(), _rowset_meta->rowset_id(), _schema, - _rowset_path_desc.filepath, _rowset_meta->version(), false, - segment_group_meta.segment_group_id(), segment_group_meta.num_segments())); + _rowset_meta->tablet_id(), _rowset_meta->rowset_id(), _schema, _tablet_path, + _rowset_meta->version(), false, segment_group_meta.segment_group_id(), + segment_group_meta.num_segments())); } if (segment_group == nullptr) { LOG(WARNING) << "fail to create olap segment_group. rowset_id='" diff --git a/be/src/olap/rowset/alpha_rowset.h b/be/src/olap/rowset/alpha_rowset.h index 9f4c838723650e..7a6dd309a420d8 100644 --- a/be/src/olap/rowset/alpha_rowset.h +++ b/be/src/olap/rowset/alpha_rowset.h @@ -47,7 +47,7 @@ class AlphaRowset : public Rowset { Status remove() override; - Status link_files_to(const FilePathDesc& dir_desc, RowsetId new_rowset_id) override; + Status link_files_to(const std::string& dir, RowsetId new_rowset_id) override; Status copy_files_to(const std::string& dir, const RowsetId& new_rowset_id) override; @@ -66,7 +66,7 @@ class AlphaRowset : public Rowset { protected: friend class RowsetFactory; - AlphaRowset(const TabletSchema* schema, const FilePathDesc& rowset_path_desc, + AlphaRowset(const TabletSchema* schema, const std::string& tablet_path, RowsetMetaSharedPtr rowset_meta); // init segment groups diff --git a/be/src/olap/rowset/alpha_rowset_reader.h b/be/src/olap/rowset/alpha_rowset_reader.h index 3dffec245796b2..7def457777b4a2 100644 --- a/be/src/olap/rowset/alpha_rowset_reader.h +++ b/be/src/olap/rowset/alpha_rowset_reader.h @@ -71,6 +71,9 @@ class AlphaRowsetReader : public RowsetReader { Version version() override; + int64_t oldest_write_timestamp() override { return 0; } + int64_t newest_write_timestamp() override { return 0; } + RowsetSharedPtr rowset() override; int64_t filtered_rows() override; diff --git a/be/src/olap/rowset/alpha_rowset_writer.cpp b/be/src/olap/rowset/alpha_rowset_writer.cpp index 138b9a73cf24ea..c4a895da37b446 100644 --- a/be/src/olap/rowset/alpha_rowset_writer.cpp +++ b/be/src/olap/rowset/alpha_rowset_writer.cpp @@ -97,8 +97,8 @@ Status AlphaRowsetWriter::add_rowset(RowsetSharedPtr rowset) { AlphaRowsetSharedPtr alpha_rowset = std::dynamic_pointer_cast(rowset); for (auto& segment_group : alpha_rowset->_segment_groups) { RETURN_NOT_OK(_init()); - RETURN_NOT_OK(segment_group->link_segments_to_path( - _rowset_writer_context.path_desc.filepath, _rowset_writer_context.rowset_id)); + RETURN_NOT_OK(segment_group->link_segments_to_path(_rowset_writer_context.tablet_path, + _rowset_writer_context.rowset_id)); _cur_segment_group->set_empty(segment_group->empty()); _cur_segment_group->set_num_segments(segment_group->num_segments()); _cur_segment_group->add_zone_maps(segment_group->get_zone_maps()); @@ -121,8 +121,8 @@ Status AlphaRowsetWriter::add_rowset_for_linked_schema_change(RowsetSharedPtr ro AlphaRowsetSharedPtr alpha_rowset = std::dynamic_pointer_cast(rowset); for (auto& segment_group : alpha_rowset->_segment_groups) { RETURN_NOT_OK(_init()); - RETURN_NOT_OK(segment_group->link_segments_to_path( - _rowset_writer_context.path_desc.filepath, _rowset_writer_context.rowset_id)); + RETURN_NOT_OK(segment_group->link_segments_to_path(_rowset_writer_context.tablet_path, + _rowset_writer_context.rowset_id)); _cur_segment_group->set_empty(segment_group->empty()); _cur_segment_group->set_num_segments(segment_group->num_segments()); _cur_segment_group->add_zone_maps_for_linked_schema_change(segment_group->get_zone_maps(), @@ -133,11 +133,6 @@ Status AlphaRowsetWriter::add_rowset_for_linked_schema_change(RowsetSharedPtr ro return Status::OK(); } -Status AlphaRowsetWriter::add_rowset_for_migration(RowsetSharedPtr rowset) { - LOG(WARNING) << "alpha_rowset_writer doesn't support add_rowset_for_migration"; - return Status::NotSupported("alpha_rowset_writer doesn't support add_rowset_for_migration"); -} - Status AlphaRowsetWriter::flush() { if (_writer_state == WRITER_FLUSHED) { return Status::OK(); @@ -226,7 +221,7 @@ RowsetSharedPtr AlphaRowsetWriter::build() { RowsetSharedPtr rowset; auto status = RowsetFactory::create_rowset(_rowset_writer_context.tablet_schema, - _rowset_writer_context.path_desc, + _rowset_writer_context.tablet_path, _current_rowset_meta, &rowset); if (!status.ok()) { LOG(WARNING) << "rowset init failed when build new rowset, res=" << status; @@ -254,15 +249,15 @@ Status AlphaRowsetWriter::_init() { return Status::OK(); } if (_is_pending_rowset) { - _cur_segment_group = new (std::nothrow) SegmentGroup( - _rowset_writer_context.tablet_id, _rowset_writer_context.rowset_id, - _rowset_writer_context.tablet_schema, _rowset_writer_context.path_desc.filepath, - false, _segment_group_id, 0, true, _rowset_writer_context.partition_id, - _rowset_writer_context.txn_id); + _cur_segment_group = new (std::nothrow) + SegmentGroup(_rowset_writer_context.tablet_id, _rowset_writer_context.rowset_id, + _rowset_writer_context.tablet_schema, + _rowset_writer_context.tablet_path, false, _segment_group_id, 0, true, + _rowset_writer_context.partition_id, _rowset_writer_context.txn_id); } else { _cur_segment_group = new (std::nothrow) SegmentGroup( _rowset_writer_context.tablet_id, _rowset_writer_context.rowset_id, - _rowset_writer_context.tablet_schema, _rowset_writer_context.path_desc.filepath, + _rowset_writer_context.tablet_schema, _rowset_writer_context.tablet_path, _rowset_writer_context.version, false, _segment_group_id, 0); } DCHECK(_cur_segment_group != nullptr) << "failed to malloc SegmentGroup"; diff --git a/be/src/olap/rowset/alpha_rowset_writer.h b/be/src/olap/rowset/alpha_rowset_writer.h index 411beabf6271dd..f594f22486af31 100644 --- a/be/src/olap/rowset/alpha_rowset_writer.h +++ b/be/src/olap/rowset/alpha_rowset_writer.h @@ -42,7 +42,6 @@ class AlphaRowsetWriter : public RowsetWriter { Status add_rowset_for_linked_schema_change(RowsetSharedPtr rowset, const SchemaMapping& schema_mapping) override; - Status add_rowset_for_migration(RowsetSharedPtr rowset) override; Status flush() override; // get a rowset diff --git a/be/src/olap/rowset/beta_rowset.cpp b/be/src/olap/rowset/beta_rowset.cpp index 1627a5bf5d616c..d8ef290cdaed06 100644 --- a/be/src/olap/rowset/beta_rowset.cpp +++ b/be/src/olap/rowset/beta_rowset.cpp @@ -17,32 +17,47 @@ #include "olap/rowset/beta_rowset.h" +#include +#include #include // for remove() #include // for link() #include -#include - +#include "common/status.h" #include "gutil/strings/substitute.h" +#include "io/fs/s3_file_system.h" +#include "olap/olap_define.h" #include "olap/rowset/beta_rowset_reader.h" #include "olap/utils.h" -#include "util/storage_backend.h" -#include "util/storage_backend_mgr.h" +#include "util/doris_metrics.h" namespace doris { -FilePathDesc BetaRowset::segment_file_path(const FilePathDesc& segment_dir_desc, +std::string BetaRowset::segment_file_path(int segment_id) { + if (is_local()) { + return local_segment_path(_tablet_path, rowset_id(), segment_id); + } + return remote_segment_path(_rowset_meta->tablet_id(), rowset_id(), segment_id); +} + +std::string BetaRowset::local_segment_path(const std::string& tablet_path, const RowsetId& rowset_id, int segment_id) { - FilePathDescStream path_desc_s; - path_desc_s << segment_dir_desc << "/" << rowset_id.to_string() << "_" << segment_id << ".dat"; - return path_desc_s.path_desc(); + // {root_path}/data/{shard_id}/{tablet_id}/{schema_hash}/{rowset_id}_{seg_num}.dat + return fmt::format("{}/{}_{}.dat", tablet_path, rowset_id.to_string(), segment_id); } -BetaRowset::BetaRowset(const TabletSchema* schema, const FilePathDesc& rowset_path_desc, +std::string BetaRowset::remote_segment_path(int64_t tablet_id, const RowsetId& rowset_id, + int segment_id) { + // data/{tablet_id}/{rowset_id}_{seg_num}.dat + return fmt::format("{}/{}/{}_{}.dat", DATA_PREFIX, tablet_id, rowset_id.to_string(), + segment_id); +} + +BetaRowset::BetaRowset(const TabletSchema* schema, const std::string& tablet_path, RowsetMetaSharedPtr rowset_meta) - : Rowset(schema, rowset_path_desc, std::move(rowset_meta)) {} + : Rowset(schema, tablet_path, std::move(rowset_meta)) {} -BetaRowset::~BetaRowset() {} +BetaRowset::~BetaRowset() = default; Status BetaRowset::init() { return Status::OK(); // no op @@ -55,13 +70,17 @@ Status BetaRowset::do_load(bool /*use_cache*/) { } Status BetaRowset::load_segments(std::vector* segments) { + auto fs = _rowset_meta->fs(); + if (!fs) { + return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED); + } for (int seg_id = 0; seg_id < num_segments(); ++seg_id) { - FilePathDesc seg_path_desc = segment_file_path(_rowset_path_desc, rowset_id(), seg_id); + auto seg_path = segment_file_path(seg_id); std::shared_ptr segment; - auto s = segment_v2::Segment::open(seg_path_desc, seg_id, _schema, &segment); + auto s = segment_v2::Segment::open(fs, seg_path, seg_id, _schema, &segment); if (!s.ok()) { - LOG(WARNING) << "failed to open segment. " << seg_path_desc.debug_string() - << " under rowset " << unique_id() << " : " << s.to_string(); + LOG(WARNING) << "failed to open segment. " << seg_path << " under rowset " + << unique_id() << " : " << s.to_string(); return Status::OLAPInternalError(OLAP_ERR_ROWSET_LOAD_FAILED); } segments->push_back(std::move(segment)); @@ -88,15 +107,18 @@ Status BetaRowset::remove() { VLOG_NOTICE << "begin to remove files in rowset " << unique_id() << ", version:" << start_version() << "-" << end_version() << ", tabletid:" << _rowset_meta->tablet_id(); + auto fs = _rowset_meta->fs(); + if (!fs) { + return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED); + } bool success = true; + Status st; for (int i = 0; i < num_segments(); ++i) { - FilePathDesc path_desc = segment_file_path(_rowset_path_desc, rowset_id(), i); - LOG(INFO) << "deleting " << path_desc.debug_string(); - fs::BlockManager* block_mgr = fs::fs_util::block_manager(path_desc); - if (!block_mgr->delete_block(path_desc).ok()) { - char errmsg[64]; - VLOG_NOTICE << "failed to delete file. err=" << strerror_r(errno, errmsg, 64) << ", " - << path_desc.debug_string(); + auto seg_path = segment_file_path(i); + LOG(INFO) << "deleting " << seg_path; + st = fs->delete_file(seg_path); + if (!st.ok()) { + LOG(WARNING) << st.to_string(); success = false; } } @@ -111,23 +133,25 @@ void BetaRowset::do_close() { // do nothing. } -Status BetaRowset::link_files_to(const FilePathDesc& dir_desc, RowsetId new_rowset_id) { +Status BetaRowset::link_files_to(const std::string& dir, RowsetId new_rowset_id) { + DCHECK(is_local()); + auto fs = _rowset_meta->fs(); + if (!fs) { + return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED); + } for (int i = 0; i < num_segments(); ++i) { - FilePathDesc dst_link_path_desc = segment_file_path(dir_desc, new_rowset_id, i); + auto dst_path = local_segment_path(dir, new_rowset_id, i); // TODO(lingbin): use Env API? or EnvUtil? - if (FileUtils::check_exist(dst_link_path_desc.filepath)) { - LOG(WARNING) << "failed to create hard link, file already exist: " - << dst_link_path_desc.filepath; + if (FileUtils::check_exist(dst_path)) { + LOG(WARNING) << "failed to create hard link, file already exist: " << dst_path; return Status::OLAPInternalError(OLAP_ERR_FILE_ALREADY_EXIST); } - FilePathDesc src_file_path_desc = segment_file_path(_rowset_path_desc, rowset_id(), i); + auto src_path = segment_file_path(i); // TODO(lingbin): how external storage support link? // use copy? or keep refcount to avoid being delete? - fs::BlockManager* block_mgr = fs::fs_util::block_manager(dir_desc); - if (!block_mgr->link_file(src_file_path_desc, dst_link_path_desc).ok()) { - LOG(WARNING) << "fail to create hard link. from=" << src_file_path_desc.debug_string() - << ", " - << "to=" << dst_link_path_desc.debug_string() << ", errno=" << Errno::no(); + if (!fs->link_file(src_path, dst_path).ok()) { + LOG(WARNING) << "fail to create hard link. from=" << src_path << ", " + << "to=" << dst_path << ", errno=" << Errno::no(); return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); } } @@ -135,97 +159,70 @@ Status BetaRowset::link_files_to(const FilePathDesc& dir_desc, RowsetId new_rows } Status BetaRowset::copy_files_to(const std::string& dir, const RowsetId& new_rowset_id) { + DCHECK(is_local()); for (int i = 0; i < num_segments(); ++i) { - FilePathDesc dst_path_desc = segment_file_path(dir, new_rowset_id, i); - Status status = Env::Default()->path_exists(dst_path_desc.filepath); + auto dst_path = local_segment_path(dir, new_rowset_id, i); + Status status = Env::Default()->path_exists(dst_path); if (status.ok()) { - LOG(WARNING) << "file already exist: " << dst_path_desc.filepath; + LOG(WARNING) << "file already exist: " << dst_path; return Status::OLAPInternalError(OLAP_ERR_FILE_ALREADY_EXIST); } if (!status.is_not_found()) { - LOG(WARNING) << "file check exist error: " << dst_path_desc.filepath; + LOG(WARNING) << "file check exist error: " << dst_path; return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); } - FilePathDesc src_path_desc = segment_file_path(_rowset_path_desc, rowset_id(), i); - if (!Env::Default()->copy_path(src_path_desc.filepath, dst_path_desc.filepath).ok()) { - LOG(WARNING) << "fail to copy file. from=" << src_path_desc.filepath - << ", to=" << dst_path_desc.filepath << ", errno=" << Errno::no(); + auto src_path = segment_file_path(i); + if (!Env::Default()->copy_path(src_path, dst_path).ok()) { + LOG(WARNING) << "fail to copy file. from=" << src_path << ", to=" << dst_path + << ", errno=" << Errno::no(); return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); } } return Status::OK(); } -Status BetaRowset::upload_files_to(const FilePathDesc& dir_desc, const RowsetId& new_rowset_id, - bool delete_src) { - std::shared_ptr storage_backend = - StorageBackendMgr::instance()->get_storage_backend(dir_desc.storage_name); - if (storage_backend == nullptr) { - LOG(WARNING) << "storage_backend is invalid: " << dir_desc.debug_string(); - return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); +Status BetaRowset::upload_to(io::RemoteFileSystem* dest_fs, const RowsetId& new_rowset_id) { + DCHECK(is_local()); + if (num_segments() < 1) { + return Status::OK(); } + std::vector local_paths; + local_paths.reserve(num_segments()); + std::vector dest_paths; + dest_paths.reserve(num_segments()); for (int i = 0; i < num_segments(); ++i) { - FilePathDesc dst_path_desc = segment_file_path(dir_desc, new_rowset_id, i); - Status status = storage_backend->exist(dst_path_desc.remote_path); - if (status.ok()) { - LOG(WARNING) << "file already exist: " << dst_path_desc.remote_path; - return Status::OLAPInternalError(OLAP_ERR_FILE_ALREADY_EXIST); - } else if (!status.is_not_found()) { - LOG(WARNING) << "file check exist error: " << dst_path_desc.remote_path; - return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); - } - FilePathDesc src_path_desc = segment_file_path(_rowset_path_desc, rowset_id(), i); - - if (!storage_backend->upload(src_path_desc.filepath, dst_path_desc.remote_path).ok()) { - LOG(WARNING) << "fail to upload file. from=" << src_path_desc.filepath - << ", to=" << dst_path_desc.remote_path << ", errno=" << Errno::no(); - return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); - } - if (delete_src && !Env::Default()->delete_file(src_path_desc.filepath).ok()) { - LOG(WARNING) << "fail to delete local file: " << src_path_desc.filepath - << ", errno=" << Errno::no(); - return Status::OLAPInternalError(OLAP_ERR_OS_ERROR); - } - LOG(INFO) << "succeed to upload file. from " << src_path_desc.filepath << " to " - << dst_path_desc.remote_path; + // Note: Here we use relative path for remote. + dest_paths.push_back(remote_segment_path(_rowset_meta->tablet_id(), new_rowset_id, i)); + local_paths.push_back(segment_file_path(i)); } - return Status::OK(); + auto st = dest_fs->batch_upload(local_paths, dest_paths); + if (st.ok()) { + DorisMetrics::instance()->upload_rowset_count->increment(1); + DorisMetrics::instance()->upload_total_byte->increment(data_disk_size()); + } else { + DorisMetrics::instance()->upload_fail_count->increment(1); + } + return st; } bool BetaRowset::check_path(const std::string& path) { - std::set valid_paths; for (int i = 0; i < num_segments(); ++i) { - FilePathDesc path_desc = segment_file_path(_rowset_path_desc, rowset_id(), i); - valid_paths.insert(path_desc.filepath); + auto seg_path = segment_file_path(i); + if (seg_path == path) { + return true; + } } - return valid_paths.find(path) != valid_paths.end(); + return false; } bool BetaRowset::check_file_exist() { - if (_rowset_path_desc.is_remote()) { - std::shared_ptr storage_backend = - StorageBackendMgr::instance()->get_storage_backend(_rowset_path_desc.storage_name); - if (storage_backend == nullptr) { - LOG(WARNING) << "storage_backend is invalid: " << _rowset_path_desc.debug_string(); + for (int i = 0; i < num_segments(); ++i) { + auto seg_path = segment_file_path(i); + if (!Env::Default()->path_exists(seg_path).ok()) { + LOG(WARNING) << "data file not existed: " << seg_path + << " for rowset_id: " << rowset_id(); return false; } - for (int i = 0; i < num_segments(); ++i) { - FilePathDesc path_desc = segment_file_path(_rowset_path_desc, rowset_id(), i); - if (!storage_backend->exist(path_desc.remote_path).ok()) { - LOG(WARNING) << "data file not existed: " << path_desc.remote_path - << " for rowset_id: " << rowset_id(); - return false; - } - } - } else { - for (int i = 0; i < num_segments(); ++i) { - FilePathDesc path_desc = segment_file_path(_rowset_path_desc, rowset_id(), i); - if (!Env::Default()->path_exists(path_desc.filepath).ok()) { - LOG(WARNING) << "data file not existed: " << path_desc.filepath - << " for rowset_id: " << rowset_id(); - return false; - } - } } return true; } diff --git a/be/src/olap/rowset/beta_rowset.h b/be/src/olap/rowset/beta_rowset.h index 0478f832e19650..9ccaa7b2b903d0 100644 --- a/be/src/olap/rowset/beta_rowset.h +++ b/be/src/olap/rowset/beta_rowset.h @@ -18,6 +18,8 @@ #ifndef DORIS_SRC_OLAP_ROWSET_BETA_ROWSET_H_ #define DORIS_SRC_OLAP_ROWSET_BETA_ROWSET_H_ +#include + #include "olap/olap_common.h" #include "olap/olap_define.h" #include "olap/rowset/rowset.h" @@ -39,8 +41,13 @@ class BetaRowset : public Rowset { Status create_reader(RowsetReaderSharedPtr* result) override; - static FilePathDesc segment_file_path(const FilePathDesc& segment_dir_desc, - const RowsetId& rowset_id, int segment_id); + std::string segment_file_path(int segment_id); + + static std::string local_segment_path(const std::string& tablet_path, const RowsetId& rowset_id, + int segment_id); + + static std::string remote_segment_path(int64_t tablet_id, const RowsetId& rowset_id, + int segment_id); Status split_range(const RowCursor& start_key, const RowCursor& end_key, uint64_t request_block_row_count, size_t key_num, @@ -48,12 +55,11 @@ class BetaRowset : public Rowset { Status remove() override; - Status link_files_to(const FilePathDesc& dir_desc, RowsetId new_rowset_id) override; + Status link_files_to(const std::string& dir, RowsetId new_rowset_id) override; Status copy_files_to(const std::string& dir, const RowsetId& new_rowset_id) override; - Status upload_files_to(const FilePathDesc& dir_desc, const RowsetId& new_rowset_id, - bool delete_src = false) override; + Status upload_to(io::RemoteFileSystem* dest_fs, const RowsetId& new_rowset_id) override; // only applicable to alpha rowset, no op here Status remove_old_files(std::vector* files_to_remove) override { @@ -67,7 +73,7 @@ class BetaRowset : public Rowset { Status load_segments(std::vector* segments); protected: - BetaRowset(const TabletSchema* schema, const FilePathDesc& rowset_path_desc, + BetaRowset(const TabletSchema* schema, const std::string& tablet_path, RowsetMetaSharedPtr rowset_meta); // init segment groups diff --git a/be/src/olap/rowset/beta_rowset_reader.h b/be/src/olap/rowset/beta_rowset_reader.h index de1251f13f5e01..a6c7caf57c6da4 100644 --- a/be/src/olap/rowset/beta_rowset_reader.h +++ b/be/src/olap/rowset/beta_rowset_reader.h @@ -43,6 +43,9 @@ class BetaRowsetReader : public RowsetReader { Version version() override { return _rowset->version(); } + int64_t oldest_write_timestamp() override { return _rowset->oldest_write_timestamp(); } + int64_t newest_write_timestamp() override { return _rowset->newest_write_timestamp(); } + RowsetSharedPtr rowset() override { return std::dynamic_pointer_cast(_rowset); } // Return the total number of filtered rows, will be used for validation of schema change diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp b/be/src/olap/rowset/beta_rowset_writer.cpp index f0891234d23eb1..439327ef79a344 100644 --- a/be/src/olap/rowset/beta_rowset_writer.cpp +++ b/be/src/olap/rowset/beta_rowset_writer.cpp @@ -23,6 +23,7 @@ #include "common/logging.h" #include "env/env.h" #include "gutil/strings/substitute.h" +#include "io/fs/file_writer.h" #include "olap/fs/fs_util.h" #include "olap/memtable.h" #include "olap/olap_define.h" @@ -33,8 +34,6 @@ #include "olap/rowset/segment_v2/segment_writer.h" #include "olap/storage_engine.h" #include "runtime/exec_env.h" -#include "util/storage_backend.h" -#include "util/storage_backend_mgr.h" namespace doris { @@ -50,27 +49,18 @@ BetaRowsetWriter::~BetaRowsetWriter() { // TODO(lingbin): Should wrapper exception logic, no need to know file ops directly. if (!_already_built) { // abnormal exit, remove all files generated _segment_writer.reset(); // ensure all files are closed - Status st; - if (_context.path_desc.is_remote()) { - std::shared_ptr storage_backend = - StorageBackendMgr::instance()->get_storage_backend( - _context.path_desc.storage_name); - if (storage_backend == nullptr) { - LOG(WARNING) << "storage_backend is invalid: " << _context.path_desc.debug_string(); - return; - } - WARN_IF_ERROR(storage_backend->rmdir(_context.path_desc.remote_path), - strings::Substitute("Failed to delete remote file=$0", - _context.path_desc.remote_path)); + auto fs = _rowset_meta->fs(); + if (!fs) { + return; } for (int i = 0; i < _num_segment; ++i) { - auto path_desc = - BetaRowset::segment_file_path(_context.path_desc, _context.rowset_id, i); + auto seg_path = + BetaRowset::local_segment_path(_context.tablet_path, _context.rowset_id, i); // Even if an error is encountered, these files that have not been cleaned up // will be cleaned up by the GC background. So here we only print the error // message when we encounter an error. - WARN_IF_ERROR(Env::Default()->delete_file(path_desc.filepath), - strings::Substitute("Failed to delete file=$0", path_desc.filepath)); + WARN_IF_ERROR(fs->delete_file(seg_path), + strings::Substitute("Failed to delete file=$0", seg_path)); } } } @@ -78,6 +68,9 @@ BetaRowsetWriter::~BetaRowsetWriter() { Status BetaRowsetWriter::init(const RowsetWriterContext& rowset_writer_context) { _context = rowset_writer_context; _rowset_meta.reset(new RowsetMeta); + if (_context.data_dir) { + _rowset_meta->set_fs(_context.data_dir->fs()); + } _rowset_meta->set_rowset_id(_context.rowset_id); _rowset_meta->set_partition_id(_context.partition_id); _rowset_meta->set_tablet_id(_context.tablet_id); @@ -91,9 +84,10 @@ Status BetaRowsetWriter::init(const RowsetWriterContext& rowset_writer_context) _rowset_meta->set_load_id(_context.load_id); } else { _rowset_meta->set_version(_context.version); + _rowset_meta->set_oldest_write_timestamp(_context.oldest_write_timestamp); + _rowset_meta->set_newest_write_timestamp(_context.newest_write_timestamp); } _rowset_meta->set_tablet_uid(_context.tablet_uid); - return Status::OK(); } @@ -161,7 +155,7 @@ template Status BetaRowsetWriter::_add_row(const ContiguousRow& row); Status BetaRowsetWriter::add_rowset(RowsetSharedPtr rowset) { assert(rowset->rowset_meta()->rowset_type() == BETA_ROWSET); - RETURN_NOT_OK(rowset->link_files_to(_context.path_desc, _context.rowset_id)); + RETURN_NOT_OK(rowset->link_files_to(_context.tablet_path, _context.rowset_id)); _num_rows_written += rowset->num_rows(); _total_data_size += rowset->rowset_meta()->data_disk_size(); _total_index_size += rowset->rowset_meta()->index_disk_size(); @@ -179,42 +173,6 @@ Status BetaRowsetWriter::add_rowset_for_linked_schema_change(RowsetSharedPtr row return add_rowset(rowset); } -Status BetaRowsetWriter::add_rowset_for_migration(RowsetSharedPtr rowset) { - Status res = Status::OK(); - assert(rowset->rowset_meta()->rowset_type() == BETA_ROWSET); - if (!rowset->rowset_path_desc().is_remote() && !_context.path_desc.is_remote()) { - res = rowset->copy_files_to(_context.path_desc.filepath, _context.rowset_id); - if (!res.ok()) { - LOG(WARNING) << "copy_files failed. src: " << rowset->rowset_path_desc().filepath - << ", dest: " << _context.path_desc.filepath; - return res; - } - } else if (!rowset->rowset_path_desc().is_remote() && _context.path_desc.is_remote()) { - res = rowset->upload_files_to(_context.path_desc, _context.rowset_id); - if (!res.ok()) { - LOG(WARNING) << "upload_files failed. src: " - << rowset->rowset_path_desc().debug_string() - << ", dest: " << _context.path_desc.debug_string(); - return res; - } - } else { - LOG(WARNING) << "add_rowset_for_migration failed. storage_medium is invalid. src: " - << rowset->rowset_path_desc().debug_string() - << ", dest: " << _context.path_desc.debug_string(); - return Status::OLAPInternalError(OLAP_ERR_ROWSET_ADD_MIGRATION_V2); - } - - _num_rows_written += rowset->num_rows(); - _total_data_size += rowset->rowset_meta()->data_disk_size(); - _total_index_size += rowset->rowset_meta()->index_disk_size(); - _num_segment += rowset->num_segments(); - // TODO update zonemap - if (rowset->rowset_meta()->has_delete_predicate()) { - _rowset_meta->set_delete_predicate(rowset->rowset_meta()->delete_predicate()); - } - return Status::OK(); -} - Status BetaRowsetWriter::flush() { if (_segment_writer != nullptr) { RETURN_NOT_OK(_flush_segment_writer(&_segment_writer)); @@ -268,8 +226,8 @@ Status BetaRowsetWriter::flush_single_memtable(const vectorized::Block* block) { RowsetSharedPtr BetaRowsetWriter::build() { // TODO(lingbin): move to more better place, or in a CreateBlockBatch? - for (auto& wblock : _wblocks) { - wblock->close(); + for (auto& file_writer : _file_writers) { + file_writer->close(); } // When building a rowset, we must ensure that the current _segment_writer has been // flushed, that is, the current _segment_writer is nullptr @@ -291,8 +249,16 @@ RowsetSharedPtr BetaRowsetWriter::build() { _rowset_meta->set_rowset_state(VISIBLE); } + if (_rowset_meta->oldest_write_timestamp() == -1) { + _rowset_meta->set_oldest_write_timestamp(UnixSeconds()); + } + + if (_rowset_meta->newest_write_timestamp() == -1) { + _rowset_meta->set_newest_write_timestamp(UnixSeconds()); + } + RowsetSharedPtr rowset; - auto status = RowsetFactory::create_rowset(_context.tablet_schema, _context.path_desc, + auto status = RowsetFactory::create_rowset(_context.tablet_schema, _context.tablet_path, _rowset_meta, &rowset); if (!status.ok()) { LOG(WARNING) << "rowset init failed when build new rowset, res=" << status; @@ -304,29 +270,28 @@ RowsetSharedPtr BetaRowsetWriter::build() { Status BetaRowsetWriter::_create_segment_writer( std::unique_ptr* writer) { - auto path_desc = - BetaRowset::segment_file_path(_context.path_desc, _context.rowset_id, _num_segment++); - // TODO(lingbin): should use a more general way to get BlockManager object - // and tablets with the same type should share one BlockManager object; - fs::BlockManager* block_mgr = fs::fs_util::block_manager(_context.path_desc); - std::unique_ptr wblock; - fs::CreateBlockOptions opts(path_desc); - DCHECK(block_mgr != nullptr); - Status st = block_mgr->create_block(opts, &wblock); + auto path = BetaRowset::local_segment_path(_context.tablet_path, _context.rowset_id, + _num_segment++); + auto fs = _rowset_meta->fs(); + if (!fs) { + return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED); + } + std::unique_ptr file_writer; + Status st = fs->create_file(path, &file_writer); if (!st.ok()) { - LOG(WARNING) << "failed to create writable block. path=" << path_desc.filepath + LOG(WARNING) << "failed to create writable file. path=" << path << ", err: " << st.get_error_msg(); return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED); } - DCHECK(wblock != nullptr); + DCHECK(file_writer != nullptr); segment_v2::SegmentWriterOptions writer_options; - writer->reset(new segment_v2::SegmentWriter(wblock.get(), _num_segment, _context.tablet_schema, - _context.data_dir, _context.max_rows_per_segment, - writer_options)); + writer->reset(new segment_v2::SegmentWriter(file_writer.get(), _num_segment, + _context.tablet_schema, _context.data_dir, + _context.max_rows_per_segment, writer_options)); { std::lock_guard l(_lock); - _wblocks.push_back(std::move(wblock)); + _file_writers.push_back(std::move(file_writer)); } auto s = (*writer)->init(config::push_write_mbytes_per_sec); diff --git a/be/src/olap/rowset/beta_rowset_writer.h b/be/src/olap/rowset/beta_rowset_writer.h index 69a957a3ce0504..bc0b91fdb6f286 100644 --- a/be/src/olap/rowset/beta_rowset_writer.h +++ b/be/src/olap/rowset/beta_rowset_writer.h @@ -18,18 +18,16 @@ #pragma once #include "olap/rowset/rowset_writer.h" -#include "vector" namespace doris { - -namespace fs { -class WritableBlock; -} - namespace segment_v2 { class SegmentWriter; } // namespace segment_v2 +namespace io { +class FileWriter; +} // namespace io + class BetaRowsetWriter : public RowsetWriter { public: BetaRowsetWriter(); @@ -50,11 +48,10 @@ class BetaRowsetWriter : public RowsetWriter { Status add_rowset_for_linked_schema_change(RowsetSharedPtr rowset, const SchemaMapping& schema_mapping) override; - Status add_rowset_for_migration(RowsetSharedPtr rowset) override; - Status flush() override; // Return the file size flushed to disk in "flush_size" + // This method is thread-safe. Status flush_single_memtable(MemTable* memtable, int64_t* flush_size) override; Status flush_single_memtable(const vectorized::Block* block) override; @@ -89,7 +86,7 @@ class BetaRowsetWriter : public RowsetWriter { std::unique_ptr _segment_writer; mutable SpinLock _lock; // lock to protect _wblocks. // TODO(lingbin): it is better to wrapper in a Batch? - std::vector> _wblocks; + std::vector> _file_writers; // counters and statistics maintained during data write std::atomic _num_rows_written; diff --git a/be/src/olap/rowset/rowset.cpp b/be/src/olap/rowset/rowset.cpp index 7003fd5929df0a..55fe81ffd32f04 100644 --- a/be/src/olap/rowset/rowset.cpp +++ b/be/src/olap/rowset/rowset.cpp @@ -21,13 +21,12 @@ namespace doris { -Rowset::Rowset(const TabletSchema* schema, const FilePathDesc& rowset_path_desc, +Rowset::Rowset(const TabletSchema* schema, const std::string& tablet_path, RowsetMetaSharedPtr rowset_meta) : _schema(schema), - _rowset_path_desc(rowset_path_desc), + _tablet_path(tablet_path), _rowset_meta(std::move(rowset_meta)), - _refs_by_reader(0), - _rowset_state_machine(RowsetStateMachine()) { + _refs_by_reader(0) { _is_pending = !_rowset_meta->has_version(); if (_is_pending) { _is_cumulative = false; diff --git a/be/src/olap/rowset/rowset.h b/be/src/olap/rowset/rowset.h index f1fded38badbf4..158848be89bcc4 100644 --- a/be/src/olap/rowset/rowset.h +++ b/be/src/olap/rowset/rowset.h @@ -25,6 +25,7 @@ #include "env/env.h" #include "gen_cpp/olap_file.pb.h" #include "gutil/macros.h" +#include "io/fs/remote_file_system.h" #include "olap/rowset/rowset_meta.h" #include "olap/tablet_schema.h" @@ -133,6 +134,8 @@ class Rowset : public std::enable_shared_from_this { bool is_pending() const { return _is_pending; } + bool is_local() const { return _rowset_meta->is_local(); } + // publish rowset to make it visible to read void make_visible(Version version); @@ -155,6 +158,8 @@ class Rowset : public std::enable_shared_from_this { int64_t num_segments() const { return rowset_meta()->num_segments(); } void to_rowset_pb(RowsetMetaPB* rs_meta) const { return rowset_meta()->to_rowset_pb(rs_meta); } const RowsetMetaPB& get_rowset_pb() const { return rowset_meta()->get_rowset_pb(); } + int64_t oldest_write_timestamp() const { return rowset_meta()->oldest_write_timestamp(); } + int64_t newest_write_timestamp() const { return rowset_meta()->newest_write_timestamp(); } KeysType keys_type() { return _schema->keys_type(); } // remove all files in this rowset @@ -192,13 +197,12 @@ class Rowset : public std::enable_shared_from_this { } // hard link all files in this rowset to `dir` to form a new rowset with id `new_rowset_id`. - virtual Status link_files_to(const FilePathDesc& dir_desc, RowsetId new_rowset_id) = 0; + virtual Status link_files_to(const std::string& dir, RowsetId new_rowset_id) = 0; // copy all files to `dir` virtual Status copy_files_to(const std::string& dir, const RowsetId& new_rowset_id) = 0; - virtual Status upload_files_to(const FilePathDesc& dir_desc, const RowsetId&, - bool delete_src = false) { + virtual Status upload_to(io::RemoteFileSystem* dest_fs, const RowsetId& new_rowset_id) { return Status::OK(); } @@ -211,7 +215,7 @@ class Rowset : public std::enable_shared_from_this { // return an unique identifier string for this rowset std::string unique_id() const { - return _rowset_path_desc.filepath + "/" + rowset_id().to_string(); + return fmt::format("{}/{}", _tablet_path, rowset_id().to_string()); } bool need_delete_file() const { return _need_delete_file; } @@ -222,7 +226,7 @@ class Rowset : public std::enable_shared_from_this { return rowset_meta()->version().contains(version); } - FilePathDesc rowset_path_desc() { return _rowset_path_desc; } + const std::string& tablet_path() const { return _tablet_path; } static bool comparator(const RowsetSharedPtr& left, const RowsetSharedPtr& right) { return left->end_version() < right->end_version(); @@ -259,7 +263,7 @@ class Rowset : public std::enable_shared_from_this { DISALLOW_COPY_AND_ASSIGN(Rowset); // this is non-public because all clients should use RowsetFactory to obtain pointer to initialized Rowset - Rowset(const TabletSchema* schema, const FilePathDesc& rowset_path_desc, + Rowset(const TabletSchema* schema, const std::string& tablet_path, RowsetMetaSharedPtr rowset_meta); // this is non-public because all clients should use RowsetFactory to obtain pointer to initialized Rowset @@ -275,7 +279,7 @@ class Rowset : public std::enable_shared_from_this { virtual void make_visible_extra(Version version) {} const TabletSchema* _schema; - FilePathDesc _rowset_path_desc; + std::string _tablet_path; RowsetMetaSharedPtr _rowset_meta; // init in constructor bool _is_pending; // rowset is pending iff it's not in visible state diff --git a/be/src/olap/rowset/rowset_factory.cpp b/be/src/olap/rowset/rowset_factory.cpp index 361290fe51dce5..fb600c0bb6a62c 100644 --- a/be/src/olap/rowset/rowset_factory.cpp +++ b/be/src/olap/rowset/rowset_factory.cpp @@ -28,15 +28,14 @@ namespace doris { -Status RowsetFactory::create_rowset(const TabletSchema* schema, - const FilePathDesc& rowset_path_desc, +Status RowsetFactory::create_rowset(const TabletSchema* schema, const std::string& tablet_path, RowsetMetaSharedPtr rowset_meta, RowsetSharedPtr* rowset) { if (rowset_meta->rowset_type() == ALPHA_ROWSET) { - rowset->reset(new AlphaRowset(schema, rowset_path_desc, rowset_meta)); + rowset->reset(new AlphaRowset(schema, tablet_path, rowset_meta)); return (*rowset)->init(); } if (rowset_meta->rowset_type() == BETA_ROWSET) { - rowset->reset(new BetaRowset(schema, rowset_path_desc, rowset_meta)); + rowset->reset(new BetaRowset(schema, tablet_path, rowset_meta)); return (*rowset)->init(); } return Status::OLAPInternalError(OLAP_ERR_ROWSET_TYPE_NOT_FOUND); // should never happen diff --git a/be/src/olap/rowset/rowset_factory.h b/be/src/olap/rowset/rowset_factory.h index 48d82eeb2fbd96..91b36db1ed074a 100644 --- a/be/src/olap/rowset/rowset_factory.h +++ b/be/src/olap/rowset/rowset_factory.h @@ -31,7 +31,7 @@ class RowsetFactory { public: // return OLAP_SUCCESS and set inited rowset in `*rowset`. // return others if failed to create or init rowset. - static Status create_rowset(const TabletSchema* schema, const FilePathDesc& rowset_path_desc, + static Status create_rowset(const TabletSchema* schema, const std::string& tablet_path, RowsetMetaSharedPtr rowset_meta, RowsetSharedPtr* rowset); // create and init rowset writer. diff --git a/be/src/olap/rowset/rowset_meta.h b/be/src/olap/rowset/rowset_meta.h index b55fc9402183a4..e4153e234572d9 100644 --- a/be/src/olap/rowset/rowset_meta.h +++ b/be/src/olap/rowset/rowset_meta.h @@ -25,6 +25,9 @@ #include "common/logging.h" #include "gen_cpp/olap_file.pb.h" #include "google/protobuf/util/message_differencer.h" +#include "io/fs/file_system.h" +#include "io/fs/file_system_map.h" +#include "io/fs/local_file_system.h" #include "json2pb/json_to_pb.h" #include "json2pb/pb_to_json.h" #include "olap/olap_common.h" @@ -36,7 +39,7 @@ using RowsetMetaSharedPtr = std::shared_ptr; class RowsetMeta { public: - virtual ~RowsetMeta() {} + virtual ~RowsetMeta() = default; virtual bool init(const std::string& pb_rowset_meta) { bool ret = _deserialize_from_pb(pb_rowset_meta); @@ -71,6 +74,29 @@ class RowsetMeta { return ret; } + // This method may return nullptr. + io::FileSystem* fs() { + if (!_fs) { + if (is_local()) { + return io::global_local_filesystem(); + } else { + _fs = io::FileSystemMap::instance()->get(resource_id()); + LOG_IF(WARNING, !_fs) << "Cannot get file system: " << resource_id(); + } + } + return _fs.get(); + } + + void set_fs(io::FileSystemPtr fs) { _fs = std::move(fs); } + + const io::ResourceId& resource_id() const { return _rowset_meta_pb.resource_id(); } + + void set_resource_id(io::ResourceId resource_id) { + _rowset_meta_pb.set_resource_id(std::move(resource_id)); + } + + bool is_local() const { return !_rowset_meta_pb.has_resource_id(); } + RowsetId rowset_id() const { return _rowset_id; } void set_rowset_id(const RowsetId& rowset_id) { @@ -276,6 +302,18 @@ class RowsetMeta { return _rowset_meta_pb.alpha_rowset_extra_meta_pb(); } + void set_oldest_write_timestamp(int64_t timestamp) { + _rowset_meta_pb.set_oldest_write_timestamp(timestamp); + } + + void set_newest_write_timestamp(int64_t timestamp) { + _rowset_meta_pb.set_newest_write_timestamp(timestamp); + } + + int64_t oldest_write_timestamp() const { return _rowset_meta_pb.oldest_write_timestamp(); } + + int64_t newest_write_timestamp() const { return _rowset_meta_pb.newest_write_timestamp(); } + private: friend class AlphaRowsetMeta; bool _deserialize_from_pb(const std::string& value) { @@ -334,6 +372,7 @@ class RowsetMeta { private: RowsetMetaPB _rowset_meta_pb; RowsetId _rowset_id; + io::FileSystemPtr _fs; bool _is_removed_from_rowset_meta = false; }; diff --git a/be/src/olap/rowset/rowset_reader.h b/be/src/olap/rowset/rowset_reader.h index d3026376ff18ff..d3dcb30d767cf0 100644 --- a/be/src/olap/rowset/rowset_reader.h +++ b/be/src/olap/rowset/rowset_reader.h @@ -61,6 +61,9 @@ class RowsetReader { virtual int64_t filtered_rows() = 0; virtual RowsetTypePB type() const = 0; + + virtual int64_t oldest_write_timestamp() = 0; + virtual int64_t newest_write_timestamp() = 0; }; } // namespace doris diff --git a/be/src/olap/rowset/rowset_writer.h b/be/src/olap/rowset/rowset_writer.h index 49824cac01af8e..dfe5186ad80989 100644 --- a/be/src/olap/rowset/rowset_writer.h +++ b/be/src/olap/rowset/rowset_writer.h @@ -53,8 +53,6 @@ class RowsetWriter { virtual Status add_rowset_for_linked_schema_change(RowsetSharedPtr rowset, const SchemaMapping& schema_mapping) = 0; - virtual Status add_rowset_for_migration(RowsetSharedPtr rowset) = 0; - // explicit flush all buffered rows into segment file. // note that `add_row` could also trigger flush when certain conditions are met virtual Status flush() = 0; diff --git a/be/src/olap/rowset/rowset_writer_context.h b/be/src/olap/rowset/rowset_writer_context.h index 539845fe5ff13b..e73fe90349b82d 100644 --- a/be/src/olap/rowset/rowset_writer_context.h +++ b/be/src/olap/rowset/rowset_writer_context.h @@ -69,7 +69,7 @@ struct RowsetWriterContext { int64_t tablet_schema_hash; int64_t partition_id; RowsetTypePB rowset_type; - FilePathDesc path_desc; + std::string tablet_path; const TabletSchema* tablet_schema; // PREPARED/COMMITTED for pending rowset // VISIBLE for non-pending rowset @@ -93,6 +93,9 @@ struct RowsetWriterContext { // ATTN: not support for RowsetConvertor. // (because it hard to refactor, and RowsetConvertor will be deprecated in future) DataDir* data_dir = nullptr; + + int64_t oldest_write_timestamp; + int64_t newest_write_timestamp; }; } // namespace doris diff --git a/be/src/olap/rowset/segment_group.cpp b/be/src/olap/rowset/segment_group.cpp index d2eb084424ab8e..d8ada41cc3937f 100644 --- a/be/src/olap/rowset/segment_group.cpp +++ b/be/src/olap/rowset/segment_group.cpp @@ -911,7 +911,7 @@ Status SegmentGroup::remove_old_files(std::vector* links_to_remove) } } } - std::string pending_delta_path = _rowset_path_prefix + PENDING_DELTA_PREFIX; + std::string pending_delta_path = _rowset_path_prefix + "/" + PENDING_DELTA_PREFIX; if (FileUtils::check_exist(pending_delta_path)) { LOG(INFO) << "remove pending delta path:" << pending_delta_path; RETURN_WITH_WARN_IF_ERROR(FileUtils::remove_all(pending_delta_path), diff --git a/be/src/olap/rowset/segment_v2/bitmap_index_reader.cpp b/be/src/olap/rowset/segment_v2/bitmap_index_reader.cpp index d3603e99ee6615..cbaecedd78c0dc 100644 --- a/be/src/olap/rowset/segment_v2/bitmap_index_reader.cpp +++ b/be/src/olap/rowset/segment_v2/bitmap_index_reader.cpp @@ -27,8 +27,8 @@ Status BitmapIndexReader::load(bool use_page_cache, bool kept_in_memory) { const IndexedColumnMetaPB& bitmap_meta = _bitmap_index_meta->bitmap_column(); _has_null = _bitmap_index_meta->has_null(); - _dict_column_reader.reset(new IndexedColumnReader(_path_desc, dict_meta)); - _bitmap_column_reader.reset(new IndexedColumnReader(_path_desc, bitmap_meta)); + _dict_column_reader.reset(new IndexedColumnReader(_fs, _path, dict_meta)); + _bitmap_column_reader.reset(new IndexedColumnReader(_fs, _path, bitmap_meta)); RETURN_IF_ERROR(_dict_column_reader->load(use_page_cache, kept_in_memory)); RETURN_IF_ERROR(_bitmap_column_reader->load(use_page_cache, kept_in_memory)); return Status::OK(); diff --git a/be/src/olap/rowset/segment_v2/bitmap_index_reader.h b/be/src/olap/rowset/segment_v2/bitmap_index_reader.h index 71cd5b5c59c209..77231bc049bb00 100644 --- a/be/src/olap/rowset/segment_v2/bitmap_index_reader.h +++ b/be/src/olap/rowset/segment_v2/bitmap_index_reader.h @@ -21,6 +21,7 @@ #include "common/status.h" #include "gen_cpp/segment_v2.pb.h" +#include "io/fs/file_system.h" #include "olap/column_block.h" #include "olap/rowset/segment_v2/common.h" #include "olap/rowset/segment_v2/indexed_column_reader.h" @@ -38,9 +39,10 @@ class IndexedColumnIterator; class BitmapIndexReader { public: - explicit BitmapIndexReader(const FilePathDesc& path_desc, + explicit BitmapIndexReader(io::FileSystem* fs, const std::string& path, const BitmapIndexPB* bitmap_index_meta) - : _path_desc(path_desc), + : _fs(fs), + _path(path), _type_info(get_scalar_type_info()), _bitmap_index_meta(bitmap_index_meta) {} @@ -56,7 +58,8 @@ class BitmapIndexReader { private: friend class BitmapIndexIterator; - FilePathDesc _path_desc; + io::FileSystem* _fs; + std::string _path; const TypeInfo* _type_info; const BitmapIndexPB* _bitmap_index_meta; bool _has_null = false; diff --git a/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp b/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp index 93cac8b5d20bc4..d09deff66a0c35 100644 --- a/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp @@ -99,7 +99,7 @@ class BitmapIndexWriterImpl : public BitmapIndexWriter { _rid += count; } - Status finish(fs::WritableBlock* wblock, ColumnIndexMetaPB* index_meta) override { + Status finish(io::FileWriter* file_writer, ColumnIndexMetaPB* index_meta) override { index_meta->set_type(BITMAP_INDEX); BitmapIndexPB* meta = index_meta->mutable_bitmap_index(); @@ -113,7 +113,7 @@ class BitmapIndexWriterImpl : public BitmapIndexWriter { options.encoding = EncodingInfo::get_default_encoding(_type_info, true); options.compression = LZ4F; - IndexedColumnWriter dict_column_writer(options, _type_info, wblock); + IndexedColumnWriter dict_column_writer(options, _type_info, file_writer); RETURN_IF_ERROR(dict_column_writer.init()); for (auto const& it : _mem_index) { RETURN_IF_ERROR(dict_column_writer.add(&(it.first))); @@ -148,7 +148,7 @@ class BitmapIndexWriterImpl : public BitmapIndexWriter { // we already store compressed bitmap, use NO_COMPRESSION to save some cpu options.compression = NO_COMPRESSION; - IndexedColumnWriter bitmap_column_writer(options, bitmap_type_info, wblock); + IndexedColumnWriter bitmap_column_writer(options, bitmap_type_info, file_writer); RETURN_IF_ERROR(bitmap_column_writer.init()); faststring buf; diff --git a/be/src/olap/rowset/segment_v2/bitmap_index_writer.h b/be/src/olap/rowset/segment_v2/bitmap_index_writer.h index acf62d703f39bc..93142728a472b6 100644 --- a/be/src/olap/rowset/segment_v2/bitmap_index_writer.h +++ b/be/src/olap/rowset/segment_v2/bitmap_index_writer.h @@ -28,8 +28,8 @@ namespace doris { class TypeInfo; -namespace fs { -class WritableBlock; +namespace io { +class FileWriter; } namespace segment_v2 { @@ -45,7 +45,7 @@ class BitmapIndexWriter { virtual void add_nulls(uint32_t count) = 0; - virtual Status finish(fs::WritableBlock* file, ColumnIndexMetaPB* index_meta) = 0; + virtual Status finish(io::FileWriter* file_writer, ColumnIndexMetaPB* index_meta) = 0; virtual uint64_t size() const = 0; diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp index d35e2be7fe8e88..d7ad5bc2abd0fc 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.cpp @@ -26,7 +26,7 @@ namespace segment_v2 { Status BloomFilterIndexReader::load(bool use_page_cache, bool kept_in_memory) { const IndexedColumnMetaPB& bf_index_meta = _bloom_filter_index_meta->bloom_filter(); - _bloom_filter_reader.reset(new IndexedColumnReader(_path_desc, bf_index_meta)); + _bloom_filter_reader.reset(new IndexedColumnReader(_fs, _path, bf_index_meta)); RETURN_IF_ERROR(_bloom_filter_reader->load(use_page_cache, kept_in_memory)); return Status::OK(); } diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h index 42aa415be3e55c..f01fd070ab586d 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_reader.h @@ -22,6 +22,7 @@ #include "common/status.h" #include "gen_cpp/segment_v2.pb.h" +#include "io/fs/file_system.h" #include "olap/column_block.h" #include "olap/rowset/segment_v2/common.h" #include "olap/rowset/segment_v2/indexed_column_reader.h" @@ -41,9 +42,10 @@ class BloomFilter; class BloomFilterIndexReader { public: - explicit BloomFilterIndexReader(const FilePathDesc& path_desc, + explicit BloomFilterIndexReader(io::FileSystem* fs, const std::string& path, const BloomFilterIndexPB* bloom_filter_index_meta) - : _path_desc(path_desc), + : _fs(fs), + _path(path), _type_info(get_scalar_type_info()), _bloom_filter_index_meta(bloom_filter_index_meta) {} @@ -57,7 +59,8 @@ class BloomFilterIndexReader { private: friend class BloomFilterIndexIterator; - FilePathDesc _path_desc; + io::FileSystem* _fs; + std::string _path; const TypeInfo* _type_info; const BloomFilterIndexPB* _bloom_filter_index_meta; std::unique_ptr _bloom_filter_reader; diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp index 5b3156078e0418..c20ebca5f2c084 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp @@ -20,8 +20,6 @@ #include #include -#include "env/env.h" -#include "olap/fs/block_manager.h" #include "olap/rowset/segment_v2/bloom_filter.h" // for BloomFilterOptions, BloomFilter #include "olap/rowset/segment_v2/common.h" #include "olap/rowset/segment_v2/encoding_info.h" @@ -119,7 +117,7 @@ class BloomFilterIndexWriterImpl : public BloomFilterIndexWriter { return Status::OK(); } - Status finish(fs::WritableBlock* wblock, ColumnIndexMetaPB* index_meta) override { + Status finish(io::FileWriter* file_writer, ColumnIndexMetaPB* index_meta) override { if (_values.size() > 0) { RETURN_IF_ERROR(flush()); } @@ -134,7 +132,7 @@ class BloomFilterIndexWriterImpl : public BloomFilterIndexWriter { options.write_ordinal_index = true; options.write_value_index = false; options.encoding = PLAIN_ENCODING; - IndexedColumnWriter bf_writer(options, bf_type_info, wblock); + IndexedColumnWriter bf_writer(options, bf_type_info, file_writer); RETURN_IF_ERROR(bf_writer.init()); for (auto& bf : _bfs) { Slice data(bf->data(), bf->size()); diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h index f418359aa8a814..8b9a945e1aa15a 100644 --- a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h +++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.h @@ -28,8 +28,8 @@ namespace doris { class TypeInfo; -namespace fs { -class WritableBlock; +namespace io { +class FileWriter; } namespace segment_v2 { @@ -50,7 +50,7 @@ class BloomFilterIndexWriter { virtual Status flush() = 0; - virtual Status finish(fs::WritableBlock* wblock, ColumnIndexMetaPB* index_meta) = 0; + virtual Status finish(io::FileWriter* file_writer, ColumnIndexMetaPB* index_meta) = 0; virtual uint64_t size() = 0; diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp index d68ffc66c08033..351433c344b021 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.cpp +++ b/be/src/olap/rowset/segment_v2/column_reader.cpp @@ -39,11 +39,11 @@ namespace segment_v2 { using strings::Substitute; Status ColumnReader::create(const ColumnReaderOptions& opts, const ColumnMetaPB& meta, - uint64_t num_rows, const FilePathDesc& path_desc, + uint64_t num_rows, io::FileSystem* fs, const std::string& path, std::unique_ptr* reader) { if (is_scalar_type((FieldType)meta.type())) { std::unique_ptr reader_local( - new ColumnReader(opts, meta, num_rows, path_desc)); + new ColumnReader(opts, meta, num_rows, fs, path)); RETURN_IF_ERROR(reader_local->init()); *reader = std::move(reader_local); return Status::OK(); @@ -55,25 +55,25 @@ Status ColumnReader::create(const ColumnReaderOptions& opts, const ColumnMetaPB& std::unique_ptr item_reader; RETURN_IF_ERROR(ColumnReader::create(opts, meta.children_columns(0), - meta.children_columns(0).num_rows(), path_desc, + meta.children_columns(0).num_rows(), fs, path, &item_reader)); std::unique_ptr offset_reader; RETURN_IF_ERROR(ColumnReader::create(opts, meta.children_columns(1), - meta.children_columns(1).num_rows(), path_desc, + meta.children_columns(1).num_rows(), fs, path, &offset_reader)); std::unique_ptr null_reader; if (meta.is_nullable()) { RETURN_IF_ERROR(ColumnReader::create(opts, meta.children_columns(2), - meta.children_columns(2).num_rows(), path_desc, + meta.children_columns(2).num_rows(), fs, path, &null_reader)); } // The num rows of the array reader equals to the num rows of the length reader. num_rows = meta.children_columns(1).num_rows(); std::unique_ptr array_reader( - new ColumnReader(opts, meta, num_rows, path_desc)); + new ColumnReader(opts, meta, num_rows, fs, path)); // array reader do not need to init array_reader->_sub_readers.resize(meta.children_columns_size()); array_reader->_sub_readers[0] = std::move(item_reader); @@ -92,8 +92,8 @@ Status ColumnReader::create(const ColumnReaderOptions& opts, const ColumnMetaPB& } ColumnReader::ColumnReader(const ColumnReaderOptions& opts, const ColumnMetaPB& meta, - uint64_t num_rows, FilePathDesc path_desc) - : _meta(meta), _opts(opts), _num_rows(num_rows), _path_desc(path_desc) {} + uint64_t num_rows, io::FileSystem* fs, const std::string& path) + : _meta(meta), _opts(opts), _num_rows(num_rows), _fs(fs), _path(path) {} ColumnReader::~ColumnReader() = default; @@ -121,17 +121,15 @@ Status ColumnReader::init() { _bf_index_meta = &index_meta.bloom_filter_index(); break; default: - return Status::Corruption( - strings::Substitute("Bad file $0: invalid column index type $1", - _path_desc.filepath, index_meta.type())); + return Status::Corruption(strings::Substitute( + "Bad file $0: invalid column index type $1", _path, index_meta.type())); } } // ArrayColumnWriter writes a single empty array and flushes. In this scenario, // the item writer doesn't write any data and the corresponding ordinal index is empty. if (_ordinal_index_meta == nullptr && !is_empty()) { - return Status::Corruption( - strings::Substitute("Bad file $0: missing ordinal index for column $1", - _path_desc.filepath, _meta.column_id())); + return Status::Corruption(strings::Substitute( + "Bad file $0: missing ordinal index for column $1", _path, _meta.column_id())); } return Status::OK(); } @@ -147,7 +145,7 @@ Status ColumnReader::read_page(const ColumnIteratorOptions& iter_opts, const Pag BlockCompressionCodec* codec) { iter_opts.sanity_check(); PageReadOptions opts; - opts.rblock = iter_opts.rblock; + opts.file_reader = iter_opts.file_reader; opts.page_pointer = pp; opts.codec = codec; opts.stats = iter_opts.stats; @@ -296,13 +294,13 @@ Status ColumnReader::get_row_ranges_by_bloom_filter(CondColumn* cond_column, Status ColumnReader::_load_ordinal_index(bool use_page_cache, bool kept_in_memory) { DCHECK(_ordinal_index_meta != nullptr); - _ordinal_index.reset(new OrdinalIndexReader(_path_desc, _ordinal_index_meta, _num_rows)); + _ordinal_index.reset(new OrdinalIndexReader(_fs, _path, _ordinal_index_meta, _num_rows)); return _ordinal_index->load(use_page_cache, kept_in_memory); } Status ColumnReader::_load_zone_map_index(bool use_page_cache, bool kept_in_memory) { if (_zone_map_index_meta != nullptr) { - _zone_map_index.reset(new ZoneMapIndexReader(_path_desc, _zone_map_index_meta)); + _zone_map_index.reset(new ZoneMapIndexReader(_fs, _path, _zone_map_index_meta)); return _zone_map_index->load(use_page_cache, kept_in_memory); } return Status::OK(); @@ -310,7 +308,7 @@ Status ColumnReader::_load_zone_map_index(bool use_page_cache, bool kept_in_memo Status ColumnReader::_load_bitmap_index(bool use_page_cache, bool kept_in_memory) { if (_bitmap_index_meta != nullptr) { - _bitmap_index.reset(new BitmapIndexReader(_path_desc, _bitmap_index_meta)); + _bitmap_index.reset(new BitmapIndexReader(_fs, _path, _bitmap_index_meta)); return _bitmap_index->load(use_page_cache, kept_in_memory); } return Status::OK(); @@ -318,7 +316,7 @@ Status ColumnReader::_load_bitmap_index(bool use_page_cache, bool kept_in_memory Status ColumnReader::_load_bloom_filter_index(bool use_page_cache, bool kept_in_memory) { if (_bf_index_meta != nullptr) { - _bloom_filter_index.reset(new BloomFilterIndexReader(_path_desc, _bf_index_meta)); + _bloom_filter_index.reset(new BloomFilterIndexReader(_fs, _path, _bf_index_meta)); return _bloom_filter_index->load(use_page_cache, kept_in_memory); } return Status::OK(); diff --git a/be/src/olap/rowset/segment_v2/column_reader.h b/be/src/olap/rowset/segment_v2/column_reader.h index 47250a96cb5d15..de3927f2ae7940 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.h +++ b/be/src/olap/rowset/segment_v2/column_reader.h @@ -22,8 +22,9 @@ #include // for unique_ptr #include "common/logging.h" -#include "common/status.h" // for Status -#include "gen_cpp/segment_v2.pb.h" // for ColumnMetaPB +#include "common/status.h" // for Status +#include "gen_cpp/segment_v2.pb.h" // for ColumnMetaPB +#include "io/fs/file_system.h" #include "olap/olap_cond.h" // for CondColumn #include "olap/rowset/segment_v2/bitmap_index_reader.h" // for BitmapIndexReader #include "olap/rowset/segment_v2/common.h" @@ -63,7 +64,7 @@ struct ColumnReaderOptions { }; struct ColumnIteratorOptions { - fs::ReadableBlock* rblock = nullptr; + io::FileReader* file_reader = nullptr; // reader statistics OlapReaderStatistics* stats = nullptr; bool use_page_cache = false; @@ -73,7 +74,7 @@ struct ColumnIteratorOptions { PageTypePB type; void sanity_check() const { - CHECK_NOTNULL(rblock); + CHECK_NOTNULL(file_reader); CHECK_NOTNULL(stats); } }; @@ -87,7 +88,7 @@ class ColumnReader { // Create an initialized ColumnReader in *reader. // This should be a lightweight operation without I/O. static Status create(const ColumnReaderOptions& opts, const ColumnMetaPB& meta, - uint64_t num_rows, const FilePathDesc& path_desc, + uint64_t num_rows, io::FileSystem* fs, const std::string& path, std::unique_ptr* reader); ~ColumnReader(); @@ -136,7 +137,7 @@ class ColumnReader { private: ColumnReader(const ColumnReaderOptions& opts, const ColumnMetaPB& meta, uint64_t num_rows, - FilePathDesc path_desc); + io::FileSystem* fs, const std::string& path); Status init(); // Read and load necessary column indexes into memory if it hasn't been loaded. @@ -172,7 +173,9 @@ class ColumnReader { ColumnMetaPB _meta; ColumnReaderOptions _opts; uint64_t _num_rows; - FilePathDesc _path_desc; + + io::FileSystem* _fs; + std::string _path; TypeInfoPtr _type_info = TypeInfoPtr(nullptr, nullptr); // initialized in init(), may changed by subclasses. diff --git a/be/src/olap/rowset/segment_v2/column_writer.cpp b/be/src/olap/rowset/segment_v2/column_writer.cpp index 6c76ddff62645b..014754ba716ae9 100644 --- a/be/src/olap/rowset/segment_v2/column_writer.cpp +++ b/be/src/olap/rowset/segment_v2/column_writer.cpp @@ -22,7 +22,6 @@ #include "common/logging.h" #include "env/env.h" #include "gutil/strings/substitute.h" -#include "olap/fs/block_manager.h" #include "olap/rowset/segment_v2/bitmap_index_writer.h" #include "olap/rowset/segment_v2/bloom_filter.h" #include "olap/rowset/segment_v2/bloom_filter_index_writer.h" @@ -77,12 +76,12 @@ class NullBitmapBuilder { }; Status ColumnWriter::create(const ColumnWriterOptions& opts, const TabletColumn* column, - fs::WritableBlock* _wblock, std::unique_ptr* writer) { + io::FileWriter* file_writer, std::unique_ptr* writer) { std::unique_ptr field(FieldFactory::create(*column)); DCHECK(field.get() != nullptr); if (is_scalar_type(column->type())) { std::unique_ptr writer_local = std::unique_ptr( - new ScalarColumnWriter(opts, std::move(field), _wblock)); + new ScalarColumnWriter(opts, std::move(field), file_writer)); *writer = std::move(writer_local); return Status::OK(); } else { @@ -107,7 +106,7 @@ Status ColumnWriter::create(const ColumnWriterOptions& opts, const TabletColumn* } std::unique_ptr item_writer; RETURN_IF_ERROR( - ColumnWriter::create(item_options, &item_column, _wblock, &item_writer)); + ColumnWriter::create(item_options, &item_column, file_writer, &item_writer)); // create length writer FieldType length_type = FieldType::OLAP_FIELD_TYPE_UNSIGNED_INT; @@ -134,7 +133,7 @@ Status ColumnWriter::create(const ColumnWriterOptions& opts, const TabletColumn* length_column.set_index_length(-1); // no short key index std::unique_ptr bigint_field(FieldFactory::create(length_column)); auto* length_writer = - new ScalarColumnWriter(length_options, std::move(bigint_field), _wblock); + new ScalarColumnWriter(length_options, std::move(bigint_field), file_writer); // if nullable, create null writer ScalarColumnWriter* null_writer = nullptr; @@ -161,7 +160,8 @@ Status ColumnWriter::create(const ColumnWriterOptions& opts, const TabletColumn* null_column.set_name("nullable"); null_column.set_index_length(-1); // no short key index std::unique_ptr null_field(FieldFactory::create(null_column)); - null_writer = new ScalarColumnWriter(null_options, std::move(null_field), _wblock); + null_writer = + new ScalarColumnWriter(null_options, std::move(null_field), file_writer); } std::unique_ptr writer_local = std::unique_ptr( @@ -237,10 +237,10 @@ Status ColumnWriter::append(const uint8_t* nullmap, const void* data, size_t num /////////////////////////////////////////////////////////////////////////////////// ScalarColumnWriter::ScalarColumnWriter(const ColumnWriterOptions& opts, - std::unique_ptr field, fs::WritableBlock* wblock) + std::unique_ptr field, io::FileWriter* file_writer) : ColumnWriter(std::move(field), opts.meta->is_nullable()), _opts(opts), - _wblock(wblock), + _file_writer(file_writer), _data_size(0) { // these opts.meta fields should be set by client DCHECK(opts.meta->has_column_id()); @@ -250,7 +250,7 @@ ScalarColumnWriter::ScalarColumnWriter(const ColumnWriterOptions& opts, DCHECK(opts.meta->has_encoding()); DCHECK(opts.meta->has_compression()); DCHECK(opts.meta->has_is_nullable()); - DCHECK(wblock != nullptr); + DCHECK(file_writer != nullptr); } ScalarColumnWriter::~ScalarColumnWriter() { @@ -420,35 +420,35 @@ Status ScalarColumnWriter::write_data() { footer.mutable_dict_page_footer()->set_encoding(PLAIN_ENCODING); PagePointer dict_pp; - RETURN_IF_ERROR(PageIO::compress_and_write_page(_compress_codec.get(), - _opts.compression_min_space_saving, _wblock, - {dict_body.slice()}, footer, &dict_pp)); + RETURN_IF_ERROR(PageIO::compress_and_write_page( + _compress_codec.get(), _opts.compression_min_space_saving, _file_writer, + {dict_body.slice()}, footer, &dict_pp)); dict_pp.to_proto(_opts.meta->mutable_dict_page()); } return Status::OK(); } Status ScalarColumnWriter::write_ordinal_index() { - return _ordinal_index_builder->finish(_wblock, _opts.meta->add_indexes()); + return _ordinal_index_builder->finish(_file_writer, _opts.meta->add_indexes()); } Status ScalarColumnWriter::write_zone_map() { if (_opts.need_zone_map) { - return _zone_map_index_builder->finish(_wblock, _opts.meta->add_indexes()); + return _zone_map_index_builder->finish(_file_writer, _opts.meta->add_indexes()); } return Status::OK(); } Status ScalarColumnWriter::write_bitmap_index() { if (_opts.need_bitmap_index) { - return _bitmap_index_builder->finish(_wblock, _opts.meta->add_indexes()); + return _bitmap_index_builder->finish(_file_writer, _opts.meta->add_indexes()); } return Status::OK(); } Status ScalarColumnWriter::write_bloom_filter_index() { if (_opts.need_bloom_filter) { - return _bloom_filter_index_builder->finish(_wblock, _opts.meta->add_indexes()); + return _bloom_filter_index_builder->finish(_file_writer, _opts.meta->add_indexes()); } return Status::OK(); } @@ -460,7 +460,7 @@ Status ScalarColumnWriter::_write_data_page(Page* page) { for (auto& data : page->data) { compressed_body.push_back(data.slice()); } - RETURN_IF_ERROR(PageIO::write_page(_wblock, compressed_body, page->footer, &pp)); + RETURN_IF_ERROR(PageIO::write_page(_file_writer, compressed_body, page->footer, &pp)); _ordinal_index_builder->append_entry(page->footer.data_page_footer().first_ordinal(), pp); return Status::OK(); } diff --git a/be/src/olap/rowset/segment_v2/column_writer.h b/be/src/olap/rowset/segment_v2/column_writer.h index 2f50ebf075744f..0a212aa1039ce2 100644 --- a/be/src/olap/rowset/segment_v2/column_writer.h +++ b/be/src/olap/rowset/segment_v2/column_writer.h @@ -32,8 +32,8 @@ namespace doris { class TypeInfo; class BlockCompressionCodec; -namespace fs { -class WritableBlock; +namespace io { +class FileWriter; } namespace segment_v2 { @@ -50,7 +50,7 @@ struct ColumnWriterOptions { bool need_zone_map = false; bool need_bitmap_index = false; bool need_bloom_filter = false; - std::string to_string() { + std::string to_string() const { std::stringstream ss; ss << std::boolalpha << "meta=" << meta->DebugString() << ", data_page_size=" << data_page_size @@ -72,7 +72,7 @@ class ZoneMapIndexWriter; class ColumnWriter { public: static Status create(const ColumnWriterOptions& opts, const TabletColumn* column, - fs::WritableBlock* _wblock, std::unique_ptr* writer); + io::FileWriter* file_writer, std::unique_ptr* writer); explicit ColumnWriter(std::unique_ptr field, bool is_nullable) : _field(std::move(field)), _is_nullable(is_nullable) {} @@ -164,7 +164,7 @@ class FlushPageCallback { class ScalarColumnWriter final : public ColumnWriter { public: ScalarColumnWriter(const ColumnWriterOptions& opts, std::unique_ptr field, - fs::WritableBlock* output_file); + io::FileWriter* file_writer); ~ScalarColumnWriter() override; @@ -241,7 +241,7 @@ class ScalarColumnWriter final : public ColumnWriter { Status _write_data_page(Page* page); private: - fs::WritableBlock* _wblock = nullptr; + io::FileWriter* _file_writer = nullptr; // total size of data page list uint64_t _data_size; diff --git a/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp b/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp index 6a0b2ca81e212b..3137a588a25363 100644 --- a/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp +++ b/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp @@ -18,6 +18,8 @@ #include "olap/rowset/segment_v2/indexed_column_reader.h" #include "gutil/strings/substitute.h" // for Substitute +#include "io/fs/file_system_map.h" +#include "io/fs/local_file_system.h" #include "olap/key_coder.h" #include "olap/rowset/segment_v2/encoding_info.h" // for EncodingInfo #include "olap/rowset/segment_v2/page_io.h" @@ -39,15 +41,16 @@ Status IndexedColumnReader::load(bool use_page_cache, bool kept_in_memory) { RETURN_IF_ERROR(EncodingInfo::get(_type_info, _meta.encoding(), &_encoding_info)); _value_key_coder = get_key_coder(_type_info->type()); - std::unique_ptr rblock; - fs::BlockManager* block_mgr = fs::fs_util::block_manager(_path_desc); - RETURN_IF_ERROR(block_mgr->open_block(_path_desc, &rblock)); + std::unique_ptr file_reader; + RETURN_IF_ERROR(_fs->open_file(_path, &file_reader)); + // read and parse ordinal index page when exists if (_meta.has_ordinal_index_meta()) { if (_meta.ordinal_index_meta().is_root_data_page()) { _sole_data_page = PagePointer(_meta.ordinal_index_meta().root_page()); } else { - RETURN_IF_ERROR(load_index_page(rblock.get(), _meta.ordinal_index_meta().root_page(), + RETURN_IF_ERROR(load_index_page(file_reader.get(), + _meta.ordinal_index_meta().root_page(), &_ordinal_index_page_handle, &_ordinal_index_reader)); _has_index_page = true; } @@ -58,7 +61,7 @@ Status IndexedColumnReader::load(bool use_page_cache, bool kept_in_memory) { if (_meta.value_index_meta().is_root_data_page()) { _sole_data_page = PagePointer(_meta.value_index_meta().root_page()); } else { - RETURN_IF_ERROR(load_index_page(rblock.get(), _meta.value_index_meta().root_page(), + RETURN_IF_ERROR(load_index_page(file_reader.get(), _meta.value_index_meta().root_page(), &_value_index_page_handle, &_value_index_reader)); _has_index_page = true; } @@ -67,23 +70,23 @@ Status IndexedColumnReader::load(bool use_page_cache, bool kept_in_memory) { return Status::OK(); } -Status IndexedColumnReader::load_index_page(fs::ReadableBlock* rblock, const PagePointerPB& pp, +Status IndexedColumnReader::load_index_page(io::FileReader* file_reader, const PagePointerPB& pp, PageHandle* handle, IndexPageReader* reader) { Slice body; PageFooterPB footer; std::unique_ptr local_compress_codec; RETURN_IF_ERROR(get_block_compression_codec(_meta.compression(), local_compress_codec)); - RETURN_IF_ERROR(read_page(rblock, PagePointer(pp), handle, &body, &footer, INDEX_PAGE, + RETURN_IF_ERROR(read_page(file_reader, PagePointer(pp), handle, &body, &footer, INDEX_PAGE, local_compress_codec.get())); RETURN_IF_ERROR(reader->parse(body, footer.index_page_footer())); return Status::OK(); } -Status IndexedColumnReader::read_page(fs::ReadableBlock* rblock, const PagePointer& pp, +Status IndexedColumnReader::read_page(io::FileReader* file_reader, const PagePointer& pp, PageHandle* handle, Slice* body, PageFooterPB* footer, PageTypePB type, BlockCompressionCodec* codec) const { PageReadOptions opts; - opts.rblock = rblock; + opts.file_reader = file_reader; opts.page_pointer = pp; opts.codec = codec; OlapReaderStatistics tmp_stats; @@ -100,13 +103,14 @@ Status IndexedColumnReader::read_page(fs::ReadableBlock* rblock, const PagePoint Status IndexedColumnIterator::_read_data_page(const PagePointer& pp) { // there is not init() for IndexedColumnIterator, so do it here - if (!_compress_codec.get()) + if (!_compress_codec) { RETURN_IF_ERROR(get_block_compression_codec(_reader->get_compression(), _compress_codec)); + } PageHandle handle; Slice body; PageFooterPB footer; - RETURN_IF_ERROR(_reader->read_page(_rblock.get(), pp, &handle, &body, &footer, DATA_PAGE, + RETURN_IF_ERROR(_reader->read_page(_file_reader.get(), pp, &handle, &body, &footer, DATA_PAGE, _compress_codec.get())); // parse data page // note that page_index is not used in IndexedColumnIterator, so we pass 0 diff --git a/be/src/olap/rowset/segment_v2/indexed_column_reader.h b/be/src/olap/rowset/segment_v2/indexed_column_reader.h index ae86b064a6769a..8544cf970b26d5 100644 --- a/be/src/olap/rowset/segment_v2/indexed_column_reader.h +++ b/be/src/olap/rowset/segment_v2/indexed_column_reader.h @@ -22,6 +22,9 @@ #include "common/status.h" #include "env/env.h" #include "gen_cpp/segment_v2.pb.h" +#include "io/fs/file_reader.h" +#include "io/fs/file_system.h" +#include "io/fs/file_system_map.h" #include "olap/column_block.h" #include "olap/fs/fs_util.h" #include "olap/rowset/segment_v2/common.h" @@ -45,13 +48,14 @@ class IndexedColumnIterator; // thread-safe reader for IndexedColumn (see comments of `IndexedColumnWriter` to understand what IndexedColumn is) class IndexedColumnReader { public: - explicit IndexedColumnReader(const FilePathDesc& path_desc, const IndexedColumnMetaPB& meta) - : _path_desc(path_desc), _meta(meta) {}; + explicit IndexedColumnReader(io::FileSystem* fs, const std::string& path, + const IndexedColumnMetaPB& meta) + : _fs(fs), _path(path), _meta(meta) {}; Status load(bool use_page_cache, bool kept_in_memory); // read a page specified by `pp' from `file' into `handle' - Status read_page(fs::ReadableBlock* rblock, const PagePointer& pp, PageHandle* handle, + Status read_page(io::FileReader* file_reader, const PagePointer& pp, PageHandle* handle, Slice* body, PageFooterPB* footer, PageTypePB type, BlockCompressionCodec* codec) const; @@ -64,12 +68,13 @@ class IndexedColumnReader { CompressionTypePB get_compression() const { return _meta.compression(); } private: - Status load_index_page(fs::ReadableBlock* rblock, const PagePointerPB& pp, PageHandle* handle, + Status load_index_page(io::FileReader* file_reader, const PagePointerPB& pp, PageHandle* handle, IndexPageReader* reader); friend class IndexedColumnIterator; - FilePathDesc _path_desc; + io::FileSystem* _fs; + std::string _path; IndexedColumnMetaPB _meta; bool _use_page_cache; @@ -96,10 +101,11 @@ class IndexedColumnIterator { : _reader(reader), _ordinal_iter(&reader->_ordinal_index_reader), _value_iter(&reader->_value_index_reader) { - fs::BlockManager* block_manager = fs::fs_util::block_manager(_reader->_path_desc); - auto st = block_manager->open_block(_reader->_path_desc, &_rblock); + io::FileSystem* fs = _reader->_fs; + auto st = fs->open_file(_reader->_path, &_file_reader); + DCHECK(st.ok()); - WARN_IF_ERROR(st, "open file failed:" + _reader->_path_desc.filepath); + WARN_IF_ERROR(st, "open file failed:" + _reader->_path); } // Seek to the given ordinal entry. Entry 0 is the first entry. @@ -146,7 +152,7 @@ class IndexedColumnIterator { // next_batch() will read from this position ordinal_t _current_ordinal = 0; // open file handle - std::unique_ptr _rblock; + std::unique_ptr _file_reader; // iterator owned compress codec, should NOT be shared by threads, initialized before used std::unique_ptr _compress_codec; }; diff --git a/be/src/olap/rowset/segment_v2/indexed_column_writer.cpp b/be/src/olap/rowset/segment_v2/indexed_column_writer.cpp index 4c7259c90c6d44..b6e34302533906 100644 --- a/be/src/olap/rowset/segment_v2/indexed_column_writer.cpp +++ b/be/src/olap/rowset/segment_v2/indexed_column_writer.cpp @@ -21,7 +21,6 @@ #include "common/logging.h" #include "env/env.h" -#include "olap/fs/block_manager.h" #include "olap/key_coder.h" #include "olap/rowset/segment_v2/encoding_info.h" #include "olap/rowset/segment_v2/index_page.h" @@ -37,10 +36,10 @@ namespace doris { namespace segment_v2 { IndexedColumnWriter::IndexedColumnWriter(const IndexedColumnWriterOptions& options, - const TypeInfo* type_info, fs::WritableBlock* wblock) + const TypeInfo* type_info, io::FileWriter* file_writer) : _options(options), _type_info(type_info), - _wblock(wblock), + _file_writer(file_writer), _mem_pool("IndexedColumnWriter"), _num_values(0), _num_data_pages(0), @@ -110,9 +109,9 @@ Status IndexedColumnWriter::_finish_current_data_page() { footer.mutable_data_page_footer()->set_num_values(num_values_in_page); footer.mutable_data_page_footer()->set_nullmap_size(0); - RETURN_IF_ERROR(PageIO::compress_and_write_page(_compress_codec.get(), - _options.compression_min_space_saving, _wblock, - {page_body.slice()}, footer, &_last_data_page)); + RETURN_IF_ERROR(PageIO::compress_and_write_page( + _compress_codec.get(), _options.compression_min_space_saving, _file_writer, + {page_body.slice()}, footer, &_last_data_page)); _num_data_pages++; if (_options.write_ordinal_index) { @@ -159,7 +158,7 @@ Status IndexedColumnWriter::_flush_index(IndexPageBuilder* index_builder, BTreeM PagePointer pp; RETURN_IF_ERROR(PageIO::compress_and_write_page( - _compress_codec.get(), _options.compression_min_space_saving, _wblock, + _compress_codec.get(), _options.compression_min_space_saving, _file_writer, {page_body.slice()}, page_footer, &pp)); meta->set_is_root_data_page(false); diff --git a/be/src/olap/rowset/segment_v2/indexed_column_writer.h b/be/src/olap/rowset/segment_v2/indexed_column_writer.h index 285ba890b23d2e..8be96d7b351314 100644 --- a/be/src/olap/rowset/segment_v2/indexed_column_writer.h +++ b/be/src/olap/rowset/segment_v2/indexed_column_writer.h @@ -35,8 +35,8 @@ class BlockCompressionCodec; class KeyCoder; class TypeInfo; -namespace fs { -class WritableBlock; +namespace io { +class FileWriter; } namespace segment_v2 { @@ -70,7 +70,7 @@ struct IndexedColumnWriterOptions { class IndexedColumnWriter { public: explicit IndexedColumnWriter(const IndexedColumnWriterOptions& options, - const TypeInfo* type_info, fs::WritableBlock* wblock); + const TypeInfo* type_info, io::FileWriter* file_writer); ~IndexedColumnWriter(); @@ -88,7 +88,7 @@ class IndexedColumnWriter { IndexedColumnWriterOptions _options; const TypeInfo* _type_info; - fs::WritableBlock* _wblock; + io::FileWriter* _file_writer; // only used for `_first_value` MemPool _mem_pool; diff --git a/be/src/olap/rowset/segment_v2/ordinal_page_index.cpp b/be/src/olap/rowset/segment_v2/ordinal_page_index.cpp index b00e4db8cc594e..ca3fe9cbf4425a 100644 --- a/be/src/olap/rowset/segment_v2/ordinal_page_index.cpp +++ b/be/src/olap/rowset/segment_v2/ordinal_page_index.cpp @@ -19,6 +19,9 @@ #include "common/logging.h" #include "env/env.h" +#include "io/fs/file_system_map.h" +#include "io/fs/file_writer.h" +#include "io/fs/local_file_system.h" #include "olap/fs/fs_util.h" #include "olap/key_coder.h" #include "olap/rowset/segment_v2/page_handle.h" @@ -34,9 +37,9 @@ void OrdinalIndexWriter::append_entry(ordinal_t ordinal, const PagePointer& data _last_pp = data_pp; } -Status OrdinalIndexWriter::finish(fs::WritableBlock* wblock, ColumnIndexMetaPB* meta) { +Status OrdinalIndexWriter::finish(io::FileWriter* file_writer, ColumnIndexMetaPB* meta) { CHECK(_page_builder->count() > 0) - << "no entry has been added, filepath=" << wblock->path_desc().filepath; + << "no entry has been added, filepath=" << file_writer->path(); meta->set_type(ORDINAL_INDEX); BTreeMetaPB* root_page_meta = meta->mutable_ordinal_index()->mutable_root_page(); @@ -51,7 +54,7 @@ Status OrdinalIndexWriter::finish(fs::WritableBlock* wblock, ColumnIndexMetaPB* // write index page (currently it's not compressed) PagePointer pp; - RETURN_IF_ERROR(PageIO::write_page(wblock, {page_body.slice()}, page_footer, &pp)); + RETURN_IF_ERROR(PageIO::write_page(file_writer, {page_body.slice()}, page_footer, &pp)); root_page_meta->set_is_root_data_page(false); pp.to_proto(root_page_meta->mutable_root_page()); @@ -69,12 +72,11 @@ Status OrdinalIndexReader::load(bool use_page_cache, bool kept_in_memory) { return Status::OK(); } // need to read index page - std::unique_ptr rblock; - fs::BlockManager* block_mgr = fs::fs_util::block_manager(_path_desc); - RETURN_IF_ERROR(block_mgr->open_block(_path_desc, &rblock)); + std::unique_ptr file_reader; + RETURN_IF_ERROR(_fs->open_file(_path, &file_reader)); PageReadOptions opts; - opts.rblock = rblock.get(); + opts.file_reader = file_reader.get(); opts.page_pointer = PagePointer(_index_meta->root_page().root_page()); opts.codec = nullptr; // ordinal index page uses NO_COMPRESSION right now OlapReaderStatistics tmp_stats; diff --git a/be/src/olap/rowset/segment_v2/ordinal_page_index.h b/be/src/olap/rowset/segment_v2/ordinal_page_index.h index 9d558fa102fe94..81419a08d61411 100644 --- a/be/src/olap/rowset/segment_v2/ordinal_page_index.h +++ b/be/src/olap/rowset/segment_v2/ordinal_page_index.h @@ -24,6 +24,7 @@ #include "common/status.h" #include "env/env.h" #include "gutil/macros.h" +#include "io/fs/file_system.h" #include "olap/rowset/segment_v2/common.h" #include "olap/rowset/segment_v2/index_page.h" #include "olap/rowset/segment_v2/page_pointer.h" @@ -32,8 +33,8 @@ namespace doris { -namespace fs { -class WritableBlock; +namespace io { +class FileWriter; } namespace segment_v2 { @@ -50,7 +51,7 @@ class OrdinalIndexWriter { uint64_t size() { return _page_builder->size(); } - Status finish(fs::WritableBlock* wblock, ColumnIndexMetaPB* meta); + Status finish(io::FileWriter* file_writer, ColumnIndexMetaPB* meta); private: DISALLOW_COPY_AND_ASSIGN(OrdinalIndexWriter); @@ -62,9 +63,9 @@ class OrdinalPageIndexIterator; class OrdinalIndexReader { public: - explicit OrdinalIndexReader(const FilePathDesc& path_desc, const OrdinalIndexPB* index_meta, - ordinal_t num_values) - : _path_desc(path_desc), _index_meta(index_meta), _num_values(num_values) {} + explicit OrdinalIndexReader(io::FileSystem* fs, const std::string& path, + const OrdinalIndexPB* index_meta, ordinal_t num_values) + : _fs(fs), _path(path), _index_meta(index_meta), _num_values(num_values) {} // load and parse the index page into memory Status load(bool use_page_cache, bool kept_in_memory); @@ -87,7 +88,8 @@ class OrdinalIndexReader { private: friend OrdinalPageIndexIterator; - FilePathDesc _path_desc; + io::FileSystem* _fs; + std::string _path; const OrdinalIndexPB* _index_meta; // total number of values (including NULLs) in the indexed column, // equals to 1 + 'last ordinal of last data pages' diff --git a/be/src/olap/rowset/segment_v2/page_io.cpp b/be/src/olap/rowset/segment_v2/page_io.cpp index 1c649aa44c3864..ef8cacd645b66d 100644 --- a/be/src/olap/rowset/segment_v2/page_io.cpp +++ b/be/src/olap/rowset/segment_v2/page_io.cpp @@ -21,9 +21,8 @@ #include #include "common/logging.h" -#include "env/env.h" #include "gutil/strings/substitute.h" -#include "olap/fs/block_manager.h" +#include "io/fs/file_writer.h" #include "olap/page_cache.h" #include "util/block_compression.h" #include "util/coding.h" @@ -65,7 +64,7 @@ Status PageIO::compress_page_body(const BlockCompressionCodec* codec, double min return Status::OK(); } -Status PageIO::write_page(fs::WritableBlock* wblock, const std::vector& body, +Status PageIO::write_page(io::FileWriter* writer, const std::vector& body, const PageFooterPB& footer, PagePointer* result) { // sanity check of page footer CHECK(footer.has_type()) << "type must be set"; @@ -101,11 +100,11 @@ Status PageIO::write_page(fs::WritableBlock* wblock, const std::vector& b encode_fixed32_le(checksum_buf, checksum); page.emplace_back(checksum_buf, sizeof(uint32_t)); - uint64_t offset = wblock->bytes_appended(); - RETURN_IF_ERROR(wblock->appendv(&page[0], page.size())); + uint64_t offset = writer->bytes_appended(); + RETURN_IF_ERROR(writer->appendv(&page[0], page.size())); result->offset = offset; - result->size = wblock->bytes_appended() - offset; + result->size = writer->bytes_appended() - offset; return Status::OK(); } @@ -116,7 +115,7 @@ Status PageIO::read_and_decompress_page(const PageReadOptions& opts, PageHandle* auto cache = StoragePageCache::instance(); PageCacheHandle cache_handle; - StoragePageCache::CacheKey cache_key(opts.rblock->path_desc().filepath, + StoragePageCache::CacheKey cache_key(opts.file_reader->path().native(), opts.page_pointer.offset); if (opts.use_page_cache && cache->is_cache_available(opts.type) && cache->lookup(cache_key, &cache_handle, opts.type)) { @@ -145,7 +144,10 @@ Status PageIO::read_and_decompress_page(const PageReadOptions& opts, PageHandle* Slice page_slice(page.get(), page_size); { SCOPED_RAW_TIMER(&opts.stats->io_ns); - RETURN_IF_ERROR(opts.rblock->read(opts.page_pointer.offset, page_slice)); + size_t bytes_read = 0; + RETURN_IF_ERROR( + opts.file_reader->read_at(opts.page_pointer.offset, page_slice, &bytes_read)); + DCHECK_EQ(bytes_read, page_size); opts.stats->compressed_bytes_read += page_size; } diff --git a/be/src/olap/rowset/segment_v2/page_io.h b/be/src/olap/rowset/segment_v2/page_io.h index ee7d457afb3be4..65f2c513887271 100644 --- a/be/src/olap/rowset/segment_v2/page_io.h +++ b/be/src/olap/rowset/segment_v2/page_io.h @@ -22,6 +22,7 @@ #include "common/logging.h" #include "common/status.h" #include "gen_cpp/segment_v2.pb.h" +#include "io/fs/file_reader.h" #include "olap/rowset/segment_v2/encoding_info.h" #include "olap/rowset/segment_v2/page_handle.h" #include "olap/rowset/segment_v2/page_pointer.h" @@ -37,11 +38,15 @@ class ReadableBlock; class WritableBlock; } // namespace fs +namespace io { +class FileWriter; +} // namespace io + namespace segment_v2 { struct PageReadOptions { // block to read page - fs::ReadableBlock* rblock = nullptr; + doris::io::FileReader* file_reader = nullptr; // location of the page PagePointer page_pointer; // decompressor for page body (null means page body is not compressed) @@ -63,7 +68,7 @@ struct PageReadOptions { const EncodingInfo* encoding_info = nullptr; void sanity_check() const { - CHECK_NOTNULL(rblock); + CHECK_NOTNULL(file_reader); CHECK_NOTNULL(stats); } }; @@ -87,21 +92,21 @@ class PageIO { // Encode page from `body' and `footer' and write to `file'. // `body' could be either uncompressed or compressed. // On success, the file pointer to the written page is stored in `result'. - static Status write_page(fs::WritableBlock* wblock, const std::vector& body, + static Status write_page(io::FileWriter* writer, const std::vector& body, const PageFooterPB& footer, PagePointer* result); // Convenient function to compress page body and write page in one go. static Status compress_and_write_page(const BlockCompressionCodec* codec, - double min_space_saving, fs::WritableBlock* wblock, + double min_space_saving, io::FileWriter* writer, const std::vector& body, const PageFooterPB& footer, PagePointer* result) { DCHECK_EQ(footer.uncompressed_size(), Slice::compute_total_size(body)); OwnedSlice compressed_body; RETURN_IF_ERROR(compress_page_body(codec, min_space_saving, body, &compressed_body)); if (compressed_body.slice().empty()) { // uncompressed - return write_page(wblock, body, footer, result); + return write_page(writer, body, footer, result); } - return write_page(wblock, {compressed_body.slice()}, footer, result); + return write_page(writer, {compressed_body.slice()}, footer, result); } // Read and parse a page according to `opts'. diff --git a/be/src/olap/rowset/segment_v2/segment.cpp b/be/src/olap/rowset/segment_v2/segment.cpp index 7970d0da5b261a..7faed426aed482 100644 --- a/be/src/olap/rowset/segment_v2/segment.cpp +++ b/be/src/olap/rowset/segment_v2/segment.cpp @@ -37,19 +37,17 @@ namespace segment_v2 { using strings::Substitute; -Status Segment::open(const FilePathDesc& path_desc, uint32_t segment_id, +Status Segment::open(io::FileSystem* fs, const std::string& path, uint32_t segment_id, const TabletSchema* tablet_schema, std::shared_ptr* output) { - std::shared_ptr segment(new Segment(path_desc, segment_id, tablet_schema)); - if (!path_desc.is_remote()) { - RETURN_IF_ERROR(segment->_open()); - } + std::shared_ptr segment(new Segment(fs, path, segment_id, tablet_schema)); + RETURN_IF_ERROR(segment->_open()); output->swap(segment); return Status::OK(); } -Segment::Segment(const FilePathDesc& path_desc, uint32_t segment_id, +Segment::Segment(io::FileSystem* fs, const std::string& path, uint32_t segment_id, const TabletSchema* tablet_schema) - : _path_desc(path_desc), _segment_id(segment_id), _tablet_schema(tablet_schema) { + : _fs(fs), _path(path), _segment_id(segment_id), _tablet_schema(tablet_schema) { #ifndef BE_TEST _mem_tracker = StorageEngine::instance()->tablet_mem_tracker(); #else @@ -99,53 +97,52 @@ Status Segment::new_iterator(const Schema& schema, const StorageReadOptions& rea Status Segment::_parse_footer() { // Footer := SegmentFooterPB, FooterPBSize(4), FooterPBChecksum(4), MagicNumber(4) - std::unique_ptr rblock; - fs::BlockManager* block_mgr = fs::fs_util::block_manager(_path_desc); - RETURN_IF_ERROR(block_mgr->open_block(_path_desc, &rblock)); - - uint64_t file_size; - RETURN_IF_ERROR(rblock->size(&file_size)); + std::unique_ptr file_reader; + RETURN_IF_ERROR(_fs->open_file(_path, &file_reader)); + auto file_size = file_reader->size(); if (file_size < 12) { - return Status::Corruption(strings::Substitute("Bad segment file $0: file size $1 < 12", - _path_desc.filepath, file_size)); + return Status::Corruption( + strings::Substitute("Bad segment file $0: file size $1 < 12", _path, file_size)); } uint8_t fixed_buf[12]; - RETURN_IF_ERROR(rblock->read(file_size - 12, Slice(fixed_buf, 12))); + size_t bytes_read = 0; + RETURN_IF_ERROR(file_reader->read_at(file_size - 12, Slice(fixed_buf, 12), &bytes_read)); + DCHECK_EQ(bytes_read, 12); // validate magic number if (memcmp(fixed_buf + 8, k_segment_magic, k_segment_magic_length) != 0) { - return Status::Corruption(strings::Substitute("Bad segment file $0: magic number not match", - _path_desc.filepath)); + return Status::Corruption( + strings::Substitute("Bad segment file $0: magic number not match", _path)); } // read footer PB uint32_t footer_length = decode_fixed32_le(fixed_buf); if (file_size < 12 + footer_length) { return Status::Corruption(strings::Substitute("Bad segment file $0: file size $1 < $2", - _path_desc.filepath, file_size, - 12 + footer_length)); + _path, file_size, 12 + footer_length)); } _mem_tracker->consume(footer_length); std::string footer_buf; footer_buf.resize(footer_length); - RETURN_IF_ERROR(rblock->read(file_size - 12 - footer_length, footer_buf)); + RETURN_IF_ERROR(file_reader->read_at(file_size - 12 - footer_length, footer_buf, &bytes_read)); + DCHECK_EQ(bytes_read, footer_length); // validate footer PB's checksum uint32_t expect_checksum = decode_fixed32_le(fixed_buf + 4); uint32_t actual_checksum = crc32c::Value(footer_buf.data(), footer_buf.size()); if (actual_checksum != expect_checksum) { return Status::Corruption(strings::Substitute( - "Bad segment file $0: footer checksum not match, actual=$1 vs expect=$2", - _path_desc.filepath, actual_checksum, expect_checksum)); + "Bad segment file $0: footer checksum not match, actual=$1 vs expect=$2", _path, + actual_checksum, expect_checksum)); } // deserialize footer PB if (!_footer.ParseFromString(footer_buf)) { - return Status::Corruption(strings::Substitute( - "Bad segment file $0: failed to parse SegmentFooterPB", _path_desc.filepath)); + return Status::Corruption( + strings::Substitute("Bad segment file $0: failed to parse SegmentFooterPB", _path)); } return Status::OK(); } @@ -153,12 +150,12 @@ Status Segment::_parse_footer() { Status Segment::_load_index() { return _load_index_once.call([this] { // read and parse short key index page - std::unique_ptr rblock; - fs::BlockManager* block_mgr = fs::fs_util::block_manager(_path_desc); - RETURN_IF_ERROR(block_mgr->open_block(_path_desc, &rblock)); + + std::unique_ptr file_reader; + RETURN_IF_ERROR(_fs->open_file(_path, &file_reader)); PageReadOptions opts; - opts.rblock = rblock.get(); + opts.file_reader = file_reader.get(); opts.page_pointer = PagePointer(_footer.short_key_index_page()); opts.codec = nullptr; // short key index page uses NO_COMPRESSION for now OlapReaderStatistics tmp_stats; @@ -195,7 +192,7 @@ Status Segment::_create_column_readers() { opts.kept_in_memory = _tablet_schema->is_in_memory(); std::unique_ptr reader; RETURN_IF_ERROR(ColumnReader::create(opts, _footer.columns(iter->second), - _footer.num_rows(), _path_desc, &reader)); + _footer.num_rows(), _fs, _path, &reader)); _column_readers[ordinal] = std::move(reader); } return Status::OK(); diff --git a/be/src/olap/rowset/segment_v2/segment.h b/be/src/olap/rowset/segment_v2/segment.h index a6fc9fe7feded6..11925e35e95854 100644 --- a/be/src/olap/rowset/segment_v2/segment.h +++ b/be/src/olap/rowset/segment_v2/segment.h @@ -25,6 +25,7 @@ #include "common/status.h" // Status #include "gen_cpp/segment_v2.pb.h" #include "gutil/macros.h" +#include "io/fs/file_system.h" #include "olap/iterators.h" #include "olap/rowset/segment_v2/page_handle.h" #include "olap/short_key_index.h" @@ -59,7 +60,7 @@ using SegmentSharedPtr = std::shared_ptr; // change finished, client should disable all cached Segment for old TabletSchema. class Segment : public std::enable_shared_from_this { public: - static Status open(const FilePathDesc& path_desc, uint32_t segment_id, + static Status open(io::FileSystem* fs, const std::string& path, uint32_t segment_id, const TabletSchema* tablet_schema, std::shared_ptr* output); ~Segment(); @@ -104,7 +105,8 @@ class Segment : public std::enable_shared_from_this { private: DISALLOW_COPY_AND_ASSIGN(Segment); - Segment(const FilePathDesc& path_desc, uint32_t segment_id, const TabletSchema* tablet_schema); + Segment(io::FileSystem* fs, const std::string& path, uint32_t segment_id, + const TabletSchema* tablet_schema); // open segment file and read the minimum amount of necessary information (footer) Status _open(); Status _parse_footer(); @@ -115,7 +117,9 @@ class Segment : public std::enable_shared_from_this { private: friend class SegmentIterator; - FilePathDesc _path_desc; + io::FileSystem* _fs; + std::string _path; + uint32_t _segment_id; const TabletSchema* _tablet_schema; diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index 6d3bfcd5ecb6dd..7d6de12240a572 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -137,8 +137,9 @@ Status SegmentIterator::init(const StorageReadOptions& opts) { Status SegmentIterator::_init(bool is_vec) { DorisMetrics::instance()->segment_read_total->increment(1); // get file handle from file descriptor of segment - fs::BlockManager* block_mgr = fs::fs_util::block_manager(_segment->_path_desc); - RETURN_IF_ERROR(block_mgr->open_block(_segment->_path_desc, &_rblock)); + auto fs = _segment->_fs; + RETURN_IF_ERROR(fs->open_file(_segment->_path, &_file_reader)); + _row_bitmap.addRange(0, _segment->num_rows()); RETURN_IF_ERROR(_init_return_column_iterators()); RETURN_IF_ERROR(_init_bitmap_index_iterators()); @@ -220,7 +221,7 @@ Status SegmentIterator::_prepare_seek(const StorageReadOptions::KeyRange& key_ra RETURN_IF_ERROR(_segment->new_column_iterator(cid, &_column_iterators[cid])); ColumnIteratorOptions iter_opts; iter_opts.stats = _opts.stats; - iter_opts.rblock = _rblock.get(); + iter_opts.file_reader = _file_reader.get(); RETURN_IF_ERROR(_column_iterators[cid]->init(iter_opts)); } } @@ -349,7 +350,7 @@ Status SegmentIterator::_init_return_column_iterators() { ColumnIteratorOptions iter_opts; iter_opts.stats = _opts.stats; iter_opts.use_page_cache = _opts.use_page_cache; - iter_opts.rblock = _rblock.get(); + iter_opts.file_reader = _file_reader.get(); RETURN_IF_ERROR(_column_iterators[cid]->init(iter_opts)); } } diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h b/be/src/olap/rowset/segment_v2/segment_iterator.h index 0ce9975456abb9..b088dbd431a4f3 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.h +++ b/be/src/olap/rowset/segment_v2/segment_iterator.h @@ -22,6 +22,7 @@ #include #include "common/status.h" +#include "io/fs/file_reader.h" #include "olap/olap_common.h" #include "olap/olap_cond.h" #include "olap/rowset/segment_v2/common.h" @@ -180,7 +181,7 @@ class SegmentIterator : public RowwiseIterator { std::unique_ptr _seek_block; // block for file to read - std::unique_ptr _rblock; + std::unique_ptr _file_reader; // char_type columns cid std::vector _char_type_idx; diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp index f95b62c9c24693..d9f3fa3c5db825 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.cpp +++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp @@ -19,8 +19,8 @@ #include "common/logging.h" // LOG #include "env/env.h" // Env +#include "io/fs/file_writer.h" #include "olap/data_dir.h" -#include "olap/fs/block_manager.h" #include "olap/row.h" // ContiguousRow #include "olap/row_cursor.h" // RowCursor #include "olap/rowset/segment_v2/column_writer.h" // ColumnWriter @@ -37,7 +37,7 @@ namespace segment_v2 { const char* k_segment_magic = "D0R1"; const uint32_t k_segment_magic_length = 4; -SegmentWriter::SegmentWriter(fs::WritableBlock* wblock, uint32_t segment_id, +SegmentWriter::SegmentWriter(io::FileWriter* file_writer, uint32_t segment_id, const TabletSchema* tablet_schema, DataDir* data_dir, uint32_t max_row_per_segment, const SegmentWriterOptions& opts) : _segment_id(segment_id), @@ -45,11 +45,11 @@ SegmentWriter::SegmentWriter(fs::WritableBlock* wblock, uint32_t segment_id, _data_dir(data_dir), _max_row_per_segment(max_row_per_segment), _opts(opts), - _wblock(wblock), + _file_writer(file_writer), _mem_tracker(MemTracker::create_virtual_tracker( -1, "SegmentWriter:Segment-" + std::to_string(segment_id))), _olap_data_convertor(tablet_schema) { - CHECK_NOTNULL(_wblock); + CHECK_NOTNULL(file_writer); size_t num_short_key_column = _tablet_schema->num_short_key_columns(); for (size_t cid = 0; cid < num_short_key_column; ++cid) { const auto& column = _tablet_schema->column(cid); @@ -104,7 +104,7 @@ Status SegmentWriter::init(uint32_t write_mbytes_per_sec __attribute__((unused)) } std::unique_ptr writer; - RETURN_IF_ERROR(ColumnWriter::create(opts, &column, _wblock, &writer)); + RETURN_IF_ERROR(ColumnWriter::create(opts, &column, _file_writer, &writer)); RETURN_IF_ERROR(writer->init()); _column_writers.push_back(std::move(writer)); } @@ -243,16 +243,16 @@ Status SegmentWriter::finalize(uint64_t* segment_file_size, uint64_t* index_size RETURN_IF_ERROR(column_writer->finish()); } RETURN_IF_ERROR(_write_data()); - uint64_t index_offset = _wblock->bytes_appended(); + uint64_t index_offset = _file_writer->bytes_appended(); RETURN_IF_ERROR(_write_ordinal_index()); RETURN_IF_ERROR(_write_zone_map()); RETURN_IF_ERROR(_write_bitmap_index()); RETURN_IF_ERROR(_write_bloom_filter_index()); RETURN_IF_ERROR(_write_short_key_index()); - *index_size = _wblock->bytes_appended() - index_offset; + *index_size = _file_writer->bytes_appended() - index_offset; RETURN_IF_ERROR(_write_footer()); - RETURN_IF_ERROR(_wblock->finalize()); - *segment_file_size = _wblock->bytes_appended(); + RETURN_IF_ERROR(_file_writer->finalize()); + *segment_file_size = _file_writer->bytes_appended(); return Status::OK(); } @@ -299,7 +299,7 @@ Status SegmentWriter::_write_short_key_index() { RETURN_IF_ERROR(_index_builder->finalize(_row_count, &body, &footer)); PagePointer pp; // short key index page is not compressed right now - RETURN_IF_ERROR(PageIO::write_page(_wblock, body, footer, &pp)); + RETURN_IF_ERROR(PageIO::write_page(_file_writer, body, footer, &pp)); pp.to_proto(_footer.mutable_short_key_index_page()); return Status::OK(); } @@ -328,7 +328,7 @@ Status SegmentWriter::_write_footer() { } Status SegmentWriter::_write_raw_data(const std::vector& slices) { - RETURN_IF_ERROR(_wblock->appendv(&slices[0], slices.size())); + RETURN_IF_ERROR(_file_writer->appendv(&slices[0], slices.size())); return Status::OK(); } diff --git a/be/src/olap/rowset/segment_v2/segment_writer.h b/be/src/olap/rowset/segment_v2/segment_writer.h index 67fb9e02a2c854..1c14b9909df326 100644 --- a/be/src/olap/rowset/segment_v2/segment_writer.h +++ b/be/src/olap/rowset/segment_v2/segment_writer.h @@ -42,9 +42,9 @@ class TabletColumn; class ShortKeyIndexBuilder; class KeyCoder; -namespace fs { -class WritableBlock; -} +namespace io { +class FileWriter; +} // namespace io namespace segment_v2 { @@ -59,7 +59,7 @@ struct SegmentWriterOptions { class SegmentWriter { public: - explicit SegmentWriter(fs::WritableBlock* block, uint32_t segment_id, + explicit SegmentWriter(io::FileWriter* file_writer, uint32_t segment_id, const TabletSchema* tablet_schema, DataDir* data_dir, uint32_t max_row_per_segment, const SegmentWriterOptions& opts); ~SegmentWriter(); @@ -75,7 +75,7 @@ class SegmentWriter { uint64_t estimate_segment_size(); - uint32_t num_rows_written() { return _row_count; } + uint32_t num_rows_written() const { return _row_count; } Status finalize(uint64_t* segment_file_size, uint64_t* index_size); @@ -104,7 +104,7 @@ class SegmentWriter { SegmentWriterOptions _opts; // Not owned. owned by RowsetWriter - fs::WritableBlock* _wblock; + io::FileWriter* _file_writer; SegmentFooterPB _footer; std::unique_ptr _index_builder; diff --git a/be/src/olap/rowset/segment_v2/zone_map_index.cpp b/be/src/olap/rowset/segment_v2/zone_map_index.cpp index 0fbe5a1f033590..1cb14988cf6940 100644 --- a/be/src/olap/rowset/segment_v2/zone_map_index.cpp +++ b/be/src/olap/rowset/segment_v2/zone_map_index.cpp @@ -18,7 +18,6 @@ #include "olap/rowset/segment_v2/zone_map_index.h" #include "olap/column_block.h" -#include "olap/fs/block_manager.h" #include "olap/olap_define.h" #include "olap/rowset/segment_v2/encoding_info.h" #include "olap/rowset/segment_v2/indexed_column_reader.h" @@ -97,7 +96,7 @@ Status ZoneMapIndexWriter::flush() { return Status::OK(); } -Status ZoneMapIndexWriter::finish(fs::WritableBlock* wblock, ColumnIndexMetaPB* index_meta) { +Status ZoneMapIndexWriter::finish(io::FileWriter* file_writer, ColumnIndexMetaPB* index_meta) { index_meta->set_type(ZONE_MAP_INDEX); ZoneMapIndexPB* meta = index_meta->mutable_zone_map_index(); // store segment zone map @@ -112,7 +111,7 @@ Status ZoneMapIndexWriter::finish(fs::WritableBlock* wblock, ColumnIndexMetaPB* options.encoding = EncodingInfo::get_default_encoding(type_info, false); options.compression = NO_COMPRESSION; // currently not compressed - IndexedColumnWriter writer(options, type_info, wblock); + IndexedColumnWriter writer(options, type_info, file_writer); RETURN_IF_ERROR(writer.init()); for (auto& value : _values) { @@ -123,7 +122,7 @@ Status ZoneMapIndexWriter::finish(fs::WritableBlock* wblock, ColumnIndexMetaPB* } Status ZoneMapIndexReader::load(bool use_page_cache, bool kept_in_memory) { - IndexedColumnReader reader(_path_desc, _index_meta->page_zone_maps()); + IndexedColumnReader reader(_fs, _path, _index_meta->page_zone_maps()); RETURN_IF_ERROR(reader.load(use_page_cache, kept_in_memory)); IndexedColumnIterator iter(&reader); diff --git a/be/src/olap/rowset/segment_v2/zone_map_index.h b/be/src/olap/rowset/segment_v2/zone_map_index.h index f8ddfbb3525b03..5de7e4804b0d59 100644 --- a/be/src/olap/rowset/segment_v2/zone_map_index.h +++ b/be/src/olap/rowset/segment_v2/zone_map_index.h @@ -24,6 +24,7 @@ #include "common/status.h" #include "env/env.h" #include "gen_cpp/segment_v2.pb.h" +#include "io/fs/file_system.h" #include "olap/field.h" #include "olap/rowset/segment_v2/binary_plain_page.h" #include "runtime/mem_pool.h" @@ -31,9 +32,9 @@ namespace doris { -namespace fs { -class WritableBlock; -} +namespace io { +class FileWriter; +} // namespace io namespace segment_v2 { @@ -54,7 +55,7 @@ struct ZoneMap { bool pass_all = false; - void to_proto(ZoneMapPB* dst, Field* field) { + void to_proto(ZoneMapPB* dst, Field* field) const { if (pass_all) { dst->set_min(""); dst->set_max(""); @@ -83,11 +84,11 @@ class ZoneMapIndexWriter { // mark the end of one data page so that we can finalize the corresponding zone map Status flush(); - Status finish(fs::WritableBlock* wblock, ColumnIndexMetaPB* index_meta); + Status finish(io::FileWriter* file_writer, ColumnIndexMetaPB* index_meta); void moidfy_index_before_flush(ZoneMap& zone_map); - uint64_t size() { return _estimated_size; } + uint64_t size() const { return _estimated_size; } void reset_page_zone_map(); void reset_segment_zone_map(); @@ -117,8 +118,9 @@ class ZoneMapIndexWriter { class ZoneMapIndexReader { public: - explicit ZoneMapIndexReader(const FilePathDesc& path_desc, const ZoneMapIndexPB* index_meta) - : _path_desc(path_desc), _index_meta(index_meta) {} + explicit ZoneMapIndexReader(io::FileSystem* fs, const std::string& path, + const ZoneMapIndexPB* index_meta) + : _fs(fs), _path(path), _index_meta(index_meta) {} // load all page zone maps into memory Status load(bool use_page_cache, bool kept_in_memory); @@ -128,7 +130,8 @@ class ZoneMapIndexReader { int32_t num_pages() const { return _page_zone_maps.size(); } private: - FilePathDesc _path_desc; + io::FileSystem* _fs; + std::string _path; const ZoneMapIndexPB* _index_meta; std::vector _page_zone_maps; diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index 68724d29bcd9ea..c49b618201b166 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -1104,6 +1104,8 @@ Status SchemaChangeWithSorting::_inner_process(RowsetReaderSharedPtr rowset_read _temp_delta_versions.first = _temp_delta_versions.second; SegmentsOverlapPB segments_overlap = rowset->rowset_meta()->segments_overlap(); + int64_t oldest_write_timestamp = rowset->oldest_write_timestamp(); + int64_t newest_write_timestamp = rowset->newest_write_timestamp(); RowBlock* ref_row_block = nullptr; rowset_reader->next_block(&ref_row_block); while (ref_row_block != nullptr && ref_row_block->has_remaining()) { @@ -1135,7 +1137,8 @@ Status SchemaChangeWithSorting::_inner_process(RowsetReaderSharedPtr rowset_read if (!_internal_sorting( row_block_arr, Version(_temp_delta_versions.second, _temp_delta_versions.second), - new_tablet, segments_overlap, &rowset)) { + oldest_write_timestamp, newest_write_timestamp, new_tablet, + segments_overlap, &rowset)) { LOG(WARNING) << "failed to sorting internally."; return Status::OLAPInternalError(OLAP_ERR_ALTER_STATUS_ERR); } @@ -1186,7 +1189,8 @@ Status SchemaChangeWithSorting::_inner_process(RowsetReaderSharedPtr rowset_read if (!_internal_sorting(row_block_arr, Version(_temp_delta_versions.second, _temp_delta_versions.second), - new_tablet, segments_overlap, &rowset)) { + oldest_write_timestamp, newest_write_timestamp, new_tablet, + segments_overlap, &rowset)) { LOG(WARNING) << "failed to sorting internally."; return Status::OLAPInternalError(OLAP_ERR_ALTER_STATUS_ERR); } @@ -1219,10 +1223,10 @@ Status SchemaChangeWithSorting::_inner_process(RowsetReaderSharedPtr rowset_read return res; } -bool SchemaChangeWithSorting::_internal_sorting(const std::vector& row_block_arr, - const Version& version, TabletSharedPtr new_tablet, - SegmentsOverlapPB segments_overlap, - RowsetSharedPtr* rowset) { +bool SchemaChangeWithSorting::_internal_sorting( + const std::vector& row_block_arr, const Version& version, + int64_t oldest_write_timestamp, int64_t newest_write_timestamp, TabletSharedPtr new_tablet, + SegmentsOverlapPB segments_overlap, RowsetSharedPtr* rowset) { uint64_t merged_rows = 0; RowBlockMerger merger(new_tablet); @@ -1230,7 +1234,9 @@ bool SchemaChangeWithSorting::_internal_sorting(const std::vector& ro << ", block_row_size=" << new_tablet->num_rows_per_row_block(); std::unique_ptr rowset_writer; - if (!new_tablet->create_rowset_writer(version, VISIBLE, segments_overlap, &rowset_writer)) { + if (!new_tablet->create_rowset_writer(version, VISIBLE, segments_overlap, + oldest_write_timestamp, newest_write_timestamp, + &rowset_writer)) { return false; } @@ -1757,8 +1763,10 @@ Status SchemaChangeHandler::_convert_historical_rowsets(const SchemaChangeParams std::unique_ptr rowset_writer; Status status = new_tablet->create_rowset_writer( rs_reader->version(), VISIBLE, - rs_reader->rowset()->rowset_meta()->segments_overlap(), &rowset_writer); - if (!status) { + rs_reader->rowset()->rowset_meta()->segments_overlap(), + rs_reader->oldest_write_timestamp(), rs_reader->newest_write_timestamp(), + &rowset_writer); + if (!status.ok()) { res = Status::OLAPInternalError(OLAP_ERR_ROWSET_BUILDER_INIT); return process_alter_exit(); } diff --git a/be/src/olap/schema_change.h b/be/src/olap/schema_change.h index 2f820ae79e5900..3a95073c44da4d 100644 --- a/be/src/olap/schema_change.h +++ b/be/src/olap/schema_change.h @@ -192,7 +192,8 @@ class SchemaChangeWithSorting : public SchemaChange { TabletSharedPtr new_tablet, TabletSharedPtr base_tablet) override; bool _internal_sorting(const std::vector& row_block_arr, - const Version& temp_delta_versions, TabletSharedPtr new_tablet, + const Version& temp_delta_versions, int64_t oldest_write_timestamp, + int64_t newest_write_timestamp, TabletSharedPtr new_tablet, SegmentsOverlapPB segments_overlap, RowsetSharedPtr* rowset); bool _external_sorting(std::vector& src_rowsets, RowsetWriter* rowset_writer, diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp index 87ccef34ada79b..72bf6512400655 100644 --- a/be/src/olap/snapshot_manager.cpp +++ b/be/src/olap/snapshot_manager.cpp @@ -94,9 +94,6 @@ Status SnapshotManager::release_snapshot(const string& snapshot_path) { SCOPED_SWITCH_THREAD_LOCAL_MEM_TRACKER(_mem_tracker); auto stores = StorageEngine::instance()->get_stores(); for (auto store : stores) { - if (store->is_remote()) { - continue; - } std::string abs_path; RETURN_WITH_WARN_IF_ERROR(Env::Default()->canonicalize(store->path(), &abs_path), Status::OLAPInternalError(OLAP_ERR_DIR_NOT_EXIST), @@ -118,20 +115,19 @@ Status SnapshotManager::release_snapshot(const string& snapshot_path) { // TODO support beta rowset // For now, alpha and beta rowset meta have same fields, so we can just use // AlphaRowsetMeta here. -Status SnapshotManager::convert_rowset_ids(const FilePathDesc& clone_dir_desc, int64_t tablet_id, +Status SnapshotManager::convert_rowset_ids(const std::string& clone_dir, int64_t tablet_id, int64_t replica_id, const int32_t& schema_hash) { SCOPED_SWITCH_THREAD_LOCAL_MEM_TRACKER(_mem_tracker); Status res = Status::OK(); // check clone dir existed - if (!FileUtils::check_exist(clone_dir_desc.filepath)) { + if (!FileUtils::check_exist(clone_dir)) { res = Status::OLAPInternalError(OLAP_ERR_DIR_NOT_EXIST); - LOG(WARNING) << "clone dir not existed when convert rowsetids. clone_dir=" - << clone_dir_desc.debug_string(); + LOG(WARNING) << "clone dir not existed when convert rowsetids. clone_dir=" << clone_dir; return res; } // load original tablet meta - string cloned_meta_file = clone_dir_desc.filepath + "/" + std::to_string(tablet_id) + ".hdr"; + auto cloned_meta_file = fmt::format("{}/{}.hdr", clone_dir, tablet_id); TabletMeta cloned_tablet_meta; if ((res = cloned_tablet_meta.create_from_file(cloned_meta_file)) != Status::OK()) { LOG(WARNING) << "fail to load original tablet meta after clone. " @@ -159,9 +155,17 @@ Status SnapshotManager::convert_rowset_ids(const FilePathDesc& clone_dir_desc, i std::unordered_map rs_version_map; for (auto& visible_rowset : cloned_tablet_meta_pb.rs_metas()) { RowsetMetaPB* rowset_meta = new_tablet_meta_pb.add_rs_metas(); - RowsetId rowset_id = StorageEngine::instance()->next_rowset_id(); - RETURN_NOT_OK(_rename_rowset_id(visible_rowset, clone_dir_desc, tablet_schema, rowset_id, - rowset_meta)); + + if (!visible_rowset.has_resource_id()) { + // src be local rowset + RowsetId rowset_id = StorageEngine::instance()->next_rowset_id(); + RETURN_NOT_OK(_rename_rowset_id(visible_rowset, clone_dir, tablet_schema, rowset_id, + rowset_meta)); + } else { + // remote rowset + *rowset_meta = visible_rowset; + } + // FIXME(cyx): Redundant? rowset_meta->set_tablet_id(tablet_id); rowset_meta->set_tablet_schema_hash(schema_hash); Version rowset_version = {visible_rowset.start_version(), visible_rowset.end_version()}; @@ -175,16 +179,24 @@ Status SnapshotManager::convert_rowset_ids(const FilePathDesc& clone_dir_desc, i continue; } RowsetMetaPB* rowset_meta = new_tablet_meta_pb.add_stale_rs_metas(); - RowsetId rowset_id = StorageEngine::instance()->next_rowset_id(); - RETURN_NOT_OK(_rename_rowset_id(stale_rowset, clone_dir_desc, tablet_schema, rowset_id, - rowset_meta)); + + if (!stale_rowset.has_resource_id()) { + // src be local rowset + RowsetId rowset_id = StorageEngine::instance()->next_rowset_id(); + RETURN_NOT_OK(_rename_rowset_id(stale_rowset, clone_dir, tablet_schema, rowset_id, + rowset_meta)); + } else { + // remote rowset + *rowset_meta = stale_rowset; + } + // FIXME(cyx): Redundant? rowset_meta->set_tablet_id(tablet_id); rowset_meta->set_tablet_schema_hash(schema_hash); } res = TabletMeta::save(cloned_meta_file, new_tablet_meta_pb); if (!res.ok()) { - LOG(WARNING) << "fail to save converted tablet meta to dir='" << clone_dir_desc.filepath; + LOG(WARNING) << "fail to save converted tablet meta to dir='" << clone_dir; return res; } @@ -192,7 +204,7 @@ Status SnapshotManager::convert_rowset_ids(const FilePathDesc& clone_dir_desc, i } Status SnapshotManager::_rename_rowset_id(const RowsetMetaPB& rs_meta_pb, - const FilePathDesc& new_path_desc, + const std::string& new_tablet_path, TabletSchema& tablet_schema, const RowsetId& rowset_id, RowsetMetaPB* new_rs_meta_pb) { Status res = Status::OK(); @@ -205,7 +217,7 @@ Status SnapshotManager::_rename_rowset_id(const RowsetMetaPB& rs_meta_pb, RowsetMetaSharedPtr alpha_rowset_meta(new AlphaRowsetMeta()); alpha_rowset_meta->init_from_pb(rs_meta_pb); RowsetSharedPtr org_rowset; - RETURN_NOT_OK(RowsetFactory::create_rowset(&tablet_schema, new_path_desc, alpha_rowset_meta, + RETURN_NOT_OK(RowsetFactory::create_rowset(&tablet_schema, new_tablet_path, alpha_rowset_meta, &org_rowset)); // do not use cache to load index // because the index file may conflict @@ -218,10 +230,12 @@ Status SnapshotManager::_rename_rowset_id(const RowsetMetaPB& rs_meta_pb, context.partition_id = org_rowset_meta->partition_id(); context.tablet_schema_hash = org_rowset_meta->tablet_schema_hash(); context.rowset_type = org_rowset_meta->rowset_type(); - context.path_desc = new_path_desc; + context.tablet_path = new_tablet_path; context.tablet_schema = &tablet_schema; context.rowset_state = org_rowset_meta->rowset_state(); context.version = org_rowset_meta->version(); + context.oldest_write_timestamp = org_rowset_meta->oldest_write_timestamp(); + context.newest_write_timestamp = org_rowset_meta->newest_write_timestamp(); // keep segments_overlap same as origin rowset context.segments_overlap = alpha_rowset_meta->segments_overlap(); @@ -248,7 +262,7 @@ Status SnapshotManager::_rename_rowset_id(const RowsetMetaPB& rs_meta_pb, // get snapshot path: curtime.seq.timeout // eg: 20190819221234.3.86400 Status SnapshotManager::_calc_snapshot_id_path(const TabletSharedPtr& tablet, int64_t timeout_s, - string* out_path) { + std::string* out_path) { Status res = Status::OK(); if (out_path == nullptr) { LOG(WARNING) << "output parameter cannot be null"; @@ -263,38 +277,31 @@ Status SnapshotManager::_calc_snapshot_id_path(const TabletSharedPtr& tablet, in return res; } - std::stringstream snapshot_id_path_stream; std::unique_lock auto_lock( _snapshot_mutex); // will automatically unlock when function return. - snapshot_id_path_stream << tablet->data_dir()->path() << SNAPSHOT_PREFIX << "/" << time_str - << "." << _snapshot_base_id++ << "." << timeout_s; - *out_path = snapshot_id_path_stream.str(); + *out_path = fmt::format("{}/{}/{}.{}.{}", tablet->data_dir()->path(), SNAPSHOT_PREFIX, time_str, + _snapshot_base_id++, timeout_s); return res; } -// location: /path/to/data/DATA_PREFIX/shard_id +// prefix: /path/to/data/DATA_PREFIX/shard_id // return: /path/to/data/DATA_PREFIX/shard_id/tablet_id/schema_hash -FilePathDesc SnapshotManager::get_schema_hash_full_path(const TabletSharedPtr& ref_tablet, - const FilePathDesc& location_desc) const { - FilePathDescStream schema_full_path_desc_s; - schema_full_path_desc_s << location_desc << "/" << ref_tablet->tablet_id() << "/" - << ref_tablet->schema_hash(); - return schema_full_path_desc_s.path_desc(); +std::string SnapshotManager::get_schema_hash_full_path(const TabletSharedPtr& ref_tablet, + const std::string& prefix) { + return fmt::format("{}/{}/{}", prefix, ref_tablet->tablet_id(), ref_tablet->schema_hash()); } std::string SnapshotManager::_get_header_full_path(const TabletSharedPtr& ref_tablet, const std::string& schema_hash_path) const { - std::stringstream header_name_stream; - header_name_stream << schema_hash_path << "/" << ref_tablet->tablet_id() << ".hdr"; - return header_name_stream.str(); + return fmt::format("{}/{}.hdr", schema_hash_path, ref_tablet->tablet_id()); } Status SnapshotManager::_link_index_and_data_files( - const FilePathDesc& schema_hash_path_desc, const TabletSharedPtr& ref_tablet, + const std::string& schema_hash_path, const TabletSharedPtr& ref_tablet, const std::vector& consistent_rowsets) { Status res = Status::OK(); for (auto& rs : consistent_rowsets) { - RETURN_NOT_OK(rs->link_files_to(schema_hash_path_desc, rs->rowset_id())); + RETURN_NOT_OK(rs->link_files_to(schema_hash_path, rs->rowset_id())); } return res; } @@ -315,7 +322,7 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet // snapshot_id_path: // /data/shard_id/tablet_id/snapshot/time_str/id.timeout/ - string snapshot_id_path; + std::string snapshot_id_path; int64_t timeout_s = config::snapshot_expire_time_sec; if (request.__isset.timeout) { timeout_s = request.timeout; @@ -329,18 +336,18 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet // schema_full_path_desc.filepath: // /snapshot_id_path/tablet_id/schema_hash/ - FilePathDesc schema_full_path_desc = get_schema_hash_full_path(ref_tablet, snapshot_id_path); + auto schema_full_path = get_schema_hash_full_path(ref_tablet, snapshot_id_path); // header_path: // /schema_full_path/tablet_id.hdr - string header_path = _get_header_full_path(ref_tablet, schema_full_path_desc.filepath); - if (FileUtils::check_exist(schema_full_path_desc.filepath)) { + auto header_path = _get_header_full_path(ref_tablet, schema_full_path); + if (FileUtils::check_exist(schema_full_path)) { VLOG_TRACE << "remove the old schema_full_path."; - FileUtils::remove_all(schema_full_path_desc.filepath); + FileUtils::remove_all(schema_full_path); } - RETURN_WITH_WARN_IF_ERROR(FileUtils::create_dir(schema_full_path_desc.filepath), + RETURN_WITH_WARN_IF_ERROR(FileUtils::create_dir(schema_full_path), Status::OLAPInternalError(OLAP_ERR_CANNOT_CREATE_DIR), - "create path " + schema_full_path_desc.filepath + " failed"); + "create path " + schema_full_path + " failed"); string snapshot_id; RETURN_WITH_WARN_IF_ERROR(FileUtils::canonicalize(snapshot_id_path, &snapshot_id), @@ -426,9 +433,12 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet std::vector rs_metas; for (auto& rs : consistent_rowsets) { - res = rs->link_files_to(schema_full_path_desc, rs->rowset_id()); - if (!res.ok()) { - break; + if (rs->is_local()) { + // local rowset + res = rs->link_files_to(schema_full_path, rs->rowset_id()); + if (!res.ok()) { + break; + } } rs_metas.push_back(rs->rowset_meta()); VLOG_NOTICE << "add rowset meta to clone list. " diff --git a/be/src/olap/snapshot_manager.h b/be/src/olap/snapshot_manager.h index aab134297fb6f7..7f67efac76e09f 100644 --- a/be/src/olap/snapshot_manager.h +++ b/be/src/olap/snapshot_manager.h @@ -51,8 +51,8 @@ class SnapshotManager { Status make_snapshot(const TSnapshotRequest& request, std::string* snapshot_path, bool* allow_incremental_clone); - FilePathDesc get_schema_hash_full_path(const TabletSharedPtr& ref_tablet, - const FilePathDesc& location_desc) const; + std::string static get_schema_hash_full_path(const TabletSharedPtr& ref_tablet, + const std::string& prefix); // @brief 释放snapshot // @param snapshot_path [in] 要被释放的snapshot的路径,只包含到ID @@ -60,8 +60,8 @@ class SnapshotManager { static SnapshotManager* instance(); - Status convert_rowset_ids(const FilePathDesc& clone_dir_desc, int64_t tablet_id, - int64_t replica_id, const int32_t& schema_hash); + Status convert_rowset_ids(const std::string& clone_dir, int64_t tablet_id, int64_t replica_id, + const int32_t& schema_hash); private: SnapshotManager() : _snapshot_base_id(0) { @@ -75,7 +75,7 @@ class SnapshotManager { std::string _get_header_full_path(const TabletSharedPtr& ref_tablet, const std::string& schema_hash_path) const; - Status _link_index_and_data_files(const FilePathDesc& header_path_desc, + Status _link_index_and_data_files(const std::string& header_path, const TabletSharedPtr& ref_tablet, const std::vector& consistent_rowsets); @@ -85,7 +85,7 @@ class SnapshotManager { Status _prepare_snapshot_dir(const TabletSharedPtr& ref_tablet, std::string* snapshot_id_path); - Status _rename_rowset_id(const RowsetMetaPB& rs_meta_pb, const FilePathDesc& new_path_desc, + Status _rename_rowset_id(const RowsetMetaPB& rs_meta_pb, const std::string& new_tablet_path, TabletSchema& tablet_schema, const RowsetId& next_id, RowsetMetaPB* new_rs_meta_pb); diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index 5a75a4f88de2d0..65cd9484c99042 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -189,6 +189,9 @@ void StorageEngine::load_data_dirs(const std::vector& data_dirs) { } Status StorageEngine::_open() { + // NOTE: must init before _init_store_map. + _file_cache.reset(new_lru_cache("FileHandlerCache", config::file_descriptor_cache_capacity)); + // init store_map RETURN_NOT_OK_STATUS_WITH_WARN(_init_store_map(), "_init_store_map failed"); @@ -202,8 +205,6 @@ Status StorageEngine::_open() { _index_stream_lru_cache = new_lru_cache("SegmentIndexCache", config::index_stream_cache_capacity); - _file_cache.reset(new_lru_cache("FileHandlerCache", config::file_descriptor_cache_capacity)); - auto dirs = get_stores(); load_data_dirs(dirs); @@ -382,7 +383,8 @@ Status StorageEngine::get_all_data_dir_info(std::vector* data_dir_i std::lock_guard l(_store_lock); auto data_dir = _store_map.find(path.first); DCHECK(data_dir != _store_map.end()); - data_dir->second->update_user_data_size(path.second.data_used_capacity); + data_dir->second->update_local_data_size(path.second.local_used_capacity); + data_dir->second->update_remote_data_size(path.second.remote_used_capacity); } // add path info to data_dir_infos @@ -397,7 +399,7 @@ Status StorageEngine::get_all_data_dir_info(std::vector* data_dir_i return res; } -int64_t StorageEngine::get_file_or_directory_size(std::filesystem::path file_path) { +int64_t StorageEngine::get_file_or_directory_size(const std::string& file_path) { if (!std::filesystem::exists(file_path)) { return 0; } @@ -491,9 +493,7 @@ std::vector StorageEngine::get_stores_for_create_tablet( for (auto& it : _store_map) { if (it.second->is_used()) { if (_available_storage_medium_type_count == 1 || - it.second->storage_medium() == storage_medium || - (it.second->storage_medium() == TStorageMedium::REMOTE_CACHE && - FilePathDesc::is_remote(storage_medium))) { + it.second->storage_medium() == storage_medium) { stores.push_back(it.second); } } @@ -691,7 +691,7 @@ Status StorageEngine::start_trash_sweep(double* usage, bool ignore_guard) { double tmp_usage = 0.0; for (DataDirInfo& info : data_dir_infos) { - LOG(INFO) << "Start to sweep path " << info.path_desc.filepath; + LOG(INFO) << "Start to sweep path " << info.path; if (!info.is_used) { continue; } @@ -700,21 +700,18 @@ Status StorageEngine::start_trash_sweep(double* usage, bool ignore_guard) { tmp_usage = std::max(tmp_usage, curr_usage); Status curr_res = Status::OK(); - FilePathDesc snapshot_path_desc(info.path_desc.filepath + SNAPSHOT_PREFIX); - curr_res = _do_sweep(snapshot_path_desc, local_now, snapshot_expire); + auto snapshot_path = fmt::format("{}/{}", info.path, SNAPSHOT_PREFIX); + curr_res = _do_sweep(snapshot_path, local_now, snapshot_expire); if (!curr_res.ok()) { - LOG(WARNING) << "failed to sweep snapshot. path=" << snapshot_path_desc.filepath + LOG(WARNING) << "failed to sweep snapshot. path=" << snapshot_path << ", err_code=" << curr_res; res = curr_res; } - FilePathDescStream trash_path_desc_s; - trash_path_desc_s << info.path_desc << TRASH_PREFIX; - FilePathDesc trash_path_desc = trash_path_desc_s.path_desc(); - curr_res = - _do_sweep(trash_path_desc, local_now, curr_usage > guard_space ? 0 : trash_expire); + auto trash_path = fmt::format("{}/{}", info.path, TRASH_PREFIX); + curr_res = _do_sweep(trash_path, local_now, curr_usage > guard_space ? 0 : trash_expire); if (!curr_res.ok()) { - LOG(WARNING) << "failed to sweep trash. path=" << trash_path_desc.filepath + LOG(WARNING) << "failed to sweep trash. path=" << trash_path << ", err_code=" << curr_res; res = curr_res; } @@ -791,9 +788,6 @@ void StorageEngine::_clean_unused_rowset_metas() { }; auto data_dirs = get_stores(); for (auto data_dir : data_dirs) { - if (data_dir->is_remote()) { - continue; - } RowsetMetaManager::traverse_rowset_metas(data_dir->get_meta(), clean_rowset_func); for (auto& rowset_meta : invalid_rowset_metas) { RowsetMetaManager::remove(data_dir->get_meta(), rowset_meta->tablet_uid(), @@ -824,10 +818,10 @@ void StorageEngine::_clean_unused_txns() { } } -Status StorageEngine::_do_sweep(const FilePathDesc& scan_root_desc, const time_t& local_now, +Status StorageEngine::_do_sweep(const std::string& scan_root, const time_t& local_now, const int32_t expire) { Status res = Status::OK(); - if (!FileUtils::check_exist(scan_root_desc.filepath)) { + if (!FileUtils::check_exist(scan_root)) { // dir not existed. no need to sweep trash. return res; } @@ -835,7 +829,7 @@ Status StorageEngine::_do_sweep(const FilePathDesc& scan_root_desc, const time_t try { // Sort pathes by name, that is by delete time. std::vector sorted_pathes; - std::copy(directory_iterator(path(scan_root_desc.filepath)), directory_iterator(), + std::copy(directory_iterator(scan_root), directory_iterator(), std::back_inserter(sorted_pathes)); std::sort(sorted_pathes.begin(), sorted_pathes.end()); for (const auto& sorted_path : sorted_pathes) { @@ -860,48 +854,10 @@ Status StorageEngine::_do_sweep(const FilePathDesc& scan_root_desc, const time_t string path_name = sorted_path.string(); if (difftime(local_now, mktime(&local_tm_create)) >= actual_expire) { - std::string storage_name_path = path_name + "/" + STORAGE_NAME; - if (scan_root_desc.is_remote() && FileUtils::check_exist(storage_name_path)) { - FilePathDesc remote_path_desc = scan_root_desc; - if (!env_util::read_file_to_string(Env::Default(), storage_name_path, - &(remote_path_desc.storage_name)) - .ok()) { - LOG(WARNING) << "read storage_name failed: " << storage_name_path; - continue; - } - boost::algorithm::trim(remote_path_desc.storage_name); - std::shared_ptr storage_backend = - StorageBackendMgr::instance()->get_storage_backend( - remote_path_desc.storage_name); - // if storage_backend is nullptr, the remote storage is invalid. - // Only the local path need to be removed. - if (storage_backend != nullptr) { - std::string remote_root_path; - if (!StorageBackendMgr::instance()->get_root_path( - remote_path_desc.storage_name, &remote_root_path)) { - LOG(WARNING) << "read storage root_path failed: " - << remote_path_desc.storage_name; - continue; - } - remote_path_desc.remote_path = remote_root_path + TRASH_PREFIX; - std::filesystem::path local_path(path_name); - std::stringstream remote_file_stream; - remote_file_stream << remote_path_desc.remote_path << "/" - << local_path.filename().string(); - Status ret = storage_backend->rmdir(remote_file_stream.str()); - if (!ret.ok()) { - LOG(WARNING) - << "fail to remove file or directory. path=" - << remote_file_stream.str() << ", error=" << ret.to_string(); - res = Status::OLAPInternalError(OLAP_ERR_OS_ERROR); - continue; - } - } - } Status ret = FileUtils::remove_all(path_name); if (!ret.ok()) { - LOG(WARNING) << "fail to remove file or directory. path_desc: " - << scan_root_desc.debug_string() << ", error=" << ret.to_string(); + LOG(WARNING) << "fail to remove file or directory. path_desc: " << scan_root + << ", error=" << ret.to_string(); res = Status::OLAPInternalError(OLAP_ERR_OS_ERROR); continue; } @@ -911,8 +867,7 @@ Status StorageEngine::_do_sweep(const FilePathDesc& scan_root_desc, const time_t } } } catch (...) { - LOG(WARNING) << "Exception occur when scan directory. path_desc=" - << scan_root_desc.debug_string(); + LOG(WARNING) << "Exception occur when scan directory. path_desc=" << scan_root; res = Status::OLAPInternalError(OLAP_ERR_IO_ERROR); } @@ -941,12 +896,16 @@ void StorageEngine::start_delete_unused_rowset() { if (it->second.use_count() != 1) { ++it; } else if (it->second->need_delete_file()) { - VLOG_NOTICE << "start to remove rowset:" << it->second->rowset_id() - << ", version:" << it->second->version().first << "-" - << it->second->version().second; - Status status = it->second->remove(); - VLOG_NOTICE << "remove rowset:" << it->second->rowset_id() - << " finished. status:" << status; + // FIXME(cyx): Currently remote unused rowsets are generated by compaction gc, + // we cannot remove them directly as other BE may need them. + if (it->second->is_local()) { + VLOG_NOTICE << "start to remove rowset:" << it->second->rowset_id() + << ", version:" << it->second->version().first << "-" + << it->second->version().second; + Status status = it->second->remove(); + VLOG_NOTICE << "remove rowset:" << it->second->rowset_id() + << " finished. status:" << status; + } it = _unused_rowsets.erase(it); } } @@ -1010,7 +969,7 @@ Status StorageEngine::obtain_shard_path(TStorageMedium::type storage_medium, } std::stringstream root_path_stream; - root_path_stream << stores[0]->path() << DATA_PREFIX << "/" << shard; + root_path_stream << stores[0]->path() << "/" << DATA_PREFIX << "/" << shard; *shard_path = root_path_stream.str(); *store = stores[0]; diff --git a/be/src/olap/storage_engine.h b/be/src/olap/storage_engine.h index e54a8d5f914aa1..fb13506b0b6b34 100644 --- a/be/src/olap/storage_engine.h +++ b/be/src/olap/storage_engine.h @@ -96,7 +96,7 @@ class StorageEngine { // @brief 获取所有root_path信息 Status get_all_data_dir_info(std::vector* data_dir_infos, bool need_update); - int64_t get_file_or_directory_size(std::filesystem::path file_path); + int64_t get_file_or_directory_size(const std::string& file_path); // get root path for creating tablet. The returned vector of root path should be random, // for avoiding that all the tablet would be deployed one disk. @@ -219,7 +219,7 @@ class StorageEngine { void _clean_unused_rowset_metas(); - Status _do_sweep(const FilePathDesc& scan_root_desc, const time_t& local_tm_now, + Status _do_sweep(const std::string& scan_root, const time_t& local_tm_now, const int32_t expire); // All these xxx_callback() functions are for Background threads @@ -275,6 +275,8 @@ class StorageEngine { Status _handle_quick_compaction(TabletSharedPtr); void _adjust_compaction_thread_num(); + + void _cooldown_tasks_producer_callback(); private: struct CompactionCandidate { @@ -408,6 +410,14 @@ class StorageEngine { std::shared_ptr _cumulative_compaction_policy; + scoped_refptr _cooldown_tasks_producer_thread; + + std::unique_ptr _cooldown_thread_pool; + + std::mutex _running_cooldown_mutex; + std::unordered_map _running_cooldown_tasks_cnt; + std::unordered_set _running_cooldown_tablets; + DISALLOW_COPY_AND_ASSIGN(StorageEngine); }; diff --git a/be/src/olap/storage_migration_v2.cpp b/be/src/olap/storage_migration_v2.cpp index 1b3f56a68d3345..2a5373969a32c9 100644 --- a/be/src/olap/storage_migration_v2.cpp +++ b/be/src/olap/storage_migration_v2.cpp @@ -345,6 +345,8 @@ Status StorageMigrationV2Handler::_convert_historical_rowsets( writer_context.rowset_state = VISIBLE; writer_context.version = rs_reader->version(); writer_context.segments_overlap = rs_reader->rowset()->rowset_meta()->segments_overlap(); + writer_context.oldest_write_timestamp = rs_reader->oldest_write_timestamp(); + writer_context.newest_write_timestamp = rs_reader->newest_write_timestamp(); std::unique_ptr rowset_writer; Status status = RowsetFactory::create_rowset_writer(writer_context, &rowset_writer); diff --git a/be/src/olap/storage_policy_mgr.cpp b/be/src/olap/storage_policy_mgr.cpp new file mode 100644 index 00000000000000..a91ef6d6bd7123 --- /dev/null +++ b/be/src/olap/storage_policy_mgr.cpp @@ -0,0 +1,74 @@ +#include "olap/storage_policy_mgr.h" + +#include "io/fs/file_system.h" +#include "io/fs/file_system_map.h" +#include "io/fs/s3_file_system.h" +#include "util/s3_util.h" + +namespace doris { + +void StoragePolicyMgr::update(const std::string& name, StoragePolicyPtr policy) { + std::lock_guard l(_mutex); + auto it = _policy_map.find(name); + if (it != _policy_map.end()) { + // just support change ak, sk, cooldown_ttl, cooldown_datetime + LOG(INFO) << "update storage policy name: " << name; + auto s3_fs = std::dynamic_pointer_cast( + io::FileSystemMap::instance()->get(name)); + DCHECK(s3_fs); + s3_fs->set_ak(policy->s3_ak); + s3_fs->set_sk(policy->s3_sk); + s3_fs->connect(); + it->second = std::move(policy); + } else { + // can't find name's policy, so do nothing. + } +} + +void StoragePolicyMgr::periodic_put(const std::string& name, StoragePolicyPtr policy) { + std::lock_guard l(_mutex); + auto it = _policy_map.find(name); + if (it == _policy_map.end()) { + LOG(INFO) << "add storage policy name: " << name << " to map"; + std::map aws_properties = { + {S3_AK, policy->s3_ak}, + {S3_SK, policy->s3_sk}, + {S3_ENDPOINT, policy->s3_endpoint}, + {S3_REGION, policy->s3_region}, + {S3_MAX_CONN_SIZE, std::to_string(policy->s3_max_conn)}, + {S3_REQUEST_TIMEOUT_MS, std::to_string(policy->s3_request_timeout_ms)}, + {S3_CONN_TIMEOUT_MS, std::to_string(policy->s3_conn_timeout_ms)}}; + auto s3_fs = std::make_shared(aws_properties, policy->bucket, + policy->root_path, name); + s3_fs->connect(); + io::FileSystemMap::instance()->insert(name, std::move(s3_fs)); + _policy_map.emplace(name, std::move(policy)); + } else if (it->second->md5_sum != policy->md5_sum) { + // fe change policy + // just support change ak, sk, cooldown_ttl, cooldown_datetime + LOG(INFO) << "update storage policy name: " << name; + auto s3_fs = std::dynamic_pointer_cast( + io::FileSystemMap::instance()->get(name)); + DCHECK(s3_fs); + s3_fs->set_ak(policy->s3_ak); + s3_fs->set_sk(policy->s3_sk); + s3_fs->connect(); + it->second = std::move(policy); + } +} + +StoragePolicyMgr::StoragePolicyPtr StoragePolicyMgr::get(const std::string& name) { + std::lock_guard l(_mutex); + auto it = _policy_map.find(name); + if (it != _policy_map.end()) { + return it->second; + } + return nullptr; +} + +void StoragePolicyMgr::del(const std::string& name) { + std::lock_guard l(_mutex); + _policy_map.erase(name); +} + +} // namespace doris diff --git a/be/src/olap/storage_policy_mgr.h b/be/src/olap/storage_policy_mgr.h new file mode 100644 index 00000000000000..7c34b510ab227d --- /dev/null +++ b/be/src/olap/storage_policy_mgr.h @@ -0,0 +1,62 @@ +#pragma once + +#include + +#include +#include +#include + +#include "common/status.h" + +struct StoragePolicy { + std::string storage_policy_name; + int64_t cooldown_datetime; + int64_t cooldown_ttl; + // s3 resource + std::string s3_endpoint; + std::string s3_region; + std::string s3_ak; + std::string s3_sk; + std::string root_path; + std::string bucket; + std::string md5_sum; + int64_t s3_conn_timeout_ms; + int64_t s3_max_conn; + int64_t s3_request_timeout_ms; +}; + +inline std::ostream& operator<<(std::ostream& out, const StoragePolicy& m) { + out << "storage_policy_name: " << m.storage_policy_name + << " cooldown_datetime: " << m.cooldown_datetime << " cooldown_ttl: " << m.cooldown_ttl + << " s3_endpoint: " << m.s3_endpoint << " s3_region: " << m.s3_region + << " root_path: " << m.root_path << " bucket: " << m.bucket << " md5_sum: " << m.md5_sum + << " s3_conn_timeout_ms: " << m.s3_conn_timeout_ms << " s3_max_conn: " << m.s3_max_conn + << " s3_request_timeout_ms: " << m.s3_request_timeout_ms; + return out; +} + +namespace doris { +class ExecEnv; + +class StoragePolicyMgr { +public: + using StoragePolicyPtr = std::shared_ptr; + StoragePolicyMgr() {} + + ~StoragePolicyMgr() = default; + + // fe push update policy to be + void update(const std::string& name, StoragePolicyPtr policy); + + // periodic pull from fe + void periodic_put(const std::string& name, StoragePolicyPtr policy); + + StoragePolicyPtr get(const std::string& name); + + void del(const std::string& name); + +private: + std::mutex _mutex; + std::unordered_map _policy_map; +}; +} // namespace doris diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 2db820b9751b93..21b6ac7827693c 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -18,16 +18,26 @@ #include "olap/tablet.h" #include +#include #include #include #include #include +#include #include -#include +#include #include +#include #include - +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/status.h" +#include "io/fs/path.h" +#include "io/fs/remote_file_system.h" #include "olap/base_compaction.h" #include "olap/cumulative_compaction.h" #include "olap/olap_common.h" @@ -39,6 +49,8 @@ #include "olap/rowset/rowset_meta_manager.h" #include "olap/schema_change.h" #include "olap/storage_engine.h" +#include "olap/storage_policy_mgr.h" +#include "olap/tablet_meta.h" #include "olap/tablet_meta_manager.h" #include "util/path_util.h" #include "util/pretty_printer.h" @@ -103,7 +115,7 @@ Status Tablet::_init_once_action() { for (const auto& rs_meta : _tablet_meta->all_rs_metas()) { Version version = rs_meta->version(); RowsetSharedPtr rowset; - res = RowsetFactory::create_rowset(&_schema, _tablet_path_desc, rs_meta, &rowset); + res = RowsetFactory::create_rowset(&_schema, _tablet_path, rs_meta, &rowset); if (!res.ok()) { LOG(WARNING) << "fail to init rowset. tablet_id=" << tablet_id() << ", schema_hash=" << schema_hash() << ", version=" << version @@ -117,7 +129,7 @@ Status Tablet::_init_once_action() { for (auto& stale_rs_meta : _tablet_meta->all_stale_rs_metas()) { Version version = stale_rs_meta->version(); RowsetSharedPtr rowset; - res = RowsetFactory::create_rowset(&_schema, _tablet_path_desc, stale_rs_meta, &rowset); + res = RowsetFactory::create_rowset(&_schema, _tablet_path, stale_rs_meta, &rowset); if (!res.ok()) { LOG(WARNING) << "fail to init stale rowset. tablet_id:" << tablet_id() << ", schema_hash:" << schema_hash() << ", version=" << version @@ -187,7 +199,7 @@ Status Tablet::revise_tablet_meta(const std::vector& rowset for (auto& rs_meta : rowsets_to_clone) { Version version = {rs_meta->start_version(), rs_meta->end_version()}; RowsetSharedPtr rowset; - res = RowsetFactory::create_rowset(&_schema, _tablet_path_desc, rs_meta, &rowset); + res = RowsetFactory::create_rowset(&_schema, _tablet_path, rs_meta, &rowset); if (!res.ok()) { LOG(WARNING) << "fail to init rowset. version=" << version; return res; @@ -991,10 +1003,10 @@ void Tablet::delete_all_files() { bool Tablet::check_path(const std::string& path_to_check) const { std::shared_lock rdlock(_meta_lock); - if (path_to_check == _tablet_path_desc.filepath) { + if (path_to_check == _tablet_path) { return true; } - std::string tablet_id_dir = path_util::dir_name(_tablet_path_desc.filepath); + auto tablet_id_dir = io::Path(_tablet_path).parent_path(); if (path_to_check == tablet_id_dir) { return true; } @@ -1097,8 +1109,11 @@ void Tablet::find_alpha_rowsets(std::vector* rowsets) const { void Tablet::pick_candidate_rowsets_to_base_compaction(vector* candidate_rowsets) { std::shared_lock rdlock(_meta_lock); + // FIXME(cyx): If there are delete predicate rowsets in tablet, + // remote rowsets cannot apply these delete predicate, which can cause + // incorrect query result. for (auto& it : _rs_version_map) { - if (it.first.first < _cumulative_point) { + if (it.first.first < _cumulative_point && it.second->is_local()) { candidate_rowsets->push_back(it.second); } } @@ -1138,7 +1153,7 @@ void Tablet::get_compaction_status(std::string* json_result) { } rapidjson::Value cumulative_policy_type; std::string policy_type_str = "cumulative compaction policy not initializied"; - if (_cumulative_compaction_policy.get() != nullptr) { + if (_cumulative_compaction_policy != nullptr) { policy_type_str = _cumulative_compaction_policy->name(); } cumulative_policy_type.SetString(policy_type_str.c_str(), policy_type_str.length(), @@ -1320,7 +1335,7 @@ void Tablet::build_tablet_report_info(TTabletInfo* tablet_info, tablet_info->tablet_id = _tablet_meta->tablet_id(); tablet_info->schema_hash = _tablet_meta->schema_hash(); tablet_info->row_count = _tablet_meta->num_rows(); - tablet_info->data_size = _tablet_meta->tablet_footprint(); + tablet_info->data_size = _tablet_meta->tablet_local_size(); // Here we need to report to FE if there are any missing versions of tablet. // We start from the initial version and traverse backwards until we meet a discontinuous version. @@ -1374,16 +1389,12 @@ void Tablet::build_tablet_report_info(TTabletInfo* tablet_info, // Useless but it is a required filed in TTabletInfo tablet_info->version_hash = 0; tablet_info->__set_partition_id(_tablet_meta->partition_id()); - if (FilePathDesc::is_remote(_data_dir->storage_medium())) { - tablet_info->__set_storage_medium( - fs::fs_util::get_t_storage_medium(_tablet_meta->storage_medium())); - } else { - tablet_info->__set_storage_medium(_data_dir->storage_medium()); - } + tablet_info->__set_storage_medium(_data_dir->storage_medium()); tablet_info->__set_version_count(_tablet_meta->version_count()); tablet_info->__set_path_hash(_data_dir->path_hash()); tablet_info->__set_is_in_memory(_tablet_meta->tablet_schema().is_in_memory()); tablet_info->__set_replica_id(replica_id()); + tablet_info->__set_remote_data_size(_tablet_meta->tablet_remote_size()); } // should use this method to get a copy of current tablet meta @@ -1553,7 +1564,7 @@ Status Tablet::create_initial_rowset(const int64_t req_version) { do { // there is no data in init rowset, so overlapping info is unknown. std::unique_ptr rs_writer; - res = create_rowset_writer(version, VISIBLE, OVERLAP_UNKNOWN, &rs_writer); + res = create_rowset_writer(version, VISIBLE, OVERLAP_UNKNOWN, -1, -1, &rs_writer); if (!res.ok()) { LOG(WARNING) << "failed to init rowset writer for tablet " << full_name(); break; @@ -1584,11 +1595,14 @@ Status Tablet::create_initial_rowset(const int64_t req_version) { Status Tablet::create_rowset_writer(const Version& version, const RowsetStatePB& rowset_state, const SegmentsOverlapPB& overlap, + int64_t oldest_write_timestamp, int64_t newest_write_timestamp, std::unique_ptr* rowset_writer) { RowsetWriterContext context; context.version = version; context.rowset_state = rowset_state; context.segments_overlap = overlap; + context.oldest_write_timestamp = oldest_write_timestamp; + context.newest_write_timestamp = newest_write_timestamp; _init_context_common_fields(context); return RowsetFactory::create_rowset_writer(context, rowset_writer); } @@ -1602,6 +1616,8 @@ Status Tablet::create_rowset_writer(const int64_t& txn_id, const PUniqueId& load context.load_id = load_id; context.rowset_state = rowset_state; context.segments_overlap = overlap; + context.oldest_write_timestamp = -1; + context.newest_write_timestamp = -1; _init_context_common_fields(context); return RowsetFactory::create_rowset_writer(context, rowset_writer); } @@ -1619,13 +1635,13 @@ void Tablet::_init_context_common_fields(RowsetWriterContext& context) { if (context.rowset_type == ALPHA_ROWSET) { context.rowset_type = StorageEngine::instance()->default_rowset_type(); } - context.path_desc = tablet_path_desc(); + context.tablet_path = tablet_path(); context.tablet_schema = &(tablet_schema()); context.data_dir = data_dir(); } Status Tablet::create_rowset(RowsetMetaSharedPtr rowset_meta, RowsetSharedPtr* rowset) { - return RowsetFactory::create_rowset(&tablet_schema(), tablet_path_desc(), rowset_meta, rowset); + return RowsetFactory::create_rowset(&tablet_schema(), tablet_path(), rowset_meta, rowset); } std::shared_ptr& Tablet::get_compaction_mem_tracker(CompactionType compaction_type) { @@ -1636,4 +1652,156 @@ std::shared_ptr& Tablet::get_compaction_mem_tracker(CompactionType c } } +Status Tablet::cooldown() { + std::unique_lock schema_change_lock(_schema_change_lock); + // Check executing serially with compaction task. + std::unique_lock base_compaction_lock(_base_compaction_lock, std::try_to_lock); + if (!base_compaction_lock.owns_lock()) { + LOG(WARNING) << "base compaction is running. tablet=" << full_name(); + return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR); + } + std::unique_lock cumu_compaction_lock(_cumulative_compaction_lock, std::try_to_lock); + if (!cumu_compaction_lock.owns_lock()) { + LOG(WARNING) << "cumulative compaction is running. tablet=" << full_name(); + return Status::OLAPInternalError(OLAP_ERR_BE_TRY_BE_LOCK_ERROR); + } + auto dest_fs = io::FileSystemMap::instance()->get(cooldown_resource()); + if (!dest_fs) { + return Status::OLAPInternalError(OLAP_ERR_NOT_INITED); + } + DCHECK(dest_fs->type() == io::FileSystemType::S3); + auto old_rowset = pick_cooldown_rowset(); + if (!old_rowset) { + LOG(WARNING) << "Cannot pick cooldown rowset in tablet " << tablet_id(); + return Status::OK(); + } + RowsetId new_rowset_id = StorageEngine::instance()->next_rowset_id(); + + auto start = std::chrono::steady_clock::now(); + + RETURN_IF_ERROR(old_rowset->upload_to(reinterpret_cast(dest_fs.get()), + new_rowset_id)); + + auto duration = std::chrono::duration(std::chrono::steady_clock::now() - start); + LOG(INFO) << "Upload rowset " << old_rowset->version() << " " << new_rowset_id.to_string() + << " to " << dest_fs->root_path().native() << ", tablet_id=" << tablet_id() + << ", duration=" << duration.count() << ", capacity=" << old_rowset->data_disk_size() + << ", tp=" << old_rowset->data_disk_size() / duration.count(); + + // gen a new rowset + auto new_rowset_meta = std::make_shared(*old_rowset->rowset_meta()); + new_rowset_meta->set_rowset_id(new_rowset_id); + new_rowset_meta->set_resource_id(dest_fs->resource_id()); + new_rowset_meta->set_fs(dest_fs); + new_rowset_meta->set_creation_time(time(nullptr)); + RowsetSharedPtr new_rowset; + RowsetFactory::create_rowset(&_schema, _tablet_path, std::move(new_rowset_meta), &new_rowset); + + std::vector to_add {std::move(new_rowset)}; + std::vector to_delete {std::move(old_rowset)}; + + std::unique_lock meta_wlock(_meta_lock); + modify_rowsets(to_add, to_delete); + save_meta(); + return Status::OK(); +} + +RowsetSharedPtr Tablet::pick_cooldown_rowset() { + RowsetSharedPtr rowset; + { + std::shared_lock meta_rlock(_meta_lock); + + // We pick the rowset with smallest start version in local. + int64_t smallest_version = std::numeric_limits::max(); + for (const auto& it : _rs_version_map) { + auto& rs = it.second; + if (rs->is_local() && rs->start_version() < smallest_version) { + smallest_version = rs->start_version(); + rowset = rs; + } + } + } + return rowset; +} + +bool Tablet::need_cooldown(int64_t* cooldown_timestamp, size_t* file_size) { + std::shared_lock meta_rlock(_meta_lock); + if (cooldown_resource().empty()) { + VLOG_DEBUG << "tablet does not need cooldown, tablet id: " << tablet_id(); + return false; + } + auto policy = ExecEnv::GetInstance()->storage_policy_mgr()->get(cooldown_resource()); + if (!policy) { + LOG(WARNING) << "Cannot get storage policy: " << cooldown_resource(); + return false; + } + auto cooldown_ttl_sec = policy->cooldown_ttl; + auto cooldown_datetime = policy->cooldown_datetime; + RowsetSharedPtr rowset = pick_cooldown_rowset(); + if (!rowset) { + VLOG_DEBUG << "pick cooldown rowset, get null, tablet id: " << tablet_id(); + return false; + } + + int64_t oldest_cooldown_time = std::numeric_limits::max(); + if (cooldown_ttl_sec >= 0) { + oldest_cooldown_time = rowset->oldest_write_timestamp() + cooldown_ttl_sec; + } + if (cooldown_datetime > 0) { + oldest_cooldown_time = std::min(oldest_cooldown_time, cooldown_datetime); + } + + int64_t newest_cooldown_time = std::numeric_limits::max(); + if (cooldown_ttl_sec >= 0) { + newest_cooldown_time = rowset->newest_write_timestamp() + cooldown_ttl_sec; + } + if (cooldown_datetime > 0) { + newest_cooldown_time = std::min(newest_cooldown_time, cooldown_datetime); + } + + if (oldest_cooldown_time + config::cooldown_lag_time_sec < UnixSeconds()) { + *cooldown_timestamp = oldest_cooldown_time; + VLOG_DEBUG << "tablet need cooldown, tablet id: " << tablet_id() + << " cooldown_timestamp: " << *cooldown_timestamp; + return true; + } + + if (newest_cooldown_time < UnixSeconds()) { + *file_size = rowset->data_disk_size(); + VLOG_DEBUG << "tablet need cooldown, tablet id: " << tablet_id() + << " file_size: " << *file_size; + return true; + } + + VLOG_DEBUG << "tablet does not need cooldown, tablet id: " << tablet_id() + << " ttl sec: " << cooldown_ttl_sec << " cooldown datetime: " << cooldown_datetime + << " oldest write time: " << rowset->oldest_write_timestamp() + << " newest write time: " << rowset->newest_write_timestamp(); + return false; +} + +void Tablet::remove_all_remote_rowsets() { + std::unique_lock meta_wlock(_meta_lock); + DCHECK(_state == TabletState::TABLET_SHUTDOWN); + Status st; + for (auto& it : _rs_version_map) { + auto& rs = it.second; + if (!rs->is_local()) { + st = rs->remove(); + LOG_IF(WARNING, !st.ok()) << "Failed to remove rowset " << rs->version() << " " + << rs->rowset_id().to_string() << " in tablet " << tablet_id() + << ": " << st.to_string(); + } + } + for (auto& it : _stale_rs_version_map) { + auto& rs = it.second; + if (!rs->is_local()) { + st = rs->remove(); + LOG_IF(WARNING, !st.ok()) << "Failed to remove rowset " << rs->version() << " " + << rs->rowset_id().to_string() << " in tablet " << tablet_id() + << ": " << st.to_string(); + } + } +} + } // namespace doris diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index e0f124988dea82..994f5525d669e3 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -79,7 +79,13 @@ class Tablet : public BaseTablet { const int64_t cumulative_layer_point() const; void set_cumulative_layer_point(int64_t new_point); - size_t tablet_footprint(); // disk space occupied by tablet + // Disk space occupied by tablet, contain local and remote. + size_t tablet_footprint(); + // Local disk space occupied by tablet. + size_t tablet_local_size(); + // Remote disk space occupied by tablet. + size_t tablet_remote_size(); + size_t num_rows(); int version_count() const; Version max_version() const; @@ -271,7 +277,8 @@ class Tablet : public BaseTablet { void find_alpha_rowsets(std::vector* rowsets) const; Status create_rowset_writer(const Version& version, const RowsetStatePB& rowset_state, - const SegmentsOverlapPB& overlap, + const SegmentsOverlapPB& overlap, int64_t oldest_write_timestamp, + int64_t newest_write_timestamp, std::unique_ptr* rowset_writer); Status create_rowset_writer(const int64_t& txn_id, const PUniqueId& load_id, @@ -279,6 +286,17 @@ class Tablet : public BaseTablet { std::unique_ptr* rowset_writer); Status create_rowset(RowsetMetaSharedPtr rowset_meta, RowsetSharedPtr* rowset); + // Cooldown to remote fs. + Status cooldown(); + + RowsetSharedPtr pick_cooldown_rowset(); + + bool need_cooldown(); + + bool need_cooldown(int64_t* cooldown_timestamp, size_t* file_size); + + // Physically remove remote rowsets. + void remove_all_remote_rowsets(); private: Status _init_once_action(); @@ -415,6 +433,16 @@ inline size_t Tablet::tablet_footprint() { return _tablet_meta->tablet_footprint(); } +inline size_t Tablet::tablet_local_size() { + std::shared_lock rdlock(_meta_lock); + return _tablet_meta->tablet_local_size(); +} + +inline size_t Tablet::tablet_remote_size() { + std::shared_lock rdlock(_meta_lock); + return _tablet_meta->tablet_remote_size(); +} + // TODO(lingbin): Why other methods which need to get information from _tablet_meta // are not locked, here needs a comment to explain. inline size_t Tablet::num_rows() { diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index e002b07cde4ee8..e129867f455823 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -109,9 +110,9 @@ Status TabletManager::_add_tablet_unlocked(TTabletId tablet_id, const TabletShar // During restore process, the tablet is exist and snapshot loader will replace the tablet's rowsets // and then reload the tablet, the tablet's path will the same if (!force) { - if (existed_tablet->tablet_path_desc().filepath == tablet->tablet_path_desc().filepath) { + if (existed_tablet->tablet_path() == tablet->tablet_path()) { LOG(WARNING) << "add the same tablet twice! tablet_id=" << tablet_id - << ", tablet_path=" << tablet->tablet_path_desc().filepath; + << ", tablet_path=" << tablet->tablet_path(); return Status::OLAPInternalError(OLAP_ERR_ENGINE_INSERT_EXISTS_TABLE); } if (existed_tablet->data_dir() == tablet->data_dir()) { @@ -167,8 +168,8 @@ Status TabletManager::_add_tablet_unlocked(TTabletId tablet_id, const TabletShar << ", tablet_id=" << tablet_id << ", old_version=" << old_version << ", new_version=" << new_version << ", old_time=" << old_time << ", new_time=" << new_time - << ", old_tablet_path=" << existed_tablet->tablet_path_desc().debug_string() - << ", new_tablet_path=" << tablet->tablet_path_desc().debug_string(); + << ", old_tablet_path=" << existed_tablet->tablet_path() + << ", new_tablet_path=" << tablet->tablet_path(); return res; } @@ -222,26 +223,6 @@ Status TabletManager::create_tablet(const TCreateTabletReq& request, std::vector int64_t tablet_id = request.tablet_id; LOG(INFO) << "begin to create tablet. tablet_id=" << tablet_id; - if (FilePathDesc::is_remote(request.storage_medium)) { - FilePathDesc path_desc; - path_desc.storage_medium = request.storage_medium; - path_desc.storage_name = request.storage_param.storage_name; - StorageParamPB storage_param; - Status st = StorageBackendMgr::instance()->get_storage_param( - request.storage_param.storage_name, &storage_param); - if (!st.ok() || - storage_param.DebugString() != - fs::fs_util::get_storage_param_pb(request.storage_param).DebugString()) { - LOG(INFO) << "remote storage need to change, create it. storage_name: " - << request.storage_param.storage_name; - RETURN_NOT_OK_STATUS_WITH_WARN( - StorageBackendMgr::instance()->create_remote_storage( - fs::fs_util::get_storage_param_pb(request.storage_param)), - "remote storage create failed. storage_name: " + - request.storage_param.storage_name); - } - } - std::lock_guard wrlock(_get_tablets_shard_lock(tablet_id)); TRACE("got tablets shard lock"); // Make create_tablet operation to be idempotent: @@ -272,9 +253,7 @@ Status TabletManager::create_tablet(const TCreateTabletReq& request, std::vector // If we are doing schema-change, we should use the same data dir // TODO(lingbin): A litter trick here, the directory should be determined before // entering this method - if (request.storage_medium == base_tablet->data_dir()->path_desc().storage_medium || - (FilePathDesc::is_remote(request.storage_medium) && - base_tablet->data_dir()->is_remote())) { + if (request.storage_medium == base_tablet->data_dir()->storage_medium()) { stores.clear(); stores.push_back(base_tablet->data_dir()); } @@ -744,6 +723,7 @@ Status TabletManager::load_tablet_from_meta(DataDir* data_dir, TTabletId tablet_ << ", path=" << data_dir->path(); return Status::OLAPInternalError(OLAP_ERR_HEADER_PB_PARSE_FAILED); } + tablet_meta->init_rs_metas_fs(data_dir->fs()); // check if tablet meta is valid if (tablet_meta->tablet_id() != tablet_id || tablet_meta->schema_hash() != schema_hash) { @@ -785,9 +765,9 @@ Status TabletManager::load_tablet_from_meta(DataDir* data_dir, TTabletId tablet_ // For case 2, If a tablet has just been copied to local BE, // it may be cleared by gc-thread(see perform_path_gc_by_tablet) because the tablet meta may not be loaded to memory. // So clone task should check path and then failed and retry in this case. - if (check_path && !Env::Default()->path_exists(tablet->tablet_path_desc().filepath).ok()) { + if (check_path && !Env::Default()->path_exists(tablet->tablet_path()).ok()) { LOG(WARNING) << "tablet path not exists, create tablet failed, path=" - << tablet->tablet_path_desc().filepath; + << tablet->tablet_path(); return Status::OLAPInternalError(OLAP_ERR_TABLE_ALREADY_DELETED_ERROR); } @@ -913,6 +893,7 @@ Status TabletManager::build_all_report_tablets_info(std::map TTabletStat t_tablet_stat; t_tablet_stat.__set_tablet_id(tablet_info.tablet_id); t_tablet_stat.__set_data_size(tablet_info.data_size); + t_tablet_stat.__set_remote_data_size(tablet_info.remote_data_size); t_tablet_stat.__set_row_num(tablet_info.row_count); t_tablet_stat.__set_version_count(tablet_info.version_count); local_cache->emplace_back(std::move(t_tablet_stat)); @@ -983,18 +964,15 @@ Status TabletManager::start_trash_sweep() { continue; } // move data to trash - FilePathDesc tablet_path_desc = (*it)->tablet_path_desc(); - if (Env::Default()->path_exists(tablet_path_desc.filepath).ok()) { + const auto& tablet_path = (*it)->tablet_path(); + if (Env::Default()->path_exists(tablet_path).ok()) { // take snapshot of tablet meta - string meta_file_path = path_util::join_path_segments( - tablet_path_desc.filepath, std::to_string((*it)->tablet_id()) + ".hdr"); + auto meta_file_path = fmt::format("{}/{}.hdr", tablet_path, (*it)->tablet_id()); (*it)->tablet_meta()->save(meta_file_path); - LOG(INFO) << "start to move tablet to trash. " - << tablet_path_desc.debug_string(); - Status rm_st = (*it)->data_dir()->move_to_trash(tablet_path_desc); + LOG(INFO) << "start to move tablet to trash. " << tablet_path; + Status rm_st = (*it)->data_dir()->move_to_trash(tablet_path); if (rm_st != Status::OK()) { - LOG(WARNING) - << "fail to move dir to trash. " << tablet_path_desc.debug_string(); + LOG(WARNING) << "fail to move dir to trash. " << tablet_path; ++it; continue; } @@ -1005,13 +983,13 @@ Status TabletManager::start_trash_sweep() { LOG(INFO) << "successfully move tablet to trash. " << "tablet_id=" << (*it)->tablet_id() << ", schema_hash=" << (*it)->schema_hash() - << ", tablet_path=" << tablet_path_desc.debug_string(); + << ", tablet_path=" << tablet_path; it = _shutdown_tablets.erase(it); ++clean_num; } else { // if could not find tablet info in meta store, then check if dir existed - FilePathDesc tablet_path_desc = (*it)->tablet_path_desc(); - if (Env::Default()->path_exists(tablet_path_desc.filepath).ok()) { + const auto& tablet_path = (*it)->tablet_path(); + if (Env::Default()->path_exists(tablet_path).ok()) { LOG(WARNING) << "errors while load meta from store, skip this tablet. " << "tablet_id=" << (*it)->tablet_id() << ", schema_hash=" << (*it)->schema_hash(); @@ -1020,7 +998,7 @@ Status TabletManager::start_trash_sweep() { LOG(INFO) << "could not find tablet dir, skip it and remove it from gc-queue. " << "tablet_id=" << (*it)->tablet_id() << ", schema_hash=" << (*it)->schema_hash() - << ", tablet_path=" << tablet_path_desc.filepath; + << ", tablet_path=" << tablet_path; it = _shutdown_tablets.erase(it); } } @@ -1073,58 +1051,7 @@ void TabletManager::try_delete_unused_tablet_path(DataDir* data_dir, TTabletId t // TODO(ygl): may do other checks in the future if (Env::Default()->path_exists(schema_hash_path).ok()) { LOG(INFO) << "start to move tablet to trash. tablet_path = " << schema_hash_path; - FilePathDesc segment_desc(schema_hash_path); - string remote_file_param_path = schema_hash_path + REMOTE_FILE_PARAM; - if (data_dir->is_remote() && FileUtils::check_exist(remote_file_param_path)) { - // it means you must remove remote file for this segment first - string json_buf; - Status s = env_util::read_file_to_string(Env::Default(), remote_file_param_path, - &json_buf); - if (!s.ok()) { - LOG(WARNING) << "delete unused file error when read remote_file_param_path: " - << remote_file_param_path; - return; - } - // json_buf format: {"tablet_uid": "a84cfb67d3ad3d62-87fd8b3ae9bdad84", "storage_name": "s3_name"} - std::string storage_name = nullptr; - std::string tablet_uid = nullptr; - rapidjson::Document dom; - if (!dom.Parse(json_buf.c_str()).HasParseError()) { - if (dom.HasMember(TABLET_UID.c_str()) && dom[TABLET_UID.c_str()].IsString() && - dom.HasMember(STORAGE_NAME.c_str()) && dom[STORAGE_NAME.c_str()].IsString()) { - storage_name = dom[STORAGE_NAME.c_str()].GetString(); - tablet_uid = dom[TABLET_UID.c_str()].GetString(); - } - } - if (!tablet_uid.empty() && !storage_name.empty()) { - segment_desc.storage_name = storage_name; - StorageParamPB storage_param; - if (!StorageBackendMgr::instance() - ->get_storage_param(storage_name, &storage_param) - .ok()) { - LOG(WARNING) << "storage_name is invalid: " << storage_name; - return; - } - - // remote file may be exist, check and mv it to trash - std::filesystem::path local_segment_path(schema_hash_path); - std::stringstream remote_file_stream; - remote_file_stream - << data_dir->path_desc().remote_path << DATA_PREFIX << "/" - << local_segment_path.parent_path() - .parent_path() - .filename() - .string() // shard - << "/" - << local_segment_path.parent_path().filename().string() // tablet_path - << "/" << local_segment_path.filename().string() // segment_path - << "/" << tablet_uid; - segment_desc.storage_medium = - fs::fs_util::get_t_storage_medium(storage_param.storage_medium()); - segment_desc.remote_path = remote_file_stream.str(); - } - } - Status rm_st = data_dir->move_to_trash(segment_desc); + Status rm_st = data_dir->move_to_trash(schema_hash_path); if (!rm_st.ok()) { LOG(WARNING) << "fail to move dir to trash. dir=" << schema_hash_path; } else { @@ -1136,20 +1063,20 @@ void TabletManager::try_delete_unused_tablet_path(DataDir* data_dir, TTabletId t void TabletManager::update_root_path_info(std::map* path_map, size_t* tablet_count) { - DCHECK(tablet_count != 0); + DCHECK(tablet_count); *tablet_count = 0; for (const auto& tablets_shard : _tablets_shards) { std::shared_lock rdlock(tablets_shard.lock); for (const auto& item : tablets_shard.tablet_map) { TabletSharedPtr tablet = item.second; ++(*tablet_count); - int64_t data_size = tablet->tablet_footprint(); auto iter = path_map->find(tablet->data_dir()->path()); if (iter == path_map->end()) { continue; } if (iter->second.is_used) { - iter->second.data_used_capacity += data_size; + iter->second.local_used_capacity += tablet->tablet_local_size(); + iter->second.remote_used_capacity += tablet->tablet_remote_size(); } } } @@ -1345,18 +1272,55 @@ void TabletManager::get_tablets_distribution_on_different_disks( Status TabletManager::_get_storage_param(DataDir* data_dir, const std::string& storage_name, StorageParamPB* storage_param) { - if (data_dir->is_remote()) { - RETURN_WITH_WARN_IF_ERROR( - StorageBackendMgr::instance()->get_storage_param(storage_name, storage_param), - Status::OLAPInternalError(OLAP_ERR_OTHER_ERROR), - "get_storage_param failed for storage_name: " + storage_name); - } else { - storage_param->set_storage_medium( - fs::fs_util::get_storage_medium_pb(data_dir->storage_medium())); - } + storage_param->set_storage_medium( + fs::fs_util::get_storage_medium_pb(data_dir->storage_medium())); return Status::OK(); } +struct SortCtx { + SortCtx(TabletSharedPtr tablet, int64_t cooldown_timestamp, int64_t file_size) + : tablet(tablet), cooldown_timestamp(cooldown_timestamp), file_size(file_size) {} + TabletSharedPtr tablet; + int64_t cooldown_timestamp; + int64_t file_size; +}; + +void TabletManager::get_cooldwon_tablets(std::vector* tablets) { + std::vector sort_ctx_vec; + for (const auto& tablets_shard : _tablets_shards) { + std::shared_lock rdlock(tablets_shard.lock); + for (const auto& item : tablets_shard.tablet_map) { + const TabletSharedPtr& tablet = item.second; + int64_t cooldown_timestamp = -1; + size_t file_size = -1; + if (tablet->need_cooldown(&cooldown_timestamp, &file_size)) { + sort_ctx_vec.emplace_back(tablet, cooldown_timestamp, file_size); + } + } + } + + std::sort(sort_ctx_vec.begin(), sort_ctx_vec.end(), [](SortCtx a, SortCtx b) { + if (a.cooldown_timestamp != -1 && b.cooldown_timestamp != -1) { + return a.cooldown_timestamp < b.cooldown_timestamp; + } + + if (a.cooldown_timestamp != -1 && b.cooldown_timestamp == -1) { + return true; + } + + if (a.cooldown_timestamp == -1 && b.cooldown_timestamp != -1) { + return false; + } + + return a.file_size > b.file_size; + }); + + for (SortCtx& ctx : sort_ctx_vec) { + VLOG_DEBUG << "get cooldown tablet: " << ctx.tablet->tablet_id(); + tablets->push_back(std::move(ctx.tablet)); + } +} + void TabletManager::get_all_tablets_storage_format(TCheckStorageFormatResult* result) { DCHECK(result != nullptr); for (const auto& tablets_shard : _tablets_shards) { diff --git a/be/src/olap/tablet_manager.h b/be/src/olap/tablet_manager.h index 1e2af26d99a394..0283c7c729aed1 100644 --- a/be/src/olap/tablet_manager.h +++ b/be/src/olap/tablet_manager.h @@ -137,6 +137,7 @@ class TabletManager { void get_tablets_distribution_on_different_disks( std::map>& tablets_num_on_disk, std::map>>& tablets_info_on_disk); + void get_cooldwon_tablets(std::vector* tables); void get_all_tablets_storage_format(TCheckStorageFormatResult* result); diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index ffaef7b2d1c5b1..bd3161fd306ceb 100644 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -43,7 +43,8 @@ Status TabletMeta::create(const TCreateTabletReq& request, const TabletUid& tabl request.tablet_schema.schema_hash, shard_id, request.tablet_schema, next_unique_id, col_ordinal_to_unique_id, tablet_uid, request.__isset.tablet_type ? request.tablet_type : TTabletType::TABLET_TYPE_DISK, - request.storage_medium, request.storage_param.storage_name, request.compression_type)); + request.storage_medium, request.storage_param.storage_name, request.compression_type, + request.storage_policy)); return Status::OK(); } @@ -55,7 +56,7 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id const std::unordered_map& col_ordinal_to_unique_id, TabletUid tablet_uid, TTabletType::type tabletType, TStorageMedium::type t_storage_medium, const std::string& storage_name, - TCompressionType::type compression_type) + TCompressionType::type compression_type, const std::string& storage_policy) : _tablet_uid(0, 0), _schema(new TabletSchema) { TabletMetaPB tablet_meta_pb; tablet_meta_pb.set_table_id(table_id); @@ -74,6 +75,7 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id : TabletTypePB::TABLET_TYPE_MEMORY); tablet_meta_pb.set_storage_medium(fs::fs_util::get_storage_medium_pb(t_storage_medium)); tablet_meta_pb.set_remote_storage_name(storage_name); + tablet_meta_pb.set_storage_policy(storage_policy); TabletSchemaPB* schema = tablet_meta_pb.mutable_schema(); schema->set_num_short_key_columns(tablet_schema.short_key_column_count); schema->set_num_rows_per_row_block(config::default_num_rows_per_column_file_block); @@ -194,7 +196,8 @@ TabletMeta::TabletMeta(const TabletMeta& b) _in_restore_mode(b._in_restore_mode), _preferred_rowset_type(b._preferred_rowset_type), _remote_storage_name(b._remote_storage_name), - _storage_medium(b._storage_medium) {}; + _storage_medium(b._storage_medium), + _cooldown_resource(b._cooldown_resource) {}; void TabletMeta::_init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn, ColumnPB* column) { @@ -372,6 +375,19 @@ Status TabletMeta::deserialize(const string& meta_binary) { return Status::OK(); } +void TabletMeta::init_rs_metas_fs(const io::FileSystemPtr& fs) { + for (auto& rs_meta : _rs_metas) { + if (rs_meta->is_local()) { + rs_meta->set_fs(fs); + } + } + for (auto& rs_meta : _stale_rs_metas) { + if (rs_meta->is_local()) { + rs_meta->set_fs(fs); + } + } +} + void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) { _table_id = tablet_meta_pb.table_id(); _partition_id = tablet_meta_pb.partition_id(); @@ -439,6 +455,7 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) { _remote_storage_name = tablet_meta_pb.remote_storage_name(); _storage_medium = tablet_meta_pb.storage_medium(); + _cooldown_resource = tablet_meta_pb.storage_policy(); } void TabletMeta::to_meta_pb(TabletMetaPB* tablet_meta_pb) { @@ -487,6 +504,7 @@ void TabletMeta::to_meta_pb(TabletMetaPB* tablet_meta_pb) { tablet_meta_pb->set_remote_storage_name(_remote_storage_name); tablet_meta_pb->set_storage_medium(_storage_medium); + tablet_meta_pb->set_storage_policy(_cooldown_resource); } uint32_t TabletMeta::mem_size() const { @@ -703,6 +721,7 @@ bool operator==(const TabletMeta& a, const TabletMeta& b) { if (a._preferred_rowset_type != b._preferred_rowset_type) return false; if (a._storage_medium != b._storage_medium) return false; if (a._remote_storage_name != b._remote_storage_name) return false; + if (a._cooldown_resource != b._cooldown_resource) return false; return true; } diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index 8624d04333fdd8..e0010d8f415191 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -17,6 +17,7 @@ #pragma once +#include #include #include #include @@ -24,6 +25,7 @@ #include "common/logging.h" #include "gen_cpp/olap_file.pb.h" +#include "io/fs/file_system.h" #include "olap/delete_handler.h" #include "olap/olap_common.h" #include "olap/olap_define.h" @@ -85,7 +87,8 @@ class TabletMeta { const std::unordered_map& col_ordinal_to_unique_id, TabletUid tablet_uid, TTabletType::type tabletType, TStorageMedium::type t_storage_medium, const std::string& remote_storage_name, - TCompressionType::type compression_type); + TCompressionType::type compression_type, + const std::string& storage_policy = std::string()); // If need add a filed in TableMeta, filed init copy in copy construct function TabletMeta(const TabletMeta& tablet_meta); TabletMeta(TabletMeta&& tablet_meta) = delete; @@ -103,6 +106,8 @@ class TabletMeta { Status serialize(std::string* meta_binary); Status deserialize(const std::string& meta_binary); void init_from_pb(const TabletMetaPB& tablet_meta_pb); + // Init `RowsetMeta._fs` if rowset is local. + void init_rs_metas_fs(const io::FileSystemPtr& fs); void to_meta_pb(TabletMetaPB* tablet_meta_pb); void to_json(std::string* json_string, json2pb::Pb2JsonOptions& options); @@ -123,8 +128,12 @@ class TabletMeta { void set_cumulative_layer_point(int64_t new_point); size_t num_rows() const; - // disk space occupied by tablet + // Disk space occupied by tablet, contain local and remote. size_t tablet_footprint() const; + // Local disk space occupied by tablet. + size_t tablet_local_size() const; + // Remote disk space occupied by tablet. + size_t tablet_remote_size() const; size_t version_count() const; Version max_version() const; @@ -178,6 +187,18 @@ class TabletMeta { StorageMediumPB storage_medium() const { return _storage_medium; } + const io::ResourceId& cooldown_resource() const { + std::shared_lock rlock(_meta_lock); + return _cooldown_resource; + } + + void set_cooldown_resource(io::ResourceId resource) { + std::unique_lock wlock(_meta_lock); + VLOG_NOTICE << "set tablet_id : " << _table_id << " cooldown resource from " + << _cooldown_resource << " to " << resource; + _cooldown_resource = std::move(resource); + } + private: Status _save_meta(DataDir* data_dir); void _init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn, ColumnPB* column); @@ -214,7 +235,11 @@ class TabletMeta { RowsetTypePB _preferred_rowset_type = BETA_ROWSET; std::string _remote_storage_name; StorageMediumPB _storage_medium = StorageMediumPB::HDD; - std::shared_mutex _meta_lock; + + // FIXME(cyx): Currently `cooldown_resource` is equivalent to `storage_policy`. + io::ResourceId _cooldown_resource; + + mutable std::shared_mutex _meta_lock; }; static const std::string SEQUENCE_COL = "__DORIS_SEQUENCE_COL__"; @@ -283,6 +308,26 @@ inline size_t TabletMeta::tablet_footprint() const { return total_size; } +inline size_t TabletMeta::tablet_local_size() const { + size_t total_size = 0; + for (auto& rs : _rs_metas) { + if (rs->is_local()) { + total_size += rs->data_disk_size(); + } + } + return total_size; +} + +inline size_t TabletMeta::tablet_remote_size() const { + size_t total_size = 0; + for (auto& rs : _rs_metas) { + if (!rs->is_local()) { + total_size += rs->data_disk_size(); + } + } + return total_size; +} + inline size_t TabletMeta::version_count() const { return _rs_metas.size(); } diff --git a/be/src/olap/task/engine_batch_load_task.cpp b/be/src/olap/task/engine_batch_load_task.cpp index 7add21582315b7..4213461e12ae47 100644 --- a/be/src/olap/task/engine_batch_load_task.cpp +++ b/be/src/olap/task/engine_batch_load_task.cpp @@ -149,7 +149,7 @@ 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; + *download_path = root_path + "/" + DPP_PREFIX; // Check path exist std::filesystem::path full_path(*download_path); diff --git a/be/src/olap/task/engine_clone_task.cpp b/be/src/olap/task/engine_clone_task.cpp index 84dc1e40681b2c..44085722eecfcf 100644 --- a/be/src/olap/task/engine_clone_task.cpp +++ b/be/src/olap/task/engine_clone_task.cpp @@ -87,7 +87,7 @@ Status EngineCloneTask::_do_clone() { } // get download path - string local_data_path = tablet->tablet_path_desc().filepath + CLONE_PREFIX; + auto local_data_path = fmt::format("{}/{}", tablet->tablet_path(), CLONE_PREFIX); bool allow_incremental_clone = false; // try to incremental clone @@ -274,14 +274,13 @@ void EngineCloneTask::_set_tablet_info(Status status, bool is_new_tablet) { /// 3. Convert rowset ids of downloaded snapshots. /// 4. Release the snapshots on source BE. Status EngineCloneTask::_make_and_download_snapshots(DataDir& data_dir, - const string& local_data_path, + 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(); - std::string local_path = local_data_path + "/"; const auto& token = _master_info.token; int timeout_s = 0; @@ -325,7 +324,7 @@ Status EngineCloneTask::_make_and_download_snapshots(DataDir& data_dir, remote_url_prefix = ss.str(); } - st = _download_files(&data_dir, remote_url_prefix, local_path); + 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(); @@ -336,10 +335,10 @@ Status EngineCloneTask::_make_and_download_snapshots(DataDir& data_dir, if (status.ok()) { // change all rowset ids because they maybe its id same with local rowset auto olap_st = SnapshotManager::instance()->convert_rowset_ids( - local_path, _clone_req.tablet_id, _clone_req.replica_id, + 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_path + 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; @@ -528,7 +527,7 @@ Status EngineCloneTask::_download_files(DataDir* data_dir, const std::string& re /// This method will do the following things: /// 1. Linke all files from CLONE dir to tablet dir if file does not exist in tablet dir /// 2. Call _finish_xx_clone() to revise the tablet meta. -Status EngineCloneTask::_finish_clone(Tablet* tablet, const string& clone_dir, +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; @@ -544,15 +543,14 @@ Status EngineCloneTask::_finish_clone(Tablet* tablet, const string& clone_dir, // 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.c_str(); + LOG(WARNING) << "clone dir not existed when clone. clone_dir=" << clone_dir; break; } // Load src header. // The tablet meta info is downloaded from source BE as .hdr file. // So we load it and generate cloned_tablet_meta. - string cloned_tablet_meta_file = - clone_dir + "/" + std::to_string(tablet->tablet_id()) + ".hdr"; + 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()) { @@ -576,7 +574,7 @@ Status EngineCloneTask::_finish_clone(Tablet* tablet, const string& clone_dir, } set local_files; - string tablet_dir = tablet->tablet_path_desc().filepath; + 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=" @@ -597,12 +595,12 @@ Status EngineCloneTask::_finish_clone(Tablet* tablet, const string& clone_dir, continue; } - string from = clone_dir + "/" + clone_file; - string to = tablet_dir + "/" + clone_file; + 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.c_str() << " to=" << to.c_str(); + << " from=" << from << " to=" << to; res = Status::OLAPInternalError(OLAP_ERR_OS_ERROR); break; } @@ -632,8 +630,7 @@ Status EngineCloneTask::_finish_clone(Tablet* tablet, const string& clone_dir, } } // clear clone dir - std::filesystem::path clone_dir_path(clone_dir); - std::filesystem::remove_all(clone_dir_path); + 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; @@ -769,9 +766,9 @@ Status EngineCloneTask::_finish_full_clone(Tablet* tablet, TabletMeta* cloned_ta // but some rowset is useless, so that remove them here for (auto& rs_meta_ptr : rs_metas_found_in_src) { RowsetSharedPtr rowset_to_remove; - auto s = RowsetFactory::create_rowset(&(cloned_tablet_meta->tablet_schema()), - tablet->tablet_path_desc().filepath, rs_meta_ptr, - &rowset_to_remove); + auto s = + RowsetFactory::create_rowset(&(cloned_tablet_meta->tablet_schema()), + tablet->tablet_path(), rs_meta_ptr, &rowset_to_remove); if (!s.ok()) { LOG(WARNING) << "failed to init rowset to remove: " << rs_meta_ptr->rowset_id().to_string(); diff --git a/be/src/olap/task/engine_clone_task.h b/be/src/olap/task/engine_clone_task.h index 6d714750b6de23..604563e03fda1e 100644 --- a/be/src/olap/task/engine_clone_task.h +++ b/be/src/olap/task/engine_clone_task.h @@ -51,7 +51,7 @@ class EngineCloneTask : public EngineTask { Status _finish_full_clone(Tablet* tablet, TabletMeta* cloned_tablet_meta); - Status _make_and_download_snapshots(DataDir& data_dir, const string& local_data_path, + 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, diff --git a/be/src/olap/task/engine_storage_migration_task.cpp b/be/src/olap/task/engine_storage_migration_task.cpp index ff9acb8ddd532c..ac31948439a6b2 100644 --- a/be/src/olap/task/engine_storage_migration_task.cpp +++ b/be/src/olap/task/engine_storage_migration_task.cpp @@ -200,7 +200,7 @@ Status EngineStorageMigrationTask::_migrate() { // try hold migration lock first Status res = Status::OK(); uint64_t shard = 0; - string full_path; + std::string full_path; { std::unique_lock migration_wlock(_tablet->get_migration_lock(), std::try_to_lock); @@ -229,11 +229,8 @@ Status EngineStorageMigrationTask::_migrate() { LOG(WARNING) << "fail to get shard from store: " << _dest_store->path(); return res; } - FilePathDescStream root_path_desc_s; - root_path_desc_s << _dest_store->path_desc() << DATA_PREFIX << "/" << shard; - FilePathDesc full_path_desc = SnapshotManager::instance()->get_schema_hash_full_path( - _tablet, root_path_desc_s.path_desc()); - full_path = full_path_desc.filepath; + 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)) { diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 6fda2e6a1fd7e8..27994be75fefbd 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -60,6 +60,7 @@ class WebPageHandler; class StreamLoadExecutor; class RoutineLoadTaskExecutor; class SmallFileMgr; +class StoragePolicyMgr; class BackendServiceClient; class FrontendServiceClient; @@ -145,6 +146,7 @@ class ExecEnv { LoadChannelMgr* load_channel_mgr() { return _load_channel_mgr; } LoadStreamMgr* load_stream_mgr() { return _load_stream_mgr; } SmallFileMgr* small_file_mgr() { return _small_file_mgr; } + StoragePolicyMgr* storage_policy_mgr() { return _storage_policy_mgr; } const std::vector& store_paths() const { return _store_paths; } size_t store_path_to_index(const std::string& path) { return _store_path_map[path]; } @@ -228,6 +230,7 @@ class ExecEnv { RoutineLoadTaskExecutor* _routine_load_task_executor = nullptr; SmallFileMgr* _small_file_mgr = nullptr; HeartbeatFlags* _heartbeat_flags = nullptr; + StoragePolicyMgr* _storage_policy_mgr = nullptr; }; template <> diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index a1cdf32b6379f8..f6a84b154335ad 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -24,6 +24,7 @@ #include "olap/page_cache.h" #include "olap/segment_loader.h" #include "olap/storage_engine.h" +#include "olap/storage_policy_mgr.h" #include "runtime/broker_mgr.h" #include "runtime/bufferpool/buffer_pool.h" #include "runtime/bufferpool/reservation_tracker.h" @@ -141,6 +142,7 @@ Status ExecEnv::_init(const std::vector& store_paths) { _stream_load_executor = new StreamLoadExecutor(this); _routine_load_task_executor = new RoutineLoadTaskExecutor(this); _small_file_mgr = new SmallFileMgr(this, config::small_file_dir); + _storage_policy_mgr = new StoragePolicyMgr(); _backend_client_cache->init_metrics("backend"); _frontend_client_cache->init_metrics("frontend"); diff --git a/be/src/runtime/load_path_mgr.cpp b/be/src/runtime/load_path_mgr.cpp index 061beac6053be7..726d74f9e0cb22 100644 --- a/be/src/runtime/load_path_mgr.cpp +++ b/be/src/runtime/load_path_mgr.cpp @@ -53,15 +53,12 @@ LoadPathMgr::~LoadPathMgr() { Status LoadPathMgr::init() { _path_vec.clear(); for (auto& path : _exec_env->store_paths()) { - if (FilePathDesc::is_remote(path.storage_medium)) { - continue; - } - _path_vec.push_back(path.path + MINI_PREFIX); + _path_vec.push_back(path.path + "/" + MINI_PREFIX); } LOG(INFO) << "Load path configured to [" << boost::join(_path_vec, ",") << "]"; // error log is saved in first root path - _error_log_dir = _exec_env->store_paths()[0].path + ERROR_LOG_PREFIX; + _error_log_dir = _exec_env->store_paths()[0].path + "/" + ERROR_LOG_PREFIX; // check and make dir RETURN_IF_ERROR(FileUtils::create_dir(_error_log_dir)); diff --git a/be/src/runtime/snapshot_loader.cpp b/be/src/runtime/snapshot_loader.cpp index 95126e2376173b..5c16eebcd19e23 100644 --- a/be/src/runtime/snapshot_loader.cpp +++ b/be/src/runtime/snapshot_loader.cpp @@ -362,8 +362,8 @@ Status SnapshotLoader::download(const std::map& src_to // MUST hold tablet's header lock, push lock, cumulative lock and base compaction lock Status SnapshotLoader::move(const std::string& snapshot_path, TabletSharedPtr tablet, bool overwrite) { - std::string tablet_path = tablet->tablet_path_desc().filepath; - std::string store_path = tablet->data_dir()->path_desc().filepath; + auto tablet_path = tablet->tablet_path(); + auto store_path = tablet->data_dir()->path(); LOG(INFO) << "begin to move snapshot files. from: " << snapshot_path << ", to: " << tablet_path << ", store: " << store_path << ", job: " << _job_id << ", task id: " << _task_id; @@ -396,16 +396,14 @@ Status SnapshotLoader::move(const std::string& snapshot_path, TabletSharedPtr ta return Status::InternalError(ss.str()); } - std::filesystem::path tablet_dir(tablet_path); - std::filesystem::path snapshot_dir(snapshot_path); - if (!std::filesystem::exists(tablet_dir)) { + if (!std::filesystem::exists(tablet_path)) { std::stringstream ss; ss << "tablet path does not exist: " << tablet_path; LOG(WARNING) << ss.str(); return Status::InternalError(ss.str()); } - if (!std::filesystem::exists(snapshot_dir)) { + if (!std::filesystem::exists(snapshot_path)) { std::stringstream ss; ss << "snapshot path does not exist: " << snapshot_path; LOG(WARNING) << ss.str(); @@ -432,10 +430,10 @@ Status SnapshotLoader::move(const std::string& snapshot_path, TabletSharedPtr ta // This remove seems soft enough, because we already get // tablet id and schema hash from this path, which // means this path is a valid path. - std::filesystem::remove_all(tablet_dir); - VLOG_CRITICAL << "remove dir: " << tablet_dir; - std::filesystem::create_directory(tablet_dir); - VLOG_CRITICAL << "re-create dir: " << tablet_dir; + std::filesystem::remove_all(tablet_path); + VLOG_CRITICAL << "remove dir: " << tablet_path; + std::filesystem::create_directory(tablet_path); + VLOG_CRITICAL << "re-create dir: " << tablet_path; } catch (const std::filesystem::filesystem_error& e) { std::stringstream ss; ss << "failed to move tablet path: " << tablet_path << ". err: " << e.what(); @@ -447,8 +445,8 @@ Status SnapshotLoader::move(const std::string& snapshot_path, TabletSharedPtr ta // files in snapshot dir will be moved in snapshot clean process std::vector linked_files; for (auto& file : snapshot_files) { - std::string full_src_path = snapshot_path + "/" + file; - std::string full_dest_path = tablet_path + "/" + file; + auto full_src_path = fmt::format("{}/{}", snapshot_path, file); + auto full_dest_path = fmt::format("{}/{}", tablet_path, file); if (link(full_src_path.c_str(), full_dest_path.c_str()) != 0) { LOG(WARNING) << "failed to link file from " << full_src_path << " to " << full_dest_path << ", err: " << std::strerror(errno); diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index 97e35e1597bd63..e01e62df1a2634 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -211,8 +211,7 @@ int64_t BackendService::get_trash_used_capacity() { StorageEngine::instance()->get_all_data_dir_info(&data_dir_infos, false /*do not update */); for (const auto& root_path_info : data_dir_infos) { - std::string lhs_trash_path = root_path_info.path_desc.filepath + TRASH_PREFIX; - std::filesystem::path trash_path(lhs_trash_path); + auto trash_path = fmt::format("{}/{}", root_path_info.path, TRASH_PREFIX); result += StorageEngine::instance()->get_file_or_directory_size(trash_path); } return result; @@ -225,12 +224,11 @@ void BackendService::get_disk_trash_used_capacity(std::vector& d for (const auto& root_path_info : data_dir_infos) { TDiskTrashInfo diskTrashInfo; - diskTrashInfo.__set_root_path(root_path_info.path_desc.filepath); + diskTrashInfo.__set_root_path(root_path_info.path); diskTrashInfo.__set_state(root_path_info.is_used ? "ONLINE" : "OFFLINE"); - std::string lhs_trash_path = root_path_info.path_desc.filepath + TRASH_PREFIX; - std::filesystem::path trash_path(lhs_trash_path); + auto trash_path = fmt::format("{}/{}", root_path_info.path, TRASH_PREFIX); diskTrashInfo.__set_trash_used_capacity( StorageEngine::instance()->get_file_or_directory_size(trash_path)); diff --git a/be/src/service/http_service.cpp b/be/src/service/http_service.cpp index 3ce77f6bb63e42..94f9a9862cb3a5 100644 --- a/be/src/service/http_service.cpp +++ b/be/src/service/http_service.cpp @@ -69,9 +69,6 @@ Status HttpService::start() { // register download action std::vector allow_paths; for (auto& path : _env->store_paths()) { - if (FilePathDesc::is_remote(path.storage_medium)) { - continue; - } allow_paths.emplace_back(path.path); } DownloadAction* download_action = _pool.add(new DownloadAction(_env, allow_paths)); diff --git a/be/src/util/doris_metrics.cpp b/be/src/util/doris_metrics.cpp index 8bd9b058477f55..70dfc36a396760 100644 --- a/be/src/util/doris_metrics.cpp +++ b/be/src/util/doris_metrics.cpp @@ -174,6 +174,22 @@ DEFINE_GAUGE_CORE_METRIC_PROTOTYPE_2ARG(query_cache_memory_total_byte, MetricUni DEFINE_GAUGE_CORE_METRIC_PROTOTYPE_2ARG(query_cache_sql_total_count, MetricUnit::NOUNIT); DEFINE_GAUGE_CORE_METRIC_PROTOTYPE_2ARG(query_cache_partition_total_count, MetricUnit::NOUNIT); +DEFINE_GAUGE_CORE_METRIC_PROTOTYPE_2ARG(upload_total_byte, MetricUnit::BYTES); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(upload_rowset_count, MetricUnit::ROWSETS); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(upload_fail_count, MetricUnit::ROWSETS); + +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(local_file_reader_total, MetricUnit::FILESYSTEM); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(s3_file_reader_total, MetricUnit::FILESYSTEM); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(local_file_writer_total, MetricUnit::FILESYSTEM); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(file_created_total, MetricUnit::FILESYSTEM); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(local_bytes_read_total, MetricUnit::FILESYSTEM); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(s3_bytes_read_total, MetricUnit::FILESYSTEM); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(local_bytes_written_total, MetricUnit::FILESYSTEM); + +DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(local_file_open_reading, MetricUnit::FILESYSTEM); +DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(s3_file_open_reading, MetricUnit::FILESYSTEM); +DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(local_file_open_writing, MetricUnit::FILESYSTEM); + const std::string DorisMetrics::_s_registry_name = "doris_be"; const std::string DorisMetrics::_s_hook_name = "doris_metrics"; @@ -289,11 +305,26 @@ DorisMetrics::DorisMetrics() : _metric_registry(_s_registry_name) { INT_COUNTER_METRIC_REGISTER(_server_metric_entity, switch_thread_mem_tracker_err_cb_count); INT_COUNTER_METRIC_REGISTER(_server_metric_entity, switch_bthread_count); + INT_UGAUGE_METRIC_REGISTER(_server_metric_entity, upload_total_byte); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, upload_rowset_count); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, upload_fail_count); + _server_metric_entity->register_hook(_s_hook_name, std::bind(&DorisMetrics::_update, this)); INT_UGAUGE_METRIC_REGISTER(_server_metric_entity, query_cache_memory_total_byte); INT_UGAUGE_METRIC_REGISTER(_server_metric_entity, query_cache_sql_total_count); INT_UGAUGE_METRIC_REGISTER(_server_metric_entity, query_cache_partition_total_count); + + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, local_file_reader_total); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, s3_file_reader_total); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, local_file_writer_total); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, file_created_total); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, local_bytes_read_total); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, s3_bytes_read_total); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, local_bytes_written_total); + INT_GAUGE_METRIC_REGISTER(_server_metric_entity, local_file_open_reading); + INT_GAUGE_METRIC_REGISTER(_server_metric_entity, s3_file_open_reading); + INT_GAUGE_METRIC_REGISTER(_server_metric_entity, local_file_open_writing); } void DorisMetrics::initialize(bool init_system_metrics, const std::set& disk_devices, diff --git a/be/src/util/doris_metrics.h b/be/src/util/doris_metrics.h index 726431635e85a5..34b17a7a9a9d18 100644 --- a/be/src/util/doris_metrics.h +++ b/be/src/util/doris_metrics.h @@ -170,6 +170,18 @@ class DorisMetrics { IntGauge* blocks_open_reading; IntGauge* blocks_open_writing; + // Metrics related with file reader/writer + IntCounter* local_file_reader_total; + IntCounter* s3_file_reader_total; + IntCounter* local_file_writer_total; + IntCounter* file_created_total; + IntCounter* local_bytes_read_total; + IntCounter* s3_bytes_read_total; + IntCounter* local_bytes_written_total; + IntGauge* local_file_open_reading; + IntGauge* s3_file_open_reading; + IntGauge* local_file_open_writing; + // Size of some global containers UIntGauge* rowset_count_generated_and_in_use; UIntGauge* unused_rowsets_count; @@ -207,6 +219,11 @@ class DorisMetrics { UIntGauge* send_batch_thread_pool_thread_num; UIntGauge* send_batch_thread_pool_queue_size; + // Upload metrics + UIntGauge* upload_total_byte; + IntCounter* upload_rowset_count; + IntCounter* upload_fail_count; + static DorisMetrics* instance() { static DorisMetrics instance; return &instance; diff --git a/be/src/util/file_cache.cpp b/be/src/util/file_cache.cpp index 2d85d2f945d12b..0ffc51406be714 100644 --- a/be/src/util/file_cache.cpp +++ b/be/src/util/file_cache.cpp @@ -47,9 +47,12 @@ bool FileCache::lookup(const std::string& file_name, template void FileCache::insert(const std::string& file_name, FileType* file, - OpenedFileHandle* file_handle) { + OpenedFileHandle* file_handle, + void (*deleter)(const CacheKey&, void*)) { DCHECK(_cache != nullptr); - auto deleter = [](const CacheKey& key, void* value) { delete (FileType*)value; }; + if (!deleter) { + deleter = [](const CacheKey& key, void* value) { delete (FileType*)value; }; + } CacheKey key(file_name); auto lru_handle = _cache->insert(key, file, 1, deleter); *file_handle = OpenedFileHandle(_cache.get(), lru_handle); @@ -57,5 +60,6 @@ void FileCache::insert(const std::string& file_name, FileType* file, // Explicit specialization for callers outside this compilation unit. template class FileCache; +template class FileCache; } // namespace doris diff --git a/be/src/util/file_cache.h b/be/src/util/file_cache.h index 48accca49f128a..4ebe9139ef3891 100644 --- a/be/src/util/file_cache.h +++ b/be/src/util/file_cache.h @@ -131,7 +131,8 @@ class FileCache { // insert new FileType* into lru cache // and return file_handle void insert(const std::string& file_name, FileType* file, - OpenedFileHandle* file_handle); + OpenedFileHandle* file_handle, + void (*deleter)(const CacheKey&, void*) = nullptr); private: // Name of the cache. diff --git a/be/src/util/metrics.h b/be/src/util/metrics.h index 24a640b3f07be7..648f96809f223c 100644 --- a/be/src/util/metrics.h +++ b/be/src/util/metrics.h @@ -57,7 +57,8 @@ enum class MetricUnit { ROWSETS, CONNECTIONS, PACKETS, - NOUNIT + NOUNIT, + FILESYSTEM }; std::ostream& operator<<(std::ostream& os, MetricType type); diff --git a/be/src/util/s3_uri.cpp b/be/src/util/s3_uri.cpp index 857788a6673955..177df0e1c6e97c 100644 --- a/be/src/util/s3_uri.cpp +++ b/be/src/util/s3_uri.cpp @@ -29,7 +29,7 @@ const std::string S3URI::_SCHEME_DELIM = "://"; const std::string S3URI::_PATH_DELIM = "/"; const std::string S3URI::_QUERY_DELIM = "?"; const std::string S3URI::_FRAGMENT_DELIM = "#"; -const StringCaseSet S3URI::_VALID_SCHEMES = {"http", "https", "s3", "s3a", "s3n", "bos"}; +const StringCaseSet S3URI::_VALID_SCHEMES = {"http", "https", "s3", "s3a", "s3n", "bos", "oss"}; bool S3URI::parse() { if (_location.empty()) { return false; diff --git a/be/src/util/s3_util.cpp b/be/src/util/s3_util.cpp index feb19f2e3fb245..d5bc067921eef8 100644 --- a/be/src/util/s3_util.cpp +++ b/be/src/util/s3_util.cpp @@ -88,6 +88,7 @@ std::shared_ptr ClientFactory::create( std::atoi(properties.find(S3_CONN_TIMEOUT_MS)->second.c_str()); } + aws_config.verifySSL = false; // See https://sdk.amazonaws.com/cpp/api/LATEST/class_aws_1_1_s3_1_1_s3_client.html bool use_virtual_addressing = true; if (properties.find(USE_PATH_STYLE) != properties.end()) { diff --git a/be/test/CMakeLists.txt b/be/test/CMakeLists.txt index 8928363ec67ae7..15ea119d8b65aa 100644 --- a/be/test/CMakeLists.txt +++ b/be/test/CMakeLists.txt @@ -201,6 +201,8 @@ set(OLAP_TEST_FILES olap/common_test.cpp # olap/memtable_flush_executor_test.cpp # olap/push_handler_test.cpp + olap/tablet_cooldown_test.cpp + olap/tablet_clone_test.cpp ) set(RUNTIME_TEST_FILES diff --git a/be/test/olap/delete_handler_test.cpp b/be/test/olap/delete_handler_test.cpp index 57a0ebb8f79350..b66791df871994 100644 --- a/be/test/olap/delete_handler_test.cpp +++ b/be/test/olap/delete_handler_test.cpp @@ -50,7 +50,7 @@ static void set_up() { EXPECT_NE(getcwd(buffer, MAX_PATH_LEN), nullptr); config::storage_root_path = string(buffer) + "/data_test"; FileUtils::remove_all(config::storage_root_path); - FileUtils::remove_all(string(getenv("DORIS_HOME")) + UNUSED_PREFIX); + FileUtils::remove_all(string(getenv("DORIS_HOME")) + "/" + UNUSED_PREFIX); FileUtils::create_dir(config::storage_root_path); std::vector paths; paths.emplace_back(config::storage_root_path, -1); @@ -70,7 +70,7 @@ static void tear_down() { EXPECT_NE(getcwd(buffer, MAX_PATH_LEN), nullptr); config::storage_root_path = string(buffer) + "/data_test"; FileUtils::remove_all(config::storage_root_path); - FileUtils::remove_all(string(getenv("DORIS_HOME")) + UNUSED_PREFIX); + FileUtils::remove_all(string(getenv("DORIS_HOME")) + "/" + UNUSED_PREFIX); if (k_engine != nullptr) { k_engine->stop(); delete k_engine; @@ -263,14 +263,14 @@ class TestDeleteConditionHandler : public testing::Test { EXPECT_EQ(Status::OK(), res); tablet = k_engine->tablet_manager()->get_tablet(_create_tablet.tablet_id); EXPECT_NE(tablet.get(), nullptr); - _tablet_path = tablet->tablet_path_desc().filepath; + _tablet_path = tablet->tablet_path(); set_create_duplicate_tablet_request(&_create_dup_tablet); res = k_engine->create_tablet(_create_dup_tablet); EXPECT_EQ(Status::OK(), res); dup_tablet = k_engine->tablet_manager()->get_tablet(_create_dup_tablet.tablet_id); EXPECT_TRUE(dup_tablet.get() != NULL); - _dup_tablet_path = tablet->tablet_path_desc().filepath; + _dup_tablet_path = tablet->tablet_path(); } void TearDown() { @@ -438,7 +438,7 @@ class TestDeleteConditionHandler2 : public testing::Test { EXPECT_EQ(Status::OK(), res); tablet = k_engine->tablet_manager()->get_tablet(_create_tablet.tablet_id); EXPECT_TRUE(tablet.get() != nullptr); - _tablet_path = tablet->tablet_path_desc().filepath; + _tablet_path = tablet->tablet_path(); } void TearDown() { @@ -812,7 +812,7 @@ class TestDeleteHandler : public testing::Test { EXPECT_EQ(Status::OK(), res); tablet = k_engine->tablet_manager()->get_tablet(_create_tablet.tablet_id); EXPECT_TRUE(tablet != nullptr); - _tablet_path = tablet->tablet_path_desc().filepath; + _tablet_path = tablet->tablet_path(); _data_row_cursor.init(tablet->tablet_schema()); _data_row_cursor.allocate_memory_for_string_type(tablet->tablet_schema()); diff --git a/be/test/olap/delta_writer_test.cpp b/be/test/olap/delta_writer_test.cpp index 632086f891a21d..49aea368440163 100644 --- a/be/test/olap/delta_writer_test.cpp +++ b/be/test/olap/delta_writer_test.cpp @@ -74,7 +74,7 @@ static void tear_down() { k_engine = nullptr; } EXPECT_EQ(system("rm -rf ./data_test"), 0); - FileUtils::remove_all(std::string(getenv("DORIS_HOME")) + UNUSED_PREFIX); + FileUtils::remove_all(std::string(getenv("DORIS_HOME")) + "/" + UNUSED_PREFIX); } static void create_tablet_request(int64_t tablet_id, int32_t schema_hash, diff --git a/be/test/olap/engine_storage_migration_task_test.cpp b/be/test/olap/engine_storage_migration_task_test.cpp index 36b95a8aa32afd..5eaf26392de16d 100644 --- a/be/test/olap/engine_storage_migration_task_test.cpp +++ b/be/test/olap/engine_storage_migration_task_test.cpp @@ -82,7 +82,7 @@ static void tear_down() { } EXPECT_EQ(system("rm -rf ./data_test_1"), 0); EXPECT_EQ(system("rm -rf ./data_test_2"), 0); - FileUtils::remove_all(std::string(getenv("DORIS_HOME")) + UNUSED_PREFIX); + FileUtils::remove_all(std::string(getenv("DORIS_HOME")) + "/" + UNUSED_PREFIX); } static void create_tablet_request_with_sequence_col(int64_t tablet_id, int32_t schema_hash, diff --git a/be/test/olap/memtable_flush_executor_test.cpp b/be/test/olap/memtable_flush_executor_test.cpp index 42b98b6ef6093a..f571ba5044210e 100644 --- a/be/test/olap/memtable_flush_executor_test.cpp +++ b/be/test/olap/memtable_flush_executor_test.cpp @@ -69,7 +69,7 @@ void tear_down() { delete k_engine; k_engine = nullptr; system("rm -rf ./flush_test"); - FileUtils::remove_all(std::string(getenv("DORIS_HOME")) + UNUSED_PREFIX); + FileUtils::remove_all(std::string(getenv("DORIS_HOME")) + "/" + UNUSED_PREFIX); } Schema create_schema() { diff --git a/be/test/olap/rowset/alpha_rowset_test.cpp b/be/test/olap/rowset/alpha_rowset_test.cpp index c5f82a100f0814..0f6a342e20bffd 100644 --- a/be/test/olap/rowset/alpha_rowset_test.cpp +++ b/be/test/olap/rowset/alpha_rowset_test.cpp @@ -94,7 +94,7 @@ void create_rowset_writer_context(TabletSchema* tablet_schema, rowset_writer_context->tablet_schema_hash = 1111; rowset_writer_context->partition_id = 10; rowset_writer_context->rowset_type = ALPHA_ROWSET; - rowset_writer_context->path_desc.filepath = config::storage_root_path + "/data/0/12345/1111"; + rowset_writer_context->tablet_path = config::storage_root_path + "/data/0/12345/1111"; rowset_writer_context->rowset_state = VISIBLE; rowset_writer_context->tablet_schema = tablet_schema; rowset_writer_context->version.first = 0; diff --git a/be/test/olap/rowset/beta_rowset_test.cpp b/be/test/olap/rowset/beta_rowset_test.cpp index 53b62c3fa9eeaa..9ca70ae7dd68a5 100644 --- a/be/test/olap/rowset/beta_rowset_test.cpp +++ b/be/test/olap/rowset/beta_rowset_test.cpp @@ -14,11 +14,16 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. +#include +#include + +#include #include #include #include "gen_cpp/olap_file.pb.h" #include "gtest/gtest.h" +#include "io/fs/s3_file_system.h" #include "olap/comparison_predicate.h" #include "olap/data_dir.h" #include "olap/row_block.h" @@ -43,9 +48,14 @@ namespace doris { static const uint32_t MAX_PATH_LEN = 1024; StorageEngine* k_engine = nullptr; +static const std::string kTestDir = "./data_test/data/beta_rowset_test"; class BetaRowsetTest : public testing::Test { public: + BetaRowsetTest() : _data_dir(std::make_unique(kTestDir)) { + _data_dir->update_capacity(); + } + static void TearDownTestSuite() { if (k_engine != nullptr) { k_engine->stop(); @@ -80,8 +90,7 @@ class BetaRowsetTest : public testing::Test { ExecEnv* exec_env = doris::ExecEnv::GetInstance(); exec_env->set_storage_engine(k_engine); - const std::string rowset_dir = "./data_test/data/beta_rowset_test"; - EXPECT_TRUE(FileUtils::create_dir(rowset_dir).ok()); + EXPECT_TRUE(FileUtils::create_dir(kTestDir).ok()); } void TearDown() override { @@ -138,12 +147,13 @@ class BetaRowsetTest : public testing::Test { RowsetWriterContext* rowset_writer_context) { RowsetId rowset_id; rowset_id.init(10000); + // rowset_writer_context->data_dir = _data_dir.get(); rowset_writer_context->rowset_id = rowset_id; rowset_writer_context->tablet_id = 12345; rowset_writer_context->tablet_schema_hash = 1111; rowset_writer_context->partition_id = 10; rowset_writer_context->rowset_type = BETA_ROWSET; - rowset_writer_context->path_desc.filepath = "./data_test/data/beta_rowset_test"; + rowset_writer_context->tablet_path = kTestDir; rowset_writer_context->rowset_state = VISIBLE; rowset_writer_context->tablet_schema = tablet_schema; rowset_writer_context->version.first = 10; @@ -159,6 +169,9 @@ class BetaRowsetTest : public testing::Test { s = (*result)->init(&context); EXPECT_EQ(Status::OK(), s); } + +private: + std::unique_ptr _data_dir; }; TEST_F(BetaRowsetTest, BasicFunctionTest) { @@ -359,4 +372,121 @@ TEST_F(BetaRowsetTest, BasicFunctionTest) { } } +class S3ClientMock : public Aws::S3::S3Client { + S3ClientMock() {} + S3ClientMock(const Aws::Auth::AWSCredentials& credentials, + const Aws::Client::ClientConfiguration& clientConfiguration, + Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy signPayloads, + bool use_virtual_addressing) + : Aws::S3::S3Client(credentials, clientConfiguration, signPayloads, + use_virtual_addressing) {} + + Aws::S3::Model::HeadObjectOutcome HeadObject( + const Aws::S3::Model::HeadObjectRequest& request) const override { + Aws::S3::Model::HeadObjectOutcome response; + response.success = false; + return response; + } + + Aws::S3::Model::GetObjectOutcome GetObject( + const Aws::S3::Model::GetObjectRequest& request) const override { + Aws::S3::Model::GetObjectOutcome response; + response.success = false; + return response; + } +}; + +class S3ClientMockGetError : public S3ClientMock { + Aws::S3::Model::HeadObjectOutcome HeadObject( + const Aws::S3::Model::HeadObjectRequest& request) const override { + Aws::S3::Model::HeadObjectOutcome response; + response.GetResult().SetContentLength(20); + response.success = true; + return response; + } +}; + +class S3ClientMockGetErrorData : public S3ClientMock { + Aws::S3::Model::HeadObjectOutcome HeadObject( + const Aws::S3::Model::HeadObjectRequest& request) const override { + Aws::S3::Model::HeadObjectOutcome response; + response.GetResult().SetContentLength(20); + response.success = true; + return response; + } + + Aws::S3::Model::GetObjectOutcome GetObject( + const Aws::S3::Model::GetObjectRequest& request) const override { + Aws::S3::Model::GetObjectOutcome response; + response.GetResult().SetContentLength(4); + response.success = true; + return response; + } +}; + +TEST_F(BetaRowsetTest, ReadTest) { + RowsetMetaSharedPtr rowset_meta = std::make_shared(); + BetaRowset rowset(nullptr, "", rowset_meta); + std::map properties { + {"AWS_ACCESS_KEY", "ak"}, + {"AWS_SECRET_KEY", "ak"}, + {"AWS_ENDPOINT", "endpoint"}, + {"AWS_REGION", "region"}, + }; + io::ResourceId resource_id = "test_resourse_id"; + std::shared_ptr fs = + std::make_shared(properties, "bucket", "test prefix", resource_id); + Aws::SDKOptions aws_options = Aws::SDKOptions {}; + Aws::InitAPI(aws_options); + + // failed to head object + { + Aws::Auth::AWSCredentials aws_cred("ak", "sk"); + Aws::Client::ClientConfiguration aws_config; + fs->_client.reset( + new S3ClientMock(aws_cred, aws_config, + Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, true)); + + rowset.rowset_meta()->set_num_segments(1); + rowset.rowset_meta()->set_resource_id(resource_id); + rowset.rowset_meta()->set_fs(fs); + + std::vector segments; + Status st = rowset.load_segments(&segments); + ASSERT_FALSE(st.ok()); + } + + // failed to get object + { + Aws::Auth::AWSCredentials aws_cred("ak", "sk"); + Aws::Client::ClientConfiguration aws_config; + fs->_client.reset(new S3ClientMockGetError()); + + rowset.rowset_meta()->set_num_segments(1); + rowset.rowset_meta()->set_resource_id(resource_id); + rowset.rowset_meta()->set_fs(fs); + + std::vector segments; + Status st = rowset.load_segments(&segments); + ASSERT_FALSE(st.ok()); + } + + // get error data + { + Aws::Auth::AWSCredentials aws_cred("ak", "sk"); + Aws::Client::ClientConfiguration aws_config; + fs->_client.reset(new S3ClientMockGetErrorData()); + + rowset.rowset_meta()->set_num_segments(1); + rowset.rowset_meta()->set_resource_id(resource_id); + rowset.rowset_meta()->set_fs(fs); + + std::vector segments; + Status st = rowset.load_segments(&segments); + ASSERT_FALSE(st.ok()); + } + + Aws::ShutdownAPI(aws_options); +} + } // namespace doris diff --git a/be/test/olap/rowset/segment_v2/bitmap_index_test.cpp b/be/test/olap/rowset/segment_v2/bitmap_index_test.cpp index 5a82ffba70838c..e335034c9196df 100644 --- a/be/test/olap/rowset/segment_v2/bitmap_index_test.cpp +++ b/be/test/olap/rowset/segment_v2/bitmap_index_test.cpp @@ -17,12 +17,14 @@ #include +#include #include #include "common/logging.h" #include "env/env.h" -#include "olap/fs/block_manager.h" -#include "olap/fs/fs_util.h" +#include "io/fs/file_system.h" +#include "io/fs/file_writer.h" +#include "io/fs/local_file_system.h" #include "olap/key_coder.h" #include "olap/olap_common.h" #include "olap/rowset/segment_v2/bitmap_index_reader.h" @@ -38,6 +40,7 @@ using roaring::Roaring; class BitmapIndexTest : public testing::Test { public: const std::string kTestDir = "./ut_dir/bitmap_index_test"; + void SetUp() override { if (FileUtils::check_exist(kTestDir)) { EXPECT_TRUE(FileUtils::remove_all(kTestDir).ok()); @@ -52,29 +55,27 @@ class BitmapIndexTest : public testing::Test { }; template -void write_index_file(std::string& filename, const void* values, size_t value_count, - size_t null_count, ColumnIndexMetaPB* meta) { +void write_index_file(const std::string& filename, io::FileSystem* fs, const void* values, + size_t value_count, size_t null_count, ColumnIndexMetaPB* meta) { const auto* type_info = get_scalar_type_info(); { - std::unique_ptr wblock; - fs::CreateBlockOptions opts(filename); - std::string storage_name; - EXPECT_TRUE(fs::fs_util::block_manager(storage_name)->create_block(opts, &wblock).ok()); + std::unique_ptr file_writer; + EXPECT_TRUE(fs->create_file(filename, &file_writer).ok()); std::unique_ptr writer; BitmapIndexWriter::create(type_info, &writer); writer->add_values(values, value_count); writer->add_nulls(null_count); - EXPECT_TRUE(writer->finish(wblock.get(), meta).ok()); + EXPECT_TRUE(writer->finish(file_writer.get(), meta).ok()); EXPECT_EQ(BITMAP_INDEX, meta->type()); - EXPECT_TRUE(wblock->close().ok()); + EXPECT_TRUE(file_writer->close().ok()); } } template -void get_bitmap_reader_iter(std::string& file_name, const ColumnIndexMetaPB& meta, +void get_bitmap_reader_iter(const std::string& file_name, const ColumnIndexMetaPB& meta, BitmapIndexReader** reader, BitmapIndexIterator** iter) { - *reader = new BitmapIndexReader(file_name, &meta.bitmap_index()); + *reader = new BitmapIndexReader(io::global_local_filesystem(), file_name, &meta.bitmap_index()); auto st = (*reader)->load(true, false); EXPECT_TRUE(st.ok()); @@ -91,7 +92,8 @@ TEST_F(BitmapIndexTest, test_invert) { std::string file_name = kTestDir + "/invert"; ColumnIndexMetaPB meta; - write_index_file(file_name, val, num_uint8_rows, 0, &meta); + write_index_file(file_name, io::global_local_filesystem(), val, + num_uint8_rows, 0, &meta); { std::unique_ptr rfile; BitmapIndexReader* reader = nullptr; @@ -146,7 +148,8 @@ TEST_F(BitmapIndexTest, test_invert_2) { std::string file_name = kTestDir + "/invert2"; ColumnIndexMetaPB meta; - write_index_file(file_name, val, num_uint8_rows, 0, &meta); + write_index_file(file_name, io::global_local_filesystem(), val, + num_uint8_rows, 0, &meta); { BitmapIndexReader* reader = nullptr; @@ -182,7 +185,8 @@ TEST_F(BitmapIndexTest, test_multi_pages) { std::string file_name = kTestDir + "/mul"; ColumnIndexMetaPB meta; - write_index_file(file_name, val, num_uint8_rows, 0, &meta); + write_index_file(file_name, io::global_local_filesystem(), val, + num_uint8_rows, 0, &meta); { BitmapIndexReader* reader = nullptr; BitmapIndexIterator* iter = nullptr; @@ -213,7 +217,8 @@ TEST_F(BitmapIndexTest, test_null) { std::string file_name = kTestDir + "/null"; ColumnIndexMetaPB meta; - write_index_file(file_name, val, num_uint8_rows, 30, &meta); + write_index_file(file_name, io::global_local_filesystem(), val, + num_uint8_rows, 30, &meta); { BitmapIndexReader* reader = nullptr; BitmapIndexIterator* iter = nullptr; diff --git a/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp b/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp index 9815fb79bf2108..e1b8780d2f313f 100644 --- a/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp +++ b/be/test/olap/rowset/segment_v2/bloom_filter_index_reader_writer_test.cpp @@ -19,6 +19,9 @@ #include "common/logging.h" #include "env/env.h" +#include "io/fs/file_system.h" +#include "io/fs/file_writer.h" +#include "io/fs/local_file_system.h" #include "olap/fs/block_manager.h" #include "olap/fs/fs_util.h" #include "olap/key_coder.h" @@ -36,15 +39,15 @@ const std::string dname = "./ut_dir/bloom_filter_index_reader_writer_test"; class BloomFilterIndexReaderWriterTest : public testing::Test { public: - virtual void SetUp() { - if (FileUtils::is_dir(dname)) { - std::set files; - EXPECT_TRUE(FileUtils::list_dirs_files(dname, nullptr, &files, Env::Default()).ok()); - for (const auto& file : files) { - Status s = Env::Default()->delete_file(dname + "/" + file); - EXPECT_TRUE(s.ok()) << s.to_string(); - } - EXPECT_TRUE(Env::Default()->delete_dir(dname).ok()); + void SetUp() override { + if (FileUtils::check_exist(dname)) { + EXPECT_TRUE(FileUtils::remove_all(dname).ok()); + } + EXPECT_TRUE(FileUtils::create_dir(dname).ok()); + } + void TearDown() override { + if (FileUtils::check_exist(dname)) { + EXPECT_TRUE(FileUtils::remove_all(dname).ok()); } } }; @@ -55,13 +58,11 @@ void write_bloom_filter_index_file(const std::string& file_name, const void* val ColumnIndexMetaPB* index_meta) { const auto* type_info = get_scalar_type_info(); using CppType = typename CppTypeTraits::CppType; - FileUtils::create_dir(dname); std::string fname = dname + "/" + file_name; + auto fs = io::global_local_filesystem(); { - std::unique_ptr wblock; - fs::CreateBlockOptions opts(fname); - std::string storage_name; - Status st = fs::fs_util::block_manager(storage_name)->create_block(opts, &wblock); + std::unique_ptr file_writer; + Status st = fs->create_file(fname, &file_writer); EXPECT_TRUE(st.ok()) << st.to_string(); std::unique_ptr bloom_filter_index_writer; @@ -79,9 +80,9 @@ void write_bloom_filter_index_file(const std::string& file_name, const void* val EXPECT_TRUE(st.ok()); i += 1024; } - st = bloom_filter_index_writer->finish(wblock.get(), index_meta); + st = bloom_filter_index_writer->finish(file_writer.get(), index_meta); EXPECT_TRUE(st.ok()) << "writer finish status:" << st.to_string(); - EXPECT_TRUE(wblock->close().ok()); + EXPECT_TRUE(file_writer->close().ok()); EXPECT_EQ(BLOOM_FILTER_INDEX, index_meta->type()); EXPECT_EQ(bf_options.strategy, index_meta->bloom_filter_index().hash_strategy()); } @@ -92,8 +93,8 @@ void get_bloom_filter_reader_iter(const std::string& file_name, const ColumnInde BloomFilterIndexReader** reader, std::unique_ptr* iter) { std::string fname = dname + "/" + file_name; - - *reader = new BloomFilterIndexReader(fname, &meta.bloom_filter_index()); + auto fs = io::global_local_filesystem(); + *reader = new BloomFilterIndexReader(fs, fname, &meta.bloom_filter_index()); auto st = (*reader)->load(true, false); EXPECT_TRUE(st.ok()); @@ -106,7 +107,7 @@ void test_bloom_filter_index_reader_writer_template( const std::string file_name, typename TypeTraits::CppType* val, size_t num, size_t null_num, typename TypeTraits::CppType* not_exist_value, bool is_slice_type = false) { - typedef typename TypeTraits::CppType CppType; + using CppType = typename TypeTraits::CppType; ColumnIndexMetaPB meta; write_bloom_filter_index_file(file_name, val, num, null_num, &meta); { diff --git a/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp b/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp index 126346d6638131..62af770e09b678 100644 --- a/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp +++ b/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp @@ -20,6 +20,9 @@ #include #include "env/env.h" +#include "io/fs/file_system.h" +#include "io/fs/file_writer.h" +#include "io/fs/local_file_system.h" #include "olap/column_block.h" #include "olap/decimal12.h" #include "olap/fs/fs_util.h" @@ -50,7 +53,7 @@ static const std::string TEST_DIR = "./ut_dir/column_reader_writer_test"; class ColumnReaderWriterTest : public testing::Test { public: ColumnReaderWriterTest() : _tracker(new MemTracker()), _pool(_tracker.get()) {} - virtual ~ColumnReaderWriterTest() {} + ~ColumnReaderWriterTest() override = default; protected: void SetUp() override { @@ -82,11 +85,10 @@ void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, // write data std::string fname = TEST_DIR + "/" + test_name; + auto fs = io::global_local_filesystem(); { - std::unique_ptr wblock; - fs::CreateBlockOptions opts(fname); - std::string storage_name; - Status st = fs::fs_util::block_manager(storage_name)->create_block(opts, &wblock); + std::unique_ptr file_writer; + Status st = fs->create_file(fname, &file_writer); EXPECT_TRUE(st.ok()) << st.get_error_msg(); ColumnWriterOptions writer_opts; @@ -111,7 +113,7 @@ void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, column = create_char_key(1); } std::unique_ptr writer; - ColumnWriter::create(writer_opts, &column, wblock.get(), &writer); + ColumnWriter::create(writer_opts, &column, file_writer.get(), &writer); st = writer->init(); EXPECT_TRUE(st.ok()) << st.to_string(); @@ -126,7 +128,7 @@ void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, EXPECT_TRUE(writer->write_zone_map().ok()); // close the file - EXPECT_TRUE(wblock->close().ok()); + EXPECT_TRUE(file_writer->close().ok()); } auto type_info = get_scalar_type_info(type); // read and check @@ -134,24 +136,21 @@ void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, // sequence read { ColumnReaderOptions reader_opts; - FilePathDesc path_desc; - path_desc.filepath = fname; std::unique_ptr reader; - auto st = ColumnReader::create(reader_opts, meta, num_rows, path_desc, &reader); + auto st = ColumnReader::create(reader_opts, meta, num_rows, fs, fname, &reader); EXPECT_TRUE(st.ok()); ColumnIterator* iter = nullptr; st = reader->new_iterator(&iter); EXPECT_TRUE(st.ok()); - std::unique_ptr rblock; - fs::BlockManager* block_manager = fs::fs_util::block_manager(path_desc); - block_manager->open_block(path_desc, &rblock); + std::unique_ptr file_reader; + st = fs->open_file(fname, &file_reader); EXPECT_TRUE(st.ok()); ColumnIteratorOptions iter_opts; OlapReaderStatistics stats; iter_opts.stats = &stats; - iter_opts.rblock = rblock.get(); + iter_opts.file_reader = file_reader.get(); st = iter->init(iter_opts); EXPECT_TRUE(st.ok()); @@ -194,24 +193,22 @@ void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, { ColumnReaderOptions reader_opts; - FilePathDesc path_desc; - path_desc.filepath = fname; std::unique_ptr reader; - auto st = ColumnReader::create(reader_opts, meta, num_rows, path_desc, &reader); + auto st = ColumnReader::create(reader_opts, meta, num_rows, fs, fname, &reader); EXPECT_TRUE(st.ok()); ColumnIterator* iter = nullptr; st = reader->new_iterator(&iter); EXPECT_TRUE(st.ok()); - std::unique_ptr rblock; - fs::BlockManager* block_manager = fs::fs_util::block_manager(path_desc); - block_manager->open_block(path_desc, &rblock); + + std::unique_ptr rblock; + st = fs->open_file(fname, &rblock); EXPECT_TRUE(st.ok()); ColumnIteratorOptions iter_opts; OlapReaderStatistics stats; iter_opts.stats = &stats; - iter_opts.rblock = rblock.get(); + iter_opts.file_reader = rblock.get(); st = iter->init(iter_opts); EXPECT_TRUE(st.ok()); @@ -267,11 +264,10 @@ void test_array_nullable_data(CollectionValue* src_data, uint8_t* src_is_null, i // write data std::string fname = TEST_DIR + "/" + test_name; + auto fs = io::global_local_filesystem(); { - std::unique_ptr wblock; - fs::CreateBlockOptions opts(fname); - std::string storage_name; - Status st = fs::fs_util::block_manager(storage_name)->create_block(opts, &wblock); + std::unique_ptr file_writer; + Status st = fs->create_file(fname, &file_writer); EXPECT_TRUE(st.ok()) << st.get_error_msg(); ColumnWriterOptions writer_opts; @@ -296,7 +292,7 @@ void test_array_nullable_data(CollectionValue* src_data, uint8_t* src_is_null, i child_meta->set_is_nullable(true); std::unique_ptr writer; - ColumnWriter::create(writer_opts, &list_column, wblock.get(), &writer); + ColumnWriter::create(writer_opts, &list_column, file_writer.get(), &writer); st = writer->init(); EXPECT_TRUE(st.ok()) << st.to_string(); @@ -314,30 +310,28 @@ void test_array_nullable_data(CollectionValue* src_data, uint8_t* src_is_null, i EXPECT_TRUE(st.ok()); // close the file - EXPECT_TRUE(wblock->close().ok()); + EXPECT_TRUE(file_writer->close().ok()); } auto type_info = get_type_info(&meta); // read and check { ColumnReaderOptions reader_opts; - FilePathDesc path_desc; - path_desc.filepath = fname; std::unique_ptr reader; - auto st = ColumnReader::create(reader_opts, meta, num_rows, path_desc, &reader); + auto st = ColumnReader::create(reader_opts, meta, num_rows, fs, fname, &reader); EXPECT_TRUE(st.ok()); ColumnIterator* iter = nullptr; st = reader->new_iterator(&iter); EXPECT_TRUE(st.ok()); - std::unique_ptr rblock; - fs::BlockManager* block_manager = fs::fs_util::block_manager(path_desc); - st = block_manager->open_block(path_desc, &rblock); + + std::unique_ptr rblock; + st = fs->open_file(fname, &rblock); EXPECT_TRUE(st.ok()); ColumnIteratorOptions iter_opts; OlapReaderStatistics stats; iter_opts.stats = &stats; - iter_opts.rblock = rblock.get(); + iter_opts.file_reader = rblock.get(); st = iter->init(iter_opts); EXPECT_TRUE(st.ok()); // sequence read diff --git a/be/test/olap/rowset/segment_v2/ordinal_page_index_test.cpp b/be/test/olap/rowset/segment_v2/ordinal_page_index_test.cpp index 85b3ea54584ff6..20f0b17d9a8915 100644 --- a/be/test/olap/rowset/segment_v2/ordinal_page_index_test.cpp +++ b/be/test/olap/rowset/segment_v2/ordinal_page_index_test.cpp @@ -25,6 +25,9 @@ #include "common/logging.h" #include "env/env.h" +#include "io/fs/file_system.h" +#include "io/fs/file_writer.h" +#include "io/fs/local_file_system.h" #include "olap/fs/fs_util.h" #include "olap/page_cache.h" #include "util/file_utils.h" @@ -51,6 +54,7 @@ class OrdinalPageIndexTest : public testing::Test { TEST_F(OrdinalPageIndexTest, normal) { std::string filename = kTestDir + "/normal.idx"; + auto fs = io::global_local_filesystem(); OrdinalIndexWriter builder; // generate ordinal index for 16K data pages, @@ -61,21 +65,18 @@ TEST_F(OrdinalPageIndexTest, normal) { } ColumnIndexMetaPB index_meta; { - std::unique_ptr wblock; - fs::CreateBlockOptions opts(filename); - std::string storage_name; - EXPECT_TRUE(fs::fs_util::block_manager(storage_name)->create_block(opts, &wblock).ok()); + std::unique_ptr file_writer; + EXPECT_TRUE(fs->create_file(filename, &file_writer).ok()); - EXPECT_TRUE(builder.finish(wblock.get(), &index_meta).ok()); + EXPECT_TRUE(builder.finish(file_writer.get(), &index_meta).ok()); EXPECT_EQ(ORDINAL_INDEX, index_meta.type()); EXPECT_FALSE(index_meta.ordinal_index().root_page().is_root_data_page()); - EXPECT_TRUE(wblock->close().ok()); + EXPECT_TRUE(file_writer->close().ok()); LOG(INFO) << "index page size=" << index_meta.ordinal_index().root_page().root_page().size(); } - FilePathDesc path_desc(filename); - OrdinalIndexReader index(path_desc, &index_meta.ordinal_index(), 16 * 1024 * 4096 + 1); + OrdinalIndexReader index(fs, filename, &index_meta.ordinal_index(), 16 * 1024 * 4096 + 1); EXPECT_TRUE(index.load(true, false).ok()); EXPECT_EQ(16 * 1024, index.num_data_pages()); EXPECT_EQ(1, index.get_first_ordinal(0)); @@ -129,8 +130,8 @@ TEST_F(OrdinalPageIndexTest, one_data_page) { EXPECT_EQ(data_page_pointer, root_page_pointer); } - FilePathDesc path_desc; - OrdinalIndexReader index(path_desc, &index_meta.ordinal_index(), num_values); + auto fs = io::global_local_filesystem(); + OrdinalIndexReader index(fs, "", &index_meta.ordinal_index(), num_values); EXPECT_TRUE(index.load(true, false).ok()); EXPECT_EQ(1, index.num_data_pages()); EXPECT_EQ(0, index.get_first_ordinal(0)); diff --git a/be/test/olap/rowset/segment_v2/segment_test.cpp b/be/test/olap/rowset/segment_v2/segment_test.cpp index e651efa890a6a6..e8069882331374 100644 --- a/be/test/olap/rowset/segment_v2/segment_test.cpp +++ b/be/test/olap/rowset/segment_v2/segment_test.cpp @@ -24,7 +24,9 @@ #include #include "common/logging.h" -#include "gutil/strings/substitute.h" +#include "io/fs/file_system.h" +#include "io/fs/file_writer.h" +#include "io/fs/local_file_system.h" #include "olap/comparison_predicate.h" #include "olap/data_dir.h" #include "olap/fs/block_manager.h" @@ -43,6 +45,7 @@ #include "runtime/mem_tracker.h" #include "testutil/test_util.h" #include "util/file_utils.h" + namespace doris { namespace segment_v2 { @@ -109,15 +112,16 @@ class SegmentReaderWriterTest : public ::testing::Test { static int seg_id = 0; // must use unique filename for each segment, otherwise page cache kicks in and produces // the wrong answer (it use (filename,offset) as cache key) - std::string filename = strings::Substitute("$0/seg_$1.dat", kSegmentDir, seg_id++); - std::unique_ptr wblock; - fs::CreateBlockOptions block_opts(filename); - std::string storage_name; - Status st = fs::fs_util::block_manager(storage_name)->create_block(block_opts, &wblock); + std::string filename = fmt::format("seg_{}.dat", seg_id++); + std::string path = fmt::format("{}/{}", kSegmentDir, filename); + auto fs = io::global_local_filesystem(); + + std::unique_ptr file_writer; + Status st = fs->create_file(path, &file_writer); EXPECT_TRUE(st.ok()); DataDir data_dir(kSegmentDir); data_dir.init(); - SegmentWriter writer(wblock.get(), 0, &build_schema, &data_dir, INT32_MAX, opts); + SegmentWriter writer(file_writer.get(), 0, &build_schema, &data_dir, INT32_MAX, opts); st = writer.init(10); EXPECT_TRUE(st.ok()); @@ -137,16 +141,14 @@ class SegmentReaderWriterTest : public ::testing::Test { uint64_t file_size, index_size; st = writer.finalize(&file_size, &index_size); EXPECT_TRUE(st.ok()); - EXPECT_TRUE(wblock->close().ok()); + EXPECT_TRUE(file_writer->close().ok()); - FilePathDesc path_desc; - path_desc.filepath = filename; - st = Segment::open(path_desc, 0, &query_schema, res); + st = Segment::open(fs, path, 0, &query_schema, res); EXPECT_TRUE(st.ok()); EXPECT_EQ(nrows, (*res)->num_rows()); } -private: +public: const std::string kSegmentDir = "./ut_dir/segment_test"; }; @@ -169,7 +171,7 @@ TEST_F(SegmentReaderWriterTest, normal) { StorageReadOptions read_opts; read_opts.stats = &stats; std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); @@ -225,7 +227,7 @@ TEST_F(SegmentReaderWriterTest, normal) { read_opts.stats = &stats; read_opts.key_ranges.emplace_back(lower_bound.get(), false, upper_bound.get(), true); std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 100); EXPECT_TRUE(iter->next_batch(&block).ok()); @@ -251,7 +253,7 @@ TEST_F(SegmentReaderWriterTest, normal) { read_opts.stats = &stats; read_opts.key_ranges.emplace_back(lower_bound.get(), false, nullptr, false); std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 100); EXPECT_TRUE(iter->next_batch(&block).is_end_of_file()); @@ -280,7 +282,7 @@ TEST_F(SegmentReaderWriterTest, normal) { read_opts.stats = &stats; read_opts.key_ranges.emplace_back(lower_bound.get(), false, upper_bound.get(), false); std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 100); EXPECT_TRUE(iter->next_batch(&block).is_end_of_file()); @@ -317,7 +319,7 @@ TEST_F(SegmentReaderWriterTest, LazyMaterialization) { read_opts.stats = &stats; std::unique_ptr iter; - EXPECT_TRUE(segment->new_iterator(read_schema, read_opts, &iter).ok()); + ASSERT_TRUE(segment->new_iterator(read_schema, read_opts, &iter).ok()); RowBlockV2 block(read_schema, 1024); EXPECT_TRUE(iter->next_batch(&block).ok()); @@ -341,7 +343,7 @@ TEST_F(SegmentReaderWriterTest, LazyMaterialization) { read_opts.stats = &stats; std::unique_ptr iter; - EXPECT_TRUE(segment->new_iterator(read_schema, read_opts, &iter).ok()); + ASSERT_TRUE(segment->new_iterator(read_schema, read_opts, &iter).ok()); RowBlockV2 block(read_schema, 1024); EXPECT_TRUE(iter->next_batch(&block).ok()); @@ -361,7 +363,7 @@ TEST_F(SegmentReaderWriterTest, LazyMaterialization) { read_opts.stats = &stats; std::unique_ptr iter; - EXPECT_TRUE(segment->new_iterator(read_schema, read_opts, &iter).ok()); + ASSERT_TRUE(segment->new_iterator(read_schema, read_opts, &iter).ok()); RowBlockV2 block(read_schema, 1024); EXPECT_TRUE(iter->next_batch(&block).ok()); @@ -393,7 +395,7 @@ TEST_F(SegmentReaderWriterTest, LazyMaterialization) { read_opts.stats = &stats; std::unique_ptr iter; - EXPECT_TRUE(segment->new_iterator(read_schema, read_opts, &iter).ok()); + ASSERT_TRUE(segment->new_iterator(read_schema, read_opts, &iter).ok()); RowBlockV2 block(read_schema, 1024); EXPECT_TRUE(iter->next_batch(&block).ok()); @@ -455,7 +457,7 @@ TEST_F(SegmentReaderWriterTest, TestIndex) { read_opts.conditions = conditions.get(); std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1); @@ -478,7 +480,7 @@ TEST_F(SegmentReaderWriterTest, TestIndex) { read_opts.conditions = conditions.get(); std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); @@ -538,7 +540,7 @@ TEST_F(SegmentReaderWriterTest, TestIndex) { read_opts.delete_conditions.push_back(delete_conditions.get()); std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); @@ -586,7 +588,7 @@ TEST_F(SegmentReaderWriterTest, TestIndex) { EXPECT_EQ(Status::OK(), conditions->append_condition(condition)); read_opts.conditions = conditions.get(); std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); EXPECT_TRUE(iter->next_batch(&block).is_end_of_file()); @@ -608,23 +610,18 @@ TEST_F(SegmentReaderWriterTest, estimate_segment_size) { tablet_schema->_cols.push_back(create_int_key(3)); tablet_schema->_cols.push_back(create_int_value(4)); - // segment write - std::string dname = "./ut_dir/segment_write_size"; - FileUtils::remove_all(dname); - FileUtils::create_dir(dname); - SegmentWriterOptions opts; opts.num_rows_per_block = num_rows_per_block; - std::string fname = dname + "/int_case"; - std::unique_ptr wblock; - fs::CreateBlockOptions wblock_opts(fname); - std::string storage_name; - Status st = fs::fs_util::block_manager(storage_name)->create_block(wblock_opts, &wblock); + std::string fname = kSegmentDir + "/int_case"; + auto fs = io::global_local_filesystem(); + + std::unique_ptr file_writer; + Status st = fs->create_file(fname, &file_writer); EXPECT_TRUE(st.ok()) << st.to_string(); - DataDir data_dir(dname); + DataDir data_dir(kSegmentDir); data_dir.init(); - SegmentWriter writer(wblock.get(), 0, tablet_schema.get(), &data_dir, INT32_MAX, opts); + SegmentWriter writer(file_writer.get(), 0, tablet_schema.get(), &data_dir, INT32_MAX, opts); st = writer.init(10); EXPECT_TRUE(st.ok()) << st.to_string(); @@ -650,14 +647,12 @@ TEST_F(SegmentReaderWriterTest, estimate_segment_size) { uint64_t file_size = 0; uint64_t index_size; EXPECT_TRUE(writer.finalize(&file_size, &index_size).ok()); - EXPECT_TRUE(wblock->close().ok()); + EXPECT_TRUE(file_writer->close().ok()); file_size = std::filesystem::file_size(fname); LOG(INFO) << "segment file size is:" << file_size; EXPECT_NE(segment_size, 0); - - FileUtils::remove_all(dname); } TEST_F(SegmentReaderWriterTest, TestDefaultValueColumn) { @@ -682,7 +677,7 @@ TEST_F(SegmentReaderWriterTest, TestDefaultValueColumn) { StorageReadOptions read_opts; read_opts.stats = &stats; std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); @@ -732,7 +727,7 @@ TEST_F(SegmentReaderWriterTest, TestDefaultValueColumn) { StorageReadOptions read_opts; read_opts.stats = &stats; std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); @@ -782,22 +777,18 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) { tablet_schema->_cols.push_back(create_varchar_key(4)); tablet_schema->init_field_index_for_test(); - // segment write - std::string dname = "./ut_dir/segment_test"; - FileUtils::create_dir(dname); - SegmentWriterOptions opts; opts.num_rows_per_block = num_rows_per_block; - std::string fname = dname + "/string_case"; - std::unique_ptr wblock; - fs::CreateBlockOptions wblock_opts(fname); - std::string storage_name; - Status st = fs::fs_util::block_manager(storage_name)->create_block(wblock_opts, &wblock); + std::string fname = kSegmentDir + "/string_case"; + auto fs = io::global_local_filesystem(); + + std::unique_ptr file_writer; + Status st = fs->create_file(fname, &file_writer); EXPECT_TRUE(st.ok()); - DataDir data_dir(dname); + DataDir data_dir(kSegmentDir); data_dir.init(); - SegmentWriter writer(wblock.get(), 0, tablet_schema.get(), &data_dir, INT32_MAX, opts); + SegmentWriter writer(file_writer.get(), 0, tablet_schema.get(), &data_dir, INT32_MAX, opts); st = writer.init(10); EXPECT_TRUE(st.ok()); @@ -824,13 +815,11 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) { uint64_t file_size = 0; uint64_t index_size; EXPECT_TRUE(writer.finalize(&file_size, &index_size).ok()); - EXPECT_TRUE(wblock->close().ok()); + EXPECT_TRUE(file_writer->close().ok()); { std::shared_ptr segment; - FilePathDesc path_desc; - path_desc.filepath = fname; - st = Segment::open(path_desc, 0, tablet_schema.get(), &segment); + st = Segment::open(fs, fname, 0, tablet_schema.get(), &segment); EXPECT_TRUE(st.ok()); EXPECT_EQ(4096, segment->num_rows()); Schema schema(*tablet_schema); @@ -840,7 +829,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) { StorageReadOptions read_opts; read_opts.stats = &stats; std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); @@ -893,7 +882,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) { read_opts.stats = &stats; read_opts.key_ranges.emplace_back(lower_bound.get(), false, nullptr, false); std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 100); st = iter->next_batch(&block); @@ -926,7 +915,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) { read_opts.stats = &stats; read_opts.key_ranges.emplace_back(lower_bound.get(), false, upper_bound.get(), false); std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 100); st = iter->next_batch(&block); @@ -950,7 +939,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) { read_opts.conditions = conditions.get(); std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); int left = 4 * 1024; @@ -1007,7 +996,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) { read_opts.conditions = conditions.get(); std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); @@ -1016,8 +1005,6 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) { EXPECT_EQ(0, block.num_rows()); } } - - FileUtils::remove_all(dname); } TEST_F(SegmentReaderWriterTest, TestBitmapPredicate) { @@ -1046,7 +1033,7 @@ TEST_F(SegmentReaderWriterTest, TestBitmapPredicate) { read_opts.stats = &stats; std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); EXPECT_TRUE(iter->next_batch(&block).ok()); @@ -1068,7 +1055,7 @@ TEST_F(SegmentReaderWriterTest, TestBitmapPredicate) { read_opts.stats = &stats; std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); EXPECT_TRUE(iter->next_batch(&block).ok()); @@ -1090,7 +1077,7 @@ TEST_F(SegmentReaderWriterTest, TestBitmapPredicate) { read_opts.stats = &stats; std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); EXPECT_FALSE(iter->next_batch(&block).ok()); @@ -1114,7 +1101,7 @@ TEST_F(SegmentReaderWriterTest, TestBitmapPredicate) { read_opts.stats = &stats; std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); EXPECT_TRUE(iter->next_batch(&block).ok()); @@ -1137,7 +1124,7 @@ TEST_F(SegmentReaderWriterTest, TestBitmapPredicate) { read_opts.stats = &stats; std::unique_ptr iter; - segment->new_iterator(schema, read_opts, &iter); + ASSERT_TRUE(segment->new_iterator(schema, read_opts, &iter).ok()); RowBlockV2 block(schema, 1024); diff --git a/be/test/olap/rowset/segment_v2/zone_map_index_test.cpp b/be/test/olap/rowset/segment_v2/zone_map_index_test.cpp index 1aa71e4e32ac65..81e30bfad3c417 100644 --- a/be/test/olap/rowset/segment_v2/zone_map_index_test.cpp +++ b/be/test/olap/rowset/segment_v2/zone_map_index_test.cpp @@ -24,6 +24,9 @@ #include "common/config.h" #include "env/env.h" +#include "io/fs/file_system.h" +#include "io/fs/file_writer.h" +#include "io/fs/local_file_system.h" #include "olap/fs/block_manager.h" #include "olap/fs/fs_util.h" #include "olap/page_cache.h" @@ -51,6 +54,7 @@ class ColumnZoneMapTest : public testing::Test { void test_string(std::string testname, Field* field) { std::string filename = kTestDir + "/" + testname; + auto fs = io::global_local_filesystem(); ZoneMapIndexWriter builder(field); std::vector values1 = {"aaaa", "bbbb", "cccc", "dddd", "eeee", "ffff"}; @@ -73,16 +77,14 @@ class ColumnZoneMapTest : public testing::Test { // write out zone map index ColumnIndexMetaPB index_meta; { - std::unique_ptr wblock; - fs::CreateBlockOptions opts(filename); - std::string storage_name; - EXPECT_TRUE(fs::fs_util::block_manager(storage_name)->create_block(opts, &wblock).ok()); - EXPECT_TRUE(builder.finish(wblock.get(), &index_meta).ok()); + std::unique_ptr file_writer; + EXPECT_TRUE(fs->create_file(filename, &file_writer).ok()); + EXPECT_TRUE(builder.finish(file_writer.get(), &index_meta).ok()); EXPECT_EQ(ZONE_MAP_INDEX, index_meta.type()); - EXPECT_TRUE(wblock->close().ok()); + EXPECT_TRUE(file_writer->close().ok()); } - ZoneMapIndexReader column_zone_map(filename, &index_meta.zone_map_index()); + ZoneMapIndexReader column_zone_map(fs, filename, &index_meta.zone_map_index()); Status status = column_zone_map.load(true, false); EXPECT_TRUE(status.ok()); EXPECT_EQ(3, column_zone_map.num_pages()); @@ -104,6 +106,7 @@ class ColumnZoneMapTest : public testing::Test { void test_cut_zone_map(std::string testname, Field* field) { std::string filename = kTestDir + "/" + testname; + auto fs = io::global_local_filesystem(); ZoneMapIndexWriter builder(field); char ch = 'a'; @@ -118,16 +121,14 @@ class ColumnZoneMapTest : public testing::Test { // write out zone map index ColumnIndexMetaPB index_meta; { - std::unique_ptr wblock; - fs::CreateBlockOptions opts(filename); - std::string storage_name; - EXPECT_TRUE(fs::fs_util::block_manager(storage_name)->create_block(opts, &wblock).ok()); - EXPECT_TRUE(builder.finish(wblock.get(), &index_meta).ok()); + std::unique_ptr file_writer; + EXPECT_TRUE(fs->create_file(filename, &file_writer).ok()); + EXPECT_TRUE(builder.finish(file_writer.get(), &index_meta).ok()); EXPECT_EQ(ZONE_MAP_INDEX, index_meta.type()); - EXPECT_TRUE(wblock->close().ok()); + EXPECT_TRUE(file_writer->close().ok()); } - ZoneMapIndexReader column_zone_map(filename, &index_meta.zone_map_index()); + ZoneMapIndexReader column_zone_map(fs, filename, &index_meta.zone_map_index()); Status status = column_zone_map.load(true, false); EXPECT_TRUE(status.ok()); EXPECT_EQ(1, column_zone_map.num_pages()); @@ -148,6 +149,7 @@ class ColumnZoneMapTest : public testing::Test { // Test for int TEST_F(ColumnZoneMapTest, NormalTestIntPage) { std::string filename = kTestDir + "/NormalTestIntPage"; + auto fs = io::global_local_filesystem(); TabletColumn int_column = create_int_key(0); Field* field = FieldFactory::create(int_column); @@ -169,16 +171,14 @@ TEST_F(ColumnZoneMapTest, NormalTestIntPage) { // write out zone map index ColumnIndexMetaPB index_meta; { - std::unique_ptr wblock; - fs::CreateBlockOptions opts({filename}); - std::string storage_name; - EXPECT_TRUE(fs::fs_util::block_manager(storage_name)->create_block(opts, &wblock).ok()); - EXPECT_TRUE(builder.finish(wblock.get(), &index_meta).ok()); + std::unique_ptr file_writer; + EXPECT_TRUE(fs->create_file(filename, &file_writer).ok()); + EXPECT_TRUE(builder.finish(file_writer.get(), &index_meta).ok()); EXPECT_EQ(ZONE_MAP_INDEX, index_meta.type()); - EXPECT_TRUE(wblock->close().ok()); + EXPECT_TRUE(file_writer->close().ok()); } - ZoneMapIndexReader column_zone_map(filename, &index_meta.zone_map_index()); + ZoneMapIndexReader column_zone_map(fs, filename, &index_meta.zone_map_index()); Status status = column_zone_map.load(true, false); EXPECT_TRUE(status.ok()); EXPECT_EQ(3, column_zone_map.num_pages()); diff --git a/be/test/olap/tablet_clone_test.cpp b/be/test/olap/tablet_clone_test.cpp new file mode 100644 index 00000000000000..fd2ef4bac1a1c4 --- /dev/null +++ b/be/test/olap/tablet_clone_test.cpp @@ -0,0 +1,203 @@ +#include + +#include + +#include "common/status.h" +#include "io/fs/file_system_map.h" +#include "io/fs/s3_file_system.h" +#include "olap/delta_writer.h" +#include "olap/snapshot_manager.h" +#include "olap/storage_engine.h" +#include "olap/tablet.h" +#include "runtime/descriptor_helper.h" +#include "runtime/tuple.h" +#include "util/file_utils.h" +#include "util/s3_util.h" + +namespace doris { + +static StorageEngine* k_engine = nullptr; + +static const std::string kTestDir = "./ut_dir/tablet_clone_test"; +static std::string kSnapshotDir = "./ut_dir/tablet_clone_test/snapshot"; +static const std::string kResourceId = "TabletCloneTest"; +static const int64_t kTabletId = 10005; +static const int32_t KSchemaHash = 270068377; + +static const std::string AK = "ak"; +static const std::string SK = "sk"; +static const std::string ENDPOINT = "endpoint"; +static const std::string REGION = "region"; +static const std::string BUCKET = "bucket"; +static const std::string PREFIX = "prefix"; + +// remove DISABLED_ when need run this test +#define TabletCloneTest DISABLED_TabletCloneTest +#define private public +class TabletCloneTest : public testing::Test { +public: + static void SetUpTestSuite() { + std::map properties = { + {S3_AK, AK}, {S3_SK, SK}, {S3_ENDPOINT, ENDPOINT}, {S3_REGION, REGION}}; + auto s3_fs = std::make_shared(properties, BUCKET, PREFIX, kResourceId); + s3_fs->connect(); + io::FileSystemMap::instance()->insert(kResourceId, s3_fs); + + config::storage_root_path = kTestDir; + config::min_file_descriptor_number = 1000; + FileUtils::remove_all(kTestDir); + FileUtils::create_dir(kTestDir); + + std::vector paths {{kTestDir, -1}}; + + EngineOptions options; + options.store_paths = paths; + doris::StorageEngine::open(options, &k_engine); + k_engine->start_bg_threads(); + } + + static void TearDownTestSuite() { + if (k_engine != nullptr) { + k_engine->stop(); + delete k_engine; + k_engine = nullptr; + } + } +}; + +static void create_tablet_request_with_sequence_col(int64_t tablet_id, int32_t schema_hash, + TCreateTabletReq* request) { + request->tablet_id = tablet_id; + request->__set_version(1); + request->tablet_schema.schema_hash = schema_hash; + request->tablet_schema.short_key_column_count = 2; + request->tablet_schema.keys_type = TKeysType::UNIQUE_KEYS; + request->tablet_schema.storage_type = TStorageType::COLUMN; + request->tablet_schema.__set_sequence_col_idx(2); + request->__set_storage_format(TStorageFormat::V2); + + TColumn k1; + k1.column_name = "k1"; + k1.__set_is_key(true); + k1.column_type.type = TPrimitiveType::TINYINT; + request->tablet_schema.columns.push_back(k1); + + TColumn k2; + k2.column_name = "k2"; + k2.__set_is_key(true); + k2.column_type.type = TPrimitiveType::SMALLINT; + request->tablet_schema.columns.push_back(k2); + + TColumn sequence_col; + sequence_col.column_name = SEQUENCE_COL; + sequence_col.__set_is_key(false); + sequence_col.column_type.type = TPrimitiveType::INT; + sequence_col.__set_aggregation_type(TAggregationType::REPLACE); + request->tablet_schema.columns.push_back(sequence_col); + + TColumn v1; + v1.column_name = "v1"; + v1.__set_is_key(false); + v1.column_type.type = TPrimitiveType::DATETIME; + v1.__set_aggregation_type(TAggregationType::REPLACE); + request->tablet_schema.columns.push_back(v1); +} + +static TDescriptorTable create_descriptor_tablet_with_sequence_col() { + TDescriptorTableBuilder desc_tbl_builder; + TTupleDescriptorBuilder tuple_builder; + + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_TINYINT).column_name("k1").column_pos(0).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_SMALLINT).column_name("k2").column_pos(1).build()); + tuple_builder.add_slot(TSlotDescriptorBuilder() + .type(TYPE_INT) + .column_name(SEQUENCE_COL) + .column_pos(2) + .build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_DATETIME).column_name("v1").column_pos(3).build()); + tuple_builder.build(&desc_tbl_builder); + + return desc_tbl_builder.desc_tbl(); +} + +TEST_F(TabletCloneTest, convert_rowset_ids_has_file_in_s3) { + TCreateTabletReq request; + create_tablet_request_with_sequence_col(kTabletId, KSchemaHash, &request); + Status st = k_engine->create_tablet(request); + ASSERT_EQ(Status::OK(), st); + + TDescriptorTable tdesc_tbl = create_descriptor_tablet_with_sequence_col(); + ObjectPool obj_pool; + DescriptorTbl* desc_tbl = nullptr; + DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + auto& slots = tuple_desc->slots(); + + PUniqueId load_id; + load_id.set_hi(0); + load_id.set_lo(0); + WriteRequest write_req = {kTabletId, KSchemaHash, WriteType::LOAD, 20003, + 30003, load_id, tuple_desc, &(tuple_desc->slots())}; + DeltaWriter* delta_writer = nullptr; + DeltaWriter::open(&write_req, &delta_writer); + ASSERT_NE(delta_writer, nullptr); + + MemTracker tracker; + MemPool pool(&tracker); + // Tuple 1 + { + Tuple* tuple = reinterpret_cast(pool.allocate(tuple_desc->byte_size())); + memset(tuple, 0, tuple_desc->byte_size()); + *(int8_t*)(tuple->get_slot(slots[0]->tuple_offset())) = 123; + *(int16_t*)(tuple->get_slot(slots[1]->tuple_offset())) = 456; + *(int32_t*)(tuple->get_slot(slots[2]->tuple_offset())) = 1; + ((DateTimeValue*)(tuple->get_slot(slots[3]->tuple_offset()))) + ->from_date_str("2020-07-16 19:39:43", 19); + + st = delta_writer->write(tuple); + ASSERT_EQ(Status::OK(), st); + } + + st = delta_writer->close(); + ASSERT_EQ(Status::OK(), st); + st = delta_writer->close_wait(); + ASSERT_EQ(Status::OK(), st); + + // publish version success + TabletSharedPtr tablet = + k_engine->tablet_manager()->get_tablet(write_req.tablet_id, write_req.schema_hash); + OlapMeta* meta = tablet->data_dir()->get_meta(); + Version version; + version.first = tablet->rowset_with_max_version()->end_version() + 1; + version.second = tablet->rowset_with_max_version()->end_version() + 1; + std::map tablet_related_rs; + StorageEngine::instance()->txn_manager()->get_txn_related_tablets( + write_req.txn_id, write_req.partition_id, &tablet_related_rs); + for (auto& tablet_rs : tablet_related_rs) { + RowsetSharedPtr rowset = tablet_rs.second; + rowset->rowset_meta()->set_resource_id(kResourceId); + st = k_engine->txn_manager()->publish_txn(meta, write_req.partition_id, write_req.txn_id, + write_req.tablet_id, write_req.schema_hash, + tablet_rs.first.tablet_uid, version); + ASSERT_EQ(Status::OK(), st); + st = tablet->add_inc_rowset(rowset); + ASSERT_EQ(Status::OK(), st); + } + EXPECT_EQ(1, tablet->num_rows()); + + TSnapshotRequest snapshot_req; + snapshot_req.tablet_id = kTabletId; + snapshot_req.schema_hash = KSchemaHash; + bool allow_incremental_clone = false; + st = SnapshotManager::instance()->_create_snapshot_files(tablet, snapshot_req, &kSnapshotDir, + &allow_incremental_clone); + ASSERT_EQ(Status::OK(), st); + st = SnapshotManager::instance()->convert_rowset_ids(kTestDir, kTabletId, KSchemaHash); + ASSERT_NE(Status::OK(), st); + delete delta_writer; +} + +} // namespace doris diff --git a/be/test/olap/tablet_cooldown_test.cpp b/be/test/olap/tablet_cooldown_test.cpp new file mode 100644 index 00000000000000..9ebc7159ef7d8b --- /dev/null +++ b/be/test/olap/tablet_cooldown_test.cpp @@ -0,0 +1,202 @@ +#include + +#include + +#include "common/config.h" +#include "common/status.h" +#include "io/fs/file_system_map.h" +#include "io/fs/s3_file_system.h" +#include "olap/delta_writer.h" +#include "olap/storage_engine.h" +#include "olap/storage_policy_mgr.h" +#include "olap/tablet.h" +#include "runtime/descriptor_helper.h" +#include "runtime/tuple.h" +#include "util/file_utils.h" +#include "util/s3_util.h" + +namespace doris { + +static StorageEngine* k_engine = nullptr; + +static const std::string kTestDir = "./ut_dir/tablet_cooldown_test"; +static const std::string kResourceId = "TabletCooldownTest"; + +static const std::string AK = "ak"; +static const std::string SK = "sk"; +static const std::string ENDPOINT = "endpoint"; +static const std::string REGION = "region"; +static const std::string BUCKET = "bucket"; +static const std::string PREFIX = "tablet_cooldown_test"; + +// remove DISABLED_ when need run this test +#define TabletCooldownTest DISABLED_TabletCooldownTest +class TabletCooldownTest : public testing::Test { +public: + static void SetUpTestSuite() { + std::map properties = { + {S3_AK, AK}, {S3_SK, SK}, {S3_ENDPOINT, ENDPOINT}, {S3_REGION, REGION}}; + auto s3_fs = std::make_shared(properties, BUCKET, PREFIX, kResourceId); + s3_fs->connect(); + io::FileSystemMap::instance()->insert(kResourceId, s3_fs); + + config::storage_root_path = kTestDir; + config::min_file_descriptor_number = 1000; + config::quick_cooldown = true; + + FileUtils::remove_all(kTestDir); + FileUtils::create_dir(kTestDir); + + std::vector paths {{kTestDir, -1}}; + + EngineOptions options; + options.store_paths = paths; + + ExecEnv::GetInstance()->_storage_policy_mgr = new StoragePolicyMgr(); + + doris::StorageEngine::open(options, &k_engine); + k_engine->start_bg_threads(); + } + + static void TearDownTestSuite() { + if (k_engine != nullptr) { + k_engine->stop(); + delete k_engine; + k_engine = nullptr; + } + } +}; + +static void create_tablet_request_with_sequence_col(int64_t tablet_id, int32_t schema_hash, + TCreateTabletReq* request) { + request->tablet_id = tablet_id; + request->__set_version(1); + request->tablet_schema.schema_hash = schema_hash; + request->tablet_schema.short_key_column_count = 2; + request->tablet_schema.keys_type = TKeysType::UNIQUE_KEYS; + request->tablet_schema.storage_type = TStorageType::COLUMN; + request->tablet_schema.__set_sequence_col_idx(2); + request->__set_storage_format(TStorageFormat::V2); + + TColumn k1; + k1.column_name = "k1"; + k1.__set_is_key(true); + k1.column_type.type = TPrimitiveType::TINYINT; + request->tablet_schema.columns.push_back(k1); + + TColumn k2; + k2.column_name = "k2"; + k2.__set_is_key(true); + k2.column_type.type = TPrimitiveType::SMALLINT; + request->tablet_schema.columns.push_back(k2); + + TColumn sequence_col; + sequence_col.column_name = SEQUENCE_COL; + sequence_col.__set_is_key(false); + sequence_col.column_type.type = TPrimitiveType::INT; + sequence_col.__set_aggregation_type(TAggregationType::REPLACE); + request->tablet_schema.columns.push_back(sequence_col); + + TColumn v1; + v1.column_name = "v1"; + v1.__set_is_key(false); + v1.column_type.type = TPrimitiveType::DATETIME; + v1.__set_aggregation_type(TAggregationType::REPLACE); + request->tablet_schema.columns.push_back(v1); +} + +static TDescriptorTable create_descriptor_tablet_with_sequence_col() { + TDescriptorTableBuilder desc_tbl_builder; + TTupleDescriptorBuilder tuple_builder; + + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_TINYINT).column_name("k1").column_pos(0).build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_SMALLINT).column_name("k2").column_pos(1).build()); + tuple_builder.add_slot(TSlotDescriptorBuilder() + .type(TYPE_INT) + .column_name(SEQUENCE_COL) + .column_pos(2) + .build()); + tuple_builder.add_slot( + TSlotDescriptorBuilder().type(TYPE_DATETIME).column_name("v1").column_pos(3).build()); + tuple_builder.build(&desc_tbl_builder); + + return desc_tbl_builder.desc_tbl(); +} + +TEST_F(TabletCooldownTest, normal) { + TCreateTabletReq request; + create_tablet_request_with_sequence_col(10005, 270068377, &request); + Status st = k_engine->create_tablet(request); + ASSERT_EQ(Status::OK(), st); + + TDescriptorTable tdesc_tbl = create_descriptor_tablet_with_sequence_col(); + ObjectPool obj_pool; + DescriptorTbl* desc_tbl = nullptr; + DescriptorTbl::create(&obj_pool, tdesc_tbl, &desc_tbl); + TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); + auto& slots = tuple_desc->slots(); + + PUniqueId load_id; + load_id.set_hi(0); + load_id.set_lo(0); + WriteRequest write_req = {10005, 270068377, WriteType::LOAD, 20003, + 30003, load_id, tuple_desc, &(tuple_desc->slots())}; + DeltaWriter* delta_writer = nullptr; + DeltaWriter::open(&write_req, &delta_writer); + ASSERT_NE(delta_writer, nullptr); + + MemTracker tracker; + MemPool pool(&tracker); + // Tuple 1 + { + Tuple* tuple = reinterpret_cast(pool.allocate(tuple_desc->byte_size())); + memset(tuple, 0, tuple_desc->byte_size()); + *(int8_t*)(tuple->get_slot(slots[0]->tuple_offset())) = 123; + *(int16_t*)(tuple->get_slot(slots[1]->tuple_offset())) = 456; + *(int32_t*)(tuple->get_slot(slots[2]->tuple_offset())) = 1; + ((DateTimeValue*)(tuple->get_slot(slots[3]->tuple_offset()))) + ->from_date_str("2020-07-16 19:39:43", 19); + + st = delta_writer->write(tuple); + ASSERT_EQ(Status::OK(), st); + } + + st = delta_writer->close(); + ASSERT_EQ(Status::OK(), st); + st = delta_writer->close_wait(); + ASSERT_EQ(Status::OK(), st); + + // publish version success + TabletSharedPtr tablet = + k_engine->tablet_manager()->get_tablet(write_req.tablet_id, write_req.schema_hash); + OlapMeta* meta = tablet->data_dir()->get_meta(); + Version version; + version.first = tablet->rowset_with_max_version()->end_version() + 1; + version.second = tablet->rowset_with_max_version()->end_version() + 1; + std::map tablet_related_rs; + StorageEngine::instance()->txn_manager()->get_txn_related_tablets( + write_req.txn_id, write_req.partition_id, &tablet_related_rs); + for (auto& tablet_rs : tablet_related_rs) { + RowsetSharedPtr rowset = tablet_rs.second; + st = k_engine->txn_manager()->publish_txn(meta, write_req.partition_id, write_req.txn_id, + write_req.tablet_id, write_req.schema_hash, + tablet_rs.first.tablet_uid, version); + ASSERT_EQ(Status::OK(), st); + st = tablet->add_inc_rowset(rowset); + ASSERT_EQ(Status::OK(), st); + } + EXPECT_EQ(1, tablet->num_rows()); + + tablet->set_cooldown_resource(kResourceId); + st = tablet->cooldown(); // rowset [0-1] + ASSERT_EQ(Status::OK(), st); + st = tablet->cooldown(); // rowset [2-2] + ASSERT_EQ(Status::OK(), st); + ASSERT_EQ(DorisMetrics::instance()->upload_rowset_count->value(), 1); + + delete delta_writer; +} + +} // namespace doris diff --git a/be/test/olap/tablet_mgr_test.cpp b/be/test/olap/tablet_mgr_test.cpp index 6ac994e7cd3be5..5da3af27f718f3 100644 --- a/be/test/olap/tablet_mgr_test.cpp +++ b/be/test/olap/tablet_mgr_test.cpp @@ -108,7 +108,7 @@ TEST_F(TabletMgrTest, CreateTablet) { TabletSharedPtr tablet = _tablet_mgr->get_tablet(111); EXPECT_TRUE(tablet != nullptr); // check dir exist - bool dir_exist = FileUtils::check_exist(tablet->tablet_path_desc().filepath); + bool dir_exist = FileUtils::check_exist(tablet->tablet_path()); EXPECT_TRUE(dir_exist); // check meta has this tablet TabletMetaSharedPtr new_tablet_meta(new TabletMeta()); @@ -167,8 +167,8 @@ TEST_F(TabletMgrTest, CreateTabletWithSequence) { TabletSharedPtr tablet = _tablet_mgr->get_tablet(111); EXPECT_TRUE(tablet != nullptr); // check dir exist - bool dir_exist = FileUtils::check_exist(tablet->tablet_path_desc().filepath); - EXPECT_TRUE(dir_exist) << tablet->tablet_path_desc().filepath; + bool dir_exist = FileUtils::check_exist(tablet->tablet_path()); + EXPECT_TRUE(dir_exist) << tablet->tablet_path(); // check meta has this tablet TabletMetaSharedPtr new_tablet_meta(new TabletMeta()); Status check_meta_st = TabletMetaManager::get_meta(_data_dir, 111, 3333, new_tablet_meta); @@ -222,7 +222,7 @@ TEST_F(TabletMgrTest, DropTablet) { EXPECT_TRUE(tablet != nullptr); // check dir exist - std::string tablet_path = tablet->tablet_path_desc().filepath; + std::string tablet_path = tablet->tablet_path(); bool dir_exist = FileUtils::check_exist(tablet_path); EXPECT_TRUE(dir_exist); diff --git a/be/test/olap/tablet_test.cpp b/be/test/olap/tablet_test.cpp index 9d6d36c42edd43..18af72f7e6a923 100644 --- a/be/test/olap/tablet_test.cpp +++ b/be/test/olap/tablet_test.cpp @@ -22,7 +22,10 @@ #include #include "olap/olap_define.h" +#include "olap/rowset/beta_rowset.h" +#include "olap/storage_policy_mgr.h" #include "olap/tablet_meta.h" +#include "util/time.h" using namespace std; @@ -97,6 +100,17 @@ class TestTablet : public testing::Test { pb1->set_creation_time(10000); } + void init_rs_meta(RowsetMetaSharedPtr& pb1, int64_t start, int64_t end, int64_t earliest_ts, + int64_t latest_ts) { + pb1->init_from_json(_json_rowset_meta); + pb1->set_oldest_write_timestamp(earliest_ts); + pb1->set_newest_write_timestamp(latest_ts); + pb1->set_start_version(start); + pb1->set_end_version(end); + pb1->set_creation_time(10000); + pb1->set_num_segments(2); + } + void init_all_rs_meta(std::vector* rs_metas) { RowsetMetaSharedPtr ptr1(new RowsetMeta()); init_rs_meta(ptr1, 0, 0); @@ -118,6 +132,7 @@ class TestTablet : public testing::Test { init_rs_meta(ptr5, 10, 11); rs_metas->push_back(ptr5); } + void fetch_expired_row_rs_meta(std::vector* rs_metas) { RowsetMetaSharedContainerPtr v2(new std::vector()); RowsetMetaSharedPtr ptr1(new RowsetMeta()); @@ -193,4 +208,129 @@ TEST_F(TestTablet, delete_expired_stale_rowset) { EXPECT_EQ(0, _tablet->_timestamped_version_tracker._stale_version_path_map.size()); _tablet.reset(); } + +TEST_F(TestTablet, cooldown_policy) { + std::vector rs_metas; + RowsetMetaSharedPtr ptr1(new RowsetMeta()); + init_rs_meta(ptr1, 1, 2, 100, 200); + rs_metas.push_back(ptr1); + RowsetSharedPtr rowset1 = make_shared(nullptr, "", ptr1); + + RowsetMetaSharedPtr ptr2(new RowsetMeta()); + init_rs_meta(ptr2, 3, 4, 300, 600); + rs_metas.push_back(ptr2); + RowsetSharedPtr rowset2 = make_shared(nullptr, "", ptr2); + + RowsetMetaSharedPtr ptr3(new RowsetMeta()); + init_rs_meta(ptr3, 5, 5, 800, 800); + rs_metas.push_back(ptr3); + RowsetSharedPtr rowset3 = make_shared(nullptr, "", ptr3); + + RowsetMetaSharedPtr ptr4(new RowsetMeta()); + init_rs_meta(ptr4, 6, 7, 1100, 1400); + rs_metas.push_back(ptr4); + RowsetSharedPtr rowset4 = make_shared(nullptr, "", ptr4); + + RowsetMetaSharedPtr ptr5(new RowsetMeta()); + init_rs_meta(ptr5, 8, 9, 1800, 2000); + rs_metas.push_back(ptr5); + RowsetSharedPtr rowset5 = make_shared(nullptr, "", ptr5); + + for (auto& rowset : rs_metas) { + _tablet_meta->add_rs_meta(rowset); + } + + StorageParamPB storage_param; + storage_param.set_storage_medium(StorageMediumPB::HDD); + TabletSharedPtr _tablet(new Tablet(_tablet_meta, storage_param, nullptr)); + _tablet->init(); + _tablet->set_cooldown_resource("test_policy_name"); + + _tablet->_rs_version_map[ptr1->version()] = rowset1; + _tablet->_rs_version_map[ptr2->version()] = rowset2; + _tablet->_rs_version_map[ptr3->version()] = rowset3; + _tablet->_rs_version_map[ptr4->version()] = rowset4; + _tablet->_rs_version_map[ptr5->version()] = rowset5; + + _tablet->set_cumulative_layer_point(20); + + ExecEnv::GetInstance()->_storage_policy_mgr = new StoragePolicyMgr(); + + { + StoragePolicy* policy = new StoragePolicy(); + policy->storage_policy_name = "test_policy_name"; + policy->cooldown_datetime = 250; + policy->cooldown_ttl = -1; + + std::shared_ptr policy_ptr; + policy_ptr.reset(policy); + + ExecEnv::GetInstance()->storage_policy_mgr()->_policy_map["test_policy_name"] = policy_ptr; + + int64_t cooldown_timestamp = -1; + size_t file_size = -1; + bool ret = _tablet->need_cooldown(&cooldown_timestamp, &file_size); + ASSERT_TRUE(ret); + ASSERT_EQ(cooldown_timestamp, 250); + ASSERT_EQ(file_size, -1); + } + + { + StoragePolicy* policy = new StoragePolicy(); + policy->storage_policy_name = "test_policy_name"; + policy->cooldown_datetime = -1; + policy->cooldown_ttl = 3600; + + std::shared_ptr policy_ptr; + policy_ptr.reset(policy); + + ExecEnv::GetInstance()->storage_policy_mgr()->_policy_map["test_policy_name"] = policy_ptr; + + int64_t cooldown_timestamp = -1; + size_t file_size = -1; + bool ret = _tablet->need_cooldown(&cooldown_timestamp, &file_size); + ASSERT_TRUE(ret); + ASSERT_EQ(cooldown_timestamp, 3700); + ASSERT_EQ(file_size, -1); + } + + { + StoragePolicy* policy = new StoragePolicy(); + policy->storage_policy_name = "test_policy_name"; + policy->cooldown_datetime = UnixSeconds() + 100; + policy->cooldown_ttl = -1; + + std::shared_ptr policy_ptr; + policy_ptr.reset(policy); + + ExecEnv::GetInstance()->storage_policy_mgr()->_policy_map["test_policy_name"] = policy_ptr; + + int64_t cooldown_timestamp = -1; + size_t file_size = -1; + bool ret = _tablet->need_cooldown(&cooldown_timestamp, &file_size); + ASSERT_FALSE(ret); + ASSERT_EQ(cooldown_timestamp, -1); + ASSERT_EQ(file_size, -1); + } + + { + StoragePolicy* policy = new StoragePolicy(); + policy->storage_policy_name = "test_policy_name"; + policy->cooldown_datetime = UnixSeconds() + 100; + policy->cooldown_ttl = UnixSeconds() - 250; + + std::shared_ptr policy_ptr; + policy_ptr.reset(policy); + + ExecEnv::GetInstance()->storage_policy_mgr()->_policy_map["test_policy_name"] = policy_ptr; + + int64_t cooldown_timestamp = -1; + size_t file_size = -1; + bool ret = _tablet->need_cooldown(&cooldown_timestamp, &file_size); + ASSERT_TRUE(ret); + ASSERT_EQ(cooldown_timestamp, -1); + ASSERT_EQ(file_size, 84699); + } +} + } // namespace doris diff --git a/be/test/olap/test_data/header_without_inc_rs.txt b/be/test/olap/test_data/header_without_inc_rs.txt index 12c1f81a0e0be3..c6b019977cec55 100644 --- a/be/test/olap/test_data/header_without_inc_rs.txt +++ b/be/test/olap/test_data/header_without_inc_rs.txt @@ -146,4 +146,5 @@ "storage_medium": "HDD", "remote_storage_name": "", "replica_id": 0 + "storage_policy": "" } diff --git a/be/test/olap/txn_manager_test.cpp b/be/test/olap/txn_manager_test.cpp index ace14e3c6126b7..b4c6a3f1783b3c 100644 --- a/be/test/olap/txn_manager_test.cpp +++ b/be/test/olap/txn_manager_test.cpp @@ -133,11 +133,9 @@ class TxnManagerTest : public testing::Test { RowsetMetaSharedPtr rowset_meta(new AlphaRowsetMeta()); rowset_meta->init_from_json(_json_rowset_meta); EXPECT_EQ(rowset_meta->rowset_id(), rowset_id); - FilePathDesc rowset_meta_path_desc; - rowset_meta_path_desc.filepath = rowset_meta_path; - EXPECT_EQ(Status::OK(), RowsetFactory::create_rowset(_schema.get(), rowset_meta_path_desc, + EXPECT_EQ(Status::OK(), RowsetFactory::create_rowset(_schema.get(), rowset_meta_path, rowset_meta, &_alpha_rowset)); - EXPECT_EQ(Status::OK(), RowsetFactory::create_rowset(_schema.get(), rowset_meta_path_desc, + EXPECT_EQ(Status::OK(), RowsetFactory::create_rowset(_schema.get(), rowset_meta_path, rowset_meta, &_alpha_rowset_same_id)); // init rowset meta 2 @@ -153,9 +151,7 @@ class TxnManagerTest : public testing::Test { RowsetMetaSharedPtr rowset_meta2(new AlphaRowsetMeta()); rowset_meta2->init_from_json(_json_rowset_meta); EXPECT_EQ(rowset_meta2->rowset_id(), rowset_id); - FilePathDesc rowset_meta_path_desc_2; - rowset_meta_path_desc_2.filepath = rowset_meta_path_2; - EXPECT_EQ(Status::OK(), RowsetFactory::create_rowset(_schema.get(), rowset_meta_path_desc_2, + EXPECT_EQ(Status::OK(), RowsetFactory::create_rowset(_schema.get(), rowset_meta_path_2, rowset_meta2, &_alpha_rowset_diff_id)); _tablet_uid = TabletUid(10, 10); } diff --git a/be/test/runtime/array_test.cpp b/be/test/runtime/array_test.cpp index 9ef527735238f2..2964b4baac414d 100644 --- a/be/test/runtime/array_test.cpp +++ b/be/test/runtime/array_test.cpp @@ -27,6 +27,9 @@ #include "gen_cpp/olap_file.pb.h" #include "gen_cpp/segment_v2.pb.h" +#include "io/fs/file_system.h" +#include "io/fs/file_writer.h" +#include "io/fs/local_file_system.h" #include "olap/field.h" #include "olap/fs/block_manager.h" #include "olap/fs/fs_util.h" @@ -249,8 +252,9 @@ class ArrayTest : public ::testing::Test { template void test_write_and_read_column(const ColumnPB& column_pb, const Field* field, const std::vector& arrays) { - const std::string path = TEST_DIR + "/" + generate_uuid_string(); - LOG(INFO) << "Test directory: " << path; + auto filename = generate_uuid_string(); + const std::string path = TEST_DIR + "/" + filename; + LOG(INFO) << "Test path: " << path; segment_v2::ColumnMetaPB meta; init_column_meta(&meta, column_pb); @@ -259,10 +263,10 @@ class ArrayTest : public ::testing::Test { tablet_column.init_from_pb(column_pb); Schema schema({tablet_column}, 0); { - auto wblock = create_writable_block(path); - EXPECT_NE(wblock, nullptr); - auto writer = create_column_writer(wblock.get(), meta, - column_pb); + auto file_writer = creat_file_writer(path); + EXPECT_NE(file_writer, nullptr); + auto writer = create_column_writer(file_writer.get(), + meta, column_pb); EXPECT_NE(writer, nullptr); Status st; for (auto array : arrays) { @@ -274,7 +278,7 @@ class ArrayTest : public ::testing::Test { EXPECT_TRUE(writer->write_ordinal_index().ok()); EXPECT_TRUE(writer->write_zone_map().ok()); - EXPECT_TRUE(wblock->close().ok()); + EXPECT_TRUE(file_writer->close().ok()); } { auto reader = create_column_reader(path, meta, arrays.size()); @@ -374,24 +378,21 @@ class ArrayTest : public ::testing::Test { } } - std::unique_ptr create_writable_block(const std::string& path) { - std::unique_ptr wblock; - fs::CreateBlockOptions fs_opts(path); - FilePathDesc path_desc; - path_desc.storage_medium = TStorageMedium::HDD; - auto st = fs::fs_util::block_manager(path_desc)->create_block(fs_opts, &wblock); - return st.ok() ? std::move(wblock) : nullptr; + std::unique_ptr creat_file_writer(const std::string& path) { + std::unique_ptr file_writer; + io::global_local_filesystem()->create_file(path, &file_writer); + return file_writer; } template - std::unique_ptr create_column_writer(fs::WritableBlock* wblock, + std::unique_ptr create_column_writer(io::FileWriter* file_writer, segment_v2::ColumnMetaPB& meta, const ColumnPB& column_pb) { segment_v2::ColumnWriterOptions writer_opts = {.meta = &meta}; TabletColumn column; column.init_from_pb(column_pb); std::unique_ptr writer; - auto st = segment_v2::ColumnWriter::create(writer_opts, &column, wblock, &writer); + auto st = segment_v2::ColumnWriter::create(writer_opts, &column, file_writer, &writer); if (!st.ok()) { return nullptr; } @@ -402,24 +403,19 @@ class ArrayTest : public ::testing::Test { std::unique_ptr create_column_reader( const std::string& path, const segment_v2::ColumnMetaPB& meta, size_t num_rows) { segment_v2::ColumnReaderOptions reader_opts; - FilePathDesc path_desc; - path_desc.filepath = path; std::unique_ptr reader; - auto st = segment_v2::ColumnReader::create(reader_opts, meta, num_rows, path_desc, &reader); + auto st = segment_v2::ColumnReader::create(reader_opts, meta, num_rows, + io::global_local_filesystem(), path, &reader); return st.ok() ? std::move(reader) : nullptr; } - std::unique_ptr create_readable_block(const std::string& path) { - std::unique_ptr rblock; - FilePathDesc path_desc; - path_desc.storage_medium = TStorageMedium::HDD; - path_desc.filepath = path; - auto block_manager = fs::fs_util::block_manager(path_desc); - auto st = block_manager->open_block(path_desc, &rblock); - return st.ok() ? std::move(rblock) : nullptr; + std::unique_ptr create_readable_block(const std::string& path) { + std::unique_ptr reader; + auto st = io::global_local_filesystem()->open_file(path, &reader); + return st.ok() ? std::move(reader) : nullptr; } - segment_v2::ColumnIterator* new_iterator(fs::ReadableBlock* rblock, OlapReaderStatistics* stats, + segment_v2::ColumnIterator* new_iterator(io::FileReader* rblock, OlapReaderStatistics* stats, segment_v2::ColumnReader* reader) { segment_v2::ColumnIterator* iter = nullptr; auto st = reader->new_iterator(&iter); @@ -428,7 +424,7 @@ class ArrayTest : public ::testing::Test { } segment_v2::ColumnIteratorOptions iter_opts; iter_opts.stats = stats; - iter_opts.rblock = rblock; + iter_opts.file_reader = rblock; st = iter->init(iter_opts); return st.ok() ? iter : nullptr; } diff --git a/be/test/tools/benchmark_tool.cpp b/be/test/tools/benchmark_tool.cpp index 12d214cb8f07b3..fb7eef57a3b90d 100644 --- a/be/test/tools/benchmark_tool.cpp +++ b/be/test/tools/benchmark_tool.cpp @@ -33,6 +33,9 @@ #include "common/logging.h" #include "gutil/strings/split.h" #include "gutil/strings/substitute.h" +#include "io/fs/file_system.h" +#include "io/fs/file_writer.h" +#include "io/fs/local_file_system.h" #include "olap/comparison_predicate.h" #include "olap/data_dir.h" #include "olap/fs/block_manager.h" @@ -104,7 +107,7 @@ namespace doris { class BaseBenchmark { public: BaseBenchmark(const std::string& name, int iterations) : _name(name), _iterations(iterations) {} - virtual ~BaseBenchmark() {} + virtual ~BaseBenchmark() = default; void add_name(const std::string& str) { _name += str; } @@ -293,7 +296,7 @@ class SegmentBenchmark : public BaseBenchmark { } } - const Schema& get_schema() { return *_schema.get(); } + const Schema& get_schema() { return *_schema; } virtual void init() override {} virtual void run() override {} @@ -340,15 +343,16 @@ class SegmentBenchmark : public BaseBenchmark { std::shared_ptr* res) { // must use unique filename for each segment, otherwise page cache kicks in and produces // the wrong answer (it use (filename,offset) as cache key) - std::string filename = strings::Substitute("$0/seg_$1.dat", kSegmentDir, ++seg_id); - std::unique_ptr wblock; - fs::CreateBlockOptions block_opts({filename}); - std::string storage_name; - fs::fs_util::block_manager(storage_name)->create_block(block_opts, &wblock); + std::string filename = fmt::format("seg_{}.dat", seg_id++); + std::string path = fmt::format("{}/{}", kSegmentDir, filename); + auto fs = io::global_local_filesystem(); + + std::unique_ptr file_writer; + fs->create_file(path, &file_writer); SegmentWriterOptions opts; DataDir data_dir(kSegmentDir); data_dir.init(); - SegmentWriter writer(wblock.get(), 0, &_tablet_schema, &data_dir, INT32_MAX, opts); + SegmentWriter writer(file_writer.get(), 0, &_tablet_schema, &data_dir, INT32_MAX, opts); writer.init(1024); RowCursor row; @@ -366,9 +370,9 @@ class SegmentBenchmark : public BaseBenchmark { uint64_t file_size, index_size; writer.finalize(&file_size, &index_size); - wblock->close(); + file_writer->close(); - Segment::open(filename, seg_id, &_tablet_schema, res); + Segment::open(fs, path, seg_id, &_tablet_schema, res); } std::vector> generate_dataset(int rows_number) { @@ -402,6 +406,7 @@ class SegmentBenchmark : public BaseBenchmark { return res; } +private: std::shared_ptr _tracker; MemPool _pool; TabletSchema _tablet_schema; diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index a11537eb33e6dd..d32e582f550d5d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -85,6 +85,8 @@ import java.util.List; import java.util.Map; +import static org.apache.doris.catalog.Catalog.checkStoragePolicyExist; + public class Alter { private static final Logger LOG = LogManager.getLogger(Alter.class); @@ -155,7 +157,22 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, } boolean needProcessOutsideTableLock = false; - if (currentAlterOps.hasSchemaChangeOp()) { + if (currentAlterOps.checkTableStoragePolicy(alterClauses)) { + String tableStoragePolicy = olapTable.getStoragePolicy(); + if (!tableStoragePolicy.equals("")) { + throw new DdlException("Do not support alter table's storage policy , " + + "this table [" + olapTable.getName() + "] has storage policy " + tableStoragePolicy); + } + String currentStoragePolicy = currentAlterOps.getTableStoragePolicy(alterClauses); + if (currentStoragePolicy.equals("")) { + throw new DdlException("alter table storage policy, but empty"); + } + // check currentStoragePolicy resource exist. + checkStoragePolicyExist(currentStoragePolicy); + + olapTable.setStoragePolicy(currentStoragePolicy); + needProcessOutsideTableLock = true; + } else if (currentAlterOps.hasSchemaChangeOp()) { // if modify storage type to v2, do schema change to convert all related tablets to segment v2 format schemaChangeHandler.process(alterClauses, clusterName, db, olapTable); } else if (currentAlterOps.hasRollupOp()) { @@ -188,7 +205,11 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, needProcessOutsideTableLock = true; } else { List partitionNames = clause.getPartitionNames(); - modifyPartitionsProperty(db, olapTable, partitionNames, properties); + if (!properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY)) { + modifyPartitionsProperty(db, olapTable, partitionNames, properties); + } else { + needProcessOutsideTableLock = true; + } } } else if (alterClause instanceof AddPartitionClause) { needProcessOutsideTableLock = true; @@ -415,8 +436,9 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { ModifyPartitionClause clause = ((ModifyPartitionClause) alterClause); Map properties = clause.getProperties(); List partitionNames = clause.getPartitionNames(); - // currently, only in memory property could reach here - Preconditions.checkState(properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY)); + // currently, only in memory and storage policy property could reach here + Preconditions.checkState(properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY) + || properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY)); ((SchemaChangeHandler) schemaChangeHandler).updatePartitionsInMemoryMeta( db, tableName, partitionNames, properties); OlapTable olapTable = (OlapTable) table; @@ -428,8 +450,9 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException { } } else if (alterClause instanceof ModifyTablePropertiesClause) { Map properties = alterClause.getProperties(); - // currently, only in memory property could reach here - Preconditions.checkState(properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY)); + // currently, only in memory and storage policy property could reach here + Preconditions.checkState(properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY) + || properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY)); ((SchemaChangeHandler) schemaChangeHandler).updateTableInMemoryMeta(db, tableName, properties); } else { throw new DdlException("Invalid alter operation: " + alterClause.getOpType()); @@ -696,7 +719,23 @@ public void modifyPartitionsProperty(Database db, TimeUtils.getTimeZone(), Type.DATETIME); newProperties.put(PropertyAnalyzer.PROPERTIES_REMOTE_STORAGE_COOLDOWN_TIME, dateLiteral1.getStringValue()); newProperties.putAll(properties); - // 4.3 analyze new properties + + // 4.3 modify partition storage policy + String partitionStoragePolicy = partitionInfo.getStoragePolicy(partition.getId()); + if (!partitionStoragePolicy.equals("")) { + throw new DdlException("Do not support alter table's partition storage policy , " + + "this table [" + olapTable.getName() + "] and partition [" + partitionName + "] " + + "has storage policy " + partitionStoragePolicy); + } + String currentStoragePolicy = PropertyAnalyzer.analyzeStoragePolicy(properties); + if (currentStoragePolicy.equals("")) { + throw new DdlException("alter table storage policy, but empty"); + } + // check currentStoragePolicy resource exist. + checkStoragePolicyExist(currentStoragePolicy); + partitionInfo.setStoragePolicy(partition.getId(), currentStoragePolicy); + + // 4.4 analyze new properties DataProperty newDataProperty = PropertyAnalyzer.analyzeDataProperty(newProperties, null); // 1. date property @@ -717,7 +756,7 @@ public void modifyPartitionsProperty(Database db, partitionInfo.setTabletType(partition.getId(), tTabletType); } ModifyPartitionInfo info = new ModifyPartitionInfo(db.getId(), olapTable.getId(), partition.getId(), - newDataProperty, replicaAlloc, hasInMemory ? newInMemory : oldInMemory); + newDataProperty, replicaAlloc, hasInMemory ? newInMemory : oldInMemory, currentStoragePolicy); modifyPartitionInfos.add(info); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java index 347aa36747ff56..1ff44c586264ac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java @@ -222,7 +222,8 @@ public void handleFinishAlterTask(AlterReplicaTask task) throws MetaNotFoundExce task.getSignature(), replica, task.getVersion()); boolean versionChanged = false; if (replica.getVersion() < task.getVersion()) { - replica.updateVersionInfo(task.getVersion(), replica.getDataSize(), replica.getRowCount()); + replica.updateVersionInfo(task.getVersion(), replica.getDataSize(), replica.getRemoteDataSize(), + replica.getRowCount()); versionChanged = true; } @@ -230,7 +231,7 @@ public void handleFinishAlterTask(AlterReplicaTask task) throws MetaNotFoundExce ReplicaPersistInfo info = ReplicaPersistInfo.createForClone(task.getDbId(), task.getTableId(), task.getPartitionId(), task.getIndexId(), task.getTabletId(), task.getBackendId(), replica.getId(), replica.getVersion(), -1, - replica.getDataSize(), replica.getRowCount(), + replica.getDataSize(), replica.getRemoteDataSize(), replica.getRowCount(), replica.getLastFailedVersion(), replica.getLastSuccessVersion()); Catalog.getCurrentCatalog().getEditLog().logUpdateReplica(info); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java index c60594326b66f6..46a30bf9b6b366 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java @@ -18,13 +18,17 @@ package org.apache.doris.alter; import org.apache.doris.analysis.AlterClause; +import org.apache.doris.analysis.ModifyTablePropertiesClause; import org.apache.doris.common.DdlException; import com.google.common.base.Joiner; import com.google.common.collect.Sets; +import org.apache.doris.common.util.PropertyAnalyzer; import java.util.List; +import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; /* * AlterOperations contains a set alter operations generated from a AlterStmt's alter clause. @@ -65,6 +69,18 @@ public boolean hasPartitionOp() { || currentOps.contains(AlterOpType.MODIFY_PARTITION); } + public boolean checkTableStoragePolicy(List alterClauses) { + return alterClauses.stream().filter(clause -> + clause instanceof ModifyTablePropertiesClause + ).anyMatch(clause->clause.getProperties().containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY)); + } + + public String getTableStoragePolicy(List alterClauses) { + return alterClauses.stream().filter(clause -> + clause instanceof ModifyTablePropertiesClause + ).map(c -> ((ModifyTablePropertiesClause) c).getStoragePolicy()).findFirst().orElse(""); + } + // MODIFY_TABLE_PROPERTY is also processed by SchemaChangeHandler public boolean hasSchemaChangeOp() { return currentOps.contains(AlterOpType.SCHEMA_CHANGE) || currentOps.contains(AlterOpType.MODIFY_TABLE_PROPERTY); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java index ba707233a5f955..b2c2076f20934e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -238,7 +238,8 @@ protected void runPendingJob() throws AlterCancelException { tbl.getCopiedIndexes(), tbl.isInMemory(), tabletType, - tbl.getCompressionType()); + tbl.getCompressionType(), + tbl.getStoragePolicy()); createReplicaTask.setBaseTablet(tabletIdMap.get(rollupTabletId), baseSchemaHash); if (this.storageFormat != null) { createReplicaTask.setStorageFormat(this.storageFormat); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index 6329834d59a638..1bd8468c6aa337 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -1654,12 +1654,15 @@ public void updateTableInMemoryMeta(Database db, String tableName, Map Map>> beIdToTabletIdWithHash = Maps.newHashMap(); @@ -1732,7 +1740,7 @@ public void updatePartitionInMemoryMeta(Database db, for (Map.Entry>> kv : beIdToTabletIdWithHash.entrySet()) { countDownLatch.addMark(kv.getKey(), kv.getValue()); UpdateTabletMetaInfoTask task = new UpdateTabletMetaInfoTask(kv.getKey(), kv.getValue(), - isInMemory, countDownLatch); + isInMemory, storagePolicy, countDownLatch); batchTask.addTask(task); } if (!FeConstants.runningUnitTest) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java index d91b770ec9a866..d339c2754aeaa8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java @@ -32,6 +32,16 @@ public class ModifyTablePropertiesClause extends AlterTableClause { private Map properties; + public String getStoragePolicy() { + return StoragePolicy; + } + + public void setStoragePolicy(String storagePolicy) { + StoragePolicy = storagePolicy; + } + + private String StoragePolicy; + public ModifyTablePropertiesClause(Map properties) { super(AlterOpType.MODIFY_TABLE_PROPERTY); this.properties = properties; @@ -91,8 +101,11 @@ public void analyze(Analyzer analyzer) throws AnalysisException { throw new AnalysisException("Alter tablet type not supported"); } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_REMOTE_STORAGE_RESOURCE)) { throw new AnalysisException("Alter table remote_storage_resource is not supported."); + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY)) { + this.needTableStable = false; + setStoragePolicy(properties.getOrDefault(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY, "")); } else { - throw new AnalysisException("Unknown table property: " + properties.keySet()); + throw new AnalysisException("Unknown table property: " + properties.keySet()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SinglePartitionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SinglePartitionDesc.java index 6e75f67a2f1bc0..c9727db97cef83 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SinglePartitionDesc.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SinglePartitionDesc.java @@ -32,6 +32,8 @@ import java.util.Map; +import static org.apache.doris.common.util.PropertyAnalyzer.PROPERTIES_STORAGE_POLICY; + public class SinglePartitionDesc { private boolean isAnalyzed; @@ -46,6 +48,7 @@ public class SinglePartitionDesc { private boolean isInMemory = false; private TTabletType tabletType = TTabletType.TABLET_TYPE_DISK; private Long versionInfo; + private String storagePolicy; public SinglePartitionDesc(boolean ifNotExists, String partName, PartitionKeyDesc partitionKeyDesc, Map properties) { @@ -59,6 +62,7 @@ public SinglePartitionDesc(boolean ifNotExists, String partName, PartitionKeyDes this.partitionDataProperty = DataProperty.DEFAULT_DATA_PROPERTY; this.replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION; + this.storagePolicy = ""; } public boolean isSetIfNotExists() { @@ -101,6 +105,15 @@ public void analyze(int partColNum, Map otherProperties) throws if (isAnalyzed) { return; } + boolean hasStoragePolicy = properties.keySet().stream(). + anyMatch(iter-> { + boolean equal = iter.compareToIgnoreCase(PROPERTIES_STORAGE_POLICY) == 0; + // when find has storage policy properties, here will set it in partition + if (equal) { + storagePolicy = properties.get(iter); + } + return equal; + }); FeNameFormat.checkPartitionName(partName); @@ -132,8 +145,10 @@ public void analyze(int partColNum, Map otherProperties) throws if (otherProperties == null) { // check unknown properties if (properties != null && !properties.isEmpty()) { - MapJoiner mapJoiner = Joiner.on(", ").withKeyValueSeparator(" = "); - throw new AnalysisException("Unknown properties: " + mapJoiner.join(properties)); + if (!hasStoragePolicy) { + MapJoiner mapJoiner = Joiner.on(", ").withKeyValueSeparator(" = "); + throw new AnalysisException("Unknown properties: " + mapJoiner.join(properties)); + } } } @@ -144,6 +159,10 @@ public boolean isAnalyzed() { return this.isAnalyzed; } + public String getStoragePolicy(){ + return this.storagePolicy; + } + public String toSql() { StringBuilder sb = new StringBuilder(); sb.append("PARTITION ").append(partName); diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index 78148acc1fb242..e530da8721c235 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -973,7 +973,8 @@ private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable loc localTbl.getCopiedIndexes(), localTbl.isInMemory(), localTbl.getPartitionInfo().getTabletType(restorePart.getId()), - localTbl.getCompressionType()); + localTbl.getCompressionType(), + localTbl.getStoragePolicy()); task.setInRestoreMode(true); batchTask.addTask(task); } @@ -1450,9 +1451,9 @@ private Status allTabletCommitted(boolean isReplay) { for (MaterializedIndex idx : part.getMaterializedIndices(IndexExtState.VISIBLE)) { for (Tablet tablet : idx.getTablets()) { for (Replica replica : tablet.getReplicas()) { - if (!replica.checkVersionCatchUp(part.getVisibleVersion(), false)) { - replica.updateVersionInfo(part.getVisibleVersion(), - replica.getDataSize(), replica.getRowCount()); + if (!replica.checkVersionCatchUp(part.getVisibleVersion(),false)) { + replica.updateVersionInfo(part.getVisibleVersion(), replica.getDataSize(), + replica.getRemoteDataSize(), replica.getRowCount()); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index 9bf9608a458313..9de43e4aff6d7f 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -260,6 +260,9 @@ import java.util.stream.Collectors; import javax.annotation.Nullable; +import static org.apache.doris.catalog.StoragePolicyResource.DEFAULT_STORAGE_POLICY_PROPERTY; +import static org.apache.doris.common.util.PropertyAnalyzer.PROPERTIES_STORAGE_POLICY; + public class Catalog { private static final Logger LOG = LogManager.getLogger(Catalog.class); // 0 ~ 9999 used for qe @@ -3350,7 +3353,8 @@ public HashMap getPartitionIdToStorageMediumMap() { partition.getId(), hddProperty, ReplicaAllocation.NOT_SET, - partitionInfo.getIsInMemory(partition.getId())); + partitionInfo.getIsInMemory(partition.getId()), + partitionInfo.getStoragePolicy(partitionId)); editLog.logModifyPartition(info); } } // end for partitions @@ -4058,7 +4062,7 @@ public void modifyTableReplicaAllocation(Database db, OlapTable table, partitionInfo.setReplicaAllocation(partition.getId(), replicaAlloc); // log ModifyPartitionInfo info = new ModifyPartitionInfo(db.getId(), table.getId(), partition.getId(), - newDataProperty, replicaAlloc, isInMemory); + newDataProperty, replicaAlloc, isInMemory, partitionInfo.getStoragePolicy(partition.getId())); editLog.logModifyPartition(info); LOG.debug("modify partition[{}-{}-{}] replica allocation to {}", db.getId(), table.getId(), partition.getName(), replicaAlloc.toCreateStmt()); @@ -4102,10 +4106,12 @@ public void modifyTableInMemoryMeta(Database db, OlapTable table, Map= FeMetaVersion.VERSION_108) { @@ -108,11 +113,12 @@ public void readFields(DataInput in) throws IOException { cooldownTimeMs = in.readLong(); remoteStorageResourceName = ""; remoteCooldownTimeMs = MAX_COOLDOWN_TIME_MS; + storagePolicy = Text.readString(in); } @Override public int hashCode() { - return Objects.hash(storageMedium, cooldownTimeMs, remoteStorageResourceName, remoteCooldownTimeMs); + return Objects.hash(storageMedium, cooldownTimeMs, remoteStorageResourceName, remoteCooldownTimeMs, storagePolicy); } @Override @@ -130,7 +136,8 @@ public boolean equals(Object obj) { return this.storageMedium == other.storageMedium && this.cooldownTimeMs == other.cooldownTimeMs && this.remoteCooldownTimeMs == other.remoteCooldownTimeMs - && this.remoteStorageResourceName.equals(other.remoteStorageResourceName); + && this.remoteStorageResourceName.equals(other.remoteStorageResourceName) + && this.storagePolicy.equals(other.storagePolicy); } @Override @@ -140,6 +147,7 @@ public String toString() { sb.append("cool down[").append(TimeUtils.longToTimeString(cooldownTimeMs)).append("]. "); sb.append("remote storage resource name[").append(this.remoteStorageResourceName).append("]. "); sb.append("remote cool down[").append(TimeUtils.longToTimeString(remoteCooldownTimeMs)).append("]."); + sb.append("Storage policy[").append(this.storagePolicy).append("]."); return sb.toString(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DiskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DiskInfo.java index a96b83869ea297..bf094a8cc124a1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/DiskInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DiskInfo.java @@ -53,6 +53,17 @@ public enum DiskState { private DiskState state; // path hash and storage medium are reported from Backend and no need to persist private long pathHash = 0; + + public long getRemoteUsedCapacity() { + return remoteUsedCapacity; + } + + public void setRemoteUsedCapacity(long remoteUsedCapacity) { + this.remoteUsedCapacity = remoteUsedCapacity; + } + + @SerializedName("remoteUsedCapacity") + private long remoteUsedCapacity = 0; private TStorageMedium storageMedium; private DiskInfo() { @@ -67,6 +78,7 @@ public DiskInfo(String rootPath) { this.state = DiskState.ONLINE; this.pathHash = 0; this.storageMedium = TStorageMedium.HDD; + this.remoteUsedCapacity = 0; } public String getRootPath() { @@ -163,7 +175,8 @@ public boolean exceedLimit(boolean floodStage) { public String toString() { return "DiskInfo [rootPath=" + rootPath + "(" + pathHash + "), totalCapacityB=" + totalCapacityB + ", dataUsedCapacityB=" + dataUsedCapacityB + ", diskAvailableCapacityB=" - + diskAvailableCapacityB + ", state=" + state + ", medium: " + storageMedium + "]"; + + diskAvailableCapacityB + ", state=" + state + ", medium: " + storageMedium + + "remoteUsedCapacity=" + remoteUsedCapacity + "]"; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcCatalogResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcCatalogResource.java index 4340ff2cf962d8..b859da3c7ddbe4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcCatalogResource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcCatalogResource.java @@ -91,6 +91,11 @@ private void checkProperties(String propertiesKey) throws DdlException { } } + @Override + public Map getCopiedProperties() { + return Maps.newHashMap(configs); + } + @Override public void modifyProperties(Map properties) throws DdlException { // modify properties @@ -118,12 +123,6 @@ public void checkProperties(Map properties) throws AnalysisExcep } } - @Override - public Map getCopiedProperties() { - Map copiedProperties = Maps.newHashMap(configs); - return copiedProperties; - } - public String getProperty(String propertiesKey) { // check the properties key String value = configs.get(propertiesKey); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index a117762274b83d..f5f10035310663 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -1539,6 +1539,21 @@ public void setIsInMemory(boolean isInMemory) { tableProperty.buildInMemory(); } + public void setStoragePolicy(String storagePolicy) { + if(tableProperty == null) { + tableProperty = new TableProperty(new HashMap<>()); + } + tableProperty.modifyTableProperties(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY, storagePolicy); + tableProperty.buildStoragePolicy(); + } + + public String getStoragePolicy() { + if (tableProperty != null) { + return tableProperty.getStoragePolicy(); + } + return ""; + } + public void setDataSortInfo(DataSortInfo dataSortInfo) { if (tableProperty == null) { tableProperty = new TableProperty(new HashMap<>()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java index 9d95b14b618c8e..cfbb95edb33630 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java @@ -55,6 +55,8 @@ public class PartitionInfo implements Writable { protected Map idToTempItem = Maps.newHashMap(); // partition id -> data property protected Map idToDataProperty; + // partition id -> storage policy + protected Map idToStoragePolicy; // partition id -> replication allocation protected Map idToReplicaAllocation; // true if the partition has multi partition columns @@ -72,6 +74,7 @@ public PartitionInfo() { this.idToReplicaAllocation = new HashMap<>(); this.idToInMemory = new HashMap<>(); this.idToTabletType = new HashMap<>(); + this.idToStoragePolicy = new HashMap<>(); } public PartitionInfo(PartitionType type) { @@ -80,6 +83,7 @@ public PartitionInfo(PartitionType type) { this.idToReplicaAllocation = new HashMap<>(); this.idToInMemory = new HashMap<>(); this.idToTabletType = new HashMap<>(); + this.idToStoragePolicy = new HashMap<>(); } public PartitionInfo(PartitionType type, List partitionColumns) { @@ -133,6 +137,7 @@ public PartitionItem handleNewSinglePartitionDesc(SinglePartitionDesc desc, idToDataProperty.put(partitionId, desc.getPartitionDataProperty()); idToReplicaAllocation.put(partitionId, desc.getReplicaAlloc()); idToInMemory.put(partitionId, desc.isInMemory()); + idToStoragePolicy.put(partitionId, desc.getStoragePolicy()); return partitionItem; } @@ -148,6 +153,7 @@ public void unprotectHandleNewSinglePartitionDesc(long partitionId, boolean isTe idToDataProperty.put(partitionId, dataProperty); idToReplicaAllocation.put(partitionId, replicaAlloc); idToInMemory.put(partitionId, isInMemory); + idToStoragePolicy.put(partitionId, ""); } public List> getPartitionItemEntryList(boolean isTemp, boolean isSorted) { @@ -209,6 +215,14 @@ public void setDataProperty(long partitionId, DataProperty newDataProperty) { idToDataProperty.put(partitionId, newDataProperty); } + public String getStoragePolicy(long partitionId) { + return idToStoragePolicy.getOrDefault(partitionId, ""); + } + + public void setStoragePolicy(long partitionId, String storagePolicy) { + idToStoragePolicy.put(partitionId, storagePolicy); + } + public ReplicaAllocation getReplicaAllocation(long partitionId) { if (!idToReplicaAllocation.containsKey(partitionId)) { LOG.debug("failed to get replica allocation for partition: {}", partitionId); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java index 665cd26482da23..ce15c9886c79c3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java @@ -79,6 +79,8 @@ public enum ReplicaStatus { private int schemaHash = -1; @SerializedName(value = "dataSize") private volatile long dataSize = 0; + @SerializedName(value = "remoteDataSize") + private volatile long remoteDataSize = 0; @SerializedName(value = "rowCount") private volatile long rowCount = 0; @SerializedName(value = "state") @@ -132,16 +134,16 @@ public Replica() { // for rollup // the new replica's version is -1 and last failed version is -1 public Replica(long replicaId, long backendId, int schemaHash, ReplicaState state) { - this(replicaId, backendId, -1, schemaHash, 0L, 0L, state, -1, -1); + this(replicaId, backendId, -1, schemaHash, 0L, 0L, 0L, state, -1, -1); } // for create tablet and restore public Replica(long replicaId, long backendId, ReplicaState state, long version, int schemaHash) { - this(replicaId, backendId, version, schemaHash, 0L, 0L, state, -1L, version); + this(replicaId, backendId, version, schemaHash, 0L, 0L, 0L, state, -1L, version); } public Replica(long replicaId, long backendId, long version, int schemaHash, - long dataSize, long rowCount, ReplicaState state, + long dataSize, long remoteDataSize, long rowCount, ReplicaState state, long lastFailedVersion, long lastSuccessVersion) { this.id = replicaId; @@ -150,6 +152,7 @@ public Replica(long replicaId, long backendId, long version, int schemaHash, this.schemaHash = schemaHash; this.dataSize = dataSize; + this.remoteDataSize = remoteDataSize; this.rowCount = rowCount; this.state = state; if (this.state == null) { @@ -191,6 +194,10 @@ public long getDataSize() { return dataSize; } + public long getRemoteDataSize() { + return remoteDataSize; + } + public long getRowCount() { return rowCount; } @@ -245,19 +252,21 @@ public synchronized void updateStat(long dataSize, long rowNum) { this.rowCount = rowNum; } - public synchronized void updateStat(long dataSize, long rowNum, long versionCount) { + public synchronized void updateStat(long dataSize, long remoteDataSize, long rowNum, long versionCount) { this.dataSize = dataSize; + this.remoteDataSize = remoteDataSize; this.rowCount = rowNum; this.versionCount = versionCount; } - public synchronized void updateVersionInfo(long newVersion, long newDataSize, long newRowCount) { - updateReplicaInfo(newVersion, this.lastFailedVersion, this.lastSuccessVersion, newDataSize, newRowCount); + public synchronized void updateVersionInfo(long newVersion, long newDataSize, long newRemoteDataSize, + long newRowCount) { + updateReplicaInfo(newVersion, this.lastFailedVersion, this.lastSuccessVersion, newDataSize, newRemoteDataSize, + newRowCount); } - public synchronized void updateVersionWithFailedInfo( - long newVersion, long lastFailedVersion, long lastSuccessVersion) { - updateReplicaInfo(newVersion, lastFailedVersion, lastSuccessVersion, dataSize, rowCount); + public synchronized void updateVersionWithFailedInfo(long newVersion, long lastFailedVersion, long lastSuccessVersion) { + updateReplicaInfo(newVersion, lastFailedVersion, lastSuccessVersion, dataSize, remoteDataSize, rowCount); } /* last failed version: LFV @@ -286,7 +295,7 @@ public synchronized void updateVersionWithFailedInfo( */ private void updateReplicaInfo(long newVersion, long lastFailedVersion, long lastSuccessVersion, - long newDataSize, long newRowCount) { + long newDataSize, long newRemoteDataSize, long newRowCount) { if (LOG.isDebugEnabled()) { LOG.debug("before update: {}", this.toString()); } @@ -310,6 +319,7 @@ private void updateReplicaInfo(long newVersion, this.version = newVersion; this.dataSize = newDataSize; + this.remoteDataSize = newRemoteDataSize; this.rowCount = newRowCount; // just check it @@ -363,7 +373,7 @@ private void updateReplicaInfo(long newVersion, } public synchronized void updateLastFailedVersion(long lastFailedVersion) { - updateReplicaInfo(this.version, lastFailedVersion, this.lastSuccessVersion, dataSize, rowCount); + updateReplicaInfo(this.version, lastFailedVersion, this.lastSuccessVersion, dataSize, remoteDataSize, rowCount); } /* diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Resource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Resource.java index fac6a30e45db28..fc002cc8515652 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Resource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Resource.java @@ -135,7 +135,6 @@ protected void replaceIfEffectiveValue(Map properties, String ke */ protected abstract void setProperties(Map properties) throws DdlException; - public abstract Map getCopiedProperties(); /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ResourceMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ResourceMgr.java index d7d57272ab21b7..340dd0bd362916 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ResourceMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ResourceMgr.java @@ -22,6 +22,7 @@ import org.apache.doris.analysis.DropResourceStmt; import org.apache.doris.catalog.Resource.ResourceType; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; @@ -49,6 +50,8 @@ import java.util.Optional; import java.util.Set; +import static org.apache.doris.catalog.StoragePolicyResource.DEFAULT_STORAGE_POLICY_PROPERTY; + /** * Resource manager is responsible for managing external resources used by Doris. * For example, Spark/MapReduce used for ETL, Spark/GPU used for queries, HDFS/S3 used for external storage. @@ -82,6 +85,20 @@ public void createResource(CreateResourceStmt stmt) throws DdlException { LOG.info("Create resource success. Resource: {}", resource); } + public void createDefaultStoragePolicy() throws DdlException { + if (!Config.use_default_storage_policy) { + return; + } + + if (nameToResource.putIfAbsent(Config.default_storage_policy, DEFAULT_STORAGE_POLICY_PROPERTY) != null){ + // already exist default storage policy. + return; + } + // log add + Catalog.getCurrentCatalog().getEditLog().logCreateResource(DEFAULT_STORAGE_POLICY_PROPERTY); + LOG.info("Create resource success. Resource: {}", DEFAULT_STORAGE_POLICY_PROPERTY); + } + public void createResource(Resource resource) throws DdlException { String resourceName = resource.getName(); if (nameToResource.putIfAbsent(resourceName, resource) != null) { @@ -91,6 +108,9 @@ public void createResource(Resource resource) throws DdlException { public void replayCreateResource(Resource resource) { nameToResource.put(resource.getName(), resource); + if (Config.use_default_storage_policy && resource.getName() == Config.default_storage_policy) { + StoragePolicyResource.setDefaultStoragePolicyProperties(resource.getCopiedProperties()); + } } public void dropResource(DropResourceStmt stmt) throws DdlException { @@ -98,6 +118,16 @@ public void dropResource(DropResourceStmt stmt) throws DdlException { if (!nameToResource.containsKey(resourceName)) { throw new DdlException("Resource(" + resourceName + ") does not exist"); } + if (resourceName.equalsIgnoreCase(DEFAULT_STORAGE_POLICY_PROPERTY.getName())) { + throw new DdlException("Resource(" + DEFAULT_STORAGE_POLICY_PROPERTY.getName() + ") does not drop, " + + "it's a default storage policy"); + } + + if (nameToResource.get(resourceName).getType().equals(ResourceType.STORAGE_POLICY)) { + // current not support drop storage policy. + throw new DdlException("Resource policy( " + resourceName + ") current not support drop storage policy."); + } + // Check whether the resource is in use before deleting it, except spark resource List usedTables = new ArrayList<>(); List dbIds = Catalog.getCurrentCatalog().getDbIds(); @@ -139,6 +169,10 @@ public void dropResource(DropResourceStmt stmt) throws DdlException { // Drop resource whether successful or not public void dropResource(Resource resource) { String name = resource.getName(); + if (resource.name.equalsIgnoreCase(DEFAULT_STORAGE_POLICY_PROPERTY.getName())) { + LOG.info("doesn't remove default storage policy"); + return; + } if (nameToResource.remove(name) == null) { LOG.info("resource " + name + " does not exists."); return; @@ -181,6 +215,17 @@ public int getResourceNum() { return nameToResource.size(); } + + public Map getResourceByType(ResourceType type){ + Map result = Maps.newConcurrentMap(); + for (Resource r : nameToResource.values()){ + if (r.getType() == type){ + result.put(r.name, r); + } + } + return result; + } + public List> getResourcesInfo(String name, boolean accurateMatch, Set typeSets) { List> targetRows = procNode.fetchResult().getRows(); List> returnRows = Lists.newArrayList(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/S3Resource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/S3Resource.java index 91069c2d65886a..9c1e61c6645791 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/S3Resource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/S3Resource.java @@ -141,8 +141,7 @@ public void checkProperties(Map properties) throws AnalysisExcep @Override public Map getCopiedProperties() { - Map copiedProperties = Maps.newHashMap(properties); - return copiedProperties; + return Maps.newHashMap(properties); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java index 01b1e0488b6fcb..a581fc1223834e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java @@ -159,8 +159,7 @@ public SparkResource getCopiedResource() { @Override public Map getCopiedProperties() { - Map copiedProperties = Maps.newHashMap(sparkConfigs); - return copiedProperties; + return Maps.newHashMap(sparkConfigs); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/StoragePolicyResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/StoragePolicyResource.java new file mode 100644 index 00000000000000..1987261f270678 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/StoragePolicyResource.java @@ -0,0 +1,235 @@ +// 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. + +package org.apache.doris.catalog; + +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.proc.BaseProcResult; + +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.annotations.SerializedName; +import org.apache.doris.system.SystemInfoService; +import org.apache.doris.task.AgentBatchTask; +import org.apache.doris.task.AgentTaskExecutor; +import org.apache.doris.task.NotifyUpdateStoragePolicyTask; +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +/** + * Policy resource for olap table + * + * Syntax: + * CREATE RESOURCE "storage_policy_name" + * PROPERTIES( + * "type"="storage_policy", + * "cooldown_datetime" = "2022-06-01", // time when data is transfter to medium + * "cooldown_ttl" = "3600", // data is transfter to medium after 1 hour + * "s3_*" + * ); + */ +public class StoragePolicyResource extends Resource { + private static final Logger LOG = LogManager.getLogger(StoragePolicyResource.class); + + // optional + public static final String COOLDOWN_DATETIME = "cooldown_datetime"; + public static final String COOLDOWN_TTL = "cooldown_ttl"; + public static final String S3_MAXCONN = "s3_max_connections"; + public static final String S3_REQUESTTIMEOUTMS = "s3_request_timeout_ms"; + public static final String S3_CONNTIMEOUTMS = "s3_connection_timeout_ms"; + + private static final String DEFAULT_COOLDOWN_DATETIME = "9999-01-01 00:00:00"; + private static final String DEFAULT_COOLDOWN_TTL = "3600"; + private static final String DEFAULT_S3_MAXCONN = "50"; + private static final String DEFAULT_S3_REQUESTTIMEOUTMS = "3000"; + private static final String DEFAULT_S3_CONNTIMEOUTMS = "1000"; + + // required + public static final String S3_ENDPOINT = "s3_endpoint"; + public static final String S3_REGION = "s3_region"; + public static final String S3_ROOTPATH = "s3_rootpath"; + public static final String S3_AK = "s3_access_key"; + public static final String S3_SK = "s3_secret_key"; + public static final String S3_BUCKET = "s3_bucket"; + + public static final String MD5_CHECKSUM = "md5_checksum"; + + public static StoragePolicyResource DEFAULT_STORAGE_POLICY_PROPERTY + = new StoragePolicyResource(Config.default_storage_policy); + + public static void setDefaultStoragePolicyProperties(Map properties) { + DEFAULT_STORAGE_POLICY_PROPERTY.properties = properties; + } + + @SerializedName(value = "properties") + private Map properties; + + public StoragePolicyResource(String name) { + this(name, Maps.newHashMap()); + } + + public StoragePolicyResource(String name, Map properties) { + super(name, ResourceType.STORAGE_POLICY); + this.properties = properties; + } + + public String getProperty(String propertyKey) { + return properties.get(propertyKey); + } + + @Override + protected void setProperties(Map properties) throws DdlException { + Preconditions.checkState(properties != null); + this.properties = properties; + // check properties + // required + checkRequiredProperty(S3_ENDPOINT); + checkRequiredProperty(S3_REGION); + checkRequiredProperty(S3_ROOTPATH); + checkRequiredProperty(S3_AK); + checkRequiredProperty(S3_SK); + checkRequiredProperty(S3_BUCKET); + // optional + checkOptionalProperty(COOLDOWN_DATETIME, DEFAULT_COOLDOWN_DATETIME); + checkOptionalProperty(COOLDOWN_TTL, DEFAULT_COOLDOWN_TTL); + checkOptionalProperty(S3_MAXCONN, DEFAULT_S3_MAXCONN); + checkOptionalProperty(S3_REQUESTTIMEOUTMS, DEFAULT_S3_REQUESTTIMEOUTMS); + checkOptionalProperty(S3_CONNTIMEOUTMS, DEFAULT_S3_CONNTIMEOUTMS); + + this.properties.put(MD5_CHECKSUM, calcPropertiesMd5()); + } + + private void checkRequiredProperty(String propertyKey) throws DdlException { + String value = properties.get(propertyKey); + + if (Strings.isNullOrEmpty(value)) { + throw new DdlException("Missing [" + propertyKey + "] in properties."); + } + } + + private void checkOptionalProperty(String propertyKey, String defaultValue) { + this.properties.putIfAbsent(propertyKey, defaultValue); + } + + @Override + public void modifyProperties(Map properties) throws DdlException { + // modify properties + replaceIfEffectiveValue(this.properties, COOLDOWN_DATETIME, properties.get(COOLDOWN_DATETIME)); + replaceIfEffectiveValue(this.properties, COOLDOWN_TTL, properties.get(COOLDOWN_TTL)); + replaceIfEffectiveValue(this.properties, S3_MAXCONN, properties.get(S3_MAXCONN)); + replaceIfEffectiveValue(this.properties, S3_REQUESTTIMEOUTMS, properties.get(S3_REQUESTTIMEOUTMS)); + replaceIfEffectiveValue(this.properties, S3_CONNTIMEOUTMS, properties.get(S3_CONNTIMEOUTMS)); + replaceIfEffectiveValue(this.properties, S3_AK, properties.get(S3_AK)); + replaceIfEffectiveValue(this.properties, S3_SK, properties.get(S3_SK)); + + if (Config.use_default_storage_policy && name.equalsIgnoreCase(Config.default_storage_policy)) { + replaceIfEffectiveValue(this.properties, S3_ENDPOINT, properties.get(S3_ENDPOINT)); + replaceIfEffectiveValue(this.properties, S3_REGION, properties.get(S3_REGION)); + replaceIfEffectiveValue(this.properties, S3_ROOTPATH, properties.get(S3_ROOTPATH)); + replaceIfEffectiveValue(this.properties, S3_BUCKET, properties.get(S3_BUCKET)); + checkDefaultPolicyPropertiesEnough(); + setDefaultStoragePolicyProperties(this.properties); + } + + this.properties.put(MD5_CHECKSUM, calcPropertiesMd5()); + LOG.info("notifyUpdate properties: {}", this.properties); + notifyUpdate(); + } + + public void checkDefaultPolicyPropertiesEnough() throws DdlException { + //check required + checkRequiredProperty(S3_ENDPOINT); + checkRequiredProperty(S3_REGION); + checkRequiredProperty(S3_ROOTPATH); + checkRequiredProperty(S3_AK); + checkRequiredProperty(S3_SK); + checkRequiredProperty(S3_BUCKET); + + //check option, it not exist, use default + checkOptionalProperty(COOLDOWN_DATETIME, DEFAULT_COOLDOWN_DATETIME); + checkOptionalProperty(COOLDOWN_TTL, DEFAULT_COOLDOWN_TTL); + checkOptionalProperty(S3_MAXCONN, DEFAULT_S3_MAXCONN); + checkOptionalProperty(S3_REQUESTTIMEOUTMS, DEFAULT_S3_REQUESTTIMEOUTMS); + checkOptionalProperty(S3_CONNTIMEOUTMS, DEFAULT_S3_CONNTIMEOUTMS); + } + + @Override + public void checkProperties(Map properties) throws AnalysisException { + // check properties + Map copiedProperties = Maps.newHashMap(properties); + copiedProperties.remove(COOLDOWN_DATETIME); + copiedProperties.remove(COOLDOWN_TTL); + copiedProperties.remove(S3_MAXCONN); + copiedProperties.remove(S3_REQUESTTIMEOUTMS); + copiedProperties.remove(S3_CONNTIMEOUTMS); + copiedProperties.remove(S3_ENDPOINT); + copiedProperties.remove(S3_REGION); + copiedProperties.remove(S3_ROOTPATH); + copiedProperties.remove(S3_AK); + copiedProperties.remove(S3_SK); + copiedProperties.remove(S3_BUCKET); + + if (!copiedProperties.isEmpty()) { + throw new AnalysisException("Unknown S3 resource properties: " + copiedProperties); + } + } + + @Override + public Map getCopiedProperties() { + return Maps.newHashMap(properties); + } + + @Override + protected void getProcNodeData(BaseProcResult result) { + String lowerCaseType = type.name().toLowerCase(); + for (Map.Entry entry : properties.entrySet()) { + if (entry.getKey().equalsIgnoreCase(S3_SK)) { + result.addRow(Lists.newArrayList(name, lowerCaseType, entry.getKey(), "******")); + continue; + } + result.addRow(Lists.newArrayList(name, lowerCaseType, entry.getKey(), entry.getValue())); + } + } + + private void notifyUpdate() { + SystemInfoService systemInfoService = Catalog.getCurrentSystemInfo(); + AgentBatchTask batchTask = new AgentBatchTask(); + + for (Long beId : systemInfoService.getBackendIds(true)) { + LOG.info("notifyUpdate be: {}", beId); + NotifyUpdateStoragePolicyTask createReplicaTask = new NotifyUpdateStoragePolicyTask(beId, name, properties); + batchTask.addTask(createReplicaTask); + } + + AgentTaskExecutor.submit(batchTask); + } + + private String calcPropertiesMd5(){ + List calcKey = Arrays.asList(COOLDOWN_DATETIME, COOLDOWN_TTL, S3_MAXCONN, S3_REQUESTTIMEOUTMS, + S3_CONNTIMEOUTMS, S3_AK, S3_SK); + return DigestUtils.md5Hex(calcKey.stream().map(iter -> "(" + iter + ":" + properties.get(iter) + ")").reduce("", String::concat)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java index fd70e718e9cd14..9ddeb99c9c2eb2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java @@ -58,6 +58,8 @@ public class TableProperty implements Writable { private ReplicaAllocation replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION; private boolean isInMemory = false; + private String storagePolicy = ""; + /* * the default storage format of this table. * DEFAULT: depends on BE's config 'default_rowset_type' @@ -139,6 +141,15 @@ public TableProperty buildInMemory() { return this; } + public TableProperty buildStoragePolicy() { + storagePolicy = properties.getOrDefault(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY, ""); + return this; + } + + public String getStoragePolicy(){ + return storagePolicy; + } + public TableProperty buildDataSortInfo() { HashMap dataSortInfoProperties = new HashMap<>(); for (Map.Entry entry : properties.entrySet()) { @@ -265,7 +276,8 @@ public static TableProperty read(DataInput in) throws IOException { .buildStorageFormat() .buildDataSortInfo() .buildRemoteStorageResource() - .buildCompressionType(); + .buildCompressionType() + .buildStoragePolicy(); if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_105) { // get replica num from property map and create replica allocation String repNum = tableProperty.properties.remove(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java index 09d936d79b7e3a..f252933859c7cd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java @@ -127,7 +127,8 @@ private void updateTabletStat(Long beId, TTabletStatResult result) { if (invertedIndex.getTabletMeta(stat.getTabletId()) != null) { Replica replica = invertedIndex.getReplica(stat.getTabletId(), beId); if (replica != null) { - replica.updateStat(stat.getDataSize(), stat.getRowNum(), stat.getVersionCount()); + replica.updateStat(stat.getDataSize(), stat.getRemoteDataSize(), stat.getRowNum(), + stat.getVersionCount()); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java index daff5f1e06d67e..fc9cad998fe1cf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java @@ -841,7 +841,7 @@ public CloneTask createCloneReplicaAndTask() throws SchedException { Replica cloneReplica = new Replica( Catalog.getCurrentCatalog().getNextId(), destBackendId, -1 /* version */, schemaHash, - -1 /* data size */, -1 /* row count */, + -1 /* data size */, -1, -1 /* row count */, ReplicaState.CLONE, committedVersion, /* use committed version as last failed version */ -1 /* last success version */); @@ -982,8 +982,8 @@ public void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) "replica does not exist. backend id: " + destBackendId); } - replica.updateVersionInfo(reportedTablet.getVersion(), - reportedTablet.getDataSize(), reportedTablet.getRowCount()); + replica.updateVersionInfo(reportedTablet.getVersion(), reportedTablet.getDataSize(), + reportedTablet.getRemoteDataSize(), reportedTablet.getRowCount()); if (reportedTablet.isSetPathHash()) { replica.setPathHash(reportedTablet.getPathHash()); } @@ -1004,6 +1004,7 @@ public void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) reportedTablet.getVersion(), reportedTablet.getSchemaHash(), reportedTablet.getDataSize(), + reportedTablet.getRemoteDataSize(), reportedTablet.getRowCount(), replica.getLastFailedVersion(), replica.getLastSuccessVersion()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java index 8e35c5fa239fc5..bd777de54850e2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java @@ -1165,7 +1165,7 @@ private void deleteReplicaInternal(TabletSchedCtx tabletCtx, } private void sendDeleteReplicaTask(long backendId, long tabletId, long replicaId, int schemaHash) { - DropReplicaTask task = new DropReplicaTask(backendId, tabletId, replicaId, schemaHash); + DropReplicaTask task = new DropReplicaTask(backendId, tabletId, replicaId, schemaHash, false); AgentBatchTask batchTask = new AgentBatchTask(); batchTask.addTask(task); AgentTaskExecutor.submit(batchTask); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java index 2b473d7ac00686..f17cd6c3533cd2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java @@ -1664,4 +1664,14 @@ public class Config extends ConfigBase { */ @ConfField(mutable = false, masterOnly = true) public static boolean be_rebalancer_fuzzy_test = false; + + /** + * When create a table(or partition), you can specify its storage policy. + * If not set, this specifies the default policy when created. + */ + @ConfField + public static boolean use_default_storage_policy = false; + @ConfField + public static String default_storage_policy = "default_storage_policy"; + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java index 4cc330afd24d6b..8779e4594709eb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java @@ -50,7 +50,7 @@ public class BackendsProcDir implements ProcDirInterface { .add("BackendId").add("Cluster").add("IP").add("HostName").add("HeartbeatPort") .add("BePort").add("HttpPort").add("BrpcPort").add("LastStartTime").add("LastHeartbeat").add("Alive") .add("SystemDecommissioned").add("ClusterDecommissioned").add("TabletNum") - .add("DataUsedCapacity").add("AvailCapacity").add("TotalCapacity").add("UsedPct") + .add("DataUsedCapacity").add("AvailCapacity").add("TotalCapacity").add("RemoteUsedCapacity").add("UsedPct") .add("MaxDiskUsedPct").add("Tag").add("ErrMsg").add("Version").add("Status") .build(); @@ -154,6 +154,11 @@ public static List> getClusterBackendInfos(String clusterName) { Pair totalCapacity = DebugUtil.getByteUint(totalB); backendInfo.add(DebugUtil.DECIMAL_FORMAT_SCALE_3.format(totalCapacity.first) + " " + totalCapacity.second); + // remote used capacity + long remoteB = backend.getRemoteUsedCapacityB(); + Pair totalRemoteUsedCapacity = DebugUtil.getByteUint(remoteB); + backendInfo.add(DebugUtil.DECIMAL_FORMAT_SCALE_3.format(totalRemoteUsedCapacity.first) + " " + totalRemoteUsedCapacity.second); + // used percent double used = 0.0; if (totalB <= 0) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ReplicasProcNode.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ReplicasProcNode.java index f0242ec80ef048..4a363a91f3ffb8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ReplicasProcNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ReplicasProcNode.java @@ -33,12 +33,10 @@ * show replicas' detail info within a tablet */ public class ReplicasProcNode implements ProcNodeInterface { - public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() - .add("ReplicaId").add("BackendId").add("Version") - .add("LstSuccessVersion").add("LstFailedVersion") - .add("LstFailedTime").add("SchemaHash").add("DataSize").add("RowCount").add("State") - .add("IsBad").add("VersionCount").add("PathHash").add("MetaUrl").add("CompactionStatus") - .build(); + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder().add("ReplicaId") + .add("BackendId").add("Version").add("LstSuccessVersion").add("LstFailedVersion").add("LstFailedTime") + .add("SchemaHash").add("LocalDataSize").add("RemoteDataSize").add("RowCount").add("State").add("IsBad") + .add("VersionCount").add("PathHash").add("MetaUrl").add("CompactionStatus").build(); private long tabletId; private List replicas; @@ -77,6 +75,7 @@ public ProcResult fetchResult() { TimeUtils.longToTimeString(replica.getLastFailedTimestamp()), String.valueOf(replica.getSchemaHash()), String.valueOf(replica.getDataSize()), + String.valueOf(replica.getRemoteDataSize()), String.valueOf(replica.getRowCount()), String.valueOf(replica.getState()), String.valueOf(replica.isBad()), diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletsProcDir.java index 38a3e6739357c6..49a4a6de1cc7ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletsProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletsProcDir.java @@ -45,7 +45,7 @@ public class TabletsProcDir implements ProcDirInterface { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("TabletId").add("ReplicaId").add("BackendId").add("SchemaHash").add("Version") .add("LstSuccessVersion").add("LstFailedVersion").add("LstFailedTime") - .add("DataSize").add("RowCount").add("State") + .add("LocalDataSize").add("RemoteDataSize").add("RowCount").add("State") .add("LstConsistencyCheckTime").add("CheckVersion") .add("VersionCount").add("PathHash").add("MetaUrl").add("CompactionStatus") .build(); @@ -84,6 +84,7 @@ public List> fetchComparableResult(long version, long backendId tabletInfo.add(-1); // lst failed version hash tabletInfo.add(-1); // lst failed time tabletInfo.add(-1); // data size + tabletInfo.add(-1); // remote data size tabletInfo.add(-1); // row count tabletInfo.add(FeConstants.null_string); // state tabletInfo.add(-1); // lst consistency check time @@ -113,6 +114,7 @@ public List> fetchComparableResult(long version, long backendId tabletInfo.add(replica.getLastFailedVersion()); tabletInfo.add(TimeUtils.longToTimeString(replica.getLastFailedTimestamp())); tabletInfo.add(replica.getDataSize()); + tabletInfo.add(replica.getRemoteDataSize()); tabletInfo.add(replica.getRowCount()); tabletInfo.add(replica.getState()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java index 88ac119aeb8181..ffac22c05a83ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java @@ -109,6 +109,8 @@ public class PropertyAnalyzer { public static final String PROPERTIES_DISABLE_LOAD = "disable_load"; + public static final String PROPERTIES_STORAGE_POLICY = "storage_policy"; + public static DataProperty analyzeDataProperty(Map properties, DataProperty oldDataProperty) throws AnalysisException { if (properties == null || properties.isEmpty()) { @@ -119,11 +121,13 @@ public static DataProperty analyzeDataProperty(Map properties, D long cooldownTimeStamp = DataProperty.MAX_COOLDOWN_TIME_MS; String remoteStorageResourceName = ""; long remoteCooldownTimeStamp = DataProperty.MAX_COOLDOWN_TIME_MS; + String storagePolicy = ""; boolean hasMedium = false; boolean hasCooldown = false; boolean hasRemoteStorageResource = false; boolean hasRemoteCooldown = false; + boolean hasStoragePolicy = false; for (Map.Entry entry : properties.entrySet()) { String key = entry.getKey(); String value = entry.getValue(); @@ -153,6 +157,11 @@ public static DataProperty analyzeDataProperty(Map properties, D if (remoteCooldownTimeStamp != DataProperty.MAX_COOLDOWN_TIME_MS) { hasRemoteCooldown = true; } + } else if (!hasStoragePolicy && key.equalsIgnoreCase(PROPERTIES_STORAGE_POLICY)) { + if (!Strings.isNullOrEmpty(value)) { + hasStoragePolicy = true; + storagePolicy = value; + } } } // end for properties @@ -213,7 +222,7 @@ public static DataProperty analyzeDataProperty(Map properties, D } Preconditions.checkNotNull(storageMedium); - return new DataProperty(storageMedium, cooldownTimeStamp, remoteStorageResourceName, remoteCooldownTimeStamp); + return new DataProperty(storageMedium, cooldownTimeStamp, remoteStorageResourceName, remoteCooldownTimeStamp, storagePolicy); } public static short analyzeShortKeyColumnCount(Map properties) throws AnalysisException { @@ -516,6 +525,15 @@ public static String analyzeRemoteStorageResource(Map properties return resourceName; } + public static String analyzeStoragePolicy(Map properties) throws AnalysisException { + String storagePolicy = ""; + if (properties != null && properties.containsKey(PROPERTIES_STORAGE_POLICY)) { + storagePolicy = properties.get(PROPERTIES_STORAGE_POLICY); + } + + return storagePolicy; + } + // analyze property like : "type" = "xxx"; public static String analyzeType(Map properties) throws AnalysisException { String type = null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java index 97ba9e20db55b8..9ca4daec7ddb6a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java @@ -2316,7 +2316,8 @@ public void unprotectQuorumLoadJob(LoadJob job, Database db) { LOG.warn("the replica[{}] is missing", info.getReplicaId()); continue; } - replica.updateVersionInfo(info.getVersion(), info.getDataSize(), info.getRowCount()); + replica.updateVersionInfo(info.getVersion(), info.getDataSize(), info.getRemoteDataSize(), + info.getRowCount()); } } @@ -2428,7 +2429,8 @@ public void unprotectFinishLoadJob(LoadJob job, Database db) { LOG.warn("the replica[{}] is missing", info.getReplicaId()); continue; } - replica.updateVersionInfo(info.getVersion(), info.getDataSize(), info.getRowCount()); + replica.updateVersionInfo(info.getVersion(), info.getDataSize(), info.getRemoteDataSize(), + info.getRowCount()); } } } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java b/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java index 28101560030562..d9e8c500f26130 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java @@ -641,6 +641,7 @@ private ReplicaPersistInfo updateReplicaInfo(OlapTable olapTable, Partition part long version = tTabletInfo.getVersion(); long rowCount = tTabletInfo.getRowCount(); long dataSize = tTabletInfo.getDataSize(); + long remoteDataSize = tTabletInfo.getRemoteDataSize(); if (indexId != pushIndexId) { // this may be a rollup tablet @@ -673,11 +674,11 @@ private ReplicaPersistInfo updateReplicaInfo(OlapTable olapTable, Partition part throw new MetaNotFoundException("cannot find replica in tablet[" + tabletId + "], backend[" + backendId + "]"); } - replica.updateVersionInfo(version, dataSize, rowCount); + replica.updateVersionInfo(version, dataSize, remoteDataSize, rowCount); LOG.debug("replica[{}] report schemaHash:{}", replica.getId(), schemaHash); return ReplicaPersistInfo.createForLoad(olapTable.getId(), partition.getId(), pushIndexId, tabletId, - replica.getId(), version, schemaHash, dataSize, rowCount); + replica.getId(), version, schemaHash, dataSize, remoteDataSize, rowCount); } private void finishDropReplica(AgentTask task) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index 1f5e146bd4b226..4b4056c5bfeff5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -455,14 +455,15 @@ private static void sync(Map backendTablets, ListMultimap backendTablets, ListMultimap backendTablets, ListMultimap tabletDeleteFromMeta olapTable.getCopiedIndexes(), olapTable.isInMemory(), olapTable.getPartitionInfo().getTabletType(partitionId), - olapTable.getCompressionType()); + olapTable.getCompressionType(), + olapTable.getStoragePolicy()); createReplicaTask.setIsRecoverTask(true); createReplicaBatchTask.addTask(createReplicaTask); } else { @@ -696,10 +698,8 @@ private static void deleteFromBackend(Map backendTablets, } if (needDelete) { - // drop replica - long replicaId = backendTabletInfo.getReplicaId(); - DropReplicaTask task = new DropReplicaTask(backendId, tabletId, replicaId, - backendTabletInfo.getSchemaHash()); + boolean isDropTableOrPartition = Catalog.getCurrentInvertedIndex().getTabletMeta(tabletId) == null; + DropReplicaTask task = new DropReplicaTask(backendId, tabletId, replicaId, backendTabletInfo.getSchemaHash(), isDropTableOrPartition); batchTask.addTask(task); LOG.warn("delete tablet[" + tabletId + "] from backend[" + backendId + "] because not found in meta"); ++deleteFromBackendCounter; @@ -897,6 +897,7 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon int schemaHash = backendTabletInfo.getSchemaHash(); long version = backendTabletInfo.getVersion(); long dataSize = backendTabletInfo.getDataSize(); + long remoteDataSize = backendTabletInfo.getRemoteDataSize(); long rowCount = backendTabletInfo.getRowCount(); Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId); @@ -964,14 +965,14 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon long replicaId = Catalog.getCurrentCatalog().getNextId(); Replica replica = new Replica(replicaId, backendId, version, schemaHash, - dataSize, rowCount, ReplicaState.NORMAL, + dataSize, remoteDataSize, rowCount, ReplicaState.NORMAL, lastFailedVersion, version); tablet.addReplica(replica); // write edit log ReplicaPersistInfo info = ReplicaPersistInfo.createForAdd(dbId, tableId, partitionId, indexId, tabletId, backendId, replicaId, - version, schemaHash, dataSize, rowCount, + version, schemaHash, dataSize, remoteDataSize, rowCount, lastFailedVersion, version); diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyPartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyPartitionInfo.java index 1cd1c55e13101b..6a95b6ccce0696 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyPartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyPartitionInfo.java @@ -48,19 +48,27 @@ public class ModifyPartitionInfo implements Writable { @SerializedName(value = "replicaAlloc") private ReplicaAllocation replicaAlloc; + @SerializedName(value = "storagePolicy") + private String storagePolicy; + + public String getStoragePolicy() { + return storagePolicy; + } + public ModifyPartitionInfo() { // for persist } public ModifyPartitionInfo(long dbId, long tableId, long partitionId, DataProperty dataProperty, ReplicaAllocation replicaAlloc, - boolean isInMemory) { + boolean isInMemory, String storagePolicy) { this.dbId = dbId; this.tableId = tableId; this.partitionId = partitionId; this.dataProperty = dataProperty; this.replicaAlloc = replicaAlloc; this.isInMemory = isInMemory; + this.storagePolicy = storagePolicy; } public long getDbId() { @@ -107,9 +115,9 @@ public boolean equals(Object other) { return false; } ModifyPartitionInfo otherInfo = (ModifyPartitionInfo) other; - return dbId == otherInfo.getDbId() && tableId == otherInfo.getTableId() - && dataProperty.equals(otherInfo.getDataProperty()) && replicaAlloc.equals(otherInfo.replicaAlloc) - && isInMemory == otherInfo.isInMemory(); + return dbId == otherInfo.getDbId() && tableId == otherInfo.getTableId() && + dataProperty.equals(otherInfo.getDataProperty()) && replicaAlloc.equals(otherInfo.replicaAlloc) + && isInMemory == otherInfo.isInMemory() && storagePolicy.equals(otherInfo.getStoragePolicy()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/ReplicaPersistInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/ReplicaPersistInfo.java index 6a4fe11eadaf54..d89f238ebf9d97 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/ReplicaPersistInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/ReplicaPersistInfo.java @@ -96,6 +96,7 @@ public static ReplicaOperationType findByValue(int value) { private long versionHash = 0L; private int schemaHash = -1; private long dataSize; + private long remoteDataSize; private long rowCount; private long lastFailedVersion = -1L; @@ -107,13 +108,13 @@ public static ReplicaOperationType findByValue(int value) { public static ReplicaPersistInfo createForAdd(long dbId, long tableId, long partitionId, long indexId, long tabletId, long backendId, long replicaId, long version, - int schemaHash, long dataSize, long rowCount, + int schemaHash, long dataSize, long remoteDataSize, long rowCount, long lastFailedVersion, long lastSuccessVersion) { return new ReplicaPersistInfo(ReplicaOperationType.ADD, dbId, tableId, partitionId, indexId, tabletId, backendId, - replicaId, version, schemaHash, dataSize, rowCount, + replicaId, version, schemaHash, dataSize, remoteDataSize, rowCount, lastFailedVersion, lastSuccessVersion); } @@ -121,12 +122,12 @@ public static ReplicaPersistInfo createForAdd(long dbId, long tableId, long part * this for delete stmt operation */ public static ReplicaPersistInfo createForCondDelete(long indexId, long tabletId, long replicaId, long version, - int schemaHash, long dataSize, long rowCount, + int schemaHash, long dataSize, long remoteDataSize, long rowCount, long lastFailedVersion, long lastSuccessVersion) { return new ReplicaPersistInfo(ReplicaOperationType.CROND_DELETE, -1L, -1L, -1L, indexId, tabletId, -1L, - replicaId, version, schemaHash, dataSize, rowCount, + replicaId, version, schemaHash, dataSize, remoteDataSize, rowCount, lastFailedVersion, lastSuccessVersion); } @@ -137,62 +138,62 @@ public static ReplicaPersistInfo createForDelete(long dbId, long tableId, long p long tabletId, long backendId) { return new ReplicaPersistInfo(ReplicaOperationType.DELETE, dbId, tableId, partitionId, indexId, tabletId, backendId, - -1L, -1L, -1, -1L, -1L, -1L, -1L); + -1L, -1L, -1, -1L, -1L, -1L, -1L, -1L); } public static ReplicaPersistInfo createForClone(long dbId, long tableId, long partitionId, long indexId, long tabletId, long backendId, long replicaId, long version, - int schemaHash, long dataSize, long rowCount, + int schemaHash, long dataSize, long remoteDataSize, long rowCount, long lastFailedVersion, long lastSuccessVersion) { return new ReplicaPersistInfo(ReplicaOperationType.CLONE, dbId, tableId, partitionId, indexId, tabletId, backendId, replicaId, - version, schemaHash, dataSize, rowCount, + version, schemaHash, dataSize, remoteDataSize, rowCount, lastFailedVersion, lastSuccessVersion); } // for original batch load, the last success version = version, last success version hash = version hash // last failed version = -1 public static ReplicaPersistInfo createForLoad(long tableId, long partitionId, long indexId, long tabletId, - long replicaId, long version, int schemaHash, long dataSize, long rowCount) { + long replicaId, long version, int schemaHash, long dataSize, long remoteDataSize, long rowCount) { return new ReplicaPersistInfo(ReplicaOperationType.LOAD, -1L, tableId, partitionId, indexId, tabletId, -1L, - replicaId, version, schemaHash, dataSize, + replicaId, version, schemaHash, dataSize, remoteDataSize, rowCount, -1L, version); } public static ReplicaPersistInfo createForRollup(long indexId, long tabletId, long backendId, long version, - int schemaHash, long dataSize, long rowCount, + int schemaHash, long dataSize, long remoteDataSize, long rowCount, long lastFailedVersion, long lastSuccessVersion) { return new ReplicaPersistInfo(ReplicaOperationType.ROLLUP, -1L, -1L, -1L, indexId, tabletId, backendId, -1L, - version, schemaHash, dataSize, rowCount, + version, schemaHash, dataSize, remoteDataSize, rowCount, lastFailedVersion, lastSuccessVersion); } public static ReplicaPersistInfo createForSchemaChange(long partitionId, long indexId, long tabletId, long backendId, long version, - int schemaHash, long dataSize, long rowCount, + int schemaHash, long dataSize, long remoteDataSize, long rowCount, long lastFailedVersion, long lastSuccessVersion) { return new ReplicaPersistInfo(ReplicaOperationType.SCHEMA_CHANGE, -1L, -1L, partitionId, indexId, tabletId, backendId, -1L, version, - schemaHash, dataSize, rowCount, lastFailedVersion, + schemaHash, dataSize, remoteDataSize, rowCount, lastFailedVersion, lastSuccessVersion); } public static ReplicaPersistInfo createForClearRollupInfo(long dbId, long tableId, long partitionId, long indexId) { return new ReplicaPersistInfo(ReplicaOperationType.CLEAR_ROLLUPINFO, - dbId, tableId, partitionId, indexId, -1L, -1L, -1L, -1L, -1, -1L, -1L, -1L, -1L); + dbId, tableId, partitionId, indexId, -1L, -1L, -1L, -1L, -1, -1L, -1L, -1L, -1L, -1L); } - public static ReplicaPersistInfo createForReport(long dbId, long tblId, long partitionId, long indexId, - long tabletId, long backendId, long replicaId) { - return new ReplicaPersistInfo(ReplicaOperationType.TABLET_INFO, dbId, tblId, partitionId, - indexId, tabletId, backendId, replicaId, -1L, -1, -1L, -1L, -1L, -1L); + public static ReplicaPersistInfo createForReport(long dbId, long tblId, long partitionId, long indexId, long tabletId, + long backendId, long replicaId) { + return new ReplicaPersistInfo(ReplicaOperationType.TABLET_INFO, dbId, tblId, partitionId, indexId, tabletId, backendId, replicaId, + -1L, -1, -1L, -1L, -1L, -1L, -1L); } @@ -201,7 +202,7 @@ private ReplicaPersistInfo() { private ReplicaPersistInfo(ReplicaOperationType opType, long dbId, long tableId, long partitionId, long indexId, long tabletId, long backendId, long replicaId, long version, - int schemaHash, long dataSize, long rowCount, long lastFailedVersion, + int schemaHash, long dataSize, long remoteDataSize, long rowCount, long lastFailedVersion, long lastSuccessVersion) { this.opType = opType; this.dbId = dbId; @@ -214,6 +215,7 @@ private ReplicaPersistInfo(ReplicaOperationType opType, long dbId, long tableId, this.version = version; this.schemaHash = schemaHash; this.dataSize = dataSize; + this.remoteDataSize = remoteDataSize; this.rowCount = rowCount; this.lastFailedVersion = lastFailedVersion; @@ -264,6 +266,10 @@ public long getDataSize() { return dataSize; } + public long getRemoteDataSize() { + return remoteDataSize; + } + public long getRowCount() { return rowCount; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java index 2192a4edbbd8f9..e535f9d88122d7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java @@ -30,6 +30,7 @@ import org.apache.doris.catalog.S3Resource; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.SparkResource; +import org.apache.doris.catalog.StoragePolicyResource; import org.apache.doris.catalog.StructType; import org.apache.doris.load.loadv2.LoadJob.LoadJobStateUpdateInfo; import org.apache.doris.load.loadv2.SparkLoadJob.SparkLoadJobStateUpdateInfo; @@ -116,7 +117,8 @@ public class GsonUtils { .of(Resource.class, "clazz") .registerSubtype(SparkResource.class, SparkResource.class.getSimpleName()) .registerSubtype(OdbcCatalogResource.class, OdbcCatalogResource.class.getSimpleName()) - .registerSubtype(S3Resource.class, S3Resource.class.getSimpleName()); + .registerSubtype(S3Resource.class, S3Resource.class.getSimpleName()) + .registerSubtype(StoragePolicyResource.class, StoragePolicyResource.class.getSimpleName()); // runtime adapter for class "AlterJobV2" private static RuntimeTypeAdapterFactory alterJobV2TypeAdapterFactory = RuntimeTypeAdapterFactory diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index 67aa2baff08032..bd8f143e2bc829 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -25,6 +25,7 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TableIf.TableType; +import org.apache.doris.catalog.Resource; import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.AuthenticationException; @@ -109,6 +110,9 @@ import org.apache.doris.thrift.TUpdateMiniEtlTaskStatusRequest; import org.apache.doris.thrift.TWaitingTxnStatusRequest; import org.apache.doris.thrift.TWaitingTxnStatusResult; +import org.apache.doris.thrift.TGetStoragePolicyResult; +import org.apache.doris.thrift.TGetStoragePolicy; +import org.apache.doris.thrift.TS3StorageParam; import org.apache.doris.transaction.DatabaseTransactionMgr; import org.apache.doris.transaction.TabletCommitInfo; import org.apache.doris.transaction.TransactionState; @@ -127,11 +131,30 @@ import java.net.InetAddress; import java.net.UnknownHostException; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import static org.apache.doris.catalog.Resource.ResourceType.STORAGE_POLICY; +import static org.apache.doris.catalog.StoragePolicyResource.COOLDOWN_DATETIME; +import static org.apache.doris.catalog.StoragePolicyResource.COOLDOWN_TTL; +import static org.apache.doris.catalog.StoragePolicyResource.S3_ENDPOINT; +import static org.apache.doris.catalog.StoragePolicyResource.S3_REGION; +import static org.apache.doris.catalog.StoragePolicyResource.S3_ROOTPATH; +import static org.apache.doris.catalog.StoragePolicyResource.S3_AK; +import static org.apache.doris.catalog.StoragePolicyResource.S3_SK; +import static org.apache.doris.catalog.StoragePolicyResource.S3_BUCKET; +import static org.apache.doris.catalog.StoragePolicyResource.S3_MAXCONN; +import static org.apache.doris.catalog.StoragePolicyResource.S3_REQUESTTIMEOUTMS; +import static org.apache.doris.catalog.StoragePolicyResource.S3_CONNTIMEOUTMS; +import static org.apache.doris.catalog.StoragePolicyResource.MD5_CHECKSUM; + +import static org.apache.doris.thrift.TStatusCode.NOT_IMPLEMENTED_ERROR; + // Frontend service used to serve all request for this frontend through // thrift protocol public class FrontendServiceImpl implements FrontendService.Iface { @@ -1199,4 +1222,51 @@ public TWaitingTxnStatusResult waitingTxnStatus(TWaitingTxnStatusRequest request return result; } + @Override + public TGetStoragePolicyResult refreshStoragePolicy() throws TException { + LOG.debug("refresh storage policy request"); + TGetStoragePolicyResult result = new TGetStoragePolicyResult(); + TStatus status = new TStatus(TStatusCode.OK); + result.setStatus(status); + + Map storagePolicyMap = Catalog.getCurrentCatalog().getResourceMgr().getResourceByType(STORAGE_POLICY); + for (Map.Entry entry : storagePolicyMap.entrySet()){ + Map storagePolicyProperties = entry.getValue().getCopiedProperties(); + // 9999-01-01 00:00:00 => unix timestamp + SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + Date date = null; + try { + date = dateFormat.parse(storagePolicyProperties.get(COOLDOWN_DATETIME)); + } catch (ParseException e) { + e.printStackTrace(); + result.status.setStatusCode(TStatusCode.INTERNAL_ERROR); + result.status.addToErrorMsgs("date to timestamp error : " + storagePolicyProperties.get(COOLDOWN_DATETIME)); + LOG.warn("date to timestamp error : ", storagePolicyProperties.get(COOLDOWN_DATETIME)); + continue; + } + + long secondTimestamp = date.getTime() / 1000; + TGetStoragePolicy rEntry = new TGetStoragePolicy(); + rEntry.setPolicyName(entry.getKey()); + rEntry.setCooldownDatetime(secondTimestamp); + rEntry.setMd5Checksum(storagePolicyProperties.get(MD5_CHECKSUM)); + // "3600" => 3600 + rEntry.setCooldownTtl(Integer.parseInt(storagePolicyProperties.get(COOLDOWN_TTL))); + TS3StorageParam s3Info = new TS3StorageParam(); + s3Info.setS3Endpoint(storagePolicyProperties.get(S3_ENDPOINT)); + s3Info.setS3Region(storagePolicyProperties.get(S3_REGION)); + s3Info.setRootPath(storagePolicyProperties.get(S3_ROOTPATH)); + s3Info.setS3Ak(storagePolicyProperties.get(S3_AK)); + s3Info.setS3Sk(storagePolicyProperties.get(S3_SK)); + s3Info.setBucket(storagePolicyProperties.get(S3_BUCKET)); + s3Info.setS3MaxConn(Integer.parseInt(storagePolicyProperties.get(S3_MAXCONN))); + s3Info.setS3RequestTimeoutMs(Integer.parseInt(storagePolicyProperties.get(S3_REQUESTTIMEOUTMS))); + s3Info.setS3ConnTimeoutMs(Integer.parseInt(storagePolicyProperties.get(S3_CONNTIMEOUTMS))); + rEntry.setS3StorageParam(s3Info); + result.addToResultEntrys(rEntry); + } + LOG.debug("refresh storage policy response: {}", result); + + return result; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java b/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java index d8267c21e34ae3..c4f4070a381312 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java @@ -373,6 +373,17 @@ public long getTotalCapacityB() { return totalCapacityB; } + public long getRemoteUsedCapacityB() { + ImmutableMap disks = disksRef; + long totalRemoteUsedCapacityB = 0L; + for (DiskInfo diskInfo : disks.values()) { + if (diskInfo.getState() == DiskState.ONLINE) { + totalRemoteUsedCapacityB += diskInfo.getRemoteUsedCapacity(); + } + } + return totalRemoteUsedCapacityB; + } + public long getAvailableCapacityB() { // when cluster init, disks is empty, return 1L. ImmutableMap disks = disksRef; @@ -495,6 +506,10 @@ public void updateDisks(Map backendDisks) { diskInfo.setTotalCapacityB(totalCapacityB); diskInfo.setDataUsedCapacityB(dataUsedCapacityB); diskInfo.setAvailableCapacityB(diskAvailableCapacityB); + if (tDisk.isSetRemoteUsedCapacity()) { + diskInfo.setRemoteUsedCapacity(tDisk.getRemoteUsedCapacity()); + } + if (tDisk.isSetPathHash()) { diskInfo.setPathHash(tDisk.getPathHash()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java index bd7380839cce4c..6d09d4f0344386 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java @@ -44,6 +44,7 @@ import org.apache.doris.thrift.TTaskType; import org.apache.doris.thrift.TUpdateTabletMetaInfoReq; import org.apache.doris.thrift.TUploadReq; +import org.apache.doris.thrift.TGetStoragePolicy; import com.google.common.collect.Lists; import org.apache.logging.log4j.LogManager; @@ -350,6 +351,15 @@ private TAgentTaskRequest toAgentTaskRequest(AgentTask task) { tAgentTaskRequest.setCompactionReq(request); return tAgentTaskRequest; } + case NOTIFY_UPDATE_STORAGE_POLICY: { + NotifyUpdateStoragePolicyTask notifyUpdateStoragePolicyTask = (NotifyUpdateStoragePolicyTask) task; + TGetStoragePolicy request = notifyUpdateStoragePolicyTask.toThrift(); + if (LOG.isDebugEnabled()) { + LOG.debug(request.toString()); + } + tAgentTaskRequest.setUpdatePolicy(request); + return tAgentTaskRequest; + } default: LOG.debug("could not find task type for task [{}]", task); return null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java index 0e45b86cf23e9e..749148e3c3979b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java @@ -89,6 +89,8 @@ public class CreateReplicaTask extends AgentTask { private DataSortInfo dataSortInfo; + private static String storagePolicy; + public CreateReplicaTask(long backendId, long dbId, long tableId, long partitionId, long indexId, long tabletId, long replicaId, short shortKeyColumnCount, int schemaHash, long version, KeysType keysType, TStorageType storageType, @@ -96,7 +98,8 @@ public CreateReplicaTask(long backendId, long dbId, long tableId, long partition Set bfColumns, double bfFpp, MarkedCountDownLatch latch, List indexes, boolean isInMemory, - TTabletType tabletType, TCompressionType compressionType) { + TTabletType tabletType, TCompressionType compressionType, + String storagePolicy) { super(null, backendId, TTaskType.CREATE, dbId, tableId, partitionId, indexId, tabletId); this.replicaId = replicaId; @@ -120,6 +123,7 @@ public CreateReplicaTask(long backendId, long dbId, long tableId, long partition this.isInMemory = isInMemory; this.tabletType = tabletType; + this.storagePolicy = storagePolicy; } public CreateReplicaTask(long backendId, long dbId, long tableId, long partitionId, long indexId, long tabletId, @@ -131,7 +135,8 @@ public CreateReplicaTask(long backendId, long dbId, long tableId, long partition boolean isInMemory, TTabletType tabletType, DataSortInfo dataSortInfo, - TCompressionType compressionType) { + TCompressionType compressionType, + String storagePolicy) { super(null, backendId, TTaskType.CREATE, dbId, tableId, partitionId, indexId, tabletId); this.replicaId = replicaId; @@ -156,6 +161,7 @@ public CreateReplicaTask(long backendId, long dbId, long tableId, long partition this.isInMemory = isInMemory; this.tabletType = tabletType; this.dataSortInfo = dataSortInfo; + this.storagePolicy = storagePolicy; } public void setIsRecoverTask(boolean isRecoverTask) { @@ -259,6 +265,8 @@ public TCreateTabletReq toThrift() { createTabletReq.setVersion(version); createTabletReq.setStorageMedium(storageMedium); + createTabletReq.setStoragePolicy(storagePolicy); + if (inRestoreMode) { createTabletReq.setInRestoreMode(true); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/DropReplicaTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/DropReplicaTask.java index 18643acd6c8426..ff95b53da67dde 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/DropReplicaTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/DropReplicaTask.java @@ -23,11 +23,13 @@ public class DropReplicaTask extends AgentTask { private int schemaHash; // set -1L as unknown private long replicaId; + private boolean isDropTableOrPartition; - public DropReplicaTask(long backendId, long tabletId, long replicaId, int schemaHash) { + public DropReplicaTask(long backendId, long tabletId, long replicaId, int schemaHash, boolean isDropTableOrPartition) { super(null, backendId, TTaskType.DROP, -1L, -1L, -1L, -1L, tabletId); this.schemaHash = schemaHash; this.replicaId = replicaId; + this.isDropTableOrPartition = isDropTableOrPartition; } public TDropTabletReq toThrift() { @@ -36,6 +38,7 @@ public TDropTabletReq toThrift() { request.setSchemaHash(schemaHash); } request.setReplicaId(replicaId); + request.setIsDropTableOrPartition(isDropTableOrPartition); return request; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/NotifyUpdateStoragePolicyTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/NotifyUpdateStoragePolicyTask.java new file mode 100644 index 00000000000000..3f80d4df171fbe --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/task/NotifyUpdateStoragePolicyTask.java @@ -0,0 +1,87 @@ +// 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. + +package org.apache.doris.task; + +import org.apache.doris.thrift.TGetStoragePolicy; +import org.apache.doris.thrift.TS3StorageParam; +import org.apache.doris.thrift.TTaskType; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.Map; + +import static org.apache.doris.catalog.StoragePolicyResource.COOLDOWN_DATETIME; +import static org.apache.doris.catalog.StoragePolicyResource.COOLDOWN_TTL; +import static org.apache.doris.catalog.StoragePolicyResource.S3_MAXCONN; +import static org.apache.doris.catalog.StoragePolicyResource.S3_REQUESTTIMEOUTMS; +import static org.apache.doris.catalog.StoragePolicyResource.S3_CONNTIMEOUTMS; +import static org.apache.doris.catalog.StoragePolicyResource.S3_ENDPOINT; +import static org.apache.doris.catalog.StoragePolicyResource.S3_REGION; +import static org.apache.doris.catalog.StoragePolicyResource.S3_ROOTPATH; +import static org.apache.doris.catalog.StoragePolicyResource.S3_AK; +import static org.apache.doris.catalog.StoragePolicyResource.S3_SK; +import static org.apache.doris.catalog.StoragePolicyResource.MD5_CHECKSUM; +import static org.apache.doris.catalog.StoragePolicyResource.S3_BUCKET; + +public class NotifyUpdateStoragePolicyTask extends AgentTask { + private static final Logger LOG = LogManager.getLogger(NotifyUpdateStoragePolicyTask.class); + private String policyName; + + private String md5CheckSum; + private Map properties; + + public NotifyUpdateStoragePolicyTask(long backendId, String name, Map properties) { + super(null, backendId, TTaskType.NOTIFY_UPDATE_STORAGE_POLICY, -1, -1, -1, -1, -1, -1, -1); + this.policyName = name; + this.properties = properties; + } + + public TGetStoragePolicy toThrift() { + TGetStoragePolicy ret = new TGetStoragePolicy(); + + ret.policy_name = policyName; + // 9999-01-01 00:00:00 => unix timestamp + SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + Date date = null; + try { + date = dateFormat.parse(properties.get(COOLDOWN_DATETIME)); + } catch (ParseException e) { + e.printStackTrace(); + } + LOG.info("toThrift map {}", properties); + long secondTimestamp = date.getTime() / 1000; + ret.cooldown_datetime = secondTimestamp; + ret.cooldown_ttl = Long.parseLong(properties.get(COOLDOWN_TTL)); + ret.s3_storage_param = new TS3StorageParam(); + ret.s3_storage_param.s3_max_conn = Integer.parseInt(properties.get(S3_MAXCONN)); + ret.s3_storage_param.s3_request_timeout_ms = Integer.parseInt(properties.get(S3_REQUESTTIMEOUTMS)); + ret.s3_storage_param.s3_conn_timeout_ms = Integer.parseInt(properties.get(S3_CONNTIMEOUTMS)); + ret.s3_storage_param.s3_endpoint = properties.get(S3_ENDPOINT); + ret.s3_storage_param.s3_region = properties.get(S3_REGION); + ret.s3_storage_param.root_path = properties.get(S3_ROOTPATH); + ret.s3_storage_param.s3_ak = properties.get(S3_AK); + ret.s3_storage_param.s3_sk = properties.get(S3_SK); + ret.s3_storage_param.bucket = properties.get(S3_BUCKET); + ret.md5_checksum = properties.get(MD5_CHECKSUM); + + return ret; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/UpdateTabletMetaInfoTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/UpdateTabletMetaInfoTask.java index 1e1b9d8ec96f12..6e8f971a7e2927 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/UpdateTabletMetaInfoTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/UpdateTabletMetaInfoTask.java @@ -46,6 +46,7 @@ public class UpdateTabletMetaInfoTask extends AgentTask { private Set> tableIdWithSchemaHash; private boolean isInMemory; private TTabletMetaType metaType; + private String storagePolicy; // private List> tabletToInMemory; @@ -60,9 +61,10 @@ public UpdateTabletMetaInfoTask(long backendId, Set> tableId public UpdateTabletMetaInfoTask(long backendId, Set> tableIdWithSchemaHash, - boolean isInMemory, + boolean isInMemory, String storagePolicy, MarkedCountDownLatch>> latch) { this(backendId, tableIdWithSchemaHash, TTabletMetaType.INMEMORY); + this.storagePolicy = storagePolicy; this.isInMemory = isInMemory; this.latch = latch; } @@ -131,6 +133,7 @@ public TUpdateTabletMetaInfoReq toThrift() { metaInfo.setTabletId(pair.first); metaInfo.setSchemaHash(pair.second); metaInfo.setIsInMemory(isInMemory); + metaInfo.setStoragePolicy(storagePolicy); metaInfo.setMetaType(metaType); metaInfos.add(metaInfo); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java index 85c7ba577c5882..d002739e990ad6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java @@ -447,13 +447,13 @@ public void testBatchUpdatePartitionProperties() throws Exception { stmt = "alter table test.tbl4 modify partition (p3, p4) set ('storage_medium' = 'HDD')"; DateLiteral dateLiteral = new DateLiteral("2999-12-31 00:00:00", Type.DATETIME); long cooldownTimeMs = dateLiteral.unixTimestamp(TimeUtils.getTimeZone()); - DataProperty oldDataProperty = new DataProperty(TStorageMedium.SSD, cooldownTimeMs, "", DataProperty.MAX_COOLDOWN_TIME_MS); + DataProperty oldDataProperty = new DataProperty(TStorageMedium.SSD, cooldownTimeMs, "", DataProperty.MAX_COOLDOWN_TIME_MS, ""); partitionList = Lists.newArrayList(p3, p4); for (Partition partition : partitionList) { Assert.assertEquals(oldDataProperty, tbl4.getPartitionInfo().getDataProperty(partition.getId())); } alterTable(stmt, false); - DataProperty newDataProperty = new DataProperty(TStorageMedium.HDD, DataProperty.MAX_COOLDOWN_TIME_MS, "", DataProperty.MAX_COOLDOWN_TIME_MS); + DataProperty newDataProperty = new DataProperty(TStorageMedium.HDD, DataProperty.MAX_COOLDOWN_TIME_MS, "", DataProperty.MAX_COOLDOWN_TIME_MS, ""); for (Partition partition : partitionList) { Assert.assertEquals(newDataProperty, tbl4.getPartitionInfo().getDataProperty(partition.getId())); } @@ -466,7 +466,7 @@ public void testBatchUpdatePartitionProperties() throws Exception { dateLiteral = new DateLiteral("2100-12-31 00:00:00", Type.DATETIME); cooldownTimeMs = dateLiteral.unixTimestamp(TimeUtils.getTimeZone()); - DataProperty newDataProperty1 = new DataProperty(TStorageMedium.SSD, cooldownTimeMs, "", DataProperty.MAX_COOLDOWN_TIME_MS); + DataProperty newDataProperty1 = new DataProperty(TStorageMedium.SSD, cooldownTimeMs, "", DataProperty.MAX_COOLDOWN_TIME_MS, ""); partitionList = Lists.newArrayList(p1, p2); for (Partition partition : partitionList) { Assert.assertEquals(newDataProperty1, tbl4.getPartitionInfo().getDataProperty(partition.getId())); @@ -496,7 +496,7 @@ public void testAlterRemoteStorageTableDataProperties() throws Exception { long cooldownTimeMs = dateLiteral.unixTimestamp(TimeUtils.getTimeZone()); DateLiteral dateLiteral1 = new DateLiteral("2122-12-01 20:23:00", Type.DATETIME); long remoteCooldownTimeMs = dateLiteral1.unixTimestamp(TimeUtils.getTimeZone()); - DataProperty oldDataProperty = new DataProperty(TStorageMedium.SSD, cooldownTimeMs, "remote_s3", remoteCooldownTimeMs); + DataProperty oldDataProperty = new DataProperty(TStorageMedium.SSD, cooldownTimeMs, "remote_s3", remoteCooldownTimeMs, ""); List partitionList = Lists.newArrayList(p2, p3, p4); for (Partition partition : partitionList) { Assert.assertEquals(oldDataProperty, tblRemote.getPartitionInfo().getDataProperty(partition.getId())); @@ -507,7 +507,7 @@ public void testAlterRemoteStorageTableDataProperties() throws Exception { alterTable(stmt, false); DateLiteral newDateLiteral = new DateLiteral("2100-04-01 22:22:22", Type.DATETIME); long newCooldownTimeMs = newDateLiteral.unixTimestamp(TimeUtils.getTimeZone()); - DataProperty dataProperty2 = new DataProperty(TStorageMedium.SSD, newCooldownTimeMs, "remote_s3", remoteCooldownTimeMs); + DataProperty dataProperty2 = new DataProperty(TStorageMedium.SSD, newCooldownTimeMs, "remote_s3", remoteCooldownTimeMs, ""); for (Partition partition : partitionList) { Assert.assertEquals(dataProperty2, tblRemote.getPartitionInfo().getDataProperty(partition.getId())); } @@ -516,7 +516,7 @@ public void testAlterRemoteStorageTableDataProperties() throws Exception { // alter storage_medium stmt = "alter table test.tbl_remote modify partition (p2, p3, p4) set ('storage_medium' = 'HDD')"; alterTable(stmt, false); - DataProperty dataProperty1 = new DataProperty(TStorageMedium.HDD, DataProperty.MAX_COOLDOWN_TIME_MS, "remote_s3", remoteCooldownTimeMs); + DataProperty dataProperty1 = new DataProperty(TStorageMedium.HDD, DataProperty.MAX_COOLDOWN_TIME_MS, "remote_s3", remoteCooldownTimeMs, ""); for (Partition partition : partitionList) { Assert.assertEquals(dataProperty1, tblRemote.getPartitionInfo().getDataProperty(partition.getId())); } @@ -532,7 +532,7 @@ public void testAlterRemoteStorageTableDataProperties() throws Exception { alterTable(stmt, false); DateLiteral newRemoteDate = new DateLiteral("2122-12-01 20:23:00", Type.DATETIME); long newRemoteCooldownTimeMs = newRemoteDate.unixTimestamp(TimeUtils.getTimeZone()); - DataProperty dataProperty4 = new DataProperty(TStorageMedium.HDD, DataProperty.MAX_COOLDOWN_TIME_MS, "remote_s3", newRemoteCooldownTimeMs); + DataProperty dataProperty4 = new DataProperty(TStorageMedium.HDD, DataProperty.MAX_COOLDOWN_TIME_MS, "remote_s3", newRemoteCooldownTimeMs, ""); for (Partition partition : partitionList) { Assert.assertEquals(dataProperty4, tblRemote.getPartitionInfo().getDataProperty(partition.getId())); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java index 257ea1b6951c0f..5ef3d35b6c064d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java @@ -222,6 +222,7 @@ public void testSchemaChange1() throws Exception { for (Replica shadowReplica : shadowTablet.getReplicas()) { shadowReplica.updateVersionInfo(testPartition.getVisibleVersion(), shadowReplica.getDataSize(), + shadowReplica.getRemoteDataSize(), shadowReplica.getRowCount()); } } @@ -301,6 +302,7 @@ public void testSchemaChangeWhileTabletNotStable() throws Exception { for (Replica shadowReplica : shadowTablet.getReplicas()) { shadowReplica.updateVersionInfo(testPartition.getVisibleVersion(), shadowReplica.getDataSize(), + shadowReplica.getRemoteDataSize(), shadowReplica.getRowCount()); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java index 62a6187064b249..25803983855a56 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java @@ -206,7 +206,8 @@ public void testSchemaChange1() throws Exception { MaterializedIndex shadowIndex = testPartition.getMaterializedIndices(IndexExtState.SHADOW).get(0); for (Tablet shadowTablet : shadowIndex.getTablets()) { for (Replica shadowReplica : shadowTablet.getReplicas()) { - shadowReplica.updateVersionInfo(testPartition.getVisibleVersion(), shadowReplica.getDataSize(), shadowReplica.getRowCount()); + shadowReplica.updateVersionInfo(testPartition.getVisibleVersion(), shadowReplica.getDataSize(), + shadowReplica.getRemoteDataSize(), shadowReplica.getRowCount()); } } @@ -288,7 +289,8 @@ public void testSchemaChangeWhileTabletNotStable() throws Exception { MaterializedIndex shadowIndex = testPartition.getMaterializedIndices(IndexExtState.SHADOW).get(0); for (Tablet shadowTablet : shadowIndex.getTablets()) { for (Replica shadowReplica : shadowTablet.getReplicas()) { - shadowReplica.updateVersionInfo(testPartition.getVisibleVersion(), shadowReplica.getDataSize(), shadowReplica.getRowCount()); + shadowReplica.updateVersionInfo(testPartition.getVisibleVersion(), shadowReplica.getDataSize(), + shadowReplica.getRemoteDataSize(), shadowReplica.getRowCount()); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/bdb/BDBToolTest.java b/fe/fe-core/src/test/java/org/apache/doris/bdb/BDBToolTest.java index b3f8e54187aa26..f9693fcd70e766 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/bdb/BDBToolTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/bdb/BDBToolTest.java @@ -73,7 +73,7 @@ public static void setEnv() { } // write something - ReplicaPersistInfo info = ReplicaPersistInfo.createForAdd(1, 2, 3, 4, 5, 6, 7, 8, 0, 10, 11, 12, 14); + ReplicaPersistInfo info = ReplicaPersistInfo.createForAdd(1, 2, 3, 4, 5, 6, 7, 8, 0, 10, 11, 0, 12, 14); JournalEntity entity = new JournalEntity(); entity.setOpCode(OperationType.OP_ADD_REPLICA); entity.setData(info); diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java index ab6880332a041d..42f42eff752cd4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java @@ -173,11 +173,11 @@ public static Database createSimpleDb(long dbId, long tableId, long partitionId, Catalog.getCurrentInvertedIndex().clear(); // replica - Replica replica1 = new Replica(testReplicaId1, testBackendId1, version, 0, 0L, 0L, + Replica replica1 = new Replica(testReplicaId1, testBackendId1, version, 0, 0L, 0L, 0L, ReplicaState.NORMAL, -1, 0); - Replica replica2 = new Replica(testReplicaId2, testBackendId2, version, 0, 0L, 0L, + Replica replica2 = new Replica(testReplicaId2, testBackendId2, version, 0, 0L, 0L, 0L, ReplicaState.NORMAL, -1, 0); - Replica replica3 = new Replica(testReplicaId3, testBackendId3, version, 0, 0L, 0L, + Replica replica3 = new Replica(testReplicaId3, testBackendId3, version, 0, 0L, 0L, 0L, ReplicaState.NORMAL, -1, 0); // tablet @@ -244,7 +244,7 @@ public static Database createSimpleDb(long dbId, long tableId, long partitionId, public static void createDupTable(Database db) { // replica - Replica replica = new Replica(testReplicaId4, testBackendId1, testStartVersion, 0, 0L, 0L, + Replica replica = new Replica(testReplicaId4, testBackendId1, testStartVersion, 0, 0L, 0L, 0L, ReplicaState.NORMAL, -1, 0); // tablet diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DataPropertyTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DataPropertyTest.java index 1669749cf17f71..aaa9ae3b25c032 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DataPropertyTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DataPropertyTest.java @@ -35,7 +35,7 @@ public void tesCooldownTimeMs() throws Exception { Assert.assertNotEquals(DataProperty.MAX_COOLDOWN_TIME_MS, dataProperty.getCooldownTimeMs()); long storageCooldownTimeMs = System.currentTimeMillis() + 24 * 3600 * 1000L; - dataProperty = new DataProperty(TStorageMedium.SSD, storageCooldownTimeMs, "", DataProperty.MAX_COOLDOWN_TIME_MS); + dataProperty = new DataProperty(TStorageMedium.SSD, storageCooldownTimeMs, "", DataProperty.MAX_COOLDOWN_TIME_MS, ""); Assert.assertEquals(storageCooldownTimeMs, dataProperty.getCooldownTimeMs()); dataProperty = new DataProperty(TStorageMedium.HDD); diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/ReplicaTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/ReplicaTest.java index bb56cf0343bb23..33cb8ef40e3598 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/ReplicaTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/ReplicaTest.java @@ -52,7 +52,7 @@ public void setUp() { version = 2; dataSize = 9999; rowCount = 1024; - replica = new Replica(replicaId, backendId, version, 0, dataSize, rowCount, ReplicaState.NORMAL, 0, version); + replica = new Replica(replicaId, backendId, version, 0, dataSize, 0, rowCount, ReplicaState.NORMAL, 0, version); } @Test @@ -67,7 +67,7 @@ public void getMethodTest() { long newVersion = version + 1; long newDataSize = dataSize + 100; long newRowCount = rowCount + 10; - replica.updateVersionInfo(newVersion, newDataSize, newRowCount); + replica.updateVersionInfo(newVersion, newDataSize, 0, newRowCount); Assert.assertEquals(newVersion, replica.getVersion()); Assert.assertEquals(newDataSize, replica.getDataSize()); Assert.assertEquals(newRowCount, replica.getRowCount()); @@ -89,7 +89,7 @@ public void testSerialization() throws Exception { List list2 = new ArrayList(); for (int count = 0; count < 10; ++count) { Replica olapReplica = new Replica(100L * count, 100L * count, 100L * count, 0, - 100L * count, 100 * count, ReplicaState.NORMAL, 0, 100L * count); + 100L * count, 0, 100 * count, ReplicaState.NORMAL, 0, 100L * count); list1.add(olapReplica); olapReplica.write(dos); } @@ -131,16 +131,16 @@ public void testSerialization() throws Exception { @Test public void testUpdateVersion1() { - Replica originalReplica = new Replica(10000, 20000, 3, 0, 100, 78, ReplicaState.NORMAL, 0, 3); + Replica originalReplica = new Replica(10000, 20000, 3, 0, 100, 0, 78, ReplicaState.NORMAL, 0, 3); // new version is little than original version, it is invalid the version will not update - originalReplica.updateVersionInfo(2, 100, 78); + originalReplica.updateVersionInfo(2, 100, 0, 78); Assert.assertEquals(3, originalReplica.getVersion()); } @Test public void testUpdateVersion2() { - Replica originalReplica = new Replica(10000, 20000, 3, 0, 100, 78, ReplicaState.NORMAL, 0, 0); - originalReplica.updateVersionInfo(3, 100, 78); + Replica originalReplica = new Replica(10000, 20000, 3, 0, 100, 0, 78, ReplicaState.NORMAL, 0, 0); + originalReplica.updateVersionInfo(3, 100, 0, 78); // if new version >= current version and last success version <= new version, then last success version should be updated Assert.assertEquals(3, originalReplica.getLastSuccessVersion()); Assert.assertEquals(3, originalReplica.getVersion()); @@ -149,7 +149,7 @@ public void testUpdateVersion2() { @Test public void testUpdateVersion3() { // version(3) ---> last failed version (8) ---> last success version(10) - Replica originalReplica = new Replica(10000, 20000, 3, 111, 0, 78, ReplicaState.NORMAL, 0, 0); + Replica originalReplica = new Replica(10000, 20000, 3, 111, 0, 0, 78, ReplicaState.NORMAL, 0, 0); originalReplica.updateLastFailedVersion(8); Assert.assertEquals(3, originalReplica.getLastSuccessVersion()); Assert.assertEquals(3, originalReplica.getVersion()); @@ -164,7 +164,7 @@ public void testUpdateVersion3() { Assert.assertEquals(8, originalReplica.getLastFailedVersion()); // update version to 8, the last success version and version should be 10 - originalReplica.updateVersionInfo(8, 100, 78); + originalReplica.updateVersionInfo(8, 100, 0, 78); Assert.assertEquals(10, originalReplica.getLastSuccessVersion()); Assert.assertEquals(10, originalReplica.getVersion()); Assert.assertEquals(-1, originalReplica.getLastFailedVersion()); @@ -190,13 +190,13 @@ public void testUpdateVersion3() { Assert.assertEquals(18, originalReplica.getLastFailedVersion()); // update version to 17 then version and success version is 17 - originalReplica.updateVersionInfo(17, 100, 78); + originalReplica.updateVersionInfo(17, 100, 0, 78); Assert.assertEquals(17, originalReplica.getLastSuccessVersion()); Assert.assertEquals(17, originalReplica.getVersion()); Assert.assertEquals(18, originalReplica.getLastFailedVersion()); // update version to 18, then version and last success version should be 18 and failed version should be -1 - originalReplica.updateVersionInfo(18, 100, 78); + originalReplica.updateVersionInfo(18, 100, 0, 78); Assert.assertEquals(18, originalReplica.getLastSuccessVersion()); Assert.assertEquals(18, originalReplica.getVersion()); Assert.assertEquals(-1, originalReplica.getLastFailedVersion()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/TabletTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/TabletTest.java index 0d439a587c3935..2be38da9f30fda 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/TabletTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/TabletTest.java @@ -69,9 +69,9 @@ public void makeTablet() { tablet = new Tablet(1); TabletMeta tabletMeta = new TabletMeta(10, 20, 30, 40, 1, TStorageMedium.HDD); invertedIndex.addTablet(1, tabletMeta); - replica1 = new Replica(1L, 1L, 100L, 0, 200000L, 3000L, ReplicaState.NORMAL, 0, 0); - replica2 = new Replica(2L, 2L, 100L, 0, 200000L, 3000L, ReplicaState.NORMAL, 0, 0); - replica3 = new Replica(3L, 3L, 100L, 0, 200000L, 3000L, ReplicaState.NORMAL, 0, 0); + replica1 = new Replica(1L, 1L, 100L, 0, 200000L, 0, 3000L, ReplicaState.NORMAL, 0, 0); + replica2 = new Replica(2L, 2L, 100L, 0, 200000L, 0, 3000L, ReplicaState.NORMAL, 0, 0); + replica3 = new Replica(3L, 3L, 100L, 0, 200000L, 0, 3000L, ReplicaState.NORMAL, 0, 0); tablet.addReplica(replica1); tablet.addReplica(replica2); tablet.addReplica(replica3); @@ -135,9 +135,9 @@ public void testSerialization() throws Exception { Assert.assertFalse(rTablet1.equals(this)); Tablet tablet2 = new Tablet(1); - Replica replica1 = new Replica(1L, 1L, 100L, 0, 200000L, 3000L, ReplicaState.NORMAL, 0, 0); - Replica replica2 = new Replica(2L, 2L, 100L, 0, 200000L, 3000L, ReplicaState.NORMAL, 0, 0); - Replica replica3 = new Replica(3L, 3L, 100L, 0, 200000L, 3000L, ReplicaState.NORMAL, 0, 0); + Replica replica1 = new Replica(1L, 1L, 100L, 0, 200000L, 0, 3000L, ReplicaState.NORMAL, 0, 0); + Replica replica2 = new Replica(2L, 2L, 100L, 0, 200000L, 0, 3000L, ReplicaState.NORMAL, 0, 0); + Replica replica3 = new Replica(3L, 3L, 100L, 0, 200000L, 0, 3000L, ReplicaState.NORMAL, 0, 0); tablet2.addReplica(replica1); tablet2.addReplica(replica2); Assert.assertFalse(tablet2.equals(tablet)); @@ -147,7 +147,7 @@ public void testSerialization() throws Exception { Tablet tablet3 = new Tablet(1); tablet3.addReplica(replica1); tablet3.addReplica(replica2); - tablet3.addReplica(new Replica(4L, 4L, 100L, 0, 200000L, 3000L, ReplicaState.NORMAL, 0, 0)); + tablet3.addReplica(new Replica(4L, 4L, 100L, 0, 200000L, 0, 3000L, ReplicaState.NORMAL, 0, 0)); Assert.assertFalse(tablet3.equals(tablet)); dis.close(); @@ -170,7 +170,7 @@ private final void testTabletColocateHealthStatus0(Tablet.TabletStatus exceptedT versionAndSuccessVersion = 99L; lastFailVersion = 100L; } - tablet.addReplica(new Replica(replicaId++, pair.first, versionAndSuccessVersion, 0, 200000L, 3000L, + tablet.addReplica(new Replica(replicaId++, pair.first, versionAndSuccessVersion, 0, 200000L, 0, 3000L, ReplicaState.NORMAL, lastFailVersion, versionAndSuccessVersion)); } Assert.assertEquals(tablet.getColocateHealthStatus(100L, new ReplicaAllocation((short) 3), diff --git a/fe/fe-core/src/test/java/org/apache/doris/http/DorisHttpTestCase.java b/fe/fe-core/src/test/java/org/apache/doris/http/DorisHttpTestCase.java index eab29c860d0366..7d08534f89e607 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/http/DorisHttpTestCase.java +++ b/fe/fe-core/src/test/java/org/apache/doris/http/DorisHttpTestCase.java @@ -140,11 +140,11 @@ public static OlapTable newTable(String name) { columns.add(k1); columns.add(k2); - Replica replica1 = new Replica(testReplicaId1, testBackendId1, testStartVersion, testSchemaHash, 1024000L, 2000L, + Replica replica1 = new Replica(testReplicaId1, testBackendId1, testStartVersion, testSchemaHash, 1024000L, 0, 2000L, Replica.ReplicaState.NORMAL, -1, 0); - Replica replica2 = new Replica(testReplicaId2, testBackendId2, testStartVersion, testSchemaHash, 1024000L, 2000L, + Replica replica2 = new Replica(testReplicaId2, testBackendId2, testStartVersion, testSchemaHash, 1024000L, 0, 2000L, Replica.ReplicaState.NORMAL, -1, 0); - Replica replica3 = new Replica(testReplicaId3, testBackendId3, testStartVersion, testSchemaHash, 1024000L, 2000L, + Replica replica3 = new Replica(testReplicaId3, testBackendId3, testStartVersion, testSchemaHash, 1024000L, 0, 2000L, Replica.ReplicaState.NORMAL, -1, 0); // tablet diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/LoadCheckerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/LoadCheckerTest.java index e2654b9fc400ea..b233c5c9479c4f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/LoadCheckerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/LoadCheckerTest.java @@ -317,7 +317,7 @@ public void testRunLoadingJobs() throws Exception { for (MaterializedIndex olapIndex : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : olapIndex.getTablets()) { for (Replica replica : tablet.getReplicas()) { - replica.updateVersionInfo(newVersion, 0L, 0L); + replica.updateVersionInfo(newVersion, 0L, 0L, 0L); } } } @@ -353,7 +353,7 @@ public void testRunQuorumFinishedJobs() throws Exception { for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) { for (Tablet tablet : index.getTablets()) { for (Replica replica : tablet.getReplicas()) { - replica.updateVersionInfo(newVersion, 0L, 0L); + replica.updateVersionInfo(newVersion, 0L, 0L, 0L); } TabletLoadInfo tabletLoadInfo = new TabletLoadInfo("/label/path", 1L); tabletLoadInfos.put(tablet.getId(), tabletLoadInfo); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/LoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/LoadJobTest.java index 32a3c156575867..4d4b8363e6c6e2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/LoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/LoadJobTest.java @@ -125,9 +125,9 @@ public LoadJob getLoadJob() { loadJob3.addFullTablet(2); loadJob3.addFullTablet(3); - loadJob3.addReplicaPersistInfos(ReplicaPersistInfo.createForLoad(1, 1, 1, 1, 1, 1, 0, 1, 1)); - loadJob3.addReplicaPersistInfos(ReplicaPersistInfo.createForLoad(2, 2, 2, 2, 2, 2, 0, 2, 2)); - loadJob3.addReplicaPersistInfos(ReplicaPersistInfo.createForLoad(3, 3, 3, 3, 3, 3, 0, 3, 3)); + loadJob3.addReplicaPersistInfos(ReplicaPersistInfo.createForLoad(1, 1, 1, 1, 1, 1, 0, 1, 0, 1)); + loadJob3.addReplicaPersistInfos(ReplicaPersistInfo.createForLoad(2, 2, 2, 2, 2, 2, 0, 2, 0, 2)); + loadJob3.addReplicaPersistInfos(ReplicaPersistInfo.createForLoad(3, 3, 3, 3, 3, 3, 0, 3, 0, 3)); return loadJob3; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/BatchModifyPartitionsInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/BatchModifyPartitionsInfoTest.java index c9e5d2a99303f0..15fb08a6a24e81 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/persist/BatchModifyPartitionsInfoTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/BatchModifyPartitionsInfoTest.java @@ -60,7 +60,7 @@ public void testSerializeBatchModifyPartitionsInfo() throws IOException, Analysi List partitionIds = Lists.newArrayList(PARTITION_ID_1, PARTITION_ID_2, PARTITION_ID_3); for (long partitionId : partitionIds) { modifyInfos.add(new ModifyPartitionInfo(DB_ID, TB_ID, partitionId, - DataProperty.DEFAULT_DATA_PROPERTY, ReplicaAllocation.DEFAULT_ALLOCATION, true)); + DataProperty.DEFAULT_DATA_PROPERTY, ReplicaAllocation.DEFAULT_ALLOCATION, true, "")); } BatchModifyPartitionsInfo batchModifyPartitionsInfo = new BatchModifyPartitionsInfo(modifyInfos); diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/ReplicaPersistInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/ReplicaPersistInfoTest.java index 78509823fe880d..ce621d7e1f2a12 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/persist/ReplicaPersistInfoTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/ReplicaPersistInfoTest.java @@ -41,7 +41,7 @@ public void testSerialization() throws Exception { file.createNewFile(); DataOutputStream dos = new DataOutputStream(new FileOutputStream(file)); - ReplicaPersistInfo info2 = ReplicaPersistInfo.createForLoad(1, 2, 3, 4, 5, 7, 0, 8, 9); + ReplicaPersistInfo info2 = ReplicaPersistInfo.createForLoad(1, 2, 3, 4, 5, 7, 0, 8, 0, 9); info2.write(dos); dos.flush(); @@ -59,7 +59,7 @@ public void testSerialization() throws Exception { @Test public void testGet() throws Exception { - ReplicaPersistInfo info = ReplicaPersistInfo.createForLoad(0, 1, 2, 3, 4, 5, 7, 0, 8); + ReplicaPersistInfo info = ReplicaPersistInfo.createForLoad(0, 1, 2, 3, 4, 5, 7, 0, 0, 8); Assert.assertEquals(0, info.getTableId()); Assert.assertEquals(1, info.getPartitionId()); Assert.assertEquals(2, info.getIndexId()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java index a2a6c9e107ee71..e8477bfe5194b5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java @@ -1085,7 +1085,7 @@ public void testBucketShuffleJoin() throws Exception { mIndex.setRowCount(10000); for (Tablet tablet : mIndex.getTablets()) { for (Replica replica : tablet.getReplicas()) { - replica.updateVersionInfo(2, 200000, 10000); + replica.updateVersionInfo(2, 200000, 0, 10000); } } } @@ -1099,7 +1099,7 @@ public void testBucketShuffleJoin() throws Exception { mIndex.setRowCount(10000); for (Tablet tablet : mIndex.getTablets()) { for (Replica replica : tablet.getReplicas()) { - replica.updateVersionInfo(2, 200000, 10000); + replica.updateVersionInfo(2, 200000, 0,10000); } } } @@ -1176,7 +1176,7 @@ public void testJoinWithMysqlTable() throws Exception { mIndex.setRowCount(10000); for (Tablet tablet : mIndex.getTablets()) { for (Replica replica : tablet.getReplicas()) { - replica.updateVersionInfo(2, 200000, 10000); + replica.updateVersionInfo(2, 200000, 0,10000); } } } @@ -1203,7 +1203,7 @@ public void testJoinWithMysqlTable() throws Exception { mIndex.setRowCount(0); for (Tablet tablet : mIndex.getTablets()) { for (Replica replica : tablet.getReplicas()) { - replica.updateVersionInfo(2, 0, 0); + replica.updateVersionInfo(2, 0, 0,0); } } } @@ -1223,7 +1223,7 @@ public void testJoinWithOdbcTable() throws Exception { mIndex.setRowCount(10000); for (Tablet tablet : mIndex.getTablets()) { for (Replica replica : tablet.getReplicas()) { - replica.updateVersionInfo(2, 200000, 10000); + replica.updateVersionInfo(2, 200000, 0,10000); } } } @@ -1250,7 +1250,7 @@ public void testJoinWithOdbcTable() throws Exception { mIndex.setRowCount(0); for (Tablet tablet : mIndex.getTablets()) { for (Replica replica : tablet.getReplicas()) { - replica.updateVersionInfo(2, 0, 0); + replica.updateVersionInfo(2, 0, 0,0); } } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java index 0cb1a8d4f84dae..930a24e9b95f89 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java @@ -112,10 +112,10 @@ public void setUp() throws AnalysisException { version, KeysType.AGG_KEYS, storageType, TStorageMedium.SSD, columns, null, 0, latch, null, - false, TTabletType.TABLET_TYPE_DISK, TCompressionType.LZ4F); + false, TTabletType.TABLET_TYPE_DISK, TCompressionType.LZ4F, ""); // drop - dropTask = new DropReplicaTask(backendId1, tabletId1, replicaId1, schemaHash1); + dropTask = new DropReplicaTask(backendId1, tabletId1, replicaId1, schemaHash1, false); // push pushTask = @@ -241,7 +241,7 @@ public void failedAgentTaskTest() { Assert.assertEquals(1, AgentTaskQueue.getTaskNum(backendId1, TTaskType.DROP, true)); dropTask.failed(); - DropReplicaTask dropTask2 = new DropReplicaTask(backendId2, tabletId1, replicaId1, schemaHash1); + DropReplicaTask dropTask2 = new DropReplicaTask(backendId2, tabletId1, replicaId1, schemaHash1, false); AgentTaskQueue.addTask(dropTask2); dropTask2.failed(); Assert.assertEquals(1, AgentTaskQueue.getTaskNum(backendId1, TTaskType.DROP, true)); diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto index 8c1bd00aa38d83..0d484a292d349c 100644 --- a/gensrc/proto/olap_file.proto +++ b/gensrc/proto/olap_file.proto @@ -93,6 +93,12 @@ message RowsetMetaPB { optional int64 num_segments = 22; // rowset id definition, it will replace required rowset id optional string rowset_id_v2 = 23; + // resource id + optional string resource_id = 24; + // earliest write time + optional int64 oldest_write_timestamp = 25 [default = -1]; + // latest write time + optional int64 newest_write_timestamp = 26 [default = -1]; // spare field id for future use optional AlphaRowsetExtraMetaPB alpha_rowset_extra_meta_pb = 50; // to indicate whether the data between the segments overlap @@ -263,6 +269,7 @@ message TabletMetaPB { optional StorageMediumPB storage_medium = 19 [default = HDD]; optional string remote_storage_name = 20; optional int64 replica_id = 21 [default = 0]; + optional string storage_policy = 22; } message OLAPIndexHeaderMessage { diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index 740db1fd08c002..dc64d9c52f5fbd 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -76,6 +76,20 @@ struct TS3StorageParam { 6: optional i32 s3_request_timeout_ms = 3000 7: optional i32 s3_conn_timeout_ms = 1000 8: optional string root_path + 9: optional string bucket +} + +struct TGetStoragePolicy { + 1: required string policy_name + 2: required i64 cooldown_datetime + 3: required i64 cooldown_ttl + 4: required TS3StorageParam s3_storage_param + 5: required string md5_checksum +} + +struct TGetStoragePolicyResult { + 1: required Status.TStatus status + 2: required list result_entrys } struct TStorageParam { @@ -119,12 +133,14 @@ struct TCreateTabletReq { 15: optional TStorageParam storage_param 16: optional TCompressionType compression_type = TCompressionType.LZ4F 17: optional Types.TReplicaId replica_id = 0 + 18: optional string storage_policy } struct TDropTabletReq { 1: required Types.TTabletId tablet_id 2: optional Types.TSchemaHash schema_hash 3: optional Types.TReplicaId replica_id = 0 + 4: optional bool is_drop_table_or_partition = false } struct TAlterTabletReq { @@ -333,6 +349,7 @@ struct TTabletMetaInfo { 3: optional Types.TPartitionId partition_id 4: optional TTabletMetaType meta_type 5: optional bool is_in_memory + 6: optional string storage_policy; } struct TUpdateTabletMetaInfoReq { @@ -376,6 +393,7 @@ struct TAgentTaskRequest { 26: optional TUpdateTabletMetaInfoReq update_tablet_meta_info_req 27: optional TCompactionReq compaction_req 28: optional TStorageMigrationReqV2 storage_migration_req_v2 + 29: optional TGetStoragePolicy update_policy } struct TAgentResult { diff --git a/gensrc/thrift/BackendService.thrift b/gensrc/thrift/BackendService.thrift index 8e534f46775df5..8219cbae2659ed 100644 --- a/gensrc/thrift/BackendService.thrift +++ b/gensrc/thrift/BackendService.thrift @@ -31,9 +31,11 @@ struct TExportTaskRequest { struct TTabletStat { 1: required i64 tablet_id + // local data size 2: optional i64 data_size 3: optional i64 row_num 4: optional i64 version_count + 5: optional i64 remote_data_size } struct TTabletStatResult { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index f195a11722abd6..22aae2236f7dba 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -780,4 +780,6 @@ service FrontendService { Status.TStatus snapshotLoaderReport(1: TSnapshotLoaderReportRequest request) TFrontendPingFrontendResult ping(1: TFrontendPingFrontendRequest request) + + AgentService.TGetStoragePolicyResult refreshStoragePolicy() } diff --git a/gensrc/thrift/MasterService.thrift b/gensrc/thrift/MasterService.thrift index e49d34aacf874e..7c9b9231989774 100644 --- a/gensrc/thrift/MasterService.thrift +++ b/gensrc/thrift/MasterService.thrift @@ -29,6 +29,7 @@ struct TTabletInfo { 3: required Types.TVersion version 4: required Types.TVersionHash version_hash 5: required Types.TCount row_count + // data size on local disk 6: required Types.TSize data_size 7: optional Types.TStorageMedium storage_medium 8: optional list transaction_ids @@ -39,6 +40,8 @@ struct TTabletInfo { 13: optional Types.TPartitionId partition_id 14: optional bool is_in_memory 15: optional Types.TReplicaId replica_id + // data size on remote storage + 16: optional Types.TSize remote_data_size } struct TFinishTaskRequest { @@ -67,11 +70,13 @@ struct TTablet { struct TDisk { 1: required string root_path 2: required Types.TSize disk_total_capacity + // local used capacity 3: required Types.TSize data_used_capacity 4: required bool used 5: optional Types.TSize disk_available_capacity 6: optional i64 path_hash 7: optional Types.TStorageMedium storage_medium + 8: optional Types.TSize remote_used_capacity } struct TPluginInfo { diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift index 1d6bc8c683c95a..5606e49acc2e6a 100644 --- a/gensrc/thrift/Types.thrift +++ b/gensrc/thrift/Types.thrift @@ -192,7 +192,8 @@ enum TTaskType { INSTALL_PLUGIN, UNINSTALL_PLUGIN, COMPACTION, - STORAGE_MEDIUM_MIGRATE_V2 + STORAGE_MEDIUM_MIGRATE_V2, + NOTIFY_UPDATE_STORAGE_POLICY } enum TStmtType {