[CP] Clear tablet autoinc seq cache when drop table

This commit is contained in:
Hongqin-Li
2024-03-05 03:19:34 +00:00
committed by ob-robot
parent d87ab32ed3
commit 5cb186e4a7
10 changed files with 253 additions and 6 deletions

View File

@ -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)

View File

@ -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()
{

View File

@ -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);

View File

@ -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_);

View File

@ -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<uint64_t> 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;
}

View File

@ -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<obrpc::OB_CLEAR_TABLET_AUTOINC_SEQ_CACHE>::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);

View File

@ -3998,6 +3998,8 @@ private:
DISALLOW_COPY_AND_ASSIGN(ObFetchTabletSeqRes);
};
using ObClearTabletAutoincSeqCacheArg = ObBatchRemoveTabletArg;
struct ObGetMinSSTableSchemaVersionRes
{
OB_UNIS_VERSION(1);

View File

@ -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));

View File

@ -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<common::ObTabletID> &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<ObTabletID> 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<const ObSimpleTableSchemaV2 *> 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<common::ObAddr, 8> 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;
}
}
}

View File

@ -104,6 +104,35 @@ private:
bool is_inited_;
};
class ObTabletAutoincMgrAllocHandle
{
public:
typedef LinkHashNode<ObTabletAutoincKey> 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<ObTabletID> 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<common::ObTabletID> &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<ObTabletAutoincKey, ObTabletAutoincMgr> TabletAutoincMgrMap;
typedef common::ObLinkHashMap<ObTabletAutoincKey, ObTabletAutoincMgr, ObTabletAutoincMgrAllocHandle> TabletAutoincMgrMap;
const static int INIT_NODE_MUTEX_NUM = 10243L;
bool is_inited_;
common::ObSmallAllocator node_allocator_;