diff --git a/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h b/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h index 882ae0b993..85704d9a96 100644 --- a/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h +++ b/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h @@ -838,7 +838,7 @@ PCODE_DEF(OB_CHECK_MEDIUM_INFO_LIST_CNT, 0x959) PCODE_DEF(OB_SPLIT_TABLET_DATA_FINISH_REQUEST, 0x960) PCODE_DEF(OB_MVIEW_COMPLETE_REFRESH, 0x961) -//PCODE_DEF(OB_CLEAR_TABLET_AUTOINC_SEQ_CACHE, 0x962) +PCODE_DEF(OB_CLEAR_TABLET_AUTOINC_SEQ_CACHE, 0x962) //PCODE_DEF(OB_FREEZE_SPLIT_SRC_TABLET, 0x963) PCODE_DEF(OB_CHECK_AND_CANCEL_DDL_COMPLEMENT_DAG, 0x964) //PCODE_DEF(OB_CLEAN_SPLITTED_TABLET, 0x965) diff --git a/src/observer/ob_rpc_processor_simple.cpp b/src/observer/ob_rpc_processor_simple.cpp index c6ec63b114..c6959fb5e3 100644 --- a/src/observer/ob_rpc_processor_simple.cpp +++ b/src/observer/ob_rpc_processor_simple.cpp @@ -2217,6 +2217,19 @@ int ObRpcBatchSetTabletAutoincSeqP::process() return ret; } +int ObRpcClearTabletAutoincSeqCacheP::process() +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(rpc_pkt_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid rpc pkt", K(ret)); + } else { + const int64_t abs_timeout_us = get_send_timestamp() + rpc_pkt_->get_timeout(); + ret = ObTabletAutoincrementService::get_instance().clear_tablet_autoinc_seq_cache(MTL_ID(), arg_.tablet_ids_, abs_timeout_us); + } + return ret; +} + #ifdef OB_BUILD_TDE_SECURITY int ObDumpTenantCacheMasterKeyP::process() { diff --git a/src/observer/ob_rpc_processor_simple.h b/src/observer/ob_rpc_processor_simple.h index 269a08390b..022db9957d 100644 --- a/src/observer/ob_rpc_processor_simple.h +++ b/src/observer/ob_rpc_processor_simple.h @@ -122,6 +122,7 @@ OB_DEFINE_PROCESSOR_S(Srv, OB_DDL_BUILD_SINGLE_REPLICA_REQUEST, ObRpcBuildDDLSin OB_DEFINE_PROCESSOR_S(Srv, OB_FETCH_TABLET_AUTOINC_SEQ_CACHE, ObRpcFetchTabletAutoincSeqCacheP); OB_DEFINE_PROCESSOR_S(Srv, OB_BATCH_GET_TABLET_AUTOINC_SEQ, ObRpcBatchGetTabletAutoincSeqP); OB_DEFINE_PROCESSOR_S(Srv, OB_BATCH_SET_TABLET_AUTOINC_SEQ, ObRpcBatchSetTabletAutoincSeqP); +OB_DEFINE_PROCESSOR_S(Srv, OB_CLEAR_TABLET_AUTOINC_SEQ_CACHE, ObRpcClearTabletAutoincSeqCacheP); OB_DEFINE_PROCESSOR_S(Srv, OB_CHECK_MODIFY_TIME_ELAPSED, ObRpcCheckCtxCreateTimestampElapsedP); OB_DEFINE_PROCESSOR_S(Srv, OB_UPDATE_BASELINE_SCHEMA_VERSION, ObRpcUpdateBaselineSchemaVersionP); OB_DEFINE_PROCESSOR_S(Srv, OB_SWITCH_LEADER, ObRpcSwitchLeaderP); diff --git a/src/observer/ob_srv_xlator_storage.cpp b/src/observer/ob_srv_xlator_storage.cpp index 77a4362152..5657254040 100644 --- a/src/observer/ob_srv_xlator_storage.cpp +++ b/src/observer/ob_srv_xlator_storage.cpp @@ -101,6 +101,7 @@ void oceanbase::observer::init_srv_xlator_for_storage(ObSrvRpcXlator *xlator) { RPC_PROCESSOR(ObRpcFetchTabletAutoincSeqCacheP, gctx_); RPC_PROCESSOR(ObRpcBatchGetTabletAutoincSeqP, gctx_); RPC_PROCESSOR(ObRpcBatchSetTabletAutoincSeqP, gctx_); + RPC_PROCESSOR(ObRpcClearTabletAutoincSeqCacheP, gctx_); RPC_PROCESSOR(ObRpcRemoteWriteDDLRedoLogP, gctx_); RPC_PROCESSOR(ObRpcRemoteWriteDDLCommitLogP, gctx_); RPC_PROCESSOR(ObRpcLSMigrateReplicaP, gctx_); diff --git a/src/rootserver/ob_ddl_service.cpp b/src/rootserver/ob_ddl_service.cpp index 85da285027..0658921e4e 100755 --- a/src/rootserver/ob_ddl_service.cpp +++ b/src/rootserver/ob_ddl_service.cpp @@ -38,6 +38,7 @@ #include "share/schema/ob_user_sql_service.h" #include "share/schema/ob_schema_service_sql_impl.h" #include "share/ob_autoincrement_service.h" +#include "share/ob_tablet_autoincrement_service.h" #include "share/config/ob_server_config.h" #include "share/ob_primary_zone_util.h" #include "share/ob_replica_info.h" @@ -22870,6 +22871,7 @@ int ObDDLService::drop_table(const ObDropTableArg &drop_table_arg, const obrpc:: schema_guard.set_session_id(drop_table_arg.session_id_); uint64_t tenant_id = drop_table_arg.tenant_id_; ObSchemaService *schema_service = schema_service_->get_schema_service(); + ObTabletAutoincCacheCleaner tablet_autoinc_cleaner(tenant_id); uint64_t compat_version = OB_INVALID_VERSION; if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) { @@ -23142,6 +23144,14 @@ int ObDDLService::drop_table(const ObDropTableArg &drop_table_arg, const obrpc:: } } } + + if (OB_SUCC(ret)) { + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(tablet_autoinc_cleaner.add_table(schema_guard, *table_schema))) { + LOG_WARN("failed to add table to tablet autoinc cleaner", K(tmp_ret)); + } + } + LOG_INFO("finish drop table", K(tenant_id), K(table_item), K(ret)); if (OB_ERR_TABLE_IS_REFERENCED == ret) { fail_for_fk_cons = true; @@ -23237,6 +23247,12 @@ int ObDDLService::drop_table(const ObDropTableArg &drop_table_arg, const obrpc:: ret = tmp_ret; LOG_WARN("publish schema failed", K(ret)); } + + if (OB_SUCC(ret)) { + if (OB_TMP_FAIL(tablet_autoinc_cleaner.commit())) { + LOG_WARN("failed to commit tablet autoinc cleaner", K(tmp_ret)); + } + } } return ret; } @@ -28195,6 +28211,7 @@ int ObDDLService::drop_database(const ObDropDatabaseArg &arg, ObArray table_ids; ObSchemaGetterGuard schema_guard; const ObDatabaseSchema *db_schema = NULL; + ObTabletAutoincCacheCleaner tablet_autoinc_cleaner(tenant_id); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { @@ -28261,6 +28278,12 @@ int ObDDLService::drop_database(const ObDropDatabaseArg &arg, actual_trans, tenant_id, arg.task_id_))) { LOG_WARN("update ddl task status to success failed", K(ret)); } + if (OB_SUCC(ret)) { + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(tablet_autoinc_cleaner.add_database(*db_schema))) { + LOG_WARN("failed to add database to tablet autoinc cleaner", K(tmp_ret)); + } + } } } @@ -28282,6 +28305,13 @@ int ObDDLService::drop_database(const ObDropDatabaseArg &arg, } } + if (OB_SUCC(ret)) { + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(tablet_autoinc_cleaner.commit())) { + LOG_WARN("failed to commit tablet autoinc cleaner", K(tmp_ret)); + } + } + LOG_INFO("finish drop database", K(tenant_id), K(arg), K(ret)); return ret; } diff --git a/src/rootserver/ob_rs_async_rpc_proxy.h b/src/rootserver/ob_rs_async_rpc_proxy.h index d1cc961e42..7e2197f8da 100644 --- a/src/rootserver/ob_rs_async_rpc_proxy.h +++ b/src/rootserver/ob_rs_async_rpc_proxy.h @@ -62,6 +62,8 @@ RPC_F(obrpc::OB_BATCH_GET_TABLET_AUTOINC_SEQ, obrpc::ObBatchGetTabletAutoincSeqA obrpc::ObBatchGetTabletAutoincSeqRes, ObBatchGetTabletAutoincSeqProxy); RPC_F(obrpc::OB_BATCH_SET_TABLET_AUTOINC_SEQ, obrpc::ObBatchSetTabletAutoincSeqArg, obrpc::ObBatchSetTabletAutoincSeqRes, ObBatchSetTabletAutoincSeqProxy); +RPC_F(obrpc::OB_CLEAR_TABLET_AUTOINC_SEQ_CACHE, obrpc::ObClearTabletAutoincSeqCacheArg, + obrpc::ObSrvRpcProxy::ObRpc::Response, ObClearTabletAutoincSeqCacheProxy); RPC_F(obrpc::OB_GET_LS_SYNC_SCN, obrpc::ObGetLSSyncScnArg, obrpc::ObGetLSSyncScnRes, ObGetLSSyncScnProxy); RPC_F(obrpc::OB_INIT_TENANT_CONFIG, obrpc::ObInitTenantConfigArg, obrpc::ObInitTenantConfigRes, ObInitTenantConfigProxy); diff --git a/src/share/ob_rpc_struct.h b/src/share/ob_rpc_struct.h index 376cf84924..50b295e054 100755 --- a/src/share/ob_rpc_struct.h +++ b/src/share/ob_rpc_struct.h @@ -3998,6 +3998,8 @@ private: DISALLOW_COPY_AND_ASSIGN(ObFetchTabletSeqRes); }; +using ObClearTabletAutoincSeqCacheArg = ObBatchRemoveTabletArg; + struct ObGetMinSSTableSchemaVersionRes { OB_UNIS_VERSION(1); diff --git a/src/share/ob_srv_rpc_proxy.h b/src/share/ob_srv_rpc_proxy.h index 54d680af53..ca2ade00c3 100755 --- a/src/share/ob_srv_rpc_proxy.h +++ b/src/share/ob_srv_rpc_proxy.h @@ -145,6 +145,7 @@ public: RPC_S(PR5 fetch_tablet_autoinc_seq_cache, OB_FETCH_TABLET_AUTOINC_SEQ_CACHE, (obrpc::ObFetchTabletSeqArg), obrpc::ObFetchTabletSeqRes); RPC_AP(PR5 batch_get_tablet_autoinc_seq, OB_BATCH_GET_TABLET_AUTOINC_SEQ, (obrpc::ObBatchGetTabletAutoincSeqArg), obrpc::ObBatchGetTabletAutoincSeqRes); RPC_AP(PR5 batch_set_tablet_autoinc_seq, OB_BATCH_SET_TABLET_AUTOINC_SEQ, (obrpc::ObBatchSetTabletAutoincSeqArg), obrpc::ObBatchSetTabletAutoincSeqRes); + RPC_AP(PR5 clear_tablet_autoinc_seq_cache, OB_CLEAR_TABLET_AUTOINC_SEQ_CACHE, (obrpc::ObClearTabletAutoincSeqCacheArg), obrpc::Int64); RPC_S(PRD force_create_sys_table, OB_FORCE_CREATE_SYS_TABLE, (ObForceCreateSysTableArg)); RPC_S(PRD schema_revise, OB_SCHEMA_REVISE, (ObSchemaReviseArg)); RPC_S(PRD force_set_locality, OB_FORCE_SET_LOCALITY, (ObForceSetLocalityArg)); diff --git a/src/share/ob_tablet_autoincrement_service.cpp b/src/share/ob_tablet_autoincrement_service.cpp index 4169ac9dee..d561dcc2c1 100644 --- a/src/share/ob_tablet_autoincrement_service.cpp +++ b/src/share/ob_tablet_autoincrement_service.cpp @@ -227,8 +227,7 @@ int ObTabletAutoincrementService::acquire_mgr( lib::ObMutex &mutex = init_node_mutexs_[key.tablet_id_.id() % INIT_NODE_MUTEX_NUM]; lib::ObMutexGuard guard(mutex); if (OB_ENTRY_NOT_EXIST == (ret = tablet_autoinc_mgr_map_.get(key, autoinc_mgr))) { - if (NULL == (autoinc_mgr = op_alloc(ObTabletAutoincMgr))) { - ret = OB_ALLOCATE_MEMORY_FAILED; + if (OB_FAIL(tablet_autoinc_mgr_map_.alloc_value(autoinc_mgr))) { LOG_WARN("failed to alloc table mgr", K(ret)); } else if (OB_FAIL(autoinc_mgr->init(key.tablet_id_, init_cache_size))) { LOG_WARN("fail to init tablet autoinc mgr", K(ret), K(key)); @@ -236,7 +235,7 @@ int ObTabletAutoincrementService::acquire_mgr( LOG_WARN("failed to create table node", K(ret)); } if (OB_FAIL(ret) && autoinc_mgr != nullptr) { - op_free(autoinc_mgr); + tablet_autoinc_mgr_map_.free_value(autoinc_mgr); autoinc_mgr = nullptr; } } @@ -341,5 +340,174 @@ int ObTabletAutoincrementService::get_tablet_cache_interval(const uint64_t tenan return ret; } + +int ObTabletAutoincrementService::clear_tablet_autoinc_seq_cache( + const uint64_t tenant_id, + const common::ObIArray &tablet_ids, + const int64_t abs_timeout_us) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("tablet auto increment service is not inited", K(ret)); + } + for (int64_t i = 0; OB_SUCC(ret) && i < tablet_ids.count(); i++) { + ObTabletAutoincKey key; + key.tenant_id_ = tenant_id; + key.tablet_id_ = tablet_ids.at(i); + lib::ObMutex &mutex = init_node_mutexs_[key.tablet_id_.id() % INIT_NODE_MUTEX_NUM]; + lib::ObMutexGuardWithTimeout guard(mutex, abs_timeout_us); + if (OB_FAIL(guard.get_ret())) { + LOG_WARN("failed to lock", K(ret)); + } else if (OB_FAIL(tablet_autoinc_mgr_map_.del(key))) { + if (OB_ENTRY_NOT_EXIST == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to del tablet autoinc", K(ret), K(key)); + } + } + } + return ret; +} + +int ObTabletAutoincCacheCleaner::add_single_table(const schema::ObSimpleTableSchemaV2 &table_schema) +{ + int ret = OB_SUCCESS; + if (table_schema.is_heap_table() || table_schema.is_aux_lob_meta_table()) { + const uint64_t tenant_id = table_schema.get_tenant_id(); + ObArray tablet_ids; + if (OB_UNLIKELY(tenant_id != tenant_id_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tenant id mismatch", K(ret), K(tenant_id), K(tenant_id_)); + } else if (OB_FAIL(table_schema.get_tablet_ids(tablet_ids))) { + LOG_WARN("failed to get tablet ids", K(ret)); + } else if (OB_FAIL(append(tablet_ids_, tablet_ids))) { + LOG_WARN("failed to append tablet ids", K(ret)); + } + } + return ret; +} + +// add user table and its related tables that use tablet autoinc, e.g., lob meta table +int ObTabletAutoincCacheCleaner::add_table(schema::ObSchemaGetterGuard &schema_guard, const schema::ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(add_single_table(table_schema))) { + LOG_WARN("failed to add single table", K(ret)); + } + + if (OB_SUCC(ret)) { + const uint64_t lob_meta_tid = table_schema.get_aux_lob_meta_tid(); + if (OB_INVALID_ID != lob_meta_tid) { + const ObTableSchema *lob_meta_table_schema = nullptr; + if (OB_FAIL(schema_guard.get_table_schema(table_schema.get_tenant_id(), lob_meta_tid, lob_meta_table_schema))) { + LOG_WARN("failed to get aux table schema", K(ret), K(lob_meta_tid)); + } else if (OB_ISNULL(lob_meta_table_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid table schema", K(ret), K(lob_meta_tid)); + } else if (OB_FAIL(add_single_table(*lob_meta_table_schema))) { + LOG_WARN("failed to add single table", K(ret)); + } + } + } + return ret; +} + +int ObTabletAutoincCacheCleaner::add_database(const schema::ObDatabaseSchema &database_schema) +{ + int ret = OB_SUCCESS; + ObSchemaGetterGuard schema_guard; + const uint64_t tenant_id = database_schema.get_tenant_id(); + const uint64_t database_id = database_schema.get_database_id(); + ObArray table_schemas; + ObMultiVersionSchemaService &schema_service = ObMultiVersionSchemaService::get_instance(); + if (OB_FAIL(schema_service.get_tenant_schema_guard(tenant_id, schema_guard))) { + LOG_WARN("fail to get schema guard", KR(ret), K(tenant_id)); + } else if (OB_FAIL(schema_guard.get_table_schemas_in_database(tenant_id, + database_id, + table_schemas))) { + LOG_WARN("fail to get table ids in database", K(tenant_id), K(database_id), K(ret)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) { + const ObSimpleTableSchemaV2 *table_schema = table_schemas.at(i); + if (OB_ISNULL(table_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("table schema should not be null", K(ret)); + } else if (OB_FAIL(add_single_table(*table_schema))) { + LOG_WARN("fail to lock_table", KR(ret), KPC(table_schema)); + } + } + } + return ret; +} + +int ObTabletAutoincCacheCleaner::commit(const int64_t timeout_us) +{ + int ret = OB_SUCCESS; + ObTimeGuard time_guard("ObTabletAutoincCacheCleaner", 1 * 1000 * 1000); + ObTabletAutoincrementService &tablet_autoinc_service = share::ObTabletAutoincrementService::get_instance(); + uint64_t data_version = 0; + common::ObZone zone; + common::ObSEArray server_list; + ObUnitInfoGetter ui_getter; + obrpc::ObClearTabletAutoincSeqCacheArg arg; + const ObLSID unused_ls_id = SYS_LS; + int64_t abs_timeout_us = ObTimeUtility::current_time() + timeout_us; + const ObTimeoutCtx &ctx = ObTimeoutCtx::get_ctx(); + if (THIS_WORKER.is_timeout_ts_valid()) { + abs_timeout_us = std::min(abs_timeout_us, THIS_WORKER.get_timeout_ts()); + } + if (ctx.is_timeout_set()) { + abs_timeout_us = std::min(abs_timeout_us, ctx.get_abs_timeout()); + } + if (ctx.is_trx_timeout_set()) { + abs_timeout_us = std::min(abs_timeout_us, ObTimeUtility::current_time() + ctx.get_trx_timeout_us()); + } + + const int64_t min_cluster_version = GET_MIN_CLUSTER_VERSION(); + if ((min_cluster_version >= MOCK_CLUSTER_VERSION_4_2_3_0 && min_cluster_version < CLUSTER_VERSION_4_3_0_0) + || min_cluster_version >= CLUSTER_VERSION_4_3_0_1) { + if (OB_ISNULL(GCTX.srv_rpc_proxy_) || OB_ISNULL(GCTX.sql_proxy_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("srv_rpc_proxy or sql_proxy in GCTX is null", K(ret), K(GCTX.srv_rpc_proxy_), K(GCTX.sql_proxy_)); + } else if (OB_FAIL(ui_getter.init(*GCTX.sql_proxy_, &GCONF))) { + LOG_WARN("init unit info getter failed", K(ret)); + } else if (OB_FAIL(ui_getter.get_tenant_servers(tenant_id_, server_list))) { + LOG_WARN("get tenant servers failed", K(ret)); + } else if (OB_FAIL(arg.init(tablet_ids_, unused_ls_id))) { + LOG_WARN("failed to init clear tablet autoinc arg", K(ret)); + } else { + bool need_clean_self = false; + rootserver::ObClearTabletAutoincSeqCacheProxy proxy(*GCTX.srv_rpc_proxy_, &obrpc::ObSrvRpcProxy::clear_tablet_autoinc_seq_cache); + for (int64_t i = 0; i < server_list.count(); i++) { // overwrite ret + const common::ObAddr &addr = server_list.at(i); + const int64_t cur_timeout_us = abs_timeout_us - ObTimeUtility::current_time(); + if (cur_timeout_us <= 0) { + ret = OB_TIMEOUT; + break; + } else if (GCTX.self_addr() == addr) { + need_clean_self = true; + } else { + if (OB_FAIL(proxy.call(addr, cur_timeout_us, tenant_id_, arg))) { + LOG_WARN("failed to send rpc call", K(ret), K(addr), K(timeout_us), K(tenant_id_)); + } + } + } + if (need_clean_self) { // overwrite ret + if (OB_FAIL(tablet_autoinc_service.clear_tablet_autoinc_seq_cache(tenant_id_, tablet_ids_, abs_timeout_us))) { + LOG_WARN("failed to clear tablet autoinc", K(ret)); + } + } + + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(proxy.wait())) { + LOG_WARN("wait batch result failed", K(tmp_ret), K(ret)); + ret = OB_SUCC(ret) ? tmp_ret : ret; + } + } + } + return ret; +} + } } diff --git a/src/share/ob_tablet_autoincrement_service.h b/src/share/ob_tablet_autoincrement_service.h index 3473627104..6655f0f5bb 100644 --- a/src/share/ob_tablet_autoincrement_service.h +++ b/src/share/ob_tablet_autoincrement_service.h @@ -104,6 +104,35 @@ private: bool is_inited_; }; +class ObTabletAutoincMgrAllocHandle +{ +public: + typedef LinkHashNode TabletAutoincNode; + typedef ObTabletAutoincMgr TabletAutoincMgr; + static ObTabletAutoincMgr* alloc_value() { return op_reclaim_alloc(TabletAutoincMgr); } + static void free_value(ObTabletAutoincMgr* val) { op_reclaim_free(val); val = nullptr; } + static TabletAutoincNode* alloc_node(ObTabletAutoincMgr* val) { UNUSED(val); return op_reclaim_alloc(TabletAutoincNode); } + static void free_node(TabletAutoincNode* node) { op_reclaim_free(node); node = nullptr; } +}; + +class ObTabletAutoincCacheCleaner final +{ +public: + static const int64_t DEFAULT_TIMEOUT_US = 1 * 1000 * 1000; + ObTabletAutoincCacheCleaner(const uint64_t tenant_id) : tenant_id_(tenant_id), tablet_ids_() {} + ~ObTabletAutoincCacheCleaner() {} + int add_table(schema::ObSchemaGetterGuard &schema_guard, const schema::ObTableSchema &table_schema); + int add_database(const schema::ObDatabaseSchema &database_schema); + int commit(const int64_t timeout_us = DEFAULT_TIMEOUT_US); + TO_STRING_KV(K_(tenant_id), K_(tablet_ids)); +private: + int add_single_table(const schema::ObSimpleTableSchemaV2 &table_schema); +private: + DISALLOW_COPY_AND_ASSIGN(ObTabletAutoincCacheCleaner); + uint64_t tenant_id_; + ObArray tablet_ids_; +}; + class ObTabletAutoincrementService { public: @@ -113,7 +142,7 @@ public: int get_tablet_cache_interval(const uint64_t tenant_id, ObTabletCacheInterval &interval); int get_autoinc_seq(const uint64_t tenant_id, const common::ObTabletID &tablet_id, uint64_t &autoinc_seq); - int clear_tablet_autoinc_cache(const uint64_t tenant_id, const common::ObTabletID &tablet_id); + int clear_tablet_autoinc_seq_cache(const uint64_t tenant_id, const common::ObIArray &tablet_ids, const int64_t abs_timeout_us); private: int acquire_mgr(const uint64_t tenant_id, const common::ObTabletID &tablet_id, const int64_t init_cache_size, ObTabletAutoincMgr *&autoinc_mgr); void release_mgr(ObTabletAutoincMgr *autoinc_mgr); @@ -122,7 +151,7 @@ private: ~ObTabletAutoincrementService(); private: - typedef common::ObLinkHashMap TabletAutoincMgrMap; + typedef common::ObLinkHashMap TabletAutoincMgrMap; const static int INIT_NODE_MUTEX_NUM = 10243L; bool is_inited_; common::ObSmallAllocator node_allocator_;