diff --git a/src/observer/ob_inner_sql_connection.cpp b/src/observer/ob_inner_sql_connection.cpp index 2f08ea69b8..4637dbde73 100644 --- a/src/observer/ob_inner_sql_connection.cpp +++ b/src/observer/ob_inner_sql_connection.cpp @@ -753,7 +753,7 @@ int ObInnerSQLConnection::query(sqlclient::ObIExecutor &executor, LOG_WARN("get tenant schema version failed", K(ret), K(ob_sql_)); } else if (OB_FAIL(res.schema_guard_.get_schema_version(OB_SYS_TENANT_ID, local_sys_schema_version))) { LOG_WARN("get sys tenant schema version failed", K(ret), K(ob_sql_)); - } else { + } else if (OB_UNLIKELY(is_extern_session())) { res.result_set().get_exec_context().get_task_exec_ctx().set_query_tenant_begin_schema_version(local_tenant_schema_version); res.result_set().get_exec_context().get_task_exec_ctx().set_query_sys_begin_schema_version(local_sys_schema_version); } @@ -763,8 +763,7 @@ int ObInnerSQLConnection::query(sqlclient::ObIExecutor &executor, // do nothing } else if (OB_FAIL(SMART_CALL(do_query(executor, res)))) { ret_code = ret; - LOG_WARN("execute failed", K(ret), K(tenant_id), K(executor), K(retry_cnt), - K(local_sys_schema_version), K(local_tenant_schema_version)); + LOG_WARN("execute failed", K(ret), K(tenant_id), K(executor), K(retry_cnt)); ret = process_retry(res, ret, abs_timeout_us, need_retry, retry_cnt, is_from_pl); // moved here from ObInnerSQLConnection::do_query() -> ObInnerSQLResult::open(). int close_ret = res.force_close(); diff --git a/src/sql/executor/ob_remote_executor_processor.cpp b/src/sql/executor/ob_remote_executor_processor.cpp index c7549446e7..8416286133 100644 --- a/src/sql/executor/ob_remote_executor_processor.cpp +++ b/src/sql/executor/ob_remote_executor_processor.cpp @@ -67,35 +67,24 @@ int ObRemoteBaseExecuteP::base_before_process(int64_t tenant_schema_version, int64_t tenant_local_version = -1; int64_t sys_local_version = -1; int64_t query_timeout = 0; - uint64_t tenant_id = 0; - if (OB_ISNULL(gctx_.schema_service_) || OB_ISNULL(gctx_.sql_engine_) || OB_ISNULL(gctx_.executor_rpc_) || OB_ISNULL(session_info)) { ret = OB_ERR_UNEXPECTED; LOG_ERROR("schema service or sql engine is NULL", K(ret), K(gctx_.schema_service_), K(gctx_.sql_engine_)); - } else if (GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_1_0_0 && - (tenant_schema_version == OB_INVALID_VERSION || sys_schema_version == OB_INVALID_VERSION)) { - // 4.1以及之后的版本,不允许传schema_version为-1 - ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid tenant_schema_version and sys_schema_version", - K(tenant_schema_version), K(sys_schema_version)); - } else if (FALSE_IT(tenant_id = session_info->get_effective_tenant_id())) { - // record tanent_id - } else if (tenant_id == 0) { - ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid tanent_id", K(ret)); } else if (FALSE_IT(THIS_WORKER.set_compatibility_mode( ORACLE_MODE == session_info->get_compatibility_mode() ? lib::Worker::CompatMode::ORACLE : lib::Worker::CompatMode::MYSQL))) { //设置RPC work线程的租户兼容模式 - } else if (OB_FAIL(gctx_.schema_service_->get_tenant_refreshed_schema_version(tenant_id, tenant_local_version))) { + } else if (OB_FAIL(gctx_.schema_service_->get_tenant_refreshed_schema_version( + session_info->get_effective_tenant_id(), tenant_local_version))) { if (OB_ENTRY_NOT_EXIST == ret) { // 本地schema可能还没刷出来 tenant_local_version = OB_INVALID_VERSION; ret = OB_SUCCESS; } else { - LOG_WARN("fail to get tenant refreshed schema version", K(ret), K(tenant_id)); + LOG_WARN("fail to get tenant refreshed schema version", K(ret), + K(session_info->get_effective_tenant_id())); } } else if (OB_FAIL(gctx_.schema_service_->get_tenant_refreshed_schema_version( OB_SYS_TENANT_ID, sys_local_version))) { @@ -112,50 +101,67 @@ int ObRemoteBaseExecuteP::base_before_process(int64_t tenant_schema_version, } } - if (OB_SUCC(ret)) { - // 先取一次本地最新的schema_guard - if (OB_FAIL(gctx_.schema_service_->get_tenant_schema_guard( - tenant_id, schema_guard_, tenant_local_version, sys_local_version))) { - LOG_WARN("fail to get schema guard", K(ret), K(tenant_local_version), - K(sys_local_version), K(tenant_schema_version), K(sys_schema_version)); - } - } - if (OB_SUCC(ret) && tenant_schema_version != tenant_local_version) { - // 前边一定获取过schema_guard_,并且获取的是当前server最新的schema版本 - if (OB_FAIL(ObSQLUtils::check_table_version(table_version_equal, dependency_tables, schema_guard_))) { - LOG_WARN("fail to check table_version", K(ret), K(dependency_tables)); - } else if (!table_version_equal) { - if (tenant_schema_version == OB_INVALID_VERSION) { - // 表版本不一致时控制端传过来的schema_version是-1,需要重试直到表版本一致 - ret = OB_ERR_WAIT_REMOTE_SCHEMA_REFRESH; - LOG_WARN("dependency table version is not equal, need retry", K(ret)); - } else if (tenant_schema_version > tenant_local_version) { - // 本地schema version落后。此时需要刷新schema版本,并且重新获取schema_guard - if (OB_FAIL(gctx_.schema_service_->async_refresh_schema(tenant_id, tenant_schema_version))) { - LOG_WARN("fail to push back effective_tenant_id", K(ret), K(tenant_schema_version), - K(tenant_id), K(tenant_local_version)); - } else if (OB_FAIL(gctx_.schema_service_->get_tenant_schema_guard( - tenant_id, schema_guard_, tenant_schema_version, sys_schema_version))) { - LOG_WARN("fail to get schema guard", K(ret), K(tenant_schema_version), K(sys_schema_version)); - } - } else if (tenant_schema_version <= tenant_local_version) { - // 这种场景下需要重新取一次schema_guard - if (OB_FAIL(gctx_.schema_service_->get_tenant_schema_guard( - tenant_id, schema_guard_, tenant_schema_version, sys_schema_version))) { - LOG_WARN("fail to get schema guard", K(ret), K(tenant_schema_version), K(sys_schema_version)); + // 获取schema_guard (tenant_local_version, sys_schema_version) + if (OB_FAIL(gctx_.schema_service_->get_tenant_schema_guard( + session_info->get_effective_tenant_id(), + schema_guard_))) { + LOG_WARN("fail to get schema guard", K(ret),K(tenant_local_version), K(sys_local_version), + K(tenant_schema_version), K(sys_schema_version)); + } + + if (OB_FAIL(ret)) { + // do nothing + } else if (OB_SUCC(sql::ObSQLUtils::check_table_version(table_version_equal, + dependency_tables, schema_guard_))) { + // 检查table version返回成功 + if (!table_version_equal) { + // tenant_schema_version > tenant_local_version 刷新schema,使用指定的schema版本获取guard + // tenant_schema_version < tenant_local_version 本地超前,用task指定的schema版本获取guard + if (tenant_schema_version > tenant_local_version) { + // 本地schema version落后。此时需要刷新 + if (OB_FAIL(gctx_.schema_service_->async_refresh_schema( + session_info->get_effective_tenant_id(), tenant_schema_version))) { + LOG_WARN("fail to push back effective_tenant_id", K(ret), + K(session_info->get_effective_tenant_id()), + K(tenant_schema_version), K(tenant_local_version)); + } } } } else { - // 如果table_version_equal == true,说明表schema一致, - // 此时就不需要重新获取schema_guard了,当前schema_guard就是可用的 + // check table version失败, 忽略错误,继续向下 + // 无论本地schema版本超前还是落后于,都使用task指定的schema version + ret = OB_SUCCESS; + LOG_INFO("fail to check table_schema_version", K(ret)); + if (tenant_schema_version > tenant_local_version) { + // 本地落后于task指定的schema version,刷新本地的schema + if (OB_FAIL(gctx_.schema_service_->async_refresh_schema( + session_info->get_effective_tenant_id(), tenant_schema_version))) { + LOG_WARN("fail to push back effective_tenant_id", K(ret), + K(session_info->get_effective_tenant_id()), + K(tenant_schema_version), K(tenant_local_version)); + } + } } } + // table_version_equal == false: + // 只有当table_version_equal==true时,使用local_schema_version获取schema guard, + // 其余情况都需要需要获取task指定schema版本的schema_guard + if (OB_SUCC(ret) && !table_version_equal) { + if (OB_FAIL(gctx_.schema_service_->get_tenant_schema_guard( + session_info->get_effective_tenant_id(), + schema_guard_, + tenant_schema_version, + sys_schema_version))) { + LOG_WARN("fail to get schema guard", K(ret), K(tenant_schema_version), K(sys_schema_version)); + } + } + int64_t local_tenant_schema_version = -1; if (OB_FAIL(ret)) { //do nothing - } else if (OB_FAIL(schema_guard_.get_schema_version(tenant_id, local_tenant_schema_version))) { + } else if (OB_FAIL(schema_guard_.get_schema_version(session_info->get_effective_tenant_id(), local_tenant_schema_version))) { LOG_WARN("get schema version from schema_guard failed", K(ret)); } else if (OB_FAIL(session_info->get_query_timeout(query_timeout))) { LOG_WARN("get query timeout failed", K(ret)); @@ -171,14 +177,13 @@ int ObRemoteBaseExecuteP::base_before_process(int64_t tenant_schema_version, vt_ctx.schema_guard_ = &schema_guard_; exec_ctx_.set_virtual_table_ctx(vt_ctx); } - LOG_TRACE("print tenant_schema_version for remote_execute", K(tenant_schema_version), K(tenant_local_version), - K(local_tenant_schema_version), K(table_version_equal), K(ret), K(sys_schema_version), K(sys_local_version)); + if (OB_FAIL(ret)) { if (local_tenant_schema_version != tenant_schema_version) { if (is_schema_error(ret)) { ret = OB_ERR_WAIT_REMOTE_SCHEMA_REFRESH; // 重写错误码,使得scheduler端能等待远端schema刷新并重试 } - } else if (ret == OB_TENANT_NOT_EXIST) { + } else if (-1 == tenant_schema_version && ret == OB_TENANT_NOT_EXIST) { // fix bug: https://work.aone.alibaba-inc.com/issue/45890226 // 控制端重启observer,导致租户schema没刷出来,发送过来的schema_version异常, 让对端重试 ret = OB_ERR_WAIT_REMOTE_SCHEMA_REFRESH; @@ -190,7 +195,8 @@ int ObRemoteBaseExecuteP::base_before_process(int64_t tenant_schema_version, // overwrite ret to make sure sql will retry if (OB_NOT_NULL(session_info) && OB_ERR_WAIT_REMOTE_SCHEMA_REFRESH == ret - && GSCHEMASERVICE.is_schema_error_need_retry(NULL, tenant_id)) { + && GSCHEMASERVICE.is_schema_error_need_retry( + NULL, session_info->get_effective_tenant_id())) { ret = OB_ERR_REMOTE_SCHEMA_NOT_FULL; } } diff --git a/src/storage/backup/ob_backup_task.cpp b/src/storage/backup/ob_backup_task.cpp index a1c67e28b2..a65e9a4082 100644 --- a/src/storage/backup/ob_backup_task.cpp +++ b/src/storage/backup/ob_backup_task.cpp @@ -2109,26 +2109,23 @@ int ObPrefetchBackupInfoTask::inner_process_() } else if (OB_FAIL(task_mgr_->receive(task_id, need_copy_item_list))) { LOG_WARN("failed to receive items", K(ret), K(task_id), K(need_copy_item_list)); } else { - LOG_INFO( - "receive backup items", K(task_id), K_(backup_data_type), "count", need_copy_item_list.count(), K_(param)); + LOG_INFO("receive backup items", K(task_id), K_(backup_data_type), "need_copy_count", need_copy_item_list.count(), + "no_need_copy_count", no_need_copy_item_list.count(), K_(param)); } } if (OB_SUCC(ret)) { ObArray items; - const bool has_remain = task_mgr_->has_remain(); int64_t file_id = 0; if (OB_FAIL(task_mgr_->deliver(items, file_id))) { if (OB_EAGAIN == ret) { ret = OB_SUCCESS; - const int64_t pending = task_mgr_->get_pending_count(); - const int64_t ready = task_mgr_->get_ready_count(); - if (has_remain && !is_run_out) { + if (!is_run_out) { if (OB_FAIL(generate_next_prefetch_dag_())) { LOG_WARN("failed to generate prefetch dag", K(ret)); } else { LOG_INFO("generate next prefetch dag", K(items), K_(backup_data_type)); } - } else if (is_run_out) { + } else { LOG_INFO("run out", K_(param), K_(backup_data_type), K(items)); } } else { diff --git a/src/storage/compaction/ob_tablet_merge_ctx.cpp b/src/storage/compaction/ob_tablet_merge_ctx.cpp index fbd6f696da..ea764c22c4 100644 --- a/src/storage/compaction/ob_tablet_merge_ctx.cpp +++ b/src/storage/compaction/ob_tablet_merge_ctx.cpp @@ -24,7 +24,6 @@ #include "share/scheduler/ob_dag_warning_history_mgr.h" #include "storage/compaction/ob_medium_compaction_mgr.h" #include "storage/compaction/ob_medium_compaction_func.h" -#include "src/storage/meta_mem/ob_tenant_meta_mem_mgr.h" namespace oceanbase { @@ -1172,42 +1171,6 @@ int ObTabletMergeCtx::prepare_merge_progress() return ret; } -int ObTabletMergeCtx::try_swap_tablet_handle() -{ - int ret = OB_SUCCESS; - // check need swap tablet when compaction - if (OB_UNLIKELY(is_mini_merge(param_.merge_type_))) { - ret = OB_NOT_SUPPORTED; - LOG_WARN("mini merge not support swap tablet", K(ret), K_(param)); - } else { - int64_t row_count = 0; - int64_t macro_count = 0; - const ObSSTable *table = nullptr; - for (int64_t i = 0; i < tables_handle_.get_count(); ++i) { - table = static_cast(tables_handle_.get_table(i)); - row_count += table->get_meta().get_row_count(); - macro_count += table->get_meta().get_basic_meta().get_data_macro_block_count(); - } - if (row_count >= LARGE_VOLUME_DATA_ROW_COUNT_THREASHOLD - || macro_count >= LARGE_VOLUME_DATA_MACRO_COUNT_THREASHOLD) { - ObTabletHandle alloc_handle; - const ObTabletMapKey key(param_.ls_id_, param_.tablet_id_); - if (OB_FAIL(MTL(ObTenantMetaMemMgr*)->get_tablet_with_allocator( - WashTabletPriority::WTP_HIGH, key, allocator_, alloc_handle, true/*force_alloc_new*/))) { - LOG_WARN("failed to get alloc tablet handle", K(ret), K(key)); - } else { - tablet_handle_ = alloc_handle; - if (OB_FAIL(alloc_handle.get_obj()->clear_memtables_on_table_store())) { - LOG_WARN("failed to clear memtables on table_store", K(ret), K(param_)); - } else { - LOG_INFO("success to swap tablet handle", K(ret), K(macro_count), K(row_count), K(tablet_handle_.get_obj()->get_table_store())); - } - } - } - } - return ret; -} - int ObTabletMergeCtx::generate_participant_table_info(char *buf, const int64_t buf_len) const { int ret = OB_SUCCESS; diff --git a/src/storage/compaction/ob_tablet_merge_ctx.h b/src/storage/compaction/ob_tablet_merge_ctx.h index 4b45f49458..b1175a2859 100644 --- a/src/storage/compaction/ob_tablet_merge_ctx.h +++ b/src/storage/compaction/ob_tablet_merge_ctx.h @@ -171,8 +171,6 @@ struct ObTabletMergeCtx int get_schema_and_gene_from_result(const ObGetMergeTablesResult &get_merge_table_result); int get_storage_schema_and_gene_from_result(const ObGetMergeTablesResult &get_merge_table_result); int get_storage_schema_to_merge(const ObTablesHandleArray &merge_tables_handle, const bool get_schema_on_memtable = true); - - int try_swap_tablet_handle(); public: int get_medium_compaction_info_to_store(); @@ -199,8 +197,7 @@ public: } typedef common::ObSEArray MinorParallelResultArray; - static const int64_t LARGE_VOLUME_DATA_ROW_COUNT_THREASHOLD = 1000L * 1000L; // 100w - static const int64_t LARGE_VOLUME_DATA_MACRO_COUNT_THREASHOLD = 300L; + // 1. init in dag ObTabletMergeDagParam ¶m_; common::ObIAllocator &allocator_; diff --git a/src/storage/compaction/ob_tablet_merge_task.cpp b/src/storage/compaction/ob_tablet_merge_task.cpp index b5a4f15ee0..8ecb676f49 100644 --- a/src/storage/compaction/ob_tablet_merge_task.cpp +++ b/src/storage/compaction/ob_tablet_merge_task.cpp @@ -648,8 +648,6 @@ int ObTabletMergeExecutePrepareTask::process() LOG_WARN("fail to init merge info", K(ret), K_(result), KPC(ctx_)); } else if (OB_FAIL(ctx_->prepare_index_tree())) { LOG_WARN("fail to prepare sstable index tree", K(ret), KPC(ctx_)); - } else if (OB_FAIL(ctx_->try_swap_tablet_handle())) { - LOG_WARN("failed to try swap tablet handle", K(ret)); } else if (OB_FAIL(ObBasicTabletMergeDag::generate_merge_task( *static_cast(get_dag()), *ctx_, this))) { LOG_WARN("Failed to generate_merge_sstable_task", K(ret)); @@ -819,9 +817,6 @@ int ObTabletMergePrepareTask::process() } if (OB_FAIL(ret) || skip_rest_operation) { - } else if (!is_mini_merge(ctx->param_.merge_type_) - && OB_FAIL(ctx->try_swap_tablet_handle())) { - LOG_WARN("failed to try swap tablet handle", K(ret)); } else if (OB_FAIL(ObBasicTabletMergeDag::generate_merge_task( *merge_dag_, *ctx, this))) { LOG_WARN("Failed to generate_merge_sstable_task", K(ret)); diff --git a/src/storage/meta_mem/ob_meta_pointer_map.h b/src/storage/meta_mem/ob_meta_pointer_map.h index 09e4ed2bf3..ed70241572 100644 --- a/src/storage/meta_mem/ob_meta_pointer_map.h +++ b/src/storage/meta_mem/ob_meta_pointer_map.h @@ -36,8 +36,7 @@ public: int get_meta_obj_with_external_memory( const Key &key, common::ObIAllocator &allocator, - ObMetaObjGuard &guard, - const bool force_alloc_new = false); + ObMetaObjGuard &guard); int try_get_in_memory_meta_obj(const Key &key, bool &success, ObMetaObjGuard &guard); int try_get_in_memory_meta_obj_and_addr( const Key &key, @@ -518,8 +517,7 @@ template int ObMetaPointerMap::get_meta_obj_with_external_memory( const Key &key, common::ObIAllocator &allocator, - ObMetaObjGuard &guard, - const bool force_alloc_new) + ObMetaObjGuard &guard) { int ret = common::OB_SUCCESS; ObMetaPointerHandle ptr_hdl(*this); @@ -529,21 +527,13 @@ int ObMetaPointerMap::get_meta_obj_with_external_memory( if (OB_UNLIKELY(!key.is_valid())) { ret = common::OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "invalid argument", K(ret), K(key)); - } else if (force_alloc_new) { - common::ObBucketHashRLockGuard lock_guard(ResourceMap::bucket_lock_, ResourceMap::hash_func_(key)); - if (OB_FAIL(ResourceMap::get_without_lock(key, ptr_hdl))) { - if (common::OB_ENTRY_NOT_EXIST != ret) { - STORAGE_LOG(WARN, "fail to get pointer handle", K(ret)); - } - } } else if (OB_FAIL(try_get_in_memory_meta_obj(key, ptr_hdl, guard, is_in_memory))) { if (OB_ENTRY_NOT_EXIST == ret) { STORAGE_LOG(DEBUG, "meta obj does not exist", K(ret), K(key)); } else { STORAGE_LOG(WARN, "fail to try get in memory meta obj", K(ret), K(key)); } - } - if (OB_SUCC(ret) && !is_in_memory) { + } else if (!is_in_memory) { t_ptr = ptr_hdl.get_resource_ptr(); ObMetaDiskAddr disk_addr; void *buf = allocator.alloc(sizeof(T)); diff --git a/src/storage/meta_mem/ob_tenant_meta_mem_mgr.cpp b/src/storage/meta_mem/ob_tenant_meta_mem_mgr.cpp index ea3c45ef7f..e0765eff4b 100644 --- a/src/storage/meta_mem/ob_tenant_meta_mem_mgr.cpp +++ b/src/storage/meta_mem/ob_tenant_meta_mem_mgr.cpp @@ -945,8 +945,7 @@ int ObTenantMetaMemMgr::get_tablet_with_allocator( const WashTabletPriority &priority, const ObTabletMapKey &key, common::ObIAllocator &allocator, - ObTabletHandle &handle, - const bool force_alloc_new) + ObTabletHandle &handle) { int ret = OB_SUCCESS; handle.reset(); @@ -956,7 +955,7 @@ int ObTenantMetaMemMgr::get_tablet_with_allocator( } else if (OB_UNLIKELY(!key.is_valid() || is_used_obj_pool(&allocator))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(key), KP(&allocator), KP(&allocator_)); - } else if (OB_FAIL(tablet_map_.get_meta_obj_with_external_memory(key, allocator, handle, force_alloc_new))) { + } else if (OB_FAIL(tablet_map_.get_meta_obj_with_external_memory(key, allocator, handle))) { if (OB_ENTRY_NOT_EXIST != ret) { LOG_WARN("fail to get tablet", K(ret), K(key)); } diff --git a/src/storage/meta_mem/ob_tenant_meta_mem_mgr.h b/src/storage/meta_mem/ob_tenant_meta_mem_mgr.h index cb5640e605..9fbb46210f 100644 --- a/src/storage/meta_mem/ob_tenant_meta_mem_mgr.h +++ b/src/storage/meta_mem/ob_tenant_meta_mem_mgr.h @@ -161,8 +161,7 @@ public: const WashTabletPriority &priority, const ObTabletMapKey &key, common::ObIAllocator &allocator, - ObTabletHandle &handle, - const bool force_alloc_new = false); + ObTabletHandle &handle); int get_tablet_addr(const ObTabletMapKey &key, ObMetaDiskAddr &addr); int has_tablet(const ObTabletMapKey &key, bool &is_exist); int del_tablet(const ObTabletMapKey &key); diff --git a/src/storage/tablet/ob_tablet.cpp b/src/storage/tablet/ob_tablet.cpp index 5e4df2ebaa..69a2e2e7b2 100644 --- a/src/storage/tablet/ob_tablet.cpp +++ b/src/storage/tablet/ob_tablet.cpp @@ -3313,18 +3313,5 @@ int ObTablet::set_memtable_clog_checkpoint_scn( return ret; } - -int ObTablet::clear_memtables_on_table_store() // be careful to call this func -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(!is_inited_)) { - ret = OB_NOT_INIT; - LOG_WARN("not inited", K(ret), K_(is_inited)); - } else { - table_store_.clear_memtables(); - } - return ret; -} - } // namespace storage } // namespace oceanbase diff --git a/src/storage/tablet/ob_tablet.h b/src/storage/tablet/ob_tablet.h index 710f7fc789..4912b781f4 100644 --- a/src/storage/tablet/ob_tablet.h +++ b/src/storage/tablet/ob_tablet.h @@ -388,7 +388,6 @@ public: int set_redefined_schema_version_in_tablet_pointer(const int64_t schema_version); int set_memtable_clog_checkpoint_scn( const ObMigrationTabletParam *tablet_meta); - int clear_memtables_on_table_store(); // be careful to call this func, will destroy memtables array on table_store TO_STRING_KV(KP(this), K_(wash_score), K_(ref_cnt), K_(tablet_meta), K_(table_store), K_(storage_schema), K_(medium_info_list)); private: diff --git a/src/storage/tablet/ob_tablet_table_store.cpp b/src/storage/tablet/ob_tablet_table_store.cpp index 4064322bf2..17eb722239 100644 --- a/src/storage/tablet/ob_tablet_table_store.cpp +++ b/src/storage/tablet/ob_tablet_table_store.cpp @@ -431,19 +431,6 @@ int ObTabletTableStore::update_memtables() return ret; } -int ObTabletTableStore::clear_memtables() -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(!is_valid())) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("table store is unexpected invalid", K(ret), KPC(this)); - } else { - memtables_.destroy(); - read_cache_.reset(); - } - return ret; -} - int ObTabletTableStore::init_read_cache() { int ret = OB_SUCCESS; @@ -1213,7 +1200,7 @@ int64_t ObTabletTableStore::to_string(char *buf, const int64_t buf_len) const J_OBJ_START(); J_NAME("ObTabletTableStore"); J_COLON(); - J_KV(KP(this), KP_(tablet_ptr), K_(major_tables), K_(minor_tables), K_(memtables), K_(is_ready_for_read)); + J_KV(KP(this), KP_(tablet_ptr), K_(major_tables), K_(minor_tables), K_(is_ready_for_read)); J_COMMA(); J_ARRAY_START(); for (int64_t i = 0; i < major_tables_.count_; ++i) { diff --git a/src/storage/tablet/ob_tablet_table_store.h b/src/storage/tablet/ob_tablet_table_store.h index ee168ae8ff..13e1e8544b 100644 --- a/src/storage/tablet/ob_tablet_table_store.h +++ b/src/storage/tablet/ob_tablet_table_store.h @@ -98,7 +98,6 @@ public: int get_memtables(common::ObIArray &memtables, const bool need_active = false) const; int prepare_memtables(); int update_memtables(); - int clear_memtables(); int get_first_frozen_memtable(ObITable *&table); int get_ddl_sstable_handles(ObTablesHandleArray &ddl_sstable_handles) const;