change ddl get tablet to use get_tablet_with_timeout
This commit is contained in:
		 YoungYang0820
					YoungYang0820
				
			
				
					committed by
					
						 wangzelin.wzl
						wangzelin.wzl
					
				
			
			
				
	
			
			
			 wangzelin.wzl
						wangzelin.wzl
					
				
			
						parent
						
							a937303a86
						
					
				
				
					commit
					042a195f8b
				
			| @ -24,6 +24,9 @@ | ||||
| #include "share/location_cache/ob_location_service.h" | ||||
| #include "sql/engine/ob_physical_plan.h" | ||||
| #include "sql/engine/table/ob_table_scan_op.h" | ||||
| #include "storage/tablet/ob_tablet.h" | ||||
| #include "storage/tx_storage/ob_ls_handle.h" | ||||
| #include "storage/tx_storage/ob_ls_map.h" | ||||
|  | ||||
| using namespace oceanbase::share; | ||||
| using namespace oceanbase::common; | ||||
| @ -743,3 +746,28 @@ int ObDDLUtil::find_table_scan_table_id(const ObOpSpec *spec, uint64_t &table_id | ||||
|   } | ||||
|   return ret; | ||||
| } | ||||
|  | ||||
| int ObDDLUtil::ddl_get_tablet( | ||||
|     ObLSHandle &ls_handle, | ||||
|     const ObTabletID &tablet_id, | ||||
|     storage::ObTabletHandle &tablet_handle, | ||||
|     const int64_t get_timeout_ts) | ||||
| { | ||||
|   int ret = OB_SUCCESS; | ||||
|   ObLS *ls = nullptr; | ||||
|   const int64_t DDL_GET_TABLET_RETRY_TIMEOUT = 30 * 1000 * 1000; // 30s | ||||
|   const int64_t timeout_ts = ObTimeUtility::current_time() + DDL_GET_TABLET_RETRY_TIMEOUT; | ||||
|   if (OB_ISNULL(ls = ls_handle.get_ls())) { | ||||
|     ret = OB_ERR_UNEXPECTED; | ||||
|     LOG_ERROR("ls should not be null", K(ret)); | ||||
|   } else if (OB_FAIL(ls->get_tablet_svr()->get_tablet_with_timeout(tablet_id, | ||||
|                                                                    tablet_handle, | ||||
|                                                                    timeout_ts, | ||||
|                                                                    get_timeout_ts))) { | ||||
|     LOG_WARN("fail to get tablet handle", K(ret), K(tablet_id)); | ||||
|     if (OB_ALLOCATE_MEMORY_FAILED == ret) { | ||||
|       ret = OB_TIMEOUT; | ||||
|     } | ||||
|   } | ||||
|   return ret; | ||||
| } | ||||
|  | ||||
| @ -16,6 +16,7 @@ | ||||
| #include "lib/allocator/page_arena.h" | ||||
| #include "share/schema/ob_table_schema.h" | ||||
| #include "share/schema/ob_schema_service.h" | ||||
| #include "storage/tablet/ob_tablet_common.h" | ||||
|  | ||||
| namespace oceanbase | ||||
| { | ||||
| @ -29,6 +30,11 @@ namespace sql | ||||
| class ObPhysicalPlan; | ||||
| class ObOpSpec; | ||||
| } | ||||
| namespace storage | ||||
| { | ||||
| class ObTabletHandle; | ||||
| class ObLSHandle; | ||||
| } | ||||
| namespace share | ||||
| { | ||||
| class ObLocationService; | ||||
| @ -255,6 +261,12 @@ public: | ||||
|       const bool is_oracle_mode, | ||||
|       ObSqlString &sql_string); | ||||
|  | ||||
|   static int ddl_get_tablet( | ||||
|       storage::ObLSHandle &ls_handle, | ||||
|       const ObTabletID &tablet_id, | ||||
|       storage::ObTabletHandle &tablet_handle, | ||||
|       const int64_t timeout_us = storage::ObTabletCommon::DEFAULT_GET_TABLET_TIMEOUT_US); | ||||
|  | ||||
|   static int clear_ddl_checksum(sql::ObPhysicalPlan *phy_plan); | ||||
|    | ||||
|   static bool is_table_lock_retry_ret_code(int ret) | ||||
|  | ||||
| @ -226,9 +226,9 @@ int ObExprToOutfileRow::extract_fisrt_wchar_from_varhcar(const ObObj &obj, int32 | ||||
| // 4. ASCII 0 (what is actually written following the escape character is ASCII 0, not a | ||||
| //    zero-valued byte). | ||||
| // 5. If the FIELDS ESCAPED BY character is empty, no characters are escaped and NULL is output | ||||
| //    as NULL, not \N. | ||||
| int ObExprToOutfileRow::print_field(char *buf, const int64_t buf_len, int64_t &pos, | ||||
|                                     const ObObj &obj, ObExprOutFileInfo &out_info) | ||||
| //    as NULL, not \N. | ||||
| int ObExprToOutfileRow::print_field(char *buf, const int64_t buf_len, int64_t &pos, | ||||
|                                     const ObObj &obj, ObExprOutFileInfo &out_info) | ||||
| { | ||||
|   int ret = OB_SUCCESS; | ||||
|   int64_t tmp_pos = 0; | ||||
|  | ||||
| @ -513,10 +513,7 @@ int ObUniqueIndexChecker::check_unique_index(ObIDag *dag) | ||||
|       ObLSHandle ls_handle; | ||||
|       if (OB_FAIL(MTL(ObLSService *)->get_ls(ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { | ||||
|         LOG_WARN("fail to get log stream", K(ret), K(ls_id_)); | ||||
|       } else if (OB_UNLIKELY(nullptr == ls_handle.get_ls())) { | ||||
|         ret = OB_ERR_UNEXPECTED; | ||||
|         LOG_WARN("ls is null", K(ret), K(ls_handle)); | ||||
|       } else if (OB_FAIL(ls_handle.get_ls()->get_tablet_svr()->get_tablet(tablet_id_, tablet_handle_))) { | ||||
|       } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, tablet_id_, tablet_handle_))) { | ||||
|         LOG_WARN("fail to get tablet", K(ret), K(tablet_id_), K(tablet_handle_)); | ||||
|       } else if (index_schema_->is_domain_index()) { | ||||
|         STORAGE_LOG(INFO, "do not need to check unique for domain index", "index_id", index_schema_->get_table_id()); | ||||
|  | ||||
| @ -1183,7 +1183,6 @@ int ObComplementMergeTask::add_build_hidden_table_sstable() | ||||
| { | ||||
|   int ret = OB_SUCCESS; | ||||
|   ObLSHandle ls_handle; | ||||
|   ObLS *ls = nullptr; | ||||
|   ObTablet *tablet = nullptr; | ||||
|   ObTabletHandle tablet_handle; | ||||
|   ObITable::TableKey hidden_table_key; | ||||
| @ -1198,11 +1197,7 @@ int ObComplementMergeTask::add_build_hidden_table_sstable() | ||||
|     LOG_WARN("error unexpected", K(ret), KP(param_), KP(context_)); | ||||
|   } else if (OB_FAIL(MTL(ObLSService *)->get_ls(param_->ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { | ||||
|     LOG_WARN("failed to get log stream", K(ret), K(param_->ls_id_)); | ||||
|   } else if (OB_ISNULL(ls = ls_handle.get_ls())) { | ||||
|     ret = OB_ERR_UNEXPECTED; | ||||
|     LOG_WARN("ls is unexpected null", K(ret)); | ||||
|   } else if (OB_FAIL(ls->get_tablet_svr()->get_tablet(param_->dest_tablet_id_, | ||||
|                                                       tablet_handle))) { | ||||
|   } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, param_->dest_tablet_id_, tablet_handle))) { | ||||
|     LOG_WARN("failed to get tablet", K(ret), K(param_->ls_id_), K(param_->dest_tablet_id_)); | ||||
|   } else if (OB_ISNULL(tablet = tablet_handle.get_obj())) { | ||||
|     ret = OB_ERR_UNEXPECTED; | ||||
|  | ||||
| @ -22,6 +22,7 @@ namespace oceanbase | ||||
| { | ||||
|  | ||||
| using namespace blocksstable; | ||||
| using namespace share; | ||||
|  | ||||
| namespace storage | ||||
| { | ||||
| @ -123,10 +124,7 @@ int ObDDLMacroBlockClogCb::on_success() | ||||
|       LOG_INFO("data buffer is freed, do not need to callback"); | ||||
|     } else if (OB_FAIL(MTL(ObLSService *)->get_ls(ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { | ||||
|       LOG_WARN("get ls handle failed", K(ret), K(ls_id_)); | ||||
|     } else if (OB_ISNULL(ls_handle.get_ls())) { | ||||
|       ret = OB_ERR_UNEXPECTED; | ||||
|       LOG_WARN("ls is unexpected null", K(ret)); | ||||
|     } else if (OB_FAIL(ls_handle.get_ls()->get_tablet(redo_info_.table_key_.get_tablet_id(), tablet_handle))) { | ||||
|     } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, redo_info_.table_key_.get_tablet_id(), tablet_handle))) { | ||||
|       LOG_WARN("get tablet handle failed", K(ret), K(redo_info_.table_key_)); | ||||
|     } else if (OB_FAIL(macro_block.block_handle_.set_block_id(macro_block_id_))) { | ||||
|       LOG_WARN("set macro block id failed", K(ret), K(macro_block_id_)); | ||||
|  | ||||
| @ -57,7 +57,6 @@ int ObDDLTableMergeDag::init_by_param(const share::ObIDagInitParam *param) | ||||
| { | ||||
|   int ret = OB_SUCCESS; | ||||
|   ObLSHandle ls_handle; | ||||
|   ObLS *ls = nullptr; | ||||
|   ObTablet *tablet = nullptr; | ||||
|   ObTabletHandle tablet_handle; | ||||
|   if (OB_UNLIKELY(is_inited_)) { | ||||
| @ -70,12 +69,10 @@ int ObDDLTableMergeDag::init_by_param(const share::ObIDagInitParam *param) | ||||
|     ddl_param_ = *static_cast<const ObDDLTableMergeDagParam *>(param); | ||||
|     if (OB_FAIL(MTL(ObLSService *)->get_ls(ddl_param_.ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { | ||||
|       LOG_WARN("failed to get log stream", K(ret), K(ddl_param_)); | ||||
|     } else if (OB_ISNULL(ls = ls_handle.get_ls())) { | ||||
|       ret = OB_ERR_UNEXPECTED; | ||||
|       LOG_WARN("ls is unexpected null", K(ret)); | ||||
|     } else if (OB_FAIL(ls->get_tablet_svr()->get_tablet(ddl_param_.tablet_id_, | ||||
|                                                         tablet_handle, | ||||
|                                                         ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|     } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, | ||||
|                                                  ddl_param_.tablet_id_, | ||||
|                                                  tablet_handle, | ||||
|                                                  ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|       LOG_WARN("failed to get tablet", K(ret), K(ddl_param_)); | ||||
|     } else if (OB_ISNULL(tablet = tablet_handle.get_obj())) { | ||||
|       ret = OB_ERR_UNEXPECTED; | ||||
| @ -100,9 +97,10 @@ int ObDDLTableMergeDag::create_first_task() | ||||
|   ObDDLTableMergeTask *merge_task = nullptr; | ||||
|   if (OB_FAIL(ls_service->get_ls(ddl_param_.ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { | ||||
|     LOG_WARN("get ls failed", K(ret), K(ddl_param_)); | ||||
|   } else if (OB_FAIL(ls_handle.get_ls()->get_tablet(ddl_param_.tablet_id_, | ||||
|                                                     tablet_handle, | ||||
|                                                     ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|   } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, | ||||
|                                                ddl_param_.tablet_id_, | ||||
|                                                tablet_handle, | ||||
|                                                ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|     LOG_WARN("get tablet failed", K(ret), K(ddl_param_)); | ||||
|   } else if (OB_FAIL(tablet_handle.get_obj()->get_ddl_kv_mgr(ddl_kv_mgr_handle))) { | ||||
|     if (OB_ENTRY_NOT_EXIST == ret) { | ||||
| @ -247,9 +245,10 @@ int ObDDLTableDumpTask::process() | ||||
|     LOG_WARN("not init", K(ret)); | ||||
|   } else if (OB_FAIL(MTL(ObLSService *)->get_ls(ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { | ||||
|     LOG_WARN("failed to get log stream", K(ret), K(ls_id_)); | ||||
|   } else if (OB_FAIL(ls_handle.get_ls()->get_tablet(tablet_id_, | ||||
|                                                     tablet_handle, | ||||
|                                                     ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|   } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, | ||||
|                                                tablet_id_, | ||||
|                                                tablet_handle, | ||||
|                                                ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|     LOG_WARN("failed to get tablet", K(ret), K(tablet_id_)); | ||||
|   } else if (OB_FAIL(tablet_handle.get_obj()->get_ddl_kv_mgr(ddl_kv_mgr_handle))) { | ||||
|     if (OB_ENTRY_NOT_EXIST == ret) { | ||||
| @ -328,9 +327,10 @@ int ObDDLTableMergeTask::process() | ||||
|     LOG_WARN("not init", K(ret)); | ||||
|   } else if (OB_FAIL(MTL(ObLSService *)->get_ls(merge_param_.ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { | ||||
|     LOG_WARN("failed to get log stream", K(ret), K(merge_param_)); | ||||
|   } else if (OB_FAIL(ls_handle.get_ls()->get_tablet(merge_param_.tablet_id_, | ||||
|                                                     tablet_handle, | ||||
|                                                     ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|   } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, | ||||
|                                                merge_param_.tablet_id_, | ||||
|                                                tablet_handle, | ||||
|                                                ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|     LOG_WARN("failed to get tablet", K(ret), K(merge_param_)); | ||||
|   } else if (OB_FAIL(tablet_handle.get_obj()->get_ddl_kv_mgr(ddl_kv_mgr_handle))) { | ||||
|     if (OB_ENTRY_NOT_EXIST == ret) { | ||||
| @ -497,9 +497,10 @@ int ObTabletDDLUtil::prepare_index_data_desc(const share::ObLSID &ls_id, | ||||
|     LOG_WARN("ls service is null", K(ret), K(ls_id)); | ||||
|   } else if (OB_FAIL(ls_service->get_ls(ls_id, ls_handle, ObLSGetMod::DDL_MOD))) { | ||||
|     LOG_WARN("get ls failed", K(ret), K(ls_id)); | ||||
|   } else if (OB_FAIL(ls_handle.get_ls()->get_tablet(tablet_id, | ||||
|                                                     tablet_handle, | ||||
|                                                     ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|   } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, | ||||
|                                                tablet_id, | ||||
|                                                tablet_handle, | ||||
|                                                ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|     LOG_WARN("get tablet failed", K(ret)); | ||||
|   } else if (OB_FAIL(data_desc.init(tablet_handle.get_obj()->get_storage_schema(), | ||||
|                                     ls_id, | ||||
| @ -598,9 +599,10 @@ int ObTabletDDLUtil::create_ddl_sstable(ObSSTableIndexBuilder *sstable_index_bui | ||||
|       LOG_WARN("ls service is null", K(ret), K(ddl_param)); | ||||
|     } else if (OB_FAIL(ls_service->get_ls(ddl_param.ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { | ||||
|       LOG_WARN("get ls failed", K(ret), K(ddl_param)); | ||||
|     } else if (OB_FAIL(ls_handle.get_ls()->get_tablet(ddl_param.table_key_.tablet_id_, | ||||
|                                                       tablet_handle, | ||||
|                                                       ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|     } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, | ||||
|                                                  ddl_param.table_key_.tablet_id_, | ||||
|                                                  tablet_handle, | ||||
|                                                  ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|       LOG_WARN("get tablet failed", K(ret), K(ddl_param)); | ||||
|     } else { | ||||
|       const ObStorageSchema &storage_schema = tablet_handle.get_obj()->get_storage_schema(); | ||||
| @ -842,9 +844,10 @@ int ObTabletDDLUtil::check_and_get_major_sstable(const share::ObLSID &ls_id, | ||||
|     LOG_WARN("invalid argument", K(ret), K(ls_id), K(tablet_id)); | ||||
|   } else if (OB_FAIL(MTL(ObLSService *)->get_ls(ls_id, ls_handle, ObLSGetMod::DDL_MOD))) { | ||||
|     LOG_WARN("failed to get log stream", K(ret), K(ls_id)); | ||||
|   } else if (OB_FAIL(ls_handle.get_ls()->get_tablet(tablet_id, | ||||
|                                                     tablet_handle, | ||||
|                                                     ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|   } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, | ||||
|                                                tablet_id, | ||||
|                                                tablet_handle, | ||||
|                                                ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|     LOG_WARN("get tablet handle failed", K(ret), K(ls_id), K(tablet_id)); | ||||
|   } else if (OB_UNLIKELY(nullptr == tablet_handle.get_obj())) { | ||||
|     ret = OB_ERR_SYS; | ||||
|  | ||||
| @ -214,6 +214,7 @@ int ObSSTableInsertTabletContext::init(const ObSSTableInsertTabletParam &build_p | ||||
|   const int64_t memory_limit = 1024L * 1024L * 1024L * 10L; // 10GB | ||||
|   const ObTabletID &tablet_id = build_param.tablet_id_; | ||||
|   const ObLSID &ls_id = build_param.ls_id_; | ||||
|   share::ObLocationService *location_service = GCTX.location_service_; | ||||
|   ObLS *ls = nullptr; | ||||
|   ObLSService *ls_service = nullptr; | ||||
|   lib::ObMutexGuard guard(mutex_); | ||||
| @ -228,10 +229,7 @@ int ObSSTableInsertTabletContext::init(const ObSSTableInsertTabletParam &build_p | ||||
|     LOG_ERROR("ls service should not be null", K(ret)); | ||||
|   } else if (OB_FAIL(ls_service->get_ls(ls_id, ls_handle_, ObLSGetMod::DDL_MOD))) { | ||||
|     LOG_WARN("get ls failed", K(ret), K(ls_id)); | ||||
|   } else if (OB_ISNULL(ls = ls_handle_.get_ls())) { | ||||
|     ret = OB_ERR_UNEXPECTED; | ||||
|     LOG_ERROR("ls should not be null", K(ret)); | ||||
|   } else if (OB_FAIL(ls->get_tablet(tablet_id, tablet_handle_))) { | ||||
|   } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle_, tablet_id, tablet_handle_))) { | ||||
|     LOG_WARN("fail to get tablet handle", K(ret), K(tablet_id)); | ||||
|   } else if (OB_FAIL(data_sstable_redo_writer_.init(ls_id, tablet_id))) { | ||||
|     LOG_WARN("fail to init sstable redo writer", K(ret), K(ls_id), K(tablet_id)); | ||||
| @ -694,7 +692,7 @@ int ObSSTableInsertTabletContext::create_sstable_with_clog( | ||||
|     const ObLSID &ls_id = ls->get_ls_id(); | ||||
|     const ObTabletID &tablet_id = tablet->get_tablet_meta().tablet_id_; | ||||
|     const int64_t ddl_start_log_ts = data_sstable_redo_writer_.get_start_log_ts(); | ||||
|     if (OB_FAIL(ls->get_tablet(tablet_id, tablet_handle))) { | ||||
|     if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle_, tablet_id, tablet_handle))) { | ||||
|       LOG_WARN("get tablet failed", K(ret)); | ||||
|     } else if (OB_FAIL(tablet_handle.get_obj()->get_ddl_kv_mgr(ddl_kv_mgr_handle))) { | ||||
|       LOG_WARN("get ddl kv manager failed", K(ret), K(ls_id), K(tablet_id)); | ||||
|  | ||||
| @ -23,6 +23,7 @@ | ||||
|  | ||||
| using namespace oceanbase::common; | ||||
| using namespace oceanbase::blocksstable; | ||||
| using namespace oceanbase::share; | ||||
| using namespace oceanbase::storage; | ||||
|  | ||||
| ObTabletDDLKvMgr::ObTabletDDLKvMgr() | ||||
| @ -321,9 +322,10 @@ int ObTabletDDLKvMgr::update_tablet(const int64_t start_log_ts, const int64_t sn | ||||
|     LOG_WARN("invalid argument", K(ret), K(start_log_ts), K(snapshot_version), K(ddl_checkpoint_ts)); | ||||
|   } else if (OB_FAIL(MTL(ObLSService *)->get_ls(ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { | ||||
|     LOG_WARN("failed to get log stream", K(ret), K(ls_id_)); | ||||
|   } else if (OB_FAIL(ls_handle.get_ls()->get_tablet(tablet_id_, | ||||
|                                                     tablet_handle, | ||||
|                                                     ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|   } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, | ||||
|                                                tablet_id_, | ||||
|                                                tablet_handle, | ||||
|                                                ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) { | ||||
|     LOG_WARN("get tablet handle failed", K(ret), K(ls_id_), K(tablet_id_)); | ||||
|   } else { | ||||
|     ObTableHandleV2 table_handle; // empty | ||||
|  | ||||
| @ -1463,7 +1463,14 @@ int ObLSTabletService::get_tablet_with_timeout( | ||||
| { | ||||
|   int ret = OB_SUCCESS; | ||||
|   const ObTabletMapKey key(ls_->get_ls_id(), tablet_id); | ||||
|   if (OB_FAIL(ObTabletCreateDeleteHelper::check_and_get_tablet(key, handle, get_timeout_us))) { | ||||
|   if (IS_NOT_INIT) { | ||||
|     ret = OB_NOT_INIT; | ||||
|     LOG_WARN("not inited", K(ret), K_(is_inited)); | ||||
|   } else if (OB_UNLIKELY(!tablet_id.is_valid() | ||||
|       || get_timeout_us < ObTabletCommon::DIRECT_GET_COMMITTED_TABLET_TIMEOUT_US)) { | ||||
|     ret = OB_INVALID_ARGUMENT; | ||||
|     LOG_WARN("invalid args", K(ret), K(tablet_id), K(get_timeout_us)); | ||||
|   } else if (OB_FAIL(ObTabletCreateDeleteHelper::check_and_get_tablet(key, handle, get_timeout_us))) { | ||||
|     while (OB_ALLOCATE_MEMORY_FAILED == ret && ObTimeUtility::current_time() < retry_timeout_us) { | ||||
|       ret = ObTabletCreateDeleteHelper::check_and_get_tablet(key, handle, get_timeout_us); | ||||
|     } | ||||
|  | ||||
| @ -225,6 +225,11 @@ public: | ||||
|       const common::ObTabletID &tablet_id, | ||||
|       ObTabletHandle &handle, | ||||
|       const int64_t timeout_us = ObTabletCommon::DEFAULT_GET_TABLET_TIMEOUT_US); | ||||
|   int get_tablet_with_timeout( | ||||
|       const common::ObTabletID &tablet_id, | ||||
|       ObTabletHandle &handle, | ||||
|       const int64_t retry_timeout_us, | ||||
|       const int64_t get_timeout_us = ObTabletCommon::DEFAULT_GET_TABLET_TIMEOUT_US); | ||||
|   int remove_tablets(const common::ObIArray<common::ObTabletID> &tablet_id_array); | ||||
|   int get_ls_min_end_log_ts_in_old_tablets(int64_t &end_log_ts); | ||||
|   int get_tx_data_memtable_mgr(ObMemtableMgrHandle &mgr_handle); | ||||
| @ -458,11 +463,6 @@ private: | ||||
|       const common::ObTabletID &tablet_id, | ||||
|       ObTabletHandle &handle, | ||||
|       const int64_t timeout_us = ObTabletCommon::DEFAULT_GET_TABLET_TIMEOUT_US); | ||||
|   int get_tablet_with_timeout( | ||||
|       const common::ObTabletID &tablet_id, | ||||
|       ObTabletHandle &handle, | ||||
|       const int64_t retry_timeout_us, | ||||
|       const int64_t get_timeout_us = ObTabletCommon::DEFAULT_GET_TABLET_TIMEOUT_US); | ||||
|   int direct_get_tablet( | ||||
|       const common::ObTabletID &tablet_id, | ||||
|       ObTabletHandle &handle); | ||||
|  | ||||
| @ -5,6 +5,4 @@ config: | ||||
|   enable_metadb_used: 'false' | ||||
|   enable_qa_mode: 'TRUE' | ||||
|   location_expire_period: '1' | ||||
| version: 4.0.0 | ||||
| release: 20221031174602.el7 | ||||
|  | ||||
|  | ||||
| @ -1,4 +1,4 @@ | ||||
| obproxy: | ||||
| obproxy-ce: | ||||
|   servers: | ||||
|     - 127.0.0.1 | ||||
|   global: | ||||
|  | ||||
		Reference in New Issue
	
	Block a user