diff --git a/be/src/agent/agent_server.cpp b/be/src/agent/agent_server.cpp index 80dd40ff56..2302be05e4 100644 --- a/be/src/agent/agent_server.cpp +++ b/be/src/agent/agent_server.cpp @@ -91,8 +91,7 @@ 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_THREAD(REFRESH_STORAGE_POLICY, _storage_refresh_policy_workers); - CREATE_AND_START_POOL(UPDATE_STORAGE_POLICY, _storage_update_policy_workers); + CREATE_AND_START_POOL(PUSH_STORAGE_POLICY, _push_storage_policy_workers); #undef CREATE_AND_START_POOL #undef CREATE_AND_START_THREAD @@ -155,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::NOTIFY_UPDATE_STORAGE_POLICY, _storage_update_policy_workers, - update_policy); + HANDLE_TYPE(TTaskType::PUSH_STORAGE_POLICY, _push_storage_policy_workers, + push_storage_policy_req); case TTaskType::REALTIME_PUSH: case TTaskType::PUSH: diff --git a/be/src/agent/agent_server.h b/be/src/agent/agent_server.h index 3914f3a236..34eab9a77b 100644 --- a/be/src/agent/agent_server.h +++ b/be/src/agent/agent_server.h @@ -81,8 +81,7 @@ private: std::unique_ptr _submit_table_compaction_workers; - std::unique_ptr _storage_refresh_policy_workers; - std::unique_ptr _storage_update_policy_workers; + std::unique_ptr _push_storage_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 ae5248c943..07ffed16a2 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -17,6 +17,7 @@ #include "agent/task_worker_pool.h" +#include #include #include @@ -24,18 +25,21 @@ #include #include #include +#include #include #include +#include "common/logging.h" #include "common/status.h" #include "env/env.h" #include "gen_cpp/Types_types.h" #include "gutil/strings/substitute.h" +#include "io/fs/s3_file_system.h" #include "olap/data_dir.h" #include "olap/olap_common.h" #include "olap/snapshot_manager.h" #include "olap/storage_engine.h" -#include "olap/storage_policy_mgr.h" +#include "olap/storage_policy.h" #include "olap/tablet.h" #include "olap/task/engine_alter_tablet_task.h" #include "olap/task/engine_batch_load_task.h" @@ -195,14 +199,9 @@ void TaskWorkerPool::start() { cb = std::bind(&TaskWorkerPool::_submit_table_compaction_worker_thread_callback, this); break; - case TaskWorkerType::REFRESH_STORAGE_POLICY: - cb = std::bind( - &TaskWorkerPool::_storage_refresh_storage_policy_worker_thread_callback, this); - break; - case TaskWorkerType::UPDATE_STORAGE_POLICY: + case TaskWorkerType::PUSH_STORAGE_POLICY: _worker_count = 1; - cb = std::bind(&TaskWorkerPool::_storage_update_storage_policy_worker_thread_callback, - this); + cb = std::bind(&TaskWorkerPool::_push_storage_policy_worker_thread_callback, this); break; case TaskWorkerType::PUSH_COOLDOWN_CONF: _worker_count = 1; @@ -269,6 +268,10 @@ void TaskWorkerPool::notify_thread() { } bool TaskWorkerPool::_register_task_info(const TTaskType::type task_type, int64_t signature) { + if (task_type == TTaskType::type::PUSH_STORAGE_POLICY) { + // no need to report task of these types + return true; + } std::lock_guard task_signatures_lock(_s_task_signatures_lock); std::set& signature_set = _s_task_signatures[task_type]; return signature_set.insert(signature).second; @@ -283,9 +286,7 @@ void TaskWorkerPool::_remove_task_info(const TTaskType::type task_type, int64_t queue_size = signature_set.size(); } - std::string type_str; - EnumToString(TTaskType, task_type, type_str); - VLOG_NOTICE << "remove task info. type=" << type_str << ", signature=" << signature + VLOG_NOTICE << "remove task info. type=" << task_type << ", signature=" << signature << ", queue_size=" << queue_size; TRACE("remove task info"); } @@ -854,25 +855,27 @@ void TaskWorkerPool::_update_tablet_meta_worker_thread_callback() { tablet->set_partition_id(tablet_meta_info.partition_id); } else { switch (tablet_meta_info.meta_type) { - case TTabletMetaType::PARTITIONID: + case TTabletMetaType::PARTITIONID: // FIXME(plat1ko): deprecate? tablet->set_partition_id(tablet_meta_info.partition_id); break; case TTabletMetaType::INMEMORY: - if (tablet_meta_info.storage_policy.empty()) { + if (tablet_meta_info.__isset.storage_policy_id) { + LOG(INFO) << "set tablet storage_policy_id=" + << tablet_meta_info.storage_policy_id; + tablet->tablet_meta()->set_storage_policy_id( + tablet_meta_info.storage_policy_id); + } + if (tablet_meta_info.__isset.is_in_memory) { tablet->tablet_meta()->mutable_tablet_schema()->set_is_in_memory( tablet_meta_info.is_in_memory); // The field is_in_memory should not be in the tablet_schema. // it should be in the tablet_meta. - for (auto rowset_meta : tablet->tablet_meta()->all_mutable_rs_metas()) { + for (auto& rowset_meta : tablet->tablet_meta()->all_mutable_rs_metas()) { rowset_meta->tablet_schema()->set_is_in_memory( tablet_meta_info.is_in_memory); } tablet->get_max_version_schema(wrlock)->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_storage_policy(tablet_meta_info.storage_policy); } break; } @@ -1232,6 +1235,23 @@ void TaskWorkerPool::_report_tablet_worker_thread_callback() { DorisMetrics::instance()->tablet_base_max_compaction_score->value()); request.__set_tablet_max_compaction_score(max_compaction_score); request.__set_report_version(report_version); + + // report storage policy and resource + auto& storage_policy_list = request.storage_policy; + for (auto [id, version] : get_storage_policy_ids()) { + auto& storage_policy = storage_policy_list.emplace_back(); + storage_policy.__set_id(id); + storage_policy.__set_version(version); + } + request.__isset.storage_policy = true; + auto& resource_list = request.resource; + for (auto [id, version] : get_storage_resource_ids()) { + auto& resource = resource_list.emplace_back(); + resource.__set_id(id); + resource.__set_version(version); + } + request.__isset.resource = true; + _handle_report(request, ReportType::TABLET); } StorageEngine::instance()->deregister_report_listener(this); @@ -1603,63 +1623,9 @@ void TaskWorkerPool::_submit_table_compaction_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() && result.status.status_code == TStatusCode::OK) { - // update storage policy mgr. - StoragePolicyMgr* spm = ExecEnv::GetInstance()->storage_policy_mgr(); - for (const auto& iter : result.result_entrys) { - auto policy_ptr = std::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_EVERY_N(INFO, 12) << "refresh storage policy task. policy=" << *policy_ptr; - spm->periodic_put(iter.policy_name, policy_ptr); - } - } - } -} - -void TaskWorkerPool::_storage_update_storage_policy_worker_thread_callback() { +void TaskWorkerPool::_push_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); _worker_thread_condition_variable.wait( @@ -1669,31 +1635,68 @@ void TaskWorkerPool::_storage_update_storage_policy_worker_thread_callback() { } agent_task_req = _tasks.front(); - get_storage_policy_req = agent_task_req.update_policy; _tasks.pop_front(); } - - StoragePolicyMgr* spm = ExecEnv::GetInstance()->storage_policy_mgr(); - auto policy_ptr = std::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. policy=" << *policy_ptr; - - spm->update(get_storage_policy_req.policy_name, policy_ptr); - _remove_task_info(agent_task_req.task_type, agent_task_req.signature); + TPushStoragePolicyReq& push_storage_policy_req = agent_task_req.push_storage_policy_req; + // refresh resource + for (auto& resource : push_storage_policy_req.resource) { + auto existed_resource = get_storage_resource(resource.id); + if (existed_resource.version >= resource.version) { + continue; + } + if (resource.__isset.s3_storage_param) { + S3Conf s3_conf; + s3_conf.ak = std::move(resource.s3_storage_param.ak); + s3_conf.sk = std::move(resource.s3_storage_param.sk); + s3_conf.endpoint = std::move(resource.s3_storage_param.endpoint); + s3_conf.region = std::move(resource.s3_storage_param.region); + s3_conf.prefix = std::move(resource.s3_storage_param.root_path); + s3_conf.bucket = std::move(resource.s3_storage_param.bucket); + s3_conf.connect_timeout_ms = resource.s3_storage_param.conn_timeout_ms; + s3_conf.max_connections = resource.s3_storage_param.max_conn; + s3_conf.request_timeout_ms = resource.s3_storage_param.request_timeout_ms; + std::shared_ptr s3_fs; + if (existed_resource.fs == nullptr) { + s3_fs = io::S3FileSystem::create(s3_conf, std::to_string(resource.id)); + } else { + s3_fs = std::static_pointer_cast(existed_resource.fs); + s3_fs->set_conf(s3_conf); + } + auto st = s3_fs->connect(); + if (!st.ok()) { + LOG(WARNING) << "update s3 resource failed: " << st; + } else { + LOG_INFO("successfully update s3 resource") + .tag("resource_id", resource.id) + .tag("resource_name", resource.name) + .tag("s3_conf", s3_conf.to_string()); + put_storage_resource(resource.id, {std::move(s3_fs), resource.version}); + } + } else { + LOG(WARNING) << "unknown resource=" << resource; + } + } + // drop storage policy + for (auto policy_id : push_storage_policy_req.dropped_storage_policy) { + delete_storage_policy(policy_id); + } + // refresh storage policy + for (auto& storage_policy : push_storage_policy_req.storage_policy) { + auto existed_storage_policy = get_storage_policy(storage_policy.id); + if (existed_storage_policy == nullptr || + existed_storage_policy->version < storage_policy.version) { + auto storage_policy1 = std::make_shared(); + storage_policy1->name = std::move(storage_policy.name); + storage_policy1->version = storage_policy.version; + storage_policy1->cooldown_datetime = storage_policy.cooldown_datetime; + storage_policy1->cooldown_ttl = storage_policy.cooldown_ttl; + storage_policy1->resource_id = storage_policy.resource_id; + LOG_INFO("successfully update storage policy") + .tag("storage_policy_id", storage_policy.id) + .tag("storage_policy", storage_policy1->to_string()); + put_storage_policy(storage_policy.id, std::move(storage_policy1)); + } + } } } diff --git a/be/src/agent/task_worker_pool.h b/be/src/agent/task_worker_pool.h index f94d8d2ad8..13c8257b64 100644 --- a/be/src/agent/task_worker_pool.h +++ b/be/src/agent/task_worker_pool.h @@ -66,9 +66,8 @@ public: RECOVER_TABLET, UPDATE_TABLET_META_INFO, SUBMIT_TABLE_COMPACTION, - REFRESH_STORAGE_POLICY, - UPDATE_STORAGE_POLICY, - PUSH_COOLDOWN_CONF + PUSH_COOLDOWN_CONF, + PUSH_STORAGE_POLICY, }; enum ReportType { TASK, DISK, TABLET }; @@ -124,12 +123,10 @@ public: return "UPDATE_TABLET_META_INFO"; case SUBMIT_TABLE_COMPACTION: return "SUBMIT_TABLE_COMPACTION"; - case REFRESH_STORAGE_POLICY: - return "REFRESH_STORAGE_POLICY"; - case UPDATE_STORAGE_POLICY: - return "UPDATE_STORAGE_POLICY"; case PUSH_COOLDOWN_CONF: return "PUSH_COOLDOWN_CONF"; + case PUSH_STORAGE_POLICY: + return "PUSH_STORAGE_POLICY"; default: return "Unknown"; } @@ -193,9 +190,8 @@ private: void _move_dir_thread_callback(); void _update_tablet_meta_worker_thread_callback(); void _submit_table_compaction_worker_thread_callback(); - void _storage_refresh_storage_policy_worker_thread_callback(); - void _storage_update_storage_policy_worker_thread_callback(); void _push_cooldown_conf_worker_thread_callback(); + void _push_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); diff --git a/be/src/agent/utils.cpp b/be/src/agent/utils.cpp index 14698f3528..7c0a552c5f 100644 --- a/be/src/agent/utils.cpp +++ b/be/src/agent/utils.cpp @@ -131,46 +131,6 @@ 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 refresh storge policy. " - << "host=" << _master_info.network_address.hostname - << ", port=" << _master_info.network_address.port - << ", code=" << client_status.code() << ", reason=" << e.what(); - 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 654d8093cd..2785e562e4 100644 --- a/be/src/agent/utils.h +++ b/be/src/agent/utils.h @@ -49,15 +49,6 @@ public: // * 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/io/CMakeLists.txt b/be/src/io/CMakeLists.txt index 684600fa37..249d4e3c27 100644 --- a/be/src/io/CMakeLists.txt +++ b/be/src/io/CMakeLists.txt @@ -34,7 +34,6 @@ set(IO_FILES s3_reader.cpp s3_writer.cpp fs/file_reader_options.cpp - fs/file_system_map.cpp fs/local_file_reader.cpp fs/local_file_system.cpp fs/local_file_writer.cpp diff --git a/be/src/io/file_factory.cpp b/be/src/io/file_factory.cpp index 62696552a5..f90a2b5489 100644 --- a/be/src/io/file_factory.cpp +++ b/be/src/io/file_factory.cpp @@ -27,6 +27,7 @@ #include "io/fs/file_system.h" #include "io/fs/hdfs_file_system.h" #include "io/fs/local_file_system.h" +#include "io/fs/remote_file_system.h" #include "io/fs/s3_file_system.h" #include "io/hdfs_file_reader.h" #include "io/hdfs_writer.h" @@ -153,12 +154,11 @@ Status FileFactory::create_file_reader(RuntimeProfile* /*profile*/, std::shared_ptr* file_system, io::FileReaderSPtr* file_reader, IOContext* io_ctx) { TFileType::type type = system_properties.system_type; - std::string cache_policy = "no_cache"; + auto cache_policy = io::FileCachePolicy::NO_CACHE; if (config::enable_file_cache) { - cache_policy = "file_block_cache"; + cache_policy = io::FileCachePolicy::FILE_BLOCK_CACHE; } - io::FileReaderOptions reader_options(io::cache_type_from_string(cache_policy), - io::FileBlockCachePathPolicy()); + io::FileReaderOptions reader_options(cache_policy, io::FileBlockCachePathPolicy()); switch (type) { case TFileType::FILE_LOCAL: { RETURN_IF_ERROR(io::global_local_filesystem()->open_file( @@ -235,9 +235,10 @@ Status FileFactory::create_s3_reader(const std::map& p } S3Conf s3_conf; RETURN_IF_ERROR(ClientFactory::convert_properties_to_s3_conf(prop, s3_uri, &s3_conf)); - *s3_file_system = io::S3FileSystem::create(s3_conf, ""); - RETURN_IF_ERROR((std::static_pointer_cast(*s3_file_system))->connect()); - RETURN_IF_ERROR((*s3_file_system)->open_file(s3_uri.get_key(), reader_options, reader, io_ctx)); + std::shared_ptr tmp_fs = io::S3FileSystem::create(std::move(s3_conf), ""); + RETURN_IF_ERROR(tmp_fs->connect()); + RETURN_IF_ERROR(tmp_fs->open_file(s3_uri.get_key(), reader_options, reader, io_ctx)); + *s3_file_system = std::move(tmp_fs); return Status::OK(); } diff --git a/be/src/io/fs/file_system.h b/be/src/io/fs/file_system.h index f1fd331038..110d6e0f9f 100644 --- a/be/src/io/fs/file_system.h +++ b/be/src/io/fs/file_system.h @@ -32,9 +32,6 @@ 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, @@ -74,15 +71,15 @@ public: 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 std::string& id() const { return _id; } const FileSystemType type() const { return _type; } protected: - FileSystem(Path&& root_path, ResourceId&& resource_id, FileSystemType type) - : _root_path(std::move(root_path)), _resource_id(std::move(resource_id)), _type(type) {} + FileSystem(Path&& root_path, std::string&& id, FileSystemType type) + : _root_path(std::move(root_path)), _id(std::move(id)), _type(type) {} Path _root_path; - ResourceId _resource_id; + std::string _id; FileSystemType _type; }; diff --git a/be/src/io/fs/file_system_map.cpp b/be/src/io/fs/file_system_map.cpp deleted file mode 100644 index 18d7f454e8..0000000000 --- a/be/src/io/fs/file_system_map.cpp +++ /dev/null @@ -1,45 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "io/fs/file_system_map.h" - -#include - -namespace doris { -namespace io { - -FileSystemMap* FileSystemMap::instance() { - static FileSystemMap map; - return ↦ -} - -void FileSystemMap::insert(ResourceId id, FileSystemSPtr fs) { - std::unique_lock wlock(_mu); - _map.try_emplace(std::move(id), std::move(fs)); -} - -FileSystemSPtr 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 deleted file mode 100644 index c208db59aa..0000000000 --- a/be/src/io/fs/file_system_map.h +++ /dev/null @@ -1,48 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include -#include - -#include "io/fs/file_system.h" - -namespace doris { -namespace io { - -class FileSystemMap { -public: - static FileSystemMap* instance(); - ~FileSystemMap() = default; - - void insert(ResourceId id, FileSystemSPtr fs); - - // If `id` is not in `_map`, return nullptr. - FileSystemSPtr 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/local_file_system.cpp b/be/src/io/fs/local_file_system.cpp index 88650d95f9..3827de2008 100644 --- a/be/src/io/fs/local_file_system.cpp +++ b/be/src/io/fs/local_file_system.cpp @@ -24,13 +24,12 @@ namespace doris { namespace io { -std::shared_ptr LocalFileSystem::create(Path path, ResourceId resource_id) { - return std::shared_ptr( - new LocalFileSystem(std::move(path), std::move(resource_id))); +std::shared_ptr LocalFileSystem::create(Path path, std::string id) { + return std::shared_ptr(new LocalFileSystem(std::move(path), std::move(id))); } -LocalFileSystem::LocalFileSystem(Path root_path, ResourceId resource_id) - : FileSystem(std::move(root_path), std::move(resource_id), FileSystemType::LOCAL) {} +LocalFileSystem::LocalFileSystem(Path&& root_path, std::string&& id) + : FileSystem(std::move(root_path), std::move(id), FileSystemType::LOCAL) {} LocalFileSystem::~LocalFileSystem() = default; diff --git a/be/src/io/fs/local_file_system.h b/be/src/io/fs/local_file_system.h index bc9faa1d98..62bb4b2c12 100644 --- a/be/src/io/fs/local_file_system.h +++ b/be/src/io/fs/local_file_system.h @@ -18,14 +18,13 @@ #pragma once #include "io/fs/file_system.h" -#include "util/file_cache.h" namespace doris { namespace io { class LocalFileSystem final : public FileSystem { public: - static std::shared_ptr create(Path path, ResourceId resource_id = ""); + static std::shared_ptr create(Path path, std::string id = ""); ~LocalFileSystem() override; @@ -53,7 +52,7 @@ public: Status list(const Path& path, std::vector* files) override; private: - LocalFileSystem(Path root_path, ResourceId resource_id = ResourceId()); + LocalFileSystem(Path&& root_path, std::string&& id = ""); Path absolute_path(const Path& path) const; }; diff --git a/be/src/io/fs/remote_file_system.cpp b/be/src/io/fs/remote_file_system.cpp index 563cc79fba..3525cdcf8d 100644 --- a/be/src/io/fs/remote_file_system.cpp +++ b/be/src/io/fs/remote_file_system.cpp @@ -47,7 +47,7 @@ Status RemoteFileSystem::open_file(const Path& path, const FileReaderOptions& re case io::FileCachePolicy::FILE_BLOCK_CACHE: { DCHECK(io_ctx); StringPiece str(raw_reader->path().native()); - std::string cache_path = reader_options.path_policy.get_cache_path(str.as_string()); + std::string cache_path = reader_options.path_policy.get_cache_path(path.native()); *reader = std::make_shared(std::move(raw_reader), cache_path, io_ctx); break; diff --git a/be/src/io/fs/remote_file_system.h b/be/src/io/fs/remote_file_system.h index 195d424d93..d51fbb53ce 100644 --- a/be/src/io/fs/remote_file_system.h +++ b/be/src/io/fs/remote_file_system.h @@ -24,8 +24,8 @@ 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(Path&& root_path, std::string&& id, FileSystemType type) + : FileSystem(std::move(root_path), std::move(id), type) {} ~RemoteFileSystem() override = default; // `local_path` should be an absolute path on local filesystem. diff --git a/be/src/io/fs/s3_file_system.cpp b/be/src/io/fs/s3_file_system.cpp index 7f872586c6..4eceed3699 100644 --- a/be/src/io/fs/s3_file_system.cpp +++ b/be/src/io/fs/s3_file_system.cpp @@ -49,21 +49,20 @@ namespace io { } #endif -std::shared_ptr S3FileSystem::create(S3Conf s3_conf, ResourceId resource_id) { - return std::shared_ptr( - new S3FileSystem(std::move(s3_conf), std::move(resource_id))); +std::shared_ptr S3FileSystem::create(S3Conf s3_conf, std::string id) { + return std::shared_ptr(new S3FileSystem(std::move(s3_conf), std::move(id))); } -S3FileSystem::S3FileSystem(S3Conf s3_conf, ResourceId resource_id) +S3FileSystem::S3FileSystem(S3Conf&& s3_conf, std::string&& id) : RemoteFileSystem( fmt::format("{}/{}/{}", s3_conf.endpoint, s3_conf.bucket, s3_conf.prefix), - std::move(resource_id), FileSystemType::S3), + std::move(id), FileSystemType::S3), _s3_conf(std::move(s3_conf)) { if (_s3_conf.prefix.size() > 0 && _s3_conf.prefix[0] == '/') { _s3_conf.prefix = _s3_conf.prefix.substr(1); } _executor = Aws::MakeShared( - resource_id.c_str(), config::s3_transfer_executor_pool_size); + id.c_str(), config::s3_transfer_executor_pool_size); } S3FileSystem::~S3FileSystem() = default; diff --git a/be/src/io/fs/s3_file_system.h b/be/src/io/fs/s3_file_system.h index 93472bebd2..70940cb7d0 100644 --- a/be/src/io/fs/s3_file_system.h +++ b/be/src/io/fs/s3_file_system.h @@ -35,7 +35,7 @@ namespace io { // This class is thread-safe.(Except `set_xxx` method) class S3FileSystem final : public RemoteFileSystem { public: - static std::shared_ptr create(S3Conf s3_conf, ResourceId resource_id); + static std::shared_ptr create(S3Conf s3_conf, std::string id); ~S3FileSystem() override; @@ -71,16 +71,14 @@ public: }; // Guarded by external lock. - void set_ak(std::string ak) { _s3_conf.ak = std::move(ak); } - - // Guarded by external lock. - void set_sk(std::string sk) { _s3_conf.sk = std::move(sk); } + void set_conf(S3Conf s3_conf) { _s3_conf = std::move(s3_conf); } std::string get_key(const Path& path) const; private: - S3FileSystem(S3Conf s3_conf, ResourceId resource_id); + S3FileSystem(S3Conf&& s3_conf, std::string&& id); +private: S3Conf _s3_conf; // FIXME(cyx): We can use std::atomic since c++20. diff --git a/be/src/olap/CMakeLists.txt b/be/src/olap/CMakeLists.txt index 4373f2fbbe..3ae9d90ba2 100644 --- a/be/src/olap/CMakeLists.txt +++ b/be/src/olap/CMakeLists.txt @@ -107,7 +107,7 @@ add_library(Olap STATIC task/engine_publish_version_task.cpp task/engine_alter_tablet_task.cpp segment_loader.cpp - storage_policy_mgr.cpp + storage_policy.cpp ) if (NOT USE_MEM_TRACKER) diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h index bf04460f31..ca97cfb5cf 100644 --- a/be/src/olap/base_tablet.h +++ b/be/src/olap/base_tablet.h @@ -33,7 +33,7 @@ class DataDir; // Base class for all tablet classes, currently only olap/Tablet // The fields and methods in this class is not final, it will change as memory // storage engine evolves. -class BaseTablet : public std::enable_shared_from_this { +class BaseTablet { public: BaseTablet(TabletMetaSharedPtr tablet_meta, DataDir* data_dir); virtual ~BaseTablet(); @@ -59,9 +59,9 @@ public: int32_t schema_hash() const; int16_t shard_id() const; - const std::string& storage_policy() const { return _tablet_meta->storage_policy(); } + int64_t storage_policy_id() const { return _tablet_meta->storage_policy_id(); } - void set_storage_policy(const std::string& policy) { _tablet_meta->set_storage_policy(policy); } + void set_storage_policy_id(int64_t id) { _tablet_meta->set_storage_policy_id(id); } // properties encapsulated in TabletSchema virtual TabletSchemaSPtr tablet_schema() const; diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp index e29fca2276..4b2d1af949 100644 --- a/be/src/olap/data_dir.cpp +++ b/be/src/olap/data_dir.cpp @@ -17,6 +17,8 @@ #include "olap/data_dir.h" +#include "olap/storage_policy.h" + #ifndef __APPLE__ #include #include @@ -836,7 +838,7 @@ void DataDir::perform_remote_rowset_gc() { auto rowset_id = key.substr(REMOTE_ROWSET_GC_PREFIX.size()); RemoteRowsetGcPB gc_pb; gc_pb.ParseFromString(val); - auto fs = io::FileSystemMap::instance()->get(gc_pb.resource_id()); + auto fs = get_filesystem(gc_pb.resource_id()); if (!fs) { LOG(WARNING) << "Cannot get file system: " << gc_pb.resource_id(); continue; @@ -847,8 +849,7 @@ void DataDir::perform_remote_rowset_gc() { auto seg_path = BetaRowset::remote_segment_path(gc_pb.tablet_id(), rowset_id, i); st = fs->delete_file(seg_path); if (!st.ok()) { - LOG(WARNING) << st.to_string(); - break; + LOG(WARNING) << "failed to perform remote rowset gc, err=" << st; } } if (st.ok()) { @@ -871,12 +872,13 @@ void DataDir::perform_remote_tablet_gc() { std::vector deleted_keys; for (auto& [key, resource] : tablet_gc_kvs) { auto tablet_id = key.substr(REMOTE_TABLET_GC_PREFIX.size()); - auto fs = io::FileSystemMap::instance()->get(resource); + auto fs = get_filesystem(resource); if (!fs) { LOG(WARNING) << "could not get file system. resource_id=" << resource; continue; } auto st = fs->delete_directory(DATA_PREFIX + '/' + tablet_id); + LOG(INFO) << "perform remote tablet gc. tablet_id=" << tablet_id; if (st.ok()) { deleted_keys.push_back(std::move(key)); } else { diff --git a/be/src/olap/rowset/rowset_meta.h b/be/src/olap/rowset/rowset_meta.h index 5c6c296022..bb122c9834 100644 --- a/be/src/olap/rowset/rowset_meta.h +++ b/be/src/olap/rowset/rowset_meta.h @@ -26,11 +26,11 @@ #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" +#include "olap/storage_policy.h" #include "olap/tablet_schema.h" #include "olap/tablet_schema_cache.h" @@ -92,7 +92,7 @@ public: if (is_local()) { _fs = io::global_local_filesystem(); } else { - _fs = io::FileSystemMap::instance()->get(resource_id()); + _fs = get_filesystem(resource_id()); LOG_IF(WARNING, !_fs) << "Cannot get file system: " << resource_id(); } } @@ -101,14 +101,14 @@ public: void set_fs(io::FileSystemSPtr fs) { if (fs && fs->type() != io::FileSystemType::LOCAL) { - _rowset_meta_pb.set_resource_id(fs->resource_id()); + _rowset_meta_pb.set_resource_id(fs->id()); } _fs = std::move(fs); } - const io::ResourceId& resource_id() const { return _rowset_meta_pb.resource_id(); } + const std::string& resource_id() const { return _rowset_meta_pb.resource_id(); } - void set_resource_id(io::ResourceId resource_id) { + void set_resource_id(std::string resource_id) { _rowset_meta_pb.set_resource_id(std::move(resource_id)); } 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 0d6296fa6a..bc6dd603bc 100644 --- a/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp +++ b/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp @@ -18,7 +18,6 @@ #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 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 6f95a219ea..7e695616a7 100644 --- a/be/src/olap/rowset/segment_v2/indexed_column_reader.h +++ b/be/src/olap/rowset/segment_v2/indexed_column_reader.h @@ -25,7 +25,6 @@ #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/rowset/segment_v2/common.h" #include "olap/rowset/segment_v2/index_page.h" #include "olap/rowset/segment_v2/page_handle.h" 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 6b99dc3aa8..d9456fe8d0 100644 --- a/be/src/olap/rowset/segment_v2/ordinal_page_index.cpp +++ b/be/src/olap/rowset/segment_v2/ordinal_page_index.cpp @@ -19,7 +19,6 @@ #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/key_coder.h" diff --git a/be/src/olap/storage_policy.cpp b/be/src/olap/storage_policy.cpp new file mode 100644 index 0000000000..e2cb882b57 --- /dev/null +++ b/be/src/olap/storage_policy.cpp @@ -0,0 +1,105 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "olap/storage_policy.h" + +#include +#include + +namespace doris { + +struct StoragePolicyMgr { + std::mutex mtx; + std::unordered_map map; +}; + +static StoragePolicyMgr s_storage_policy_mgr; + +StoragePolicyPtr get_storage_policy(int64_t id) { + std::lock_guard lock(s_storage_policy_mgr.mtx); + if (auto it = s_storage_policy_mgr.map.find(id); it != s_storage_policy_mgr.map.end()) { + return it->second; + } + return nullptr; +} + +void put_storage_policy(int64_t id, StoragePolicyPtr policy) { + std::lock_guard lock(s_storage_policy_mgr.mtx); + s_storage_policy_mgr.map[id] = std::move(policy); +} + +void delete_storage_policy(int64_t id) { + std::lock_guard lock(s_storage_policy_mgr.mtx); + s_storage_policy_mgr.map.erase(id); +} + +std::vector> get_storage_policy_ids() { + std::vector> res; + res.reserve(s_storage_policy_mgr.map.size()); + std::lock_guard lock(s_storage_policy_mgr.mtx); + for (auto& [id, policy] : s_storage_policy_mgr.map) { + res.emplace_back(id, policy->version); + } + return res; +} + +struct StorageResouceMgr { + std::mutex mtx; + std::unordered_map map; +}; + +static StorageResouceMgr s_storage_resource_mgr; + +io::FileSystemSPtr get_filesystem(const std::string& resource_id) { + int64_t id = std::atol(resource_id.c_str()); + std::lock_guard lock(s_storage_resource_mgr.mtx); + if (auto it = s_storage_resource_mgr.map.find(id); it != s_storage_resource_mgr.map.end()) { + return it->second.fs; + } + return nullptr; +} + +StorageResource get_storage_resource(int64_t resource_id) { + std::lock_guard lock(s_storage_resource_mgr.mtx); + if (auto it = s_storage_resource_mgr.map.find(resource_id); + it != s_storage_resource_mgr.map.end()) { + return it->second; + } + return StorageResource {nullptr, -1}; +} + +void put_storage_resource(int64_t resource_id, StorageResource resource) { + std::lock_guard lock(s_storage_resource_mgr.mtx); + s_storage_resource_mgr.map[resource_id] = std::move(resource); +} + +void delete_storage_resource(int64_t resource_id) { + std::lock_guard lock(s_storage_resource_mgr.mtx); + s_storage_resource_mgr.map.erase(resource_id); +} + +std::vector> get_storage_resource_ids() { + std::vector> res; + res.reserve(s_storage_resource_mgr.map.size()); + std::lock_guard lock(s_storage_resource_mgr.mtx); + for (auto& [id, resource] : s_storage_resource_mgr.map) { + res.emplace_back(id, resource.version); + } + return res; +} + +} // namespace doris diff --git a/be/src/olap/storage_policy.h b/be/src/olap/storage_policy.h new file mode 100644 index 0000000000..4ddac3d36f --- /dev/null +++ b/be/src/olap/storage_policy.h @@ -0,0 +1,71 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include "io/fs/file_system.h" + +namespace doris { + +struct StoragePolicy { + std::string name; + int64_t version; + int64_t cooldown_datetime; + int64_t cooldown_ttl; + int64_t resource_id; + + std::string to_string() const { + return fmt::format( + "(name={}, version={}, cooldown_date_time={}, cooldown_ttl={}, resource_id={})", + name, version, cooldown_datetime, cooldown_ttl, resource_id); + } +}; + +using StoragePolicyPtr = std::shared_ptr; + +// return nullptr if not found +StoragePolicyPtr get_storage_policy(int64_t id); + +// always success +void put_storage_policy(int64_t id, StoragePolicyPtr policy); + +void delete_storage_policy(int64_t id); + +// return [id, version] of all storage policies +std::vector> get_storage_policy_ids(); + +struct StorageResource { + io::FileSystemSPtr fs; + int64_t version = -1; +}; + +// return nullptr if not found +// use string `resource_id` for compatibility +io::FileSystemSPtr get_filesystem(const std::string& resource_id); + +// return [nullptr, -1] if not found +StorageResource get_storage_resource(int64_t resource_id); + +// always success +void put_storage_resource(int64_t resource_id, StorageResource resource); + +void delete_storage_resource(int64_t resource_id); + +// return [id, version] of all resources +std::vector> get_storage_resource_ids(); + +} // namespace doris diff --git a/be/src/olap/storage_policy_mgr.cpp b/be/src/olap/storage_policy_mgr.cpp deleted file mode 100644 index 58da3ff03e..0000000000 --- a/be/src/olap/storage_policy_mgr.cpp +++ /dev/null @@ -1,105 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "olap/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, const StoragePolicyPtr& policy) { - std::shared_ptr s3_fs; - { - std::lock_guard l(_mutex); - auto it = _policy_map.find(name); - if (it != _policy_map.end()) { - LOG(INFO) << "update storage policy name: " << name; - it->second = policy; - s3_fs = std::dynamic_pointer_cast( - io::FileSystemMap::instance()->get(name)); - if (s3_fs) { - s3_fs->set_ak(policy->s3_ak); - s3_fs->set_sk(policy->s3_sk); - } else { - DCHECK(false) << "s3_fs is null"; - } - } - } - if (s3_fs) { - auto st = s3_fs->connect(); - if (!st.ok()) { - LOG(ERROR) << st; - } - } -} - -void StoragePolicyMgr::periodic_put(const std::string& name, const StoragePolicyPtr& policy) { - std::shared_ptr s3_fs; - { - 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"; - S3Conf s3_conf; - s3_conf.ak = policy->s3_ak; - s3_conf.sk = policy->s3_sk; - s3_conf.endpoint = policy->s3_endpoint; - s3_conf.region = policy->s3_region; - s3_conf.max_connections = policy->s3_max_conn; - s3_conf.request_timeout_ms = policy->s3_request_timeout_ms; - s3_conf.connect_timeout_ms = policy->s3_conn_timeout_ms; - s3_conf.bucket = policy->bucket; - s3_conf.prefix = policy->root_path; - s3_fs = io::S3FileSystem::create(std::move(s3_conf), name); - io::FileSystemMap::instance()->insert(name, s3_fs); - _policy_map.emplace(name, policy); - } else if (it->second->md5_sum != policy->md5_sum) { - LOG(INFO) << "update storage policy name: " << name; - it->second = policy; - 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); - } - } - if (s3_fs) { - auto st = s3_fs->connect(); - if (!st.ok()) { - LOG(ERROR) << st; - } - } -} - -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 deleted file mode 100644 index 2da2c29f2d..0000000000 --- a/be/src/olap/storage_policy_mgr.h +++ /dev/null @@ -1,79 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include - -#include -#include -#include - -#include "common/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() = default; - - ~StoragePolicyMgr() = default; - - // fe push update policy to be - void update(const std::string& name, const StoragePolicyPtr& policy); - - // periodic pull from fe - void periodic_put(const std::string& name, const 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 d7afed0875..a75d9fe3bc 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -56,7 +56,7 @@ #include "olap/rowset/segment_v2/column_reader.h" #include "olap/schema_change.h" #include "olap/storage_engine.h" -#include "olap/storage_policy_mgr.h" +#include "olap/storage_policy.h" #include "olap/tablet_meta.h" #include "olap/tablet_meta_manager.h" #include "olap/tablet_schema.h" @@ -1390,7 +1390,7 @@ void Tablet::build_tablet_report_info(TTabletInfo* tablet_info, 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()); - tablet_info->__set_is_cooldown(!_tablet_meta->storage_policy().empty()); + tablet_info->__set_is_cooldown(_tablet_meta->storage_policy_id() > 0); if (tablet_info->is_cooldown) { tablet_info->__set_cooldown_replica_id(_tablet_meta->cooldown_replica_id()); } @@ -1651,12 +1651,19 @@ Status Tablet::cooldown() { } Status Tablet::_cooldown_data() { - auto dest_fs = io::FileSystemMap::instance()->get(storage_policy()); - if (!dest_fs) { - return Status::Error(); + auto storage_policy = get_storage_policy(storage_policy_id()); + if (storage_policy == nullptr) { + return Status::InternalError("could not find storage_policy, storage_policy_id={}", + storage_policy_id()); } - DCHECK(dest_fs->type() == io::FileSystemType::S3); - + auto resource = get_storage_resource(storage_policy->resource_id); + auto& dest_fs = resource.fs; + if (dest_fs == nullptr) { + return Status::InternalError("could not find resource, resouce_id={}", + storage_policy->resource_id); + } + DCHECK(atol(dest_fs->id().c_str()) == storage_policy->resource_id); + DCHECK(dest_fs->type() != io::FileSystemType::LOCAL); auto old_rowset = pick_cooldown_rowset(); if (!old_rowset) { LOG(WARNING) << "Cannot pick cooldown rowset in tablet " << tablet_id(); @@ -1666,11 +1673,10 @@ Status Tablet::_cooldown_data() { auto start = std::chrono::steady_clock::now(); - auto st = old_rowset->upload_to(reinterpret_cast(dest_fs.get()), - new_rowset_id); + auto st = + old_rowset->upload_to(static_cast(dest_fs.get()), new_rowset_id); if (!st.ok()) { - record_unused_remote_rowset(new_rowset_id, dest_fs->resource_id(), - old_rowset->num_segments()); + record_unused_remote_rowset(new_rowset_id, dest_fs->id(), old_rowset->num_segments()); return st; } @@ -1683,7 +1689,7 @@ Status Tablet::_cooldown_data() { // 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_resource_id(dest_fs->id()); new_rowset_meta->set_fs(dest_fs); new_rowset_meta->set_creation_time(time(nullptr)); @@ -1764,11 +1770,19 @@ Status Tablet::_write_remote_tablet_meta(FileSystemSPtr fs, const TabletMetaPB& } Status Tablet::_follow_cooldowned_data() { - auto dest_fs = io::FileSystemMap::instance()->get(storage_policy()); - if (!dest_fs) { - return Status::InternalError("storage_policy doesn't exist: " + storage_policy()); + auto storage_policy = get_storage_policy(storage_policy_id()); + if (storage_policy == nullptr) { + return Status::InternalError("could not find storage_policy, storage_policy_id={}", + storage_policy_id()); } - DCHECK(dest_fs->type() == io::FileSystemType::S3); + auto resource = get_storage_resource(storage_policy->resource_id); + auto& dest_fs = resource.fs; + if (dest_fs == nullptr) { + return Status::InternalError("could not find resource, resouce_id={}", + storage_policy->resource_id); + } + DCHECK(atol(dest_fs->id().c_str()) == storage_policy->resource_id); + DCHECK(dest_fs->type() != io::FileSystemType::LOCAL); TabletMetaPB remote_tablet_meta_pb; RETURN_IF_ERROR(_read_remote_tablet_meta(dest_fs, &remote_tablet_meta_pb)); int64_t max_version = -1; @@ -1827,18 +1841,18 @@ RowsetSharedPtr Tablet::pick_cooldown_rowset() { } bool Tablet::need_cooldown(int64_t* cooldown_timestamp, size_t* file_size) { - // std::shared_lock meta_rlock(_meta_lock); - if (storage_policy().empty()) { + int64_t id = storage_policy_id(); + if (id <= 0) { VLOG_DEBUG << "tablet does not need cooldown, tablet id: " << tablet_id(); return false; } - auto policy = ExecEnv::GetInstance()->storage_policy_mgr()->get(storage_policy()); - if (!policy) { - LOG(WARNING) << "Cannot get storage policy: " << storage_policy(); + auto storage_policy = get_storage_policy(id); + if (!storage_policy) { + LOG(WARNING) << "Cannot get storage policy: " << id; return false; } - auto cooldown_ttl_sec = policy->cooldown_ttl; - auto cooldown_datetime = policy->cooldown_datetime; + auto cooldown_ttl_sec = storage_policy->cooldown_ttl; + auto cooldown_datetime = storage_policy->cooldown_datetime; RowsetSharedPtr rowset = pick_cooldown_rowset(); if (!rowset) { VLOG_DEBUG << "pick cooldown rowset, get null, tablet id: " << tablet_id(); @@ -1882,7 +1896,7 @@ bool Tablet::need_cooldown(int64_t* cooldown_timestamp, size_t* file_size) { return false; } -void Tablet::record_unused_remote_rowset(const RowsetId& rowset_id, const io::ResourceId& resource, +void Tablet::record_unused_remote_rowset(const RowsetId& rowset_id, const std::string& resource, int64_t num_segments) { auto gc_key = REMOTE_ROWSET_GC_PREFIX + rowset_id.to_string(); RemoteRowsetGcPB gc_pb; @@ -1897,11 +1911,13 @@ void Tablet::record_unused_remote_rowset(const RowsetId& rowset_id, const io::Re Status Tablet::remove_all_remote_rowsets() { DCHECK(_state == TABLET_SHUTDOWN); - if (storage_policy().empty()) { + if (storage_policy_id() == 0) { return Status::OK(); } auto tablet_gc_key = REMOTE_TABLET_GC_PREFIX + std::to_string(tablet_id()); - return _data_dir->get_meta()->put(META_COLUMN_FAMILY_INDEX, tablet_gc_key, storage_policy()); + auto policy = get_storage_policy(storage_policy_id()); + return _data_dir->get_meta()->put(META_COLUMN_FAMILY_INDEX, tablet_gc_key, + std::to_string(policy->resource_id)); } TabletSchemaSPtr Tablet::tablet_schema() const { diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index f7e80614e0..27b05f2b10 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -333,7 +333,7 @@ public: // REQUIRES: held _meta_lock void update_self_owned_remote_rowsets(const std::vector& rowsets_in_snapshot); - void record_unused_remote_rowset(const RowsetId& rowset_id, const io::ResourceId& resource, + void record_unused_remote_rowset(const RowsetId& rowset_id, const std::string& resource, int64_t num_segments); bool check_all_rowset_segment(); diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index d3f30227b9..18634b94e9 100644 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -44,7 +44,7 @@ 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.compression_type, request.storage_policy, + request.compression_type, request.storage_policy_id, request.__isset.enable_unique_key_merge_on_write ? request.enable_unique_key_merge_on_write : false)); @@ -61,7 +61,7 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id const TTabletSchema& tablet_schema, uint32_t next_unique_id, const std::unordered_map& col_ordinal_to_unique_id, TabletUid tablet_uid, TTabletType::type tabletType, - TCompressionType::type compression_type, const std::string& storage_policy, + TCompressionType::type compression_type, int64_t storage_policy_id, bool enable_unique_key_merge_on_write) : _tablet_uid(0, 0), _schema(new TabletSchema), @@ -82,7 +82,7 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id ? TabletTypePB::TABLET_TYPE_DISK : TabletTypePB::TABLET_TYPE_MEMORY); tablet_meta_pb.set_enable_unique_key_merge_on_write(enable_unique_key_merge_on_write); - tablet_meta_pb.set_storage_policy(storage_policy); + tablet_meta_pb.set_storage_policy_id(storage_policy_id); 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); @@ -254,9 +254,9 @@ TabletMeta::TabletMeta(const TabletMeta& b) _stale_rs_metas(b._stale_rs_metas), _in_restore_mode(b._in_restore_mode), _preferred_rowset_type(b._preferred_rowset_type), - _storage_policy(b._storage_policy), _cooldown_replica_id(b._cooldown_replica_id), _cooldown_term(b._cooldown_term), + _storage_policy_id(b._storage_policy_id), _enable_unique_key_merge_on_write(b._enable_unique_key_merge_on_write), _delete_bitmap(b._delete_bitmap) {}; @@ -524,9 +524,9 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) { _preferred_rowset_type = tablet_meta_pb.preferred_rowset_type(); } - _storage_policy = tablet_meta_pb.storage_policy(); _cooldown_replica_id = -1; _cooldown_term = -1; + _storage_policy_id = tablet_meta_pb.storage_policy_id(); if (tablet_meta_pb.has_enable_unique_key_merge_on_write()) { _enable_unique_key_merge_on_write = tablet_meta_pb.enable_unique_key_merge_on_write(); } @@ -599,7 +599,7 @@ void TabletMeta::to_meta_pb(bool only_include_remote_rowset, TabletMetaPB* table tablet_meta_pb->set_preferred_rowset_type(_preferred_rowset_type); } - tablet_meta_pb->set_storage_policy(_storage_policy); + tablet_meta_pb->set_storage_policy_id(_storage_policy_id); tablet_meta_pb->set_enable_unique_key_merge_on_write(_enable_unique_key_merge_on_write); if (_enable_unique_key_merge_on_write) { @@ -877,13 +877,13 @@ bool operator==(const TabletMeta& a, const TabletMeta& b) { } if (a._in_restore_mode != b._in_restore_mode) return false; if (a._preferred_rowset_type != b._preferred_rowset_type) return false; - if (a._storage_policy != b._storage_policy) return false; if (a._cooldown_replica_id != b._cooldown_replica_id) { return false; } if (a._cooldown_term != b._cooldown_term) { return false; } + if (a._storage_policy_id != b._storage_policy_id) return false; return true; } diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index f287662ad2..e715ec7973 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -82,15 +82,12 @@ public: TabletMetaSharedPtr* tablet_meta); TabletMeta(); - // Only remote_storage_name is needed in meta, it is a key used to get remote params from fe. - // The config of storage is saved in fe. TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id, int64_t replica_id, int32_t schema_hash, uint64_t shard_id, const TTabletSchema& tablet_schema, uint32_t next_unique_id, const std::unordered_map& col_ordinal_to_unique_id, TabletUid tablet_uid, TTabletType::type tabletType, - TCompressionType::type compression_type, - const std::string& storage_policy = std::string(), + TCompressionType::type compression_type, int64_t storage_policy_id = 0, bool enable_unique_key_merge_on_write = false); // If need add a filed in TableMeta, filed init copy in copy construct function TabletMeta(const TabletMeta& tablet_meta); @@ -189,16 +186,12 @@ public: bool all_beta() const; - const std::string& storage_policy() const { - std::shared_lock rlock(_meta_lock); - return _storage_policy; - } + int64_t storage_policy_id() const { return _storage_policy_id; } - void set_storage_policy(const std::string& policy) { - std::unique_lock wlock(_meta_lock); - VLOG_NOTICE << "set tablet_id : " << _table_id << " storage policy from " << _storage_policy - << " to " << policy; - _storage_policy = policy; + void set_storage_policy_id(int64_t id) { + VLOG_NOTICE << "set tablet_id : " << _table_id << " storage policy from " + << _storage_policy_id << " to " << id; + _storage_policy_id = id; } const int64_t cooldown_replica_id() const { return _cooldown_replica_id; } @@ -255,9 +248,9 @@ private: bool _in_restore_mode = false; RowsetTypePB _preferred_rowset_type = BETA_ROWSET; - std::string _storage_policy; int64_t _cooldown_replica_id = -1; int64_t _cooldown_term = -1; + int64_t _storage_policy_id = 0; // <= 0 means no storage policy // For unique key data model, the feature Merge-on-Write will leverage a primary // key index and a delete-bitmap to mark duplicate keys as deleted in load stage, diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index c6123d7fbe..f8239bde8e 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -159,7 +159,6 @@ public: LoadStreamMgr* load_stream_mgr() { return _load_stream_mgr; } NewLoadStreamMgr* new_load_stream_mgr() { return _new_load_stream_mgr; } SmallFileMgr* small_file_mgr() { return _small_file_mgr; } - StoragePolicyMgr* storage_policy_mgr() { return _storage_policy_mgr; } BlockSpillManager* block_spill_mgr() { return _block_spill_mgr; } const std::vector& store_paths() const { return _store_paths; } @@ -242,7 +241,6 @@ private: RoutineLoadTaskExecutor* _routine_load_task_executor = nullptr; SmallFileMgr* _small_file_mgr = nullptr; HeartbeatFlags* _heartbeat_flags = nullptr; - StoragePolicyMgr* _storage_policy_mgr = nullptr; doris::vectorized::ScannerScheduler* _scanner_scheduler = nullptr; BlockSpillManager* _block_spill_mgr = nullptr; diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index ef88739bf4..d18d5a83dc 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -25,7 +25,6 @@ #include "olap/rowset/segment_v2/inverted_index_cache.h" #include "olap/segment_loader.h" #include "olap/storage_engine.h" -#include "olap/storage_policy_mgr.h" #include "pipeline/task_scheduler.h" #include "runtime/block_spill_manager.h" #include "runtime/broker_mgr.h" @@ -123,7 +122,6 @@ Status ExecEnv::_init(const std::vector& store_paths) { _stream_load_executor = new StreamLoadExecutor(this); _routine_load_task_executor = new RoutineLoadTaskExecutor(this); _small_file_mgr = new SmallFileMgr(this, config::small_file_dir); - _storage_policy_mgr = new StoragePolicyMgr(); _block_spill_mgr = new BlockSpillManager(_store_paths); _backend_client_cache->init_metrics("backend"); diff --git a/be/src/util/s3_util.h b/be/src/util/s3_util.h index 156bcf5444..4adcc72874 100644 --- a/be/src/util/s3_util.h +++ b/be/src/util/s3_util.h @@ -56,19 +56,15 @@ struct S3Conf { int connect_timeout_ms = -1; bool use_virtual_addressing = true; - std::string to_string() const; + std::string to_string() const { + return fmt::format( + "(ak={}, sk=*, endpoint={}, region={}, bucket={}, prefix={}, max_connections={}, " + "request_timeout_ms={}, connect_timeout_ms={}, use_virtual_addressing={})", + ak, endpoint, region, bucket, prefix, max_connections, request_timeout_ms, + connect_timeout_ms, use_virtual_addressing); + } }; -inline std::string S3Conf::to_string() const { - std::stringstream ss; - ss << "ak: " << ak << ", sk: " << sk << ", endpoint: " << endpoint << ", region: " << region - << ", bucket: " << bucket << ", prefix: " << prefix - << ", max_connections: " << max_connections << ", request_timeout_ms: " << request_timeout_ms - << ", connect_timeout_ms: " << connect_timeout_ms - << ", use_virtual_addressing: " << use_virtual_addressing; - return ss.str(); -} - class ClientFactory { public: ~ClientFactory(); diff --git a/be/test/io/cache/remote_file_cache_test.cpp b/be/test/io/cache/remote_file_cache_test.cpp index 1a1ad93750..74aa73ede7 100644 --- a/be/test/io/cache/remote_file_cache_test.cpp +++ b/be/test/io/cache/remote_file_cache_test.cpp @@ -62,7 +62,7 @@ static StorageEngine* k_engine = nullptr; static std::string kSegmentDir = "./ut_dir/remote_file_cache_test"; static int64_t tablet_id = 0; static RowsetId rowset_id; -static io::ResourceId resource_id = "test_resourse_id"; +static std::string resource_id = "10000"; class RemoteFileCacheTest : public ::testing::Test { protected: @@ -164,8 +164,7 @@ protected: // just use to create s3 filesystem, otherwise won't use cache S3Conf s3_conf; - std::shared_ptr fs = - io::S3FileSystem::create(std::move(s3_conf), resource_id); + auto fs = io::S3FileSystem::create(std::move(s3_conf), resource_id); rowset.rowset_meta()->set_resource_id(resource_id); rowset.rowset_meta()->set_num_segments(1); rowset.rowset_meta()->set_fs(fs); diff --git a/be/test/olap/ordered_data_compaction_test.cpp b/be/test/olap/ordered_data_compaction_test.cpp index 1cd6c17212..e345447220 100644 --- a/be/test/olap/ordered_data_compaction_test.cpp +++ b/be/test/olap/ordered_data_compaction_test.cpp @@ -309,7 +309,7 @@ protected: TabletMetaSharedPtr tablet_meta( new TabletMeta(2, 2, 2, 2, 2, 2, t_tablet_schema, 2, col_ordinal_to_unique_id, UniqueId(1, 2), TTabletType::TABLET_TYPE_DISK, - TCompressionType::LZ4F, "", enable_unique_key_merge_on_write)); + TCompressionType::LZ4F, 0, enable_unique_key_merge_on_write)); TabletSharedPtr tablet(new Tablet(tablet_meta, _data_dir.get())); tablet->init(); diff --git a/be/test/olap/remote_rowset_gc_test.cpp b/be/test/olap/remote_rowset_gc_test.cpp index 65d4895119..6b09183af5 100644 --- a/be/test/olap/remote_rowset_gc_test.cpp +++ b/be/test/olap/remote_rowset_gc_test.cpp @@ -23,11 +23,11 @@ #include "common/status.h" #include "exec/tablet_info.h" #include "gen_cpp/internal_service.pb.h" -#include "io/fs/file_system_map.h" #include "io/fs/s3_file_system.h" #include "olap/delta_writer.h" #include "olap/rowset/beta_rowset.h" #include "olap/storage_engine.h" +#include "olap/storage_policy.h" #include "olap/tablet.h" #include "runtime/descriptor_helper.h" #include "util/file_utils.h" @@ -38,7 +38,8 @@ namespace doris { static StorageEngine* k_engine = nullptr; static const std::string kTestDir = "./ut_dir/remote_rowset_gc_test"; -static const std::string kResourceId = "RemoteRowsetGcTest"; +static constexpr int64_t kResourceId = 10000; +static constexpr int64_t kStoragePolicyId = 10002; // remove DISABLED_ when need run this test #define RemoteRowsetGcTest DISABLED_RemoteRowsetGcTest @@ -52,9 +53,14 @@ public: s3_conf.region = config::test_s3_region; s3_conf.bucket = config::test_s3_bucket; s3_conf.prefix = "remote_rowset_gc_test"; - auto s3_fs = io::S3FileSystem::create(std::move(s3_conf), kResourceId); + auto s3_fs = io::S3FileSystem::create(std::move(s3_conf), std::to_string(kResourceId)); ASSERT_TRUE(s3_fs->connect().ok()); - io::FileSystemMap::instance()->insert(kResourceId, s3_fs); + put_storage_resource(kResourceId, {s3_fs, 1}); + auto storage_policy = std::make_shared(); + storage_policy->name = "TabletCooldownTest"; + storage_policy->version = 1; + storage_policy->resource_id = kResourceId; + put_storage_policy(kStoragePolicyId, storage_policy); constexpr uint32_t MAX_PATH_LEN = 1024; char buffer[MAX_PATH_LEN]; @@ -190,7 +196,7 @@ TEST_F(RemoteRowsetGcTest, normal) { } EXPECT_EQ(0, tablet->num_rows()); - tablet->set_storage_policy(kResourceId); + tablet->set_storage_policy_id(kStoragePolicyId); st = tablet->cooldown(); // rowset [0-1] ASSERT_EQ(Status::OK(), st); st = tablet->cooldown(); // rowset [2-2] @@ -199,7 +205,7 @@ TEST_F(RemoteRowsetGcTest, normal) { delete delta_writer; - auto fs = io::FileSystemMap::instance()->get(kResourceId); + auto fs = get_storage_resource(kResourceId).fs; auto rowset = tablet->get_rowset_by_version({2, 2}); ASSERT_TRUE(rowset); auto seg_path = BetaRowset::remote_segment_path(10005, rowset->rowset_id(), 0); diff --git a/be/test/olap/rowid_conversion_test.cpp b/be/test/olap/rowid_conversion_test.cpp index 129f2cb513..ddbc5303d9 100644 --- a/be/test/olap/rowid_conversion_test.cpp +++ b/be/test/olap/rowid_conversion_test.cpp @@ -265,7 +265,7 @@ protected: TabletMetaSharedPtr tablet_meta( new TabletMeta(1, 1, 1, 1, 1, 1, t_tablet_schema, 1, col_ordinal_to_unique_id, UniqueId(1, 2), TTabletType::TABLET_TYPE_DISK, - TCompressionType::LZ4F, "", enable_unique_key_merge_on_write)); + TCompressionType::LZ4F, 0, enable_unique_key_merge_on_write)); TabletSharedPtr tablet(new Tablet(tablet_meta, nullptr)); tablet->init(); diff --git a/be/test/olap/rowset/beta_rowset_test.cpp b/be/test/olap/rowset/beta_rowset_test.cpp index 8766faab6c..de05810d21 100644 --- a/be/test/olap/rowset/beta_rowset_test.cpp +++ b/be/test/olap/rowset/beta_rowset_test.cpp @@ -230,9 +230,8 @@ TEST_F(BetaRowsetTest, ReadTest) { s3_conf.region = "region"; s3_conf.bucket = "bucket"; s3_conf.prefix = "prefix"; - io::ResourceId resource_id = "test_resourse_id"; - std::shared_ptr fs = - io::S3FileSystem::create(std::move(s3_conf), resource_id); + std::string resource_id = "10000"; + auto fs = io::S3FileSystem::create(std::move(s3_conf), resource_id); Aws::SDKOptions aws_options = Aws::SDKOptions {}; Aws::InitAPI(aws_options); // failed to head object diff --git a/be/test/olap/tablet_cooldown_test.cpp b/be/test/olap/tablet_cooldown_test.cpp index 62a67e88ee..6fde083581 100644 --- a/be/test/olap/tablet_cooldown_test.cpp +++ b/be/test/olap/tablet_cooldown_test.cpp @@ -23,10 +23,11 @@ #include "common/status.h" #include "exec/tablet_info.h" #include "gen_cpp/internal_service.pb.h" -#include "io/fs/file_system_map.h" #include "io/fs/s3_file_system.h" #include "olap/delta_writer.h" +#include "olap/rowset/beta_rowset.h" #include "olap/storage_engine.h" +#include "olap/storage_policy.h" #include "olap/tablet.h" #include "runtime/descriptor_helper.h" #include "util/file_utils.h" @@ -37,7 +38,8 @@ namespace doris { static StorageEngine* k_engine = nullptr; static const std::string kTestDir = "./ut_dir/tablet_cooldown_test"; -static const std::string kResourceId = "TabletCooldownTest"; +static constexpr int64_t kResourceId = 10000; +static constexpr int64_t kStoragePolicyId = 10002; // remove DISABLED_ when need run this test #define TabletCooldownTest DISABLED_TabletCooldownTest @@ -50,10 +52,15 @@ public: s3_conf.endpoint = config::test_s3_endpoint; s3_conf.region = config::test_s3_region; s3_conf.bucket = config::test_s3_bucket; - s3_conf.prefix = "tablet_cooldown_test"; - auto s3_fs = io::S3FileSystem::create(std::move(s3_conf), kResourceId); + s3_conf.prefix = config::test_s3_prefix + "/tablet_cooldown_test"; + auto s3_fs = io::S3FileSystem::create(std::move(s3_conf), std::to_string(kResourceId)); ASSERT_TRUE(s3_fs->connect().ok()); - io::FileSystemMap::instance()->insert(kResourceId, s3_fs); + put_storage_resource(kResourceId, {s3_fs, 1}); + auto storage_policy = std::make_shared(); + storage_policy->name = "TabletCooldownTest"; + storage_policy->version = 1; + storage_policy->resource_id = kResourceId; + put_storage_policy(kStoragePolicyId, storage_policy); constexpr uint32_t MAX_PATH_LEN = 1024; char buffer[MAX_PATH_LEN]; @@ -139,6 +146,7 @@ static TDescriptorTable create_descriptor_tablet_with_sequence_col() { } TEST_F(TabletCooldownTest, normal) { + // create tablet TCreateTabletReq request; create_tablet_request_with_sequence_col(10005, 270068377, &request); Status st = k_engine->create_tablet(request); @@ -151,6 +159,7 @@ TEST_F(TabletCooldownTest, normal) { TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0); OlapTableSchemaParam param; + // write data PUniqueId load_id; load_id.set_hi(0); load_id.set_lo(0); @@ -160,13 +169,38 @@ TEST_F(TabletCooldownTest, normal) { DeltaWriter::open(&write_req, &delta_writer); ASSERT_NE(delta_writer, nullptr); - MemTracker tracker; - MemPool pool(&tracker); + vectorized::Block block; + for (const auto& slot_desc : tuple_desc->slots()) { + block.insert(vectorized::ColumnWithTypeAndName(slot_desc->get_empty_mutable_column(), + slot_desc->get_data_type_ptr(), + slot_desc->col_name())); + } + + auto columns = block.mutate_columns(); + { + int8_t c1 = 123; + columns[0]->insert_data((const char*)&c1, sizeof(c1)); + + int16_t c2 = 456; + columns[1]->insert_data((const char*)&c2, sizeof(c2)); + + int32_t c3 = 1; + columns[2]->insert_data((const char*)&c3, sizeof(c2)); + + DateTimeValue c4; + c4.from_date_str("2020-07-16 19:39:43", 19); + int64_t c4_int = c4.to_int64(); + columns[3]->insert_data((const char*)&c4_int, sizeof(c4)); + + st = delta_writer->write(&block, {0}); + ASSERT_EQ(Status::OK(), st); + } st = delta_writer->close(); ASSERT_EQ(Status::OK(), st); st = delta_writer->close_wait(PSlaveTabletNodes(), false); ASSERT_EQ(Status::OK(), st); + delete delta_writer; // publish version success TabletSharedPtr tablet = @@ -187,16 +221,23 @@ TEST_F(TabletCooldownTest, normal) { st = tablet->add_inc_rowset(rowset); ASSERT_EQ(Status::OK(), st); } - EXPECT_EQ(0, tablet->num_rows()); + EXPECT_EQ(1, tablet->num_rows()); - tablet->set_storage_policy(kResourceId); + // test cooldown + tablet->set_storage_policy_id(kStoragePolicyId); 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); + auto rs = tablet->get_rowset_by_version({2, 2}); + ASSERT_FALSE(rs->is_local()); - delete delta_writer; + // test read + ASSERT_EQ(Status::OK(), st); + std::vector segments; + st = std::static_pointer_cast(rs)->load_segments(&segments); + ASSERT_EQ(Status::OK(), st); + ASSERT_EQ(segments.size(), 1); } } // namespace doris diff --git a/be/test/olap/tablet_test.cpp b/be/test/olap/tablet_test.cpp index db3ffb02df..d0d0a70a6d 100644 --- a/be/test/olap/tablet_test.cpp +++ b/be/test/olap/tablet_test.cpp @@ -25,7 +25,7 @@ #include "olap/olap_define.h" #include "olap/rowset/beta_rowset.h" #include "olap/storage_engine.h" -#include "olap/storage_policy_mgr.h" +#include "olap/storage_policy.h" #include "olap/tablet_meta.h" #include "olap/tablet_schema_cache.h" #include "testutil/mock_rowset.h" @@ -98,10 +98,9 @@ public: } TabletMetaSharedPtr new_tablet_meta(TTabletSchema schema, bool enable_merge_on_write = false) { - return static_cast( - new TabletMeta(1, 2, 15673, 15674, 4, 5, schema, 6, {{7, 8}}, UniqueId(9, 10), - TTabletType::TABLET_TYPE_DISK, TCompressionType::LZ4F, std::string(), - enable_merge_on_write)); + return static_cast(new TabletMeta( + 1, 2, 15673, 15674, 4, 5, schema, 6, {{7, 8}}, UniqueId(9, 10), + TTabletType::TABLET_TYPE_DISK, TCompressionType::LZ4F, 0, enable_merge_on_write)); } void init_rs_meta(RowsetMetaSharedPtr& pb1, int64_t start, int64_t end) { @@ -313,7 +312,8 @@ TEST_F(TestTablet, cooldown_policy) { TabletSharedPtr _tablet(new Tablet(_tablet_meta, nullptr)); _tablet->init(); - _tablet->set_storage_policy("test_policy_name"); + constexpr int64_t storage_policy_id = 10000; + _tablet->set_storage_policy_id(storage_policy_id); _tablet->_rs_version_map[ptr1->version()] = rowset1; _tablet->_rs_version_map[ptr2->version()] = rowset2; @@ -323,18 +323,11 @@ TEST_F(TestTablet, cooldown_policy) { _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; + auto storage_policy = std::make_shared(); + storage_policy->cooldown_datetime = 250; + storage_policy->cooldown_ttl = -1; + put_storage_policy(storage_policy_id, storage_policy); int64_t cooldown_timestamp = -1; size_t file_size = -1; @@ -345,15 +338,10 @@ TEST_F(TestTablet, cooldown_policy) { } { - 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; + auto storage_policy = std::make_shared(); + storage_policy->cooldown_datetime = -1; + storage_policy->cooldown_ttl = 3600; + put_storage_policy(storage_policy_id, storage_policy); int64_t cooldown_timestamp = -1; size_t file_size = -1; @@ -364,15 +352,10 @@ TEST_F(TestTablet, cooldown_policy) { } { - 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; + auto storage_policy = std::make_shared(); + storage_policy->cooldown_datetime = UnixSeconds() + 100; + storage_policy->cooldown_ttl = -1; + put_storage_policy(storage_policy_id, storage_policy); int64_t cooldown_timestamp = -1; size_t file_size = -1; @@ -383,15 +366,10 @@ TEST_F(TestTablet, cooldown_policy) { } { - 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; + auto storage_policy = std::make_shared(); + storage_policy->cooldown_datetime = UnixSeconds() + 100; + storage_policy->cooldown_ttl = UnixSeconds() - 250; + put_storage_policy(storage_policy_id, storage_policy); int64_t cooldown_timestamp = -1; size_t file_size = -1; 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 490bf0495b..88b0422354 100644 --- a/be/test/olap/test_data/header_without_inc_rs.txt +++ b/be/test/olap/test_data/header_without_inc_rs.txt @@ -106,6 +106,6 @@ "preferred_rowset_type": "BETA_ROWSET", "tablet_type": "TABLET_TYPE_DISK", "replica_id": 0, - "storage_policy": "", - "enable_unique_key_merge_on_write": false + "enable_unique_key_merge_on_write": false, + "storage_policy_id": 0 } diff --git a/be/test/vec/olap/vertical_compaction_test.cpp b/be/test/vec/olap/vertical_compaction_test.cpp index 67df8dacd9..3818020491 100644 --- a/be/test/vec/olap/vertical_compaction_test.cpp +++ b/be/test/vec/olap/vertical_compaction_test.cpp @@ -301,7 +301,7 @@ protected: TabletMetaSharedPtr tablet_meta( new TabletMeta(2, 2, 2, 2, 2, 2, t_tablet_schema, 2, col_ordinal_to_unique_id, UniqueId(1, 2), TTabletType::TABLET_TYPE_DISK, - TCompressionType::LZ4F, "", enable_unique_key_merge_on_write)); + TCompressionType::LZ4F, 0, enable_unique_key_merge_on_write)); TabletSharedPtr tablet(new Tablet(tablet_meta, nullptr)); tablet->init(); 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 e47021f0cf..5bb3323d82 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 @@ -475,7 +475,7 @@ public class Alter { // 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( + ((SchemaChangeHandler) schemaChangeHandler).updatePartitionsProperties( db, tableName, partitionNames, properties); OlapTable olapTable = (OlapTable) table; olapTable.writeLockOrDdlException(); @@ -489,7 +489,7 @@ public class Alter { // 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); + ((SchemaChangeHandler) schemaChangeHandler).updateTableProperties(db, tableName, properties); } else { throw new DdlException("Invalid alter operation: " + alterClause.getOpType()); } 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 8429d9e975..a330923582 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 @@ -74,6 +74,8 @@ import org.apache.doris.common.util.Util; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.persist.RemoveAlterJobV2OperationLog; import org.apache.doris.persist.TableAddOrDropColumnsInfo; +import org.apache.doris.policy.Policy; +import org.apache.doris.policy.PolicyTypeEnum; import org.apache.doris.qe.ConnectContext; import org.apache.doris.task.AgentBatchTask; import org.apache.doris.task.AgentTaskExecutor; @@ -102,6 +104,7 @@ import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -1852,10 +1855,27 @@ public class SchemaChangeHandler extends AlterHandler { LOG.info("send clear alter task for table {}, number: {}", olapTable.getName(), batchTask.getTaskNum()); } + static long storagePolicyNameToId(String storagePolicy) throws DdlException { + if (storagePolicy == null) { + return -1; // don't update storage policy + } + // if storagePolicy is "", means to reset the storage policy of this table + if (storagePolicy.isEmpty()) { + return 0; // reset storage policy + } else { + Optional policy = Env.getCurrentEnv().getPolicyMgr() + .findPolicy(storagePolicy, PolicyTypeEnum.STORAGE); + if (!policy.isPresent()) { + throw new DdlException("StoragePolicy[" + storagePolicy + "] not exist"); + } + return policy.get().getId(); + } + } + /** - * Update all partitions' in-memory property of table + * Update all partitions' properties of table */ - public void updateTableInMemoryMeta(Database db, String tableName, Map properties) + public void updateTableProperties(Database db, String tableName, Map properties) throws UserException { List partitions = Lists.newArrayList(); OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, Table.TableType.OLAP); @@ -1866,43 +1886,61 @@ public class SchemaChangeHandler extends AlterHandler { olapTable.readUnlock(); } - boolean isInMemory = Boolean.parseBoolean(properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY)); - String storagePolicy = properties.getOrDefault(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY, ""); - if (isInMemory == olapTable.isInMemory() && storagePolicy.equalsIgnoreCase("")) { - LOG.info("isInMemory == olapTable.isInMemory() and storagePolicy empty" - + isInMemory + " " + olapTable.isInMemory() + " " + storagePolicy); + String inMemory = properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY); + int isInMemory = -1; // < 0 means don't update inMemory properties + if (inMemory != null) { + isInMemory = Boolean.parseBoolean(inMemory) ? 1 : 0; + if ((isInMemory > 0) == olapTable.isInMemory()) { + // already up-to-date + isInMemory = -1; + } + } + String storagePolicy = properties.get(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY); + long storagePolicyId = storagePolicyNameToId(storagePolicy); + + if (isInMemory < 0 && storagePolicyId < 0) { + LOG.info("Properties already up-to-date"); return; } for (Partition partition : partitions) { - updatePartitionInMemoryMeta(db, olapTable.getName(), partition.getName(), storagePolicy, isInMemory); + updatePartitionProperties(db, olapTable.getName(), partition.getName(), storagePolicyId, isInMemory); } olapTable.writeLockOrDdlException(); try { - Env.getCurrentEnv().modifyTableInMemoryMeta(db, olapTable, properties); + Env.getCurrentEnv().modifyTableProperties(db, olapTable, properties); } finally { olapTable.writeUnlock(); } } /** - * Update some specified partitions' in-memory property of table + * Update some specified partitions' properties of table */ - public void updatePartitionsInMemoryMeta(Database db, String tableName, List partitionNames, + public void updatePartitionsProperties(Database db, String tableName, List partitionNames, Map properties) throws DdlException, MetaNotFoundException { OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, Table.TableType.OLAP); - boolean isInMemory = Boolean.parseBoolean(properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY)); - String storagePolicy = properties.getOrDefault(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY, ""); - if (isInMemory == olapTable.isInMemory() && storagePolicy.equalsIgnoreCase("")) { - LOG.info("isInMemory == olapTable.isInMemory() and storagePolicy empty" - + isInMemory + " " + olapTable.isInMemory() + " " + storagePolicy); + String inMemory = properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY); + int isInMemory = -1; // < 0 means don't update inMemory properties + if (inMemory != null) { + isInMemory = Boolean.parseBoolean(inMemory) ? 1 : 0; + if ((isInMemory > 0) == olapTable.isInMemory()) { + // already up-to-date + isInMemory = -1; + } + } + String storagePolicy = properties.get(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY); + long storagePolicyId = storagePolicyNameToId(storagePolicy); + + if (isInMemory < 0 && storagePolicyId < 0) { + LOG.info("Properties already up-to-date"); return; } for (String partitionName : partitionNames) { try { - updatePartitionInMemoryMeta(db, olapTable.getName(), partitionName, storagePolicy, isInMemory); + updatePartitionProperties(db, olapTable.getName(), partitionName, storagePolicyId, isInMemory); } catch (Exception e) { String errMsg = "Failed to update partition[" + partitionName + "]'s 'in_memory' property. " + "The reason is [" + e.getMessage() + "]"; @@ -1912,15 +1950,11 @@ public class SchemaChangeHandler extends AlterHandler { } /** - * Update one specified partition's in-memory property by partition name of table - * This operation may return partial successfully, with a exception to inform user to retry + * Update one specified partition's properties by partition name of table + * This operation may return partial successfully, with an exception to inform user to retry */ - public void updatePartitionInMemoryMeta(Database db, - String tableName, - String partitionName, - String storagePolicy, - boolean isInMemory) throws UserException { - + public void updatePartitionProperties(Database db, String tableName, String partitionName, long storagePolicyId, + int isInMemory) throws UserException { // be id -> Map>> beIdToTabletIdWithHash = Maps.newHashMap(); OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, Table.TableType.OLAP); @@ -1952,7 +1986,7 @@ public class SchemaChangeHandler extends AlterHandler { for (Map.Entry>> kv : beIdToTabletIdWithHash.entrySet()) { countDownLatch.addMark(kv.getKey(), kv.getValue()); UpdateTabletMetaInfoTask task = new UpdateTabletMetaInfoTask(kv.getKey(), kv.getValue(), - isInMemory, storagePolicy, countDownLatch); + isInMemory, storagePolicyId, countDownLatch); batchTask.addTask(task); } if (!FeConstants.runningUnitTest) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index 0a238e9dbd..c37e810d5d 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -4226,7 +4226,7 @@ public class Env { } // The caller need to hold the table write lock - public void modifyTableInMemoryMeta(Database db, OlapTable table, Map properties) { + public void modifyTableProperties(Database db, OlapTable table, Map properties) { Preconditions.checkArgument(table.isWriteLockHeldByCurrentThread()); TableProperty tableProperty = table.getTableProperty(); if (tableProperty == null) { 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 4abfadcba2..cbd942b2ac 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 @@ -28,13 +28,6 @@ import org.apache.doris.common.proc.BaseProcResult; import org.apache.doris.datasource.CatalogIf; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; -import org.apache.doris.policy.Policy; -import org.apache.doris.policy.PolicyTypeEnum; -import org.apache.doris.policy.StoragePolicy; -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 com.google.common.base.Strings; import com.google.common.collect.Maps; @@ -45,10 +38,9 @@ import org.apache.logging.log4j.Logger; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Optional; +import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; public abstract class Resource implements Writable, GsonPostProcessable { @@ -92,6 +84,28 @@ public abstract class Resource implements Writable, GsonPostProcessable { protected ResourceType type; @SerializedName(value = "references") protected Map references = Maps.newHashMap(); + @SerializedName(value = "id") + protected long id = -1; + @SerializedName(value = "version") + protected long version = -1; + + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); + + public void writeLock() { + lock.writeLock().lock(); + } + + public void writeUnlock() { + lock.writeLock().unlock(); + } + + public void readLock() { + lock.readLock().lock(); + } + + public void readUnlock() { + lock.readLock().unlock(); + } public Resource() { } @@ -103,10 +117,20 @@ public abstract class Resource implements Writable, GsonPostProcessable { public static Resource fromStmt(CreateResourceStmt stmt) throws DdlException { Resource resource = getResourceInstance(stmt.getResourceType(), stmt.getResourceName()); + resource.id = Env.getCurrentEnv().getNextId(); + resource.version = 0; resource.setProperties(stmt.getProperties()); return resource; } + public long getId() { + return this.id; + } + + public long getVersion() { + return this.version; + } + public synchronized boolean removeReference(String referenceName, ReferenceType type) { String fullName = referenceName + REFERENCE_SPLIT + type.name(); if (references.remove(fullName) != null) { @@ -252,48 +276,7 @@ public abstract class Resource implements Writable, GsonPostProcessable { private void notifyUpdate() { references.entrySet().stream().collect(Collectors.groupingBy(Entry::getValue)).forEach((type, refs) -> { - if (type == ReferenceType.POLICY) { - SystemInfoService systemInfoService = Env.getCurrentSystemInfo(); - AgentBatchTask batchTask = new AgentBatchTask(); - - Map copiedProperties = getCopiedProperties(); - - for (Long beId : systemInfoService.getBackendIds(true)) { - for (Map.Entry ref : refs) { - String policyName = ref.getKey().split(REFERENCE_SPLIT)[0]; - List policiesByType = Env.getCurrentEnv().getPolicyMgr() - .getCopiedPoliciesByType(PolicyTypeEnum.STORAGE); - Optional findPolicy = policiesByType.stream() - .filter(p -> p.getType() == PolicyTypeEnum.STORAGE - && policyName.equals(p.getPolicyName())) - .findAny(); - LOG.info("find policy in {} ", policiesByType); - if (!findPolicy.isPresent()) { - return; - } - // add policy's coolDown ttl、coolDown data、policy name to map - Map tmpMap = Maps.newHashMap(copiedProperties); - StoragePolicy used = (StoragePolicy) findPolicy.get(); - tmpMap.put(StoragePolicy.COOLDOWN_DATETIME, - String.valueOf(used.getCooldownTimestampMs())); - - final String[] cooldownTtl = {"-1"}; - Optional.ofNullable(used.getCooldownTtl()) - .ifPresent(date -> cooldownTtl[0] = used.getCooldownTtl()); - tmpMap.put(StoragePolicy.COOLDOWN_TTL, cooldownTtl[0]); - - tmpMap.put(StoragePolicy.MD5_CHECKSUM, used.getMd5Checksum()); - - NotifyUpdateStoragePolicyTask modifyS3ResourcePropertiesTask = - new NotifyUpdateStoragePolicyTask(beId, used.getPolicyName(), tmpMap); - LOG.info("notify be: {}, policy name: {}, " - + "properties: {} to modify S3 resource batch task.", - beId, used.getPolicyName(), tmpMap); - batchTask.addTask(modifyS3ResourcePropertiesTask); - } - } - AgentTaskExecutor.submit(batchTask); - } else if (type == ReferenceType.CATALOG) { + if (type == ReferenceType.CATALOG) { for (Map.Entry ref : refs) { String catalogName = ref.getKey().split(REFERENCE_SPLIT)[0]; CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogName); 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 d9a79b8616..70db926073 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 @@ -47,7 +47,9 @@ import java.io.DataOutput; import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; +import java.util.stream.Collectors; /** * Resource manager is responsible for managing external resources used by Doris. @@ -167,6 +169,11 @@ public class ResourceMgr implements Writable { return nameToResource.size(); } + public List getResource(ResourceType type) { + return nameToResource.values().stream().filter(resource -> resource.getType() == type) + .collect(Collectors.toList()); + } + 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 dd7355621a..e1ace54e1b 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 @@ -91,13 +91,14 @@ public class S3Resource extends Resource { @SerializedName(value = "properties") private Map properties; - public S3Resource(String name) { - this(name, Maps.newHashMap()); - } + // for Gson fromJson + // TODO(plat1ko): other Resource subclass also MUST define default ctor, otherwise when reloading object from json + // some not serialized field (i.e. `lock`) will be `null`. + public S3Resource() {} - public S3Resource(String name, Map properties) { + public S3Resource(String name) { super(name, ResourceType.S3); - this.properties = properties; + properties = Maps.newHashMap(); } public String getProperty(String propertyKey) { @@ -186,9 +187,12 @@ public class S3Resource extends Resource { } } // modify properties + writeLock(); for (Map.Entry kv : properties.entrySet()) { replaceIfEffectiveValue(this.properties, kv.getKey(), kv.getValue()); } + ++version; + writeUnlock(); super.modifyProperties(properties); } @@ -200,6 +204,9 @@ public class S3Resource extends Resource { @Override protected void getProcNodeData(BaseProcResult result) { String lowerCaseType = type.name().toLowerCase(); + result.addRow(Lists.newArrayList(name, lowerCaseType, "id", String.valueOf(id))); + readLock(); + result.addRow(Lists.newArrayList(name, lowerCaseType, "version", String.valueOf(version))); for (Map.Entry entry : properties.entrySet()) { // it's dangerous to show password in show odbc resource, // so we use empty string to replace the real password @@ -209,6 +216,7 @@ public class S3Resource extends Resource { result.addRow(Lists.newArrayList(name, lowerCaseType, entry.getKey(), entry.getValue())); } } + readUnlock(); } public static Map getS3HadoopProperties(Map properties) { 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 f3090fa212..e14f87d375 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 @@ -28,6 +28,8 @@ import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.Replica.ReplicaState; import org.apache.doris.catalog.ReplicaAllocation; +import org.apache.doris.catalog.Resource; +import org.apache.doris.catalog.Resource.ResourceType; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Tablet; import org.apache.doris.catalog.Tablet.TabletStatus; @@ -45,6 +47,9 @@ import org.apache.doris.metric.MetricRepo; import org.apache.doris.persist.BackendReplicasInfo; import org.apache.doris.persist.BackendTabletsInfo; import org.apache.doris.persist.ReplicaPersistInfo; +import org.apache.doris.policy.Policy; +import org.apache.doris.policy.PolicyTypeEnum; +import org.apache.doris.policy.StoragePolicy; import org.apache.doris.system.Backend; import org.apache.doris.system.Backend.BackendStatus; import org.apache.doris.system.SystemInfoService; @@ -57,6 +62,7 @@ import org.apache.doris.task.CreateReplicaTask; import org.apache.doris.task.DropReplicaTask; import org.apache.doris.task.MasterTask; import org.apache.doris.task.PublishVersionTask; +import org.apache.doris.task.PushStoragePolicyTask; import org.apache.doris.task.StorageMediaMigrationTask; import org.apache.doris.task.UpdateTabletMetaInfoTask; import org.apache.doris.thrift.TBackend; @@ -67,6 +73,8 @@ import org.apache.doris.thrift.TReportRequest; import org.apache.doris.thrift.TStatus; import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TStorageMedium; +import org.apache.doris.thrift.TStoragePolicy; +import org.apache.doris.thrift.TStorageResource; import org.apache.doris.thrift.TStorageType; import org.apache.doris.thrift.TTablet; import org.apache.doris.thrift.TTabletInfo; @@ -84,11 +92,13 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.thrift.TException; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.BlockingQueue; +import java.util.stream.Collectors; public class ReportHandler extends Daemon { private static final Logger LOG = LogManager.getLogger(ReportHandler.class); @@ -164,7 +174,8 @@ public class ReportHandler extends Daemon { backend.setTabletMaxCompactionScore(request.getTabletMaxCompactionScore()); } - ReportTask reportTask = new ReportTask(beId, tasks, disks, tablets, reportVersion); + ReportTask reportTask = new ReportTask(beId, tasks, disks, tablets, reportVersion, + request.getStoragePolicy(), request.getResource()); try { putToQueue(reportTask); } catch (Exception e) { @@ -212,14 +223,20 @@ public class ReportHandler extends Daemon { private Map tablets; private long reportVersion; + private List storagePolicies; + private List storageResources; + public ReportTask(long beId, Map> tasks, Map disks, - Map tablets, long reportVersion) { + Map tablets, long reportVersion, + List storagePolicies, List storageResources) { this.beId = beId; this.tasks = tasks; this.disks = disks; this.tablets = tablets; this.reportVersion = reportVersion; + this.storagePolicies = storagePolicies; + this.storageResources = storageResources; } @Override @@ -230,6 +247,10 @@ public class ReportHandler extends Daemon { if (disks != null) { ReportHandler.diskReport(beId, disks); } + if (Config.enable_storage_policy && storagePolicies != null && storageResources != null) { + storagePolicyReport(beId, storagePolicies, storageResources); + } + if (tablets != null) { long backendReportVersion = Env.getCurrentSystemInfo().getBackendReportVersion(beId); if (reportVersion < backendReportVersion) { @@ -242,6 +263,122 @@ public class ReportHandler extends Daemon { } } + private static void handlePushStoragePolicy(long backendId, List policyToPush, + List resourceToPush, List policyToDrop) { + AgentBatchTask batchTask = new AgentBatchTask(); + PushStoragePolicyTask pushStoragePolicyTask = new PushStoragePolicyTask(backendId, policyToPush, + resourceToPush, policyToDrop); + batchTask.addTask(pushStoragePolicyTask); + AgentTaskExecutor.submit(batchTask); + } + + private static void storagePolicyReport(long backendId, + List storagePoliciesInBe, + List storageResourcesInBe) { + LOG.info("backend[{}] reports policies {}, report resources: {}", + backendId, storagePoliciesInBe, storageResourcesInBe); + // do the diff. find out (intersection) / (be - meta) / (meta - be) + List policiesInFe = Env.getCurrentEnv().getPolicyMgr().getCopiedPoliciesByType(PolicyTypeEnum.STORAGE); + List resourcesInFe = Env.getCurrentEnv().getResourceMgr().getResource(ResourceType.S3); + + List resourceToPush = new ArrayList<>(); + List policyToPush = new ArrayList<>(); + List policyToDrop = new ArrayList<>(); + + diffPolicy(storagePoliciesInBe, policiesInFe, policyToPush, policyToDrop); + diffResource(storageResourcesInBe, resourcesInFe, resourceToPush); + + if (policyToPush.isEmpty() && resourceToPush.isEmpty() && policyToDrop.isEmpty()) { + return; + } + LOG.info("after diff policy, policyToPush {}, policyToDrop {}, and resourceToPush {}", + policyToPush.stream() + .map(p -> "StoragePolicy(name=" + p.getPolicyName() + " id=" + p.getId() + " version=" + + p.getVersion()).collect(Collectors.toList()), + policyToDrop, resourceToPush.stream() + .map(r -> "Resource(name=" + r.getName() + " id=" + r.getId() + " version=" + + r.getVersion()).collect(Collectors.toList())); + // send push rpc + handlePushStoragePolicy(backendId, policyToPush, resourceToPush, policyToDrop); + } + + private static void diffPolicy(List storagePoliciesInBe, List policiesInFe, + List policyToPush, List policyToDrop) { + // fe - be + for (Policy policy : policiesInFe) { + if (policy.getId() <= 0 || ((StoragePolicy) policy).getStorageResource() == null) { + continue; // ignore policy with invalid id or storage resource + } + boolean beHasIt = false; + for (TStoragePolicy tStoragePolicy : storagePoliciesInBe) { + if (policy.getId() == tStoragePolicy.getId()) { + beHasIt = true; + // find id eq + if (policy.getVersion() == tStoragePolicy.getVersion()) { + // find version eq + } else if (policy.getVersion() > tStoragePolicy.getVersion()) { + // need to add + policyToPush.add(policy); + } else { + // impossible + LOG.warn("fe policy version {} litter than be {}, impossible", + policy.getVersion(), tStoragePolicy.getVersion()); + } + break; + } + } + if (!beHasIt) { + policyToPush.add(policy); + } + } + + // be - fe + for (TStoragePolicy tStoragePolicy : storagePoliciesInBe) { + boolean feHasIt = false; + for (Policy policy : policiesInFe) { + if (policy.getId() == tStoragePolicy.getId()) { + feHasIt = true; + // find id eq + break; + } + } + if (!feHasIt) { + policyToDrop.add(tStoragePolicy.getId()); + } + } + } + + private static void diffResource(List storageResourcesInBe, List resourcesInFe, + List resourceToPush) { + // fe - be + for (Resource resource : resourcesInFe) { + if (resource.getId() <= 0) { + continue; // ignore resource with invalid id + } + boolean beHasIt = false; + for (TStorageResource tStorageResource : storageResourcesInBe) { + if (resource.getId() == tStorageResource.getId()) { + beHasIt = true; + // find id eq + if (resource.getVersion() == tStorageResource.getVersion()) { + // find version eq + } else if (resource.getVersion() > tStorageResource.getVersion()) { + // need to add + resourceToPush.add(resource); + } else { + // impossible + LOG.warn("fe resource version {} litter than be {}, impossible", + resource.getVersion(), tStorageResource.getVersion()); + } + break; + } + } + if (!beHasIt) { + resourceToPush.add(resource); + } + } + } + private static void tabletReport(long backendId, Map backendTablets, long backendReportVersion) { long start = System.currentTimeMillis(); LOG.info("backend[{}] reports {} tablet(s). report version: {}", diff --git a/fe/fe-core/src/main/java/org/apache/doris/policy/Policy.java b/fe/fe-core/src/main/java/org/apache/doris/policy/Policy.java index 388bb4d939..bfedaa88cf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/policy/Policy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/policy/Policy.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.persist.gson.GsonPostProcessable; @@ -39,6 +40,8 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.List; +import java.util.Map; +import java.util.concurrent.locks.ReentrantReadWriteLock; /** * Base class for Policy. @@ -48,8 +51,8 @@ public abstract class Policy implements Writable, GsonPostProcessable { private static final Logger LOG = LogManager.getLogger(Policy.class); - @SerializedName(value = "policyId") - protected long policyId = -1; + @SerializedName(value = "id") + protected long id = -1; @SerializedName(value = "type") protected PolicyTypeEnum type = null; @@ -57,6 +60,30 @@ public abstract class Policy implements Writable, GsonPostProcessable { @SerializedName(value = "policyName") protected String policyName = null; + @SerializedName(value = "version") + protected long version = -1; + + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); + + public void writeLock() { + lock.writeLock().lock(); + } + + public void writeUnlock() { + lock.writeLock().unlock(); + } + + public void readLock() { + lock.readLock().lock(); + } + + public void readUnlock() { + lock.readLock().unlock(); + } + + // just for subclass lombok @Data + public Policy() {} + public Policy(PolicyTypeEnum type) { this.type = type; } @@ -67,10 +94,11 @@ public abstract class Policy implements Writable, GsonPostProcessable { * @param type policy type * @param policyName policy name */ - public Policy(long policyId, final PolicyTypeEnum type, final String policyName) { - this.policyId = policyId; + public Policy(long id, final PolicyTypeEnum type, final String policyName) { + this.id = id; this.type = type; this.policyName = policyName; + this.version = 0; } /** @@ -98,6 +126,9 @@ public abstract class Policy implements Writable, GsonPostProcessable { } } + public void modifyProperties(Map properties) throws DdlException, AnalysisException { + } + /** * Use for SHOW POLICY. **/ diff --git a/fe/fe-core/src/main/java/org/apache/doris/policy/RowPolicy.java b/fe/fe-core/src/main/java/org/apache/doris/policy/RowPolicy.java index e5de712ef4..3ef7de1d18 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/policy/RowPolicy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/policy/RowPolicy.java @@ -142,7 +142,7 @@ public class RowPolicy extends Policy { @Override public RowPolicy clone() { - return new RowPolicy(this.policyId, this.policyName, this.dbId, this.user, this.originStmt, this.tableId, + return new RowPolicy(this.id, this.policyName, this.dbId, this.user, this.originStmt, this.tableId, this.filterType, this.wherePredicate); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/policy/StoragePolicy.java b/fe/fe-core/src/main/java/org/apache/doris/policy/StoragePolicy.java index f6c1662bef..48adb56a64 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/policy/StoragePolicy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/policy/StoragePolicy.java @@ -25,29 +25,20 @@ import org.apache.doris.catalog.S3Resource; import org.apache.doris.catalog.ScalarType; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; -import org.apache.doris.common.io.Text; import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ShowResultSetMetaData; -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 com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.gson.annotations.SerializedName; import lombok.Data; -import org.apache.commons.codec.digest.DigestUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.io.DataInput; import java.io.IOException; import java.text.ParseException; import java.text.SimpleDateFormat; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Optional; @@ -76,11 +67,12 @@ public class StoragePolicy extends Policy { public static final ShowResultSetMetaData STORAGE_META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("PolicyName", ScalarType.createVarchar(100))) + .addColumn(new Column("Id", ScalarType.createVarchar(20))) + .addColumn(new Column("Version", ScalarType.createVarchar(20))) .addColumn(new Column("Type", ScalarType.createVarchar(20))) .addColumn(new Column("StorageResource", ScalarType.createVarchar(20))) .addColumn(new Column("CooldownDatetime", ScalarType.createVarchar(20))) .addColumn(new Column("CooldownTtl", ScalarType.createVarchar(20))) - .addColumn(new Column("properties", ScalarType.createVarchar(65535))) .build(); private static final Logger LOG = LogManager.getLogger(StoragePolicy.class); @@ -100,10 +92,6 @@ public class StoragePolicy extends Policy { private static final long ONE_DAY_MS = 24 * ONE_HOUR_MS; private static final long ONE_WEEK_MS = 7 * ONE_DAY_MS; - public static final String MD5_CHECKSUM = "md5_checksum"; - @SerializedName(value = "md5Checksum") - private String md5Checksum = null; - @SerializedName(value = "storageResource") private String storageResource = null; @@ -118,6 +106,7 @@ public class StoragePolicy extends Policy { private Map props; + // for Gson fromJson public StoragePolicy() { super(PolicyTypeEnum.STORAGE); } @@ -201,7 +190,6 @@ public class StoragePolicy extends Policy { if (!addResourceReference() && !ifNotExists) { throw new AnalysisException("this policy has been added to s3 resource once, policy has been created."); } - this.md5Checksum = calcPropertiesMd5(); } private static Resource checkIsS3ResourceAndExist(final String storageResource) throws AnalysisException { @@ -229,25 +217,18 @@ public class StoragePolicy extends Policy { * Use for SHOW POLICY. **/ public List getShowInfo() throws AnalysisException { - final String[] props = {""}; - if (Env.getCurrentEnv().getResourceMgr().containsResource(this.storageResource)) { - props[0] = Env.getCurrentEnv().getResourceMgr().getResource(this.storageResource).toString(); + readLock(); + try { + if (cooldownTimestampMs == -1) { + return Lists.newArrayList(this.policyName, String.valueOf(this.id), String.valueOf(this.version), + this.type.name(), this.storageResource, "-1", this.cooldownTtl); + } + return Lists.newArrayList(this.policyName, String.valueOf(this.id), String.valueOf(this.version), + this.type.name(), this.storageResource, TimeUtils.longToTimeString(this.cooldownTimestampMs), + this.cooldownTtl); + } finally { + readUnlock(); } - if (!props[0].equals("")) { - // s3_secret_key => ****** - S3Resource s3Resource = GsonUtils.GSON.fromJson(props[0], S3Resource.class); - Optional.ofNullable(s3Resource).ifPresent(s3 -> { - Map copyMap = s3.getCopiedProperties(); - copyMap.put(S3Resource.S3_SECRET_KEY, "******"); - props[0] = GsonUtils.GSON.toJson(copyMap); - }); - } - if (cooldownTimestampMs == -1) { - return Lists.newArrayList(this.policyName, this.type.name(), this.storageResource, "-1", this.cooldownTtl, - props[0]); - } - return Lists.newArrayList(this.policyName, this.type.name(), this.storageResource, - TimeUtils.longToTimeString(this.cooldownTimestampMs), this.cooldownTtl, props[0]); } @Override @@ -255,7 +236,7 @@ public class StoragePolicy extends Policy { @Override public StoragePolicy clone() { - return new StoragePolicy(this.policyId, this.policyName, this.storageResource, this.cooldownTimestampMs, + return new StoragePolicy(this.id, this.policyName, this.storageResource, this.cooldownTimestampMs, this.cooldownTtl, this.cooldownTtlMs); } @@ -324,25 +305,6 @@ public class StoragePolicy extends Policy { return cooldownTtlMs; } - // be use this md5Sum to determine whether storage policy has been changed. - // if md5Sum not eq previous value, be change its storage policy. - private String calcPropertiesMd5() { - List calcKey = Arrays.asList(COOLDOWN_DATETIME, COOLDOWN_TTL, S3Resource.S3_MAX_CONNECTIONS, - S3Resource.S3_REQUEST_TIMEOUT_MS, S3Resource.S3_CONNECTION_TIMEOUT_MS, - S3Resource.S3_ACCESS_KEY, S3Resource.S3_SECRET_KEY); - Map copiedStoragePolicyProperties = Env.getCurrentEnv().getResourceMgr() - .getResource(this.storageResource).getCopiedProperties(); - - copiedStoragePolicyProperties.put(COOLDOWN_DATETIME, String.valueOf(this.cooldownTimestampMs)); - copiedStoragePolicyProperties.put(COOLDOWN_TTL, this.cooldownTtl); - - LOG.info("calcPropertiesMd5 map {}", copiedStoragePolicyProperties); - - return DigestUtils.md5Hex(calcKey.stream() - .map(iter -> "(" + iter + ":" + copiedStoragePolicyProperties.get(iter) + ")") - .reduce("", String::concat)); - } - public void checkProperties(Map properties) throws AnalysisException { // check properties Map copiedProperties = Maps.newHashMap(properties); @@ -357,58 +319,45 @@ public class StoragePolicy extends Policy { } public void modifyProperties(Map properties) throws DdlException, AnalysisException { - Optional.ofNullable(properties.get(COOLDOWN_TTL)).ifPresent(this::setCooldownTtl); - Optional.ofNullable(properties.get(COOLDOWN_DATETIME)).ifPresent(date -> { + this.toString(); + // some check + long cooldownTtlMs = -1; + String cooldownTtl = properties.get(COOLDOWN_TTL); + if (cooldownTtl != null) { + cooldownTtlMs = getMsByCooldownTtl(cooldownTtl); + } + long cooldownTimestampMs = -1; + String cooldownDatetime = properties.get(COOLDOWN_DATETIME); + if (cooldownDatetime != null) { SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); try { - this.cooldownTimestampMs = df.parse(properties.get(COOLDOWN_DATETIME)).getTime(); + cooldownTimestampMs = df.parse(cooldownDatetime).getTime(); } catch (ParseException e) { throw new RuntimeException(e); } - }); - - if (policyName.equalsIgnoreCase(DEFAULT_STORAGE_POLICY_NAME) && storageResource == null) { - // here first time set S3 resource to default storage policy. - String alterStorageResource = Optional.ofNullable(properties.get(STORAGE_RESOURCE)).orElseThrow( - () -> new DdlException("first time set default storage policy, but not give storageResource")); - // check alterStorageResource resource exist. - checkIsS3ResourceAndExist(alterStorageResource); - storageResource = alterStorageResource; } - - md5Checksum = calcPropertiesMd5(); - notifyUpdate(); - } - - private void notifyUpdate() { - SystemInfoService systemInfoService = Env.getCurrentSystemInfo(); - AgentBatchTask batchTask = new AgentBatchTask(); - - for (Long beId : systemInfoService.getBackendIds(true)) { - Map copiedProperties = Env.getCurrentEnv().getResourceMgr().getResource(storageResource) - .getCopiedProperties(); - - Map tmpMap = Maps.newHashMap(copiedProperties); - - tmpMap.put(COOLDOWN_DATETIME, String.valueOf(this.cooldownTimestampMs)); - - Optional.ofNullable(this.getCooldownTtl()).ifPresent(date -> { - tmpMap.put(COOLDOWN_TTL, this.getCooldownTtl()); - }); - tmpMap.put(MD5_CHECKSUM, this.getMd5Checksum()); - NotifyUpdateStoragePolicyTask notifyUpdateStoragePolicyTask = new NotifyUpdateStoragePolicyTask(beId, - getPolicyName(), tmpMap); - batchTask.addTask(notifyUpdateStoragePolicyTask); - LOG.info("update policy info to be: {}, policy name: {}, " - + "properties: {} to modify S3 resource batch task.", beId, getPolicyName(), tmpMap); + String storageResource = properties.get(STORAGE_RESOURCE); + if (storageResource != null) { + checkIsS3ResourceAndExist(storageResource); } - - AgentTaskExecutor.submit(batchTask); - } - - public static StoragePolicy read(DataInput in) throws IOException { - String json = Text.readString(in); - return GsonUtils.GSON.fromJson(json, StoragePolicy.class); + if (this.policyName.equalsIgnoreCase(DEFAULT_STORAGE_POLICY_NAME) && this.storageResource == null + && storageResource == null) { + throw new DdlException("first time set default storage policy, but not give storageResource"); + } + // modify properties + writeLock(); + if (cooldownTtlMs > 0) { + this.cooldownTtl = cooldownTtl; + this.cooldownTtlMs = cooldownTtlMs; + } + if (cooldownTimestampMs > 0) { + this.cooldownTimestampMs = cooldownTimestampMs; + } + if (storageResource != null) { + this.storageResource = storageResource; + } + ++version; + writeUnlock(); } public boolean addResourceReference() { 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 0d90cc9825..19dde33b65 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 @@ -26,7 +26,6 @@ import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.HMSResource; import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.S3Resource; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.TableIf.TableType; @@ -54,9 +53,6 @@ import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.master.MasterImpl; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.planner.StreamLoadPlanner; -import org.apache.doris.policy.Policy; -import org.apache.doris.policy.PolicyTypeEnum; -import org.apache.doris.policy.StoragePolicy; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ConnectProcessor; import org.apache.doris.qe.QeProcessorImpl; @@ -84,8 +80,6 @@ import org.apache.doris.thrift.TFrontendPingFrontendResult; import org.apache.doris.thrift.TFrontendPingFrontendStatusCode; import org.apache.doris.thrift.TGetDbsParams; import org.apache.doris.thrift.TGetDbsResult; -import org.apache.doris.thrift.TGetStoragePolicy; -import org.apache.doris.thrift.TGetStoragePolicyResult; import org.apache.doris.thrift.TGetTablesParams; import org.apache.doris.thrift.TGetTablesResult; import org.apache.doris.thrift.TIcebergMetadataType; @@ -111,7 +105,6 @@ import org.apache.doris.thrift.TReportExecStatusParams; import org.apache.doris.thrift.TReportExecStatusResult; import org.apache.doris.thrift.TReportRequest; import org.apache.doris.thrift.TRow; -import org.apache.doris.thrift.TS3StorageParam; import org.apache.doris.thrift.TShowVariableRequest; import org.apache.doris.thrift.TShowVariableResult; import org.apache.doris.thrift.TSnapshotLoaderReportRequest; @@ -147,11 +140,9 @@ import org.jetbrains.annotations.NotNull; import java.time.Instant; import java.time.LocalDateTime; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -1233,61 +1224,6 @@ public class FrontendServiceImpl implements FrontendService.Iface { return result; } - @Override - public TGetStoragePolicyResult refreshStoragePolicy() throws TException { - TGetStoragePolicyResult result = new TGetStoragePolicyResult(); - TStatus status = new TStatus(TStatusCode.OK); - result.setStatus(status); - - List policyList = Env.getCurrentEnv().getPolicyMgr().getCopiedPoliciesByType(PolicyTypeEnum.STORAGE); - policyList.stream().filter(p -> p instanceof StoragePolicy).map(p -> (StoragePolicy) p).forEach( - iter -> { - // default policy not init. - if (iter.getStorageResource() == null) { - return; - } - TGetStoragePolicy rEntry = new TGetStoragePolicy(); - rEntry.setPolicyName(iter.getPolicyName()); - //java 8 not support ifPresentOrElse - final long[] ttlCoolDown = {-1}; - Optional.ofNullable(iter.getCooldownTtl()).ifPresent(ttl -> ttlCoolDown[0] = Integer.parseInt(ttl)); - rEntry.setCooldownTtl(ttlCoolDown[0]); - - //timestamp : ms -> s - rEntry.setCooldownDatetime( - iter.getCooldownTimestampMs() == -1 ? -1 : iter.getCooldownTimestampMs() / 1000); - - Optional.ofNullable(iter.getMd5Checksum()).ifPresent(rEntry::setMd5Checksum); - TS3StorageParam s3Info = new TS3StorageParam(); - Optional.ofNullable(iter.getStorageResource()).ifPresent(resource -> { - Map storagePolicyProperties = Env.getCurrentEnv().getResourceMgr() - .getResource(resource).getCopiedProperties(); - s3Info.setS3Endpoint(storagePolicyProperties.get(S3Resource.S3_ENDPOINT)); - s3Info.setS3Region(storagePolicyProperties.get(S3Resource.S3_REGION)); - s3Info.setRootPath(storagePolicyProperties.get(S3Resource.S3_ROOT_PATH)); - s3Info.setS3Ak(storagePolicyProperties.get(S3Resource.S3_ACCESS_KEY)); - s3Info.setS3Sk(storagePolicyProperties.get(S3Resource.S3_SECRET_KEY)); - s3Info.setBucket(storagePolicyProperties.get(S3Resource.S3_BUCKET)); - s3Info.setS3MaxConn( - Integer.parseInt(storagePolicyProperties.get(S3Resource.S3_MAX_CONNECTIONS))); - s3Info.setS3RequestTimeoutMs(Integer.parseInt( - storagePolicyProperties.get(S3Resource.S3_REQUEST_TIMEOUT_MS))); - s3Info.setS3ConnTimeoutMs(Integer.parseInt( - storagePolicyProperties.get(S3Resource.S3_CONNECTION_TIMEOUT_MS))); - }); - - rEntry.setS3StorageParam(s3Info); - result.addToResultEntrys(rEntry); - } - ); - if (!result.isSetResultEntrys()) { - result.setResultEntrys(new ArrayList<>()); - } - - LOG.debug("refresh storage policy request: {}", result); - return result; - } - @Override public TInitExternalCtlMetaResult initExternalCtlMeta(TInitExternalCtlMetaRequest request) throws TException { if (request.isSetCatalogId() && request.isSetDbId()) { 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 3c10b8d1a3..23db696a97 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 @@ -33,12 +33,12 @@ import org.apache.doris.thrift.TCompactionReq; import org.apache.doris.thrift.TCreateTabletReq; import org.apache.doris.thrift.TDownloadReq; import org.apache.doris.thrift.TDropTabletReq; -import org.apache.doris.thrift.TGetStoragePolicy; import org.apache.doris.thrift.TMoveDirReq; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TPublishVersionRequest; import org.apache.doris.thrift.TPushCooldownConfReq; import org.apache.doris.thrift.TPushReq; +import org.apache.doris.thrift.TPushStoragePolicyReq; import org.apache.doris.thrift.TReleaseSnapshotRequest; import org.apache.doris.thrift.TSnapshotRequest; import org.apache.doris.thrift.TStorageMediumMigrateReq; @@ -347,13 +347,13 @@ public class AgentBatchTask implements Runnable { tAgentTaskRequest.setCompactionReq(request); return tAgentTaskRequest; } - case NOTIFY_UPDATE_STORAGE_POLICY: { - NotifyUpdateStoragePolicyTask notifyUpdateStoragePolicyTask = (NotifyUpdateStoragePolicyTask) task; - TGetStoragePolicy request = notifyUpdateStoragePolicyTask.toThrift(); + case PUSH_STORAGE_POLICY: { + PushStoragePolicyTask pushStoragePolicyTask = (PushStoragePolicyTask) task; + TPushStoragePolicyReq request = pushStoragePolicyTask.toThrift(); if (LOG.isDebugEnabled()) { LOG.debug(request.toString()); } - tAgentTaskRequest.setUpdatePolicy(request); + tAgentTaskRequest.setPushStoragePolicyReq(request); return tAgentTaskRequest; } case PUSH_COOLDOWN_CONF: { 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 12277033d3..fcb139fd8f 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 @@ -20,10 +20,13 @@ package org.apache.doris.task; import org.apache.doris.alter.SchemaChangeHandler; import org.apache.doris.analysis.DataSortInfo; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.Index; import org.apache.doris.catalog.KeysType; import org.apache.doris.common.MarkedCountDownLatch; import org.apache.doris.common.Status; +import org.apache.doris.policy.Policy; +import org.apache.doris.policy.PolicyTypeEnum; import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TCompressionType; import org.apache.doris.thrift.TCreateTabletReq; @@ -42,6 +45,7 @@ import org.apache.logging.log4j.Logger; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.Set; public class CreateReplicaTask extends AgentTask { @@ -88,7 +92,7 @@ public class CreateReplicaTask extends AgentTask { private boolean isRecoverTask = false; private DataSortInfo dataSortInfo; - private String storagePolicy; + private long storagePolicyId = 0; // <= 0 means no storage policy private boolean enableUniqueKeyMergeOnWrite; @@ -134,7 +138,13 @@ public class CreateReplicaTask extends AgentTask { this.tabletType = tabletType; this.dataSortInfo = dataSortInfo; this.enableUniqueKeyMergeOnWrite = (keysType == KeysType.UNIQUE_KEYS && enableUniqueKeyMergeOnWrite); - this.storagePolicy = storagePolicy; + if (storagePolicy != null && !storagePolicy.isEmpty()) { + Optional policy = Env.getCurrentEnv().getPolicyMgr() + .findPolicy(storagePolicy, PolicyTypeEnum.STORAGE); + if (policy.isPresent()) { + this.storagePolicyId = policy.get().getId(); + } + } this.disableAutoCompaction = disableAutoCompaction; this.storeRowColumn = storeRowColumn; } @@ -242,7 +252,9 @@ public class CreateReplicaTask extends AgentTask { createTabletReq.setVersion(version); createTabletReq.setStorageMedium(storageMedium); - createTabletReq.setStoragePolicy(storagePolicy); + if (storagePolicyId > 0) { + createTabletReq.setStoragePolicyId(storagePolicyId); + } if (inRestoreMode) { createTabletReq.setInRestoreMode(true); } 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 deleted file mode 100644 index 87c9b06d66..0000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/task/NotifyUpdateStoragePolicyTask.java +++ /dev/null @@ -1,70 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.task; - -import org.apache.doris.catalog.S3Resource; -import org.apache.doris.policy.StoragePolicy; -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.util.Map; - -public class NotifyUpdateStoragePolicyTask extends AgentTask { - private static final Logger LOG = LogManager.getLogger(NotifyUpdateStoragePolicyTask.class); - private String policyName; - 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; - // cooldown_datetime in BE is in seconds - ret.setCooldownDatetime(Long.parseLong(properties.get(StoragePolicy.COOLDOWN_DATETIME)) / 1000); - ret.setCooldownTtl(Long.parseLong(properties.get(StoragePolicy.COOLDOWN_TTL))); - ret.s3_storage_param = new TS3StorageParam(); - ret.s3_storage_param.s3_max_conn = Integer.parseInt( - properties.getOrDefault(S3Resource.S3_MAX_CONNECTIONS, - S3Resource.DEFAULT_S3_MAX_CONNECTIONS)); - ret.s3_storage_param.s3_request_timeout_ms = Integer.parseInt( - properties.getOrDefault(S3Resource.S3_REQUEST_TIMEOUT_MS, - S3Resource.DEFAULT_S3_REQUEST_TIMEOUT_MS)); - ret.s3_storage_param.s3_conn_timeout_ms = Integer.parseInt( - properties.getOrDefault(S3Resource.S3_CONNECTION_TIMEOUT_MS, - S3Resource.DEFAULT_S3_CONNECTION_TIMEOUT_MS)); - ret.s3_storage_param.s3_endpoint = properties.get(S3Resource.S3_ENDPOINT); - ret.s3_storage_param.s3_region = properties.get(S3Resource.S3_REGION); - ret.s3_storage_param.root_path = properties.get(S3Resource.S3_ROOT_PATH); - ret.s3_storage_param.s3_ak = properties.get(S3Resource.S3_ACCESS_KEY); - ret.s3_storage_param.s3_sk = properties.get(S3Resource.S3_SECRET_KEY); - ret.s3_storage_param.bucket = properties.get(S3Resource.S3_BUCKET); - ret.md5_checksum = properties.get(StoragePolicy.MD5_CHECKSUM); - - LOG.info("TGetStoragePolicy toThrift : {}", ret); - return ret; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/PushStoragePolicyTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/PushStoragePolicyTask.java new file mode 100644 index 0000000000..42ed500d4b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/task/PushStoragePolicyTask.java @@ -0,0 +1,115 @@ +// 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.catalog.Env; +import org.apache.doris.catalog.Resource; +import org.apache.doris.catalog.Resource.ResourceType; +import org.apache.doris.catalog.S3Resource; +import org.apache.doris.policy.Policy; +import org.apache.doris.policy.StoragePolicy; +import org.apache.doris.thrift.TPushStoragePolicyReq; +import org.apache.doris.thrift.TS3StorageParam; +import org.apache.doris.thrift.TStoragePolicy; +import org.apache.doris.thrift.TStorageResource; +import org.apache.doris.thrift.TTaskType; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.List; + +public class PushStoragePolicyTask extends AgentTask { + private static final Logger LOG = LogManager.getLogger(PushStoragePolicyTask.class); + + private List storagePolicy; + private List resource; + private List droppedStoragePolicy; + + public PushStoragePolicyTask(long backendId, List storagePolicy, + List resource, List droppedStoragePolicy) { + super(null, backendId, TTaskType.PUSH_STORAGE_POLICY, -1, -1, -1, -1, -1, -1, -1); + this.storagePolicy = storagePolicy; + this.resource = resource; + this.droppedStoragePolicy = droppedStoragePolicy; + } + + public TPushStoragePolicyReq toThrift() { + TPushStoragePolicyReq ret = new TPushStoragePolicyReq(); + List tStoragePolicies = new ArrayList<>(); + storagePolicy.forEach(p -> { + TStoragePolicy item = new TStoragePolicy(); + p.readLock(); + try { + item.setId(p.getId()); + item.setName(p.getPolicyName()); + item.setVersion(p.getVersion()); + StoragePolicy storagePolicy = (StoragePolicy) p; + String resourceName = storagePolicy.getStorageResource(); + Resource resource = Env.getCurrentEnv().getResourceMgr().getResource(resourceName); + if (resource == null || resource.getType() != ResourceType.S3) { + LOG.warn("can't find s3 resource by name {}", resourceName); + return; + } + item.setResourceId(resource.getId()); + long coolDownDatetime = storagePolicy.getCooldownTimestampMs() / 1000; + item.setCooldownDatetime(coolDownDatetime); + long coolDownTtl = storagePolicy.getCooldownTtlMs() / 1000; + item.setCooldownTtl(coolDownTtl); + } finally { + p.readUnlock(); + } + tStoragePolicies.add(item); + }); + ret.setStoragePolicy(tStoragePolicies); + + List tStorageResources = new ArrayList<>(); + resource.forEach(r -> { + TStorageResource item = new TStorageResource(); + r.readLock(); + item.setId(r.getId()); + item.setName(r.getName()); + item.setVersion(r.getVersion()); + TS3StorageParam s3Info = new TS3StorageParam(); + S3Resource s3Resource = (S3Resource) r; + s3Info.setEndpoint(s3Resource.getProperty(S3Resource.S3_ENDPOINT)); + s3Info.setRegion(s3Resource.getProperty(S3Resource.S3_REGION)); + s3Info.setAk(s3Resource.getProperty(S3Resource.S3_ACCESS_KEY)); + s3Info.setSk(s3Resource.getProperty(S3Resource.S3_SECRET_KEY)); + s3Info.setRootPath(s3Resource.getProperty(S3Resource.S3_ROOT_PATH)); + s3Info.setBucket(s3Resource.getProperty(S3Resource.S3_BUCKET)); + String maxConnections = s3Resource.getProperty(S3Resource.S3_MAX_CONNECTIONS); + s3Info.setMaxConn(Integer.parseInt(maxConnections == null + ? S3Resource.DEFAULT_S3_MAX_CONNECTIONS : maxConnections)); + String requestTimeoutMs = s3Resource.getProperty(S3Resource.S3_REQUEST_TIMEOUT_MS); + s3Info.setMaxConn(Integer.parseInt(requestTimeoutMs == null + ? S3Resource.DEFAULT_S3_REQUEST_TIMEOUT_MS : requestTimeoutMs)); + String connTimeoutMs = s3Resource.getProperty(S3Resource.S3_CONNECTION_TIMEOUT_MS); + s3Info.setMaxConn(Integer.parseInt(connTimeoutMs == null + ? S3Resource.DEFAULT_S3_CONNECTION_TIMEOUT_MS : connTimeoutMs)); + r.readUnlock(); + item.setS3StorageParam(s3Info); + tStorageResources.add(item); + }); + ret.setResource(tStorageResources); + + ret.setDroppedStoragePolicy(droppedStoragePolicy); + 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 f56809441d..a39197f5ad 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 @@ -44,9 +44,9 @@ public class UpdateTabletMetaInfoTask extends AgentTask { private MarkedCountDownLatch>> latch; private Set> tableIdWithSchemaHash; - private boolean isInMemory; + private int inMemory = -1; // < 0 means not to update inMemory property, > 0 means true, == 0 means false private TTabletMetaType metaType; - private String storagePolicy; + private long storagePolicyId = -1; // < 0 means not to update storage policy, == 0 means to reset storage policy // private List> tabletToInMemory; @@ -61,11 +61,11 @@ public class UpdateTabletMetaInfoTask extends AgentTask { public UpdateTabletMetaInfoTask(long backendId, Set> tableIdWithSchemaHash, - boolean isInMemory, String storagePolicy, + int inMemory, long storagePolicyId, MarkedCountDownLatch>> latch) { this(backendId, tableIdWithSchemaHash, TTabletMetaType.INMEMORY); - this.storagePolicy = storagePolicy; - this.isInMemory = isInMemory; + this.storagePolicyId = storagePolicyId; + this.inMemory = inMemory; this.latch = latch; } @@ -132,8 +132,12 @@ public class UpdateTabletMetaInfoTask extends AgentTask { TTabletMetaInfo metaInfo = new TTabletMetaInfo(); metaInfo.setTabletId(pair.first); metaInfo.setSchemaHash(pair.second); - metaInfo.setIsInMemory(isInMemory); - metaInfo.setStoragePolicy(storagePolicy); + if (inMemory >= 0) { + metaInfo.setIsInMemory(inMemory > 0); + } + if (storagePolicyId >= 0) { + metaInfo.setStoragePolicyId(storagePolicyId); + } metaInfo.setMetaType(metaType); metaInfos.add(metaInfo); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/ResourcePersistTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/ResourcePersistTest.java new file mode 100644 index 0000000000..d69fb57a06 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/ResourcePersistTest.java @@ -0,0 +1,57 @@ +// 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.persist; + +import org.apache.doris.catalog.Resource; +import org.apache.doris.catalog.S3Resource; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; + +public class ResourcePersistTest { + @Test + public void test() throws IOException { + Resource resource = new S3Resource("s3_resource"); + File file = new File("./ResourcePersistTest"); + try { + // 1. Write objects to file + file.createNewFile(); + DataOutputStream dos = new DataOutputStream(new FileOutputStream(file)); + resource.write(dos); + dos.flush(); + dos.close(); + + // 2. Read objects from file + DataInputStream dis = new DataInputStream(new FileInputStream(file)); + S3Resource resource1 = (S3Resource) Resource.read(dis); + dis.close(); + Assert.assertEquals(resource1.toString(), resource.toString()); + resource1.readLock(); + resource1.readUnlock(); + } finally { + file.delete(); + } + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/StoragePolicyPersistTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/StoragePolicyPersistTest.java index 29791f1bcc..24f3ad19dc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/persist/StoragePolicyPersistTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/StoragePolicyPersistTest.java @@ -37,22 +37,24 @@ public class StoragePolicyPersistTest { // 1. Write objects to file File file = new File("./StoregaPolicyPersistTest"); - file.createNewFile(); - DataOutputStream dos = new DataOutputStream(new FileOutputStream(file)); - storagePolicy.write(dos); - dos.flush(); - dos.close(); + try { + file.createNewFile(); + DataOutputStream dos = new DataOutputStream(new FileOutputStream(file)); + storagePolicy.write(dos); + dos.flush(); + dos.close(); - // 2. Read objects from file - DataInputStream dis = new DataInputStream(new FileInputStream(file)); - StoragePolicy anotherStoragePolicy = StoragePolicy.read(dis); - Assert.assertEquals(cooldownTime, anotherStoragePolicy.getCooldownTimestampMs()); + // 2. Read objects from file + DataInputStream dis = new DataInputStream(new FileInputStream(file)); + StoragePolicy storagePolicy1 = (StoragePolicy) StoragePolicy.read(dis); + dis.close(); + Assert.assertEquals(cooldownTime, storagePolicy1.getCooldownTimestampMs()); + Assert.assertTrue(storagePolicy1.getLock() != null); - StoragePolicy clonePolicy = anotherStoragePolicy.clone(); - Assert.assertEquals(cooldownTime, clonePolicy.getCooldownTimestampMs()); - - // 3. delete files - dis.close(); - file.delete(); + StoragePolicy clonePolicy = storagePolicy1.clone(); + Assert.assertEquals(cooldownTime, clonePolicy.getCooldownTimestampMs()); + } finally { + file.delete(); + } } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/policy/PolicyTest.java b/fe/fe-core/src/test/java/org/apache/doris/policy/PolicyTest.java index e57953af61..71084fb429 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/policy/PolicyTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/policy/PolicyTest.java @@ -221,7 +221,7 @@ public class PolicyTest extends TestWithFeService { Policy newPolicy = Policy.read(input); Assertions.assertTrue(newPolicy instanceof RowPolicy); RowPolicy newRowPolicy = (RowPolicy) newPolicy; - Assertions.assertEquals(rowPolicy.getPolicyId(), newRowPolicy.getPolicyId()); + Assertions.assertEquals(rowPolicy.getId(), newRowPolicy.getId()); Assertions.assertEquals(type, newRowPolicy.getType()); Assertions.assertEquals(policyName, newRowPolicy.getPolicyName()); Assertions.assertEquals(dbId, newRowPolicy.getDbId()); diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto index 3d103bfc2b..7fad92e321 100644 --- a/gensrc/proto/olap_file.proto +++ b/gensrc/proto/olap_file.proto @@ -284,10 +284,11 @@ 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; + // optional string storage_policy = 22; optional DeleteBitmapPB delete_bitmap = 23; // Use primary key index to speed up tabel unique key model optional bool enable_unique_key_merge_on_write = 24 [default = false]; + optional int64 storage_policy_id = 25; } message OLAPRawDeltaHeaderMessage { diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index 71bcaf42e7..5ce5dfd3ce 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -57,28 +57,38 @@ enum TTabletType { } struct TS3StorageParam { - 1: optional string s3_endpoint - 2: optional string s3_region - 3: optional string s3_ak - 4: optional string s3_sk - 5: optional i32 s3_max_conn = 50 - 6: optional i32 s3_request_timeout_ms = 3000 - 7: optional i32 s3_conn_timeout_ms = 1000 + 1: optional string endpoint + 2: optional string region + 3: optional string ak + 4: optional string sk + 5: optional i32 max_conn = 50 + 6: optional i32 request_timeout_ms = 3000 + 7: optional i32 conn_timeout_ms = 1000 8: optional string root_path 9: optional string bucket } -struct TGetStoragePolicy { - 1: optional string policy_name - 2: optional i64 cooldown_datetime - 3: optional i64 cooldown_ttl - 4: optional TS3StorageParam s3_storage_param - 5: optional string md5_checksum +struct TStoragePolicy { + 1: optional i64 id + 2: optional string name + 3: optional i64 version // alter version + 4: optional i64 cooldown_datetime + 5: optional i64 cooldown_ttl + 6: optional i64 resource_id } -struct TGetStoragePolicyResult { - 1: optional Status.TStatus status - 2: optional list result_entrys +struct TStorageResource { + 1: optional i64 id + 2: optional string name + 3: optional i64 version // alter version + 4: optional TS3StorageParam s3_storage_param + // more storage resource type +} + +struct TPushStoragePolicyReq { + 1: optional list storage_policy + 2: optional list resource + 3: optional list dropped_storage_policy } enum TCompressionType { @@ -116,8 +126,9 @@ 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 + // 18: optional string storage_policy 19: optional bool enable_unique_key_merge_on_write = false + 20: optional i64 storage_policy_id } struct TDropTabletReq { @@ -336,7 +347,8 @@ struct TTabletMetaInfo { 3: optional Types.TPartitionId partition_id 4: optional TTabletMetaType meta_type 5: optional bool is_in_memory - 6: optional string storage_policy; + // 6: optional string storage_policy; + 7: optional i64 storage_policy_id } struct TUpdateTabletMetaInfoReq { @@ -390,8 +402,9 @@ 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 + // DEPRECATED 29: optional TGetStoragePolicy update_policy 30: optional TPushCooldownConfReq push_cooldown_conf + 31: optional TPushStoragePolicyReq push_storage_policy_req } struct TAgentResult { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 32c73a5663..7d1e4e2481 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -749,7 +749,6 @@ service FrontendService { TFrontendPingFrontendResult ping(1: TFrontendPingFrontendRequest request) - AgentService.TGetStoragePolicyResult refreshStoragePolicy() TInitExternalCtlMetaResult initExternalCtlMeta(1: TInitExternalCtlMetaRequest request) TFetchSchemaTableDataResult fetchSchemaTableData(1: TFetchSchemaTableDataRequest request) diff --git a/gensrc/thrift/MasterService.thrift b/gensrc/thrift/MasterService.thrift index 6852f21426..70112f4a5c 100644 --- a/gensrc/thrift/MasterService.thrift +++ b/gensrc/thrift/MasterService.thrift @@ -98,6 +98,8 @@ struct TReportRequest { // the max compaction score of all tablets on a backend, // this field should be set along with tablet report 8: optional i64 tablet_max_compaction_score + 9: optional list storage_policy // only id and version + 10: optional list resource // only id and version } struct TMasterResult { diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift index 5ae9ad46c5..8eb88e1d3d 100644 --- a/gensrc/thrift/Types.thrift +++ b/gensrc/thrift/Types.thrift @@ -203,8 +203,9 @@ enum TTaskType { UNINSTALL_PLUGIN, COMPACTION, STORAGE_MEDIUM_MIGRATE_V2, - NOTIFY_UPDATE_STORAGE_POLICY, - PUSH_COOLDOWN_CONF + NOTIFY_UPDATE_STORAGE_POLICY, // deprecated + PUSH_COOLDOWN_CONF, + PUSH_STORAGE_POLICY } enum TStmtType { diff --git a/regression-test/suites/cold_heat_separation/policy/alter.groovy b/regression-test/suites/cold_heat_separation/policy/alter.groovy index bf58d24b35..e56c235bb6 100644 --- a/regression-test/suites/cold_heat_separation/policy/alter.groovy +++ b/regression-test/suites/cold_heat_separation/policy/alter.groovy @@ -92,6 +92,7 @@ suite("alter_policy") { def show_alter_result = order_sql """ SHOW RESOURCES WHERE NAME = "has_resouce_policy_alter"; """ + logger.info(show_alter_result.toString()) // [[has_resouce_policy_alter, s3, AWS_ACCESS_KEY, 6666], // [has_resouce_policy_alter, s3, AWS_BUCKET, test-bucket], @@ -102,7 +103,9 @@ suite("alter_policy") { // [has_resouce_policy_alter, s3, AWS_REQUEST_TIMEOUT_MS, 7777], // [has_resouce_policy_alter, s3, AWS_ROOT_PATH, path/to/rootaaaa], // [has_resouce_policy_alter, s3, AWS_SECRET_KEY, ******], - // [has_resouce_policy_alter, s3, type, s3]] + // [has_resouce_policy_alter, s3, id, {id}], + // [has_resouce_policy_alter, s3, type, s3] + // [has_resouce_policy_alter, s3, version, {version}]] // AWS_ACCESS_KEY assertEquals(show_alter_result[0][3], "6666") // AWS_BUCKET @@ -121,8 +124,6 @@ suite("alter_policy") { assertEquals(show_alter_result[7][3], "path/to/rootaaaa") // AWS_SECRET_KEY assertEquals(show_alter_result[8][3], "******") - // type - assertEquals(show_alter_result[10][3], "s3") def storage_exist = { name -> def show_storage_policy = sql """ diff --git a/regression-test/suites/cold_heat_separation/policy/create.groovy b/regression-test/suites/cold_heat_separation/policy/create.groovy index 21b217d7b5..19f259065e 100644 --- a/regression-test/suites/cold_heat_separation/policy/create.groovy +++ b/regression-test/suites/cold_heat_separation/policy/create.groovy @@ -45,7 +45,7 @@ suite("create_policy") { def create_sucess = sql """ SHOW RESOURCES WHERE NAME = "crete_policy_1"; """ - assertEquals(create_sucess.size(), 11) + assertEquals(create_sucess.size(), 13) def failed_cannot_create_duplicate_name_resources = try_sql """ CREATE RESOURCE "crete_policy_1" diff --git a/regression-test/suites/cold_heat_separation/policy/show.groovy b/regression-test/suites/cold_heat_separation/policy/show.groovy index 3a62e50e43..c18f3cab8b 100644 --- a/regression-test/suites/cold_heat_separation/policy/show.groovy +++ b/regression-test/suites/cold_heat_separation/policy/show.groovy @@ -15,24 +15,22 @@ // specific language governing permissions and limitations // under the License. -import groovy.json.JsonSlurper - suite("show_policy") { sql """ADMIN SET FRONTEND CONFIG ("enable_storage_policy" = "true");""" - def storage_exist = { name -> + def get_storage_policy = { name -> def show_storage_policy = sql """ SHOW STORAGE POLICY; """ - for(iter in show_storage_policy){ - if(name == iter[0]){ - return true; + for(policy in show_storage_policy){ + if(name == policy[0]){ + return policy; } } - return false; + return []; } - if (!storage_exist.call("showPolicy_1_policy")){ + if (get_storage_policy.call("showPolicy_1_policy").isEmpty()){ def create_s3_resource = try_sql """ CREATE RESOURCE "showPolicy_1_resource" PROPERTIES( @@ -58,10 +56,8 @@ suite("show_policy") { SHOW STORAGE POLICY; """ - def jsonSlurper = new JsonSlurper() - if (show_result.size != 0){ - def json_ret = jsonSlurper.parseText(show_result[0][5]) - assertEquals(json_ret["AWS_SECRET_KEY"], "******") - } - assertEquals(storage_exist.call("showPolicy_1_policy"), true) + def storage_policy = get_storage_policy("showPolicy_1_policy") + assertEquals(storage_policy.size(), 7) + assertEquals(storage_policy[4], "showPolicy_1_resource") + assertEquals(storage_policy[5], "2022-06-08 00:00:00") } diff --git a/regression-test/suites/cold_heat_separation/use_policy/create_table_use_partition_policy.groovy b/regression-test/suites/cold_heat_separation/use_policy/create_table_use_partition_policy.groovy index d60fc1e0d7..8f1fcba080 100644 --- a/regression-test/suites/cold_heat_separation/use_policy/create_table_use_partition_policy.groovy +++ b/regression-test/suites/cold_heat_separation/use_policy/create_table_use_partition_policy.groovy @@ -15,15 +15,10 @@ // specific language governing permissions and limitations // under the License. -import java.text.SimpleDateFormat; -import java.util.Date; - suite("create_table_use_partition_policy") { sql """ADMIN SET FRONTEND CONFIG ("enable_storage_policy" = "true");""" - SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - Date date = new Date(System.currentTimeMillis() + 3600000) - def cooldownTime = format.format(date) + def cooldown_ttl = "10" def create_table_partition_use_not_create_policy = try_sql """ CREATE TABLE IF NOT EXISTS create_table_partition_use_not_create_policy @@ -70,7 +65,7 @@ suite("create_table_use_partition_policy") { CREATE STORAGE POLICY test_create_table_partition_use_policy_1 PROPERTIES( "storage_resource" = "test_create_table_partition_use_resource_1", - "cooldown_datetime" = "$cooldownTime" + "cooldown_ttl" = "$cooldown_ttl" ); """ assertEquals(storage_exist.call("test_create_table_partition_use_policy_1"), true) @@ -95,7 +90,7 @@ suite("create_table_use_partition_policy") { CREATE STORAGE POLICY test_create_table_partition_use_policy_2 PROPERTIES( "storage_resource" = "test_create_table_partition_use_resource_2", - "cooldown_datetime" = "$cooldownTime" + "cooldown_ttl" = "$cooldown_ttl" ); """ assertEquals(storage_exist.call("test_create_table_partition_use_policy_2"), true) diff --git a/regression-test/suites/cold_heat_separation/use_policy/create_table_use_policy.groovy b/regression-test/suites/cold_heat_separation/use_policy/create_table_use_policy.groovy index bb70c834ed..021ffd33e1 100644 --- a/regression-test/suites/cold_heat_separation/use_policy/create_table_use_policy.groovy +++ b/regression-test/suites/cold_heat_separation/use_policy/create_table_use_policy.groovy @@ -15,15 +15,10 @@ // specific language governing permissions and limitations // under the License. -import java.text.SimpleDateFormat; -import java.util.Date; - suite("create_table_use_policy") { sql """ADMIN SET FRONTEND CONFIG ("enable_storage_policy" = "true");""" - SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - Date date = new Date(System.currentTimeMillis() + 3600000) - def cooldownTime = format.format(date) + def cooldown_ttl = "10" def create_table_use_not_create_policy = try_sql """ CREATE TABLE IF NOT EXISTS create_table_use_not_create_policy @@ -72,11 +67,11 @@ suite("create_table_use_policy") { CREATE STORAGE POLICY IF NOT EXISTS test_create_table_use_policy PROPERTIES( "storage_resource" = "test_create_table_use_resource", - "cooldown_datetime" = "$cooldownTime" + "cooldown_ttl" = "$cooldown_ttl" ); """ - sql """ALTER STORAGE POLICY test_create_table_use_policy PROPERTIES("cooldown_datetime" = "$cooldownTime")""" + sql """ALTER STORAGE POLICY test_create_table_use_policy PROPERTIES("cooldown_ttl" = "$cooldown_ttl")""" assertEquals(storage_exist.call("test_create_table_use_policy"), true)