451 lines
		
	
	
		
			16 KiB
		
	
	
	
		
			C++
		
	
	
	
	
	
			
		
		
	
	
			451 lines
		
	
	
		
			16 KiB
		
	
	
	
		
			C++
		
	
	
	
	
	
| /**
 | |
|  * 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.
 | |
|  */
 | |
| 
 | |
| #define USING_LOG_PREFIX STORAGE
 | |
| #include "ob_macro_block_iterator.h"
 | |
| #include "lib/container/ob_fixed_array_iterator.h"
 | |
| #include "ob_sstable.h"
 | |
| 
 | |
| using namespace oceanbase::common;
 | |
| using namespace oceanbase::blocksstable;
 | |
| 
 | |
| namespace oceanbase {
 | |
| namespace storage {
 | |
| ObMacroBlockDesc::ObMacroBlockDesc()
 | |
|     : data_version_(0),
 | |
|       block_idx_(0),
 | |
|       block_cnt_(0),
 | |
|       macro_block_ctx_(),
 | |
|       full_meta_(),
 | |
|       range_(),
 | |
|       row_store_type_(FLAT_ROW_STORE),
 | |
|       schema_version_(0),
 | |
|       data_seq_(0),
 | |
|       row_count_(0),
 | |
|       occupy_size_(0),
 | |
|       micro_block_count_(0),
 | |
|       data_checksum_(0),
 | |
|       snapshot_version_(0),
 | |
|       row_count_delta_(0),
 | |
|       progressive_merge_round_(0),
 | |
|       contain_uncommitted_row_(true)
 | |
| {}
 | |
| /**
 | |
|  * -----------------------------------------------------------ObMacroBlockRowComparor-------------------------------------------------------
 | |
|  */
 | |
| 
 | |
| ObMacroBlockRowComparor::ObMacroBlockRowComparor()
 | |
|     : ret_(OB_SUCCESS),
 | |
|       use_collation_free_(false),
 | |
|       is_prefix_check_(false),
 | |
|       full_meta_(),
 | |
|       macro_block_rowkey_(),
 | |
|       sstable_(nullptr)
 | |
| {}
 | |
| 
 | |
| ObMacroBlockRowComparor::~ObMacroBlockRowComparor()
 | |
| {}
 | |
| 
 | |
| int ObMacroBlockRowComparor::compare_(
 | |
|     const blocksstable::MacroBlockId& block_id, const common::ObStoreRowkey& rowkey, int& cmp_ret)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
| 
 | |
|   if (OB_ISNULL(sstable_)) {
 | |
|     ret = OB_ERR_SYS;
 | |
|     LOG_WARN("error sys, sstable must not be null", K(ret));
 | |
|   } else if (OB_FAIL(sstable_->get_meta(block_id, full_meta_))) {
 | |
|     LOG_WARN("fail to get meta", K(ret), K(block_id));
 | |
|   } else if (OB_UNLIKELY(!full_meta_.is_valid())) {
 | |
|     ret = OB_INVALID_ARGUMENT;
 | |
|     STORAGE_LOG(ERROR, "Invalid macro meta, ", K(ret), K(full_meta_));
 | |
|   } else {
 | |
|     const ObMacroBlockMetaV2* macro_meta = full_meta_.meta_;
 | |
|     if (use_collation_free_) {
 | |
|       if (OB_ISNULL(macro_meta->collation_free_endkey_)) {
 | |
|         macro_block_rowkey_.assign(macro_meta->endkey_, macro_meta->rowkey_column_number_);
 | |
|       } else {
 | |
|         macro_block_rowkey_.assign(macro_meta->collation_free_endkey_, macro_meta->rowkey_column_number_);
 | |
|       }
 | |
|     } else {
 | |
|       macro_block_rowkey_.assign(macro_meta->endkey_, macro_meta->rowkey_column_number_);
 | |
|     }
 | |
|     if (is_prefix_check_) {
 | |
|       ret = macro_block_rowkey_.compare_prefix(rowkey, cmp_ret);
 | |
|     } else {
 | |
|       ret = macro_block_rowkey_.compare(rowkey, cmp_ret);
 | |
|     }
 | |
|   }
 | |
| 
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| bool ObMacroBlockRowComparor::operator()(
 | |
|     const blocksstable::MacroBlockId& block_id, const common::ObStoreRowkey& rowkey)
 | |
| {
 | |
|   bool bret = false;
 | |
|   int32_t cmp_ret = 0;
 | |
| 
 | |
|   if (OB_UNLIKELY(OB_SUCCESS != (ret_ = compare_(block_id, rowkey, cmp_ret)))) {
 | |
|     STORAGE_LOG(WARN, "Failed to compare block and rowkey", K(rowkey), K_(ret));
 | |
|   } else {
 | |
|     bret = cmp_ret < 0;
 | |
|   }
 | |
| 
 | |
|   return bret;
 | |
| }
 | |
| 
 | |
| bool ObMacroBlockRowComparor::operator()(
 | |
|     const common::ObStoreRowkey& rowkey, const blocksstable::MacroBlockId& block_id)
 | |
| {
 | |
|   bool bret = false;
 | |
|   int32_t cmp_ret = 0;
 | |
| 
 | |
|   if (OB_UNLIKELY(OB_SUCCESS != (ret_ = compare_(block_id, rowkey, cmp_ret)))) {
 | |
|     STORAGE_LOG(WARN, "Failed to compare block and rowkey", K(block_id), K(rowkey), K_(ret));
 | |
|   } else {
 | |
|     bret = cmp_ret > 0;
 | |
|   }
 | |
| 
 | |
|   return bret;
 | |
| }
 | |
| 
 | |
| void ObMacroBlockRowComparor::reset()
 | |
| {
 | |
|   ret_ = OB_SUCCESS;
 | |
|   use_collation_free_ = false;
 | |
|   full_meta_.reset();
 | |
|   is_prefix_check_ = false;
 | |
|   sstable_ = nullptr;
 | |
| }
 | |
| 
 | |
| /**
 | |
|  * -----------------------------------------------------------ObMacroBlockIterator-----------------------------------------------------------
 | |
|  */
 | |
| ObMacroBlockIterator::ObMacroBlockIterator()
 | |
|     : sstable_(NULL), is_reverse_scan_(false), step_(1), cur_idx_(-1), begin_(-1), end_(-1), check_lob_(false)
 | |
| {}
 | |
| 
 | |
| ObMacroBlockIterator::~ObMacroBlockIterator()
 | |
| {}
 | |
| 
 | |
| void ObMacroBlockIterator::reset()
 | |
| {
 | |
|   sstable_ = NULL;
 | |
|   is_reverse_scan_ = false;
 | |
|   step_ = 1;
 | |
|   cur_idx_ = -1;
 | |
|   begin_ = -1;
 | |
|   end_ = -1;
 | |
|   check_lob_ = false;
 | |
| }
 | |
| 
 | |
| int ObMacroBlockIterator::open(ObSSTable& sstable, const ObExtStoreRowkey& ext_rowkey)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (OB_UNLIKELY(!sstable.is_valid())) {
 | |
|     ret = OB_INVALID_ARGUMENT;
 | |
|     STORAGE_LOG(WARN, "Invalid argument, ", K(ret), K(sstable));
 | |
|   } else {
 | |
|     sstable_ = &sstable;
 | |
|     if (OB_FAIL(locate_macro_block(ext_rowkey, cur_idx_))) {
 | |
|       STORAGE_LOG(WARN, "Fail to locate macro block, ", K(ret), K(ext_rowkey));
 | |
|     } else {
 | |
|       begin_ = end_ = cur_idx_;
 | |
|     }
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int ObMacroBlockIterator::open(ObSSTable& sstable, const bool is_reverse, const bool check_lob)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
| 
 | |
|   if (OB_UNLIKELY(!sstable.is_valid())) {
 | |
|     ret = OB_INVALID_ARGUMENT;
 | |
|     STORAGE_LOG(WARN, "Invalid argument, ", K(ret), K(sstable));
 | |
|   } else if (OB_UNLIKELY(check_lob && is_reverse)) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     STORAGE_LOG(WARN, "Macro block scan with lob conflict with reverse scan", K(ret));
 | |
|   } else {
 | |
|     sstable_ = &sstable;
 | |
|     check_lob_ = check_lob;
 | |
|     is_reverse_scan_ = is_reverse;
 | |
|     begin_ = 0;
 | |
|     if (check_lob) {
 | |
|       end_ = sstable.get_total_macro_blocks().count() - 1;
 | |
|     } else {
 | |
|       end_ = sstable.get_macro_block_ids().count() - 1;
 | |
|     }
 | |
|     if (is_reverse) {
 | |
|       cur_idx_ = end_;
 | |
|       step_ = -1;
 | |
|     } else {
 | |
|       cur_idx_ = begin_;
 | |
|       step_ = 1;
 | |
|     }
 | |
|   }
 | |
| 
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int ObMacroBlockIterator::open(ObSSTable& sstable, const common::ObExtStoreRange& ext_range, const bool is_reverse)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
| 
 | |
|   if (OB_UNLIKELY(!sstable.is_valid())) {
 | |
|     ret = OB_INVALID_ARGUMENT;
 | |
|     STORAGE_LOG(WARN, "Invalid argument, ", K(ret), K(sstable));
 | |
|   } else {
 | |
|     sstable_ = &sstable;
 | |
|     is_reverse_scan_ = is_reverse;
 | |
|     step_ = is_reverse ? -1 : 1;
 | |
| 
 | |
|     if (OB_FAIL(locate_macro_block(ext_range.get_ext_start_key(), begin_))) {
 | |
|       STORAGE_LOG(WARN, "Fail to locate start macro block, ", K(ret));
 | |
|     } else if (-1 == begin_) {
 | |
|       // not exist range
 | |
|       end_ = cur_idx_ = -1;
 | |
|     } else if (OB_FAIL(locate_macro_block(ext_range.get_ext_end_key(), end_))) {
 | |
|       STORAGE_LOG(WARN, "Fail to locate end macro block, ", K(ret));
 | |
|     } else {
 | |
|       if (-1 == end_) {
 | |
|         end_ = sstable.get_macro_block_ids().count() - 1;
 | |
|       }
 | |
|       if (!ext_range.get_range().get_border_flag().inclusive_start()) {
 | |
|         const MacroBlockId& block_id = sstable_->get_macro_block_ids().at(begin_);
 | |
|         ObFullMacroBlockMeta full_meta;
 | |
|         if (OB_FAIL(sstable_->get_meta(block_id, full_meta))) {
 | |
|           STORAGE_LOG(WARN, "fail to get meta", K(ret));
 | |
|         } else {
 | |
|           const ObMacroBlockMetaV2* meta = full_meta.meta_;
 | |
|           ObStoreRowkey rowkey(meta->endkey_, meta->rowkey_column_number_);
 | |
|           if (ext_range.get_range().get_start_key() == rowkey) {
 | |
|             ++begin_;
 | |
|           }
 | |
|         }
 | |
|       }
 | |
| 
 | |
|       cur_idx_ = is_reverse ? end_ : begin_;
 | |
|     }
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int ObMacroBlockIterator::get_next_macro_block(blocksstable::ObMacroBlockCtx& macro_block_ctx)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (OB_UNLIKELY(check_lob_)) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     STORAGE_LOG(WARN, "Unexpected check lob mode in ObMacroBlockIterator", K(ret));
 | |
|   } else if (cur_idx_ < begin_ || cur_idx_ > end_ || cur_idx_ < 0) {
 | |
|     ret = OB_ITER_END;
 | |
|   } else if (OB_FAIL(sstable_->get_macro_block_ctx(cur_idx_, macro_block_ctx))) {
 | |
|     STORAGE_LOG(WARN, "Fail to get_macro_ctx", K(ret), K(cur_idx_));
 | |
|   } else {
 | |
|     STORAGE_LOG(DEBUG, "Success to get macro block id, ", K_(cur_idx), K(macro_block_ctx));
 | |
|     cur_idx_ += step_;
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int ObMacroBlockIterator::get_next_macro_block(
 | |
|     blocksstable::ObMacroBlockCtx& macro_block_ctx, ObFullMacroBlockMeta& macro_meta)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (OB_UNLIKELY(check_lob_)) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     STORAGE_LOG(WARN, "Unexpected check lob mode in ObMacroBlockIterator", K(ret));
 | |
|   } else if (cur_idx_ < begin_ || cur_idx_ > end_ || cur_idx_ < 0) {
 | |
|     ret = OB_ITER_END;
 | |
|   } else if (OB_FAIL(sstable_->get_macro_block_ctx(cur_idx_, macro_block_ctx))) {
 | |
|     STORAGE_LOG(WARN, "Fail to get_macro_ctx", K(ret), K(cur_idx_));
 | |
|   } else if (OB_FAIL(sstable_->get_meta(macro_block_ctx.sstable_block_id_.macro_block_id_, macro_meta))) {
 | |
|     STORAGE_LOG(WARN, "fail to get meta", K(ret));
 | |
|   } else if (OB_UNLIKELY(!macro_meta.is_valid())) {
 | |
|     ret = OB_ERR_UNEXPECTED;
 | |
|     STORAGE_LOG(WARN, "Unexpected error, ", K(ret), K(macro_meta));
 | |
|   } else {
 | |
|     STORAGE_LOG(DEBUG, "Success to get macro block id, ", K_(cur_idx), K(macro_block_ctx));
 | |
|     cur_idx_ += step_;
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int ObMacroBlockIterator::get_next_macro_block(ObMacroBlockDesc& block_desc)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   ObFullMacroBlockMeta full_meta;
 | |
| 
 | |
|   if (cur_idx_ < begin_ || cur_idx_ > end_ || cur_idx_ < 0) {
 | |
|     ret = OB_ITER_END;
 | |
|   } else if (cur_idx_ >= sstable_->get_macro_block_count()) {
 | |
|     const int64_t lob_idx = cur_idx_ - sstable_->get_macro_block_count();
 | |
|     if (OB_UNLIKELY(!check_lob_)) {
 | |
|       ret = OB_ERR_UNEXPECTED;
 | |
|       STORAGE_LOG(WARN,
 | |
|           "Unexpected macro block iter index without check lob",
 | |
|           K_(cur_idx),
 | |
|           "macro_block_count",
 | |
|           sstable_->get_macro_block_count(),
 | |
|           K(ret));
 | |
|     } else if (OB_FAIL(sstable_->get_lob_macro_block_ctx(lob_idx, block_desc.macro_block_ctx_))) {
 | |
|       LOG_WARN("Failed to get lob macro block ctx", K(ret), K(lob_idx));
 | |
|     } else if (OB_FAIL(sstable_->get_meta(block_desc.macro_block_ctx_.sstable_block_id_.macro_block_id_, full_meta))) {
 | |
|       LOG_WARN("fail to get meta", K(ret));
 | |
|     } else {
 | |
|       block_desc.range_.get_start_key().assign(full_meta.meta_->endkey_, full_meta.meta_->rowkey_column_number_);
 | |
|       block_desc.range_.get_end_key().assign(full_meta.meta_->endkey_, full_meta.meta_->rowkey_column_number_);
 | |
|     }
 | |
|   } else {
 | |
|     if (0 == cur_idx_) {
 | |
|       block_desc.range_.get_start_key().set_min();
 | |
|     } else {
 | |
|       if (OB_FAIL(sstable_->get_meta(sstable_->get_macro_block_ids().at(cur_idx_ - 1), full_meta))) {
 | |
|         STORAGE_LOG(WARN, "fail to get meta", K(ret));
 | |
|       } else if (!full_meta.is_valid()) {
 | |
|         ret = OB_ERR_UNEXPECTED;
 | |
|         STORAGE_LOG(WARN, "Invalid macro meta, ", K(ret), K(full_meta));
 | |
|       } else {
 | |
|         block_desc.range_.get_start_key().assign(full_meta.meta_->endkey_, full_meta.meta_->rowkey_column_number_);
 | |
|       }
 | |
|     }
 | |
| 
 | |
|     if (OB_SUCC(ret)) {
 | |
|       if (OB_FAIL(sstable_->get_meta(sstable_->get_macro_block_ids().at(cur_idx_), full_meta))) {
 | |
|         STORAGE_LOG(WARN, "fail to get meta", K(ret));
 | |
|       } else if (!full_meta.is_valid()) {
 | |
|         ret = OB_ERR_UNEXPECTED;
 | |
|         STORAGE_LOG(WARN, "Invalid macro meta, ", K(ret), K(full_meta));
 | |
|       } else if (OB_FAIL(sstable_->get_macro_block_ctx(cur_idx_, block_desc.macro_block_ctx_))) {
 | |
|         STORAGE_LOG(WARN, "failed to get macro block ctx", K(ret), K(cur_idx_));
 | |
|       } else {
 | |
|         if (cur_idx_ == sstable_->get_macro_block_ids().count() - 1) {
 | |
|           block_desc.range_.get_end_key().set_max();
 | |
|         } else {
 | |
|           block_desc.range_.get_end_key().assign(full_meta.meta_->endkey_, full_meta.meta_->rowkey_column_number_);
 | |
|         }
 | |
|       }
 | |
|     }
 | |
|   }
 | |
| 
 | |
|   if (OB_SUCC(ret)) {
 | |
|     STORAGE_LOG(DEBUG,
 | |
|         "Success to get macro block id, ",
 | |
|         K_(cur_idx),
 | |
|         "macro_block_id",
 | |
|         sstable_->get_macro_block_ids().at(cur_idx_));
 | |
|     block_desc.block_idx_ = cur_idx_;
 | |
|     block_desc.data_version_ = full_meta.meta_->data_version_;
 | |
|     block_desc.row_store_type_ = full_meta.meta_->row_store_type_;
 | |
|     if (check_lob_) {
 | |
|       block_desc.block_cnt_ = sstable_->get_total_macro_blocks().count();
 | |
|     } else {
 | |
|       block_desc.block_cnt_ = sstable_->get_macro_block_ids().count();
 | |
|     }
 | |
|     block_desc.full_meta_ = full_meta;
 | |
|     block_desc.range_.get_border_flag().unset_inclusive_start();
 | |
|     block_desc.range_.get_border_flag().set_inclusive_end();
 | |
|     block_desc.schema_version_ = full_meta.meta_->schema_version_;
 | |
|     block_desc.data_seq_ = full_meta.meta_->data_seq_;
 | |
|     block_desc.row_count_ = full_meta.meta_->row_count_;
 | |
|     block_desc.occupy_size_ = full_meta.meta_->occupy_size_;
 | |
|     block_desc.micro_block_count_ = full_meta.meta_->micro_block_count_;
 | |
|     block_desc.data_checksum_ = full_meta.meta_->data_checksum_;
 | |
|     block_desc.snapshot_version_ = full_meta.meta_->snapshot_version_;
 | |
|     block_desc.row_count_delta_ = full_meta.meta_->row_count_delta_;
 | |
|     block_desc.progressive_merge_round_ = full_meta.meta_->progressive_merge_round_;
 | |
|     block_desc.contain_uncommitted_row_ = full_meta.meta_->contain_uncommitted_row_;
 | |
|     cur_idx_ += step_;
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int ObMacroBlockIterator::get_macro_block_count(int64_t& macro_block_count)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   if (begin_ < 0 || end_ < 0) {
 | |
|     macro_block_count = 0;
 | |
|   } else {
 | |
|     macro_block_count = end_ - begin_ + 1;
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int ObMacroBlockIterator::locate_macro_block(const ObExtStoreRowkey& ext_rowkey, int64_t& block_idx)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   block_idx = -1;
 | |
| 
 | |
|   if (sstable_->get_macro_block_count() == 0) {
 | |
|     // pass empty sstable
 | |
|   } else if (sstable_->is_rowkey_helper_valid() &&
 | |
|              sstable_->get_rowkey_helper().is_oracle_mode() == share::is_oracle_mode()) {
 | |
|     if (OB_FAIL(sstable_->get_rowkey_helper().locate_block_idx(ext_rowkey, cur_idx_, block_idx))) {
 | |
|       STORAGE_LOG(WARN, "Failed to locate macro block with rowkey helper", K(ret));
 | |
|     }
 | |
|   } else {
 | |
|     ret = locate_macro_block_without_helper(ext_rowkey, block_idx);
 | |
|   }
 | |
| 
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| int ObMacroBlockIterator::locate_macro_block_without_helper(const ObExtStoreRowkey& ext_rowkey, int64_t& block_idx)
 | |
| {
 | |
|   int ret = OB_SUCCESS;
 | |
|   const ObStoreRowkey* cmp_rowkey = NULL;
 | |
|   bool use_collation_free = false;
 | |
|   block_idx = -1;
 | |
| 
 | |
|   if (OB_FAIL(ext_rowkey.check_use_collation_free(sstable_->exist_invalid_collation_free_meta_, use_collation_free))) {
 | |
|     STORAGE_LOG(WARN, "Fail to check use collation free, ", K(ret), K(ext_rowkey));
 | |
|   } else {
 | |
|     ObSSTable::MacroBlockArray::iterator begin = sstable_->meta_.macro_block_array_.begin();
 | |
|     ObSSTable::MacroBlockArray::iterator end = sstable_->meta_.macro_block_array_.end();
 | |
| 
 | |
|     comparor_.reset();
 | |
|     comparor_.set_use_collation_free(use_collation_free);
 | |
|     comparor_.set_sstable(*sstable_);
 | |
|     if (use_collation_free) {
 | |
|       cmp_rowkey = &(ext_rowkey.get_collation_free_store_rowkey());
 | |
|     } else {
 | |
|       cmp_rowkey = &(ext_rowkey.get_store_rowkey());
 | |
|     }
 | |
|     comparor_.set_prefix_check(cmp_rowkey->get_obj_cnt() < sstable_->meta_.rowkey_column_count_);
 | |
|     if (!comparor_.is_prefix_check() && cur_idx_ >= 0 && cur_idx_ < sstable_->get_macro_block_ids().count()) {
 | |
|       // the rowkey may be in the same macro block, so check last macro block first
 | |
|       if (!comparor_(sstable_->get_macro_block_ids().at(cur_idx_), *cmp_rowkey)) {
 | |
|         if (0 == cur_idx_ || comparor_(sstable_->get_macro_block_ids().at(cur_idx_ - 1), *cmp_rowkey)) {
 | |
|           block_idx = cur_idx_;
 | |
|         }
 | |
|       }
 | |
|     }
 | |
| 
 | |
|     if (-1 == block_idx) {
 | |
|       // binary search
 | |
|       ObSSTable::MacroBlockArray::iterator iter = std::lower_bound(begin, end, *cmp_rowkey, comparor_);
 | |
|       if (OB_FAIL(comparor_.get_ret())) {
 | |
|         STORAGE_LOG(ERROR, "Fail to find macro, ", K(ret));
 | |
|       } else {
 | |
|         if (iter < sstable_->meta_.macro_block_array_.end()) {
 | |
|           block_idx = iter - sstable_->meta_.macro_block_array_.begin();
 | |
|         }
 | |
|       }
 | |
|     }
 | |
|   }
 | |
|   return ret;
 | |
| }
 | |
| 
 | |
| }  // namespace storage
 | |
| }  // namespace oceanbase
 | 
