/** * Copyright (c) 2021 OceanBase * OceanBase CE is licensed under Mulan PubL v2. * You can use this software according to the terms and conditions of the Mulan PubL v2. * You may obtain a copy of Mulan PubL v2 at: * http://license.coscl.org.cn/MulanPubL-2.0 * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. * See the Mulan PubL v2 for more details. */ #include "ob_sstable_estimator.h" #include "storage/blocksstable/ob_micro_block_scanner.h" #include "storage/ob_file_system_util.h" namespace oceanbase { using namespace common; using namespace blocksstable; namespace storage { const int64_t MACRO_BLOCK_COUNT_THRESHOLD = 1024; ObSSTableEstimateContext::ObSSTableEstimateContext() : sstable_(NULL), rowkeys_(NULL) {} ObSSTableEstimateContext::~ObSSTableEstimateContext() {} void ObSSTableEstimateContext::reset() { sstable_ = NULL; rowkeys_ = NULL; macro_blocks_.reset(); cache_context_.reset(); multi_version_range_.reset(); } bool ObSSTableEstimateContext::is_valid() const { return NULL != sstable_ && NULL != rowkeys_ && cache_context_.is_valid(); } ObStoreRowMultiGetEstimator::ObStoreRowMultiGetEstimator() : ObISSTableEstimator() {} ObStoreRowMultiGetEstimator::~ObStoreRowMultiGetEstimator() {} int ObStoreRowMultiGetEstimator::set_context(ObSSTableEstimateContext& context) { int ret = OB_SUCCESS; if (OB_FAIL(ObISSTableEstimator::set_context(context))) { STORAGE_LOG(WARN, "failed to set context", K(ret)); } else if (OB_FAIL(context_.sstable_->find_macros(*context_.rowkeys_, context_.macro_blocks_))) { STORAGE_LOG(WARN, "failed to find macro blocks", K(ret)); } return ret; } int ObStoreRowMultiGetEstimator::estimate_row_count(ObPartitionEst& part_est) { int ret = OB_SUCCESS; int tmp_ret = OB_SUCCESS; bool is_contain = true; ObRowValueHandle handle; ObStorageFileHandle file_handle; ObStorageFile* file = nullptr; if (OB_FAIL(file_handle.assign(context_.sstable_->get_storage_file_handle()))) { STORAGE_LOG(WARN, "fail to get file handle", K(ret), K(context_.sstable_->get_storage_file_handle())); } else if (OB_ISNULL(file = file_handle.get_storage_file())) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(WARN, "fail to get pg file", K(ret), K(file_handle)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < context_.rowkeys_->count(); ++i) { const ObStoreRowkey& rowkey = context_.rowkeys_->at(i).get_store_rowkey(); const ObMacroBlockCtx& block_ctx = context_.macro_blocks_.at(i); const MacroBlockId& block_id = block_ctx.get_macro_block_id(); if (block_id.is_valid()) { if (OB_SUCCESS != (tmp_ret = context_.cache_context_.bf_cache_->may_contain( context_.sstable_->get_table_id(), block_id, file->get_file_id(), rowkey, is_contain))) { if (OB_ENTRY_NOT_EXIST != tmp_ret) { STORAGE_LOG(WARN, "failed to check may contain", K(tmp_ret), K_(context), K(block_ctx), K(rowkey)); } } if (is_contain || (OB_SUCCESS != ret)) { part_est.logical_row_count_++; } } } } return ret; } int ObSSTableScanEstimator::alloc_estimator(const ObSSTable& sstable) { int ret = OB_SUCCESS; void* buf = NULL; if (sstable.is_multi_version_minor_sstable()) { if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObMultiVersionSingleScanEstimator)))) { ret = OB_ALLOCATE_MEMORY_FAILED; } else { estimator_ = new (buf) ObMultiVersionSingleScanEstimator(allocator_); } } else { if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObStoreRowSingleScanEstimator)))) { ret = OB_ALLOCATE_MEMORY_FAILED; STORAGE_LOG(WARN, "failed to alloc ObStoreRowSingleScanEstimator", K(ret)); } else { estimator_ = new (buf) ObStoreRowSingleScanEstimator(); } } return ret; } ObStoreRowSingleScanEstimator::ObStoreRowSingleScanEstimator() : ObISSTableEstimator(), is_empty_scan_(false), is_first_scan_(true) {} ObStoreRowSingleScanEstimator::~ObStoreRowSingleScanEstimator() {} int ObStoreRowSingleScanEstimator::check_bf(ObMacroBlockCtx& macro_block_ctx) { int ret = OB_SUCCESS; int tmp_ret = OB_SUCCESS; // check bloom filter to identify if the scan is empty const MacroBlockId& macro_block_id = macro_block_ctx.get_macro_block_id(); ObStoreRowkey rowkey; ObStorageFileHandle file_handle; ObStorageFile* file = nullptr; if (OB_FAIL(file_handle.assign(context_.sstable_->get_storage_file_handle()))) { STORAGE_LOG(WARN, "fail to get file handle", K(ret), K(context_.sstable_->get_storage_file_handle())); } else if (OB_ISNULL(file = file_handle.get_storage_file())) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(WARN, "fail to get pg file", K(ret), K(file_handle)); } else if (OB_FAIL(get_common_rowkey(context_.range_->get_range(), rowkey))) { STORAGE_LOG(WARN, "failed to get common rowkey", K(ret), K(context_.range_)); } else if (rowkey.get_obj_cnt() > 0) { // allow check contain fail, should not overwrite ret bool is_contain = false; if (OB_SUCCESS != (tmp_ret = context_.cache_context_.bf_cache_->may_contain( context_.sstable_->get_table_id(), macro_block_id, file->get_file_id(), rowkey, is_contain))) { if (OB_ENTRY_NOT_EXIST != tmp_ret) { STORAGE_LOG(WARN, "failed to check may contain", K(tmp_ret), K_(context), K(macro_block_id), K(rowkey)); } } else if (!is_contain) { is_empty_scan_ = true; } } return ret; } void ObStoreRowSingleScanEstimator::reset() { ObISSTableEstimator::reset(); is_empty_scan_ = false; is_first_scan_ = true; } int ObStoreRowSingleScanEstimator::estimate_row_count(ObPartitionEst& part_est) { int ret = OB_SUCCESS; int64_t total_macro_block_count = 0; ObMacroBlockIterator macro_iter; if (!context_.range_->get_range().is_valid()) { } else if (context_.range_->get_range().is_whole_range()) { part_est.logical_row_count_ = context_.sstable_->get_meta().row_count_; part_est.physical_row_count_ = part_est.logical_row_count_; } else if (OB_FAIL(macro_iter.open(*context_.sstable_, *context_.range_))) { STORAGE_LOG(WARN, "fail to open macro iter,", K(ret)); } else if (OB_FAIL(macro_iter.get_macro_block_count(total_macro_block_count))) { STORAGE_LOG(WARN, "fail to get macro block count,", K(ret)); } else if (0 == total_macro_block_count) { } else if (total_macro_block_count > MACRO_BLOCK_COUNT_THRESHOLD) { // there are too many block, do estimate in fast way part_est.logical_row_count_ = (double)total_macro_block_count / context_.sstable_->get_meta().macro_block_count_ * context_.sstable_->get_meta().row_count_; part_est.physical_row_count_ = part_est.logical_row_count_; } else { // do estimate by macro block scan ObMacroBlockCtx macro_block_ctx; int64_t idx = 0; while (OB_SUCC(ret)) { bool is_start_block = 0 == idx; bool is_last_block = total_macro_block_count - 1 == idx; if (OB_FAIL(macro_iter.get_next_macro_block(macro_block_ctx))) { if (OB_ITER_END != ret) { STORAGE_LOG(WARN, "fail to get next macro block, ", K(ret)); } else { ret = OB_SUCCESS; break; } } else { if (1 == total_macro_block_count && OB_FAIL(check_bf(macro_block_ctx))) { STORAGE_LOG(WARN, "failed to open single scan estimator", K(ret)); } else if (is_empty_scan_) { } else if (OB_FAIL(estimate_macro_row_count(macro_block_ctx, is_start_block, is_last_block, part_est))) { STORAGE_LOG(WARN, "cannot estimate cost of macro block.", K(ret), K(macro_block_ctx), K(idx), K(total_macro_block_count)); } } idx++; } part_est.physical_row_count_ = part_est.logical_row_count_; } return ret; } int ObStoreRowSingleScanEstimator::estimate_macro_row_count(const ObMacroBlockCtx& macro_block_ctx, const bool is_start_block, const bool is_last_block, ObPartitionEst& part_est) { int ret = OB_SUCCESS; if (!macro_block_ctx.is_valid()) { ret = OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "invalid args", K(ret), K(macro_block_ctx)); } else { ObFullMacroBlockMeta meta; ObArray micro_infos; if (OB_FAIL(context_.sstable_->get_meta(macro_block_ctx.get_macro_block_id(), meta))) { STORAGE_LOG(WARN, "Fail to get macro meta, ", K(ret), K(macro_block_ctx)); } else if (!meta.is_valid()) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(WARN, "The macro block meta is NULL, ", K(ret), K(macro_block_ctx)); } else if ((!is_start_block) && (!is_last_block)) { // in the middle of scan ranges; part_est.logical_row_count_ += meta.meta_->row_count_; } else if (OB_FAIL(context_.cache_context_.block_index_cache_->get_micro_infos(context_.sstable_->get_table_id(), macro_block_ctx, context_.range_->get_range(), is_start_block, is_last_block, micro_infos))) { if (OB_BEYOND_THE_RANGE == ret) { ret = OB_SUCCESS; } else { STORAGE_LOG(WARN, "failed to get micro infos", K(ret), K_(context), K(macro_block_ctx)); } } else { // FIXME: // 1. io_micro_block_ costs need multiply block cache hit percent. // 2. row costs are not precise, need check in micro block. int64_t average_row_count = 0; int64_t average_row_size = 0; int64_t micro_block_count = micro_infos.count(); if (0 != meta.meta_->micro_block_count_) { average_row_count = meta.meta_->row_count_ / meta.meta_->micro_block_count_; } if (0 != meta.meta_->row_count_) { average_row_size = (meta.meta_->micro_block_index_offset_ - meta.meta_->micro_block_data_offset_) / meta.meta_->row_count_; } if (OB_SUCCESS == ret && is_start_block && micro_block_count > 0) { if (OB_FAIL(estimate_border_cost(micro_infos, macro_block_ctx, true /*check_start*/, average_row_count, average_row_size, micro_block_count, part_est))) { STORAGE_LOG(WARN, "failed to estimate border cost", K(ret)); } } if (OB_SUCCESS == ret && is_last_block && micro_block_count > 0) { if (OB_FAIL(estimate_border_cost(micro_infos, macro_block_ctx, false /*check_start*/, average_row_count, average_row_size, micro_block_count, part_est))) { STORAGE_LOG(WARN, "failed to estimate border cost", K(ret)); } } if (OB_SUCCESS == ret && micro_block_count > 0) { int64_t other_row_count = average_row_count * micro_block_count; part_est.logical_row_count_ += other_row_count; } } } return ret; } int ObStoreRowSingleScanEstimator::estimate_border_cost( const common::ObIArray& micro_infos, const ObMacroBlockCtx& macro_block_ctx, const bool check_start, const int64_t average_row_count, const int64_t average_row_size, int64_t& micro_block_count, ObPartitionEst& part_est) { int ret = OB_SUCCESS; int64_t result_row_count = 0; int64_t first = check_start ? 0 : micro_infos.count() - 1; if (first < 0) { ret = OB_ERR_SYS; STORAGE_LOG(WARN, "invalid first", K(ret), K(first), K(micro_infos.count()), K(check_start)); } else if (!macro_block_ctx.is_valid() || average_row_count < 0 || average_row_size < 0) { ret = OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "invalid args", K(ret), K(macro_block_ctx), K(average_row_count), K(average_row_size)); } else if (micro_block_count > 0) { if (OB_FAIL(estimate_border_count_( micro_infos.at(first), macro_block_ctx, average_row_size, result_row_count, micro_block_count, part_est))) { STORAGE_LOG(WARN, "fail to estimate_border_block_row_count, ", K(ret)); } else if (micro_block_count > 0 && 0 == result_row_count) { int64_t second = check_start ? 1 : micro_infos.count() - 2; if (second < 0) { ret = OB_ERR_SYS; STORAGE_LOG(WARN, "invalid second , ", K(ret), K(second), K(check_start), K(micro_infos.count())); } else if (OB_FAIL(estimate_border_count_(micro_infos.at(second), macro_block_ctx, average_row_size, result_row_count, micro_block_count, part_est))) { STORAGE_LOG(WARN, "fail to estimate_border_block_row_count, ", K(ret)); } } } return ret; } int ObStoreRowSingleScanEstimator::estimate_border_count_(const ObMicroBlockInfo& micro_info, const blocksstable::ObMacroBlockCtx& macro_block_ctx, const int64_t average_row_size, int64_t& result_row_count, int64_t& micro_block_count, ObPartitionEst& part_est) { int ret = OB_SUCCESS; int64_t logical_row_count = 0; if (!micro_info.is_valid() || !macro_block_ctx.is_valid() || average_row_size < 0) { ret = OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "invalid args, ", K(ret), K(micro_info), K(macro_block_ctx), K(average_row_size)); } else if (OB_FAIL( estimate_border_row_count(micro_info, macro_block_ctx, false, logical_row_count, result_row_count))) { STORAGE_LOG(WARN, "failed to estimate_border_row_count, ", K(ret)); } else if (result_row_count < 0) { ret = OB_ERR_SYS; STORAGE_LOG(ERROR, "invalid result_row_count, ", K(ret), K(result_row_count)); } else { --micro_block_count; part_est.logical_row_count_ += logical_row_count; } return ret; } int ObISSTableEstimator::estimate_border_row_count(const ObMicroBlockInfo& micro_info, const blocksstable::ObMacroBlockCtx& macro_block_ctx, bool consider_multi_version, int64_t& logical_row_count, int64_t& physical_row_count) { int ret = OB_SUCCESS; ObFullMacroBlockMeta meta; ObBlockCacheWorkingSet* block_cache_ws = context_.cache_context_.block_cache_ws_; ObMicroBlockBufferHandle micro_handle; ObMicroBlockData block_data; ObMacroBlockReader reader; ObMacroBlockHandle macro_handle; ObArenaAllocator allocator; ObStorageFileHandle file_handle; ObStorageFile* file = nullptr; logical_row_count = 0; physical_row_count = 0; if (!micro_info.is_valid() || !macro_block_ctx.is_valid()) { ret = OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "invalid args", K(ret), K(micro_info), K(macro_block_ctx), K(physical_row_count)); } else if (NULL == block_cache_ws) { ret = OB_ERR_SYS; STORAGE_LOG(ERROR, "block cache ws must not null", K(ret)); } else if (OB_FAIL(file_handle.assign(context_.sstable_->get_storage_file_handle()))) { STORAGE_LOG(WARN, "fail to get file handle", K(ret), K(context_.sstable_->get_storage_file_handle())); } else if (OB_ISNULL(file = file_handle.get_storage_file())) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(WARN, "fail to get pg file", K(ret), K(file_handle)); } else if (OB_FAIL(context_.sstable_->get_meta(macro_block_ctx.get_macro_block_id(), meta))) { STORAGE_LOG(WARN, "Fail to get macro meta, ", K(ret)); } else if (!meta.is_valid()) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(WARN, "unexpect error, meta is NULL.", K(ret), K(meta), KP(block_cache_ws), K(macro_block_ctx)); } else if (OB_FAIL(block_cache_ws->get_cache_block(context_.sstable_->get_table_id(), macro_block_ctx.get_macro_block_id(), file->get_file_id(), micro_info.offset_, micro_info.size_, micro_handle))) { if (OB_ENTRY_NOT_EXIST != ret) { STORAGE_LOG(WARN, "failed to get cache block", K(ret), K_(context), K(macro_block_ctx)); } else { // we cannot always get micro block from block cache // now we still need to load data from disk and overwrite the return code bool is_compressed = false; ObMacroBlockReadInfo read_info; read_info.macro_block_ctx_ = ¯o_block_ctx; read_info.offset_ = micro_info.offset_; read_info.size_ = micro_info.size_; read_info.io_desc_.category_ = USER_IO; read_info.io_desc_.wait_event_no_ = ObWaitEventIds::DB_FILE_DATA_READ; macro_handle.set_file(file); if (OB_FAIL(file->read_block(read_info, macro_handle))) { STORAGE_LOG(WARN, "Fail to read micro block from io, ", K(ret)); } else if (OB_FAIL(ObRecordHeaderV3::deserialize_and_check_record( macro_handle.get_buffer(), micro_info.size_, MICRO_BLOCK_HEADER_MAGIC))) { STORAGE_LOG(WARN, "micro block data is corrupted.", K(ret)); } else if (OB_FAIL(reader.decompress_data(meta, macro_handle.get_buffer(), micro_info.size_, block_data.get_buf(), block_data.get_buf_size(), is_compressed))) { STORAGE_LOG(WARN, "Fail to decompress data, ", K(ret)); } else { int tmp_ret = OB_SUCCESS; ObMicroBlockCacheKey tmp_key(context_.sstable_->get_table_id(), macro_block_ctx.get_macro_block_id(), file->get_file_id(), micro_info.offset_, micro_info.size_); ObMicroBlockCacheValue tmp_value(block_data.get_buf(), block_data.get_buf_size()); if (OB_SUCCESS != (tmp_ret = block_cache_ws->put(tmp_key, tmp_value))) { STORAGE_LOG(WARN, "Fail to put micro block to block cache, ", K(tmp_ret)); } STORAGE_LOG(DEBUG, "Success to decompress micro block, ", K(macro_block_ctx)); } } } else if (!micro_handle.is_valid()) { ret = OB_ERR_SYS; STORAGE_LOG(ERROR, "invalid micro_handle", K(ret), K(micro_handle)); } else { block_data = *micro_handle.get_block_data(); } if (OB_SUCC(ret)) { ObMicroBlockScanner* block_scanner = NULL; ObColumnMap* column_map = NULL; common::ObArray columns; if (NULL == (column_map = OB_NEW(ObColumnMap, ObModIds::OB_CS_SSTABLE_READER))) { ret = OB_ALLOCATE_MEMORY_FAILED; STORAGE_LOG(ERROR, "allocate memory for column map failed.", K(ret)); } else if (NULL == (block_scanner = OB_NEW(ObMicroBlockScanner, ObModIds::OB_CS_SSTABLE_READER))) { ret = OB_ALLOCATE_MEMORY_FAILED; STORAGE_LOG(ERROR, "allocate memory for macro block scanner failed.", K(ret)); } else if (OB_FAIL(columns.reserve(meta.meta_->rowkey_column_number_))) { STORAGE_LOG(WARN, "reserve memory for rowkey columns desc failed.", K(ret)); } else if (OB_FAIL(get_rowkey_column_desc(meta, columns))) { STORAGE_LOG(WARN, "Fail to get_rowkey_column_desc ", K(ret)); } else if (OB_FAIL(column_map->init(allocator, meta.meta_->schema_version_, meta.meta_->rowkey_column_number_, meta.meta_->column_number_, columns))) { STORAGE_LOG(WARN, "Fail to init column map, ", K(ret)); } else if (OB_FAIL(block_scanner->estimate_row_count(allocator, context_.range_->get_range(), *column_map, block_data, static_cast(meta.meta_->row_store_type_), consider_multi_version, logical_row_count, physical_row_count))) { STORAGE_LOG(WARN, "Fail to set scan param for micro block scanner, ", K(ret)); } if (NULL != column_map) { OB_DELETE(ObColumnMap, ObModIds::OB_CS_SSTABLE_READER, column_map); column_map = NULL; } if (NULL != block_scanner) { OB_DELETE(ObMicroBlockScanner, ObModIds::OB_CS_SSTABLE_READER, block_scanner); block_scanner = NULL; } } return ret; } int ObISSTableEstimator::get_rowkey_column_desc( const blocksstable::ObFullMacroBlockMeta& meta, common::ObIArray& columns) { // static func, no need to check inner stat int ret = OB_SUCCESS; share::schema::ObColDesc desc; if (!meta.is_valid() || columns.count() != 0) { ret = OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "invalid args", K(ret), K(meta), K(columns)); } for (int64_t i = 0; OB_SUCC(ret) && i < meta.meta_->rowkey_column_number_; ++i) { desc.col_id_ = meta.schema_->column_id_array_[i]; desc.col_type_ = meta.schema_->column_type_array_[i]; desc.col_order_ = meta.schema_->column_order_array_[i]; if (OB_FAIL(columns.push_back(desc))) { STORAGE_LOG(WARN, "push column desc failed.", K(ret), K(desc)); } } return ret; } int ObStoreRowSingleScanEstimator::get_common_rowkey(const ObStoreRange& range, ObStoreRowkey& rowkey) { // static func, no need to check inner stat int ret = OB_SUCCESS; int64_t prefix_len = 0; if (!range.is_valid()) { ret = OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "invalid args", K(ret), K(range)); } else if (OB_FAIL(ObStoreRowkey::get_common_prefix_length(range.get_start_key(), range.get_end_key(), prefix_len))) { STORAGE_LOG(WARN, "failed to get common prefix length", K(ret), K(range)); } else { rowkey.reset(); rowkey.assign(const_cast(range.get_start_key().get_obj_ptr()), prefix_len); } return ret; } int ObMultiVersionSingleScanEstimator::set_context(ObSSTableEstimateContext& context) { int ret = OB_SUCCESS; ObVersionRange version_range; version_range.snapshot_version_ = ObVersionRange::MAX_VERSION; version_range.base_version_ = ObVersionRange::MAX_VERSION; version_range.multi_version_start_ = ObVersionRange::MAX_VERSION; if (OB_FAIL(ObISSTableEstimator::set_context(context))) { STORAGE_LOG(WARN, "failed to set context", K(ret)); } else if (OB_FAIL(ObVersionStoreRangeConversionHelper::range_to_multi_version_range( *context.range_, version_range, allocator_, context_.multi_version_range_))) { STORAGE_LOG(WARN, "failed to convert to multi version range", K(ret)); } else if (OB_FAIL(context_.sstable_->find_macros(context_.multi_version_range_, context_.macro_blocks_))) { STORAGE_LOG(WARN, "failed to find macros", K(ret), K_(context_.multi_version_range)); } return ret; } int ObMultiVersionSingleScanEstimator::estimate_row_count(ObPartitionEst& part_est) { int ret = OB_SUCCESS; int64_t total_macro_block_count = context_.macro_blocks_.count(); MacroBlockId macro_block_id; bool is_left_border = false; bool is_right_border = false; int64_t gap_size = 0; int64_t purged_phy_row_count = 0; const bool is_whole_range = context_.range_->get_range().is_whole_range(); for (int64_t i = 0; OB_SUCC(ret) && i < total_macro_block_count; ++i) { const ObMacroBlockCtx& macro_block_ctx = context_.macro_blocks_.at(i); is_left_border = is_whole_range ? false : 0 == i; is_right_border = is_whole_range ? false : total_macro_block_count - 1 == i; if (OB_FAIL(estimate_macro_row_count( macro_block_ctx, is_left_border, is_right_border, part_est, gap_size, purged_phy_row_count))) { STORAGE_LOG(WARN, "cannot estimate cost of macro block.", K(ret), K(macro_block_id), K(is_left_border), K(is_right_border), K(i), K(total_macro_block_count)); } } return ret; } int ObMultiVersionSingleScanEstimator::estimate_macro_row_count(const blocksstable::ObMacroBlockCtx& macro_block_ctx, const bool is_left_border, const bool is_right_border, ObPartitionEst& part_est, int64_t& gap_size, int64_t& purged_phy_row_count) { int ret = OB_SUCCESS; ObFullMacroBlockMeta meta; if (!macro_block_ctx.is_valid()) { ret = OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "macro block id is invalid", K(ret)); } else if (OB_FAIL(context_.sstable_->get_meta(macro_block_ctx.get_macro_block_id(), meta))) { STORAGE_LOG(WARN, "failed to get macro block meta", K(ret), K(macro_block_ctx)); } else if (!meta.is_valid()) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(WARN, "macro block meta is null", K(ret), K(macro_block_ctx)); } else { const int64_t delta = meta.meta_->row_count_delta_; // if delta < 0, it means this macro block really delete rows already exist in older table // so if this delta is believable, we should minus gap_size+delta from physical row count // since rows from older table can be skipped int64_t macro_purged_row_count = 0; bool need_reset_gap = false; if (!(is_left_border || is_right_border)) { part_est.logical_row_count_ += meta.meta_->row_count_delta_; part_est.physical_row_count_ += meta.meta_->row_count_; if (meta.meta_->macro_block_deletion_flag_) { gap_size += meta.meta_->row_count_; purged_phy_row_count += meta.meta_->row_count_ - (delta < 0 ? delta : 0); } else { need_reset_gap = true; if (delta <= 0) { if (OB_FAIL(context_.cache_context_.block_index_cache_->cal_macro_purged_row_count( context_.sstable_->get_table_id(), macro_block_ctx, macro_purged_row_count))) { STORAGE_LOG(WARN, "failed to cal_macro_phy_row_count", K(ret), K(macro_block_ctx)); } else { part_est.physical_row_count_ -= macro_purged_row_count; } } } } else { need_reset_gap = true; bool need_check_micro_block = false; ObSEArray micro_infos; if (OB_FAIL(context_.cache_context_.block_index_cache_->cal_border_row_count(context_.sstable_->get_table_id(), macro_block_ctx, context_.multi_version_range_.get_range(), is_left_border, is_right_border, part_est.logical_row_count_, part_est.physical_row_count_, need_check_micro_block))) { STORAGE_LOG(WARN, "failed to get row count delta", K_(context), K(macro_block_ctx), K(ret)); } else if (need_check_micro_block) { // Fix problem in issue #32813975 (the estimation is imprecise after minor freeze) // open the specific micro block to get the row count and take multiple versions of row keys // into consideration. int64_t logical_row_count = 0, physical_row_count = 0; if (OB_FAIL(context_.cache_context_.block_index_cache_->get_micro_infos(context_.sstable_->get_table_id(), macro_block_ctx, context_.multi_version_range_.get_range(), is_left_border, is_right_border, micro_infos))) { if (OB_BEYOND_THE_RANGE == ret) { ret = OB_SUCCESS; } else { STORAGE_LOG(WARN, "failed to get micro infos", K(ret), K_(context), K(macro_block_ctx)); } } else if (1 != micro_infos.count()) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(WARN, "unexpected error, should only 1 micro block, ", K(ret), K(micro_infos.count())); } else if (OB_FAIL(estimate_border_row_count( micro_infos.at(0), macro_block_ctx, true, logical_row_count, physical_row_count))) { STORAGE_LOG(WARN, "failed to estimate_border_row_count for multi version, ", K(ret)); } else { part_est.logical_row_count_ = logical_row_count; part_est.physical_row_count_ = physical_row_count; } } } if (OB_SUCC(ret) && need_reset_gap) { if (gap_size >= OB_SKIP_RANGE_LIMIT) { part_est.physical_row_count_ -= purged_phy_row_count; } gap_size = 0; purged_phy_row_count = 0; } STORAGE_LOG(DEBUG, "estimate macro row count", K(part_est), K(delta), K(macro_block_ctx), K(gap_size), K(purged_phy_row_count)); } return ret; } ObStoreRowMultiScanEstimator::ObStoreRowMultiScanEstimator() : ObISSTableEstimator() {} ObStoreRowMultiScanEstimator::~ObStoreRowMultiScanEstimator() {} int ObStoreRowMultiScanEstimator::estimate_row_count(ObPartitionEst& part_est) { int ret = OB_SUCCESS; ObQueryFlag query_flag; ObArray rowkeys; ObPartitionEst tmp_cost; for (int64_t i = 0; OB_SUCC(ret) && i < context_.ranges_->count(); ++i) { const ObExtStoreRange& range = context_.ranges_->at(i); if (OB_FAIL(context_.sstable_->estimate_scan_row_count( query_flag, context_.sstable_->get_table_id(), range, tmp_cost))) { STORAGE_LOG(WARN, "fail to estimate scan cost of sstable", K(ret)); } else if (OB_FAIL(part_est.add(tmp_cost))) { STORAGE_LOG(WARN, "failed to add cost metrics", K(ret)); } } return ret; } } // namespace storage } // namespace oceanbase