diff --git a/deps/oblib/src/lib/stat/ob_latch_define.h b/deps/oblib/src/lib/stat/ob_latch_define.h index 67988bbafc..01b8ebf903 100644 --- a/deps/oblib/src/lib/stat/ob_latch_define.h +++ b/deps/oblib/src/lib/stat/ob_latch_define.h @@ -304,6 +304,8 @@ LATCH_DEF(REWRITE_RULE_ITEM_LOCK, 292, "rewrite rule item lock", LATCH_FIFO, 200 // LATCH_DEF(TENANT_MGR_TENANT_BUCKET_LOCK, 290, "tenant mgr tenant bucket lock", LATCH_READ_PREFER, INT64_MAX, 0, TENANT_MGR_TENANT_BUCKET_LOCK_WAIT, "tenant mgr tenant bucket lock") LATCH_DEF(SRS_LOCK, 292, "srs lock", LATCH_READ_PREFER, 2000, 0, SRS_LOCK_WAIT, "srs lock") LATCH_DEF(DDL_EXECUTE_LOCK, 293, "ddl execute lock", LATCH_FIFO, 2000, 0, DDL_EXECUTE_LOCK_WAIT, "ddl execute lock") +LATCH_DEF(TENANT_IO_CONFIG_LOCK, 294, "tenant io config lock", LATCH_FIFO, 2000, 0, TENANT_IO_CONFIG_WAIT, "tenant io config lock") + LATCH_DEF(LATCH_END, 99999, "latch end", LATCH_FIFO, 2000, 0, WAIT_EVENT_END, "latch end") #endif diff --git a/deps/oblib/src/lib/wait_event/ob_wait_event.h b/deps/oblib/src/lib/wait_event/ob_wait_event.h index 998a5a19f8..befa4a3a03 100644 --- a/deps/oblib/src/lib/wait_event/ob_wait_event.h +++ b/deps/oblib/src/lib/wait_event/ob_wait_event.h @@ -294,6 +294,8 @@ WAIT_EVENT_DEF(DEADLOCK_DETECT_LOCK_WAIT, 15247, "spinlock: deadlock detect lock //WAIT_EVENT_DEF(BACKUP_ARCHIVE_SERVICE_COND_WAIT, 15250, "backup archive service condition wait", "address", "", "", CONCURRENCY, "backup archive service condition wait", true) WAIT_EVENT_DEF(SRS_LOCK_WAIT, 15251, "latch: srs lock wait", "address", "number", "tries", CONCURRENCY, "latch: srs lock wait", true) WAIT_EVENT_DEF(CLOG_CKPT_RWLOCK_WAIT, 15252, "rwlock: clog checkpoint rwlock wait", "address", "number", "tries", CONCURRENCY, "rwlock: clog checkpoint rwlock wait", true) +WAIT_EVENT_DEF(TENANT_IO_CONFIG_WAIT, 15253, "rwlock: tenant io config wait", "address", "number", "tries", CONCURRENCY, "rwlock: tenant io config wait", true) + //transaction WAIT_EVENT_DEF(END_TRANS_WAIT, 16001, "wait end trans", "rollback", "trans_hash_value", "participant_count", COMMIT,"wait end trans", false) diff --git a/mittest/mtlenv/storage/blocksstable/ob_index_block_data_prepare.h b/mittest/mtlenv/storage/blocksstable/ob_index_block_data_prepare.h index 4de6120ed8..42645637fc 100644 --- a/mittest/mtlenv/storage/blocksstable/ob_index_block_data_prepare.h +++ b/mittest/mtlenv/storage/blocksstable/ob_index_block_data_prepare.h @@ -444,7 +444,6 @@ void TestIndexBlockDataPrepare::prepare_data() ObMacroBlockHandle macro_handle; const int64_t macro_block_size = 2 * 1024 * 1024; ASSERT_EQ(OB_SUCCESS, root_desc.addr_.get_block_addr(read_info.macro_block_id_, read_info.offset_, read_info.size_)); - read_info.io_desc_.set_category(ObIOCategory::USER_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); read_info.offset_ = 0; read_info.size_ = macro_block_size; diff --git a/mittest/mtlenv/storage/blocksstable/test_index_block_tree_cursor.cpp b/mittest/mtlenv/storage/blocksstable/test_index_block_tree_cursor.cpp index 3a28b32d89..ec8e174acb 100644 --- a/mittest/mtlenv/storage/blocksstable/test_index_block_tree_cursor.cpp +++ b/mittest/mtlenv/storage/blocksstable/test_index_block_tree_cursor.cpp @@ -307,7 +307,6 @@ TEST_F(TestIndexBlockTreeCursor, test_bare_micro_block_iterator) read_info.macro_block_id_ = macro_block_id; read_info.offset_ = 0; read_info.size_ = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); ASSERT_EQ(OB_SUCCESS, ObBlockManager::async_read_block(read_info, macro_handle)); ASSERT_EQ(OB_SUCCESS, macro_handle.wait(DEFAULT_IO_WAIT_TIME_MS)); diff --git a/mittest/mtlenv/storage/blocksstable/test_index_tree.cpp b/mittest/mtlenv/storage/blocksstable/test_index_tree.cpp index 9c71c5141b..e2287f1336 100644 --- a/mittest/mtlenv/storage/blocksstable/test_index_tree.cpp +++ b/mittest/mtlenv/storage/blocksstable/test_index_tree.cpp @@ -614,7 +614,6 @@ TEST_F(TestIndexTree, test_macro_id_index_block) ObMacroBlockHandle macro_handle; const int64_t macro_block_size = 2 * 1024 * 1024; read_info.macro_block_id_ = first_macro_id; - read_info.io_desc_.set_category(ObIOCategory::USER_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); read_info.offset_ = 0; read_info.size_ = macro_block_size; @@ -1326,7 +1325,6 @@ TEST_F(TestIndexTree, test_rebuilder) ObMacroBlockReadInfo info; ObMacroBlockHandle macro_handle; const int64_t macro_block_size = 2 * 1024 * 1024; - info.io_desc_.set_category(ObIOCategory::USER_IO); info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); info.offset_ = 0; info.size_ = macro_block_size; diff --git a/mittest/mtlenv/storage/blocksstable/test_shared_macro_block.cpp b/mittest/mtlenv/storage/blocksstable/test_shared_macro_block.cpp index 631af7caad..9e84cc3556 100644 --- a/mittest/mtlenv/storage/blocksstable/test_shared_macro_block.cpp +++ b/mittest/mtlenv/storage/blocksstable/test_shared_macro_block.cpp @@ -112,7 +112,6 @@ TEST_F(TestSharedMacroBlk, test_rebuild_sstable) read_info.offset_ = sstable_.meta_.macro_info_.nested_offset_; read_info.size_ = sstable_.meta_.macro_info_.nested_size_; read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); ASSERT_EQ(OB_SUCCESS, ObBlockManager::read_block(read_info, old_handle)); ObMacroBlockHandle new_handle; @@ -157,7 +156,6 @@ TEST_F(TestSharedMacroBlk, test_invalid_write) read_info.macro_block_id_ = block_info.macro_id_; read_info.size_ = block_info.nested_size_; read_info.offset_ = block_info.nested_offset_; - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); ObMacroBlockHandle read_handle; ASSERT_EQ(OB_SUCCESS, ObBlockManager::read_block(read_info, read_handle)); diff --git a/mittest/mtlenv/storage/test_physical_copy_task.cpp b/mittest/mtlenv/storage/test_physical_copy_task.cpp index 02606fc84f..2bd9626434 100644 --- a/mittest/mtlenv/storage/test_physical_copy_task.cpp +++ b/mittest/mtlenv/storage/test_physical_copy_task.cpp @@ -149,7 +149,6 @@ void TestRootBlockInfo::prepare_block_root() char *io_buf = static_cast(allocator_.alloc(buf_size)); ASSERT_TRUE(nullptr != io_buf); MEMCPY(io_buf + block_addr_.offset_, buf, block_addr_.size_); - write_info.io_desc_.set_category(ObIOCategory::SYS_IO); write_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_WRITE); write_info.buffer_ = io_buf; write_info.size_ = buf_size; diff --git a/src/observer/ob_rpc_processor_simple.cpp b/src/observer/ob_rpc_processor_simple.cpp index 6ffbd71e8a..3ede320fe4 100644 --- a/src/observer/ob_rpc_processor_simple.cpp +++ b/src/observer/ob_rpc_processor_simple.cpp @@ -1905,7 +1905,6 @@ int ObRpcRemoteWriteDDLRedoLogP::process() // restruct write_info write_info.buffer_ = arg_.redo_info_.data_buffer_.ptr(); write_info.size_= arg_.redo_info_.data_buffer_.length(); - write_info.io_desc_.set_category(ObIOCategory::SYS_IO); write_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_WRITE); const int64_t io_timeout_ms = max(DDL_FLUSH_MACRO_BLOCK_TIMEOUT / 1000L, GCONF._data_storage_io_timeout / 1000L); if (OB_FAIL(ObBlockManager::async_write_block(write_info, macro_handle))) { diff --git a/src/observer/omt/ob_multi_tenant.cpp b/src/observer/omt/ob_multi_tenant.cpp index 4142d8cbac..74539a6338 100644 --- a/src/observer/omt/ob_multi_tenant.cpp +++ b/src/observer/omt/ob_multi_tenant.cpp @@ -1192,9 +1192,7 @@ int ObMultiTenant::modify_tenant_io(const uint64_t tenant_id, const ObUnitConfig io_config.callback_thread_count_ = tenant_config->_io_callback_thread_count; static const char *trace_mod_name = "io_tracer"; io_config.enable_io_tracer_ = 0 == strncasecmp(trace_mod_name, GCONF.leak_mod_to_check.get_value(), strlen(trace_mod_name)); - if (OB_FAIL(io_config.parse_category_config(tenant_config->io_category_config))) { - LOG_WARN("parse io category config failed", K(ret)); - } else if (OB_FAIL(OB_IO_MANAGER.refresh_tenant_io_config(tenant_id, io_config))) { + if (OB_FAIL(OB_IO_MANAGER.refresh_tenant_io_config(tenant_id, io_config))) { LOG_WARN("refresh tenant io config failed", K(ret), K(tenant_id), K(io_config)); } } diff --git a/src/observer/omt/ob_tenant.cpp b/src/observer/omt/ob_tenant.cpp index 9011c321d0..62a321d2ee 100644 --- a/src/observer/omt/ob_tenant.cpp +++ b/src/observer/omt/ob_tenant.cpp @@ -1876,6 +1876,10 @@ void ObTenant::check_resource_manager_plan() plan_name))) { LOG_WARN("fail get tenant variable", K(id_), K(plan_name), K(ret)); // skip + } else if (OB_FAIL(rule_mgr.refresh_group_mapping_rule(id_, plan_name))) { + LOG_WARN("refresh group id name mapping rule fail." + "Tenant resource isolation may not work", + K(id_), K(plan_name), K(ret)); } else if (OB_FAIL(plan_mgr.refresh_resource_plan(id_, plan_name))) { LOG_WARN("refresh resource plan fail." "Tenant resource isolation may not work", diff --git a/src/observer/virtual_table/ob_all_virtual_io_status.cpp b/src/observer/virtual_table/ob_all_virtual_io_status.cpp index 8cdc0c01cf..44e6f7ee16 100644 --- a/src/observer/virtual_table/ob_all_virtual_io_status.cpp +++ b/src/observer/virtual_table/ob_all_virtual_io_status.cpp @@ -288,7 +288,7 @@ int ObAllVirtualIOBenchmark::inner_get_next_row(common::ObNewRow *&row) ObAllVirtualIOQuota::QuotaInfo::QuotaInfo() : tenant_id_(OB_INVALID_TENANT_ID), - category_(ObIOCategory::MAX_CATEGORY), + group_id_(0), mode_(ObIOMode::MAX_MODE), size_(0), real_iops_(0), @@ -336,33 +336,70 @@ int ObAllVirtualIOQuota::init(const common::ObAddr &addr) LOG_WARN("tenant not exist", K(ret), K(cur_tenant_id)); } } else { - const ObIOUsage &io_usage = tenant_holder.get_ptr()->get_io_usage(); + ObIOUsage &io_usage = tenant_holder.get_ptr()->get_io_usage(); const ObTenantIOConfig &io_config = tenant_holder.get_ptr()->get_io_config(); ObIOUsage::AvgItems avg_iops, avg_size, avg_rt; + io_usage.calculate_io_usage(); io_usage.get_io_usage(avg_iops, avg_size, avg_rt); - for (int64_t i = 0; OB_SUCC(ret) && i < static_cast(ObIOCategory::MAX_CATEGORY); ++i) { + for (int64_t i = 0; i < io_config.group_num_; ++i) { + if (io_config.group_configs_.at(i).deleted_) { + continue; + } for (int64_t j = 0; OB_SUCC(ret) && j < static_cast(ObIOMode::MAX_MODE); ++j) { - if (avg_size[i][j] > std::numeric_limits::epsilon()) { + if (avg_size.at(i+1).at(j) > std::numeric_limits::epsilon()) { QuotaInfo item; item.tenant_id_ = cur_tenant_id; - item.category_ = static_cast(i); item.mode_ = static_cast(j); - item.size_ = avg_size[i][j]; - item.real_iops_ = avg_iops[i][j]; - int64_t category_min_iops = 0, category_max_iops = 0, category_iops_weight = 0; + item.group_id_ = io_config.group_ids_.at(i); + item.size_ = avg_size.at(i+1).at(j); + item.real_iops_ = avg_iops.at(i+1).at(j); + int64_t group_min_iops = 0, group_max_iops = 0, group_iops_weight = 0; double iops_scale = 0; - if (OB_FAIL(io_config.get_category_config(static_cast(i), - category_min_iops, - category_max_iops, - category_iops_weight))) { - LOG_WARN("get category config failed", K(ret), "category", get_io_category_name(static_cast(i))); - } else if (OB_FAIL(ObIOCalibration::get_instance().get_iops_scale(static_cast(j), avg_size[i][j], iops_scale))) { + if (OB_FAIL(io_config.get_group_config(i, + group_min_iops, + group_max_iops, + group_iops_weight))) { + LOG_WARN("get group config failed", K(ret), K(i)); + } else if (OB_FAIL(ObIOCalibration::get_instance().get_iops_scale(static_cast(j), + avg_size.at(i+1).at(j), + iops_scale))) { LOG_WARN("get iops scale failed", K(ret), "mode", get_io_mode_string(static_cast(j))); } else { - item.min_iops_ = category_min_iops * iops_scale; - item.max_iops_ = category_max_iops * iops_scale; + item.min_iops_ = group_min_iops * iops_scale; + item.max_iops_ = group_max_iops * iops_scale; if (OB_FAIL(quota_infos_.push_back(item))) { - LOG_WARN("push back io quota item failed", K(ret), K(item)); + LOG_WARN("push back io group item failed", K(j), K(ret), K(item)); + } + } + } + } + } + if (OB_SUCC(ret)) { + // OTHER_GROUPS + for (int64_t k = 0; OB_SUCC(ret) && k < static_cast(ObIOMode::MAX_MODE); ++k) { + if (avg_size.at(0).at(k) > std::numeric_limits::epsilon()) { + QuotaInfo item; + item.tenant_id_ = cur_tenant_id; + item.mode_ = static_cast(k); + item.group_id_ = 0; + item.size_ = avg_size.at(0).at(k); + item.real_iops_ = avg_iops.at(0).at(k); + int64_t group_min_iops = 0, group_max_iops = 0, group_iops_weight = 0; + double iops_scale = 0; + if (OB_FAIL(io_config.get_group_config(INT64_MAX, + group_min_iops, + group_max_iops, + group_iops_weight))) { + LOG_WARN("get other group config failed", K(ret), "gruop_info", io_config.other_group_config_); + } else if (OB_FAIL(ObIOCalibration::get_instance().get_iops_scale(static_cast(k), + avg_size.at(0).at(k), + iops_scale))) { + LOG_WARN("get iops scale failed", K(ret), "mode", get_io_mode_string(static_cast(k))); + } else { + item.min_iops_ = group_min_iops * iops_scale; + item.max_iops_ = group_max_iops * iops_scale; + if (OB_FAIL(quota_infos_.push_back(item))) { + LOG_WARN("push back other group item failed", K(k), K(ret), K(item)); } } } @@ -413,9 +450,8 @@ int ObAllVirtualIOQuota::inner_get_next_row(common::ObNewRow *&row) cells[i].set_int(item.tenant_id_); break; } - case CATEGORY: { - cells[i].set_varchar(get_io_category_name(item.category_)); - cells[i].set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + case GROUP_ID: { + cells[i].set_int(item.group_id_); break; } case MODE: { @@ -466,6 +502,211 @@ int ObAllVirtualIOQuota::inner_get_next_row(common::ObNewRow *&row) return ret; } +/****************** IOScheduler *******************/ +ObAllVirtualIOScheduler::ScheduleInfo::ScheduleInfo() + : thread_id_ (-1), + tenant_id_(OB_INVALID_TENANT_ID), + group_id_(0), + queuing_count_(0), + reservation_ts_(INT_MAX64), + group_limitation_ts_(INT_MAX64), + tenant_limitation_ts_(INT_MAX64), + proportion_ts_(INT_MAX64) +{ + +} + +ObAllVirtualIOScheduler::ScheduleInfo::~ScheduleInfo() +{ + +} + +ObAllVirtualIOScheduler::ObAllVirtualIOScheduler() + : schedule_pos_(0), schedule_infos_() +{ + +} + +ObAllVirtualIOScheduler::~ObAllVirtualIOScheduler() +{ + +} + +int ObAllVirtualIOScheduler::init(const common::ObAddr &addr) +{ + int ret = OB_SUCCESS; + ObArray tenant_ids; + if (OB_FAIL(init_addr(addr))) { + LOG_WARN("init failed", K(ret), K(addr)); + } else if (OB_FAIL(OB_IO_MANAGER.get_tenant_ids(tenant_ids))) { + LOG_WARN("get tenant id failed", K(ret)); + } else { + ObIOScheduler *io_scheduler = OB_IO_MANAGER.get_scheduler(); + int64_t thread_num = io_scheduler->get_senders_count(); + for (int64_t thread_id = 0; OB_SUCC(ret) && thread_id < thread_num; ++thread_id) { + ObIOSender *cur_sender = io_scheduler->get_cur_sender(thread_id); + for (int64_t i = 0; OB_SUCC(ret) && i < tenant_ids.count(); ++i) { + const uint64_t cur_tenant_id = tenant_ids.at(i); + ObRefHolder tenant_holder; + if (OB_SERVER_TENANT_ID == cur_tenant_id) { + // tenent_id = 500, do nothing + } else if (OB_FAIL(OB_IO_MANAGER.get_tenant_io_manager(cur_tenant_id, tenant_holder))) { + if (OB_HASH_NOT_EXIST != ret) { + LOG_WARN("get tenant io manager failed", K(ret), K(cur_tenant_id)); + } else { + ret = OB_TENANT_NOT_EXIST; + LOG_WARN("tenant not exist", K(ret), K(cur_tenant_id)); + } + } else { + const ObTenantIOConfig &io_config = tenant_holder.get_ptr()->get_io_config(); + int64_t group_num = tenant_holder.get_ptr()->get_group_num(); + for (int64_t index = 0; OB_SUCC(ret) && index < group_num; ++index) { + if (io_config.group_configs_.at(index).deleted_) { + continue; + } + ScheduleInfo item; + item.thread_id_ = thread_id; + item.tenant_id_ = cur_tenant_id; + item.group_id_ = io_config.group_ids_.at(index); + ObSenderInfo sender_info; + if (OB_FAIL(cur_sender->get_sender_status(cur_tenant_id, index, sender_info))) { + LOG_WARN("get sender status failed", K(ret), K(cur_tenant_id), K(index)); + } else { + item.queuing_count_ = sender_info.queuing_count_; + item.reservation_ts_ = sender_info.reservation_ts_; + item.group_limitation_ts_ = sender_info.group_limitation_ts_; + item.tenant_limitation_ts_ = sender_info.tenant_limitation_ts_; + item.proportion_ts_ = sender_info.proportion_ts_; + if (OB_FAIL(schedule_infos_.push_back(item))) { + LOG_WARN("push back io quota item failed", K(ret), K(item)); + } + } + } + if (OB_SUCC(ret)) { + // OTHER_GROUPS + ScheduleInfo item; + item.thread_id_ = thread_id; + item.tenant_id_ = cur_tenant_id; + item.group_id_ = 0; + ObSenderInfo sender_info; + if (OB_FAIL(cur_sender->get_sender_status(cur_tenant_id, INT64_MAX, sender_info))) { + LOG_WARN("get sender status failed", K(ret), K(cur_tenant_id), K(index)); + } else { + item.queuing_count_ = sender_info.queuing_count_; + item.reservation_ts_ = sender_info.reservation_ts_; + item.group_limitation_ts_ = sender_info.group_limitation_ts_; + item.tenant_limitation_ts_ = sender_info.tenant_limitation_ts_; + item.proportion_ts_ = sender_info.proportion_ts_; + if (OB_FAIL(schedule_infos_.push_back(item))) { + LOG_WARN("push back io quota item failed", K(ret), K(item)); + } + } + } + } + } + } + if (OB_SUCC(ret)) { + is_inited_ = true; + } + } + return ret; +} + +void ObAllVirtualIOScheduler::reset() +{ + ObAllVirtualIOStatusIterator::reset(); + schedule_pos_ = 0; + schedule_infos_.reset(); +} + +int ObAllVirtualIOScheduler::inner_get_next_row(common::ObNewRow *&row) +{ + int ret = OB_SUCCESS; + row = nullptr; + ObObj *cells = cur_row_.cells_; + if (OB_UNLIKELY(!is_inited_ || nullptr == cells)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret), KP(cur_row_.cells_), K(is_inited_)); + } else if (schedule_pos_ >= schedule_infos_.count()) { + row = nullptr; + ret = OB_ITER_END; + } else { + ScheduleInfo &item = schedule_infos_.at(schedule_pos_); + for (int64_t i = 0; OB_SUCC(ret) && i < output_column_ids_.count(); ++i) { + const uint64_t column_id = output_column_ids_.at(i); + switch (column_id) { + case SVR_IP: { + cells[i].set_varchar(ip_buf_); + cells[i].set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + break; + } + case SVR_PORT: { + cells[i].set_int(addr_.get_port()); + break; + } + case THREAD_ID: { + cells[i].set_int(item.thread_id_); + break; + } + case TENANT_ID: { + cells[i].set_int(item.tenant_id_); + break; + } + case GROUP_ID: { + cells[i].set_int(item.group_id_); + break; + } + case QUEUING_COUNT: { + cells[i].set_int(item.queuing_count_); + break; + } + case RESERVATION_TS: { + if (INT_MAX64 == item.reservation_ts_) { + cells[i].set_null(); + } else { + cells[i].set_timestamp(item.reservation_ts_); + } + break; + } + case CATEGORY_LIMIT_TS: { + if (INT_MAX64 == item.group_limitation_ts_) { + cells[i].set_null(); + } else { + cells[i].set_timestamp(item.group_limitation_ts_); + } + break; + } + case TENANT_LIMIT_TS: { + if (INT_MAX64 == item.tenant_limitation_ts_) { + cells[i].set_null(); + } else { + cells[i].set_timestamp(item.tenant_limitation_ts_); + } + break; + } + case PROPORTION_TS: { + if (INT_MAX64 == item.proportion_ts_) { + cells[i].set_null(); + } else { + cells[i].set_timestamp(item.proportion_ts_); + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid column id", K(ret), K(column_id), K(i), K(output_column_ids_)); + break; + } + } // end switch + } // end for-loop + if (OB_SUCC(ret)) { + row = &cur_row_; + } + ++schedule_pos_; + } + return ret; +} + }// namespace observer }// namespace oceanbase diff --git a/src/observer/virtual_table/ob_all_virtual_io_status.h b/src/observer/virtual_table/ob_all_virtual_io_status.h index 73524e908a..a15d5a0d06 100644 --- a/src/observer/virtual_table/ob_all_virtual_io_status.h +++ b/src/observer/virtual_table/ob_all_virtual_io_status.h @@ -17,6 +17,7 @@ #include "share/ob_scanner.h" #include "common/row/ob_row.h" #include "share/io/ob_io_calibration.h" +#include "share/io/ob_io_struct.h" namespace oceanbase { @@ -104,7 +105,7 @@ private: SVR_IP = common::OB_APP_MIN_COLUMN_ID, SVR_PORT, TENANT_ID, - CATEGORY, + GROUP_ID, MODE, SIZE, MIN_IOPS, @@ -119,10 +120,10 @@ private: public: QuotaInfo(); ~QuotaInfo(); - TO_STRING_KV(K(tenant_id_), K(category_), K(mode_), K(size_), K(real_iops_), K(min_iops_), K(max_iops_)); + TO_STRING_KV(K(tenant_id_), K(group_id_), K(mode_), K(size_), K(real_iops_), K(min_iops_), K(max_iops_)); public: uint64_t tenant_id_; - common::ObIOCategory category_; + uint64_t group_id_; common::ObIOMode mode_; double size_; double real_iops_; @@ -135,6 +136,50 @@ private: int64_t quota_pos_; }; +class ObAllVirtualIOScheduler : public ObAllVirtualIOStatusIterator +{ +public: + ObAllVirtualIOScheduler(); + virtual ~ObAllVirtualIOScheduler(); + int init(const common::ObAddr &addr); + virtual void reset() override; + virtual int inner_get_next_row(common::ObNewRow *&row) override; +private: + enum COLUMN + { + SVR_IP = common::OB_APP_MIN_COLUMN_ID, + SVR_PORT, + THREAD_ID, + TENANT_ID, + GROUP_ID, + QUEUING_COUNT, + RESERVATION_TS, + CATEGORY_LIMIT_TS, + TENANT_LIMIT_TS, + PROPORTION_TS + }; + struct ScheduleInfo + { + public: + ScheduleInfo(); + ~ScheduleInfo(); + TO_STRING_KV(K(thread_id_),K(tenant_id_), K(group_id_), K(queuing_count_)); + public: + uint64_t thread_id_; + uint64_t tenant_id_; + uint64_t group_id_; + int64_t queuing_count_; + int64_t reservation_ts_; + int64_t group_limitation_ts_; + int64_t tenant_limitation_ts_; + int64_t proportion_ts_; + }; + DISALLOW_COPY_AND_ASSIGN(ObAllVirtualIOScheduler); +private: + int64_t schedule_pos_; + ObArray schedule_infos_; +}; + }// namespace observer }// namespace oceanbase diff --git a/src/observer/virtual_table/ob_all_virtual_tablet_sstable_macro_info.cpp b/src/observer/virtual_table/ob_all_virtual_tablet_sstable_macro_info.cpp index ba1eff96e2..55bd56f9ee 100644 --- a/src/observer/virtual_table/ob_all_virtual_tablet_sstable_macro_info.cpp +++ b/src/observer/virtual_table/ob_all_virtual_tablet_sstable_macro_info.cpp @@ -186,7 +186,6 @@ int ObAllVirtualTabletSSTableMacroInfo::get_macro_info( ObMacroBlockHandle macro_handle; ObMacroBlockReadInfo macro_read_info; macro_read_info.macro_block_id_ = macro_id; - macro_read_info.io_desc_.set_category(ObIOCategory::USER_IO); macro_read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); macro_read_info.offset_ = 0; macro_read_info.size_ = OB_SERVER_BLOCK_MGR.get_macro_block_size(); diff --git a/src/share/inner_table/ob_inner_table_schema.11101_11150.cpp b/src/share/inner_table/ob_inner_table_schema.11101_11150.cpp index 8593bd99fe..e8820f022b 100644 --- a/src/share/inner_table/ob_inner_table_schema.11101_11150.cpp +++ b/src/share/inner_table/ob_inner_table_schema.11101_11150.cpp @@ -2943,14 +2943,14 @@ int ObInnerTableSchema::all_virtual_io_quota_schema(ObTableSchema &table_schema) } if (OB_SUCC(ret)) { - ADD_COLUMN_SCHEMA("category", //column_name + ADD_COLUMN_SCHEMA("group_id", //column_name ++column_id, //column_id 0, //rowkey_id 0, //index_id 0, //part_key_pos - ObVarcharType, //column_type + ObIntType, //column_type CS_TYPE_INVALID, //column_collation_type - 256, //column_length + sizeof(int64_t), //column_length -1, //column_precision -1, //column_scale false, //is_nullable diff --git a/src/share/inner_table/ob_inner_table_schema.15151_15200.cpp b/src/share/inner_table/ob_inner_table_schema.15151_15200.cpp index d2ec50eef7..3f1ca61404 100644 --- a/src/share/inner_table/ob_inner_table_schema.15151_15200.cpp +++ b/src/share/inner_table/ob_inner_table_schema.15151_15200.cpp @@ -5003,6 +5003,51 @@ int ObInnerTableSchema::all_virtual_res_mgr_directive_real_agent_ora_schema(ObTa false); //is_autoincrement } + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("MIN_IOPS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("MAX_IOPS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("WEIGHT_IOPS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + if (OB_SUCC(ret)) { ADD_COLUMN_SCHEMA("GMT_CREATE", //column_name ++column_id, //column_id diff --git a/src/share/inner_table/ob_inner_table_schema.301_350.cpp b/src/share/inner_table/ob_inner_table_schema.301_350.cpp index 9acecd4ecf..d4bda5e4c0 100644 --- a/src/share/inner_table/ob_inner_table_schema.301_350.cpp +++ b/src/share/inner_table/ob_inner_table_schema.301_350.cpp @@ -338,6 +338,63 @@ int ObInnerTableSchema::all_res_mgr_directive_schema(ObTableSchema &table_schema utilization_limit_default, utilization_limit_default); //default_value } + + if (OB_SUCC(ret)) { + ObObj min_iops_default; + min_iops_default.set_int(0); + ADD_COLUMN_SCHEMA_T("min_iops", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + min_iops_default, + min_iops_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj max_iops_default; + max_iops_default.set_int(100); + ADD_COLUMN_SCHEMA_T("max_iops", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + max_iops_default, + max_iops_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj weight_iops_default; + weight_iops_default.set_int(0); + ADD_COLUMN_SCHEMA_T("weight_iops", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + weight_iops_default, + weight_iops_default); //default_value + } table_schema.set_index_using_type(USING_BTREE); table_schema.set_row_store_type(ENCODING_ROW_STORE); table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); diff --git a/src/share/inner_table/ob_inner_table_schema_def.py b/src/share/inner_table/ob_inner_table_schema_def.py index bc4c164623..86db278cd3 100644 --- a/src/share/inner_table/ob_inner_table_schema_def.py +++ b/src/share/inner_table/ob_inner_table_schema_def.py @@ -2709,7 +2709,10 @@ def_table_schema( normal_columns = [ ('comments', 'varchar:2000', 'true'), ('mgmt_p1', 'int', 'false', 100), - ('utilization_limit', 'int', 'false', 100) + ('utilization_limit', 'int', 'false', 100), + ('min_iops', 'int', 'false', 0), + ('max_iops', 'int', 'false', 100), + ('weight_iops', 'int', 'false', 0) ], ) @@ -7744,7 +7747,7 @@ def_table_schema( ('svr_ip', 'varchar:MAX_IP_ADDR_LENGTH'), ('svr_port', 'int'), ('tenant_id', 'int'), - ('category', 'varchar:256'), + ('group_id', 'int'), ('mode', 'varchar:256'), ('size', 'int'), ('min_iops', 'int'), diff --git a/src/share/inner_table/sys_package/dbms_resource_manager_body_mysql.sql b/src/share/inner_table/sys_package/dbms_resource_manager_body_mysql.sql index 307c030626..3996ce8ded 100644 --- a/src/share/inner_table/sys_package/dbms_resource_manager_body_mysql.sql +++ b/src/share/inner_table/sys_package/dbms_resource_manager_body_mysql.sql @@ -30,7 +30,10 @@ PROCEDURE create_plan_directive ( group_or_subplan VARCHAR(65535), comment VARCHAR(65535) DEFAULT '', mgmt_p1 INT DEFAULT 100, - utilization_limit INT DEFAULT 100 + utilization_limit INT DEFAULT 100, + min_iops INT DEFAULT 0, + max_iops INT DEFAULT 100, + weight_iops INT DEFAULT 0 ); pragma interface (C, CREATE_PLAN_DIRECTIVE_INNER); @@ -39,7 +42,10 @@ PROCEDURE update_plan_directive ( group_or_subplan VARCHAR(65535), new_comment VARCHAR(65535) DEFAULT NULL, new_mgmt_p1 INT DEFAULT NULL, - new_utilization_limit INT DEFAULT NULL + new_utilization_limit INT DEFAULT NULL, + new_min_iops INT DEFAULT NULL, + new_max_iops INT DEFAULT NULL, + new_weight_iops INT DEFAULT NULL ); pragma interface (C, UPDATE_PLAN_DIRECTIVE_INNER); diff --git a/src/share/inner_table/sys_package/dbms_resource_manager_mysql.sql b/src/share/inner_table/sys_package/dbms_resource_manager_mysql.sql index 9d96a127e2..75648ac26e 100644 --- a/src/share/inner_table/sys_package/dbms_resource_manager_mysql.sql +++ b/src/share/inner_table/sys_package/dbms_resource_manager_mysql.sql @@ -44,7 +44,10 @@ CREATE OR REPLACE PACKAGE dbms_resource_manager AUTHID CURRENT_USER group_or_subplan VARCHAR(65535), comment VARCHAR(65535) DEFAULT '', mgmt_p1 INT DEFAULT 100, - utilization_limit INT DEFAULT 100 + utilization_limit INT DEFAULT 100, + min_iops INT DEFAULT 0, + max_iops INT DEFAULT 100, + weight_iops INT DEFAULT 0 ); -- @@ -55,7 +58,10 @@ CREATE OR REPLACE PACKAGE dbms_resource_manager AUTHID CURRENT_USER group_or_subplan VARCHAR(65535), new_comment VARCHAR(65535) DEFAULT NULL, new_mgmt_p1 INT DEFAULT NULL, - new_utilization_limit INT DEFAULT NULL + new_utilization_limit INT DEFAULT NULL, + new_min_iops INT DEFAULT NULL, + new_max_iops INT DEFAULT NULL, + new_weight_iops INT DEFAULT NULL ); -- diff --git a/src/share/io/io_schedule/ob_io_mclock.cpp b/src/share/io/io_schedule/ob_io_mclock.cpp index 784cd4cb83..23e55241fe 100644 --- a/src/share/io/io_schedule/ob_io_mclock.cpp +++ b/src/share/io/io_schedule/ob_io_mclock.cpp @@ -105,6 +105,11 @@ bool ObMClock::is_valid() const return is_inited_ && !is_stopped_; } +bool ObMClock::is_stop() const +{ + return is_stopped_; +} + int ObMClock::calc_phy_clock(const int64_t current_ts, const double iops_scale, const double weight_scale, ObPhyQueue *phy_queue) { int ret = OB_SUCCESS; @@ -118,7 +123,7 @@ int ObMClock::calc_phy_clock(const int64_t current_ts, const double iops_scale, LOG_WARN("invalid argument", K(ret), K(current_ts), K(iops_scale), K(weight_scale)); } else { reservation_clock_.atom_update(current_ts, iops_scale, phy_queue->reservation_ts_); - limitation_clock_.atom_update(current_ts, iops_scale, phy_queue->category_limitation_ts_); + limitation_clock_.atom_update(current_ts, iops_scale, phy_queue->group_limitation_ts_); proportion_clock_.atom_update(current_ts, iops_scale * weight_scale, phy_queue->proportion_ts_); } return ret; @@ -163,8 +168,8 @@ int64_t ObMClock::get_proportion_ts() const /****************** TenantIOClock **********************/ ObTenantIOClock::ObTenantIOClock() : is_inited_(false), - category_clocks_(), - other_clock_(), + group_clocks_(), + other_group_clock_(), io_config_(), io_usage_(nullptr) { @@ -185,20 +190,42 @@ int ObTenantIOClock::init(const ObTenantIOConfig &io_config, const ObIOUsage *io } else if (OB_UNLIKELY(!io_config.is_valid() || nullptr == io_usage)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(io_config), KP(io_usage)); + } else if (OB_FAIL(io_config_.deep_copy(io_config))) { + LOG_WARN("get io config failed", K(ret), K(io_config)); + } else if (OB_FAIL(group_clocks_.reserve(io_config.group_num_))) { + LOG_WARN("reserver group failed", K(ret), K(io_config.group_num_)); } else { - io_config_ = io_config; const ObTenantIOConfig::UnitConfig &unit_config = io_config.unit_config_; - for (int64_t i = 0; OB_SUCC(ret) && i < static_cast(ObIOCategory::MAX_CATEGORY) + 1; ++i) { - const ObTenantIOConfig::CategoryConfig &cur_config = - static_cast(ObIOCategory::MAX_CATEGORY) == i ? io_config.other_config_ : io_config.category_configs_[i]; - ObMClock &cur_clock = static_cast(ObIOCategory::MAX_CATEGORY) == i ? other_clock_ : category_clocks_[i]; - if (cur_config.is_valid()) { - if (OB_FAIL(cur_clock.init(calc_iops(unit_config.min_iops_, cur_config.min_percent_), - calc_iops(unit_config.max_iops_, cur_config.max_percent_), - calc_weight(unit_config.weight_, cur_config.weight_percent_)))) { - LOG_WARN("init category io clock failed", K(ret), K(i), K(unit_config), K(cur_config)); + const int64_t all_group_num = io_config.get_all_group_num(); + for (int64_t i = 0; OB_SUCC(ret) && i < all_group_num; ++i) { + if (i == all_group_num - 1) { + //OTHER_GROUPS + const ObTenantIOConfig::GroupConfig &cur_config = io_config.other_group_config_; + if (OB_UNLIKELY(!cur_config.is_valid())) { + ret = OB_INVALID_CONFIG; + LOG_WARN("config is not valid", K(ret), K(i), K(cur_config)); + } else if (OB_FAIL(other_group_clock_.init(calc_iops(unit_config.min_iops_, cur_config.min_percent_), + calc_iops(unit_config.max_iops_, cur_config.max_percent_), + calc_weight(unit_config.weight_, cur_config.weight_percent_)))) { + LOG_WARN("init io clock failed", K(ret), K(i), K(other_group_clock_)); } else { - LOG_INFO("init category clock", K(i), K(unit_config), K(cur_config)); + LOG_INFO("init other group clock success", K(i), K(unit_config), K(cur_config)); + } + } else { + //regular groups + const ObTenantIOConfig::GroupConfig &cur_config = io_config.group_configs_.at(i); + ObMClock cur_clock; + if (OB_UNLIKELY(!cur_config.is_valid())) { + ret = OB_INVALID_CONFIG; + LOG_WARN("config is not valid", K(ret), K(i), K(cur_config)); + } else if (OB_FAIL(cur_clock.init(calc_iops(unit_config.min_iops_, cur_config.min_percent_), + calc_iops(unit_config.max_iops_, cur_config.max_percent_), + calc_weight(unit_config.weight_, cur_config.weight_percent_)))) { + LOG_WARN("init io clock failed", K(ret), K(i), K(cur_clock)); + } else if (OB_FAIL(group_clocks_.push_back(cur_clock))) { + LOG_WARN("push back group io clock failed", K(ret), K(i), K(cur_clock)); + } else { + LOG_INFO("init group clock success", K(i), K(unit_config), K(cur_config)); } } } @@ -215,10 +242,11 @@ int ObTenantIOClock::init(const ObTenantIOConfig &io_config, const ObIOUsage *io void ObTenantIOClock::destroy() { is_inited_ = false; - for (int64_t i = 0; i < static_cast(ObIOCategory::MAX_CATEGORY); ++i) { - category_clocks_[i].destroy(); + for (int64_t i = 0; i < group_clocks_.count(); ++i) { + group_clocks_.at(i).destroy(); } - other_clock_.destroy(); + other_group_clock_.destroy(); + group_clocks_.destroy(); io_usage_ = nullptr; } @@ -234,22 +262,22 @@ int ObTenantIOClock::calc_phyqueue_clock(ObPhyQueue *phy_queue, const ObIOReques } else if (req.get_flag().is_unlimited()) { const int64_t current_ts = ObTimeUtility::fast_current_time(); phy_queue->reservation_ts_ = current_ts; - phy_queue->category_limitation_ts_ = current_ts; + phy_queue->group_limitation_ts_ = current_ts; phy_queue->tenant_limitation_ts_ = current_ts; phy_queue->proportion_ts_ = current_ts; } else { const int64_t current_ts = ObTimeUtility::fast_current_time(); - int cate_index=phy_queue->category_index_; - if(cate_index < 0 || cate_index >= static_cast(ObIOCategory::MAX_CATEGORY)) { + uint64_t cur_queue_index = phy_queue->queue_index_; + if (cur_queue_index < 0 || (cur_queue_index >= group_clocks_.count() && cur_queue_index != INT64_MAX)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("index out of boundary", K(ret), K(cate_index)); + LOG_WARN("index out of boundary", K(ret), K(cur_queue_index), K(group_clocks_.count())); } else { - ObMClock &mclock = get_mclock(cate_index); - double weight_scale = get_weight_scale(cate_index); + ObMClock &mclock = get_mclock(cur_queue_index); + double weight_scale = get_weight_scale(cur_queue_index); double iops_scale = 0; if (OB_FAIL(ObIOCalibration::get_instance().get_iops_scale(req.get_mode(), - max(req.io_info_.size_, req.io_size_), - iops_scale))) { + max(req.io_info_.size_, req.io_size_), + iops_scale))) { LOG_WARN("get iops scale failed", K(ret), K(req)); } else if (OB_FAIL(mclock.calc_phy_clock(current_ts, iops_scale, weight_scale, phy_queue))) { LOG_WARN("calculate clock of the request failed", K(ret), K(mclock), K(weight_scale)); @@ -262,7 +290,7 @@ int ObTenantIOClock::calc_phyqueue_clock(ObPhyQueue *phy_queue, const ObIOReques return ret; } -int ObTenantIOClock::sync_clocks(ObIArray &io_clocks) +int ObTenantIOClock::sync_clocks(ObIArray &io_clocks) { int ret = OB_SUCCESS; int64_t min_proportion_ts = INT64_MAX; @@ -310,15 +338,15 @@ int ObTenantIOClock::sync_tenant_clock(ObTenantIOClock *io_clock) int ObTenantIOClock::adjust_reservation_clock(ObPhyQueue *phy_queue, const ObIORequest &req) { int ret = OB_SUCCESS; - int cate_index=phy_queue->category_index_; - if(cate_index < 0 || cate_index >= static_cast(ObIOCategory::MAX_CATEGORY)) { - LOG_WARN("index out of boundary", K(ret), K(cate_index)); + uint64_t cur_queue_index = phy_queue->queue_index_; + if(cur_queue_index < 0 || (cur_queue_index >= group_clocks_.count() && cur_queue_index != INT64_MAX)) { + LOG_WARN("index out of boundary", K(ret), K(cur_queue_index)); } else { - ObMClock &mclock = get_mclock(cate_index); + ObMClock &mclock = get_mclock(cur_queue_index); double iops_scale = 0; if (OB_FAIL(ObIOCalibration::get_instance().get_iops_scale(req.get_mode(), - max(req.io_info_.size_, req.io_size_), - iops_scale))) { + max(req.io_info_.size_, req.io_size_), + iops_scale))) { LOG_WARN("get iops scale failed", K(ret), K(req)); } else if (OB_FAIL(mclock.dial_back_reservation_clock(iops_scale))) { LOG_WARN("dial back reservation clock failed", K(ret), K(iops_scale), K(req), K(mclock)); @@ -330,60 +358,103 @@ int ObTenantIOClock::adjust_reservation_clock(ObPhyQueue *phy_queue, const ObIOR int ObTenantIOClock::adjust_proportion_clock(const int64_t delta_us) { int ret = OB_SUCCESS; - for (int64_t i = 0; OB_SUCC(ret) && i < static_cast(ObIOCategory::MAX_CATEGORY); ++i) { - if (category_clocks_[i].is_valid()) { - category_clocks_[i].dial_back_proportion_clock(delta_us); + for (int64_t i = 0; OB_SUCC(ret) && i < group_clocks_.count(); ++i) { + if (group_clocks_.at(i).is_valid() && !group_clocks_.at(i).is_stop()) { + group_clocks_.at(i).dial_back_proportion_clock(delta_us); } } - if (other_clock_.is_valid()) { - other_clock_.dial_back_proportion_clock(delta_us); + if (other_group_clock_.is_valid() && !other_group_clock_.is_stop()) { + other_group_clock_.dial_back_proportion_clock(delta_us); } return ret; } -int ObTenantIOClock::update_io_config(const ObTenantIOConfig &io_config) +int ObTenantIOClock::update_io_clocks(const ObTenantIOConfig &io_config) { int ret = OB_SUCCESS; - if (OB_UNLIKELY(!io_config.is_valid())) { - ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid argument", K(ret), K(io_config)); + if (OB_FAIL(io_config_.deep_copy(io_config))) { + LOG_WARN("get io config failed", K(ret), K(io_config)); } else { - io_config_ = io_config; - const int64_t min_proportion_ts = get_min_proportion_ts(); - const ObTenantIOConfig::UnitConfig &unit_config = io_config.unit_config_; - for (int64_t i = 0; OB_SUCC(ret) && i < static_cast(ObIOCategory::MAX_CATEGORY) + 1; ++i) { - const ObTenantIOConfig::CategoryConfig &cur_config = - static_cast(ObIOCategory::MAX_CATEGORY) == i ? io_config.other_config_ : io_config.category_configs_[i]; - ObMClock &cur_clock = static_cast(ObIOCategory::MAX_CATEGORY) == i ? other_clock_ : category_clocks_[i]; - if (cur_config.is_valid() && cur_clock.is_inited()) { - // update - if (OB_FAIL(cur_clock.update(calc_iops(unit_config.min_iops_, cur_config.min_percent_), - calc_iops(unit_config.max_iops_, cur_config.max_percent_), - calc_weight(unit_config.weight_, cur_config.weight_percent_), - min_proportion_ts))) { - LOG_WARN("update category io clock failed", K(ret), K(i), K(unit_config), K(cur_config)); - } else { - cur_clock.start(); - LOG_INFO("update category clock", K(i), K(unit_config), K(cur_config)); - } - } else if (cur_config.is_valid() && !cur_clock.is_inited()) { - // init the new category - if (OB_FAIL(cur_clock.init(calc_iops(unit_config.min_iops_, cur_config.min_percent_), - calc_iops(unit_config.max_iops_, cur_config.max_percent_), - calc_weight(unit_config.weight_, cur_config.weight_percent_), - min_proportion_ts))) { - LOG_WARN("init category io clock failed", K(ret), K(i), K(unit_config), K(cur_config)); - } else { - LOG_INFO("init category clock", K(i), K(unit_config), K(cur_config)); - } - } else if (!cur_config.is_valid() && cur_clock.is_inited()) { - // stop - cur_clock.stop(); + if (group_clocks_.count() < io_config.group_num_) { + if (OB_FAIL(group_clocks_.reserve(io_config.group_num_))) { + LOG_WARN("reserve group config failed", K(ret), K(group_clocks_), K(io_config.group_num_)); } } if (OB_SUCC(ret)) { - unit_clock_.iops_ = unit_config.max_iops_; - is_inited_ = true; + const int64_t all_group_num = io_config.get_all_group_num(); + for (int64_t i = 0; OB_SUCC(ret) && i < all_group_num; ++i) { + if (OB_FAIL(update_io_clock(i, io_config, all_group_num))) { + LOG_WARN("update cur clock failed", K(ret), K(i)); + } + } + if (OB_SUCC(ret)) { + unit_clock_.iops_ = io_config.unit_config_.max_iops_; + is_inited_ = true; + } + } + } + return ret; +} + +int ObTenantIOClock::update_io_clock(const int64_t index, const ObTenantIOConfig &io_config, const int64_t all_group_num) +{ + int ret = OB_SUCCESS; + const ObTenantIOConfig::UnitConfig &unit_config = io_config.unit_config_; + const int64_t min_proportion_ts = get_min_proportion_ts(); + if (index == all_group_num - 1) { + //1. update other group + const ObTenantIOConfig::GroupConfig &cur_config = io_config.other_group_config_; + if (OB_UNLIKELY(!other_group_clock_.is_inited())) { + LOG_WARN("clock is not init", K(ret), K(index), K(other_group_clock_)); + } else if (OB_UNLIKELY(!cur_config.is_valid())) { + ret = OB_INVALID_CONFIG; + LOG_WARN("config is not valid", K(ret), K(index), K(cur_config)); + // stop + other_group_clock_.stop(); + } else if (OB_FAIL(other_group_clock_.update(calc_iops(unit_config.min_iops_, cur_config.min_percent_), + calc_iops(unit_config.max_iops_, cur_config.max_percent_), + calc_weight(unit_config.weight_, cur_config.weight_percent_), + min_proportion_ts))) { + LOG_WARN("update other group io clock failed", K(ret), K(index), K(other_group_clock_)); + } else { + other_group_clock_.start(); + LOG_INFO("update other group clock success", K(index), K(unit_config), K(cur_config)); + } + } else if (index < group_clocks_.count()) { + // 2. update exist clocks + const ObTenantIOConfig::GroupConfig &cur_config = io_config.group_configs_.at(index); + if (!group_clocks_.at(index).is_inited()) { + LOG_WARN("clock is not init", K(ret), K(index), K(group_clocks_.at(index))); + } else if (group_clocks_.at(index).is_stop() || cur_config.deleted_) { + // group has been deleted, ignore + } else if (!cur_config.is_valid()) { + LOG_WARN("config is not valid", K(ret), K(index), K(cur_config), K(group_clocks_.at(index))); + // stop + group_clocks_.at(index).stop(); + } else if (OB_FAIL(group_clocks_.at(index).update(calc_iops(unit_config.min_iops_, cur_config.min_percent_), + calc_iops(unit_config.max_iops_, cur_config.max_percent_), + calc_weight(unit_config.weight_, cur_config.weight_percent_), + min_proportion_ts))) { + LOG_WARN("update group io clock failed", K(ret), K(index), K(unit_config), K(cur_config)); + } else { + group_clocks_.at(index).start(); + LOG_INFO("update group clock success", K(index), K(unit_config), K(cur_config)); + } + } else { + // 3. add new clocks + const ObTenantIOConfig::GroupConfig &cur_config = io_config.group_configs_.at(index); + ObMClock cur_clock; + if (OB_UNLIKELY(!cur_config.is_valid())) { + ret = OB_INVALID_CONFIG; + LOG_WARN("config is not valid", K(ret), K(index), K(cur_config)); + } else if (OB_FAIL(cur_clock.init(calc_iops(unit_config.min_iops_, cur_config.min_percent_), + calc_iops(unit_config.max_iops_, cur_config.max_percent_), + calc_weight(unit_config.weight_, cur_config.weight_percent_)))) { + LOG_WARN("init io clock failed", K(ret), K(index), K(cur_clock)); + } else if (OB_FAIL(group_clocks_.push_back(cur_clock))) { + LOG_WARN("push back io clock failed", K(ret), K(index), K(cur_clock)); + } else { + LOG_INFO("init new group clock success", K(index), K(unit_config), K(cur_config)); } } return ret; @@ -392,43 +463,42 @@ int ObTenantIOClock::update_io_config(const ObTenantIOConfig &io_config) int64_t ObTenantIOClock::get_min_proportion_ts() { int64_t min_proportion_ts = INT64_MAX; - for (int64_t i = 0; i < static_cast(ObIOCategory::MAX_CATEGORY); ++i) { - if (category_clocks_[i].is_valid()) { - min_proportion_ts = min(min_proportion_ts, category_clocks_[i].get_proportion_ts()); + for (int64_t i = 0; i < group_clocks_.count(); ++i) { + if (group_clocks_.at(i).is_valid()) { + min_proportion_ts = min(min_proportion_ts, group_clocks_.at(i).get_proportion_ts()); } } - if (other_clock_.is_valid()) { - min_proportion_ts = min(min_proportion_ts, other_clock_.get_proportion_ts()); + if (other_group_clock_.is_valid()) { + min_proportion_ts = min(min_proportion_ts, other_group_clock_.get_proportion_ts()); } return min_proportion_ts; } -ObMClock &ObTenantIOClock::get_mclock(const int category_index) +ObMClock &ObTenantIOClock::get_mclock(const int64_t queue_index) { - const bool use_category_clock = category_clocks_[category_index].is_valid(); - ObMClock &io_clock = use_category_clock ? category_clocks_[category_index] : other_clock_; + ObMClock &io_clock = queue_index == INT64_MAX ? other_group_clock_ : group_clocks_.at(queue_index); return io_clock; } -double ObTenantIOClock::get_weight_scale(const int category_index) +double ObTenantIOClock::get_weight_scale(const int64_t queue_index) { double weight_scale = 1; if (OB_ISNULL(io_usage_)) { // do nothing } else { - const bool is_other_category = !category_clocks_[category_index].is_valid(); int64_t sum_weight_percent = 0; bool need_add_other_weight = true; - for (int64_t i = 0; i < static_cast(ObIOCategory::MAX_CATEGORY); ++i) { - if (io_usage_->is_request_doing(static_cast(i))) { - if (category_clocks_[i].is_valid()) { - sum_weight_percent += io_config_.category_configs_[i].weight_percent_; - } else if (need_add_other_weight) { - sum_weight_percent += io_config_.other_config_.weight_percent_; - need_add_other_weight = false; + for (int64_t i = 0; i < io_config_.group_num_; ++i) { + int64_t usage_index = i + 1; + if (usage_index < io_usage_->get_io_usage_num() && io_usage_->is_request_doing(usage_index)) { + if (group_clocks_.at(i).is_valid()) { + sum_weight_percent += io_config_.group_configs_.at(i).weight_percent_; } } } + if (io_usage_->get_io_usage_num() > 0 && io_usage_->is_request_doing(0)) { + sum_weight_percent += io_config_.other_group_config_.weight_percent_; + } if (sum_weight_percent > 0) { weight_scale = 100.0 / sum_weight_percent; } @@ -445,3 +515,11 @@ int64_t ObTenantIOClock::calc_weight(const int64_t weight, const int64_t percent { return static_cast(static_cast(weight) * percentage / 100); } + + +void ObTenantIOClock::stop_clock(const uint64_t index) +{ + if (index < group_clocks_.count() && index >= 0) { + group_clocks_.at(index).stop(); + } +} \ No newline at end of file diff --git a/src/share/io/io_schedule/ob_io_mclock.h b/src/share/io/io_schedule/ob_io_mclock.h index 207bad1fcc..3ff022ae4e 100644 --- a/src/share/io/io_schedule/ob_io_mclock.h +++ b/src/share/io/io_schedule/ob_io_mclock.h @@ -15,6 +15,8 @@ #include "share/io/ob_io_define.h" #include "lib/container/ob_heap.h" +#include "lib/container/ob_array_iterator.h" +#include "lib/container/ob_array_wrap.h" namespace oceanbase { @@ -33,6 +35,7 @@ public: void destroy(); bool is_inited() const; bool is_valid() const; + bool is_stop() const; int calc_phy_clock(const int64_t current_ts, const double iops_scale, const double weight_scale, ObPhyQueue *phy_queue); int dial_back_reservation_clock(const double iops_scale); int dial_back_proportion_clock(const int64_t delta_us); @@ -47,31 +50,33 @@ private: }; struct ObTenantIOConfig; -class ObTenantIOClock : public ObIOClock +class ObTenantIOClock { public: ObTenantIOClock(); virtual ~ObTenantIOClock(); - virtual int init(const ObTenantIOConfig &io_config, const ObIOUsage *io_usage) override; - virtual void destroy() override; + int init(const ObTenantIOConfig &io_config, const ObIOUsage *io_usage); + void destroy(); int calc_phyqueue_clock(ObPhyQueue *phy_queue, const ObIORequest &req); - virtual int sync_clocks(ObIArray &io_clocks) override; + int sync_clocks(ObIArray &io_clocks); int sync_tenant_clock(ObTenantIOClock *ioclock); int adjust_reservation_clock(ObPhyQueue *phy_queue, const ObIORequest &req); int adjust_proportion_clock(const int64_t delta_us); - virtual int update_io_config(const ObTenantIOConfig &io_config) override; + int update_io_clocks(const ObTenantIOConfig &io_config); + int update_io_clock(const int64_t index, const ObTenantIOConfig &io_config, const int64_t all_group_num); int64_t get_min_proportion_ts(); - TO_STRING_KV(K(is_inited_), "category_clocks", ObArrayWrap(category_clocks_, static_cast(ObIOCategory::MAX_CATEGORY)), - K_(other_clock), K_(unit_clock), K(io_config_), K(io_usage_)); + void stop_clock(const uint64_t index); + TO_STRING_KV(K(is_inited_), "group_clocks", group_clocks_, "other_clock", other_group_clock_, + K_(unit_clock), K(io_config_), K(io_usage_)); private: - ObMClock &get_mclock(const int category_index); - double get_weight_scale(const int category_index); + ObMClock &get_mclock(const int64_t queue_index); + double get_weight_scale(const int64_t queue_index); int64_t calc_iops(const int64_t iops, const int64_t percentage); int64_t calc_weight(const int64_t weight, const int64_t percentage); private: bool is_inited_; - ObMClock category_clocks_[static_cast(ObIOCategory::MAX_CATEGORY)]; - ObMClock other_clock_; + ObSEArray group_clocks_; + ObMClock other_group_clock_; ObAtomIOClock unit_clock_; ObTenantIOConfig io_config_; const ObIOUsage *io_usage_; diff --git a/src/share/io/ob_io_calibration.cpp b/src/share/io/ob_io_calibration.cpp index 5617380636..ffbf62e6f3 100644 --- a/src/share/io/ob_io_calibration.cpp +++ b/src/share/io/ob_io_calibration.cpp @@ -388,7 +388,7 @@ void ObIOBenchRunner::run1() io_info.size_ = load_.size_; io_info.buf_ = ObIOMode::READ == load_.mode_ ? nullptr : write_buf_; io_info.flag_.set_mode(load_.mode_); - io_info.flag_.set_category(ObIOCategory::SYS_IO); + io_info.flag_.set_group_id(0); io_info.flag_.set_wait_event(ObIOMode::READ == load_.mode_ ? ObWaitEventIds::DB_FILE_DATA_READ : ObWaitEventIds::DB_FILE_COMPACT_WRITE); io_info.flag_.set_unlimited(true); diff --git a/src/share/io/ob_io_define.cpp b/src/share/io/ob_io_define.cpp index e15415b8c3..b98e2cf513 100644 --- a/src/share/io/ob_io_define.cpp +++ b/src/share/io/ob_io_define.cpp @@ -16,6 +16,7 @@ #include "share/io/ob_io_define.h" #include "share/io/ob_io_struct.h" #include "share/io/ob_io_manager.h" +#include "share/resource_manager/ob_resource_manager.h" #include "lib/time/ob_time_utility.h" using namespace oceanbase::lib; @@ -56,54 +57,6 @@ ObIOMode oceanbase::common::get_io_mode_enum(const char *mode_string) return mode; } -/****************** IOCategory **********************/ -static const char *log_category_name = "LOG"; -static const char *user_category_name = "USER"; -static const char *sys_category_name = "SYS"; -static const char *prewarm_category_name = "PREWARM"; -static const char *large_query_category_name = "LARGE"; -const char *oceanbase::common::get_io_category_name(ObIOCategory category) -{ - const char *ret_name = "UNKNOWN"; - switch (category) { - case ObIOCategory::LOG_IO: - ret_name = log_category_name; - break; - case ObIOCategory::USER_IO: - ret_name = user_category_name; - break; - case ObIOCategory::SYS_IO: - ret_name = sys_category_name; - break; - case ObIOCategory::PREWARM_IO: - ret_name = prewarm_category_name; - break; - case ObIOCategory::LARGE_QUERY_IO: - ret_name = large_query_category_name; - break; - default: - break; - } - return ret_name; -} - -ObIOCategory oceanbase::common::get_io_category_enum(const char *category_name) -{ - ObIOCategory io_category = ObIOCategory::MAX_CATEGORY; - if (0 == strncasecmp(category_name, log_category_name, strlen(log_category_name))) { - io_category = ObIOCategory::LOG_IO; - } else if (0 == strncasecmp(category_name, user_category_name, strlen(user_category_name))) { - io_category = ObIOCategory::USER_IO; - } else if (0 == strncasecmp(category_name, sys_category_name, strlen(sys_category_name))) { - io_category = ObIOCategory::SYS_IO; - } else if (0 == strncasecmp(category_name, prewarm_category_name, strlen(prewarm_category_name))) { - io_category = ObIOCategory::PREWARM_IO; - } else if (0 == strncasecmp(category_name, large_query_category_name, strlen(large_query_category_name))) { - io_category = ObIOCategory::LARGE_QUERY_IO; - } - return io_category; -} - /****************** IOFlag **********************/ ObIOFlag::ObIOFlag() : flag_(0) @@ -124,7 +77,7 @@ void ObIOFlag::reset() bool ObIOFlag::is_valid() const { return mode_ >= 0 && mode_ < static_cast(ObIOMode::MAX_MODE) - && category_ >= 0 && category_ < static_cast(ObIOCategory::MAX_CATEGORY) + && group_id_ >= 0 && wait_event_id_ > 0; } @@ -138,14 +91,9 @@ ObIOMode ObIOFlag::get_mode() const return static_cast(mode_); } -void ObIOFlag::set_category(ObIOCategory category) +void ObIOFlag::set_group_id(int64_t group_id) { - category_ = static_cast(category); -} - -ObIOCategory ObIOFlag::get_category() const -{ - return static_cast(category_); + group_id_ = group_id; } void ObIOFlag::set_wait_event(int64_t wait_event_id) @@ -153,6 +101,11 @@ void ObIOFlag::set_wait_event(int64_t wait_event_id) wait_event_id_ = wait_event_id; } +int64_t ObIOFlag::get_group_id() const +{ + return group_id_; +} + int64_t ObIOFlag::get_wait_event() const { return wait_event_id_; @@ -465,6 +418,22 @@ int64_t ObIORequest::get_data_size() const return data_size; } +int64_t ObIORequest::get_group_id() const +{ + return io_info_.flag_.get_group_id(); +} + +uint64_t ObIORequest::get_io_usage_index() +{ + uint64_t index = 0; + if (get_group_id() < GROUP_START_ID) { + //other group , do nothing + } else { + index = tenant_io_mgr_.get_ptr()->get_usage_index(get_group_id()); + } + return index; +} + const char *ObIORequest::get_data() { const char *buf = nullptr; @@ -486,11 +455,6 @@ const ObIOFlag &ObIORequest::get_flag() const return io_info_.flag_; } -ObIOCategory ObIORequest::get_category() const -{ - return io_info_.flag_.get_category(); -} - ObIOMode ObIORequest::get_mode() const { return io_info_.flag_.get_mode(); @@ -709,15 +673,16 @@ void ObIORequest::dec_out_ref() ObPhyQueue::ObPhyQueue() : is_inited_(false), + stop_accept_(false), reservation_ts_(INT_MAX64), - category_limitation_ts_(INT_MAX64), + group_limitation_ts_(INT_MAX64), tenant_limitation_ts_(INT_MAX64), proportion_ts_(INT_MAX64), - is_category_ready_(false), + is_group_ready_(false), is_tenant_ready_(false), - category_index_(-1), + queue_index_(-1), reservation_pos_(-1), - category_limitation_pos_(-1), + group_limitation_pos_(-1), tenant_limitation_pos_(-1), proportion_pos_(-1), req_list_() @@ -730,17 +695,17 @@ ObPhyQueue::~ObPhyQueue() destroy(); } -int ObPhyQueue::init(const int index) +int ObPhyQueue::init(const int64_t index) { int ret = OB_SUCCESS; if (OB_UNLIKELY(is_inited_)) { ret = OB_INIT_TWICE; LOG_WARN("phy queue init twice", K(ret), K(is_inited_)); - } else if (index < 0 || index > static_cast(ObIOCategory::MAX_CATEGORY)){ + } else if (index < 0 || index > INT64_MAX){ ret = OB_INVALID_ARGUMENT; LOG_WARN("index out of boundary", K(ret), K(index)); } else { - category_index_=index; + queue_index_ = index; is_inited_ = true; } if (OB_UNLIKELY(!is_inited_)) { @@ -752,33 +717,34 @@ int ObPhyQueue::init(const int index) void ObPhyQueue::destroy() { is_inited_ = false; + stop_accept_ = true; reservation_ts_ = INT_MAX64; - category_limitation_ts_ = INT_MAX64; + group_limitation_ts_ = INT_MAX64; tenant_limitation_ts_ = INT_MAX64; proportion_ts_ = INT_MAX64; - is_category_ready_ = false; + is_group_ready_ = false; is_tenant_ready_ = false; reservation_pos_ = -1; - category_limitation_pos_ = -1; + group_limitation_pos_ = -1; tenant_limitation_pos_ = -1; proportion_pos_ = -1; - category_index_ = -1; + queue_index_ = -1; } void ObPhyQueue::reset_time_info() { reservation_ts_ = INT_MAX64; - category_limitation_ts_ = INT_MAX64; + group_limitation_ts_ = INT_MAX64; tenant_limitation_ts_ = INT_MAX64; proportion_ts_ = INT_MAX64; } void ObPhyQueue::reset_queue_info() { - is_category_ready_ = false; + is_group_ready_ = false; is_tenant_ready_ = false; reservation_pos_ = -1; - category_limitation_pos_ = -1; + group_limitation_pos_ = -1; tenant_limitation_pos_ = -1; proportion_pos_ = -1; } @@ -998,34 +964,56 @@ bool ObTenantIOConfig::UnitConfig::is_valid() const return min_iops_ > 0 && max_iops_ >= min_iops_ && weight_ >= 0; } -ObTenantIOConfig::CategoryConfig::CategoryConfig() - : min_percent_(0), max_percent_(0), weight_percent_(0) +ObTenantIOConfig::GroupConfig::GroupConfig() + : deleted_(false), cleared_(false), min_percent_(0), max_percent_(0), weight_percent_(0) { } -bool ObTenantIOConfig::CategoryConfig::is_valid() const +ObTenantIOConfig::GroupConfig::~GroupConfig() { - return min_percent_ > 0 && max_percent_ >= min_percent_ && max_percent_ <= 100 && weight_percent_ >= 0 && weight_percent_ <= 100; + +} + +bool ObTenantIOConfig::GroupConfig::is_valid() const +{ + return min_percent_ >= 0 && min_percent_ <= 100 && + max_percent_ >= 0 && max_percent_ <= 100 && + weight_percent_ >= 0 && weight_percent_ <= 100 && + max_percent_ >= min_percent_; } ObTenantIOConfig::ObTenantIOConfig() - : memory_limit_(0), callback_thread_count_(0), enable_io_tracer_(false) + : memory_limit_(0), callback_thread_count_(0), group_num_(0), group_ids_(), group_configs_(), + other_group_config_(), group_config_change_(false), enable_io_tracer_(false) { } +ObTenantIOConfig::~ObTenantIOConfig() +{ + destroy(); +} + +void ObTenantIOConfig::destroy() +{ + group_ids_.destroy(); + group_configs_.destroy(); +} + const ObTenantIOConfig &ObTenantIOConfig::default_instance() { static ObTenantIOConfig instance; instance.memory_limit_ = 512L * 1024L * 1024L; // min_tenant_memory: 512M instance.callback_thread_count_ = 8; - instance.other_config_.min_percent_ = 100; - instance.other_config_.max_percent_ = 100; - instance.other_config_.weight_percent_ = 100; + instance.group_num_ = 0; instance.unit_config_.min_iops_ = 10000; instance.unit_config_.max_iops_ = 50000; instance.unit_config_.weight_ = 10000; + instance.other_group_config_.min_percent_ = 100; + instance.other_group_config_.max_percent_ = 100; + instance.other_group_config_.weight_percent_ = 100; + instance.group_config_change_ = false; instance.enable_io_tracer_ = false; return instance; } @@ -1036,21 +1024,22 @@ bool ObTenantIOConfig::is_valid() const if (bret) { int64_t sum_min_percent = 0; int64_t sum_weight_percent = 0; - for (int64_t i = 0; i < static_cast(ObIOCategory::MAX_CATEGORY); ++i) { - if (category_configs_[i].is_valid()) { - sum_min_percent += category_configs_[i].min_percent_; - sum_weight_percent += category_configs_[i].weight_percent_; + for (int64_t i = 0; i < group_num_; ++i) { + if (group_configs_.at(i).is_valid() && !group_configs_.at(i).deleted_) { + sum_min_percent += group_configs_.at(i).min_percent_; + sum_weight_percent += group_configs_.at(i).weight_percent_; } } - sum_min_percent += other_config_.min_percent_; - sum_weight_percent += other_config_.weight_percent_; - bret = 100 == sum_min_percent && 100 == sum_weight_percent; + sum_min_percent += other_group_config_.min_percent_; + sum_weight_percent += other_group_config_.weight_percent_; + bret = 100 >= sum_min_percent && 100 == sum_weight_percent; } return bret; } bool ObTenantIOConfig::operator ==(const ObTenantIOConfig &other) const { + //for basic config, group config is judged by flag bool bret = false; if (memory_limit_ != other.memory_limit_) { LOG_INFO("memory limit not equal", K(memory_limit_), K(other.memory_limit_)); @@ -1062,24 +1051,34 @@ bool ObTenantIOConfig::operator ==(const ObTenantIOConfig &other) const LOG_INFO("unit config not equal", K(unit_config_), K(other.unit_config_)); } else if (enable_io_tracer_ != other.enable_io_tracer_) { LOG_INFO("enable io tracer not equal", K(enable_io_tracer_), K(other.enable_io_tracer_)); - } else { - bret = true; - const int64_t max_category_count = static_cast(ObIOCategory::MAX_CATEGORY); - for (int64_t i = 0; bret && i < max_category_count + 1; ++i) { - const CategoryConfig &category_config = max_category_count == i ? other_config_ : category_configs_[i]; - const CategoryConfig &other_category_config = max_category_count == i ? other.other_config_ : other.category_configs_[i]; - if (category_config.weight_percent_ != other_category_config.weight_percent_ - || category_config.max_percent_ != other_category_config.max_percent_ - || category_config.min_percent_ != other_category_config.min_percent_) { - bret = false; - LOG_INFO("category config not equal", K(category_config), K(other_category_config), K(i), K(max_category_count)); - } - } } return bret; } -int ObTenantIOConfig::parse_category_config(const char *config_str) +int ObTenantIOConfig::deep_copy(const ObTenantIOConfig &other_config) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(group_configs_.assign(other_config.group_configs_))) { + LOG_WARN("fail to assign group_configs", K(ret)); + } else if (OB_FAIL(group_ids_.assign(other_config.group_ids_))) { + LOG_WARN("fail to assign group_ids", K(ret)); + } else { + group_num_ = other_config.group_num_; + other_group_config_ = other_config.other_group_config_; + } + + if (OB_SUCC(ret)) { + memory_limit_ = other_config.memory_limit_; + callback_thread_count_ = other_config.callback_thread_count_; + unit_config_ = other_config.unit_config_; + group_config_change_ = other_config.group_config_change_; + enable_io_tracer_ = other_config.enable_io_tracer_; + } + return ret; +} + +//for unittest and performance test script +int ObTenantIOConfig::parse_group_config(const char *config_str) { int ret = OB_SUCCESS; const int64_t max_config_length = 512; @@ -1103,89 +1102,149 @@ int ObTenantIOConfig::parse_category_config(const char *config_str) } while (OB_SUCC(ret) && pos < len) { const char *tmp_config_str = copied_str + pos; - char category_name[max_config_length] = { 0 }; - ObTenantIOConfig::CategoryConfig tmp_category_config; - int scan_count = sscanf(tmp_config_str, "%s %ld,%ld,%ld", - category_name, - &tmp_category_config.min_percent_, - &tmp_category_config.max_percent_, - &tmp_category_config.weight_percent_); - if (4 != scan_count) { + int64_t group_id = 0; + char group_idx[max_config_length] = { 0 }; + char group_namex[max_config_length] = { 0 }; + ObTenantIOConfig::GroupConfig tmp_group_config; + int scan_count = sscanf(tmp_config_str, "%s %s %ld,%ld,%ld", + group_idx, + group_namex, + &tmp_group_config.min_percent_, + &tmp_group_config.max_percent_, + &tmp_group_config.weight_percent_); + if (5 != scan_count) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("scan current category config failed", K(ret), K(scan_count), KCSTRING(tmp_config_str), K(tmp_category_config)); + LOG_WARN("scan current group config failed", K(ret), K(scan_count), KCSTRING(tmp_config_str), K(tmp_group_config)); } else { - ObIOCategory io_category = get_io_category_enum(category_name); - ObTenantIOConfig::CategoryConfig &target_category_config = ObIOCategory::MAX_CATEGORY == io_category ? other_config_ : category_configs_[static_cast(io_category)]; - target_category_config = tmp_category_config; - pos += strlen(tmp_config_str) + 1; + const int64_t group_id = atoi(group_idx); + if (0 == group_id) { + // other group + other_group_config_ = tmp_group_config; + pos += strlen(tmp_config_str) + 1; + } else { + // add single group + if (OB_FAIL(group_configs_.push_back(tmp_group_config))) { + LOG_WARN("push back group config failed", K(ret), K(tmp_group_config)); + } else if (OB_FAIL(group_ids_.push_back(group_id))) { + LOG_WARN("push back group id failed", K(ret), K(group_id)); + } else { + ATOMIC_INC(&group_num_); + pos += strlen(tmp_config_str) + 1; + } + } } } if (OB_SUCC(ret)) { - // decide the config of other category + // decide the config of other group int64_t sum_min_percent = 0; int64_t sum_weight_percent = 0; - for (int64_t i = 0; i < static_cast(ObIOCategory::MAX_CATEGORY); ++i) { - if (category_configs_[i].is_valid()) { - sum_min_percent += category_configs_[i].min_percent_; - sum_weight_percent += category_configs_[i].weight_percent_; + for (int64_t i = 0; i < group_configs_.count(); ++i) { + if (group_configs_.at(i).is_valid()) { + sum_min_percent += group_configs_.at(i).min_percent_; + sum_weight_percent += group_configs_.at(i).weight_percent_; } } - if (0 == other_config_.min_percent_ - && 0 == other_config_.max_percent_ - && 0 == other_config_.weight_percent_ + if (0 == other_group_config_.min_percent_ + && 0 == other_group_config_.max_percent_ + && 0 == other_group_config_.weight_percent_ && sum_min_percent < 100 && sum_weight_percent < 100) { - other_config_.min_percent_ = 100 - sum_min_percent; - other_config_.max_percent_ = 100; - other_config_.weight_percent_ = 100 - sum_weight_percent; + other_group_config_.min_percent_ = 100 - sum_min_percent; + other_group_config_.max_percent_ = 100; + other_group_config_.weight_percent_ = 100 - sum_weight_percent; } } } return ret; } -int ObTenantIOConfig::get_category_config(const ObIOCategory category, int64_t &min_iops, int64_t &max_iops, int64_t &iops_weight) const +int ObTenantIOConfig::add_single_group_config(const uint64_t tenant_id, const int64_t group_id, int64_t min_percent, int64_t max_percent, int64_t weight_percent) +{ + int ret = OB_SUCCESS; + if (group_id < GROUP_START_ID || !is_valid_tenant_id(tenant_id) || + min_percent < 0 || min_percent > 100 || + max_percent < 0 || max_percent > 100 || + weight_percent < 0 || weight_percent > 100 || + min_percent > max_percent) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid group config", K(ret), K(tenant_id), K(group_id), K(min_percent), K(max_percent), K(weight_percent)); + } else { + ObTenantIOConfig::GroupConfig tmp_group_config; + tmp_group_config.min_percent_ = min_percent; + tmp_group_config.max_percent_ = max_percent; + tmp_group_config.weight_percent_ = weight_percent; + if (OB_UNLIKELY(!tmp_group_config.is_valid())) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid group config", K(ret), K(tmp_group_config)); + } else if (OB_FAIL(group_configs_.push_back(tmp_group_config))) { + LOG_WARN("push back group config failed", K(ret), K(tmp_group_config)); + } else if (OB_FAIL(group_ids_.push_back(group_id))) { + group_configs_.pop_back(); + LOG_WARN("push back group id failed", K(ret), K(tmp_group_config)); + } + } + return ret; +} + +int ObTenantIOConfig::get_group_config(const uint64_t index, int64_t &min_iops, int64_t &max_iops, int64_t &iops_weight) const { int ret = OB_SUCCESS; min_iops = 0; max_iops = 0; iops_weight = 0; - if (OB_UNLIKELY(category >= ObIOCategory::MAX_CATEGORY)) { + if (OB_UNLIKELY(index < 0 || (index >= group_configs_.count() && index != INT64_MAX))) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid argument", K(ret)); + LOG_WARN("invalid argument", K(ret), K(index)); + } else if (index == INT64_MAX) { + //OTHER GROUPS + const GroupConfig &group_config = other_group_config_; + min_iops = unit_config_.min_iops_ * group_config.min_percent_ / 100; + max_iops = unit_config_.max_iops_ * group_config.max_percent_ / 100; + iops_weight = unit_config_.weight_ * group_config.weight_percent_ / 100; } else { - const CategoryConfig &category_config = category_configs_[static_cast(category)].is_valid() - ? category_configs_[static_cast(category)] : other_config_; - min_iops = unit_config_.min_iops_ * category_config.min_percent_ / 100; - max_iops = unit_config_.max_iops_ * category_config.max_percent_ / 100; - iops_weight = unit_config_.weight_ * category_config.weight_percent_ / 100; + const GroupConfig &group_config = group_configs_.at(index); + min_iops = unit_config_.min_iops_ * group_config.min_percent_ / 100; + max_iops = unit_config_.max_iops_ * group_config.max_percent_ / 100; + iops_weight = unit_config_.weight_ * group_config.weight_percent_ / 100; } return ret; } +int64_t ObTenantIOConfig::get_all_group_num() const +{ + int64_t all_group_num = ATOMIC_LOAD(&group_num_) +1; + //include other_groups + return all_group_num; +} + int64_t ObTenantIOConfig::to_string(char* buf, const int64_t buf_len) const { int64_t pos = 0; J_OBJ_START(); - J_KV(K(memory_limit_), K(callback_thread_count_), K(unit_config_), K_(enable_io_tracer)); - // if self invalid, print all category configs, otherwise, only print valid category configs + J_KV(K(group_num_), K(memory_limit_), K(callback_thread_count_), K(unit_config_), K_(enable_io_tracer)); + // if self invalid, print all group configs, otherwise, only print valid group configs const bool self_valid = is_valid(); - BUF_PRINTF(", category_configs:["); + BUF_PRINTF(", group_configs:["); bool need_comma = false; - for (int64_t i = 0; i < static_cast(ObIOCategory::MAX_CATEGORY); ++i) { - if (!self_valid || category_configs_[i].is_valid()) { + for (int64_t i = 0; i < group_configs_.count(); ++i) { + if (group_configs_.at(i).deleted_ || group_configs_.at(i).cleared_) { + continue; + } else if (!self_valid || group_configs_.at(i).is_valid()) { if (need_comma) { J_COMMA(); } - J_KV(get_io_category_name(static_cast(i)), category_configs_[i]); + BUF_PRINTF("group_id = "); + char group_id[8]; + snprintf(group_id, sizeof(group_id), "%ld", group_ids_.at(i)); + J_KV(group_id, group_configs_.at(i)); need_comma = true; } } - if (!self_valid || other_config_.is_valid()) { + if (!self_valid || other_group_config_.is_valid()) { if (need_comma) { J_COMMA(); } - J_KV("other", other_config_); + J_KV("other_groups", other_group_config_); } BUF_PRINTF("]"); J_OBJ_END(); @@ -1284,7 +1343,6 @@ int ObMClockQueue::pop_phyqueue(ObIORequest *&req, int64_t &deadline_ts) LOG_WARN("not init", K(ret), K(is_inited_)); } else if (r_heap_.empty()){ ret = OB_ENTRY_NOT_EXIST; - LOG_WARN("r heap is empty, phy queue not init", K(ret)); } else { ObPhyQueue *tmp_phy_queue = r_heap_.top(); if (OB_ISNULL(tmp_phy_queue)) { @@ -1342,15 +1400,15 @@ int ObMClockQueue::remove_from_heap(ObPhyQueue *phy_queue) LOG_WARN("phy_queue is null", K(ret), KP(phy_queue)); } else if (OB_FAIL(r_heap_.remove(phy_queue))) { LOG_WARN("remove phy queue from r heap failed", K(ret)); - } else if (!phy_queue->is_category_ready_ && !phy_queue->is_tenant_ready_) { + } else if (!phy_queue->is_group_ready_ && !phy_queue->is_tenant_ready_) { if (OB_FAIL(cl_heap_.remove(phy_queue))) { LOG_WARN("remove phy queue from cl heap failed", K(ret)); } - } else if (phy_queue->is_category_ready_ && !phy_queue->is_tenant_ready_) { + } else if (phy_queue->is_group_ready_ && !phy_queue->is_tenant_ready_) { if (OB_FAIL(tl_heap_.remove(phy_queue))) { LOG_WARN("remove phy queue from tl heap failed", K(ret)); } - } else if (phy_queue->is_category_ready_ && phy_queue->is_tenant_ready_) { + } else if (phy_queue->is_group_ready_ && phy_queue->is_tenant_ready_) { if (OB_FAIL(ready_heap_.remove(phy_queue))) { LOG_WARN("remove phy queue from ready heap failed", K(ret)); } @@ -1370,12 +1428,12 @@ int ObMClockQueue::pop_with_ready_queue(const int64_t current_ts, ObIORequest *& tmp_phy_queue = cl_heap_.top(); deadline_ts = 0 == iter_count ? tmp_phy_queue->tenant_limitation_ts_ : deadline_ts; ++iter_count; - if (tmp_phy_queue->category_limitation_ts_ > current_ts) { + if (tmp_phy_queue->group_limitation_ts_ > current_ts) { break; } else if (OB_FAIL(cl_heap_.pop())) { LOG_WARN("remove PhyQueue from c_limitation queue failed", K(ret)); } else { - tmp_phy_queue->is_category_ready_ = true; + tmp_phy_queue->is_group_ready_ = true; if (tmp_phy_queue->tenant_limitation_ts_ <= current_ts) { tmp_phy_queue->is_tenant_ready_ = true; if (OB_FAIL(ready_heap_.push(tmp_phy_queue))) { diff --git a/src/share/io/ob_io_define.h b/src/share/io/ob_io_define.h index a86f08d536..a91bc94282 100644 --- a/src/share/io/ob_io_define.h +++ b/src/share/io/ob_io_define.h @@ -17,9 +17,12 @@ #include "lib/profile/ob_trace_id.h" #include "lib/lock/ob_thread_cond.h" #include "lib/container/ob_rbtree.h" -#include "common/storage/ob_io_device.h" #include "lib/list/ob_list.h" #include "lib/container/ob_heap.h" +#include "lib/container/ob_array_iterator.h" +#include "lib/container/ob_array_wrap.h" +#include "common/storage/ob_io_device.h" +#include "share/resource_manager/ob_resource_plan_info.h" namespace oceanbase { @@ -28,7 +31,8 @@ namespace common static constexpr int64_t DEFAULT_IO_WAIT_TIME_MS = 5000L; // 5s static constexpr int64_t MAX_IO_WAIT_TIME_MS = 300L * 1000L; // 5min - +static constexpr int64_t GROUP_START_ID = 10000L; // start id = 10000 +static constexpr int64_t GROUP_START_NUM = 8L; enum class ObIOMode : uint8_t { READ = 0, @@ -39,19 +43,6 @@ enum class ObIOMode : uint8_t const char *get_io_mode_string(const ObIOMode mode); ObIOMode get_io_mode_enum(const char *mode_string); -enum class ObIOCategory : uint8_t -{ - LOG_IO = 0, - USER_IO = 1, - SYS_IO = 2, - PREWARM_IO = 3, - LARGE_QUERY_IO = 4, - MAX_CATEGORY -}; - -const char *get_io_category_name(ObIOCategory category); -ObIOCategory get_io_category_enum(const char *category_name); - struct ObIOFlag final { public: @@ -61,9 +52,9 @@ public: bool is_valid() const; void set_mode(ObIOMode mode); ObIOMode get_mode() const; - void set_category(ObIOCategory category); - ObIOCategory get_category() const; + void set_group_id(int64_t group_id); void set_wait_event(int64_t wait_event_id); + int64_t get_group_id() const; int64_t get_wait_event() const; void set_read(); bool is_read() const; @@ -75,16 +66,15 @@ public: void set_unlimited(const bool is_unlimited = true); bool is_unlimited() const; TO_STRING_KV("mode", common::get_io_mode_string(static_cast(mode_)), - "category", common::get_io_category_name(static_cast(category_)), - K(wait_event_id_), K(is_sync_), K(is_unlimited_), K(reserved_)); + K(group_id_), K(wait_event_id_), K(is_sync_), K(is_unlimited_), K(reserved_)); private: static constexpr int64_t IO_MODE_BIT = 4; // read, write, append - static constexpr int64_t IO_CATEGORY_BIT = 8; // clog, user, prewarm, large query, etc + static constexpr int64_t IO_GROUP_ID_BIT = 16; // for consumer group in resource manager static constexpr int64_t IO_WAIT_EVENT_BIT = 32; // for performance monitor static constexpr int64_t IO_SYNC_FLAG_BIT = 1; // indicate if the caller is waiting io finished static constexpr int64_t IO_UNLIMITED_FLAG_BIT = 1; // indicate if the io is unlimited static constexpr int64_t IO_RESERVED_BIT = 64 - IO_MODE_BIT - - IO_CATEGORY_BIT + - IO_GROUP_ID_BIT - IO_WAIT_EVENT_BIT - IO_SYNC_FLAG_BIT - IO_UNLIMITED_FLAG_BIT; @@ -93,7 +83,7 @@ private: int64_t flag_; struct { int64_t mode_ : IO_MODE_BIT; - int64_t category_ : IO_CATEGORY_BIT; + int64_t group_id_ : IO_GROUP_ID_BIT; int64_t wait_event_id_ : IO_WAIT_EVENT_BIT; bool is_sync_ : IO_SYNC_FLAG_BIT; bool is_unlimited_ : IO_UNLIMITED_FLAG_BIT; @@ -215,9 +205,10 @@ public: int init(const ObIOInfo &info); virtual void destroy(); int64_t get_data_size() const; + int64_t get_group_id() const; + uint64_t get_io_usage_index(); const char *get_data(); const ObIOFlag &get_flag() const; - ObIOCategory get_category() const; ObIOMode get_mode() const; void cancel(); int alloc_io_buf(); @@ -267,23 +258,26 @@ class ObPhyQueue final public: ObPhyQueue(); ~ObPhyQueue(); - int init(const int index); + bool is_stop_accept() { return stop_accept_; } + int init(const int64_t index); void destroy(); void reset_time_info(); void reset_queue_info(); + void set_stop_accept() { stop_accept_ = true; } public: typedef common::ObDList IOReqList; - TO_STRING_KV(K_(reservation_ts), K_(category_limitation_ts), K_(tenant_limitation_ts)); + TO_STRING_KV(K_(reservation_ts), K_(group_limitation_ts), K_(tenant_limitation_ts), K_(stop_accept)); bool is_inited_; + bool stop_accept_; int64_t reservation_ts_; - int64_t category_limitation_ts_; + int64_t group_limitation_ts_; int64_t tenant_limitation_ts_; int64_t proportion_ts_; - bool is_category_ready_; + bool is_group_ready_; bool is_tenant_ready_; - int category_index_; + int64_t queue_index_; //index in array, INT64_MAX means other int64_t reservation_pos_; - int64_t category_limitation_pos_; + int64_t group_limitation_pos_; int64_t tenant_limitation_pos_; int64_t proportion_pos_; IOReqList req_list_; @@ -329,29 +323,44 @@ public: int64_t max_iops_; int64_t weight_; }; - struct CategoryConfig + + struct GroupConfig { - CategoryConfig(); + public: + GroupConfig(); + ~GroupConfig(); bool is_valid() const; - TO_STRING_KV(K_(min_percent), K_(max_percent), K_(weight_percent)); + TO_STRING_KV(K_(deleted), K_(cleared),K_(min_percent), K_(max_percent), K_(weight_percent)); + public: + bool deleted_; //group被删除的标记 + bool cleared_; //group被清零的标记,以后有新的directive就会重置 int64_t min_percent_; int64_t max_percent_; int64_t weight_percent_; }; + public: ObTenantIOConfig(); + ~ObTenantIOConfig(); + void destroy(); static const ObTenantIOConfig &default_instance(); bool is_valid() const; bool operator ==(const ObTenantIOConfig &other) const; - int parse_category_config(const char *config_str); - int get_category_config(const ObIOCategory category, int64_t &min_iops, int64_t &max_iops, int64_t &iops_weight) const; + int deep_copy(const ObTenantIOConfig &other_config); + int parse_group_config(const char *config_str); + int add_single_group_config(const uint64_t tenant_id, const int64_t group_id, int64_t min_percent, int64_t max_percent, int64_t weight_percent); + int get_group_config(const uint64_t index, int64_t &min_iops, int64_t &max_iops, int64_t &iops_weight) const; + int64_t get_all_group_num() const; int64_t to_string(char* buf, const int64_t buf_len) const; public: int64_t memory_limit_; int64_t callback_thread_count_; + int64_t group_num_; UnitConfig unit_config_; - CategoryConfig category_configs_[static_cast(ObIOCategory::MAX_CATEGORY)]; - CategoryConfig other_config_; + ObSEArray group_ids_; + ObSEArray group_configs_; + GroupConfig other_group_config_; + bool group_config_change_; bool enable_io_tracer_; }; @@ -366,18 +375,6 @@ struct ObAtomIOClock final int64_t last_ns_; // the unit is nano sescond for max iops of 1 billion }; -class ObIOClock -{ -public: - ObIOClock() {} - virtual ~ObIOClock() {} - virtual int init(const ObTenantIOConfig &io_config, const ObIOUsage *io_usage) = 0; - virtual void destroy() = 0; - virtual int update_io_config(const ObTenantIOConfig &io_config) = 0; - virtual int sync_clocks(ObIArray &io_clocks) = 0; - DECLARE_PURE_VIRTUAL_TO_STRING; -}; - class ObIOQueue { public: @@ -413,11 +410,11 @@ private: private: bool is_inited_; HeapCompare r_cmp_; - HeapCompare cl_cmp_; + HeapCompare cl_cmp_; HeapCompare tl_cmp_; HeapCompare p_cmp_; ObRemovableHeap, &ObPhyQueue::reservation_pos_> r_heap_; - ObRemovableHeap, &ObPhyQueue::category_limitation_pos_> cl_heap_; + ObRemovableHeap, &ObPhyQueue::group_limitation_pos_> cl_heap_; ObRemovableHeap, &ObPhyQueue::tenant_limitation_pos_> tl_heap_; ObRemovableHeap, &ObPhyQueue::proportion_pos_> ready_heap_; }; diff --git a/src/share/io/ob_io_manager.cpp b/src/share/io/ob_io_manager.cpp index fd10f86f57..994e9824ef 100644 --- a/src/share/io/ob_io_manager.cpp +++ b/src/share/io/ob_io_manager.cpp @@ -54,16 +54,16 @@ int ObIOManager::init(const int64_t memory_limit, if (OB_UNLIKELY(is_inited_)) { ret = OB_INIT_TWICE; LOG_WARN("init twice", K(ret), K(is_inited_)); - } else if (OB_UNLIKELY(memory_limit <= 0|| queue_depth <= 0 || schedule_queue_count <= 0 || schedule_media_id < 0)) { + } else if (OB_UNLIKELY(memory_limit <= 0|| schedule_queue_count <= 0 || schedule_media_id < 0)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid arguments", K(ret), K(memory_limit), K(queue_depth), K(schedule_queue_count), K(schedule_media_id)); + LOG_WARN("invalid arguments", K(ret), K(memory_limit), K(schedule_queue_count), K(schedule_media_id)); } else if (OB_FAIL(allocator_.init(OB_MALLOC_MIDDLE_BLOCK_SIZE, "IO_MGR", OB_SERVER_TENANT_ID, memory_limit))) { LOG_WARN("init io allocator failed", K(ret)); } else if (OB_FAIL(channel_map_.create(7, "IO_CHANNEL_MAP"))) { LOG_WARN("create channel map failed", K(ret)); } else if (OB_FAIL(tenant_map_.create(7, "IO_TENANT_MAP"))) { LOG_WARN("create tenant map failed", K(ret)); - } else if (OB_FAIL(io_scheduler_.init(schedule_queue_count, queue_depth, schedule_media_id))) { + } else if (OB_FAIL(io_scheduler_.init(schedule_queue_count, schedule_media_id))) { LOG_WARN("init io scheduler failed", K(ret)); } else if (OB_FAIL(fault_detector_.init())) { LOG_WARN("init io fault detector failed", K(ret)); @@ -299,7 +299,7 @@ int ObIOManager::adjust_tenant_clock() LOG_WARN("not init", K(ret), K(is_inited_)); } else { ObTenantIOManager *tenant_io_mgr = nullptr; - ObArray io_clocks; + ObArray io_clocks; DRWLock::RDLockGuard guard(tenant_map_lock_); hash::ObHashMap::iterator iter = tenant_map_.begin(); for (; OB_SUCC(ret) && iter != tenant_map_.end(); ++iter) { @@ -447,7 +447,7 @@ int ObIOManager::add_tenant_io_manager(const uint64_t tenant_id, const ObTenantI } else if (FALSE_IT(tenant_io_mgr = new (buf) ObTenantIOManager())) { } else if (OB_FAIL(tenant_io_mgr->init(tenant_id, tenant_io_config, &io_scheduler_))) { LOG_WARN("init tenant io manager failed", K(ret), K(tenant_id), K(tenant_io_config)); - } else if (OB_FAIL(io_scheduler_.add_tenant_map(tenant_id))) { + } else if (OB_FAIL(io_scheduler_.init_group_queues(tenant_id, tenant_io_mgr->get_group_num()))) { LOG_WARN("init io map failed", K(ret), K(tenant_id), K(tenant_io_config)); } else if (OB_FAIL(tenant_io_mgr->start())) { LOG_WARN("start tenant io manager failed", K(ret), K(tenant_id)); @@ -457,7 +457,7 @@ int ObIOManager::add_tenant_io_manager(const uint64_t tenant_id, const ObTenantI if (OB_FAIL(tenant_map_.set_refactored(tenant_id, tenant_io_mgr))) { LOG_WARN("put into tenant map failed", K(ret), K(tenant_id), KP(tenant_io_mgr)); } else { - LOG_INFO("add tenant io manager", K(tenant_id), KPC(tenant_io_mgr)); + LOG_INFO("add tenant io manager success", K(tenant_id), KPC(tenant_io_mgr)); tenant_io_mgr = nullptr; } } @@ -484,10 +484,12 @@ int ObIOManager::remove_tenant_io_manager(const uint64_t tenant_id) } else if (OB_UNLIKELY(tenant_id <= 0)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(tenant_id)); + } else if (OB_FAIL(io_scheduler_.remove_phyqueues(tenant_id))) { + LOG_WARN("remove phy_queues from map failed", K(ret), K(tenant_id)); } else { DRWLock::WRLockGuard guard(tenant_map_lock_); if (OB_FAIL(tenant_map_.erase_refactored(tenant_id, &tenant_io_mgr))) { - LOG_WARN("remove tenant io manager failed", K(ret), K(tenant_id)); + LOG_WARN("remove tenant io manager failed", K(ret), K(tenant_id), KP(tenant_io_mgr)); } else { LOG_INFO("remove tenant io manager success", K(tenant_id), KP(tenant_io_mgr)); } @@ -495,9 +497,6 @@ int ObIOManager::remove_tenant_io_manager(const uint64_t tenant_id) if (OB_SUCC(ret) && nullptr != tenant_io_mgr) { tenant_io_mgr->stop(); tenant_io_mgr->dec_ref(); - if (OB_FAIL(io_scheduler_.remove_tenant_map(tenant_id))) { - LOG_WARN("remove phy_queues from map failed", K(ret), K(tenant_id)); - } } return ret; } @@ -509,12 +508,15 @@ int ObIOManager::refresh_tenant_io_config(const uint64_t tenant_id, const ObTena if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; LOG_WARN("not init", K(ret), K(is_inited_)); - } else if (OB_UNLIKELY(tenant_id <= 0 || !tenant_io_config.is_valid())) { + } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id) || + tenant_io_config.memory_limit_ <= 0 || + tenant_io_config.callback_thread_count_ <= 0 || + !tenant_io_config.unit_config_.is_valid())) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid argument", K(ret), K(tenant_id), K(tenant_io_config)); + LOG_WARN("invalid argument", K(ret), K(tenant_id)); } else if (OB_FAIL(get_tenant_io_manager(tenant_id, tenant_holder))) { LOG_WARN("get tenant io manager failed", K(ret), K(tenant_id)); - } else if (OB_FAIL(tenant_holder.get_ptr()->update_io_config(tenant_io_config))) { + } else if (OB_FAIL(tenant_holder.get_ptr()->update_basic_io_config(tenant_io_config))) { LOG_WARN("update tenant io config failed", K(ret), K(tenant_id), K(tenant_io_config)); } return ret; @@ -547,6 +549,10 @@ int ObIOManager::get_tenant_ids(ObIArray &tenant_ids) return ret; } +ObIOScheduler *ObIOManager::get_scheduler() +{ + return &io_scheduler_; +} /****************** TenantIOManager **********************/ int ObTenantIOManager::mtl_init(ObTenantIOManager *&io_service) @@ -594,7 +600,8 @@ ObTenantIOManager::ObTenantIOManager() io_clock_(nullptr), io_allocator_(), io_scheduler_(nullptr), - callback_mgr_() + callback_mgr_(), + io_config_lock_(ObLatchIds::TENANT_IO_CONFIG_LOCK) { } @@ -612,7 +619,7 @@ int ObTenantIOManager::init(const uint64_t tenant_id, if (OB_UNLIKELY(is_inited_)) { ret = OB_INIT_TWICE; LOG_WARN("init twice", K(ret), K(is_inited_)); - } else if (OB_UNLIKELY(tenant_id <= 0 + } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id) || !io_config.is_valid() || nullptr == io_scheduler)) { ret = OB_INVALID_ARGUMENT; @@ -623,11 +630,16 @@ int ObTenantIOManager::init(const uint64_t tenant_id, LOG_WARN("init io tracer failed", K(ret)); } else if (OB_FAIL(alloc_io_clock(io_allocator_, io_clock_))) { LOG_WARN("alloc io clock failed", K(ret)); + } else if (OB_FAIL(io_usage_.init(io_config.group_num_))) { + LOG_WARN("init io usage failed", K(ret), K(io_usage_)); } else if (OB_FAIL(io_clock_->init(io_config, &io_usage_))) { LOG_WARN("init io clock failed", K(ret), K(io_config)); + } else if (OB_FAIL(init_group_index_map(io_config))) { + LOG_WARN("init group map failed", K(ret)); + } else if (OB_FAIL(io_config_.deep_copy(io_config))) { + LOG_WARN("copy io config failed", K(ret), K(io_config_)); } else { tenant_id_ = tenant_id; - io_config_ = io_config; io_scheduler_ = io_scheduler; is_inited_ = true; } @@ -640,16 +652,18 @@ int ObTenantIOManager::init(const uint64_t tenant_id, void ObTenantIOManager::destroy() { ATOMIC_SET(&is_working_, false); - callback_mgr_.destroy(); + if (OB_NOT_NULL(io_clock_)) { io_clock_->destroy(); io_allocator_.free(io_clock_); io_clock_ = nullptr; } + callback_mgr_.destroy(); io_tracer_.destroy(); io_scheduler_ = nullptr; tenant_id_ = 0; io_allocator_.destroy(); + group_id_index_map_.destroy(); is_inited_ = false; } @@ -724,34 +738,33 @@ int ObTenantIOManager::enqueue_callback(ObIORequest &req) return ret; } -int ObTenantIOManager::update_io_config(const ObTenantIOConfig &io_config) +int ObTenantIOManager::update_basic_io_config(const ObTenantIOConfig &io_config) { int ret = OB_SUCCESS; if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; LOG_WARN("not init", K(ret), K(is_inited_)); - } else if (OB_UNLIKELY(!io_config.is_valid())) { - ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid argument", K(ret), K(io_config)); } else if (OB_UNLIKELY(!is_working())) { ret = OB_STATE_NOT_MATCH; LOG_WARN("tenant not working", K(ret), K(tenant_id_)); } else if (io_config == io_config_) { - // config not change, do nothing + // basic config not change, do nothing } else { if (OB_FAIL(io_allocator_.update_memory_limit(io_config.memory_limit_))) { LOG_WARN("update memory limit failed", K(ret), K(io_config.memory_limit_)); } else if (OB_FAIL(callback_mgr_.update_thread_count(io_config.callback_thread_count_))) { LOG_WARN("callback manager adjust thread failed", K(ret), K(io_config)); - } else if (OB_FAIL(io_clock_->update_io_config(io_config))) { - LOG_WARN("update tenant io config failed", K(ret), K(io_config)); } else { - LOG_INFO("update io config success", K(tenant_id_), "old_config", io_config_, "new_config", io_config); - io_config_ = io_config; + // just update basic config + DRWLock::WRLockGuard guard(io_config_lock_); + io_config_.memory_limit_ = io_config.memory_limit_; + io_config_.callback_thread_count_ = io_config.callback_thread_count_; + io_config_.unit_config_ = io_config.unit_config_; if (!io_config_.enable_io_tracer_) { ATOMIC_SET(&io_config_.enable_io_tracer_, false); io_tracer_.reuse(); } + LOG_INFO("update basic io config success", K(tenant_id_), K(io_config_)); } } return ret; @@ -776,7 +789,7 @@ int ObTenantIOManager::alloc_io_request(ObIAllocator &allocator, const int64_t c return ret; } -int ObTenantIOManager::alloc_io_clock(ObIAllocator &allocator, ObIOClock *&io_clock) +int ObTenantIOManager::alloc_io_clock(ObIAllocator &allocator, ObTenantIOClock *&io_clock) { int ret = OB_SUCCESS; io_clock = nullptr; @@ -790,6 +803,319 @@ int ObTenantIOManager::alloc_io_clock(ObIAllocator &allocator, ObIOClock *&io_cl return ret; } +int ObTenantIOManager::init_group_index_map(const ObTenantIOConfig &io_config) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(group_id_index_map_.create(7, "GROUP_INDEX_MAP"))) { + LOG_WARN("create group index map failed", K(ret)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < io_config.group_num_; ++i) { + if(OB_FAIL(group_id_index_map_.set_refactored(io_config.group_ids_.at(i), i, 1 /*overwrite*/))) { + LOG_WARN("init group_index_map failed", K(ret), K(i)); + } + } + } + return ret; +} + +int ObTenantIOManager::get_group_index(const int64_t group_id, uint64_t &index) +{ + int ret = group_id_index_map_.get_refactored(group_id, index); + if (OB_FAIL(ret)) { + if(OB_HASH_NOT_EXIST != ret) { + LOG_WARN("get index from map failed", K(ret), K(group_id), K(index)); + } + } else if (OB_UNLIKELY(index == INT64_MAX)) { + //index == INT64_MAX means group has been deleted + ret = OB_STATE_NOT_MATCH; + } + return ret; +} + +int ObTenantIOManager::modify_group_io_config(const uint64_t index, + const int64_t min_percent, + const int64_t max_percent, + const int64_t weight_percent, + const bool deleted, + const bool cleared) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_working())) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("tenant not working", K(ret), K(tenant_id_)); + } else if (index < 0 || (index >= io_config_.group_configs_.count() && index != INT64_MAX) || + min_percent < 0 || min_percent > 100 || max_percent < 0 || max_percent > 100 || + max_percent < min_percent || weight_percent < 0 || weight_percent > 100) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid group config", K(index), K(min_percent), K(max_percent), K(weight_percent)); + } else { + if (INT64_MAX == index) { + // other groups + io_config_.other_group_config_.min_percent_ = min_percent; + io_config_.other_group_config_.max_percent_ = max_percent; + io_config_.other_group_config_.weight_percent_ = weight_percent; + io_config_.other_group_config_.cleared_ = cleared; + io_config_.other_group_config_.deleted_ = deleted; + } else { + io_config_.group_configs_.at(index).min_percent_ = min_percent; + io_config_.group_configs_.at(index).max_percent_ = max_percent; + io_config_.group_configs_.at(index).weight_percent_ = weight_percent; + io_config_.group_configs_.at(index).cleared_ = cleared; + io_config_.group_configs_.at(index).deleted_ = deleted; + } + io_config_.group_config_change_ = true; + } + return ret; +} + +int ObTenantIOManager::modify_io_config(const uint64_t group_id, + const int64_t min_percent, + const int64_t max_percent, + const int64_t weight_percent) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_working())) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("tenant not working", K(ret), K(tenant_id_)); + } else { + uint64_t index = INT64_MAX; + DRWLock::WRLockGuard guard(io_config_lock_); + if (group_id < GROUP_START_ID && group_id > 0) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid group id", K(ret), K(tenant_id_), K(group_id)); + } else if (min_percent < 0 || min_percent > 100 || + max_percent < 0 || max_percent > 100 || + max_percent < min_percent || weight_percent < 0 || weight_percent > 100) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid group config", K(ret), K(tenant_id_), K(min_percent), K(max_percent), K(weight_percent)); + } else if (0 == group_id) { + //1. modify OTHER GROUPS + if (io_config_.other_group_config_.min_percent_ == min_percent && + io_config_.other_group_config_.max_percent_ == max_percent && + io_config_.other_group_config_.weight_percent_ == weight_percent) { + //config did not change, do nothing + } else if (OB_FAIL(modify_group_io_config(INT64_MAX, min_percent, max_percent, weight_percent))) { + LOG_WARN("modify group io config failed", K(ret), K(tenant_id_), K(min_percent), K(max_percent), K(weight_percent)); + } + } else if (OB_FAIL(get_group_index(group_id, index))) { + if (OB_STATE_NOT_MATCH == ret) { + //group has been deleted, do nothing + LOG_INFO("group has been deleted before flush directive", K(group_id), K(index)); + ret = OB_SUCCESS; + } else if (OB_HASH_NOT_EXIST == ret) { + //2. add new group + if (OB_FAIL(add_group_io_config(group_id, min_percent, max_percent, weight_percent))) { + LOG_WARN("add consumer group failed", K(ret), K(tenant_id_), K(group_id)); + } else { + io_config_.group_config_change_ = true; + } + } else { + LOG_WARN("get group index failed", K(ret), K(tenant_id_), K(group_id)); + } + } else { + //3. modify exits groups + if (index < 0 || (index >= io_config_.group_configs_.count() && index != INT64_MAX)) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid index", K(ret), K(index), K(io_config_.group_configs_.count())); + } else if (io_config_.group_configs_.at(index).min_percent_ == min_percent && + io_config_.group_configs_.at(index).max_percent_ == max_percent && + io_config_.group_configs_.at(index).weight_percent_ == weight_percent) { + //config did not change, do nothing + } else { + if (io_config_.group_configs_.at(index).cleared_) { + //并发状态可能先被clear + io_config_.group_configs_.at(index).cleared_ = false; + } else if (OB_FAIL(modify_group_io_config(index, min_percent, max_percent, weight_percent))) { + LOG_WARN("modify group io config failed", K(ret), K(tenant_id_), K(min_percent), K(max_percent), K(weight_percent)); + } + } + } + } + return ret; +} + +int ObTenantIOManager::add_group_io_config(const int64_t group_id, + const int64_t min_percent, + const int64_t max_percent, + const int64_t weight_percent) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_working())) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("tenant not working", K(ret), K(tenant_id_)); + } else if (group_id < GROUP_START_ID || min_percent < 0 || min_percent > 100 || + max_percent < 0 || max_percent > 100 || max_percent < min_percent || + weight_percent < 0 || weight_percent > 100) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid group config", K(ret), K(group_id), K(min_percent), K(max_percent), K(weight_percent)); + } else { + int64_t group_num = ATOMIC_LOAD(&io_config_.group_num_); + if (OB_FAIL(io_config_.add_single_group_config(tenant_id_, group_id, min_percent, max_percent, weight_percent))) { + LOG_WARN("init single group failed", K(group_id)); + } else if (OB_FAIL(group_id_index_map_.set_refactored(group_id, group_num, 1))) {// overwrite + LOG_WARN("set group_id and index into map failed", K(ret), K(group_id), K(group_num)); + } else { + ATOMIC_INC(&io_config_.group_num_); + LOG_INFO("add group config success", K(group_id), K(io_config_)); + } + } + return ret; +} + +int ObTenantIOManager::reset_all_group_config() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_working())) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("tenant not working", K(ret), K(tenant_id_)); + } else { + DRWLock::WRLockGuard guard(io_config_lock_); + for (int64_t i = 0; i < io_config_.group_num_; ++i) { + if(io_config_.group_configs_.at(i).deleted_) { + //do nothing + } else if (OB_FAIL(modify_group_io_config(i, 0, 0, 0, false, true/*cleared*/))) { + LOG_WARN("modify group io config failed", K(ret), K(i)); + } + } + if(OB_SUCC(ret)) { + LOG_INFO ("stop all group io control success when delete plan", K(tenant_id_), K(io_config_)); + } + } + return ret; +} + +int ObTenantIOManager::reset_consumer_group_config(const int64_t group_id) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_working())) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("tenant not working", K(ret), K(tenant_id_)); + } else if (group_id < GROUP_START_ID) { + ret = OB_INVALID_CONFIG; + LOG_WARN("cannot reset other group io config", K(ret), K(group_id)); + } else { + // 对应group资源清零 + uint64_t index = INT_MAX64; + DRWLock::WRLockGuard guard(io_config_lock_); + if (OB_FAIL(get_group_index(group_id, index))) { + if (OB_HASH_NOT_EXIST == ret) { + //directive not flush yet, do nothing + ret = OB_SUCCESS; + LOG_INFO("directive not flush yet", K(group_id)); + } else if (OB_STATE_NOT_MATCH == ret) { + ret = OB_SUCCESS; + LOG_INFO("group has been deleted", K(group_id)); + } else { + LOG_WARN("get index from map failed", K(ret), K(group_id), K(index)); + } + } else if (OB_UNLIKELY(index == INT64_MAX || index < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid index, maybe try to reset OTHER_GROUPS or deleted_groups", K(ret), K(index), K(group_id)); + } else if (OB_FAIL(modify_group_io_config(index, 0, 0, 0, false, true/*cleared*/))) { + LOG_WARN("modify group io config failed", K(ret), K(tenant_id_), K(index)); + } else { + LOG_INFO ("stop group io control success when delete directive", K(tenant_id_), K(group_id), K(index), K(io_config_)); + } + } + return ret; +} + +int ObTenantIOManager::delete_consumer_group_config(const int64_t group_id) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_working())) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("tenant not working", K(ret), K(tenant_id_)); + } else if (group_id < GROUP_START_ID) { + ret = OB_INVALID_CONFIG; + LOG_WARN("cannot delete other group io config", K(ret), K(group_id)); + } else { + // 1.map设置非法值 + // 2.config设为unusable,资源清零 + // 3.phyqueue停止接受新请求,但是不会析构 + // 4.clock设置为stop,但是不会析构 + // 5.io_usage暂不处理 + uint64_t index = INT_MAX64; + DRWLock::WRLockGuard guard(io_config_lock_); + if (OB_FAIL(get_group_index(group_id, index))) { + if (OB_HASH_NOT_EXIST == ret) { + //GROUP 没有在map里,可能是没有指定资源,io未对其进行隔离或还未下刷 + ret = OB_SUCCESS; + LOG_INFO("io control not active for this group", K(group_id)); + if (OB_FAIL(group_id_index_map_.set_refactored(group_id, INT64_MAX, 1))) { //使用非法值覆盖 + LOG_WARN("stop phy queues failed", K(ret), K(tenant_id_), K(index)); + } + } else if (OB_STATE_NOT_MATCH == ret) { + // group delete twice + ret = OB_ERR_UNEXPECTED; + } else { + LOG_WARN("get index from map failed", K(ret), K(group_id), K(index)); + } + } else if (OB_UNLIKELY(index == INT64_MAX || index < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid index, maybe try to delete OTHER_GROUPS or deleted_groups", K(ret), K(index), K(group_id)); + } else { + if (OB_FAIL(group_id_index_map_.set_refactored(group_id, INT64_MAX, 1))) { //使用非法值覆盖 + LOG_WARN("stop phy queues failed", K(ret), K(tenant_id_), K(index)); + } else if (OB_FAIL(modify_group_io_config(index, 0, 0, 0, true/*deleted*/, false))) { + LOG_WARN("modify group io config failed", K(ret), K(tenant_id_), K(index)); + } + } + if (OB_SUCC(ret) && index != INT64_MAX) { + if (OB_FAIL(io_scheduler_->stop_phy_queues(tenant_id_, index))) { + LOG_WARN("stop phy queues failed", K(ret), K(tenant_id_), K(index)); + } else { + io_clock_->stop_clock(index); + LOG_INFO ("stop group io control success when delete group", K(tenant_id_), K(group_id), K(index), K(io_config_)); + } + } + } + return ret; +} + +int ObTenantIOManager::refresh_group_io_config() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_working())) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("tenant not working", K(ret), K(tenant_id_)); + } else if (OB_LIKELY(!io_config_.group_config_change_)) { + // group config not change, do nothing + } else if (OB_FAIL(io_usage_.refresh_group_num(io_config_.group_num_))) { + LOG_WARN("refresh io usage array failed", K(ret), K(io_usage_.get_io_usage_num()), K(io_config_.group_num_)); + } else if (OB_FAIL(io_scheduler_->update_group_queues(tenant_id_, io_config_.group_num_))) { + LOG_WARN("refresh phyqueue num failed", K(ret), K(io_config_.group_num_)); + } else if (OB_FAIL(io_clock_->update_io_clocks(io_config_))) { + LOG_WARN("refresh io clock failed", K(ret), K(io_config_)); + } else { + LOG_INFO("refresh group io config success", K(tenant_id_), K(io_config_)); + io_config_.group_config_change_ = false; + } + + return ret; +} + const ObTenantIOConfig &ObTenantIOManager::get_io_config() { return io_config_; @@ -809,30 +1135,79 @@ int ObTenantIOManager::trace_request_if_need(const ObIORequest *req, const char* return ret; } +int64_t ObTenantIOManager::get_group_num() +{ + DRWLock::RDLockGuard guard(io_config_lock_); + int64_t group_num = io_config_.group_num_; + return group_num; +} + +uint64_t ObTenantIOManager::get_usage_index(const int64_t group_id) +{ + uint64_t index = INT64_MAX; + int ret = group_id_index_map_.get_refactored(group_id, index); + if (OB_FAIL(ret) || index == INT64_MAX) { + //maybe deleted or reset + index = 0; + } else { + //other group occupies the first place, so return index + 1 + index += 1; + } + return index; +} void ObTenantIOManager::print_io_status() { - if (is_working()) { + if (is_working() && is_inited_) { + char io_status[1024] = { 0 }; + bool need_print_io_config = false; ObIOUsage::AvgItems avg_iops, avg_size, avg_rt; io_usage_.calculate_io_usage(); io_usage_.get_io_usage(avg_iops, avg_size, avg_rt); - char io_status[1024] = { 0 }; - bool need_print_io_config = false; - for (int64_t i = 0; i < static_cast(ObIOCategory::MAX_CATEGORY); ++i) { - if (avg_size[i][static_cast(ObIOMode::READ)] > std::numeric_limits::epsilon()) { - snprintf(io_status, sizeof(io_status), "category: %8s, mode: read, size: %10.2f, iops: %8.2f, rt: %8.2f", - get_io_category_name((ObIOCategory)i), - avg_size[i][static_cast(ObIOMode::READ)], avg_iops[i][static_cast(ObIOMode::READ)], avg_rt[i][static_cast(ObIOMode::READ)]); + for (int64_t i = 1; i < io_usage_.get_io_usage_num(); ++i) { + if (io_config_.group_configs_.at(i-1).deleted_) { + continue; + } + if (avg_size.at(i).at(static_cast(ObIOMode::READ)) > std::numeric_limits::epsilon()) { + snprintf(io_status, sizeof(io_status), "group_id: %ld, mode: read, size: %10.2f, iops: %8.2f, rt: %8.2f", + io_config_.group_ids_.at(i-1), + avg_size.at(i).at(static_cast(ObIOMode::READ)), + avg_iops.at(i).at(static_cast(ObIOMode::READ)), + avg_rt.at(i).at(static_cast(ObIOMode::READ))); LOG_INFO("[IO STATUS]", K_(tenant_id), KCSTRING(io_status)); need_print_io_config = true; } - if (avg_size[i][static_cast(ObIOMode::WRITE)] > std::numeric_limits::epsilon()) { - snprintf(io_status, sizeof(io_status), "category: %8s, mode: write, size: %10.2f, iops: %8.2f, rt: %8.2f", - get_io_category_name((ObIOCategory)i), - avg_size[i][static_cast(ObIOMode::WRITE)], avg_iops[i][static_cast(ObIOMode::WRITE)], avg_rt[i][static_cast(ObIOMode::WRITE)]); + if (avg_size.at(i).at(static_cast(ObIOMode::WRITE)) > std::numeric_limits::epsilon()) { + snprintf(io_status, sizeof(io_status), "group_id: %ld, mode: write, size: %10.2f, iops: %8.2f, rt: %8.2f", + io_config_.group_ids_.at(i-1), + avg_size.at(i).at(static_cast(ObIOMode::WRITE)), + avg_iops.at(i).at(static_cast(ObIOMode::WRITE)), + avg_rt.at(i).at(static_cast(ObIOMode::WRITE))); LOG_INFO("[IO STATUS]", K_(tenant_id), KCSTRING(io_status)); need_print_io_config = true; } } + // OTHER_GROUPS + if (avg_size.at(0).at(static_cast(ObIOMode::READ)) > std::numeric_limits::epsilon()) { + snprintf(io_status, sizeof(io_status), "group_id: %ld, group_name: %s, mode: read, size: %10.2f, iops: %8.2f, rt: %8.2f", + 0L, + "OTHER_GROUPS", + avg_size.at(0).at(static_cast(ObIOMode::READ)), + avg_iops.at(0).at(static_cast(ObIOMode::READ)), + avg_rt.at(0).at(static_cast(ObIOMode::READ))); + LOG_INFO("[IO STATUS]", K_(tenant_id), KCSTRING(io_status)); + need_print_io_config = true; + } + if (avg_size.at(0).at(static_cast(ObIOMode::WRITE)) > std::numeric_limits::epsilon()) { + snprintf(io_status, sizeof(io_status), "group_id: %ld, group_name: %s, mode: write, size: %10.2f, iops: %8.2f, rt: %8.2f", + 0L, + "OTHER_GROUPS", + avg_size.at(0).at(static_cast(ObIOMode::WRITE)), + avg_iops.at(0).at(static_cast(ObIOMode::WRITE)), + avg_rt.at(0).at(static_cast(ObIOMode::WRITE))); + LOG_INFO("[IO STATUS]", K_(tenant_id), KCSTRING(io_status)); + need_print_io_config = true; + } + if (need_print_io_config) { ObArray queue_count_array; int ret = OB_SUCCESS; diff --git a/src/share/io/ob_io_manager.h b/src/share/io/ob_io_manager.h index 00633fbe19..6c26b63c31 100644 --- a/src/share/io/ob_io_manager.h +++ b/src/share/io/ob_io_manager.h @@ -71,6 +71,7 @@ public: int refresh_tenant_io_config(const uint64_t tenant_id, const ObTenantIOConfig &tenant_io_config); int get_tenant_io_manager(const uint64_t tenant_id, ObRefHolder &tenant_holder); int get_tenant_ids(ObIArray &tenant_ids); + ObIOScheduler *get_scheduler(); private: friend class ObTenantIOManager; @@ -111,18 +112,47 @@ public: bool is_working() const; int inner_aio(const ObIOInfo &info, ObIOHandle &handle); int enqueue_callback(ObIORequest &req); - ObIOClock *get_io_clock() { return io_clock_; } - const ObIOUsage &get_io_usage() { return io_usage_; } - int update_io_config(const ObTenantIOConfig &io_config); + ObTenantIOClock *get_io_clock() { return io_clock_; } + ObIOUsage &get_io_usage() { return io_usage_; } + int update_basic_io_config(const ObTenantIOConfig &io_config); int alloc_io_request(ObIAllocator &allocator,const int64_t callback_size, ObIORequest *&req); - int alloc_io_clock(ObIAllocator &allocator, ObIOClock *&io_clock); + int alloc_io_clock(ObIAllocator &allocator, ObTenantIOClock *&io_clock); + int init_group_index_map(const ObTenantIOConfig &io_config); + int get_group_index(const int64_t group_id, uint64_t &index); + int modify_group_io_config(const uint64_t index, + const int64_t min_percent, + const int64_t max_percent, + const int64_t weight_percent, + const bool deleted = false, + const bool cleared = false); + + //for modify group config + int modify_io_config(const uint64_t group_id, + const int64_t min_percent, + const int64_t max_percent, + const int64_t weight_percent); + //for add group + int add_group_io_config(const int64_t group_id, + const int64_t min_percent, + const int64_t max_percent, + const int64_t weight_percent); + //for delete plan + int reset_all_group_config(); + //for delete directive + int reset_consumer_group_config(const int64_t group_id); + //for delete group + int delete_consumer_group_config(const int64_t group_id); + //随directive refresh而定期刷新(最晚10S一次) + int refresh_group_io_config(); const ObTenantIOConfig &get_io_config(); int trace_request_if_need(const ObIORequest *req, const char* msg, ObIOTracer::TraceType trace_type); + int64_t get_group_num(); + uint64_t get_usage_index(const int64_t group_id); void print_io_status(); void inc_ref(); void dec_ref(); TO_STRING_KV(K(is_inited_), K(ref_cnt_), K(tenant_id_), K(io_config_), K(io_clock_), - K(io_allocator_), KPC(io_scheduler_), K(callback_mgr_)); + K(io_allocator_), KPC(io_scheduler_), K(callback_mgr_)); private: friend class ObIORequest; bool is_inited_; @@ -130,16 +160,18 @@ private: int64_t ref_cnt_; uint64_t tenant_id_; ObTenantIOConfig io_config_; - ObIOClock *io_clock_; + ObTenantIOClock *io_clock_; ObIOAllocator io_allocator_; ObIOScheduler *io_scheduler_; ObIOCallbackManager callback_mgr_; ObIOUsage io_usage_; ObIOTracer io_tracer_; + DRWLock io_config_lock_; //for map and config + hash::ObHashMap group_id_index_map_; //key:group_id, value:index }; #define OB_IO_MANAGER (oceanbase::common::ObIOManager::get_instance()) -}// end namespace oceanbase +}// end namespace common }// end namespace oceanbase #endif//OCEANBASE_LIB_STORAGE_OB_IO_MANAGER_H diff --git a/src/share/io/ob_io_struct.cpp b/src/share/io/ob_io_struct.cpp index a7836cc572..b69813a9db 100644 --- a/src/share/io/ob_io_struct.cpp +++ b/src/share/io/ob_io_struct.cpp @@ -424,8 +424,15 @@ void ObIOStatDiff::reset() /****************** IOUsage **********************/ ObIOUsage::ObIOUsage() + : io_stats_(), + io_estimators_(), + group_avg_iops_(), + group_avg_byte_(), + group_avg_rt_us_(), + group_num_(0), + doing_request_count_() { - MEMSET(doing_request_count_, 0, sizeof(doing_request_count_)); + } ObIOUsage::~ObIOUsage() @@ -433,46 +440,137 @@ ObIOUsage::~ObIOUsage() } -void ObIOUsage::accumulate(const ObIORequest &req) +int ObIOUsage::init(const int64_t group_num) +{ + int ret =OB_SUCCESS; + //push other group into array + if (OB_FAIL(refresh_group_num(group_num))) { + LOG_WARN("refresh io usage array failed", K(ret), K(group_num)); + } else if (io_stats_.count() != group_num_ || + io_estimators_.count() != group_num_ || + group_avg_iops_.count() != group_num_ || + group_avg_byte_.count() != group_num_ || + group_avg_rt_us_.count() != group_num_ || + doing_request_count_.count() != group_num_) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("init io usage failed", K(group_num_)); + } + return ret; +} + +int ObIOUsage::refresh_group_num(const int64_t group_num) +{ + int ret = OB_SUCCESS; + if (group_num < 0) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid group num", K(ret), K(group_num)); + } else if (OB_FAIL(io_stats_.reserve(group_num + 1)) || + OB_FAIL(io_estimators_.reserve(group_num + 1)) || + OB_FAIL(group_avg_iops_.reserve(group_num + 1)) || + OB_FAIL(group_avg_byte_.reserve(group_num + 1)) || + OB_FAIL(group_avg_rt_us_.reserve(group_num + 1)) || + OB_FAIL(doing_request_count_.reserve(group_num + 1))) { + LOG_WARN("reserver group failed", K(ret), K(group_num)); + } else { + for (int64_t i = group_num_; OB_SUCC(ret) && i < group_num + 1; ++i) { + ObSEArray cur_stat_array; + ObSEArray cur_estimators_array; + ObSEArray cur_avg_iops; + ObSEArray cur_avg_byte; + ObSEArray cur_avg_rt_us; + + if (OB_FAIL(cur_stat_array.reserve(static_cast(ObIOMode::MAX_MODE))) || + OB_FAIL(cur_estimators_array.reserve(static_cast(ObIOMode::MAX_MODE))) || + OB_FAIL(cur_avg_iops.reserve(static_cast(ObIOMode::MAX_MODE))) || + OB_FAIL(cur_avg_byte.reserve(static_cast(ObIOMode::MAX_MODE))) || + OB_FAIL(cur_avg_rt_us.reserve(static_cast(ObIOMode::MAX_MODE)))) { + LOG_WARN("reserver group failed", K(ret), K(group_num)); + } else { + for (int64_t j = 0; OB_SUCC(ret) && j < static_cast(ObIOMode::MAX_MODE); ++j) { + ObIOStat cur_stat; + ObIOStatDiff cur_diff; + if (OB_FAIL(cur_stat_array.push_back(cur_stat))) { + LOG_WARN("push stat failed", K(ret), K(i), K(j)); + } else if (OB_FAIL(cur_estimators_array.push_back(cur_diff))) { + LOG_WARN("push estimator failed", K(ret), K(i), K(j)); + } else if (OB_FAIL(cur_avg_iops.push_back(0))) { + LOG_WARN("push avg_iops failed", K(ret), K(i), K(j)); + } else if (OB_FAIL(cur_avg_byte.push_back(0))) { + LOG_WARN("push avg_byte failed", K(ret), K(i), K(j)); + } else if (OB_FAIL(cur_avg_rt_us.push_back(0))) { + LOG_WARN("push avg_rt failed", K(ret), K(i), K(j)); + } + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(io_stats_.push_back(cur_stat_array))) { + LOG_WARN("push stat array failed", K(ret), K(i)); + } else if (OB_FAIL(io_estimators_.push_back(cur_estimators_array))) { + LOG_WARN("push estimator array failed", K(ret), K(i)); + } else if (OB_FAIL(group_avg_iops_.push_back(cur_avg_iops))) { + LOG_WARN("push avg_iops array failed", K(ret), K(i)); + } else if (OB_FAIL(group_avg_byte_.push_back(cur_avg_byte))) { + LOG_WARN("push avg_byte array failed", K(ret), K(i)); + } else if (OB_FAIL(group_avg_rt_us_.push_back(cur_avg_rt_us))) { + LOG_WARN("push avg_rt array failed", K(ret), K(i)); + } else if (OB_FAIL(doing_request_count_.push_back(0))) { + LOG_WARN("push group_doing_req failed", K(ret), K(i)); + } else { + ATOMIC_INC(&group_num_); + } + } + } + } + return ret; +} +void ObIOUsage::accumulate(ObIORequest &req) { if (req.time_log_.return_ts_ > 0) { const int64_t device_delay = get_io_interval(req.time_log_.return_ts_, req.time_log_.submit_ts_); - io_stats_[static_cast(req.get_category())][static_cast(req.get_mode())] + io_stats_.at(req.get_io_usage_index()).at(static_cast(req.get_mode())) .accumulate(1, req.io_size_, device_delay); } } void ObIOUsage::calculate_io_usage() { - for (int64_t i = 0; i < static_cast(ObIOCategory::MAX_CATEGORY); ++i) { + for (int64_t i = 0; i < group_num_; ++i) { for (int64_t j = 0; j < static_cast(ObIOMode::MAX_MODE); ++j) { - ObIOStatDiff &cur_io_estimator = io_estimators_[i][j]; - ObIOStat &cur_io_stat = io_stats_[i][j]; - cur_io_estimator.diff(cur_io_stat, avg_iops_[i][j], avg_byte_[i][j], avg_rt_us_[i][j]); + ObIOStatDiff &cur_io_estimator = io_estimators_.at(i).at(j); + ObIOStat &cur_io_stat = io_stats_.at(i).at(j); + cur_io_estimator.diff(cur_io_stat, + group_avg_iops_.at(i).at(j), + group_avg_byte_.at(i).at(j), + group_avg_rt_us_.at(i).at(j)); } } } -void ObIOUsage::get_io_usage(AvgItems &avg_iops, AvgItems &avg_bytes, AvgItems &avg_rt_us) const +void ObIOUsage::get_io_usage(AvgItems &avg_iops, AvgItems &avg_bytes, AvgItems &avg_rt_us) { - memcpy(&avg_iops, &avg_iops_, sizeof(AvgItems)); - memcpy(&avg_bytes, &avg_byte_, sizeof(AvgItems)); - memcpy(&avg_rt_us, &avg_rt_us_, sizeof(AvgItems)); + avg_iops.assign(group_avg_iops_); + avg_bytes.assign(group_avg_byte_); + avg_rt_us.assign(group_avg_rt_us_); } -void ObIOUsage::record_request_start(const ObIORequest &req) +void ObIOUsage::record_request_start(ObIORequest &req) { - ATOMIC_INC(&doing_request_count_[static_cast(req.get_category())]); + ATOMIC_INC(&doing_request_count_.at(req.get_io_usage_index())); } -void ObIOUsage::record_request_finish(const ObIORequest &req) +void ObIOUsage::record_request_finish(ObIORequest &req) { - ATOMIC_DEC(&doing_request_count_[static_cast(req.get_category())]); + ATOMIC_DEC(&doing_request_count_.at(req.get_io_usage_index())); } -bool ObIOUsage::is_request_doing(const ObIOCategory category) const +bool ObIOUsage::is_request_doing(const int64_t index) const { - return ATOMIC_LOAD(&doing_request_count_[static_cast(category)]) > 0; + return ATOMIC_LOAD(&doing_request_count_.at(index)) > 0; +} + +int64_t ObIOUsage::get_io_usage_num() const +{ + return group_num_; } int64_t ObIOUsage::to_string(char* buf, const int64_t buf_len) const @@ -481,11 +579,13 @@ int64_t ObIOUsage::to_string(char* buf, const int64_t buf_len) const J_OBJ_START(); BUF_PRINTF("doing_request_count:["); bool need_comma = false; - for (int64_t i = 0; i < static_cast(ObIOCategory::MAX_CATEGORY); ++i) { + for (int64_t i = 0; i < group_num_; ++i) { if (need_comma) { J_COMMA(); } - J_KV(get_io_category_name(static_cast(i)), doing_request_count_[i]); + char ret[8]; + snprintf(ret, sizeof(ret), "%ld", i); + J_KV(ret, doing_request_count_.at(i)); need_comma = true; } BUF_PRINTF("]"); @@ -634,49 +734,93 @@ void ObIOTuner::print_io_status() } } -/****************** ObIOCategoryQueues **********************/ -ObIOCategoryQueues::ObIOCategoryQueues() - : is_inited_(false) +/****************** ObIOGroupQueues **********************/ +ObIOGroupQueues::ObIOGroupQueues(ObIAllocator &allocator) + : is_inited_(false), + allocator_(allocator), + group_phy_queues_(), + other_phy_queue_() { } -ObIOCategoryQueues::~ObIOCategoryQueues() +ObIOGroupQueues::~ObIOGroupQueues() { destroy(); } -int ObIOCategoryQueues::init() +int ObIOGroupQueues::init(const int64_t group_num) { int ret = OB_SUCCESS; if (OB_UNLIKELY(is_inited_)) { ret = OB_INIT_TWICE; LOG_WARN("phy queue init twice", K(ret), K(is_inited_)); + } else if (group_num < 0) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid group num", K(ret), K(group_num)); } else { - for (int64_t i = 0; OB_SUCC(ret) && i < static_cast(ObIOCategory::MAX_CATEGORY) + 1; ++i) { - if (OB_FAIL(phy_queues_[i].init(i))){ - LOG_WARN("phy queue init failed", K(ret)); + for (int64_t i = 0; OB_SUCC(ret) && i < group_num; ++i) { + void *buf = nullptr; + ObPhyQueue *tmp_phyqueue = nullptr; + if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObPhyQueue)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (FALSE_IT(tmp_phyqueue = new (buf) ObPhyQueue())) { + } else if (OB_FAIL(tmp_phyqueue->init(i))) { + LOG_WARN("init io phy_queue failed", K(ret), K(i), K(*tmp_phyqueue)); + } else if (OB_FAIL(group_phy_queues_.push_back(tmp_phyqueue))) { + LOG_WARN("push back io sender failed", K(ret), K(i), K(*tmp_phyqueue)); + } + if (OB_FAIL(ret) && nullptr != tmp_phyqueue) { + tmp_phyqueue->~ObPhyQueue(); + allocator_.free(tmp_phyqueue); } } - } - if (OB_SUCC(ret)) { - is_inited_ = true; + if (OB_SUCC(ret)) { + is_inited_ = true; + } } return ret; } -void ObIOCategoryQueues::destroy() + +void ObIOGroupQueues::destroy() { + for (int64_t i = 0; i < group_phy_queues_.count(); ++i) { + ObPhyQueue *tmp_phyqueue = group_phy_queues_.at(i); + if (nullptr != tmp_phyqueue) { + tmp_phyqueue->destroy(); + allocator_.free(tmp_phyqueue); + } + } + other_phy_queue_.destroy(); + group_phy_queues_.destroy(); is_inited_ = false; } + +/****************** IOSenderInfo **********************/ +ObSenderInfo::ObSenderInfo() + : queuing_count_(0), + reservation_ts_(INT_MAX64), + group_limitation_ts_(INT_MAX64), + tenant_limitation_ts_(INT_MAX64), + proportion_ts_(INT_MAX64) +{ + +} +ObSenderInfo::~ObSenderInfo() +{ + +} + /****************** IOScheduleQueue **********************/ ObIOSender::ObIOSender(ObIAllocator &allocator) - : is_inited_(false), - allocator_(allocator), - stop_submit_(false), + : sender_req_count_(0), tg_id_(-1), + is_inited_(false), + stop_submit_(false), + allocator_(allocator), io_queue_(nullptr), - queue_cond_(), - sender_req_count_(0) + queue_cond_() { } @@ -686,23 +830,20 @@ ObIOSender::~ObIOSender() destroy(); } -int ObIOSender::init(const int32_t queue_depth) +int ObIOSender::init() { int ret = OB_SUCCESS; if (OB_UNLIKELY(is_inited_)) { ret = OB_INIT_TWICE; LOG_WARN("init twice", K(ret), K(is_inited_)); - } else if (OB_UNLIKELY(queue_depth <= 0)) { - ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid argument", K(ret), K(queue_depth)); } else if (OB_FAIL(queue_cond_.init(ObWaitEventIds::IO_QUEUE_LOCK_WAIT))) { LOG_WARN("init queue condition failed", K(ret)); } else if (OB_FAIL(alloc_mclock_queue(allocator_, io_queue_))) { LOG_WARN("alloc io queue failed", K(ret)); } else if (OB_FAIL(io_queue_->init())) { LOG_WARN("init io queue failed", K(ret)); - } else if (OB_FAIL(tenant_map_.create(7, "IO_TENANT_MAP"))) { - LOG_WARN("create channel map failed", K(ret)); + } else if (OB_FAIL(tenant_groups_map_.create(7, "IO_GROUP_MAP"))) { + LOG_WARN("create tenant group map failed", K(ret)); } else if (OB_FAIL(TG_CREATE(lib::TGDefIDs::IO_SCHEDULE, tg_id_))) { LOG_WARN("create thread group id failed", K(ret)); } else { @@ -717,13 +858,13 @@ int ObIOSender::init(const int32_t queue_depth) return ret; } -struct DestroyPhyqueueMapFn +struct DestroyGroupqueueMapFn { public: - DestroyPhyqueueMapFn(ObIAllocator &allocator) : allocator_(allocator) {} - int operator () (oceanbase::common::hash::HashMapPair &entry) { + DestroyGroupqueueMapFn(ObIAllocator &allocator) : allocator_(allocator) {} + int operator () (hash::HashMapPair &entry) { if (nullptr != entry.second) { - entry.second->~ObIOCategoryQueues(); + entry.second->~ObIOGroupQueues(); allocator_.free(entry.second); } return OB_SUCCESS; @@ -757,9 +898,9 @@ void ObIOSender::destroy() TG_DESTROY(tg_id_); tg_id_ = -1; } - DestroyPhyqueueMapFn destry_phyqueue_map_fn(allocator_); - tenant_map_.foreach_refactored(destry_phyqueue_map_fn); - tenant_map_.destroy(); + DestroyGroupqueueMapFn destry_groupqueue_map_fn(allocator_); + tenant_groups_map_.foreach_refactored(destry_groupqueue_map_fn); + tenant_groups_map_.destroy(); queue_cond_.destroy(); if (nullptr != io_queue_) { io_queue_->destroy(); @@ -823,21 +964,43 @@ int ObIOSender::alloc_mclock_queue(ObIAllocator &allocator, ObMClockQueue *&io_q int ObIOSender::enqueue_request(ObIORequest &req) { int ret = OB_SUCCESS; - ObIOCategoryQueues *io_category_queues = nullptr; ObIORequest *tmp_req = &req; + ObPhyQueue *tmp_phy_queue = nullptr; if (!is_inited_) { ret = OB_NOT_INIT; LOG_WARN("Not init", K(ret), K(is_inited_)); - } else { + } else if (OB_NOT_NULL(req.tenant_io_mgr_.get_ptr())) { ObThreadCondGuard cond_guard(queue_cond_); if (OB_FAIL(cond_guard.get_ret())) { LOG_ERROR("guard queue condition failed", K(ret)); } else { - if (OB_FAIL(tenant_map_.get_refactored(tmp_req->io_info_.tenant_id_, io_category_queues))) { + ObIOGroupQueues *io_group_queues = nullptr; + if (OB_FAIL(tenant_groups_map_.get_refactored(tmp_req->io_info_.tenant_id_, io_group_queues))) { LOG_WARN("get_refactored tenant_map failed", K(ret), K(req)); } else { - const int index = static_cast(tmp_req->get_category()); - ObPhyQueue *tmp_phy_queue = &(io_category_queues->phy_queues_[index]); + uint64_t index = INT_MAX64; + const int64_t group_id = tmp_req->get_group_id(); + if (group_id < GROUP_START_ID) { //other + tmp_phy_queue = &(io_group_queues->other_phy_queue_); + } else if (OB_FAIL(req.tenant_io_mgr_.get_ptr()->get_group_index(group_id, index))) { + // 防止删除group、新建group等情况发生时在途req无法找到对应的group + if (ret == OB_HASH_NOT_EXIST || ret == OB_STATE_NOT_MATCH) { + ret = OB_SUCCESS; + tmp_phy_queue = &(io_group_queues->other_phy_queue_); + } else { + LOG_WARN("get group index failed", K(ret), K(group_id), K(index)); + } + } else if (index < 0 || index >= io_group_queues->group_phy_queues_.count()) { + tmp_phy_queue = &(io_group_queues->other_phy_queue_); + } else { + tmp_phy_queue = io_group_queues->group_phy_queues_.at(index); + if (OB_UNLIKELY(tmp_phy_queue->is_stop_accept())) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("runner is quit, stop accept new req", K(ret), K(req)); + } + } + } + if (OB_SUCC(ret)) { if (tmp_phy_queue->req_list_.is_empty()) { //new request if (OB_FAIL(io_queue_->remove_from_heap(tmp_phy_queue))) { @@ -856,9 +1019,10 @@ int ObIOSender::enqueue_request(ObIORequest &req) if (OB_NOT_NULL(req.tenant_io_mgr_.get_ptr())) { ObTenantIOClock *io_clock = static_cast(req.tenant_io_mgr_.get_ptr()->get_io_clock()); //phy_queue from idle to active + // TODO(QILU):不要每次是新请求就调一次,因为可能就是发的很快,需要增加一个判断机制空闲了一段时间才触发 int tmp_ret = io_clock->sync_tenant_clock(io_clock); if (OB_FAIL(io_clock->calc_phyqueue_clock(tmp_phy_queue, req))) { - LOG_WARN("calc phyqueue clock failed", K(ret)); + LOG_WARN("calc phyqueue clock failed", K(ret), K(tmp_phy_queue->queue_index_)); } else if (OB_UNLIKELY(OB_SUCCESS != tmp_ret)) { LOG_WARN("sync tenant clock failed", K(tmp_ret)); } @@ -947,30 +1111,48 @@ int ObIOSender::dequeue_request(ObIORequest *&req) return ret; } -int ObIOSender::remove_phy_queue(const uint64_t tenant_id) +int ObIOSender::update_group_queue(const uint64_t tenant_id, const int64_t group_num) { int ret = OB_SUCCESS; if (!is_inited_) { ret = OB_NOT_INIT; LOG_WARN("Not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id) || group_num < 0)) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid config", K(ret), K(tenant_id), K(group_num)); } else { - ObThreadCondGuard cond_guard(queue_cond_); - if (OB_FAIL(cond_guard.get_ret())) { - LOG_ERROR("guard queue condition failed", K(ret)); - } else { - ObIOCategoryQueues *io_category_queues = nullptr; - if (OB_FAIL(tenant_map_.erase_refactored(tenant_id, &io_category_queues))) { - LOG_WARN("erase phy_queues failed", K(ret), K(tenant_id)); - } else if (nullptr != io_category_queues) { - for (int64_t j = 0; OB_SUCC(ret) && j < static_cast(ObIOCategory::MAX_CATEGORY) + 1; ++j) { - ObPhyQueue *tmp_phy_queue = &(io_category_queues->phy_queues_[j]); - if (OB_FAIL(io_queue_->remove_from_heap(tmp_phy_queue))) { - LOG_WARN("remove phy queue from heap failed", K(ret)); - } + ObIOGroupQueues *io_group_queues = nullptr; + if (OB_FAIL(tenant_groups_map_.get_refactored(tenant_id, io_group_queues))) { + LOG_WARN("get_refactored form tenant_group_map failed", K(ret), K(tenant_id)); + } else if (OB_UNLIKELY(!io_group_queues->is_inited_)) { + LOG_WARN("io_group_queues not init", K(ret), K(*io_group_queues)); + } else if (io_group_queues->group_phy_queues_.count() > group_num || group_num < 0) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid group num", K(ret), K(group_num)); + } else if (io_group_queues->group_phy_queues_.count() == group_num) { + // do nothing + } else if (io_group_queues->group_phy_queues_.count() < group_num) { + // add phyqueue + int64_t cur_num = io_group_queues->group_phy_queues_.count(); + for (int64_t i = cur_num; OB_SUCC(ret) && i < group_num; ++i) { + void *buf = nullptr; + ObPhyQueue *tmp_phyqueue = nullptr; + if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObPhyQueue)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (FALSE_IT(tmp_phyqueue = new (buf) ObPhyQueue())) { + } else if (OB_FAIL(tmp_phyqueue->init(i))) { + LOG_WARN("init io phy_queue failed", K(ret), K(i), K(*tmp_phyqueue)); + } else if (OB_FAIL(io_group_queues->group_phy_queues_.push_back(tmp_phyqueue))) { + LOG_WARN("push back io sender failed", K(ret), K(i), K(*tmp_phyqueue)); + } else if (OB_FAIL(enqueue_phy_queue(*tmp_phyqueue))) { + LOG_WARN("new queue into heap failed", K(ret)); + } else { + LOG_INFO("add phy queue success", K(tenant_id), K(cur_num), K(group_num)); } - if (OB_SUCC(ret)) { - io_category_queues->~ObIOCategoryQueues(); - allocator_.free(io_category_queues); + if (OB_FAIL(ret) && nullptr != tmp_phyqueue) { + tmp_phyqueue->~ObPhyQueue(); + allocator_.free(tmp_phyqueue); } } } @@ -978,6 +1160,69 @@ int ObIOSender::remove_phy_queue(const uint64_t tenant_id) return ret; } +int ObIOSender::remove_group_queues(const uint64_t tenant_id) +{ + int ret = OB_SUCCESS; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("Not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else { + ObThreadCondGuard cond_guard(queue_cond_); + if (OB_FAIL(cond_guard.get_ret())) { + LOG_ERROR("guard queue condition failed", K(ret)); + } else { + ObIOGroupQueues *io_group_queues = nullptr; + if (OB_FAIL(tenant_groups_map_.erase_refactored(tenant_id, &io_group_queues))) { + LOG_WARN("erase phy_queues failed", K(ret), K(tenant_id)); + } else if (nullptr != io_group_queues) { + for (int64_t j = 0; OB_SUCC(ret) && j < io_group_queues->group_phy_queues_.count(); ++j) { + ObPhyQueue *tmp_phy_queue = io_group_queues->group_phy_queues_.at(j); + if (OB_FAIL(io_queue_->remove_from_heap(tmp_phy_queue))) { + LOG_WARN("remove phy queue from heap failed", K(ret)); + } + } + if (OB_SUCC(ret)) { + if(OB_FAIL(io_queue_->remove_from_heap(&(io_group_queues->other_phy_queue_)))) { + LOG_WARN("remove other phy queue from heap failed", K(ret)); + } else { + io_group_queues->~ObIOGroupQueues(); + allocator_.free(io_group_queues); + } + } + } + } + } + return ret; +} + +int ObIOSender::stop_phy_queue(const uint64_t tenant_id, const uint64_t index) +{ + int ret = OB_SUCCESS; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("Not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id) || index < 0 || INT64_MAX == index)) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid config", K(ret), K(tenant_id), K(index)); + } else { + ObThreadCondGuard cond_guard(queue_cond_); + if (OB_FAIL(cond_guard.get_ret())) { + LOG_ERROR("guard queue condition failed", K(ret)); + } else { + ObIOGroupQueues *io_group_queues = nullptr; + if (OB_FAIL(tenant_groups_map_.get_refactored(tenant_id, io_group_queues))) { + LOG_WARN("get_refactored tenant_map failed", K(ret), K(tenant_id)); + } else if (nullptr != io_group_queues && index < io_group_queues->group_phy_queues_.count()) { + io_group_queues->group_phy_queues_.at(index)->set_stop_accept(); + //TODO (QILU) tuner regularly checks whether the memory can be released + } + } + } + return ret; +} int ObIOSender::notify() { int ret = OB_SUCCESS; @@ -1000,6 +1245,38 @@ int32_t ObIOSender::get_queue_count() const return OB_ISNULL(io_queue_) ? 0 : sender_req_count_; } +int ObIOSender::get_sender_status(const uint64_t tenant_id, const uint64_t index, ObSenderInfo &sender_info) +{ + int ret = OB_SUCCESS; + ObIOGroupQueues *io_group_queues = nullptr; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("Not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id) || index < 0 || + (index >= io_group_queues->group_phy_queues_.count() && INT64_MAX != index))) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid index", K(ret), K(index)); + } else { + ObThreadCondGuard cond_guard(queue_cond_); + if (OB_FAIL(cond_guard.get_ret())) { + LOG_ERROR("guard queue condition failed", K(ret)); + } else { + if (OB_FAIL(tenant_groups_map_.get_refactored(tenant_id, io_group_queues))) { + LOG_WARN("get io_group_queues from map failed", K(ret), K(tenant_id)); + } else { + ObPhyQueue *tmp_phy_queue = index == INT64_MAX ? + &(io_group_queues->other_phy_queue_) : io_group_queues->group_phy_queues_.at(index); + sender_info.queuing_count_ = tmp_phy_queue->req_list_.get_size(); + sender_info.reservation_ts_ = tmp_phy_queue->reservation_ts_; + sender_info.group_limitation_ts_ = tmp_phy_queue->group_limitation_ts_; + sender_info.tenant_limitation_ts_ = tmp_phy_queue->tenant_limitation_ts_; + sender_info.proportion_ts_ = tmp_phy_queue->proportion_ts_; + } + } + } + return ret; +} + void ObIOSender::pop_and_submit() { int ret = OB_SUCCESS; @@ -1111,7 +1388,7 @@ ObIOScheduler::~ObIOScheduler() destroy(); } -int ObIOScheduler::init(const int64_t queue_count, const int64_t queue_depth, const int64_t schedule_media_id) +int ObIOScheduler::init(const int64_t queue_count, const int64_t schedule_media_id) { int ret = OB_SUCCESS; if (OB_UNLIKELY(is_inited_)) { @@ -1119,7 +1396,7 @@ int ObIOScheduler::init(const int64_t queue_count, const int64_t queue_depth, co LOG_WARN("io scheduler init twice", K(ret), K(is_inited_)); } else if (OB_UNLIKELY(queue_count <= 0 || queue_count <= 0 || schedule_media_id < 0)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid argument", K(queue_count), K(queue_depth), K(schedule_media_id)); + LOG_WARN("invalid argument", K(queue_count), K(schedule_media_id)); } else if (OB_FAIL(io_tuner_.init())) { LOG_WARN("init io tuner failed", K(ret)); } else { @@ -1130,8 +1407,8 @@ int ObIOScheduler::init(const int64_t queue_count, const int64_t queue_depth, co ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("allocate memory failed", K(ret)); } else if (FALSE_IT(tmp_sender = new (buf) ObIOSender(allocator_))) { - } else if (OB_FAIL(tmp_sender->init(queue_depth))) { - LOG_WARN("init io sender failed", K(ret), K(i), K(*tmp_sender), K(queue_depth)); + } else if (OB_FAIL(tmp_sender->init())) { + LOG_WARN("init io sender failed", K(ret), K(i), K(*tmp_sender)); } else if (OB_FAIL(senders_.push_back(tmp_sender))) { LOG_WARN("push back io sender failed", K(ret), K(i), K(*tmp_sender)); } @@ -1196,7 +1473,7 @@ void ObIOScheduler::stop() } } -int ObIOScheduler::schedule_request(ObIOClock &io_clock, ObIORequest &req) +int ObIOScheduler::schedule_request(ObTenantIOClock &io_clock, ObIORequest &req) { int ret = OB_SUCCESS; RequestHolder holder(&req); @@ -1224,47 +1501,105 @@ int ObIOScheduler::schedule_request(ObIOClock &io_clock, ObIORequest &req) return ret; } -int ObIOScheduler::add_tenant_map(uint64_t tenant_id) +int ObIOScheduler::init_group_queues(const uint64_t tenant_id, const int64_t group_num) { int ret = OB_SUCCESS; - for (int64_t i = 0; OB_SUCC(ret) && i < senders_.count(); ++i) { - ObIOSender *cur_sender = senders_.at(i); - ObIOCategoryQueues *io_category_queues = nullptr; - void *buf_queues = nullptr; - if (OB_ISNULL(buf_queues = cur_sender->allocator_.alloc(sizeof(ObIOCategoryQueues)))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_WARN("allocate phyqueues memory failed", K(ret)); - } else { - io_category_queues = new (buf_queues) ObIOCategoryQueues(); - if (OB_FAIL(io_category_queues->init())) { + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id) || group_num < 0)) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid config", K(ret), K(tenant_id), K(group_num)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < senders_.count(); ++i) { + ObIOSender *cur_sender = senders_.at(i); + ObIOGroupQueues *io_group_queues = nullptr; + void *buf_queues = nullptr; + if (OB_ISNULL(buf_queues = cur_sender->allocator_.alloc(sizeof(ObIOGroupQueues)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate phyqueues memory failed", K(ret)); + } else if (FALSE_IT(io_group_queues = new (buf_queues) ObIOGroupQueues(cur_sender->allocator_))) { + } else if (OB_FAIL(io_group_queues->other_phy_queue_.init(INT64_MAX))) { //other group index + LOG_WARN("init other group queue failes", K(ret)); + } else if (OB_FAIL(io_group_queues->init(group_num))) { LOG_WARN("init phyqueues failed", K(ret)); + } else if (OB_FAIL(cur_sender->enqueue_phy_queue(io_group_queues->other_phy_queue_))){ //other groups queue + LOG_WARN("other phy queue into send_queue failed", K(ret)); } else { - for(int64_t j = 0; OB_SUCC(ret) && j < static_cast(ObIOCategory::MAX_CATEGORY) + 1; j++) { - if (OB_FAIL(cur_sender->enqueue_phy_queue(io_category_queues->phy_queues_[j]))) { + for (int64_t j = 0; OB_SUCC(ret) && j < group_num; ++j) { + if (OB_FAIL(cur_sender->enqueue_phy_queue(*(io_group_queues->group_phy_queues_.at(j))))) { LOG_WARN("new phy_queue into send_queue failed", K(ret)); } } - } - if (OB_SUCC(ret)) { - if (OB_FAIL(cur_sender->tenant_map_.set_refactored(tenant_id, io_category_queues))) { - LOG_WARN("init tenant map failed", K(ret), K(i)); + if (OB_SUCC(ret)) { + if (OB_FAIL(cur_sender->tenant_groups_map_.set_refactored(tenant_id, io_group_queues))) { + LOG_WARN("init tenant group map failed", K(tenant_id), K(ret), K(i)); + } + } else { + io_group_queues->~ObIOGroupQueues(); + cur_sender->allocator_.free(io_group_queues); } - } else { - io_category_queues->~ObIOCategoryQueues(); - cur_sender->allocator_.free(io_category_queues); } } } return ret; } -int ObIOScheduler::remove_tenant_map(uint64_t tenant_id) +int ObIOScheduler::update_group_queues(const uint64_t tenant_id, const int64_t group_num) { int ret = OB_SUCCESS; - for (int64_t i = 0; OB_SUCC(ret) && i < senders_.count(); ++i) { - ObIOSender *cur_sender = senders_.at(i); - if (OB_FAIL(cur_sender->remove_phy_queue(tenant_id))) { - LOG_WARN("remove phy queue failed", K(ret), K(i), K(tenant_id)); + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id) || group_num < 0)) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid config", K(ret), K(tenant_id), K(group_num)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < senders_.count(); ++i) { + ObIOSender *cur_sender = senders_.at(i); + if (OB_FAIL(cur_sender->update_group_queue(tenant_id, group_num))) { + LOG_WARN("serder update group queue num failed", K(ret), K(tenant_id), K(group_num)); + } + } + } + return ret; +} + +int ObIOScheduler::remove_phyqueues(const uint64_t tenant_id) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < senders_.count(); ++i) { + ObIOSender *cur_sender = senders_.at(i); + if (OB_FAIL(cur_sender->remove_group_queues(tenant_id))) { + LOG_WARN("remove phy queue failed", K(ret), K(i), K(tenant_id)); + } + } + } + return ret; +} + +int ObIOScheduler::stop_phy_queues(const uint64_t tenant_id, const int64_t index) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id) || index < 0 || INT64_MAX == index)) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid config", K(ret), K(tenant_id), K(index)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < senders_.count(); ++i) { + ObIOSender *cur_sender = senders_.at(i); + if (OB_FAIL(cur_sender->stop_phy_queue(tenant_id, index))) { + LOG_WARN("stop phy queue failed", K(ret), K(index)); + } } } return ret; @@ -2215,7 +2550,7 @@ void ObIOCallbackManager::destroy() } for (int64_t i = 0; i < runners_.count(); ++i) { runners_.at(i)->wait(); - } + } for (int64_t i = 0; i < runners_.count(); ++i) { io_allocator_->free(runners_.at(i)); } @@ -2529,7 +2864,7 @@ int ObIOFaultDetector::record_read_failure(const ObIORequest &req) LOG_WARN("alloc RetryTask failed", K(ret)); } else { retry_task->io_info_ = req.io_info_; - retry_task->io_info_.flag_.set_category(ObIOCategory::PREWARM_IO); + retry_task->io_info_.flag_.set_group_id(0); retry_task->io_info_.callback_ = nullptr; retry_task->timeout_ms_ = 5000L; // 5s if (OB_FAIL(TG_PUSH_TASK(TGDefIDs::IO_HEALTH, retry_task))) { diff --git a/src/share/io/ob_io_struct.h b/src/share/io/ob_io_struct.h index f903b64d09..be0f2f3c00 100644 --- a/src/share/io/ob_io_struct.h +++ b/src/share/io/ob_io_struct.h @@ -140,21 +140,25 @@ class ObIOUsage final public: ObIOUsage(); ~ObIOUsage(); - void accumulate(const ObIORequest &req); + int init(const int64_t group_num); + int refresh_group_num (const int64_t group_num); + void accumulate(ObIORequest &req); void calculate_io_usage(); - typedef double AvgItems[static_cast(ObIOCategory::MAX_CATEGORY)][static_cast(ObIOMode::MAX_MODE)]; - void get_io_usage(AvgItems &avg_iops, AvgItems &avg_bytes, AvgItems &avg_rt_us) const; - void record_request_start(const ObIORequest &req); - void record_request_finish(const ObIORequest &req); - bool is_request_doing(const ObIOCategory category) const; + typedef ObSEArray, 2> AvgItems; + void get_io_usage(AvgItems &avg_iops, AvgItems &avg_bytes, AvgItems &avg_rt_us); + void record_request_start(ObIORequest &req); + void record_request_finish(ObIORequest &req); + bool is_request_doing(const int64_t index) const; + int64_t get_io_usage_num() const; int64_t to_string(char* buf, const int64_t buf_len) const; private: - ObIOStat io_stats_[static_cast(ObIOCategory::MAX_CATEGORY)][static_cast(ObIOMode::MAX_MODE)]; - ObIOStatDiff io_estimators_[static_cast(ObIOCategory::MAX_CATEGORY)][static_cast(ObIOMode::MAX_MODE)]; - AvgItems avg_iops_; - AvgItems avg_byte_; - AvgItems avg_rt_us_; - int64_t doing_request_count_[static_cast(ObIOCategory::MAX_CATEGORY)]; + ObSEArray, 2> io_stats_; + ObSEArray, 2> io_estimators_; + AvgItems group_avg_iops_; + AvgItems group_avg_byte_; + AvgItems group_avg_rt_us_; + int64_t group_num_; + ObSEArray doing_request_count_; }; class ObCpuUsage final @@ -187,15 +191,32 @@ private: ObIOScheduler &io_scheduler_; }; -struct ObIOCategoryQueues final { +struct ObIOGroupQueues final { public: - ObIOCategoryQueues(); - ~ObIOCategoryQueues(); - int init(); + ObIOGroupQueues(ObIAllocator &allocator); + ~ObIOGroupQueues(); + int init(const int64_t group_num); void destroy(); + TO_STRING_KV(K(is_inited_), K(other_phy_queue_), K(group_phy_queues_)); public: bool is_inited_; - ObPhyQueue phy_queues_[static_cast(ObIOCategory::MAX_CATEGORY) + 1]; + ObIAllocator &allocator_; + ObSEArray group_phy_queues_; + ObPhyQueue other_phy_queue_; +}; + + +struct ObSenderInfo final +{ +public: + ObSenderInfo(); + ~ObSenderInfo(); +public: + int64_t queuing_count_; + int64_t reservation_ts_; + int64_t group_limitation_ts_; + int64_t tenant_limitation_ts_; + int64_t proportion_ts_; }; class ObIOSender : public lib::TGRunnable @@ -203,7 +224,7 @@ class ObIOSender : public lib::TGRunnable public: ObIOSender(ObIAllocator &allocator); virtual ~ObIOSender(); - int init(const int32_t queue_depth); + int init(); void stop(); void wait(); void destroy(); @@ -217,23 +238,25 @@ public: int enqueue_request(ObIORequest &req); int enqueue_phy_queue(ObPhyQueue &phyqueue); int dequeue_request(ObIORequest *&req); - int remove_phy_queue(const uint64_t tenant_id); + int update_group_queue(const uint64_t tenant_id, const int64_t group_num); + int remove_group_queues(const uint64_t tenant_id); + int stop_phy_queue(const uint64_t tenant_id, const uint64_t index); int notify(); int32_t get_queue_count() const; + int get_sender_status(const uint64_t tenant_id, const uint64_t index, ObSenderInfo &sender_info); TO_STRING_KV(K(is_inited_), K(stop_submit_), KPC(io_queue_), K(tg_id_)); //private: void pop_and_submit(); int64_t calc_wait_timeout(const int64_t queue_deadline); int submit(ObIORequest &req); - - bool is_inited_; - ObIAllocator &allocator_; - bool stop_submit_; + int64_t sender_req_count_; int tg_id_; // thread group id + bool is_inited_; + bool stop_submit_; + ObIAllocator &allocator_; ObMClockQueue *io_queue_; ObThreadCond queue_cond_; - hash::ObHashMap tenant_map_; - int64_t sender_req_count_; + hash::ObHashMap tenant_groups_map_; }; @@ -242,14 +265,18 @@ class ObIOScheduler final public: ObIOScheduler(const ObIOConfig &io_config, ObIAllocator &allocator); ~ObIOScheduler(); - int init(const int64_t queue_count, const int64_t queue_depth, const int64_t schedule_media_id = 0); + int init(const int64_t queue_count, const int64_t schedule_media_id = 0); void destroy(); int start(); void stop(); void accumulate(const ObIORequest &req); - int schedule_request(ObIOClock &io_clock, ObIORequest &req); - int add_tenant_map(uint64_t tenant_id); - int remove_tenant_map(uint64_t tenant_id); + int schedule_request(ObTenantIOClock &io_clock, ObIORequest &req); + int init_group_queues(const uint64_t tenant_id, const int64_t group_num); + int update_group_queues(const uint64_t tenant_id, const int64_t group_num); + int remove_phyqueues(const uint64_t tenant_id); + int stop_phy_queues(const uint64_t tenant_id, const int64_t index); + ObIOSender *get_cur_sender(const int thread_id){ return senders_.at(thread_id); }; + int64_t get_senders_count() { return senders_.count(); } TO_STRING_KV(K(is_inited_), K(io_config_), K(senders_)); private: friend class ObIOTuner; diff --git a/src/share/redolog/ob_log_file_handler.cpp b/src/share/redolog/ob_log_file_handler.cpp index 05c3ea3d6a..3f6ab55d81 100644 --- a/src/share/redolog/ob_log_file_handler.cpp +++ b/src/share/redolog/ob_log_file_handler.cpp @@ -256,7 +256,7 @@ int ObLogFileHandler::inner_read(const ObIOFd &io_fd, void *buf, const int64_t s io_info.offset_ = offset + read_sz; io_info.size_ = size - read_sz; io_info.flag_.set_mode(ObIOMode::READ); - io_info.flag_.set_category(ObIOCategory::LOG_IO); + io_info.flag_.set_group_id(THIS_WORKER.get_group_id()); io_info.flag_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); io_info.buf_ = nullptr; io_info.callback_ = nullptr; @@ -330,7 +330,7 @@ int ObLogFileHandler::inner_write_impl(const ObIOFd &io_fd, void *buf, const int io_info.fd_ = io_fd; io_info.offset_ = offset; io_info.size_ = size; - io_info.flag_.set_category(ObIOCategory::LOG_IO); + io_info.flag_.set_group_id(THIS_WORKER.get_group_id()); io_info.flag_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_WRITE); io_info.buf_ = reinterpret_cast(buf); io_info.callback_ = nullptr; diff --git a/src/share/resource_manager/ob_cgroup_ctrl.cpp b/src/share/resource_manager/ob_cgroup_ctrl.cpp index 4546d68fcf..e342ff76c5 100644 --- a/src/share/resource_manager/ob_cgroup_ctrl.cpp +++ b/src/share/resource_manager/ob_cgroup_ctrl.cpp @@ -18,6 +18,7 @@ #include "lib/oblog/ob_log.h" #include "share/ob_errno.h" #include "share/config/ob_server_config.h" +#include "share/io/ob_io_manager.h" #include "share/resource_manager/ob_resource_plan_info.h" #include "share/resource_manager/ob_resource_manager.h" #include "share/resource_manager/ob_cgroup_ctrl.h" @@ -36,6 +37,28 @@ ObCgSet ObCgSet::instance_; } } +//集成IO参数 +int OBGroupIOInfo::init(int64_t min_percent, int64_t max_percent, int64_t weight_percent) +{ + int ret = OB_SUCCESS; + if (min_percent < 0 || min_percent > 100 || + max_percent < 0 || max_percent > 100 || + weight_percent < 0 || weight_percent > 100 || + min_percent > max_percent) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid io config", K(ret), K(min_percent), K(max_percent), K(weight_percent)); + } else { + min_percent_ = min_percent; + max_percent_ = max_percent; + weight_percent_ = weight_percent; + } + return ret; +} +bool OBGroupIOInfo::is_valid() const +{ + return min_percent_ >= 0 && max_percent_ >= min_percent_ && max_percent_ <= 100 && weight_percent_ >= 0 && weight_percent_ <= 100; +} + // 创建cgroup初始目录结构,将所有线程加入other组 int ObCgroupCtrl::init() { @@ -174,7 +197,7 @@ int ObCgroupCtrl::get_group_path( } else if (OB_FAIL(get_group_info_by_group_id(tenant_id, group_id, g_name))){ LOG_WARN("get group_name by id failed", K(group_id), K(ret)); } else { - group_name = const_cast(g_name.get_group_name().ptr()); + group_name = const_cast(g_name.get_value().ptr()); snprintf(group_path, path_bufsize, "%s/tenant_%04lu/%s", root_cgroup_, tenant_id, group_name); } @@ -308,6 +331,22 @@ int ObCgroupCtrl::remove_thread_from_cgroup(const pid_t tid, const uint64_t tena return ret; } +int ObCgroupCtrl::add_thread_to_group(const pid_t tid, + const uint64_t tenant_id, + const uint64_t group_id) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(add_thread_to_cgroup(tid, tenant_id, group_id))) { + LOG_WARN("fail to add thread to group", K(ret), K(tid), K(group_id), K(tenant_id)); + } else { + LOG_INFO("set backup pid to group success", K(tenant_id), K(group_id), K(tid)); + } + return ret; +} + int ObCgroupCtrl::set_cpu_shares(const int32_t cpu_shares, const uint64_t tenant_id, int64_t group_id) { int ret = OB_SUCCESS; @@ -551,6 +590,127 @@ int ObCgroupCtrl::get_cpu_time(const uint64_t tenant_id, int64_t &cpu_time) return ret; } +int ObCgroupCtrl::set_group_iops(const uint64_t tenant_id, + const int level, // UNUSED + const int64_t group_id, + const OBGroupIOInfo &group_io) +{ + int ret = OB_SUCCESS; + UNUSED(level); + + ObRefHolder tenant_holder; + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(OB_IO_MANAGER.get_tenant_io_manager(tenant_id, tenant_holder))) { + LOG_WARN("get tenant io manager failed", K(ret), K(tenant_id)); + } else if (OB_FAIL(tenant_holder.get_ptr()->modify_io_config(group_id, + group_io.min_percent_, + group_io.max_percent_, + group_io.weight_percent_))) { + LOG_WARN("modify consumer group iops failed", K(ret), K(group_id), K(tenant_id), K(group_id)); + } + return ret; +} + +int ObCgroupCtrl::reset_all_group_iops(const uint64_t tenant_id, + const int level) // UNUSED +{ + int ret = OB_SUCCESS; + UNUSED(level); + + ObRefHolder tenant_holder; + // 删除plan, IO层代表对应的所有group资源为0,0,0, 但group对应的数据结构不会被释放以防用户后续复用 + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(OB_IO_MANAGER.get_tenant_io_manager(tenant_id, tenant_holder))) { + LOG_WARN("get tenant io manager failed", K(ret), K(tenant_id)); + } else if (OB_FAIL(tenant_holder.get_ptr()->reset_all_group_config())) { + LOG_WARN("reset consumer group iops failed", K(ret), K(tenant_id)); + } else { + LOG_INFO("group iops control has reset, delete cur plan success", K(tenant_id)); + } + return ret; +} + +int ObCgroupCtrl::reset_group_iops(const uint64_t tenant_id, + const int level, // UNUSED + const common::ObString &consumer_group) +{ + int ret = OB_SUCCESS; + UNUSED(level); + + uint64_t group_id = 0; + share::ObGroupName group_name; + group_name.set_value(consumer_group); + ObResourceMappingRuleManager &rule_mgr = G_RES_MGR.get_mapping_rule_mgr(); + ObRefHolder tenant_holder; + + // 删除directive, IO层代表对应的group资源为0,0,0, 但group对应的数据结构不会被释放以防用户后续复用 + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(rule_mgr.get_group_id_by_name(tenant_id, group_name, group_id))) { + if (OB_HASH_NOT_EXIST == ret) { + //创建directive后立刻删除,可能还没有被刷到存储层或plan未生效,此时不再进行后续操作 + ret = OB_SUCCESS; + LOG_INFO("delete directive success with no_releated_io_module", K(consumer_group), K(tenant_id)); + } else { + LOG_WARN("fail get group id", K(ret), K(group_id), K(group_name)); + } + } else if (group_id < GROUP_START_ID) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid group id", K(ret), K(group_id), K(group_name)); + } else if (OB_FAIL(OB_IO_MANAGER.get_tenant_io_manager(tenant_id, tenant_holder))) { + LOG_WARN("get tenant io manager failed", K(ret), K(tenant_id)); + } else if (OB_FAIL(tenant_holder.get_ptr()->reset_consumer_group_config(group_id))) { + LOG_WARN("reset consumer group iops failed", K(ret), K(group_id), K(tenant_id), K(group_id)); + } else { + LOG_INFO("group iops control has reset, delete directive success", K(consumer_group), K(tenant_id), K(group_id)); + } + return ret; +} + +int ObCgroupCtrl::delete_group_iops(const uint64_t tenant_id, + const int level, // UNUSED + const common::ObString &consumer_group) +{ + int ret = OB_SUCCESS; + UNUSED(level); + + uint64_t group_id = 0; + share::ObGroupName group_name; + group_name.set_value(consumer_group); + ObResourceMappingRuleManager &rule_mgr = G_RES_MGR.get_mapping_rule_mgr(); + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(rule_mgr.get_group_id_by_name(tenant_id, group_name, group_id))) { + if (OB_HASH_NOT_EXIST == ret) { + //创建group后立刻删除,可能还没有被刷到存储层或plan未生效,此时不再进行后续操作 + ret = OB_SUCCESS; + LOG_INFO("delete group success with no_releated_io_module", K(consumer_group), K(tenant_id)); + } else { + LOG_WARN("fail get group id", K(ret), K(group_id), K(group_name)); + } + } else if (group_id < GROUP_START_ID) { + //OTHER_GROUPS and all cannot be deleted + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid group id", K(ret), K(group_id), K(group_name)); + } else { + ObRefHolder tenant_holder; + if (OB_FAIL(OB_IO_MANAGER.get_tenant_io_manager(tenant_id, tenant_holder))) { + LOG_WARN("get tenant io manager failed", K(ret), K(tenant_id)); + } else if (OB_FAIL(tenant_holder.get_ptr()->delete_consumer_group_config(group_id))) { + LOG_WARN("stop group iops control failed", K(ret), K(tenant_id), K(group_id)); + } else { + LOG_INFO("stop group iops_ctrl success when delete group", K(consumer_group), K(tenant_id), K(group_id)); + } + } + return ret; +} + int ObCgroupCtrl::init_cgroup_root_dir_(const char *cgroup_path) { int ret = OB_SUCCESS; @@ -619,7 +779,7 @@ int ObCgroupCtrl::write_string_to_file_(const char *filename, const char *conten } else if ((tmp_ret = write(fd, content, strlen(content))) < 0) { ret = OB_IO_ERROR; LOG_WARN("write file error", - K(filename), K(content), K(ret), K(errno), KERRMSG, K(ret)); + K(filename), K(content), K(ret), K(errno), KERRMSG, K(tmp_ret)); } else { // do nothing } diff --git a/src/share/resource_manager/ob_cgroup_ctrl.h b/src/share/resource_manager/ob_cgroup_ctrl.h index 959b1b3627..a0b263e189 100644 --- a/src/share/resource_manager/ob_cgroup_ctrl.h +++ b/src/share/resource_manager/ob_cgroup_ctrl.h @@ -15,7 +15,6 @@ #include #include - namespace oceanbase { namespace common @@ -65,6 +64,23 @@ private: const char *names_[OBCG_MAXNUM]; }; +struct OBGroupIOInfo final +{ +public: + OBGroupIOInfo() + : min_percent_(0), + max_percent_(100), + weight_percent_(0) + {} + int init(const int64_t min_percent, const int64_t max_percent, const int64_t weight_percent); + void reset(); + bool is_valid() const; +public: + uint64_t min_percent_; + uint64_t max_percent_; + uint64_t weight_percent_; +}; + class ObCgroupCtrl { public: @@ -86,6 +102,10 @@ public: // 从指定租户cgroup组移除指定tid int remove_thread_from_cgroup(const pid_t tid, const uint64_t tenant_id); + // 后台线程绑定接口 + int add_thread_to_group(const pid_t tid, + const uint64_t tenant_id, + const uint64_t group_id); // 设定指定租户cgroup组的cpu.shares int set_cpu_shares(const int32_t cpu_shares, const uint64_t tenant_id, int64_t group_id = INT64_MAX); int get_cpu_shares(int32_t &cpu_shares, const uint64_t tenant_id, int64_t group_id = INT64_MAX); @@ -110,6 +130,25 @@ public: int get_cpu_time(const uint64_t tenant_id, int64_t &cpu_time); // 获取某段时间内cpu占用率 int get_cpu_usage(const uint64_t tenant_id, int32_t &cpu_usage); + + + // 设定指定租户cgroup组的iops,直接更新到租户io_config + int set_group_iops(const uint64_t tenant_id, + const int level, // UNUSED + const int64_t group_id, + const OBGroupIOInfo &group_io); + // 删除正在使用的plan反应到IO层:重置所有IOPS + int reset_all_group_iops(const uint64_t tenant_id, + const int level);// UNUSED + // 删除directive反应到IO层:重置IOPS + int reset_group_iops(const uint64_t tenant_id, + const int level, // UNUSED + const common::ObString &consumer_group); + // 删除group反应到IO层:停用对应的group结构 + int delete_group_iops(const uint64_t tenant_id, + const int level, // UNUSED + const common::ObString &consumer_group); + // 根据 consumer group 动态创建 cgroup int create_user_tenant_group_dir( const uint64_t tenant_id, diff --git a/src/share/resource_manager/ob_resource_manager_proxy.cpp b/src/share/resource_manager/ob_resource_manager_proxy.cpp index 5c794dac3a..74835c5215 100644 --- a/src/share/resource_manager/ob_resource_manager_proxy.cpp +++ b/src/share/resource_manager/ob_resource_manager_proxy.cpp @@ -21,6 +21,8 @@ #include "share/ob_errno.h" #include "share/schema/ob_schema_utils.h" #include "share/resource_manager/ob_resource_plan_manager.h" +#include "share/resource_manager/ob_cgroup_ctrl.h" +#include "share/resource_manager/ob_resource_manager.h" #include "share/inner_table/ob_inner_table_schema_constants.h" #include "share/resource_manager/ob_resource_mapping_rule_manager.h" #include "common/ob_timeout_ctx.h" @@ -168,6 +170,26 @@ int ObResourceManagerProxy::delete_plan( LOG_WARN("fail to execute sql", K(sql), K(ret)); } } + if (OB_SUCC(ret)) { + ObResMgrVarcharValue cur_plan; + ObResourcePlanManager &plan_mgr = G_RES_MGR.get_plan_mgr(); + if (OB_FAIL(plan_mgr.get_cur_plan(tenant_id, cur_plan))) { + LOG_WARN("get cur plan failed", K(ret), K(tenant_id), K(cur_plan)); + } else if (cur_plan.get_value() != plan) { + //删除非当前使用plan,do nothing + } else { + //删除当前使用的plan,把当前所有IO资源置空 + if (OB_FAIL(GCTX.cgroup_ctrl_->reset_all_group_iops( + tenant_id, + 1))) { + LOG_WARN("reset cur plan group directive failed",K(plan), K(ret)); + } else if (OB_FAIL(reset_all_mapping_rules())) { + LOG_WARN("reset hashmap failed when delete using plan"); + } else { + LOG_INFO("reset cur plan group directive success",K(plan), K(ret)); + } + } + } return ret; } @@ -185,7 +207,7 @@ int ObResourceManagerProxy::allocate_consumer_group_id( ObSqlString sql; const char *tname = OB_ALL_RES_MGR_CONSUMER_GROUP_TNAME; if (OB_FAIL(sql.assign_fmt( - "SELECT /* ALLOC_MAX_GROUP_ID */ COALESCE(MAX(CONSUMER_GROUP_ID) + 1, 1) AS NEXT_GROUP_ID FROM %s " + "SELECT /* ALLOC_MAX_GROUP_ID */ COALESCE(MAX(CONSUMER_GROUP_ID) + 1, 10000) AS NEXT_GROUP_ID FROM %s " "WHERE TENANT_ID = %ld", tname, ObSchemaUtils::get_extract_tenant_id(tenant_id, tenant_id)))) { LOG_WARN("fail format sql", K(ret)); @@ -289,7 +311,9 @@ int ObResourceManagerProxy::delete_consumer_group( if (trans_guard.ready()) { int64_t affected_rows = 0; ObSqlString sql; + // 删除 group 时要级联删除 directive const char *tname_consumer_group = OB_ALL_RES_MGR_CONSUMER_GROUP_TNAME; + const char *tname_directive = OB_ALL_RES_MGR_DIRECTIVE_TNAME; if (OB_FAIL(sql.assign_fmt( "DELETE /* REMOVE_RES_CONSUMER_GROUP */ FROM %s " "WHERE TENANT_ID = %ld AND CONSUMER_GROUP = '%.*s'", @@ -305,24 +329,45 @@ int ObResourceManagerProxy::delete_consumer_group( } else if (1 != affected_rows) { ret = OB_ERR_CONSUMER_GROUP_NOT_EXIST; LOG_USER_ERROR(OB_ERR_CONSUMER_GROUP_NOT_EXIST, consumer_group.length(), consumer_group.ptr()); + } else if (OB_FAIL(sql.assign_fmt( + "DELETE /* REMOVE_RES_CONSUMER_GROUP */ FROM %s " + "WHERE TENANT_ID = %ld AND GROUP_OR_SUBPLAN = '%.*s'", + tname_directive, ObSchemaUtils::get_extract_tenant_id(tenant_id, tenant_id), + consumer_group.length(), consumer_group.ptr()))) { + LOG_WARN("fail append value", K(ret)); + } else if (OB_FAIL(trans.write( + tenant_id, + sql.ptr(), + affected_rows))) { + trans.reset_last_error(); + LOG_WARN("fail to execute sql", K(sql), K(ret)); + } + } + if (OB_SUCC(ret)) { + // 在这里inner sql之后就stop io_control的原因是,无法从内部表读到被删除group的信息 + if (OB_FAIL(GCTX.cgroup_ctrl_->delete_group_iops(tenant_id, 1, consumer_group))) { + LOG_WARN("fail to stop cur iops isolation", K(ret), K(tenant_id), K(consumer_group)); } } return ret; } int ObResourceManagerProxy::create_plan_directive( - uint64_t tenant_id, - const common::ObString &plan, - const common::ObString &group, - const common::ObObj &comment, - const common::ObObj &mgmt_p1, - const common::ObObj &utilization_limit) + uint64_t tenant_id, + const common::ObString &plan, + const common::ObString &group, + const common::ObObj &comment, + const common::ObObj &mgmt_p1, + const common::ObObj &utilization_limit, + const common::ObObj &min_iops, + const common::ObObj &max_iops, + const common::ObObj &weight_iops) { int ret = OB_SUCCESS; ObMySQLTransaction trans; TransGuard trans_guard(trans, tenant_id, ret); if (trans_guard.ready()) { - ret = create_plan_directive(trans, tenant_id, plan, group, comment, mgmt_p1, utilization_limit); + ret = create_plan_directive(trans, tenant_id, plan, group, comment, mgmt_p1, utilization_limit, min_iops, max_iops, weight_iops); } return ret; } @@ -330,11 +375,14 @@ int ObResourceManagerProxy::create_plan_directive( int ObResourceManagerProxy::create_plan_directive( common::ObMySQLTransaction &trans, uint64_t tenant_id, - const ObString &plan, - const ObString &group, - const ObObj &comments, - const ObObj &mgmt_p1, - const ObObj &utilization_limit) + const common::ObString &plan, + const common::ObString &group, + const common::ObObj &comments, + const common::ObObj &mgmt_p1, + const common::ObObj &utilization_limit, + const common::ObObj &min_iops, + const common::ObObj &max_iops, + const common::ObObj &weight_iops) { int ret = OB_SUCCESS; bool consumer_group_exist = true; @@ -370,6 +418,40 @@ int ObResourceManagerProxy::create_plan_directive( if (OB_SUCC(ret) && OB_SUCC(get_percentage("UTILIZATION_LIMIT", utilization_limit, v))) { SQL_COL_APPEND_VALUE(sql, values, v, "UTILIZATION_LIMIT", "%ld"); } + uint64_t tenant_data_version = 0; + if (OB_FAIL(ret)) { + // do nothing + } else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) { + LOG_WARN("get tenant data version failed", K(ret)); + } else if (tenant_data_version < DATA_VERSION_4_1_0_0 && ( + (OB_SUCC(get_percentage("MIN_IOPS", min_iops, v) && v != 0)) || + (OB_SUCC(get_percentage("MAX_IOPS", max_iops, v) && v != 100)) || + (OB_SUCC(get_percentage("WEIGHT_IOPS", weight_iops, v) && v != 0)))) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("iops setting is not suppported when tenant's data version is below 4.1.0.0", K(ret)); + } else if (tenant_data_version >= DATA_VERSION_4_1_0_0) { + int64_t iops_minimum = 0; + int64_t iops_maximum = 100; + if (OB_SUCC(ret) && OB_SUCC(get_percentage("MIN_IOPS", min_iops, v))) { + iops_minimum = v; + SQL_COL_APPEND_VALUE(sql, values, v, "MIN_IOPS", "%ld"); + } + if (OB_SUCC(ret) && OB_SUCC(get_percentage("MAX_IOPS", max_iops, v))) { + iops_maximum = v; + bool is_valid = false; + if (OB_FAIL(check_iops_validity(tenant_id, plan, group, iops_minimum, iops_maximum, is_valid))) { + LOG_WARN("check iops setting failed", K(tenant_id), K(plan), K(iops_minimum), K(iops_maximum)); + } else if (OB_UNLIKELY(!is_valid)) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid iops config", K(ret), K(tenant_id), K(iops_minimum), K(iops_maximum)); + } else { + SQL_COL_APPEND_VALUE(sql, values, v, "MAX_IOPS", "%ld"); + } + } + if (OB_SUCC(ret) && OB_SUCC(get_percentage("WEIGHT_IOPS", weight_iops, v))) { + SQL_COL_APPEND_VALUE(sql, values, v, "WEIGHT_IOPS", "%ld"); + } + } if (OB_SUCC(ret)) { int64_t affected_rows = 0; if (OB_FAIL(sql.append_fmt(") VALUES (%.*s)", @@ -556,6 +638,25 @@ int ObResourceManagerProxy::check_if_user_exist( return ret; } +int ObResourceManagerProxy::check_if_function_exist(const ObString &function_name, bool &exist) +{ + int ret = OB_SUCCESS; + if (0 == function_name.compare("COMPACTION_HIGH") || + 0 == function_name.compare("HA_HIGH") || + 0 == function_name.compare("COMPACTION_MID") || + 0 == function_name.compare("HA_MID") || + 0 == function_name.compare("COMPACTION_LOW") || + 0 == function_name.compare("HA_LOW") || + 0 == function_name.compare("DDL_HIGH") || + 0 == function_name.compare("DDL")) { + exist = true; + } else { + exist = false; + LOG_WARN("invalid function name", K(function_name)); + } + return ret; +} + int ObResourceManagerProxy::check_if_column_exist( uint64_t tenant_id, const ObString &db_name, @@ -696,13 +797,123 @@ int ObResourceManagerProxy::get_percentage(const char *name, const ObObj &obj, i return ret; } +int ObResourceManagerProxy::check_iops_validity( + const uint64_t tenant_id, + const common::ObString &plan_name, + const common::ObString &group, + const int64_t iops_minimum, + const int64_t iops_maximum, + bool &valid) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (iops_maximum < iops_minimum) { + // precheck + valid = false; + } else { + ObSEArray directives; + if (OB_FAIL(get_all_plan_directives(tenant_id, plan_name, directives))) { + LOG_WARN("fail get plan directive", K(tenant_id), K(plan_name), K(ret)); + } else { + uint64_t total_min = 0; + for (int64_t i = 0; OB_SUCC(ret) && i < directives.count(); ++i) { + ObPlanDirective &cur_directive = directives.at(i); + if (cur_directive.group_id_ < GROUP_START_ID) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected group id", K(cur_directive)); + } else if (OB_UNLIKELY(!cur_directive.is_valid())) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid group io config", K(cur_directive)); + } else if ((0 == group.compare(cur_directive.group_name_.get_value()))) { + //skip cur group + } else { + total_min += cur_directive.min_iops_; + } + } + if(OB_SUCC(ret)) { + total_min += iops_minimum; + if (total_min > 100) { + valid = false; + LOG_WARN("invalid group io config", K(total_min), K(iops_minimum), K(iops_maximum), K(plan_name)); + } else { + valid = true; + } + } + } + } + return ret; +} + +int ObResourceManagerProxy::get_user_mapping_info( + const uint64_t tenant_id, + const common::ObString &user, + ObResourceUserMappingRule &rule) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else { + ObMySQLTransaction trans; + TransGuard trans_guard(trans, tenant_id, ret); + if (trans_guard.ready()) { + ObSQLClientRetryWeak sql_client_retry_weak( + &trans, tenant_id, OB_ALL_RES_MGR_MAPPING_RULE_TID); + SMART_VAR(ObMySQLProxy::MySQLResult, res) { + common::sqlclient::ObMySQLResult *result = NULL; + ObSqlString sql; + const char *t_a_res_name = OB_ALL_RES_MGR_MAPPING_RULE_TNAME; + const char *t_b_user_name = OB_ALL_USER_TNAME; + uint64_t sql_tenant_id = ObSchemaUtils::get_extract_tenant_id(tenant_id, tenant_id); + if (OB_FAIL(sql.assign_fmt( + "SELECT /* GET_USER_ID_FROM_MAPPING_RULE */ " + "user_id FROM %s a, %s b " + "WHERE a.`value` = b.user_name " + "AND a.TENANT_ID = %ld AND b.tenant_id = %ld " + "AND a.attribute = 'USER' AND a.`value` = '%.*s'", + t_a_res_name, t_b_user_name, + sql_tenant_id, sql_tenant_id, user.length(), user.ptr()))) { + LOG_WARN("fail format sql", K(ret)); + } else if (OB_FAIL(sql_client_retry_weak.read(res, tenant_id, sql.ptr()))) { + LOG_WARN("fail to execute sql", K(sql), K(ret)); + } else if (NULL == (result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail query sql", K(t_a_res_name), K(tenant_id), K(ret)); + } else { + int64_t affected_rows = 0; + while (OB_SUCC(result->next())) { + EXTRACT_INT_FIELD_TO_CLASS_MYSQL(*result, user_id, rule, uint64_t); + ++affected_rows; + } + if (OB_ITER_END == ret) { + if (OB_UNLIKELY(!is_single_row(affected_rows))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected value. expect only 1 row affected", K(ret), K(affected_rows), K(user)); + } else { + ret = OB_SUCCESS; + } + } else { + LOG_WARN("fail get next row", K(ret), K(user), K(tenant_id)); + } + } + } + } + } + return ret; +} + int ObResourceManagerProxy::update_plan_directive( uint64_t tenant_id, const ObString &plan, const ObString &group, const ObObj &comments, const ObObj &mgmt_p1, - const ObObj &utilization_limit) + const ObObj &utilization_limit, + const ObObj &min_iops, + const ObObj &max_iops, + const ObObj &weight_iops) { int ret = OB_SUCCESS; ObMySQLTransaction trans; @@ -719,7 +930,8 @@ int ObResourceManagerProxy::update_plan_directive( ret = OB_ERR_PLAN_DIRECTIVE_NOT_EXIST; LOG_USER_ERROR(OB_ERR_PLAN_DIRECTIVE_NOT_EXIST, plan.length(), plan.ptr(), group.length(), group.ptr()); - } else if (comments.is_null() && mgmt_p1.is_null() && utilization_limit.is_null()) { + } else if (comments.is_null() && mgmt_p1.is_null() && utilization_limit.is_null() && + min_iops.is_null() && max_iops.is_null() && weight_iops.is_null()) { // 没有指定任何有效参数,什么都不做,也不报错。兼容 Oracle 行为。 ret = OB_SUCCESS; } else if (OB_FAIL(sql.assign_fmt("UPDATE /* UPDATE_PLAN_DIRECTIVE */ %s SET ", tname))) { @@ -743,6 +955,65 @@ int ObResourceManagerProxy::update_plan_directive( ret = sql.append_fmt("%s UTILIZATION_LIMIT=%ld", comma, v); comma = ","; } + uint64_t tenant_data_version = 0; + if (OB_FAIL(ret)) { + // do nothing + } else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) { + LOG_WARN("get tenant data version failed", K(ret)); + } else if (tenant_data_version < DATA_VERSION_4_1_0_0 && ( + (OB_SUCC(get_percentage("NEW_MIN_IOPS", min_iops, v) && v != 0)) || + (OB_SUCC(get_percentage("NEW_MAX_IOPS", max_iops, v) && v != 100)) || + (OB_SUCC(get_percentage("NEW_WEIGHT_IOPS", weight_iops, v) && v != 0)))) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("iops setting is not suppported when tenant's data version is below 4.1.0.0", K(ret)); + } else if (tenant_data_version >= DATA_VERSION_4_1_0_0) { + if (min_iops.is_null() && max_iops.is_null()) { + // no need to check + } else { + // check if iops_config is valid + int64_t new_iops_minimum = 0; + int64_t new_iops_maximum = 100; + if (OB_SUCC(ret)) { + ObPlanDirective directive; + if (OB_FAIL(get_iops_config(tenant_id, plan, group, directive))) { + LOG_WARN("get iops config from table failed", K(ret), K(plan), K(group), K(directive)); + } else { + new_iops_minimum = directive.min_iops_; + new_iops_maximum = directive.max_iops_; + } + } + if (OB_SUCC(ret) && + !min_iops.is_null() && + OB_SUCC(get_percentage("NEW_MIN_IOPS", min_iops, v))) { + new_iops_minimum = v; + ret = sql.append_fmt("%s MIN_IOPS=%ld", comma, v); + comma = ","; + } + if (OB_SUCC(ret) && + !max_iops.is_null() && + OB_SUCC(get_percentage("NEW_MAX_IOPS", max_iops, v))) { + new_iops_maximum = v; + } + if (OB_SUCC(ret)) { + bool is_valid = false; + if (OB_FAIL(check_iops_validity(tenant_id, plan, group, new_iops_minimum, new_iops_maximum, is_valid))) { + LOG_WARN("check iops setting failed", K(tenant_id), K(plan), K(new_iops_minimum), K(new_iops_maximum)); + } else if (OB_UNLIKELY(!is_valid)) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid iops config", K(ret), K(tenant_id), K(new_iops_minimum), K(new_iops_maximum)); + } else { + ret = sql.append_fmt("%s MAX_IOPS=%ld", comma, v); + comma = ","; + } + } + } + if (OB_SUCC(ret) && + !weight_iops.is_null() && + OB_SUCC(get_percentage("NEW_WEIGHT_IOPS", weight_iops, v))) { + ret = sql.append_fmt("%s WEIGHT_IOPS=%ld", comma, v); + comma = ","; + } + } if (OB_FAIL(ret)) { LOG_WARN("fail append value", K(ret)); } else if (OB_FAIL(sql.append_fmt( @@ -801,6 +1072,15 @@ int ObResourceManagerProxy::delete_plan_directive( LOG_WARN("affected row value not expected", K(affected_rows), K(ret)); } } + if (OB_SUCC(ret)) { + // 在这里inner sql之后就stop的原因是, 无法从内部表读到被删除group的信息 + if (OB_FAIL(GCTX.cgroup_ctrl_->reset_group_iops( + tenant_id, + 1, + group))) { + LOG_WARN("reset deleted group directive failed", K(ret), K(group)); + } + } return ret; } @@ -827,7 +1107,7 @@ int ObResourceManagerProxy::get_all_plan_directives( const char *tname = OB_ALL_RES_MGR_DIRECTIVE_TNAME; if (OB_FAIL(sql.assign_fmt( "SELECT /* GET_ALL_PLAN_DIRECTIVE_SQL */ " - "group_or_subplan group_name, mgmt_p1, utilization_limit FROM %s " + "* FROM %s " "WHERE TENANT_ID = %ld AND PLAN = '%.*s'", tname, ObSchemaUtils::get_extract_tenant_id(tenant_id, tenant_id), plan.length(), plan.ptr()))) { @@ -841,11 +1121,22 @@ int ObResourceManagerProxy::get_all_plan_directives( while (OB_SUCC(result->next())) { ObPlanDirective directive; directive.set_tenant_id(tenant_id); - EXTRACT_VARCHAR_FIELD_TO_CLASS_MYSQL(*result, group_name, directive); + EXTRACT_VARCHAR_FIELD_TO_CLASS_MYSQL(*result, group_or_subplan, directive); EXTRACT_INT_FIELD_TO_CLASS_MYSQL(*result, mgmt_p1, directive, int64_t); EXTRACT_INT_FIELD_TO_CLASS_MYSQL(*result, utilization_limit, directive, int64_t); - if (OB_SUCC(ret) && OB_FAIL(directives.push_back(directive))) { - LOG_WARN("fail push back directives", K(directive), K(ret)); + bool skip_null_error = false; + bool skip_column_error = true; + EXTRACT_INT_FIELD_MYSQL_WITH_DEFAULT_VALUE(*result, "min_iops", directive.min_iops_, int64_t, skip_null_error, skip_column_error, 0); + EXTRACT_INT_FIELD_MYSQL_WITH_DEFAULT_VALUE(*result, "max_iops", directive.max_iops_, int64_t, skip_null_error, skip_column_error, 100); + EXTRACT_INT_FIELD_MYSQL_WITH_DEFAULT_VALUE(*result, "weight_iops", directive.weight_iops_, int64_t, skip_null_error, skip_column_error, 0); + if (OB_SUCC(ret)) { + ObResourceMappingRuleManager &rule_mgr = G_RES_MGR.get_mapping_rule_mgr(); + //如果读失败了,可能是group info还没有放到map里,因此不执行该directive直到下一次刷新 + if (OB_FAIL(rule_mgr.get_group_id_by_name(tenant_id, directive.group_name_, directive.group_id_))) { + LOG_WARN("fail get group id",K(ret), K(directive.group_id_), K(directive.group_name_)); + } else if (OB_FAIL(directives.push_back(directive))) { + LOG_WARN("fail push back directives", K(directive), K(ret)); + } } } if (OB_ITER_END == ret) { @@ -893,6 +1184,10 @@ int ObResourceManagerProxy::replace_mapping_rule( if (OB_FAIL(replace_user_mapping_rule(trans, tenant_id, attribute, value, consumer_group))) { LOG_WARN("replace user mapping rule failed", K(ret)); } + } else if (0 == attribute.case_compare("function")) { + if (OB_FAIL(replace_function_mapping_rule(trans, tenant_id, attribute, value, consumer_group))) { + LOG_WARN("replace user mapping rule failed", K(ret)); + } } else if (0 == attribute.case_compare("column")) { if (OB_FAIL(replace_column_mapping_rule(trans, tenant_id, attribute, value, consumer_group, session))) { LOG_WARN("replace column mapping rule failed", K(ret)); @@ -913,6 +1208,7 @@ int ObResourceManagerProxy::replace_user_mapping_rule(ObMySQLTransaction &trans, { int ret = OB_SUCCESS; bool user_exist = true; + uint64_t user_id = 0; if (OB_SUCC(ret)) { // if user not exists, do nothing, do not throw error // https://work.aone.alibaba-inc.com/issue/34248669 @@ -922,6 +1218,15 @@ int ObResourceManagerProxy::replace_user_mapping_rule(ObMySQLTransaction &trans, LOG_WARN("fail check if user exist", K(tenant_id), K(value), K(ret)); } } + if (OB_SUCC(ret) && user_exist && consumer_group.empty()) { + // get user_id for reset map + ObResourceUserMappingRule rule; + if (OB_FAIL(get_user_mapping_info(tenant_id, value, rule))) { + LOG_WARN("get user info failed", K(ret), K(tenant_id), K(value)); + } else { + user_id = rule.user_id_; + } + } if (OB_SUCC(ret) && user_exist) { ObSqlString sql; const char *tname = OB_ALL_RES_MGR_MAPPING_RULE_TNAME; @@ -953,15 +1258,85 @@ int ObResourceManagerProxy::replace_user_mapping_rule(ObMySQLTransaction &trans, } } } + if (OB_SUCC(ret) && user_exist && consumer_group.empty()) { + // reset map + if (consumer_group.empty()) { + ObResourceMappingRuleManager &rule_mgr = G_RES_MGR.get_mapping_rule_mgr(); + if (OB_FAIL(rule_mgr.reset_group_id_by_user(tenant_id, user_id))) { + LOG_WARN("fail reset user_group map", K(ret), K(tenant_id), K(user_id), K(value)); + } + } + } + } + } + return ret; +} + +int ObResourceManagerProxy::replace_function_mapping_rule(ObMySQLTransaction &trans, uint64_t tenant_id, + const ObString &attribute, const ObString &value, + const ObString &consumer_group) +{ + int ret = OB_SUCCESS; + bool function_exist = false; + if (OB_SUCC(ret)) { + // Same as user rule, the mapping is unsuccessful but no error is thrown + if (OB_FAIL(check_if_function_exist(value, function_exist))) { + LOG_WARN("fail check if function exist", K(tenant_id), K(value), K(ret)); + } else if (OB_UNLIKELY(!function_exist)) { + LOG_USER_ERROR(OB_NOT_SUPPORTED, "invalid function name, please check"); + } + } + if (OB_SUCC(ret) && function_exist) { + ObSqlString sql; + const char *tname = OB_ALL_RES_MGR_MAPPING_RULE_TNAME; + if (OB_FAIL(sql.assign_fmt("REPLACE INTO %s (", tname))) { + STORAGE_LOG(WARN, "append table name failed, ", K(ret)); + } else { + ObSqlString values; + SQL_COL_APPEND_VALUE(sql, values, ObSchemaUtils::get_extract_tenant_id(tenant_id, tenant_id), "tenant_id", "%lu"); + SQL_COL_APPEND_STR_VALUE(sql, values, attribute.ptr(), attribute.length(), "attribute"); + SQL_COL_APPEND_STR_VALUE(sql, values, value.ptr(), value.length(), "value"); + SQL_COL_APPEND_STR_VALUE(sql, values, consumer_group.ptr(), consumer_group.length(), "consumer_group"); + if (OB_SUCC(ret)) { + int64_t affected_rows = 0; + if (OB_FAIL(sql.append_fmt(") VALUES (%.*s)", + static_cast(values.length()), + values.ptr()))) { + LOG_WARN("append sql failed, ", K(ret)); + } else if (OB_FAIL(trans.write(tenant_id, + sql.ptr(), + affected_rows))) { + trans.reset_last_error(); + LOG_WARN("fail to execute sql", K(sql), K(ret)); + } else { + if (is_single_row(affected_rows) || is_double_row(affected_rows)) { + // insert or replace + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected value. expect 1 or 2 row affected", K(affected_rows), K(sql), K(ret)); + } + } + } + if (OB_SUCC(ret) && function_exist && consumer_group.empty()) { + // reset map + if (consumer_group.empty()) { + ObResourceMappingRuleManager &rule_mgr = G_RES_MGR.get_mapping_rule_mgr(); + ObResMgrVarcharValue func; + func.set_value(value); + if (OB_FAIL(rule_mgr.reset_group_id_by_function(tenant_id, func))) { + LOG_WARN("fail reset user_group map", K(ret), K(tenant_id), K(func), K(value)); + } + } + } } } return ret; } int ObResourceManagerProxy::replace_column_mapping_rule(ObMySQLTransaction &trans, uint64_t tenant_id, - const ObString &attribute, const ObString &value, - const ObString &consumer_group, - const sql::ObSQLSessionInfo &session) + const ObString &attribute, const ObString &value, + const ObString &consumer_group, + const sql::ObSQLSessionInfo &session) { int ret = OB_SUCCESS; bool both_exist = true; @@ -1107,6 +1482,54 @@ int ObResourceManagerProxy::get_all_resource_mapping_rules( return ret; } +int ObResourceManagerProxy::get_all_group_info( + uint64_t tenant_id, + const common::ObString &plan, + common::ObIArray &rules) +{ + int ret = OB_SUCCESS; + ObMySQLTransaction trans; + TransGuard trans_guard(trans, tenant_id, ret); + if (trans_guard.ready()) { + ObSQLClientRetryWeak sql_client_retry_weak( + &trans, tenant_id, OB_ALL_RES_MGR_CONSUMER_GROUP_TID); + SMART_VAR(ObMySQLProxy::MySQLResult, res) { + common::sqlclient::ObMySQLResult *result = NULL; + ObSqlString sql; + const char *tname = OB_ALL_RES_MGR_CONSUMER_GROUP_TNAME; + if (OB_FAIL(sql.assign_fmt( + "SELECT /* GET_ALL_GROUP_INFOS */ " + "consumer_group as group_name," + "consumer_group_id as group_id " + "FROM %s WHERE TENANT_ID = %ld", + tname, ObSchemaUtils::get_extract_tenant_id(tenant_id, tenant_id)))) { + LOG_WARN("fail format sql", K(ret)); + } else if (OB_FAIL(sql_client_retry_weak.read(res, tenant_id, sql.ptr()))) { + LOG_WARN("fail to execute sql", K(sql), K(ret)); + } else if (NULL == (result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail query sql", K(tname), K(tenant_id), K(ret)); + } else { + while (OB_SUCC(result->next())) { + ObResourceUserMappingRule rule; + rule.set_tenant_id(tenant_id); + EXTRACT_INT_FIELD_TO_CLASS_MYSQL(*result, group_id, rule, uint64_t); + EXTRACT_VARCHAR_FIELD_TO_CLASS_MYSQL(*result, group_name, rule); + if (OB_SUCC(ret) && OB_FAIL(rules.push_back(rule))) { + LOG_WARN("fail push back rules", K(rule), K(ret)); + } + } + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("fail get next row", K(ret), K(tname), K(tenant_id)); + } + } + } + } + return ret; +} + // 构建当前租户下全部 group_id -> group_name 的映射表 int ObResourceManagerProxy::get_all_resource_mapping_rules_for_plan( uint64_t tenant_id, @@ -1165,6 +1588,68 @@ int ObResourceManagerProxy::get_all_resource_mapping_rules_for_plan( return ret; } +// 构建当前租户下全部后台 Function 到 group_id 的映射表 +int ObResourceManagerProxy::get_all_resource_mapping_rules_by_function( + uint64_t tenant_id, + const common::ObString &plan, + common::ObIArray &rules) +// 构建当前租户下全部 group_id -> group_name 的映射表 +{ + int ret = OB_SUCCESS; + ObMySQLTransaction trans; + TransGuard trans_guard(trans, tenant_id, ret); + if (trans_guard.ready()) { + ObSQLClientRetryWeak sql_client_retry_weak( + &trans, tenant_id, OB_ALL_RES_MGR_MAPPING_RULE_TID); + SMART_VAR(ObMySQLProxy::MySQLResult, res) { + common::sqlclient::ObMySQLResult *result = NULL; + ObSqlString sql; + const char *t_a_res_name = OB_ALL_RES_MGR_MAPPING_RULE_TNAME; + const char *t_c_consumer_group_name = OB_ALL_RES_MGR_CONSUMER_GROUP_TNAME; + const char *t_d_directive = OB_ALL_RES_MGR_DIRECTIVE_TNAME; + uint64_t sql_tenant_id = ObSchemaUtils::get_extract_tenant_id(tenant_id, tenant_id); + // 当 resource plan 为空时,或者对应的 directive 不存在时,使用默认组 OTHER_GROUPS + if (OB_FAIL(sql.assign_fmt( + "SELECT /* GET_ALL_RES_MAPPING_RULE_BY_FUNCTION */ " + "a.`value` as `value`," + "(case when d.group_or_subplan is NULL then 'OTHER_GROUPS' else c.consumer_group end) as `group`, " + "(case when d.group_or_subplan is NULL then 0 else c.consumer_group_id end) as group_id " + "FROM %s a left join %s d ON a.consumer_group = d.group_or_subplan AND d.plan = '%.*s' AND d.tenant_id = %ld," + "%s c " + "WHERE a.TENANT_ID = %ld AND c.tenant_id = %ld AND a.tenant_id = c.tenant_id " + "AND a.attribute = 'FUNCTION' AND c.consumer_group = a.consumer_group", + t_a_res_name, t_d_directive, plan.length(), plan.ptr(), sql_tenant_id, + t_c_consumer_group_name, + sql_tenant_id, sql_tenant_id))) { + LOG_WARN("fail format sql", K(ret)); + } else if (OB_FAIL(sql_client_retry_weak.read(res, tenant_id, sql.ptr()))) { + LOG_WARN("fail to execute sql", K(sql), K(ret)); + } else if (NULL == (result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail query sql", K(t_a_res_name), K(tenant_id), K(ret)); + } else { + while (OB_SUCC(result->next())) { + ObResourceMappingRule rule; + rule.set_tenant_id(tenant_id); + rule.set_attr(ObString::make_string("")); + EXTRACT_VARCHAR_FIELD_TO_CLASS_MYSQL(*result, value, rule); + EXTRACT_VARCHAR_FIELD_TO_CLASS_MYSQL(*result, group, rule); + EXTRACT_INT_FIELD_TO_CLASS_MYSQL(*result, group_id, rule, uint64_t); + if (OB_SUCC(ret) && OB_FAIL(rules.push_back(rule))) { + LOG_WARN("fail push back rules", K(rule), K(ret)); + } + } + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("fail get next row", K(ret), K(t_a_res_name), K(tenant_id)); + } + } + } + } + return ret; +} + // 构建当前租户下全部 user_id -> group_id 的映射表 int ObResourceManagerProxy::get_all_resource_mapping_rules_by_user( uint64_t tenant_id, @@ -1541,4 +2026,63 @@ int ObResourceManagerProxy::get_all_resource_mapping_rules_by_column( } } return ret; +} + +int ObResourceManagerProxy::get_iops_config( + const uint64_t tenant_id, + const common::ObString &plan, + const common::ObString &group, + ObPlanDirective &directive) +{ + int ret = OB_SUCCESS; + ObMySQLTransaction trans; + TransGuard trans_guard(trans, tenant_id, ret); + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (trans_guard.ready()) { + ObSQLClientRetryWeak sql_client_retry_weak( + &trans, tenant_id, OB_ALL_RES_MGR_DIRECTIVE_TID); + SMART_VAR(ObMySQLProxy::MySQLResult, res) { + common::sqlclient::ObMySQLResult *result = NULL; + ObSqlString sql; + const char *tname = OB_ALL_RES_MGR_DIRECTIVE_TNAME; + if (OB_FAIL(sql.assign_fmt( + "SELECT /* GET IOPS CONFIG */ min_iops, max_iops FROM %s " + "WHERE TENANT_ID = %ld AND PLAN = '%.*s' AND GROUP_OR_SUBPLAN = '%.*s'", + tname, ObSchemaUtils::get_extract_tenant_id(tenant_id, tenant_id), + plan.length(), plan.ptr(), group.length(), group.ptr()))) { + LOG_WARN("fail format sql", K(ret)); + } else if (OB_FAIL(sql_client_retry_weak.read(res, tenant_id, sql.ptr()))) { + LOG_WARN("fail to execute sql", K(sql), K(ret)); + } else if (NULL == (result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail query sql", K(tname), K(tenant_id), K(ret)); + } else { + int64_t affected_rows = 0; + while (OB_SUCC(result->next())) { + EXTRACT_INT_FIELD_TO_CLASS_MYSQL(*result, min_iops, directive, int64_t); + EXTRACT_INT_FIELD_TO_CLASS_MYSQL(*result, max_iops, directive, int64_t); + ++affected_rows; + } + if (OB_ITER_END == ret) { + if (OB_UNLIKELY(!is_single_row(affected_rows))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected value. expect only 1 row affected", K(ret), K(affected_rows), K(group), K(plan)); + } else { + ret = OB_SUCCESS; + } + } else { + LOG_WARN("fail get next row", K(ret), K(tname), K(tenant_id)); + } + } + } + } + return ret; +} + +int ObResourceManagerProxy::reset_all_mapping_rules() +{ + int ret = G_RES_MGR.get_mapping_rule_mgr().reset_mapping_rules(); + return ret; } \ No newline at end of file diff --git a/src/share/resource_manager/ob_resource_manager_proxy.h b/src/share/resource_manager/ob_resource_manager_proxy.h index 716cf8e633..4dab4ef2dd 100644 --- a/src/share/resource_manager/ob_resource_manager_proxy.h +++ b/src/share/resource_manager/ob_resource_manager_proxy.h @@ -63,14 +63,20 @@ public: const common::ObString &group, const common::ObObj &comment, const common::ObObj &mgmt_p1, - const common::ObObj &utilization_limit); + const common::ObObj &utilization_limit, + const common::ObObj &min_iops, + const common::ObObj &max_iops, + const common::ObObj &weight_iops); int create_plan_directive( uint64_t tenant_id, const common::ObString &plan, const common::ObString &group, const common::ObObj &comment, const common::ObObj &mgmt_p1, - const common::ObObj &utilization_limit); + const common::ObObj &utilization_limit, + const common::ObObj &min_iops, + const common::ObObj &max_iops, + const common::ObObj &weight_iops); // 这里之所以直接传入 ObObj,而不是传入 ObString 或 int // 是为了便于判断传入的参数是否是缺省,如果缺省则 ObObj.is_null() 是 true int update_plan_directive( @@ -79,7 +85,10 @@ public: const common::ObString &group, const common::ObObj &comment, const common::ObObj &mgmt_p1, - const common::ObObj &utilization_limit); + const common::ObObj &utilization_limit, + const common::ObObj &min_iops, + const common::ObObj &max_iops, + const common::ObObj &weight_iops); int delete_plan_directive( uint64_t tenant_id, const common::ObString &plan, @@ -102,6 +111,12 @@ public: const common::ObString &attribute, const common::ObString &value, const common::ObString &consumer_group); + int replace_function_mapping_rule( + common::ObMySQLTransaction &trans, + uint64_t tenant_id, + const common::ObString &attribute, + const common::ObString &value, + const common::ObString &consumer_group); int replace_column_mapping_rule( common::ObMySQLTransaction &trans, uint64_t tenant_id, @@ -113,6 +128,14 @@ public: int get_all_resource_mapping_rules( uint64_t tenant_id, common::ObIArray &rules); + int get_all_group_info( + uint64_t tenant_id, + const common::ObString &plan, + common::ObIArray &rules); + int get_all_resource_mapping_rules_by_function( + uint64_t tenant_id, + const common::ObString &plan, + common::ObIArray &rules); int get_all_resource_mapping_rules_for_plan( uint64_t tenant_id, const common::ObString &plan, @@ -152,6 +175,13 @@ public: common::ObString &literal_value, common::ObString &user_name, const common::ObNameCaseMode case_mode); + int get_iops_config( + const uint64_t tenant_id, + const common::ObString &plan, + const common::ObString &group, + ObPlanDirective &directive); + int reset_all_mapping_rules(); + private: int allocate_consumer_group_id( common::ObMySQLTransaction &trans, @@ -177,6 +207,7 @@ private: uint64_t tenant_id, const common::ObString &user_name, bool &exist); + int check_if_function_exist(const common::ObString &function_name, bool &exist); int check_if_column_exist( uint64_t tenant_id, const common::ObString &db_name, @@ -201,6 +232,20 @@ private: common::ObString &formalized_value); // helper func, 便于集中获取百分比的值,数值范围为 [0, 100] int get_percentage(const char *name, const common::ObObj &obj, int64_t &v); + // max_iops >= min_iops, 否则抛出错误 + int check_iops_validity( + const uint64_t tenant_id, + const common::ObString &plan_name, + const common::ObString &group, + const int64_t iops_minimum, + const int64_t iops_maximum, + bool &valid); + + // get user_info from inner mapping_table + int get_user_mapping_info( + const uint64_t tenant_id, + const common::ObString &user, + ObResourceUserMappingRule &rule); public: class TransGuard { diff --git a/src/share/resource_manager/ob_resource_mapping_rule_manager.cpp b/src/share/resource_manager/ob_resource_mapping_rule_manager.cpp index ad9b188568..d81f7e9ee6 100644 --- a/src/share/resource_manager/ob_resource_mapping_rule_manager.cpp +++ b/src/share/resource_manager/ob_resource_mapping_rule_manager.cpp @@ -26,85 +26,174 @@ int ObResourceMappingRuleManager::init() int ret = OB_SUCCESS; int rule_bucket_size = 4096; int group_bucket_size = 512; - if (rule_map_.created() || group_map_.created()) { + if (user_rule_map_.created() || group_id_name_map_.created() || + function_rule_map_.created() || group_name_id_map_.created()) { ret = OB_INIT_TWICE; LOG_WARN("mapping rule manager should not init multiple times", K(ret)); - } else if (OB_FAIL(rule_map_.create(rule_bucket_size, "UsrRuleMap", "UsrRuleMapNode"))) { + } else if (OB_FAIL(user_rule_map_.create(rule_bucket_size, "UsrRuleMap", "UsrRuleMapNode"))) { // 整个集群的用户数,一般来说不会很大,4K 的空间足够 LOG_WARN("fail create rule map", K(ret)); - } else if (OB_FAIL(group_map_.create(group_bucket_size, "GrpIdNameMap", "GrpIdNameNode"))) { + } else if (OB_FAIL(group_id_name_map_.create(group_bucket_size, "GrpIdNameMap", "GrpIdNameNode"))) { // 整个集群的 group 数,一般来说不会很大,512 的空间足够 LOG_WARN("fail create group map", K(ret)); + } else if (OB_FAIL(function_rule_map_.create(group_bucket_size, "FuncRuleMap", "FuncRuleNode"))) { + LOG_WARN("fail create function rule map", K(ret)); + } else if (OB_FAIL(group_name_id_map_.create(group_bucket_size, "GrpNameIdMap", "GrpNameIdNode"))) { + LOG_WARN("fail create name id map", K(ret)); + } + LOG_INFO("resource mapping rule manager init ok"); + return ret; +} + +int ObResourceMappingRuleManager::refresh_group_mapping_rule(const uint64_t tenant_id, const ObString &plan) +{ + int ret = OB_SUCCESS; + ObResourceManagerProxy proxy; + ObResourceUserMappingRuleSet rules; + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(proxy.get_all_group_info(tenant_id, plan, rules))) { + LOG_WARN("fail get group infos", K(tenant_id), K(ret)); + } else { + for (int64_t i = 0; i < rules.count() && OB_SUCC(ret); ++i) { + ObResourceUserMappingRule &rule = rules.at(i); + if (OB_FAIL(group_id_name_map_.set_refactored( + combine_two_ids(rule.tenant_id_, rule.group_id_), + rule.group_name_, + 0 /* don't overwrite */))) { + if (OB_HASH_EXIST == ret) { + // group_id 和 group_name 的映射是固定的,不会变化,所以无需 overwrite + ret = OB_SUCCESS; + } else { + LOG_WARN("fail set group mapping to group_map", K(rule), K(ret)); + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(group_name_id_map_.set_refactored( + share::ObTenantGroupKey(rule.tenant_id_, rule.group_name_), + rule.group_id_, + 1 /* overwrite */))) { + LOG_WARN("fail set group name mapping to group id", K(rule), K(ret)); + } + } + } } - LOG_INFO("resource plan manager init ok"); return ret; } int ObResourceMappingRuleManager::refresh_resource_mapping_rule( - uint64_t tenant_id, + const uint64_t tenant_id, const ObString &plan) { int ret = OB_SUCCESS; ObResourceManagerProxy proxy; - // 目前每个租户最多只有 2 个 活跃 directive : interactive, batch - if (OB_ISNULL(GCTX.cgroup_ctrl_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("cgroup ctrl is null", K(ret)); - } else if (!GCTX.cgroup_ctrl_->is_valid()) { - ret = OB_EAGAIN; - // cgroup ctrl 没有初始化成功,可能是没有 cgroup fs、没有权限等原因 - // 此时不再继续后继资源隔离操作 + //下面这些读内部表的操作,不受cgroup是否开启影响 + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(refresh_resource_user_mapping_rule(proxy, tenant_id, plan))) { + LOG_WARN("fail refresh user mapping rule", K(tenant_id), K(plan), K(ret)); + } else if (OB_FAIL(refresh_resource_function_mapping_rule(proxy, tenant_id, plan))) { + LOG_WARN("fail refresh function mapping rule", K(tenant_id), K(plan), K(ret)); + } else { + LOG_INFO("refresh resource mapping rule success", K(tenant_id), K(plan)); } + return ret; +} - // user_id -> group_id - if (OB_SUCC(ret)) { - ObResourceUserMappingRuleSet rules; - if (OB_FAIL(proxy.get_all_resource_mapping_rules_by_user(tenant_id, plan, rules))) { - LOG_WARN("fail get resource mapping rules", K(tenant_id), K(ret)); - } else { - for (int64_t i = 0; i < rules.count() && OB_SUCC(ret); ++i) { - ObResourceUserMappingRule &rule = rules.at(i); - // 建立 user_id => group_id 的映射 - // note: 这里没有处理删除用户名之后的情况,如果频繁地建用户,给用户设定 mapping,然后删用户 - // 会导致 rule_map 里垃圾内容堆积。但是考虑到这种情况在线上环境里很少出现,暂不处理。 - // 内存占用很小(key:value = uint64:uint64),不会有太多浪费。 - if (OB_SUCC(ret)) { - if (OB_FAIL(rule_map_.set_refactored( - sql::ObTenantUserKey(rule.tenant_id_, rule.user_id_), - rule.group_id_, - 1 /* overwrite on dup key */))) { - LOG_WARN("fail set user mapping rule to rule_map", K(rule), K(ret)); - } - } - } - LOG_INFO("refresh_resource_mapping_rule for user_id->group_id lookup", - K(ret), K(tenant_id), K(plan), K(rules)); +int ObResourceMappingRuleManager::reset_mapping_rules() +{ + int ret = OB_SUCCESS; + for (common::hash::ObHashMap::const_iterator user_iter = user_rule_map_.begin(); + OB_SUCC(ret) && user_iter != user_rule_map_.end(); ++user_iter) { + if (OB_FAIL(user_rule_map_.set_refactored(user_iter->first, 0, 1/*overwrite*/))) { + LOG_WARN("failed to reset user map", K(ret), K(user_iter->first)); } } - - // group_id -> group_name if (OB_SUCC(ret)) { - ObResourceIdNameMappingRuleSet rules; - if (OB_FAIL(proxy.get_all_resource_mapping_rules_for_plan(tenant_id, plan, rules))) { - LOG_WARN("fail get resource mapping rules", K(tenant_id), K(ret)); - } else { - for (int64_t i = 0; i < rules.count() && OB_SUCC(ret); ++i) { - ObResourceIdNameMappingRule &rule = rules.at(i); - if (OB_FAIL(group_map_.set_refactored( - common::combine_two_ids(rule.tenant_id_, rule.group_id_), - rule.group_name_, - 0 /* don't overwrite */))) { - if (OB_HASH_EXIST == ret) { - // group_id 和 group_name 的映射是固定的,不会变化,所以无需 overwrite - ret = OB_SUCCESS; - } else { - LOG_WARN("fail set group mapping to group_map", K(rule), K(ret)); - } - } + for (common::hash::ObHashMap::const_iterator func_iter = function_rule_map_.begin(); + OB_SUCC(ret) && func_iter != function_rule_map_.end(); ++func_iter) { + if (OB_FAIL(function_rule_map_.set_refactored(func_iter->first, 0, 1/*overwrite*/))) { + LOG_WARN("failed to reset user map", K(ret), K(func_iter->first)); } } - LOG_INFO("refresh_resource_mapping_rule for group_id->group_name lookup", - K(ret), K(tenant_id), K(plan), K(rules)); + } + return ret; +} + +int ObResourceMappingRuleManager::refresh_resource_user_mapping_rule( + ObResourceManagerProxy &proxy, + const uint64_t tenant_id, + const ObString &plan) +{ + int ret = OB_SUCCESS; + ObResourceUserMappingRuleSet user_rules; + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(proxy.get_all_resource_mapping_rules_by_user(tenant_id, plan, user_rules))) { + LOG_WARN("fail get resource mapping user_rules", K(tenant_id), K(ret)); + } else { + for (int64_t i = 0; i < user_rules.count() && OB_SUCC(ret); ++i) { + ObResourceUserMappingRule &rule = user_rules.at(i); + // 建立 user_id => group_id 的映射 + // note: 这里没有处理删除用户名之后的情况,如果频繁地建用户,给用户设定 mapping,然后删用户 + // 会导致 rule_map 里垃圾内容堆积。但是考虑到这种情况在线上环境里很少出现,暂不处理。 + // 内存占用很小(key:value = uint64:uint64),不会有太多浪费。 + uint64_t group_id = 0; + bool map_changed = true; + if (OB_SUCCESS == user_rule_map_.get_refactored( + sql::ObTenantUserKey(rule.tenant_id_, rule.user_id_), group_id)) { + if (group_id == rule.group_id_) { + map_changed = false; // avoid set_refactor memory fragment + } + } + if (map_changed && OB_FAIL(user_rule_map_.set_refactored( + sql::ObTenantUserKey(rule.tenant_id_, rule.user_id_), + rule.group_id_, + 1 /* overwrite on dup key */))) { + LOG_WARN("fail set user mapping rule to rule_map", K(rule), K(ret)); + } + } + LOG_INFO("refresh resource user mapping rule", K(tenant_id), K(plan), K(user_rules)); + } + return ret; +} + +int ObResourceMappingRuleManager::refresh_resource_function_mapping_rule( + ObResourceManagerProxy &proxy, + const uint64_t tenant_id, + const ObString &plan) +{ + int ret = OB_SUCCESS; + ObResourceMappingRuleSet rules; + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(proxy.get_all_resource_mapping_rules_by_function(tenant_id, plan, rules))) { + LOG_WARN("fail get resource mapping rules", K(tenant_id), K(ret)); + } else { + for (int64_t i = 0; i < rules.count() && OB_SUCC(ret); ++i) { + ObResourceMappingRule &rule = rules.at(i); + // 建立 function_name => group_id 的映射 + uint64_t group_id = 0; + bool map_changed = true; + if (OB_SUCCESS == function_rule_map_.get_refactored( + share::ObTenantFunctionKey(rule.tenant_id_, rule.value_), group_id)) { + if (rule.group_id_ == group_id) { + // no new function mapping, don't update the function_rule_map_ to avoid memory fragment + map_changed = false; + } + } + if (map_changed && OB_FAIL(function_rule_map_.set_refactored( + share::ObTenantFunctionKey(rule.tenant_id_, rule.value_), /* function name */ + rule.group_id_, /* group id */ + 1 /* overwrite on dup key */))) { + LOG_WARN("fail set user mapping rule to rule_map", K(rule), K(ret)); + } + } + LOG_INFO("refresh_resource_function_mapping_rule", K(tenant_id), K(plan), K(rules)); } return ret; diff --git a/src/share/resource_manager/ob_resource_mapping_rule_manager.h b/src/share/resource_manager/ob_resource_mapping_rule_manager.h index 245cab82b8..749a001b42 100644 --- a/src/share/resource_manager/ob_resource_mapping_rule_manager.h +++ b/src/share/resource_manager/ob_resource_mapping_rule_manager.h @@ -29,7 +29,7 @@ class ObString; } namespace share { - +class ObResourceManagerProxy; class ObResourceMappingRuleManager { public: @@ -40,11 +40,57 @@ public: ObResourceMappingRuleManager() = default; virtual ~ObResourceMappingRuleManager() = default; int init(); - int refresh_resource_mapping_rule(uint64_t tenant_id, const common::ObString &plan); - inline int64_t get_group_id_by_user(uint64_t tenant_id, uint64_t user_id, uint64_t &group_id) + int refresh_group_mapping_rule(const uint64_t tenant_id, const common::ObString &plan); + int refresh_resource_mapping_rule(const uint64_t tenant_id, const common::ObString &plan); + int reset_mapping_rules(); + inline int get_group_id_by_user(const uint64_t tenant_id, uint64_t user_id, uint64_t &group_id) { int ret = common::OB_SUCCESS; - if (OB_FAIL(rule_map_.get_refactored(sql::ObTenantUserKey(tenant_id, user_id), group_id))) { + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(user_rule_map_.get_refactored(sql::ObTenantUserKey(tenant_id, user_id), group_id))) { + if (common::OB_HASH_NOT_EXIST == ret) { + group_id = 0; // 没有定义 mapping rule,默认为 0 (OTHER_GROUPS) + ret = common::OB_SUCCESS; + } else { + LOG_WARN("get group id by user fail", K(ret), K(user_id)); + group_id = 0; // 没有定义 mapping rule,默认为 0 (OTHER_GROUPS) + ret = common::OB_SUCCESS; + } + } + return ret; + } + + inline int get_group_id_by_function_type(const uint64_t tenant_id, + const int64_t function_type, + uint64_t &group_id) + { + int ret = common::OB_SUCCESS; + const ObString &func_name = share::get_io_function_name(static_cast(function_type)); + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(get_group_id_by_function(tenant_id, func_name, group_id))) { + LOG_WARN("get group id by function fail", K(ret), K(function_type)); + // 没有拿到group_id,可能是没有建立映射关系、map未初始化(没有生效plan)等原因 + // 此时仍然需要返回一个group_id,使dag绑定到other上 + group_id = 0; + ret = common::OB_SUCCESS; + } + return ret; + } + + inline int get_group_id_by_function(const uint64_t tenant_id, + const common::ObString &func, + uint64_t &group_id) + { + int ret = common::OB_SUCCESS; + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(function_rule_map_.get_refactored( + share::ObTenantFunctionKey(tenant_id, func), group_id))) { if (common::OB_HASH_NOT_EXIST == ret) { group_id = 0; // 没有定义 mapping rule,默认为 0 (OTHER_GROUPS) ret = common::OB_SUCCESS; @@ -52,19 +98,54 @@ public: } return ret; } - inline int64_t get_group_name_by_id(uint64_t tenant_id, - uint64_t group_id, - ObGroupName &group_name) + + inline int get_group_name_by_id(const uint64_t tenant_id, + const uint64_t group_id, + ObGroupName &group_name) { - int ret = group_map_.get_refactored(common::combine_two_ids(tenant_id, group_id), group_name); + int ret = group_id_name_map_.get_refactored(common::combine_two_ids(tenant_id, group_id), group_name); return ret; } + + inline int get_group_id_by_name(const uint64_t tenant_id, + ObGroupName group_name, + uint64_t &group_id) + { + int ret = group_name_id_map_.get_refactored(share::ObTenantGroupKey(tenant_id, group_name), group_id); + return ret; + } + inline int reset_group_id_by_user(const uint64_t tenant_id, + const uint64_t user_id) + { + int ret = user_rule_map_.set_refactored(sql::ObTenantUserKey(tenant_id, user_id), 0, 1/*overwrite*/); + return ret; + } + inline int reset_group_id_by_function(const uint64_t tenant_id, + ObResMgrVarcharValue &func) + { + int ret = function_rule_map_.set_refactored(share::ObTenantFunctionKey(tenant_id, func), 0, 1/*overwrite*/); + return ret; + } + +private: + int refresh_resource_function_mapping_rule( + ObResourceManagerProxy &proxy, + const uint64_t tenant_id, + const ObString &plan); + int refresh_resource_user_mapping_rule( + ObResourceManagerProxy &proxy, + const uint64_t tenant_id, + const ObString &plan); private: /* variables */ // 将用户 id 映射到 group id,用于用户登录时快速确定登录用户所属 cgroup - common::hash::ObHashMap rule_map_; - // 将 group id 映射到 GroupName,用于快速更新 cgroup fs 目录 - common::hash::ObHashMap group_map_; + common::hash::ObHashMap user_rule_map_; + // 将 function 映射到 group id,用于后台线程快速确定后台 session 所属 cgroup + common::hash::ObHashMap function_rule_map_; + // 将 group_id 映射到 group_name, 用于快速更新 cgroup fs 目录(包括user和function使用的group) + common::hash::ObHashMap group_id_name_map_; + // 将 group_name 映射到 group_id, 用于快速根据group_name找到id(主要是用于io控制) + common::hash::ObHashMap group_name_id_map_; DISALLOW_COPY_AND_ASSIGN(ObResourceMappingRuleManager); }; } diff --git a/src/share/resource_manager/ob_resource_plan_info.cpp b/src/share/resource_manager/ob_resource_plan_info.cpp index 41c338813f..9f9a9bf5d7 100644 --- a/src/share/resource_manager/ob_resource_plan_info.cpp +++ b/src/share/resource_manager/ob_resource_plan_info.cpp @@ -17,12 +17,51 @@ using namespace oceanbase::common; using namespace oceanbase::share; +ObString oceanbase::share::get_io_function_name(ObFunctionType function_type) +{ + ObString ret_name; + switch (function_type) { + case ObFunctionType::PRIO_COMPACTION_HIGH: + ret_name = ObString("COMPACTION_HIGH"); + break; + case ObFunctionType::PRIO_HA_HIGH: + ret_name = ObString("HA_HIGH"); + break; + case ObFunctionType::PRIO_COMPACTION_MID: + ret_name = ObString("COMPACTION_MID"); + break; + case ObFunctionType::PRIO_HA_MID: + ret_name = ObString("HA_MID"); + break; + case ObFunctionType::PRIO_COMPACTION_LOW: + ret_name = ObString("COMPACTION_LOW"); + break; + case ObFunctionType::PRIO_HA_LOW: + ret_name = ObString("HA_LOW"); + break; + case ObFunctionType::PRIO_DDL: + ret_name = ObString("DDL"); + break; + case ObFunctionType::PRIO_DDL_HIGH: + ret_name = ObString("DDL_HIGH"); + break; + default: + ret_name = ObString("OTHER_GROUPS"); + break; + } + return ret_name; +} + int ObPlanDirective::assign(const ObPlanDirective &other) { int ret = OB_SUCCESS; tenant_id_ = other.tenant_id_; mgmt_p1_ = other.mgmt_p1_; utilization_limit_ = other.utilization_limit_; + min_iops_ = other.min_iops_; + max_iops_ = other.max_iops_; + weight_iops_ = other.weight_iops_; + group_id_ = other.group_id_; level_ = other.level_; ret = group_name_.assign(other.group_name_); return ret; @@ -32,6 +71,7 @@ int ObResourceMappingRule::assign(const ObResourceMappingRule &other) { int ret = OB_SUCCESS; tenant_id_ = other.tenant_id_; + group_id_ = other.group_id_; OZ(set_attr(other.attr_), other); OZ(set_value(other.value_), other); OZ(set_group(other.group_), other); diff --git a/src/share/resource_manager/ob_resource_plan_info.h b/src/share/resource_manager/ob_resource_plan_info.h index afa80a79db..8c54c809bb 100644 --- a/src/share/resource_manager/ob_resource_plan_info.h +++ b/src/share/resource_manager/ob_resource_plan_info.h @@ -26,35 +26,122 @@ class ObString; } namespace share { +enum ObFunctionType +{ + PRIO_COMPACTION_HIGH = 0, + PRIO_HA_HIGH = 1, + PRIO_COMPACTION_MID = 2, + PRIO_HA_MID = 3, + PRIO_COMPACTION_LOW = 4, + PRIO_HA_LOW = 5, + PRIO_DDL = 6, + PRIO_DDL_HIGH = 7, + MAX_FUNCTION_NUM +}; + +ObString get_io_function_name(ObFunctionType function_type); // 为了便于作为 hash value,所以把 ObString 包一下 -class ObGroupName +class ObResMgrVarcharValue { public: - ObGroupName() {}; - int set_group_name(const common::ObString &name) + ObResMgrVarcharValue() { - common::ObDataBuffer allocator(group_name_buf_, common::OB_MAX_RESOURCE_PLAN_NAME_LENGTH); - return common::ob_write_string(allocator, name, group_name_);; + MEMSET(value_buf_, 0, sizeof(value_buf_)); + } + ObResMgrVarcharValue(const ObResMgrVarcharValue &other) + { + (void)assign(other); + } + ObResMgrVarcharValue(const ObString &other) + { + (void)set_value(other); + } + int set_value(const common::ObString &value) + { + common::ObDataBuffer allocator(value_buf_, common::OB_MAX_RESOURCE_PLAN_NAME_LENGTH); + return common::ob_write_string(allocator, value, value_); } // 自动隐式类型转换成 ObString operator const common::ObString& () const { - return group_name_; + return value_; } - const common::ObString &get_group_name() const + const common::ObString &get_value() const { - return group_name_; + return value_; } - int assign(const ObGroupName &other) + int assign(const ObResMgrVarcharValue &other) { - common::ObDataBuffer allocator(group_name_buf_, common::OB_MAX_RESOURCE_PLAN_NAME_LENGTH); - return common::ob_write_string(allocator, other.group_name_, group_name_); + return set_value(other.value_); } - TO_STRING_KV(K_(group_name)); + + void reset() + { + value_.reset(); + } + uint64_t hash() const + { + return value_.hash(); + } + int compare(const ObResMgrVarcharValue &r) const + { + return value_.compare(r.value_); + } + bool operator== (const ObResMgrVarcharValue &other) const { return 0 == compare(other); } + bool operator!=(const ObResMgrVarcharValue &other) const { return !operator==(other); } + bool operator<(const ObResMgrVarcharValue &other) const { return -1 == compare(other); } + TO_STRING_KV(K_(value)); private: - common::ObString group_name_; - char group_name_buf_[common::OB_MAX_RESOURCE_PLAN_NAME_LENGTH]; + common::ObString value_; + char value_buf_[common::OB_MAX_RESOURCE_PLAN_NAME_LENGTH]; +}; + +class ObGroupName : public ObResMgrVarcharValue +{ +public: + ObGroupName() {} +}; + +class ObTenantFunctionKey { +public: + ObTenantFunctionKey() : tenant_id_(0), func_name_() + {} + ObTenantFunctionKey(const uint64_t tenant_id, const ObResMgrVarcharValue &func_name) : + tenant_id_(tenant_id), func_name_(func_name) + {} + ObTenantFunctionKey(const uint64_t tenant_id, const common::ObString &func_name) : + tenant_id_(tenant_id), func_name_(func_name) + {} + int assign(const ObTenantFunctionKey &other) + { + tenant_id_ = other.tenant_id_; + return func_name_.assign(other.func_name_); + } + uint64_t hash() const + { + return common::murmurhash(&tenant_id_, sizeof(tenant_id_), func_name_.hash()); + } + int compare(const ObTenantFunctionKey& r) const + { + int cmp = 0; + if (tenant_id_ < r.tenant_id_) { + cmp = -1; + } else if (tenant_id_ == r.tenant_id_) { + cmp = func_name_.compare(r.func_name_); + } else { + cmp = 1; + } + return cmp; + } + bool operator== (const ObTenantFunctionKey &other) const { return 0 == compare(other); } + bool operator!=(const ObTenantFunctionKey &other) const { return !operator==(other); } + bool operator<(const ObTenantFunctionKey &other) const { return -1 == compare(other); } + TO_STRING_KV(K_(tenant_id), K_(func_name)); + +public: + uint64_t tenant_id_; + ObResMgrVarcharValue func_name_; }; class ObPlanDirective @@ -64,41 +151,79 @@ public: tenant_id_(common::OB_INVALID_ID), mgmt_p1_(100), utilization_limit_(100), + min_iops_(0), + max_iops_(100), + weight_iops_(0), + group_id_(), group_name_(), level_(1) {} ~ObPlanDirective() = default; public: - int set_tenant_id(uint64_t tenant_id) + bool is_valid() const + { + bool bret = min_iops_ >= 0 && min_iops_ <= 100 && max_iops_ >= 0 && + max_iops_ <= 100 && weight_iops_ >= 0 && weight_iops_ <= 100 && + min_iops_ <= max_iops_; + return bret; + } + int set_tenant_id(const uint64_t tenant_id) { tenant_id_ = tenant_id; return common::OB_SUCCESS; } - int set_mgmt_p1(int64_t mgmt_p1) + int set_mgmt_p1(const int64_t mgmt_p1) { mgmt_p1_ = mgmt_p1; return common::OB_SUCCESS; } - int set_utilization_limit(int64_t limit) + int set_utilization_limit(const int64_t limit) { utilization_limit_ = limit; return common::OB_SUCCESS; } - int set_group_name(const common::ObString &name) + int set_min_iops(const int64_t min_iops) { - return group_name_.set_group_name(name); + min_iops_ = min_iops; + return common::OB_SUCCESS; + } + int set_max_iops(const int64_t max_iops) + { + max_iops_ = max_iops; + return common::OB_SUCCESS; + } + int set_weight_iops(const int64_t weight_iops) + { + weight_iops_ = weight_iops; + return common::OB_SUCCESS; + } + void set_group_id(const uint64_t group_id) + { + group_id_ = group_id; + } + int set_group_or_subplan(const common::ObString &name) + { + return group_name_.set_value(name); } int assign(const ObPlanDirective &other); TO_STRING_KV(K_(tenant_id), - "group_name", group_name_.get_group_name(), + "group_name", group_name_.get_value(), K_(mgmt_p1), K_(utilization_limit), + K_(min_iops), + K_(max_iops), + K_(weight_iops), + K_(group_id), K_(level)); public: uint64_t tenant_id_; int64_t mgmt_p1_; int64_t utilization_limit_; - ObGroupName group_name_; + uint64_t min_iops_; + uint64_t max_iops_; + uint64_t weight_iops_; + uint64_t group_id_; + share::ObGroupName group_name_; int level_; private: DISALLOW_COPY_AND_ASSIGN(ObPlanDirective); @@ -109,6 +234,7 @@ class ObResourceMappingRule public: ObResourceMappingRule() : tenant_id_(common::OB_INVALID_ID), + group_id_(0), //default: other attr_(), value_(), group_() @@ -120,35 +246,36 @@ public: tenant_id_ = tenant_id; return common::OB_SUCCESS; } + int set_group_id(uint64_t group_id) + { + group_id_ = group_id; + return common::OB_SUCCESS; + } int set_group(const common::ObString &group) { - common::ObDataBuffer allocator(group_buf_, common::OB_MAX_RESOURCE_PLAN_NAME_LENGTH); - return common::ob_write_string(allocator, group, group_);; + return group_.set_value(group); } int set_attr(const common::ObString &attr) { - common::ObDataBuffer allocator(attr_buf_, common::OB_MAX_RESOURCE_PLAN_NAME_LENGTH); - return common::ob_write_string(allocator, attr, attr_);; + return attr_.set_value(attr); } int set_value(const common::ObString &value) { - common::ObDataBuffer allocator(value_buf_, common::OB_MAX_RESOURCE_PLAN_NAME_LENGTH); - return common::ob_write_string(allocator, value, value_);; + return value_.set_value(value); } int assign(const ObResourceMappingRule &other); TO_STRING_KV(K_(tenant_id), + K_(group_id), K_(attr), K_(value), K_(group)); public: uint64_t tenant_id_; - common::ObString attr_; - common::ObString value_; - common::ObString group_; + uint64_t group_id_; + share::ObResMgrVarcharValue attr_; + share::ObResMgrVarcharValue value_; + share::ObGroupName group_; private: - char attr_buf_[common::OB_MAX_RESOURCE_PLAN_NAME_LENGTH]; - char value_buf_[common::OB_MAX_RESOURCE_PLAN_NAME_LENGTH]; - char group_buf_[common::OB_MAX_RESOURCE_PLAN_NAME_LENGTH]; DISALLOW_COPY_AND_ASSIGN(ObResourceMappingRule); }; @@ -172,12 +299,12 @@ public: } int set_group_name(const common::ObString &name) { - return group_name_.set_group_name(name); + return group_name_.set_value(name); } int assign(const ObResourceIdNameMappingRule &other); TO_STRING_KV(K_(tenant_id), K_(group_id), - "group_name", group_name_.get_group_name()); + "group_name", group_name_.get_value()); public: uint64_t tenant_id_; uint64_t group_id_; @@ -213,23 +340,59 @@ public: } int set_group_name(const common::ObString &name) { - return group_name_.set_group_name(name); + return group_name_.set_value(name); } int assign(const ObResourceUserMappingRule &other); TO_STRING_KV(K_(tenant_id), K_(user_id), K_(group_id), - "group_name", group_name_.get_group_name()); + "group_name", group_name_.get_value()); public: uint64_t tenant_id_; uint64_t user_id_; uint64_t group_id_; - ObGroupName group_name_; + share::ObGroupName group_name_; private: - char group_name_buf_[common::OB_MAX_RESOURCE_PLAN_NAME_LENGTH]; DISALLOW_COPY_AND_ASSIGN(ObResourceUserMappingRule); }; +class ObTenantGroupKey { +public: + ObTenantGroupKey() : tenant_id_(0), group_name_() + {} + ObTenantGroupKey(const uint64_t tenant_id, const ObGroupName &group_name) : + tenant_id_(tenant_id), group_name_(group_name) + {} + int assign(const ObTenantGroupKey &other) + { + tenant_id_ = other.tenant_id_; + return group_name_.assign(other.group_name_); + } + uint64_t hash() const + { + return common::murmurhash(&tenant_id_, sizeof(tenant_id_), group_name_.hash()); + } + int compare(const ObTenantGroupKey& r) const + { + int cmp = 0; + if (tenant_id_ < r.tenant_id_) { + cmp = -1; + } else if (tenant_id_ == r.tenant_id_) { + cmp = group_name_.compare(r.group_name_); + } else { + cmp = 1; + } + return cmp; + } + bool operator== (const ObTenantGroupKey &other) const { return 0 == compare(other); } + bool operator!=(const ObTenantGroupKey &other) const { return !operator==(other); } + bool operator<(const ObTenantGroupKey &other) const { return -1 == compare(other); } + TO_STRING_KV(K_(tenant_id), K_(group_name)); + +public: + uint64_t tenant_id_; + share::ObGroupName group_name_; +}; class ObResourceColumnMappingRule { diff --git a/src/share/resource_manager/ob_resource_plan_manager.cpp b/src/share/resource_manager/ob_resource_plan_manager.cpp index eafd1cd21c..8e9a6ba7c7 100644 --- a/src/share/resource_manager/ob_resource_plan_manager.cpp +++ b/src/share/resource_manager/ob_resource_plan_manager.cpp @@ -13,6 +13,7 @@ #define USING_LOG_PREFIX SHARE #include "ob_resource_plan_manager.h" #include "lib/string/ob_string.h" +#include "share/io/ob_io_manager.h" #include "share/resource_manager/ob_resource_manager_proxy.h" #include "share/resource_manager/ob_cgroup_ctrl.h" #include "observer/ob_server_struct.h" @@ -24,43 +25,134 @@ using namespace oceanbase::share; int ObResourcePlanManager::init() { int ret = OB_SUCCESS; - LOG_INFO("resource plan manager init ok"); - return ret; -} - -int ObResourcePlanManager::refresh_resource_plan(uint64_t tenant_id, ObString &plan_name) -{ - int ret = OB_SUCCESS; - ObResourceManagerProxy proxy; - // 目前每个租户最多只有 2 个 活跃 directive : interactive, batch - ObPlanDirectiveSet directives; - if (OB_ISNULL(GCTX.cgroup_ctrl_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("cgroup ctrl is null", K(ret)); - } else if (!GCTX.cgroup_ctrl_->is_valid()) { - ret = OB_EAGAIN; - // cgroup ctrl 没有初始化成功,可能是没有 cgroup fs、没有权限等原因 - // 此时不再继续后继资源隔离操作 - } else if (OB_FAIL(proxy.get_all_plan_directives(tenant_id, plan_name, directives))) { - LOG_WARN("fail get plan directive", K(tenant_id), K(plan_name), K(ret)); + if (tenant_plan_map_.created()) { + ret = OB_INIT_TWICE; + LOG_WARN("mapping rule manager should not init multiple times", K(ret)); + } else if (OB_FAIL(tenant_plan_map_.create(7, "TENANT_PLAN_MAP"))) { + LOG_WARN("fail create tenant_plan_map", K(ret)); } else { - // directive => cgroup share/cfs_cpu_quota 转换。2 步: - // step1: 以 100 为总值做归一化 - // step2: 将值转化成 cgroup 值 (utilization=>cfs_cpu_quota 的值和 cpu 核数等有关) - // - 如果 utilization = 100,那么 cfs_cpu_quota = -1 - if (OB_FAIL(create_cgroup_dir_if_not_exist(directives))) { - LOG_WARN("fail create cgroup dir", K(directives), K(ret)); - } else if (OB_FAIL(normalize_directives(directives))) { - LOG_WARN("fail normalize directive", K(ret)); - } else if (OB_FAIL(flush_directive_to_cgroup_fs(directives))) { - LOG_WARN("fail flush directive to cgroup fs", K(ret)); - } - LOG_INFO("refresh_resource_plan", K(tenant_id), K(plan_name), K(directives)); + LOG_INFO("resource plan manager init ok"); } return ret; } -int ObResourcePlanManager::normalize_directives(ObPlanDirectiveSet &directives) +int ObResourcePlanManager::switch_resource_plan(const uint64_t tenant_id, ObString &plan_name) +{ + int ret = OB_SUCCESS; + ObResMgrVarcharValue origin_plan; + ObResMgrVarcharValue cur_plan(plan_name); + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(tenant_plan_map_.get_refactored(tenant_id, origin_plan))) { + if (OB_HASH_NOT_EXIST == ret) { + // initialize + if (OB_FAIL(tenant_plan_map_.set_refactored(tenant_id, cur_plan))) { + LOG_WARN("set plan failed", K(ret), K(tenant_id)); + } else { + LOG_INFO("add tenant id plan success", K(tenant_id), K(cur_plan)); + } + } else { + LOG_WARN("get plan failed", K(ret), K(tenant_id)); + } + } else if (origin_plan != cur_plan) { + // switch plan,reset 原来plan下对应directive的io资源 + ObResourceManagerProxy proxy; + ObPlanDirectiveSet directives; + if (OB_FAIL(proxy.get_all_plan_directives(tenant_id, origin_plan.get_value(), directives))) { + LOG_WARN("fail get plan directive", K(tenant_id), K(origin_plan), K(ret)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < directives.count(); ++i) { + const ObPlanDirective &cur_directive = directives.at(i); + if (OB_FAIL(GCTX.cgroup_ctrl_->reset_group_iops( + tenant_id, + 1, + cur_directive.group_name_))) { + LOG_ERROR("reset old plan group directive failed", K(cur_directive), K(ret)); + } + } + } + if (OB_SUCC(ret) && plan_name.empty()) { + // reset user and function hashmap + if (OB_FAIL(proxy.reset_all_mapping_rules())) { + LOG_WARN("fail reset all group rules",K(ret)); + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(tenant_plan_map_.set_refactored(tenant_id, cur_plan, 1))) { //overrite + LOG_WARN("set plan failed", K(ret), K(tenant_id)); + } else { + LOG_INFO("switch resource plan success", K(tenant_id), K(origin_plan), K(cur_plan)); + } + } + } + return ret; +} + +int ObResourcePlanManager::refresh_resource_plan(const uint64_t tenant_id, ObString &plan_name) +{ + int ret = OB_SUCCESS; + ObResourceManagerProxy proxy; + ObPlanDirectiveSet directives; + ObPlanDirective other_directive; // for OTHER_GROUPS + other_directive.set_group_id(0); + other_directive.set_tenant_id(tenant_id); + // 首先check plan是否发生了切换,如果plan切换那么原plan中资源设置先清零 + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(switch_resource_plan(tenant_id, plan_name))) { + LOG_WARN("check resource plan failed", K(ret), K(tenant_id), K(plan_name)); + } else if (OB_FAIL(proxy.get_all_plan_directives(tenant_id, plan_name, directives))) { + LOG_WARN("fail get plan directive", K(tenant_id), K(plan_name), K(ret)); + } else if (OB_FAIL(normalize_iops_directives(tenant_id, directives, other_directive))) { + LOG_WARN("fail normalize directive", K(ret)); + } else if (OB_FAIL(flush_directive_to_iops_control(tenant_id, directives, other_directive))) { // for IOPS + LOG_WARN("fail flush directive to io control", K(ret)); + } else { + if (OB_ISNULL(GCTX.cgroup_ctrl_) || !(GCTX.cgroup_ctrl_->is_valid())) { + // do nothing,cgroup ctrl 没有初始化成功,可能是没有 cgroup fs、没有权限等原因 + // cgroup不生效无法对CPU资源进行隔离,但上述io资源隔离可以继续 + + + // directive => cgroup share/cfs_cpu_quota 转换。2 步: + // step1: 以 100 为总值做归一化 + // step2: 将值转化成 cgroup 值 (utilization=>cfs_cpu_quota 的值和 cpu 核数等有关) + // - 如果 utilization = 100,那么 cfs_cpu_quota = -1 + } else if (OB_FAIL(create_cgroup_dir_if_not_exist(directives))) { + LOG_WARN("fail create cgroup dir", K(directives), K(ret)); + } else if (OB_FAIL(normalize_cpu_directives(directives))) { + LOG_WARN("fail normalize directive", K(ret)); + } else if (OB_FAIL(flush_directive_to_cgroup_fs(directives))) { // for CPU + LOG_WARN("fail flush directive to cgroup fs", K(ret)); + } + } + if (OB_SUCC(ret)) { + LOG_INFO("refresh resource plan success", K(tenant_id), K(plan_name), K(directives)); + } + return ret; +} + +int ObResourcePlanManager::get_cur_plan(const uint64_t tenant_id, ObResMgrVarcharValue &plan_name) +{ + int ret = OB_SUCCESS; + plan_name.reset(); + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(tenant_plan_map_.get_refactored(tenant_id, plan_name))) { + if (OB_HASH_NOT_EXIST == ret) { + //plan只有被使用才会放到map里 + ret = OB_SUCCESS; + LOG_INFO("delete plan success with no_releated_io_module", K(plan_name), K(tenant_id)); + } else { + LOG_WARN("get plan failed", K(ret), K(tenant_id), K(plan_name)); + } + } + return ret; +} + +int ObResourcePlanManager::normalize_cpu_directives(ObPlanDirectiveSet &directives) { int ret = OB_SUCCESS; int64_t total_mgmt = 0; @@ -122,6 +214,54 @@ int ObResourcePlanManager::normalize_directives(ObPlanDirectiveSet &directives) return ret; } +int ObResourcePlanManager::normalize_iops_directives(const uint64_t tenant_id, + ObPlanDirectiveSet &directives, + ObPlanDirective &other_group_directive) +{ + int ret = OB_SUCCESS; + // 在本版本中,用户无法指定OTHER_GROUPS及其他默认组的资源,OTHER资源是使用其他资源组算出来的 + // OTHER MIN_IOPS = 100-SUM; MAX_IOPS = 100; WEIGHT_IOPS = 100/SUM; + // 需要在产品手册中告知,建议用户不要把所有组的min_iops总和设置成100% + + uint64_t total_weight = 0; + uint64_t total_min = 0; + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < directives.count(); ++i) { + ObPlanDirective &cur_directive = directives.at(i); + if (cur_directive.group_id_ < GROUP_START_ID) { + ret = OB_ERR_UNEXPECTED; + // 理论上不应该出现 + LOG_WARN("unexpected error!!!", K(cur_directive)); + } else if (OB_UNLIKELY(!cur_directive.is_valid())) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid group io config", K(cur_directive)); + } else { + total_weight += cur_directive.weight_iops_; + total_min += cur_directive.min_iops_; + } + } + total_weight += OTHER_GROUPS_IOPS_WEIGHT; //OTHER GROUPS WEIGHT + + if(OB_SUCC(ret)) { + if (total_min > 100) { + ret = OB_INVALID_CONFIG; + LOG_WARN("invalid group io config", K(total_min)); + } else { + for (int64_t i = 0; i < directives.count(); ++i) { + ObPlanDirective &cur_directive = directives.at(i); + cur_directive.weight_iops_ = 100 * cur_directive.weight_iops_ / total_weight; + } + other_group_directive.weight_iops_ = 100 * 100 / total_weight; + other_group_directive.min_iops_ = 100 - total_min; + } + } + } + return ret; +} + int ObResourcePlanManager::create_cgroup_dir_if_not_exist(const ObPlanDirectiveSet &directives) { int ret = OB_SUCCESS; @@ -190,10 +330,10 @@ int ObResourcePlanManager::flush_directive_to_cgroup_fs(ObPlanDirectiveSet &dire LOG_ERROR("fail set cpu shares. tenant isolation function may not functional!!", K(d), K(ret)); } else if (OB_FAIL(GCTX.cgroup_ctrl_->set_cpu_cfs_quota( - d.tenant_id_, - d.level_, - d.group_name_, - static_cast(d.utilization_limit_)))) { + d.tenant_id_, + d.level_, + d.group_name_, + static_cast(d.utilization_limit_)))) { LOG_ERROR("fail set cpu quota. tenant isolation function may not functional!!", K(d), K(ret)); } @@ -201,3 +341,62 @@ int ObResourcePlanManager::flush_directive_to_cgroup_fs(ObPlanDirectiveSet &dire } return ret; } + +int ObResourcePlanManager::flush_directive_to_iops_control(const uint64_t tenant_id, + ObPlanDirectiveSet &directives, + ObPlanDirective &other_group_directive) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < directives.count(); ++i) { + const ObPlanDirective &cur_directive = directives.at(i); + share::OBGroupIOInfo cur_io_info; + if (OB_FAIL(cur_io_info.init(cur_directive.min_iops_, cur_directive.max_iops_, cur_directive.weight_iops_))) { + LOG_ERROR("fail init group io info", K(cur_directive), K(ret)); + } else if (OB_FAIL(GCTX.cgroup_ctrl_->set_group_iops( + cur_directive.tenant_id_, + cur_directive.level_, + cur_directive.group_id_, + cur_io_info))) { + LOG_ERROR("fail set iops. tenant isolation function may not functional!!", + K(cur_directive), K(ret)); + } + // ignore ret, continue + } + if (OB_SUCC(ret)) { + share::OBGroupIOInfo other_io_info; + if (OB_FAIL(other_io_info.init(other_group_directive.min_iops_, + other_group_directive.max_iops_, + other_group_directive.weight_iops_))) { + LOG_ERROR("fail init other group io info", K(other_group_directive), K(ret)); + } else if (OB_FAIL(GCTX.cgroup_ctrl_->set_group_iops( + other_group_directive.tenant_id_, + other_group_directive.level_, + other_group_directive.group_id_, + other_io_info))) { + LOG_ERROR("fail set iops. tenant isolation function may not functional!!", + K(other_group_directive), K(ret)); + } else if (OB_FAIL(refresh_tenant_group_io_config(tenant_id))) { + LOG_WARN("refresh tenant io config failed", K(ret), K(tenant_id)); + } + } + } + return ret; +} + +int ObResourcePlanManager::refresh_tenant_group_io_config(const uint64_t tenant_id) { + int ret = OB_SUCCESS; + ObRefHolder tenant_holder; + if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_TENANT_ID; + LOG_WARN("invalid config", K(ret), K(tenant_id)); + } else if (OB_FAIL(OB_IO_MANAGER.get_tenant_io_manager(tenant_id, tenant_holder))) { + LOG_WARN("get tenant io manager failed", K(ret), K(tenant_id)); + } else if (OB_FAIL(tenant_holder.get_ptr()->refresh_group_io_config())) { + LOG_WARN("refresh group io config failed", K(ret), K(tenant_id)); + } + return ret; +} \ No newline at end of file diff --git a/src/share/resource_manager/ob_resource_plan_manager.h b/src/share/resource_manager/ob_resource_plan_manager.h index 77ca9c565f..5578f7ca71 100644 --- a/src/share/resource_manager/ob_resource_plan_manager.h +++ b/src/share/resource_manager/ob_resource_plan_manager.h @@ -27,21 +27,32 @@ class ObString; } namespace share { +static constexpr int64_t OTHER_GROUPS_IOPS_WEIGHT = 100L; class ObResourcePlanManager { public: - typedef common::ObSEArray ObPlanDirectiveSet; + typedef common::ObSEArray ObPlanDirectiveSet; public: ObResourcePlanManager() = default; virtual ~ObResourcePlanManager() = default; int init(); - int refresh_resource_plan(uint64_t tenant_id, common::ObString &plan_name); + int refresh_resource_plan(const uint64_t tenant_id, common::ObString &plan_name); + int get_cur_plan(const uint64_t tenant_id, ObResMgrVarcharValue &plan_name); private: /* functions */ + int switch_resource_plan(const uint64_t tenant_id, common::ObString &plan_name); int flush_directive_to_cgroup_fs(ObPlanDirectiveSet &directives); + int flush_directive_to_iops_control(const uint64_t tenant_id, + ObPlanDirectiveSet &directives, + ObPlanDirective &other_group_directive); int create_cgroup_dir_if_not_exist(const ObPlanDirectiveSet &directives); - int normalize_directives(ObPlanDirectiveSet &directives); + int normalize_cpu_directives(ObPlanDirectiveSet &directives); + int normalize_iops_directives(const uint64_t tenant_id, + ObPlanDirectiveSet &directives, + ObPlanDirective &other_group_directive); + int refresh_tenant_group_io_config(const uint64_t tenant_id); + common::hash::ObHashMap tenant_plan_map_; /* variables */ DISALLOW_COPY_AND_ASSIGN(ObResourcePlanManager); }; diff --git a/src/share/scheduler/ob_dag_scheduler.cpp b/src/share/scheduler/ob_dag_scheduler.cpp index 6d7e937288..7cd70c50e5 100644 --- a/src/share/scheduler/ob_dag_scheduler.cpp +++ b/src/share/scheduler/ob_dag_scheduler.cpp @@ -20,6 +20,7 @@ #include "share/scheduler/ob_dag_scheduler.h" #include "share/scheduler/ob_sys_task_stat.h" #include "share/rc/ob_context.h" +#include "share/resource_manager/ob_resource_manager.h" #include "observer/omt/ob_tenant.h" #include "observer/omt/ob_tenant_config_mgr.h" #include "lib/stat/ob_diagnose_info.h" @@ -1288,6 +1289,8 @@ ObTenantDagWorker::ObTenantDagWorker() status_(DWS_FREE), check_period_(0), last_check_time_(0), + function_type_(0), + group_id_(-1), tg_id_(-1), is_inited_(false) { @@ -1347,6 +1350,8 @@ void ObTenantDagWorker::destroy() status_ = DWS_FREE; check_period_ = 0; last_check_time_ = 0; + function_type_ = 0; + group_id_ = -1; self_ = NULL; is_inited_ = false; TG_DESTROY(tg_id_); @@ -1365,6 +1370,25 @@ void ObTenantDagWorker::resume() notify(DWS_RUNNABLE); } +int ObTenantDagWorker::set_dag_resource() +{ + int ret = OB_SUCCESS; + uint64_t group_id = 0; + if (nullptr == GCTX.cgroup_ctrl_ || OB_UNLIKELY(!GCTX.cgroup_ctrl_->is_valid())) { + //invalid cgroup, cannot bind thread and control resource + } else if (OB_FAIL(G_RES_MGR.get_mapping_rule_mgr().get_group_id_by_function_type(MTL_ID(), function_type_, group_id))) { + LOG_WARN("fail to get group id by function", K(ret), K(MTL_ID()), K(function_type_), K(group_id)); + } else if (group_id == group_id_) { + // group not change, do nothing + } else if (OB_FAIL(GCTX.cgroup_ctrl_->add_thread_to_group(static_cast(GETTID()), MTL_ID(), group_id))) { + LOG_WARN("bind back thread to group failed", K(ret), K(GETTID()), K(MTL_ID()), K(group_id)); + } else { + ATOMIC_SET(&group_id_, group_id); + THIS_WORKER.set_group_id(group_id); + } + return ret; +} + bool ObTenantDagWorker::need_wake_up() const { return (ObTimeUtility::fast_current_time() - last_check_time_) > check_period_ * 10; @@ -1393,7 +1417,9 @@ void ObTenantDagWorker::run1() COMMON_LOG(WARN, "invalid compat mode", K(ret), K(*dag)); } else { THIS_WORKER.set_compatibility_mode(compat_mode); - if (OB_FAIL(task_->do_work())) { + if (OB_FAIL(set_dag_resource())) { + LOG_WARN("isolate dag CPU and IOPS failed", K(ret)); + } else if (OB_FAIL(task_->do_work())) { if (!dag->ignore_warning()) { COMMON_LOG(WARN, "failed to do work", K(ret), K(*task_), K(compat_mode)); } @@ -2846,7 +2872,7 @@ int ObTenantDagScheduler::schedule_one(const int64_t priority) && OB_FAIL(task->generate_next_task())) { task->get_dag()->reset_task_running_status(*task, ObITask::TASK_STATUS_FAILED); COMMON_LOG(WARN, "failed to generate_next_task", K(ret)); - } else if (OB_FAIL(dispatch_task(*task, worker))) { + } else if (OB_FAIL(dispatch_task(*task, worker, priority))) { task->get_dag()->reset_task_running_status(*task, ObITask::TASK_STATUS_WAITING); COMMON_LOG(WARN, "failed to dispatch task", K(ret)); } else { @@ -2859,7 +2885,7 @@ int ObTenantDagScheduler::schedule_one(const int64_t priority) running_workers_.add_last(worker, priority); if (task != NULL) { COMMON_LOG(INFO, "schedule one task", KPC(task), "priority", OB_DAG_PRIOS[priority].dag_prio_str_, - K_(total_running_task_cnt), K(running_task_cnts_[priority]), + "group id", worker->get_group_id(), K_(total_running_task_cnt), K(running_task_cnts_[priority]), K(low_limits_[priority]), K(up_limits_[priority]), KP(task->get_dag()->get_dag_net())); } worker->resume(); @@ -2933,7 +2959,7 @@ int ObTenantDagScheduler::loop_ready_dag_lists() return ret; } -int ObTenantDagScheduler::dispatch_task(ObITask &task, ObTenantDagWorker *&ret_worker) +int ObTenantDagScheduler::dispatch_task(ObITask &task, ObTenantDagWorker *&ret_worker, const int64_t priority) { int ret = OB_SUCCESS; ret_worker = NULL; @@ -2945,6 +2971,7 @@ int ObTenantDagScheduler::dispatch_task(ObITask &task, ObTenantDagWorker *&ret_w if (OB_SUCC(ret)) { ret_worker = free_workers_.remove_first(); ret_worker->set_task(&task); + ret_worker->set_function_type(priority); } return ret; } diff --git a/src/share/scheduler/ob_dag_scheduler.h b/src/share/scheduler/ob_dag_scheduler.h index 3d30bdc5e4..d7da400c99 100644 --- a/src/share/scheduler/ob_dag_scheduler.h +++ b/src/share/scheduler/ob_dag_scheduler.h @@ -705,9 +705,12 @@ public: void run1() override; void yield(); void set_task(ObITask *task) { task_ = task; } + void set_function_type(const int64_t function_type) { function_type_ = function_type; } + int set_dag_resource(); bool need_wake_up() const; ObITask *get_task() const { return task_; } static ObTenantDagWorker *self() { return self_; } + uint64_t get_group_id() { return group_id_; } private: void notify(DagWorkerStatus status); private: @@ -719,6 +722,8 @@ private: DagWorkerStatus status_; int64_t check_period_; int64_t last_check_time_; + int64_t function_type_; + uint64_t group_id_; int tg_id_; bool is_inited_; }; @@ -874,7 +879,7 @@ private: int create_worker(); int try_reclaim_threads(); int schedule_one(const int64_t priority); - int dispatch_task(ObITask &task, ObTenantDagWorker *&ret_worker); + int dispatch_task(ObITask &task, ObTenantDagWorker *&ret_worker, const int64_t priority); void destroy_all_workers(); int set_thread_score(const int64_t priority, const int32_t concurrency); bool try_switch(ObTenantDagWorker &worker); diff --git a/src/sql/engine/basic/ob_chunk_datum_store.cpp b/src/sql/engine/basic/ob_chunk_datum_store.cpp index edb134c137..a2715d512b 100644 --- a/src/sql/engine/basic/ob_chunk_datum_store.cpp +++ b/src/sql/engine/basic/ob_chunk_datum_store.cpp @@ -2538,7 +2538,6 @@ int ObChunkDatumStore::write_file(void *buf, int64_t size) } else { file_size_ = 0; io_.tenant_id_ = tenant_id_; - io_.io_desc_.set_category(common::ObIOCategory::USER_IO); io_.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_WRITE); LOG_INFO("open file success", K_(io_.fd), K_(io_.dir_id)); } @@ -2593,7 +2592,6 @@ int ObChunkDatumStore::read_file( OX (ret = OB_ITER_END); } else { this->set_io(size, static_cast(buf)); - io_.io_desc_.set_category(common::ObIOCategory::USER_IO); io_.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_READ); if (0 == read_size && OB_FAIL(FILE_MANAGER_INSTANCE_V2.get_tmp_file_size(io_.fd_, tmp_file_size))) { @@ -2626,7 +2624,6 @@ int ObChunkDatumStore::aio_read_file( LOG_WARN("invalid argument", K(size), K(offset), KP(buf)); } else if (size > 0) { this->set_io(size, static_cast(buf)); - io_.io_desc_.set_category(common::ObIOCategory::USER_IO); io_.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_READ); if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.aio_pread(io_, offset, handle))) { if (OB_ITER_END != ret) { diff --git a/src/sql/engine/basic/ob_chunk_row_store.cpp b/src/sql/engine/basic/ob_chunk_row_store.cpp index 74a382b769..99a75e2432 100644 --- a/src/sql/engine/basic/ob_chunk_row_store.cpp +++ b/src/sql/engine/basic/ob_chunk_row_store.cpp @@ -1615,9 +1615,6 @@ int ObChunkRowStore::write_file(void *buf, int64_t size) } else { file_size_ = 0; io_.tenant_id_ = tenant_id_; - common::ObIOCategory io_category = GCONF._large_query_io_percentage.get_value() > 0 ? - common::ObIOCategory::LARGE_QUERY_IO : common::ObIOCategory::USER_IO; - io_.io_desc_.set_category(io_category); io_.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_WRITE); LOG_TRACE("open file success", K_(io_.fd), K_(io_.dir_id)); } @@ -1661,9 +1658,6 @@ int ObChunkRowStore::read_file(void *buf, const int64_t size, const int64_t offs OX (ret = OB_ITER_END); } else { this->set_io(size, static_cast(buf)); - common::ObIOCategory io_category = GCONF._large_query_io_percentage.get_value() > 0 ? - common::ObIOCategory::LARGE_QUERY_IO : common::ObIOCategory::USER_IO; - io_.io_desc_.set_category(io_category); io_.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_READ); blocksstable::ObTmpFileIOHandle handle; if (0 == read_size diff --git a/src/sql/engine/basic/ob_ra_datum_store.cpp b/src/sql/engine/basic/ob_ra_datum_store.cpp index c26e380717..8d546c3a20 100644 --- a/src/sql/engine/basic/ob_ra_datum_store.cpp +++ b/src/sql/engine/basic/ob_ra_datum_store.cpp @@ -1133,7 +1133,6 @@ int ObRADatumStore::write_file(BlockIndex &bi, void *buf, int64_t size) io.buf_ = static_cast(buf); io.size_ = size; io.tenant_id_ = tenant_id_; - io.io_desc_.set_category(ObIOCategory::USER_IO); io.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_WRITE); const uint64_t start = rdtsc(); if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.write(io, timeout_ms))) { @@ -1174,7 +1173,6 @@ int ObRADatumStore::read_file(void *buf, const int64_t size, const int64_t offse io.buf_ = static_cast(buf); io.size_ = size; io.tenant_id_ = tenant_id_; - io.io_desc_.set_category(ObIOCategory::USER_IO); io.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_READ); const uint64_t start = rdtsc(); blocksstable::ObTmpFileIOHandle handle; diff --git a/src/sql/engine/basic/ob_ra_row_store.cpp b/src/sql/engine/basic/ob_ra_row_store.cpp index f3ac302560..4eef79cd38 100644 --- a/src/sql/engine/basic/ob_ra_row_store.cpp +++ b/src/sql/engine/basic/ob_ra_row_store.cpp @@ -958,9 +958,6 @@ int ObRARowStore::write_file(BlockIndex &bi, void *buf, int64_t size) io.buf_ = static_cast(buf); io.size_ = size; io.tenant_id_ = tenant_id_; - common::ObIOCategory io_category = GCONF._large_query_io_percentage.get_value() > 0 ? - common::ObIOCategory::LARGE_QUERY_IO : common::ObIOCategory::USER_IO; - io.io_desc_.set_category(io_category); io.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_WRITE); if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.write(io, timeout_ms))) { LOG_WARN("write to file failed", K(ret), K(io), K(timeout_ms)); @@ -995,9 +992,6 @@ int ObRARowStore::read_file(void *buf, const int64_t size, const int64_t offset) io.buf_ = static_cast(buf); io.size_ = size; io.tenant_id_ = tenant_id_; - common::ObIOCategory io_category = GCONF._large_query_io_percentage.get_value() > 0 ? - common::ObIOCategory::LARGE_QUERY_IO : common::ObIOCategory::USER_IO; - io.io_desc_.set_category(io_category); io.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_READ); blocksstable::ObTmpFileIOHandle handle; if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.pread(io, offset, timeout_ms, handle))) { diff --git a/src/storage/access/ob_sstable_row_whole_scanner.cpp b/src/storage/access/ob_sstable_row_whole_scanner.cpp index b0c71ba12b..85a6f9bb69 100644 --- a/src/storage/access/ob_sstable_row_whole_scanner.cpp +++ b/src/storage/access/ob_sstable_row_whole_scanner.cpp @@ -255,7 +255,6 @@ int ObSSTableRowWholeScanner::open( read_info.macro_block_id_ = macro_desc.macro_block_id_; read_info.offset_ = sstable_->get_macro_offset(); read_info.size_ = sstable_->get_macro_read_size(); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); if (OB_FAIL(ObBlockManager::async_read_block(read_info, scan_handle.macro_io_handle_))) { LOG_WARN("Fail to read macro block", K(ret), K(read_info)); @@ -365,7 +364,6 @@ int ObSSTableRowWholeScanner::prefetch() read_info.macro_block_id_ = scan_handle.macro_block_desc_.macro_block_id_; read_info.offset_ = sstable_->get_macro_offset(); read_info.size_ = sstable_->get_macro_read_size(); - read_info.io_desc_.set_category(common::ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(common::ObWaitEventIds::DB_FILE_COMPACT_READ); if (OB_FAIL(ObBlockManager::async_read_block(read_info, scan_handle.macro_io_handle_))) { LOG_WARN("Fail to read macro block, ", K(ret), K(read_info)); diff --git a/src/storage/backup/ob_backup_reader.cpp b/src/storage/backup/ob_backup_reader.cpp index 9965ba76de..158fb6d6c2 100644 --- a/src/storage/backup/ob_backup_reader.cpp +++ b/src/storage/backup/ob_backup_reader.cpp @@ -271,7 +271,6 @@ int ObMacroBlockBackupReader::get_macro_read_info_( read_info.macro_block_id_ = block_info_.macro_id_; read_info.offset_ = block_info_.nested_offset_; read_info.size_ = block_info_.nested_size_; - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_MIGRATE_READ); } return ret; diff --git a/src/storage/backup/ob_backup_tmp_file.cpp b/src/storage/backup/ob_backup_tmp_file.cpp index 9be9382089..50129b59d9 100644 --- a/src/storage/backup/ob_backup_tmp_file.cpp +++ b/src/storage/backup/ob_backup_tmp_file.cpp @@ -95,7 +95,6 @@ int ObBackupTmpFile::get_io_info_(const char *buf, const int64_t size, ObTmpFile io_info.reset(); io_info.fd_ = file_fd_; io_info.tenant_id_ = tenant_id_; - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); io_info.buf_ = const_cast(buf); io_info.size_ = size; diff --git a/src/storage/backup/ob_backup_tmp_file.h b/src/storage/backup/ob_backup_tmp_file.h index d10144b4d2..83e5706c3f 100644 --- a/src/storage/backup/ob_backup_tmp_file.h +++ b/src/storage/backup/ob_backup_tmp_file.h @@ -143,7 +143,6 @@ int ObBackupIndexBufferNode::get_backup_index(T &backup_index) blocksstable::ObTmpFileIOHandle handle; io_info.fd_ = tmp_file_.get_fd(); io_info.tenant_id_ = tmp_file_.get_tenant_id(); - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); io_info.size_ = std::min(need_read_size, estimate_size_ - read_offset_); common::ObArenaAllocator allocator; diff --git a/src/storage/blocksstable/ob_block_manager.cpp b/src/storage/blocksstable/ob_block_manager.cpp index 1c31b1dda8..c1074a641e 100644 --- a/src/storage/blocksstable/ob_block_manager.cpp +++ b/src/storage/blocksstable/ob_block_manager.cpp @@ -1335,7 +1335,6 @@ int ObBlockManager::InspectBadBlockTask::check_block(const MacroBlockId ¯o_i read_info.macro_block_id_ = macro_id; read_info.offset_ = 0; read_info.size_ = blk_mgr_.get_macro_block_size(); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); if (OB_FAIL(ObBlockManager::async_read_block(read_info, macro_handle))) { diff --git a/src/storage/blocksstable/ob_bloom_filter_data_reader.cpp b/src/storage/blocksstable/ob_bloom_filter_data_reader.cpp index ff191b501e..9ce6ba7e82 100644 --- a/src/storage/blocksstable/ob_bloom_filter_data_reader.cpp +++ b/src/storage/blocksstable/ob_bloom_filter_data_reader.cpp @@ -75,7 +75,6 @@ int ObBloomFilterMacroBlockReader::read_macro_block(const MacroBlockId ¯o_id } else { ObMacroBlockReadInfo macro_read_info; macro_read_info.macro_block_id_ = macro_id; - macro_read_info.io_desc_.set_category(is_sys_read_ ? ObIOCategory::SYS_IO : ObIOCategory::USER_IO); macro_read_info.io_desc_.set_wait_event(is_sys_read_ ? ObWaitEventIds::DB_FILE_COMPACT_READ : ObWaitEventIds::DB_FILE_DATA_READ); macro_read_info.offset_ = 0; macro_read_info.size_ = OB_SERVER_BLOCK_MGR.get_macro_block_size(); diff --git a/src/storage/blocksstable/ob_bloom_filter_data_writer.cpp b/src/storage/blocksstable/ob_bloom_filter_data_writer.cpp index e9d15f98b4..51b6965ee9 100644 --- a/src/storage/blocksstable/ob_bloom_filter_data_writer.cpp +++ b/src/storage/blocksstable/ob_bloom_filter_data_writer.cpp @@ -347,7 +347,6 @@ int ObBloomFilterMacroBlockWriter::flush_macro_block() macro_write_info.buffer_ = data_buffer_.data(); macro_write_info.size_ = data_buffer_.capacity(); macro_write_info.size_ = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - macro_write_info.io_desc_.set_category(ObIOCategory::SYS_IO); macro_write_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_WRITE); if (OB_FAIL(ObBlockManager::write_block(macro_write_info, macro_handle))) { STORAGE_LOG(WARN, "Failed to write bloomfilter macro block", K(ret)); diff --git a/src/storage/blocksstable/ob_index_block_builder.cpp b/src/storage/blocksstable/ob_index_block_builder.cpp index 8cacd7c316..b4c3b50bad 100644 --- a/src/storage/blocksstable/ob_index_block_builder.cpp +++ b/src/storage/blocksstable/ob_index_block_builder.cpp @@ -973,7 +973,6 @@ int ObSSTableIndexBuilder::rewrite_small_sstable(ObSSTableMergeRes &res) read_info.offset_ = 0; read_info.size_ = upper_align(roots_[0]->last_macro_size_, DIO_READ_ALIGN_SIZE); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); const int64_t io_timeout_ms = std::max(GCONF._data_storage_io_timeout / 1000, DEFAULT_IO_WAIT_TIME_MS); if (OB_FAIL(ObBlockManager::async_read_block(read_info, read_handle))) { @@ -1027,7 +1026,6 @@ int ObSSTableIndexBuilder::do_check_and_rewrite_sstable(ObBlockInfo &block_info) read_info.offset_ = 0; read_info.size_ = OB_SERVER_BLOCK_MGR.get_macro_block_size(); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); const int64_t io_timeout_ms = std::max(GCONF._data_storage_io_timeout / 1000, DEFAULT_IO_WAIT_TIME_MS); ObSSTableMacroBlockHeader macro_header; diff --git a/src/storage/blocksstable/ob_index_block_tree_cursor.cpp b/src/storage/blocksstable/ob_index_block_tree_cursor.cpp index f5512848da..66b1518932 100644 --- a/src/storage/blocksstable/ob_index_block_tree_cursor.cpp +++ b/src/storage/blocksstable/ob_index_block_tree_cursor.cpp @@ -1010,7 +1010,6 @@ int ObIndexBlockTreeCursor::get_next_level_block( read_info.offset_ = absolute_offset; read_info.size_ = idx_row_header.get_block_size(); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); - read_info.io_desc_.set_category(ObIOCategory::USER_IO); idx_row_header.fill_deserialize_meta(block_des_meta); if (OB_FAIL(ObBlockManager::read_block(read_info, macro_handle))) { LOG_WARN("Fail to read micro block from sync io", K(ret)); diff --git a/src/storage/blocksstable/ob_macro_block.cpp b/src/storage/blocksstable/ob_macro_block.cpp index fb48c5cdb3..4dc9d22aa3 100644 --- a/src/storage/blocksstable/ob_macro_block.cpp +++ b/src/storage/blocksstable/ob_macro_block.cpp @@ -687,7 +687,6 @@ int ObMacroBlock::flush(ObMacroBlockHandle ¯o_handle, ObMacroBlockWriteInfo write_info; write_info.buffer_ = data_.data(); write_info.size_ = data_.capacity(); - write_info.io_desc_.set_category(ObIOCategory::SYS_IO); write_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_WRITE); if (OB_FAIL(macro_handle.async_write(write_info))) { STORAGE_LOG(WARN, "Fail to async write block", K(ret), K(macro_handle)); diff --git a/src/storage/blocksstable/ob_macro_block_handle.cpp b/src/storage/blocksstable/ob_macro_block_handle.cpp index b4fa641f93..2703a7bcce 100644 --- a/src/storage/blocksstable/ob_macro_block_handle.cpp +++ b/src/storage/blocksstable/ob_macro_block_handle.cpp @@ -128,6 +128,7 @@ int ObMacroBlockHandle::async_read(const ObMacroBlockReadInfo &read_info) io_info.callback_ = read_info.io_callback_; io_info.fd_.first_id_ = read_info.macro_block_id_.first_id(); io_info.fd_.second_id_ = read_info.macro_block_id_.second_id(); + io_info.flag_.set_group_id(THIS_WORKER.get_group_id()); io_info.flag_.set_read(); if (OB_FAIL(ObIOManager::get_instance().aio_read(io_info, io_handle_))) { @@ -154,6 +155,7 @@ int ObMacroBlockHandle::async_write(const ObMacroBlockWriteInfo &write_info) io_info.flag_ = write_info.io_desc_; io_info.fd_.first_id_ = macro_id_.first_id(); io_info.fd_.second_id_ = macro_id_.second_id(); + io_info.flag_.set_group_id(THIS_WORKER.get_group_id()); io_info.flag_.set_write(); if (OB_FAIL(ObIOManager::get_instance().aio_write(io_info, io_handle_))) { diff --git a/src/storage/blocksstable/ob_macro_block_writer.cpp b/src/storage/blocksstable/ob_macro_block_writer.cpp index da0a1816d1..b7ac7dcbff 100755 --- a/src/storage/blocksstable/ob_macro_block_writer.cpp +++ b/src/storage/blocksstable/ob_macro_block_writer.cpp @@ -1282,7 +1282,6 @@ int ObMacroBlockWriter::check_write_complete(const MacroBlockId ¯o_block_id) ObMacroBlockReadInfo read_info; read_info.macro_block_id_ = macro_block_id; read_info.size_ = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); const int64_t io_timeout_ms = std::max(GCONF._data_storage_io_timeout / 1000, DEFAULT_IO_WAIT_TIME_MS); ObMacroBlockHandle read_handle; diff --git a/src/storage/blocksstable/ob_micro_block_cache.cpp b/src/storage/blocksstable/ob_micro_block_cache.cpp index 4cd4f9670a..2828d05e6d 100644 --- a/src/storage/blocksstable/ob_micro_block_cache.cpp +++ b/src/storage/blocksstable/ob_micro_block_cache.cpp @@ -929,9 +929,6 @@ int ObIMicroBlockCache::prefetch( // fill read info ObMacroBlockReadInfo read_info; read_info.macro_block_id_ = macro_id; - read_info.io_desc_.set_category(flag.is_prewarm() ? ObIOCategory::PREWARM_IO - : flag.is_large_query() && GCONF._large_query_io_percentage.get_value() > 0 - ? ObIOCategory::LARGE_QUERY_IO : ObIOCategory::USER_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); read_info.io_callback_ = &callback; common::align_offset_size( @@ -980,9 +977,6 @@ int ObIMicroBlockCache::prefetch( // fill read info ObMacroBlockReadInfo read_info; read_info.macro_block_id_ = macro_id; - read_info.io_desc_.set_category(flag.is_prewarm() ? ObIOCategory::PREWARM_IO - : flag.is_large_query() && GCONF._large_query_io_percentage.get_value() > 0 - ? ObIOCategory::LARGE_QUERY_IO : ObIOCategory::USER_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); read_info.io_callback_ = &callback; common::align_offset_size(offset, size, read_info.offset_, read_info.size_); @@ -1096,7 +1090,6 @@ int ObDataMicroBlockCache::load_block( LOG_WARN("Invalid argument", K(ret), K(micro_block_id), KP(macro_reader)); } else { macro_read_info.macro_block_id_ = micro_block_id.macro_id_; - macro_read_info.io_desc_.set_category(ObIOCategory::USER_IO); macro_read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); macro_read_info.offset_ = micro_block_id.offset_; macro_read_info.size_ = micro_block_id.size_; @@ -1214,7 +1207,6 @@ int ObIndexMicroBlockCache::load_block( LOG_WARN("Invalid argument", K(ret), K(micro_block_id), KP(read_info), KP(allocator)); } else { macro_read_info.macro_block_id_ = micro_block_id.macro_id_; - macro_read_info.io_desc_.set_category(ObIOCategory::USER_IO); macro_read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); macro_read_info.offset_ = micro_block_id.offset_; macro_read_info.size_ = micro_block_id.size_; diff --git a/src/storage/blocksstable/ob_shared_macro_block_manager.cpp b/src/storage/blocksstable/ob_shared_macro_block_manager.cpp index a89b233236..5d713c347d 100644 --- a/src/storage/blocksstable/ob_shared_macro_block_manager.cpp +++ b/src/storage/blocksstable/ob_shared_macro_block_manager.cpp @@ -182,7 +182,6 @@ int ObSharedMacroBlockMgr::write_block( } else { ObMacroBlockWriteInfo write_info; write_info.buffer_ = buf; - write_info.io_desc_.set_category(ObIOCategory::SYS_IO); write_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_WRITE); write_info.size_ = size; lib::ObMutexGuard guard(mutex_); @@ -252,7 +251,6 @@ int ObSharedMacroBlockMgr::check_write_complete(const MacroBlockId ¯o_id, co read_info.macro_block_id_ = macro_id; read_info.size_ = macro_size; read_info.offset_ = offset_; - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); const int64_t io_timeout_ms = std::max(GCONF._data_storage_io_timeout / 1000, DEFAULT_IO_WAIT_TIME_MS); ObMacroBlockHandle read_handle; @@ -289,7 +287,6 @@ int ObSharedMacroBlockMgr::try_switch_macro_block() write_info.buffer_ = common_header_buf_; write_info.size_ = header_size_; write_info.offset_ = 0; - write_info.io_desc_.set_category(ObIOCategory::SYS_IO); write_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_WRITE); if (OB_FAIL(do_write_block(write_info, block_info))) { LOG_WARN("fail to write common header to the shared macro block", K(ret), K(block_info)); @@ -745,7 +742,6 @@ int ObSharedMacroBlockMgr::read_sstable_block( read_info.macro_block_id_ = macro_info.get_data_block_ids().at(0); read_info.offset_ = macro_info.get_nested_offset(); read_info.size_ = upper_align(macro_info.get_nested_size(), DIO_READ_ALIGN_SIZE); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); if (OB_FAIL(ObBlockManager::read_block(read_info, block_handle))) { diff --git a/src/storage/blocksstable/ob_sstable_meta_info.cpp b/src/storage/blocksstable/ob_sstable_meta_info.cpp index ac5689a900..bfe580a602 100644 --- a/src/storage/blocksstable/ob_sstable_meta_info.cpp +++ b/src/storage/blocksstable/ob_sstable_meta_info.cpp @@ -265,7 +265,6 @@ int ObRootBlockInfo::read_block_data( blocksstable::ObMacroBlockReadInfo read_info; handle.reset(); read_info.io_desc_.set_mode(ObIOMode::READ); - read_info.io_desc_.set_category( ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); if (OB_FAIL(addr.get_block_addr(read_info.macro_block_id_, read_info.offset_, read_info.size_))) { LOG_WARN("fail to get block address", K(ret), K(addr)); diff --git a/src/storage/compaction/ob_index_block_micro_iterator.cpp b/src/storage/compaction/ob_index_block_micro_iterator.cpp index ace76b99ad..90637ec4fb 100644 --- a/src/storage/compaction/ob_index_block_micro_iterator.cpp +++ b/src/storage/compaction/ob_index_block_micro_iterator.cpp @@ -173,7 +173,6 @@ int ObIndexBlockMicroIterator::init( read_info.macro_block_id_ = macro_id; read_info.offset_ = sstable->get_macro_offset(); read_info.size_ = sstable->get_macro_read_size(); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); if (OB_FAIL(ObBlockManager::async_read_block(read_info, macro_handle_))) { LOG_WARN("async read block failed, ", K(ret), K(read_info), K(macro_id)); diff --git a/src/storage/ddl/ob_ddl_redo_log_replayer.cpp b/src/storage/ddl/ob_ddl_redo_log_replayer.cpp index 8b53d6ca1d..b8bab4f63a 100644 --- a/src/storage/ddl/ob_ddl_redo_log_replayer.cpp +++ b/src/storage/ddl/ob_ddl_redo_log_replayer.cpp @@ -126,7 +126,6 @@ int ObDDLRedoLogReplayer::replay_redo(const ObDDLRedoLog &log, const SCN &scn) ObMacroBlockHandle macro_handle; write_info.buffer_ = redo_info.data_buffer_.ptr(); write_info.size_= redo_info.data_buffer_.length(); - write_info.io_desc_.set_category(ObIOCategory::SYS_IO); write_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_WRITE); const int64_t io_timeout_ms = max(DDL_FLUSH_MACRO_BLOCK_TIMEOUT / 1000L, GCONF._data_storage_io_timeout / 1000L); ObDDLMacroBlock macro_block; diff --git a/src/storage/high_availability/ob_storage_ha_macro_block_writer.cpp b/src/storage/high_availability/ob_storage_ha_macro_block_writer.cpp index 3401be762d..d8c96efb4f 100644 --- a/src/storage/high_availability/ob_storage_ha_macro_block_writer.cpp +++ b/src/storage/high_availability/ob_storage_ha_macro_block_writer.cpp @@ -114,7 +114,6 @@ int ObStorageHAMacroBlockWriter::process(blocksstable::ObMacroBlocksWriteCtx &co int64_t log_seq_num = 0; int64_t data_size = 0; obrpc::ObCopyMacroBlockHeader header; - write_info.io_desc_.set_category(ObIOCategory::SYS_IO); write_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_MIGRATE_WRITE); if (OB_UNLIKELY(!is_inited_)) { diff --git a/src/storage/high_availability/ob_storage_ha_reader.cpp b/src/storage/high_availability/ob_storage_ha_reader.cpp index 68725569ab..35ae664c03 100644 --- a/src/storage/high_availability/ob_storage_ha_reader.cpp +++ b/src/storage/high_availability/ob_storage_ha_reader.cpp @@ -731,7 +731,6 @@ int ObCopyMacroBlockObProducer::prefetch_() read_info.macro_block_id_ = macro_meta.get_macro_id(); read_info.offset_ = sstable_->get_macro_offset(); read_info.size_ = sstable_->get_macro_read_size(); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_MIGRATE_READ); if (OB_FAIL(ObBlockManager::async_read_block(read_info, copy_macro_block_handle_[handle_idx_].read_handle_))) { STORAGE_LOG(WARN, "Fail to async read block, ", K(ret), K(read_info)); diff --git a/src/storage/ob_bloom_filter_task.cpp b/src/storage/ob_bloom_filter_task.cpp index 575ccfdbb0..3b33998a99 100644 --- a/src/storage/ob_bloom_filter_task.cpp +++ b/src/storage/ob_bloom_filter_task.cpp @@ -152,7 +152,6 @@ int ObBloomFilterBuildTask::build_bloom_filter() read_info.macro_block_id_ = macro_id_; read_info.offset_ = 0; read_info.size_ = OB_DEFAULT_MACRO_BLOCK_SIZE; - read_info.io_desc_.set_category(ObIOCategory::USER_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); if (OB_FAIL(ObBlockManager::read_block(read_info, macro_handle))) { LOG_WARN("Fail to read macro block", K(ret), K(read_info)); diff --git a/src/storage/ob_parallel_external_sort.h b/src/storage/ob_parallel_external_sort.h index 156c677d38..7e9f221a3c 100644 --- a/src/storage/ob_parallel_external_sort.h +++ b/src/storage/ob_parallel_external_sort.h @@ -308,7 +308,6 @@ int ObFragmentWriterV2::flush_buffer() io_info.size_ = buf_size_; io_info.tenant_id_ = tenant_id_; io_info.buf_ = buf_; - io_info.io_desc_.set_category(common::ObIOCategory::SYS_IO); io_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_INDEX_BUILD_WRITE); if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.aio_write(io_info, file_io_handle_))) { STORAGE_LOG(WARN, "fail to do aio write macro file", K(ret), K(io_info)); @@ -561,7 +560,6 @@ int ObFragmentReaderV2::prefetch() io_info.size_ = buf_size_; io_info.tenant_id_ = tenant_id_; io_info.buf_ = buf_; - io_info.io_desc_.set_category(common::ObIOCategory::SYS_IO); io_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_INDEX_BUILD_READ); if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.aio_read(io_info, file_io_handles_[handle_cursor_ % MAX_HANDLE_COUNT]))) { if (common::OB_ITER_END != ret) { diff --git a/src/storage/slog_ckpt/ob_linked_macro_block_reader.cpp b/src/storage/slog_ckpt/ob_linked_macro_block_reader.cpp index 6001c15c58..80b1895e3d 100644 --- a/src/storage/slog_ckpt/ob_linked_macro_block_reader.cpp +++ b/src/storage/slog_ckpt/ob_linked_macro_block_reader.cpp @@ -58,7 +58,6 @@ int ObLinkedMacroBlockReader::get_meta_blocks(const MacroBlockId &entry_block) read_info.offset_ = 0; read_info.size_ = sizeof(ObMacroBlockCommonHeader) + sizeof(ObLinkedMacroBlockHeader); read_info.io_desc_.set_mode(ObIOMode::READ); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); if (entry_block.second_id() >= 0) { read_info.macro_block_id_ = entry_block; @@ -109,7 +108,6 @@ int ObLinkedMacroBlockReader::prefetch_block() read_info.offset_ = 0; read_info.size_ = OB_SERVER_BLOCK_MGR.get_macro_block_size(); read_info.io_desc_.set_mode(ObIOMode::READ); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); read_info.macro_block_id_ = macros_handle_.at(prefetch_macro_block_idx_); handles_[handle_pos_].reset(); @@ -153,7 +151,6 @@ int ObLinkedMacroBlockReader::pread_block(const ObMetaDiskAddr &addr, ObMacroBlo ObMacroBlockReadInfo read_info; handler.reset(); read_info.io_desc_.set_mode(ObIOMode::READ); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); if (OB_FAIL(addr.get_block_addr(read_info.macro_block_id_, read_info.offset_, read_info.size_))) { LOG_WARN("fail to get block address", K(ret), K(addr)); @@ -174,7 +171,6 @@ int ObLinkedMacroBlockReader::read_block_by_id( read_info.offset_ = 0; read_info.size_ = OB_SERVER_BLOCK_MGR.get_macro_block_size(); read_info.io_desc_.set_mode(ObIOMode::READ); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); read_info.macro_block_id_ = block_id; handler.reset(); diff --git a/src/storage/slog_ckpt/ob_linked_macro_block_writer.cpp b/src/storage/slog_ckpt/ob_linked_macro_block_writer.cpp index c986652c4c..c2a18977d4 100644 --- a/src/storage/slog_ckpt/ob_linked_macro_block_writer.cpp +++ b/src/storage/slog_ckpt/ob_linked_macro_block_writer.cpp @@ -37,7 +37,6 @@ int ObLinkedMacroBlockWriter::init() ret = OB_INIT_TWICE; LOG_WARN("ObLinkedMacroBlockWriter has not been inited", K(ret)); } else { - io_desc_.set_category(ObIOCategory::SYS_IO); io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_WRITE); is_inited_ = true; } diff --git a/tools/ob_admin/dumpsst/ob_admin_dumpsst_executor.cpp b/tools/ob_admin/dumpsst/ob_admin_dumpsst_executor.cpp index 2154a581b0..1e8cd0df5a 100644 --- a/tools/ob_admin/dumpsst/ob_admin_dumpsst_executor.cpp +++ b/tools/ob_admin/dumpsst/ob_admin_dumpsst_executor.cpp @@ -290,7 +290,6 @@ int ObAdminDumpsstExecutor::dump_macro_block(const ObDumpMacroBlockContext ¯ ObMacroBlockReadInfo read_info; read_info.macro_block_id_.set_block_index(macro_block_context.second_id_); - read_info.io_desc_.set_category(ObIOCategory::SYS_IO); read_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_READ); read_info.offset_ = 0; read_info.size_ = OB_DEFAULT_MACRO_BLOCK_SIZE; diff --git a/unittest/storage/blocksstable/test_sstable_meta.cpp b/unittest/storage/blocksstable/test_sstable_meta.cpp index 4c5e9879c2..cdf242bbcf 100644 --- a/unittest/storage/blocksstable/test_sstable_meta.cpp +++ b/unittest/storage/blocksstable/test_sstable_meta.cpp @@ -149,7 +149,6 @@ void TestRootBlockInfo::prepare_block_root() char *io_buf = static_cast(allocator_.alloc(buf_size)); ASSERT_TRUE(nullptr != io_buf); MEMCPY(io_buf + block_addr_.offset_, buf, block_addr_.size_); - write_info.io_desc_.set_category(ObIOCategory::SYS_IO); write_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_COMPACT_WRITE); write_info.buffer_ = io_buf; write_info.size_ = buf_size; diff --git a/unittest/storage/blocksstable/test_tmp_file.cpp b/unittest/storage/blocksstable/test_tmp_file.cpp index 72e593dae9..8c12ef5e6e 100644 --- a/unittest/storage/blocksstable/test_tmp_file.cpp +++ b/unittest/storage/blocksstable/test_tmp_file.cpp @@ -255,7 +255,6 @@ void TestTmpFileStress::write_data(const int64_t macro_block_size) io_info.fd_ = fd_; io_info.size_ = macro_block_size; io_info.tenant_id_ = 1; - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); char *buf = new char[BUF_COUNT * macro_block_size]; const int64_t timeout_ms = 5000; @@ -276,7 +275,6 @@ void TestTmpFileStress::write_plain_data(char *&buf, const int64_t macro_block_s ASSERT_EQ(OB_SUCCESS, ret); io_info.fd_ = fd_; io_info.tenant_id_ = 1; - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); buf = new char[BUF_COUNT * macro_block_size]; const int64_t timeout_ms = 5000; @@ -302,7 +300,6 @@ void TestTmpFileStress::read_data(const int64_t macro_block_size) io_info.fd_ = fd_; io_info.size_ = macro_block_size; io_info.tenant_id_ = 1; - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); char *buf = new char[macro_block_size]; for (int64_t i = 0; i < BUF_COUNT; ++i) { @@ -325,7 +322,6 @@ void TestTmpFileStress::read_plain_data(const char *read_buf, const int64_t macr io_info.fd_ = fd_; io_info.size_ = macro_block_size; io_info.tenant_id_ = 1; - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); char *buf = new char[BUF_COUNT * macro_block_size]; int64_t offset = 0; @@ -584,7 +580,6 @@ TEST_F(TestTmpFile, test_big_file) char *read_buf = (char *)malloc(write_size); io_info.fd_ = fd; io_info.tenant_id_ = 1; - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); io_info.buf_ = write_buf; io_info.size_ = write_size; @@ -720,7 +715,6 @@ TEST_F(TestTmpFile, test_inner_read_offset_and_seek) char *read_buf = new char [macro_block_size + 256]; io_info.fd_ = fd; io_info.tenant_id_ = 1; - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); io_info.buf_ = write_buf; io_info.size_ = macro_block_size + 256; @@ -832,7 +826,6 @@ TEST_F(TestTmpFile, test_aio_read_and_write) io_info.fd_ = fd; io_info.tenant_id_ = 1; - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); io_info.buf_ = write_buf; io_info.size_ = macro_block_size + 256; @@ -1075,7 +1068,6 @@ TEST_F(TestTmpFile, test_write_less_than_macro_block_size) char *read_buf = new char [256]; io_info.fd_ = fd; io_info.tenant_id_ = 1; - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); io_info.buf_ = write_buf; io_info.size_ = 256; @@ -1163,7 +1155,6 @@ TEST_F(TestTmpFile, test_write_more_than_one_macro_block) char *read_buf = new char [macro_block_size + 256]; io_info.fd_ = fd; io_info.tenant_id_ = 1; - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); io_info.buf_ = write_buf; io_info.size_ = macro_block_size + 256; @@ -1266,7 +1257,6 @@ TEST_F(TestTmpFile, test_single_dir_two_file) ASSERT_EQ(OB_SUCCESS, ret); io_info1.fd_ = fd_1; io_info1.tenant_id_ = 1; - io_info1.io_desc_.set_category(ObIOCategory::USER_IO); io_info1.io_desc_.set_wait_event(2); io_info1.buf_ = write_buf; io_info1.size_ = macro_block_size + 256; @@ -1275,7 +1265,6 @@ TEST_F(TestTmpFile, test_single_dir_two_file) ASSERT_EQ(OB_SUCCESS, ret); io_info2.fd_ = fd_2; io_info2.tenant_id_ = 1; - io_info2.io_desc_.set_category(ObIOCategory::USER_IO); io_info2.io_desc_.set_wait_event(2); io_info2.buf_ = write_buf; io_info2.size_ = macro_block_size + 256; @@ -1346,14 +1335,12 @@ TEST_F(TestTmpFile, test_single_dir_two_file) new_io_info.fd_ = new_fd; new_io_info.tenant_id_ = 1; - new_io_info.io_desc_.set_category(ObIOCategory::USER_IO); new_io_info.io_desc_.set_wait_event(2); new_io_info.buf_ = write_buf; new_io_info.size_ = macro_block_size + 256; old_io_info.fd_ = old_fd; old_io_info.tenant_id_ = 1; - old_io_info.io_desc_.set_category(ObIOCategory::USER_IO); old_io_info.io_desc_.set_wait_event(2); old_io_info.buf_ = write_buf; old_io_info.size_ = macro_block_size + 256; @@ -1496,7 +1483,6 @@ TEST_F(TestTmpFile, test_handle_double_wait) char *read_buf = new char [256]; io_info.fd_ = fd; io_info.tenant_id_ = 1; - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); io_info.buf_ = write_buf; io_info.size_ = 256; @@ -1554,7 +1540,6 @@ TEST_F(TestTmpFile, test_sql_workload) io_info.fd_ = fd; io_info.tenant_id_ = 1; - io_info.io_desc_.set_category(ObIOCategory::USER_IO); io_info.io_desc_.set_wait_event(2); io_info.buf_ = write_buf; io_info.size_ = write_size; diff --git a/unittest/storage/test_io_manager.cpp b/unittest/storage/test_io_manager.cpp index 0cf96bfaa6..d1b8dca0dc 100644 --- a/unittest/storage/test_io_manager.cpp +++ b/unittest/storage/test_io_manager.cpp @@ -18,6 +18,7 @@ #include "share/io/ob_io_manager.h" #include "share/io/ob_io_calibration.h" #include "share/io/io_schedule/ob_io_mclock.h" +#include "share/resource_manager/ob_cgroup_ctrl.h" #undef private #include "share/ob_local_device.h" #include "lib/thread/thread_pool.h" @@ -96,11 +97,12 @@ static void TearDownTestCase() static ObIOInfo get_random_io_info() { ObIOInfo io_info; + const int64_t max_group_num = 0; //不包括OTHER GROUPS的其他group io_info.tenant_id_ = OB_SERVER_TENANT_ID; io_info.fd_.first_id_ = ObRandom::rand(0, 10000); io_info.fd_.second_id_ = ObRandom::rand(0, 10000); io_info.flag_.set_mode(static_cast(ObRandom::rand(0, (int)ObIOMode::MAX_MODE - 1))); - io_info.flag_.set_category(static_cast(ObRandom::rand(0, (int)ObIOCategory::MAX_CATEGORY - 1))); + io_info.flag_.set_group_id(0); // 0 means default io_info.flag_.set_wait_event(ObRandom::rand(1, 9999)); io_info.offset_ = ObRandom::rand(1, 1000L * 1000L * 1000L); io_info.size_ = ObRandom::rand(1, 1000L * 10L); @@ -113,12 +115,14 @@ static ObTenantIOConfig default_tenant_io_config() ObTenantIOConfig tenant_config; tenant_config.callback_thread_count_ = 2; tenant_config.memory_limit_ = 1024L * 1024L * 1024L; + tenant_config.group_num_ = 0; tenant_config.unit_config_.min_iops_ = 1000; tenant_config.unit_config_.max_iops_ = 1000; tenant_config.unit_config_.weight_ = 1000; - tenant_config.other_config_.min_percent_ = 100; - tenant_config.other_config_.max_percent_ = 100; - tenant_config.other_config_.weight_percent_ = 100; + tenant_config.other_group_config_.min_percent_ = 100; + tenant_config.other_group_config_.max_percent_ = 100; + tenant_config.other_group_config_.weight_percent_ = 100; + tenant_config.group_config_change_ = false; return tenant_config; } }; @@ -156,7 +160,7 @@ TEST_F(TestIOStruct, IOFlag) // normal usage flag.set_mode(ObIOMode::READ); - flag.set_category(ObIOCategory::USER_IO); + flag.set_group_id(0); flag.set_wait_event(99); ASSERT_TRUE(flag.is_valid()); @@ -168,12 +172,10 @@ TEST_F(TestIOStruct, IOFlag) flag2.set_mode((ObIOMode)88); ASSERT_FALSE(flag2.is_valid()); - // test io category + // test io group flag2 = flag; ASSERT_TRUE(flag2.is_valid()); - flag2.set_category(ObIOCategory::MAX_CATEGORY); - ASSERT_FALSE(flag2.is_valid()); - flag2.set_category((ObIOCategory)88); + flag2.set_group_id(-1); ASSERT_FALSE(flag2.is_valid()); // test wait event number @@ -204,7 +206,7 @@ TEST_F(TestIOStruct, IOInfo) info.tenant_id_ = OB_SERVER_TENANT_ID; info.fd_ = fd; info.flag_.set_mode(ObIOMode::READ); - info.flag_.set_category(ObIOCategory::USER_IO); + info.flag_.set_group_id(0); info.flag_.set_wait_event(1); info.offset_ = 80; info.size_ = 1; @@ -302,11 +304,13 @@ TEST_F(TestIOStruct, IORequest) read_info.tenant_id_ = OB_SERVER_TENANT_ID; read_info.fd_ = fd; read_info.flag_.set_mode(ObIOMode::READ); - read_info.flag_.set_category(ObIOCategory::USER_IO); + read_info.flag_.set_group_id(0); read_info.flag_.set_wait_event(1); read_info.offset_ = 89; read_info.size_ = 1; + ASSERT_EQ(req.get_group_id(), 0); ASSERT_TRUE(read_info.is_valid()); + ASSERT_SUCC(req.tenant_io_mgr_.get_ptr()->io_usage_.init(0)); ASSERT_SUCC(req.init(read_info)); ASSERT_TRUE(req.is_inited_); ASSERT_EQ(req.io_buf_, nullptr); // read buf allocation is delayed @@ -482,7 +486,7 @@ TEST_F(TestIOStruct, IOScheduler) ASSERT_TRUE(io_config.is_valid()); ObIOScheduler scheduler(io_config, io_allocator); ASSERT_FALSE(scheduler.is_inited_); - ASSERT_SUCC(scheduler.init(2, 10000)); + ASSERT_SUCC(scheduler.init(2)); ASSERT_TRUE(scheduler.is_inited_); // test schedule @@ -494,6 +498,9 @@ TEST_F(TestIOStruct, IOScheduler) ObTenantIOClock io_clock; ObIOUsage io_usage; ASSERT_SUCC(io_clock.init(tenant_config, &io_usage)); + io_clock.destroy(); + ASSERT_SUCC(io_usage.init(0)); + ASSERT_SUCC(io_clock.init(tenant_config, &io_usage)); //ASSERT_SUCC(scheduler.schedule_request(io_clock, req)); // test destroy @@ -505,24 +512,36 @@ TEST_F(TestIOStruct, IOScheduler) TEST_F(TestIOStruct, MClockQueue) { + ObIOAllocator io_allocator; + ASSERT_SUCC(io_allocator.init(TEST_TENANT_ID, IO_MEMORY_LIMIT)); ObTenantIOConfig io_config; io_config.callback_thread_count_ = 2; io_config.memory_limit_ = 1024L * 1024L * 1024L; + io_config.group_num_ = 2; io_config.unit_config_.min_iops_ = 100; io_config.unit_config_.max_iops_ = 10000L; io_config.unit_config_.weight_ = 1000; - io_config.category_configs_[0].min_percent_ = 1; - io_config.category_configs_[0].max_percent_ = 90; - io_config.category_configs_[0].weight_percent_ = 60; - io_config.category_configs_[1].min_percent_ = 1; - io_config.category_configs_[1].max_percent_ = 90; - io_config.category_configs_[1].weight_percent_ = 30; - io_config.other_config_.min_percent_ = 98; - io_config.other_config_.max_percent_ = 100; - io_config.other_config_.weight_percent_ = 10; + + ObTenantIOConfig::GroupConfig group_config_1; + group_config_1.min_percent_ = 1; + group_config_1.max_percent_ = 90; + group_config_1.weight_percent_ = 60; + ObTenantIOConfig::GroupConfig group_config_2; + group_config_2.min_percent_ = 1; + group_config_2.max_percent_ = 90; + group_config_2.weight_percent_ = 30; + io_config.group_configs_.push_back(group_config_1); + io_config.group_configs_.push_back(group_config_2); + io_config.group_ids_.push_back(1); + io_config.group_ids_.push_back(2); + io_config.other_group_config_.min_percent_ = 98; + io_config.other_group_config_.max_percent_ = 100; + io_config.other_group_config_.weight_percent_ = 10; ASSERT_TRUE(io_config.is_valid()); ObTenantIOClock tenant_clock; ObIOUsage io_usage; + ASSERT_SUCC(io_usage.init(2)); + ASSERT_SUCC(io_usage.refresh_group_num(2)); ASSERT_SUCC(tenant_clock.init(io_config, &io_usage)); ObMClockQueue mqueue1; ObMClockQueue mqueue2; @@ -671,11 +690,12 @@ TEST_F(TestIOStruct, IOManager) class TestIOManager : public TestIOStruct { + // basic use resource manager public: virtual void SetUp() { ObIOManager::get_instance().destroy(); - const int64_t memory_limit = 10L * 1024L * 1024L * 1024L; // 4GB + const int64_t memory_limit = 10L * 1024L * 1024L * 1024L; // 10GB ASSERT_SUCC(ObIOManager::get_instance().init(memory_limit)); ASSERT_SUCC(ObIOManager::get_instance().start()); @@ -687,12 +707,12 @@ public: ObTenantIOConfig io_config; io_config.memory_limit_ = memory_limit; io_config.callback_thread_count_ = 2; - io_config.other_config_.min_percent_ = 100; - io_config.other_config_.max_percent_ = 100; - io_config.other_config_.weight_percent_ = 100; io_config.unit_config_.min_iops_ = 10000; io_config.unit_config_.max_iops_ = 100000; io_config.unit_config_.weight_ = 100; + io_config.other_group_config_.min_percent_ = 100; + io_config.other_group_config_.max_percent_ = 100; + io_config.other_group_config_.weight_percent_ = 100; ASSERT_SUCC(OB_IO_MANAGER.add_tenant_io_manager(tenant_id, io_config)); } virtual void TearDown() @@ -725,7 +745,7 @@ TEST_F(TestIOManager, simple) io_info.tenant_id_ = 500; io_info.fd_ = fd; io_info.flag_.set_write(); - io_info.flag_.set_category(ObIOCategory::USER_IO); + io_info.flag_.set_group_id(0); io_info.flag_.set_wait_event(100); io_info.offset_ = 0; io_info.size_ = write_io_size; @@ -839,24 +859,22 @@ enum class IOPerfMode struct IOPerfLoad { IOPerfLoad() - : tenant_id_(0), category_(ObIOCategory::MAX_CATEGORY), device_id_(0), - mode_(ObIOMode::MAX_MODE), size_(0), depth_(0), iops_(0), - thread_count_(0), is_sequence_(false), start_delay_ts_(0), stop_delay_ts_(0), + : tenant_id_(0), device_id_(0), mode_(ObIOMode::MAX_MODE), size_(0), depth_(0), iops_(0), + thread_count_(0), is_sequence_(false), group_id_(0), start_delay_ts_(0), stop_delay_ts_(0), device_(nullptr), perf_mode_(IOPerfMode::UNKNOWN) {} - TO_STRING_KV(K(tenant_id_), "category", get_io_category_name(category_), K(device_id_), + TO_STRING_KV(K(tenant_id_), K(device_id_), "mode", ObIOMode::READ == mode_ ? "read" : ObIOMode::WRITE == mode_ ? "write" : "unknown", - "io_size", size_, "io_depth", depth_, "target_iops", iops_, + "group_id", group_id_, "io_size", size_, "io_depth", depth_, "target_iops", iops_, K(thread_count_), K(is_sequence_), K(start_delay_ts_), K(stop_delay_ts_), KP(device_), K(perf_mode_)); bool is_valid() const { - return tenant_id_ > 0 && category_ < ObIOCategory::MAX_CATEGORY && device_id_ > 0 + return tenant_id_ > 0 && group_id_ >= 0 && device_id_ > 0 && mode_ < ObIOMode::MAX_MODE && size_ > 0 && depth_ > 0 && iops_ >= 0 && thread_count_ > 0 && start_delay_ts_ >= 0 && stop_delay_ts_ > start_delay_ts_ && size_ > 0 && (ObIOMode::WRITE == mode_ ? is_io_aligned(size_) : true) && perf_mode_ != IOPerfMode::UNKNOWN; } int32_t tenant_id_; - ObIOCategory category_; int32_t device_id_; ObIOMode mode_; int32_t size_; @@ -864,6 +882,7 @@ struct IOPerfLoad int64_t iops_; int32_t thread_count_; bool is_sequence_; + uint64_t group_id_; int64_t start_delay_ts_; int64_t stop_delay_ts_; IOPerfDevice *device_; @@ -936,7 +955,7 @@ public: const int64_t max_iops, const int64_t weight, IOPerfTenant &curr_tenant); - TO_STRING_KV(K(modify_delay_ts_), K(fd_), K(curr_tenant_)); + TO_STRING_KV(K(load_), K(modify_delay_ts_), K(fd_), K(curr_tenant_)); public: int64_t modify_init_ts_; int64_t modify_delay_ts_; @@ -946,10 +965,10 @@ public: ObIOFd fd_; }; -#define PERF_CONFIG_FILE "io_perf.conf" +#define GROUP_PERF_CONFIG_FILE "io_perf.conf" -void write_perf_config(); -int parse_perf_config(const char *config_file_path, +void write_group_perf_config(); +int parse_group_perf_config(const char *config_file_path, IOPerfScheduler &scheduler_config, ObIArray &perf_devices, ObIArray &perf_tenants, @@ -1009,7 +1028,7 @@ TEST_F(TestIOManager, tenant) ASSERT_SUCC(OB_IO_MANAGER.add_tenant_io_manager(1002, default_config)); int64_t current_ts = ObTimeUtility::fast_current_time(); IOPerfLoad load; - load.category_ = ObIOCategory::USER_IO; + load.group_id_ = 0; load.depth_ = 1; IOPerfDevice device; device.device_id_ = 1; @@ -1039,24 +1058,24 @@ TEST_F(TestIOManager, tenant) TEST_F(TestIOManager, perf) { - // uset multi thread to do some io stress, maybe use test_io_performance + // use multi thread to do some io stress, maybe use test_io_performance bool is_perf_config_exist = false; - ASSERT_SUCC(FileDirectoryUtils::is_exists(PERF_CONFIG_FILE, is_perf_config_exist)); + ASSERT_SUCC(FileDirectoryUtils::is_exists(GROUP_PERF_CONFIG_FILE, is_perf_config_exist)); if (!is_perf_config_exist) { - write_perf_config(); + write_group_perf_config(); } // parse configs IOPerfScheduler scheduler_config; ObArray perf_devices; ObArray perf_tenants; ObArray perf_loads; - ASSERT_SUCC(parse_perf_config(PERF_CONFIG_FILE, scheduler_config, perf_devices, perf_tenants, perf_loads)); + ASSERT_SUCC(parse_group_perf_config(GROUP_PERF_CONFIG_FILE, scheduler_config, perf_devices, perf_tenants, perf_loads)); ASSERT_TRUE(perf_devices.count() > 0); ASSERT_TRUE(perf_tenants.count() > 0); ASSERT_TRUE(perf_loads.count() > 0); ObIOManager::get_instance().destroy(); - const int64_t memory_limit = 30L * 1024L * 1024L * 1024L; // 10GB + const int64_t memory_limit = 30L * 1024L * 1024L * 1024L; // 30GB const int64_t queue_depth = 100L; ASSERT_SUCC(ObIOManager::get_instance().init(memory_limit, queue_depth, scheduler_config.sender_count_, scheduler_config.schedule_media_id_)); ASSERT_SUCC(ObIOManager::get_instance().start()); @@ -1077,6 +1096,9 @@ TEST_F(TestIOManager, perf) IOPerfTenant &curr_config = perf_tenants.at(i); LOG_INFO("wenqu: tenant config", K(curr_config), K(i)); ASSERT_SUCC(OB_IO_MANAGER.add_tenant_io_manager(curr_config.tenant_id_, curr_config.config_)); + ObRefHolder tenant_holder; + ASSERT_SUCC(OB_IO_MANAGER.get_tenant_io_manager(curr_config.tenant_id_, tenant_holder)); + ASSERT_SUCC(tenant_holder.get_ptr()->refresh_group_io_config()); } // prepare perf runners char *runner_buf = (char *)malloc(perf_loads.count() * sizeof(IOPerfRunner)); @@ -1110,24 +1132,24 @@ TEST_F(TestIOManager, perf) TEST_F(TestIOManager, modify) { - // uset multi thread to do some io stress, maybe use test_io_performance + // use multi thread to do modify group_io_config bool is_perf_config_exist = false; - ASSERT_SUCC(FileDirectoryUtils::is_exists(PERF_CONFIG_FILE, is_perf_config_exist)); + ASSERT_SUCC(FileDirectoryUtils::is_exists(GROUP_PERF_CONFIG_FILE, is_perf_config_exist)); if (!is_perf_config_exist) { - write_perf_config(); + write_group_perf_config(); } // parse configs IOPerfScheduler scheduler_config; ObArray perf_devices; ObArray perf_tenants; ObArray perf_loads; - ASSERT_SUCC(parse_perf_config(PERF_CONFIG_FILE, scheduler_config, perf_devices, perf_tenants, perf_loads)); + ASSERT_SUCC(parse_group_perf_config(GROUP_PERF_CONFIG_FILE, scheduler_config, perf_devices, perf_tenants, perf_loads)); ASSERT_TRUE(perf_devices.count() > 0); ASSERT_TRUE(perf_tenants.count() > 0); ASSERT_TRUE(perf_loads.count() > 0); ObIOManager::get_instance().destroy(); - const int64_t memory_limit = 30L * 1024L * 1024L * 1024L; // 10GB + const int64_t memory_limit = 30L * 1024L * 1024L * 1024L; // 30GB const int64_t queue_depth = 100L; ASSERT_SUCC(ObIOManager::get_instance().init(memory_limit, queue_depth, scheduler_config.sender_count_, scheduler_config.schedule_media_id_)); ASSERT_SUCC(ObIOManager::get_instance().start()); @@ -1148,6 +1170,9 @@ TEST_F(TestIOManager, modify) IOPerfTenant &curr_config = perf_tenants.at(i); LOG_INFO("wenqu: tenant config", K(curr_config), K(i)); ASSERT_SUCC(OB_IO_MANAGER.add_tenant_io_manager(curr_config.tenant_id_, curr_config.config_)); + ObRefHolder tenant_holder; + ASSERT_SUCC(OB_IO_MANAGER.get_tenant_io_manager(curr_config.tenant_id_, tenant_holder)); + ASSERT_SUCC(tenant_holder.get_ptr()->refresh_group_io_config()); } // prepare perf runners char *runner_buf = (char *)malloc(perf_loads.count() * sizeof(IOPerfRunner)); @@ -1161,10 +1186,13 @@ TEST_F(TestIOManager, modify) ASSERT_SUCC(runner->init(start_ts, cur_load)); ASSERT_SUCC(runners.push_back(runner)); LOG_INFO("runner start now"); + } + //prepare modifyer + for (int64_t i = 0; i < perf_tenants.count(); ++i) { IOConfModify *modifyer=new (modifyer_buf + i * sizeof(IOConfModify)) IOConfModify(); IOPerfTenant &curr_tenant = perf_tenants.at(i); - int64_t modify_init_ts = start_ts+cur_load.start_delay_ts_; - int64_t modify_delay_ts = 3000000L; + int64_t modify_init_ts = start_ts; + int64_t modify_delay_ts = 3000000L; //2s后开始修改 ASSERT_SUCC(modifyer->init(modify_init_ts, modify_delay_ts, curr_tenant)); ASSERT_SUCC(modifyers.push_back(modifyer)); } @@ -1270,10 +1298,10 @@ int TestIOCallback::inner_process(const bool is_success) } /**************** io perf ******************/ -void write_perf_config() +void write_group_perf_config() { int fd = -1; - const char *file_name = PERF_CONFIG_FILE; + const char *file_name = GROUP_PERF_CONFIG_FILE; if (0 > (fd = ::open(file_name, O_RDWR | O_CREAT | O_TRUNC, 0644))) { LOG_WARN("open perf config file failed", K(fd), K(file_name)); } else { @@ -1284,14 +1312,15 @@ void write_perf_config() "device_id media_id async_channel sync_channel max_io_depth file_size_gb file_path\n" "1 0 8 1 64 1 ./perf_test\n" "\n" - "tenant_id min_iops max_iops weight category\n" - "1002 5000 50000 700 user: 90, 100, 80; large: 1, 90, 10; other: 9, 100, 10;\n" - "1001 50000 100000 1000 other: 100, 100, 100;\n" + "tenant_id min_iops max_iops weight group\n" + "1001 5000 100000 700 10001: testgroup1: 80, 100, 60; 10002: testgroup2: 10, 60, 30; 0: OTHER_GROUPS: 10, 100, 10;\n" + "1002 1000 50000 1000 0: testgroup1: 100, 100, 100;\n" "\n" - "tenant_id device_id category io_mode io_size_byte io_depth perf_mode target_iops thread_count is_sequence start_s stop_s\n" - "1001 1 user r 16384 10 rolling 0 16 0 0 2\n" - "1002 1 large r 16384 10 rolling 4000 16 0 1 3\n" - "#1002 1 sys w 2097152 10 rolling 40000 16 0 2 3\n" + "tenant_id device_id group io_mode io_size_byte io_depth perf_mode target_iops thread_count is_sequence start_s stop_s\n" + "1001 1 0 r 16384 10 rolling 0 16 0 0 8\n" + "1001 1 10001 r 16384 10 rolling 0 16 0 2 7\n" + "1001 1 10002 r 16384 10 rolling 0 16 0 0 6\n" + "1002 1 0 r 16384 10 rolling 0 16 0 0 5\n" ; const int64_t file_len = strlen(file_buf); int write_ret = ::write(fd, file_buf, file_len); @@ -1302,7 +1331,7 @@ void write_perf_config() } } -int parse_perf_config(const char *config_file_path, +int parse_group_perf_config(const char *config_file_path, IOPerfScheduler &scheduler_config, ObIArray &perf_devices, ObIArray &perf_tenants, @@ -1323,8 +1352,8 @@ int parse_perf_config(const char *config_file_path, char curr_line[1024] = { 0 }; const char *scheduler_header = "sender_count schedule_media_id io_greed"; const char *device_header = "device_id media_id async_channel sync_channel max_io_depth file_size_gb file_path"; - const char *tenant_header = "tenant_id min_iops max_iops weight category"; - const char *load_header = "tenant_id device_id category io_mode io_size_byte io_depth perf_mode target_iops thread_count is_sequence start_s stop_s"; + const char *tenant_header = "tenant_id min_iops max_iops weight group"; + const char *load_header = "tenant_id device_id group io_mode io_size_byte io_depth perf_mode target_iops thread_count is_sequence start_s stop_s"; enum class PerfConfigType { SCHEDULER, DEVICE, TENANT, LOAD, MAX }; PerfConfigType config_type = PerfConfigType::MAX; while (OB_SUCC(ret)) { @@ -1349,7 +1378,7 @@ int parse_perf_config(const char *config_file_path, ret = OB_ERR_UNEXPECTED; LOG_WARN("scan config file failed", K(ret), K(scan_ret)); } - LOG_INFO("wenqu: parse scheduler config", K(ret), K(scheduler_config)); + LOG_INFO("qilu: parse scheduler config", K(ret), K(scheduler_config)); } else if (PerfConfigType::DEVICE == config_type) { IOPerfDevice item; int scan_ret = sscanf(curr_line, "%d%d%d%d%d%ld%s\n", @@ -1365,38 +1394,37 @@ int parse_perf_config(const char *config_file_path, } else if (OB_FAIL(perf_devices.push_back(item))) { LOG_WARN("add item failed", K(ret), K(item)); } - LOG_INFO("wenqu: parse device", K(ret), K(item)); + LOG_INFO("qilu: parse device", K(ret), K(item)); } } else if (PerfConfigType::TENANT == config_type) { IOPerfTenant item; - char category_config[1024] = { 0 }; + char group_config[1024] = { 0 }; int scan_ret = sscanf(curr_line, "%d%ld%ld%ld%[^\n]\n", &item.tenant_id_, &item.config_.unit_config_.min_iops_, &item.config_.unit_config_.max_iops_, - &item.config_.unit_config_.weight_, category_config); + &item.config_.unit_config_.weight_, group_config); if (OB_UNLIKELY(5 != scan_ret)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("scan config file failed", K(ret), K(scan_ret)); } else { item.config_.memory_limit_ = IO_MEMORY_LIMIT; item.config_.callback_thread_count_ = 1; - // parse category config - if (OB_FAIL(item.config_.parse_category_config(category_config))) { - LOG_WARN("parse category config failed", K(ret), K(category_config)); + // parse group config + if (OB_FAIL(item.config_.parse_group_config(group_config))) { + LOG_WARN("parse group config failed", K(ret), K(group_config)); } else if (OB_UNLIKELY(!item.is_valid())) { ret = OB_INVALID_DATA; LOG_WARN("invalid data", K(ret), K(item)); } else if (OB_FAIL(perf_tenants.push_back(item))) { LOG_WARN("add item failed", K(ret), K(item)); } - LOG_INFO("wenqu: parse tenant", K(ret), K(item), K(category_config)); + LOG_INFO("qilu: parse tenant", K(ret), K(item), K(group_config), K(item.config_)); } } else if (PerfConfigType::LOAD == config_type) { IOPerfLoad item; - char category_name[64] = { 0 }; char io_mode[16] = { 0 }; char perf_mode[16] = { 0 }; - int scan_ret = sscanf(curr_line, "%d%d%s%s%d%d%s%ld%d%d%ld%ld\n", - &item.tenant_id_, &item.device_id_, category_name, io_mode, + int scan_ret = sscanf(curr_line, "%d%d%ld%s%d%d%s%ld%d%d%ld%ld\n", + &item.tenant_id_, &item.device_id_, &item.group_id_, io_mode, &item.size_, &item.depth_, perf_mode, &item.iops_, &item.thread_count_, (int *)&item.is_sequence_, &item.start_delay_ts_, &item.stop_delay_ts_); if (OB_UNLIKELY(12 != scan_ret)) { @@ -1431,14 +1459,13 @@ int parse_perf_config(const char *config_file_path, break; } } - item.category_ = get_io_category_enum(category_name); if (OB_UNLIKELY(!item.is_valid())) { ret = OB_INVALID_DATA; LOG_WARN("invalid data", K(ret), K(item)); } else if (OB_FAIL(perf_loads.push_back(item))) { LOG_WARN("add item failed", K(ret), K(item)); } - LOG_INFO("wenqu: parse load", K(ret), K(item), K(io_mode), K(category_name)); + LOG_INFO("qilu: parse load", K(ret), K(item), K(io_mode), K(item.group_id_)); } } else { ret = OB_ERR_UNEXPECTED; @@ -1551,7 +1578,7 @@ int IOPerfRunner::do_perf_batch() const int64_t check_count = load_.iops_ / load_.thread_count_ / (1000L / check_interval_ms); int64_t last_check_ts = ObTimeUtility::fast_current_time(); const bool need_control_io_speed = 0 != load_.iops_; - LOG_INFO("perf start", K(load_.tenant_id_), K(load_.category_)); + LOG_INFO("perf start", K(load_.tenant_id_), K(load_.group_id_)); while (!has_set_stop()) { (void) do_batch_io(); if (need_control_io_speed) { @@ -1574,7 +1601,7 @@ int IOPerfRunner::do_perf_rolling() int ret = OB_SUCCESS; ObIOInfo info; info.tenant_id_ = load_.tenant_id_; - info.flag_.set_category(load_.category_); + info.flag_.set_group_id(load_.group_id_); info.flag_.set_mode(load_.mode_); info.flag_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); info.fd_ = fd_; @@ -1667,7 +1694,7 @@ int IOPerfRunner::do_batch_io() int ret = OB_SUCCESS; ObIOInfo info; info.tenant_id_ = load_.tenant_id_; - info.flag_.set_category(load_.category_); + info.flag_.set_group_id(load_.group_id_); info.flag_.set_mode(load_.mode_); info.flag_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); info.fd_ = fd_;