[FEAT MERGE] Transaction processing query optimization

Co-authored-by: haitaoyang <haitaoy3.14@gmail.com>
This commit is contained in:
mjhmllover 2024-06-18 03:05:13 +00:00 committed by ob-robot
parent b29a7dd782
commit 024359ccf0
65 changed files with 4049 additions and 700 deletions

View File

@ -34,6 +34,7 @@ public:
virtual void SetUp();
virtual void TearDown();
protected:
int prepare_tmp_rowkey(ObDatumRowkey &rowkey);
ObFixedArray<ObColDesc, common::ObIAllocator> full_index_cols_;
};
@ -74,6 +75,21 @@ void TestIndexBlockRowScanner::TearDown()
TestIndexBlockDataPrepare::TearDown();
}
int TestIndexBlockRowScanner::prepare_tmp_rowkey(ObDatumRowkey &rowkey)
{
int ret = OB_SUCCESS;
// schema_rowkey + mvcc
const int64_t index_rokwey_cnt = TEST_ROWKEY_COLUMN_CNT + 2;
void *buf = allocator_.alloc(sizeof(ObStorageDatum) * index_rokwey_cnt);
if (OB_ISNULL(buf)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
} else {
ObStorageDatum *datums = new (buf) ObStorageDatum[index_rokwey_cnt];
ret = rowkey.assign(datums, index_rokwey_cnt);
}
return ret;
}
TEST_F(TestIndexBlockRowScanner, transform)
{
ObIndexBlockDataTransformer index_block_transformer;
@ -84,8 +100,11 @@ TEST_F(TestIndexBlockRowScanner, transform)
const ObIndexBlockDataHeader *idx_blk_header
= reinterpret_cast<const ObIndexBlockDataHeader *>(root_block_data_buf_.get_extra_buf());
ASSERT_TRUE(idx_blk_header->is_valid());
ObDatumRowkey tmp_rowkey;
ASSERT_EQ(OB_SUCCESS, prepare_tmp_rowkey(tmp_rowkey));
for (int64_t i = 0; i < idx_blk_header->row_cnt_; ++i) {
STORAGE_LOG(INFO, "Show transformed root block rowkey", K(idx_blk_header->rowkey_array_[i]));
ASSERT_EQ(OB_SUCCESS, idx_blk_header->rowkey_vector_->get_rowkey(i, tmp_rowkey));
STORAGE_LOG(INFO, "Show transformed root block rowkey", K(tmp_rowkey));
}
ObDatumRow row;
ASSERT_EQ(OB_SUCCESS, row.init(allocator_, TEST_COLUMN_CNT));
@ -93,7 +112,8 @@ TEST_F(TestIndexBlockRowScanner, transform)
ObDatumRowkey last_rowkey;
ASSERT_EQ(OB_SUCCESS, last_rowkey.assign(row.storage_datums_, TEST_ROWKEY_COLUMN_CNT));
ObDatumRowkey index_rowkey;
ASSERT_EQ(OB_SUCCESS, index_rowkey.assign(idx_blk_header->rowkey_array_[idx_blk_header->row_cnt_ - 1].datums_, TEST_ROWKEY_COLUMN_CNT));
ASSERT_EQ(OB_SUCCESS, idx_blk_header->rowkey_vector_->get_rowkey(idx_blk_header->row_cnt_ - 1, tmp_rowkey));
ASSERT_EQ(OB_SUCCESS, index_rowkey.assign(tmp_rowkey.datums_, TEST_ROWKEY_COLUMN_CNT));
ASSERT_EQ(last_rowkey, index_rowkey);
// Test update index block on deep copy
@ -109,10 +129,14 @@ TEST_F(TestIndexBlockRowScanner, transform)
ASSERT_TRUE(new_idx_blk_header.is_valid());
ASSERT_EQ(new_idx_blk_header.row_cnt_, idx_blk_header->row_cnt_);
ASSERT_EQ(new_idx_blk_header.col_cnt_, idx_blk_header->col_cnt_);
ObDatumRowkey tmp_new_rowkey;
ASSERT_EQ(OB_SUCCESS, prepare_tmp_rowkey(tmp_new_rowkey));
for (int64_t i = 0; i < idx_blk_header->row_cnt_; ++i) {
ASSERT_EQ(OB_SUCCESS, idx_blk_header->rowkey_vector_->get_rowkey(i, tmp_rowkey));
ASSERT_EQ(OB_SUCCESS, new_idx_blk_header.rowkey_vector_->get_rowkey(i, tmp_new_rowkey));
STORAGE_LOG(INFO, "cmp deep copy rowkey", K(i),
K(new_idx_blk_header.rowkey_array_[i]), K(idx_blk_header->rowkey_array_[i]));
ASSERT_EQ(new_idx_blk_header.rowkey_array_[i], idx_blk_header->rowkey_array_[i]);
K(tmp_new_rowkey), K(tmp_rowkey));
ASSERT_EQ(tmp_new_rowkey, tmp_rowkey);
}
// clear src block extra buf
@ -121,7 +145,8 @@ TEST_F(TestIndexBlockRowScanner, transform)
ObIndexBlockRowParser idx_row_parser;
for (int64_t i = 0; i < new_idx_blk_header.row_cnt_; ++i) {
idx_row_parser.reset();
const ObDatumRowkey &rowkey = new_idx_blk_header.rowkey_array_[i];
ASSERT_EQ(OB_SUCCESS, new_idx_blk_header.rowkey_vector_->get_rowkey(i, tmp_new_rowkey));
const ObDatumRowkey &rowkey = tmp_new_rowkey;
for (int64_t j = 0; j < rowkey.get_datum_cnt(); ++j) {
ASSERT_NE(nullptr, rowkey.datums_[j].ptr_);
}
@ -164,8 +189,11 @@ TEST_F(TestIndexBlockRowScanner, prefetch_and_scan)
ASSERT_EQ(OB_SUCCESS, root_blk_header->get_index_data(root_row_id, index_data_ptr, index_data_len));
ASSERT_EQ(OB_SUCCESS, idx_row_parser.init(index_data_ptr, index_data_len));
ASSERT_EQ(OB_SUCCESS, idx_row_parser.get_header(idx_row_header));
ObDatumRowkey tmp_rowkey;
ASSERT_EQ(OB_SUCCESS, prepare_tmp_rowkey(tmp_rowkey));
ASSERT_EQ(OB_SUCCESS, root_blk_header->rowkey_vector_->get_rowkey(root_row_id, tmp_rowkey));
ObMicroIndexInfo idx_row;
idx_row.endkey_ = &root_blk_header->rowkey_array_[root_row_id];
idx_row.endkey_.set_compact_rowkey(&tmp_rowkey);
idx_row.row_header_ = idx_row_header;
ASSERT_EQ(OB_SUCCESS, index_block_cache.prefetch(
table_schema_.get_tenant_id(),
@ -197,11 +225,11 @@ TEST_F(TestIndexBlockRowScanner, prefetch_and_scan)
ASSERT_EQ(OB_SUCCESS, idx_scanner.open(
ObIndexBlockRowHeader::DEFAULT_IDX_ROW_MACRO_ID,
*read_block,
root_blk_header->rowkey_array_[root_row_id]));
tmp_rowkey));
ASSERT_EQ(OB_SUCCESS, raw_idx_scanner.open(
ObIndexBlockRowHeader::DEFAULT_IDX_ROW_MACRO_ID,
*raw_block,
root_blk_header->rowkey_array_[root_row_id]));
tmp_rowkey));
idx_scanner.reuse();
raw_idx_scanner.reuse();

View File

@ -65,6 +65,7 @@ ob_set_subtarget(ob_share cache
cache/ob_working_set_mgr.cpp
cache/ob_kvcache_hazard_version.cpp
cache/ob_kvcache_pre_warmer.cpp
cache/ob_kvcache_pointer_swizzle.cpp
)
ob_set_subtarget(ob_share common

View File

@ -334,6 +334,8 @@ public:
other.mb_handle_ = nullptr;
other.reset();
}
inline ObKVMemBlockHandle* get_mb_handle() const { return mb_handle_; }
inline void set_mb_handle(ObKVMemBlockHandle *mb_handle) { mb_handle_ = mb_handle; }
TO_STRING_KV(KP_(mb_handle));
private:
template<class Key, class Value> friend class ObIKVCache;

View File

@ -0,0 +1,162 @@
/* Copyright (c) 2023 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_kvcache_pointer_swizzle.h"
#include "ob_kvcache_inst_map.h"
#include "lib/utility/ob_macro_utils.h"
#include "storage/blocksstable/ob_micro_block_cache.h"
namespace oceanbase
{
namespace common
{
ObPointerSwizzleNode::ObPointerSwizzleNode()
: node_version_({0}),
mb_handle_(nullptr),
value_(nullptr),
seq_num_(0)
{}
ObPointerSwizzleNode::~ObPointerSwizzleNode()
{}
void ObPointerSwizzleNode::operator=(const ObPointerSwizzleNode &ps_node)
{
mb_handle_ = ps_node.mb_handle_;
value_ = ps_node.value_;
seq_num_ = ps_node.seq_num_;
}
int ObPointerSwizzleNode::swizzle(const blocksstable::ObMicroBlockBufferHandle &handle)
{
int ret = OB_SUCCESS;
ObKVMemBlockHandle *mb_handle = handle.get_mb_handle();
const blocksstable::ObMicroBlockCacheValue *value = handle.get_micro_block();
if (nullptr == value || !value->get_block_data().is_normal_block()) {
// We only focus on INDEX_BLOCK and DATA_BLOCK
} else if (nullptr == mb_handle || LFU != mb_handle->policy_) {
// Skip the action of moving memory blocks to prevent invalidating the node memory.
} else if (nullptr != value_) {
// The attempt to check if value_ is not nullptr (a dirty read) indicates that
// swizzling has occurred, then skip it at this time.
} else {
ObPointerSwizzleGuard guard(node_version_);
if (guard.is_multi_thd_safe() && nullptr == mb_handle_ && nullptr == value_) {
set(mb_handle, value);
COMMON_LOG(DEBUG, "swizzle successfully", KPC(this));
}
}
return ret;
}
int ObPointerSwizzleNode::access_mem_ptr(blocksstable::ObMicroBlockBufferHandle &handle)
{
int ret = OB_SUCCESS;
ObPointerSwizzleNode tmp_ps_node;
if (nullptr == value_) {
// The attempt to check if value_ is nullptr (a dirty read) might indicate that
// swizzling has never occurred or that it has already been unswizzled.
ret = OB_READ_NOTHING;
} else if (!load_node(tmp_ps_node)) {
// The multithreaded safe loading of the node failed
ret = OB_READ_NOTHING;
} else if (!add_handle_ref(tmp_ps_node.mb_handle_, tmp_ps_node.seq_num_)) {
// The memory for value_ corresponding to the node has been released;
// the node is reset to improve efficiency, whether the node is reset or not is
// independent of the release of memory for value_.
unswizzle();
ret = OB_READ_NOTHING;
} else {
++tmp_ps_node.mb_handle_->recent_get_cnt_;
ATOMIC_AAF(&tmp_ps_node.mb_handle_->get_cnt_, 1);
tmp_ps_node.mb_handle_->inst_->status_.total_hit_cnt_.inc();
handle.set_mb_handle(tmp_ps_node.mb_handle_);
handle.set_micro_block(reinterpret_cast<const blocksstable::ObMicroBlockCacheValue*>(tmp_ps_node.value_));
COMMON_LOG(DEBUG, "access the memory successfully which the swizzling pointer points to", KPC(this));
}
return ret;
}
void ObPointerSwizzleNode::unswizzle()
{
ObPointerSwizzleGuard guard(node_version_);
if (guard.is_multi_thd_safe() && (nullptr != mb_handle_ || nullptr != value_)) {
reset();
}
}
bool ObPointerSwizzleNode::load_node(ObPointerSwizzleNode &tmp_ps_node)
{
bool bret = false;
ObNodeVersion observed_version(ATOMIC_LOAD(&node_version_.value_));
if (1 == observed_version.write_flag_) {
// There are some threads that are writing to the current node
} else if (FALSE_IT(tmp_ps_node = *this)) {
} else if (ATOMIC_LOAD(&node_version_.value_) != observed_version.value_) {
// Check if there are threads that write to the current node
} else if (nullptr == tmp_ps_node.value_ || nullptr == tmp_ps_node.mb_handle_) {
// Make sure that the value_ is valid
} else {
// There is no need to assign a value to status at this point,
// as subsequent processes will overwrite it
bret = true;
}
return bret;
}
void ObPointerSwizzleNode::reset()
{
mb_handle_ = nullptr;
value_ = nullptr;
seq_num_ = 0;
}
void ObPointerSwizzleNode::set(ObKVMemBlockHandle *mb_handle, const ObIKVCacheValue *value)
{
mb_handle_ = mb_handle;
value_ = value;
seq_num_ = mb_handle->get_seq_num();
}
bool ObPointerSwizzleNode::add_handle_ref(ObKVMemBlockHandle *mb_handle, const uint32_t seq_num)
{
bool bret = false;
if (NULL != mb_handle) {
bret = (OB_SUCCESS == mb_handle->handle_ref_.check_seq_num_and_inc_ref_cnt(seq_num));
}
return bret;
}
ObPointerSwizzleNode::ObPointerSwizzleGuard::ObPointerSwizzleGuard(ObNodeVersion &cur_version)
: cur_version_(cur_version),
is_multi_thd_safe_(false)
{
ObNodeVersion tmp_version(ATOMIC_LOAD(&cur_version_.value_));
ObNodeVersion cmp_version(tmp_version.version_value_, 0);
ObNodeVersion new_version(tmp_version.version_value_ + 1, 1);
// Check the version number while also checking if the top bit is zero.
if (ATOMIC_BCAS(&cur_version_.value_, cmp_version.value_, new_version.value_)) {
is_multi_thd_safe_ = true;
}
}
ObPointerSwizzleNode::ObPointerSwizzleGuard::~ObPointerSwizzleGuard()
{
if (is_multi_thd_safe_) {
// This location can be safely read because writes from other threads have been blocked.
ObNodeVersion targ_version(cur_version_.version_value_, 0);
ATOMIC_STORE(&cur_version_.value_, targ_version.value_);
}
}
}//end namespace common
}//end namespace oceanbase

View File

@ -0,0 +1,73 @@
/**
* Copyright (c) 2023 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.
*/
#ifndef OCEANBASE_CACHE_OB_KVCACHE_POINTER_SWIZZLE_H_
#define OCEANBASE_CACHE_OB_KVCACHE_POINTER_SWIZZLE_H_
#include "share/cache/ob_kvcache_hazard_version.h"
#include "share/cache/ob_kvcache_struct.h"
namespace oceanbase
{
namespace blocksstable
{
class ObMicroBlockBufferHandle;
}
namespace common
{
class ObPointerSwizzleNode final
{
public:
ObPointerSwizzleNode();
~ObPointerSwizzleNode();
void operator=(const ObPointerSwizzleNode &ps_node);
int swizzle(const blocksstable::ObMicroBlockBufferHandle &handle);
int access_mem_ptr(blocksstable::ObMicroBlockBufferHandle &handle);
TO_STRING_KV(K_(seq_num), KPC_(mb_handle), KP_(value), K_(node_version));
private:
void unswizzle();
bool load_node(ObPointerSwizzleNode &tmp_ps_node);
void reset();
void set(ObKVMemBlockHandle *mb_handle, const ObIKVCacheValue *value);
bool add_handle_ref(ObKVMemBlockHandle *mb_handle, const uint32_t seq_num);
private:
union ObNodeVersion
{
uint64_t value_;
struct {
uint64_t version_value_: 63;
uint64_t write_flag_: 1;
};
ObNodeVersion(const uint64_t &value) : value_(value) {}
ObNodeVersion(const uint64_t &version_value, const uint64_t &write_flag) : version_value_(version_value), write_flag_(write_flag) {}
TO_STRING_KV(K_(value), K_(version_value), K_(write_flag));
} node_version_;
ObKVMemBlockHandle *mb_handle_;
const ObIKVCacheValue *value_;
int32_t seq_num_;
private:
class ObPointerSwizzleGuard final
{
public:
ObPointerSwizzleGuard(ObNodeVersion &cur_version);
~ObPointerSwizzleGuard();
bool is_multi_thd_safe() { return is_multi_thd_safe_; }
private:
ObNodeVersion &cur_version_;
bool is_multi_thd_safe_;
};
};
}//end namespace common
}//end namespace oceanbase
#endif //OCEANBASE_CACHE_OB_KVCACHE_POINTER_SWIZZLE_H_

View File

@ -28,7 +28,8 @@ ObDataBlockCachePreWarmer::ObDataBlockCachePreWarmer()
update_step_(0),
kvpair_(nullptr),
inst_handle_(),
cache_handle_()
cache_handle_(),
col_descs_(nullptr)
{
}
@ -44,6 +45,7 @@ void ObDataBlockCachePreWarmer::reset()
rest_size_ = 0;
warm_size_percentage_ = 100;
update_step_ = 0;
col_descs_ = nullptr;
reuse();
}
@ -54,9 +56,10 @@ void ObDataBlockCachePreWarmer::reuse()
cache_handle_.reset();
}
int ObDataBlockCachePreWarmer::init()
int ObDataBlockCachePreWarmer::init(const ObIArray<share::schema::ObColDesc> *col_desc_array)
{
int ret = OB_SUCCESS;
col_descs_ = col_desc_array;
cache_ = &OB_STORE_CACHE.get_block_cache();
warm_size_percentage_ = DATA_BLOCK_CACHE_PERCENTAGE;
inner_update_rest();
@ -194,9 +197,10 @@ ObIndexBlockCachePreWarmer::~ObIndexBlockCachePreWarmer()
{
}
int ObIndexBlockCachePreWarmer::init()
int ObIndexBlockCachePreWarmer::init(const ObIArray<share::schema::ObColDesc> *col_desc_array)
{
int ret = OB_SUCCESS;
col_descs_ = col_desc_array;
cache_ = &OB_STORE_CACHE.get_index_block_cache();
warm_size_percentage_ = INDEX_BLOCK_BASE_PERCENTAGE;
inner_update_rest();
@ -217,7 +221,7 @@ int ObIndexBlockCachePreWarmer::do_reserve_kvpair(
allocator_.reuse();
blocksstable::ObMicroBlockData micro_data(micro_block_desc.get_block_buf(), micro_block_desc.get_block_size());
char *allocated_buf = nullptr;
if (OB_FAIL(idx_transformer_.transform(micro_data, value_.get_block_data(), allocator_, allocated_buf))) {
if (OB_FAIL(idx_transformer_.transform(micro_data, value_.get_block_data(), allocator_, allocated_buf, col_descs_))) {
COMMON_LOG(WARN, "Fail to transform index block to memory format", K(ret));
} else {
kvpair_size = sizeof(blocksstable::ObMicroBlockCacheKey) + value_.size();

View File

@ -29,7 +29,7 @@ public:
virtual ~ObDataBlockCachePreWarmer();
void reset();
void reuse();
int init();
int init(const ObIArray<share::schema::ObColDesc> *col_desc_array);
OB_INLINE bool is_valid() const { return nullptr != cache_; }
int reserve_kvpair(const blocksstable::ObMicroBlockDesc &micro_block_desc, const int64_t level = 0);
int update_and_put_kvpair(const blocksstable::ObMicroBlockDesc &micro_block_desc);
@ -58,6 +58,7 @@ protected:
ObKVCachePair *kvpair_;
ObKVCacheInstHandle inst_handle_;
ObKVCacheHandle cache_handle_;
const ObIArray<share::schema::ObColDesc> *col_descs_;
};
class ObIndexBlockCachePreWarmer : public ObDataBlockCachePreWarmer
@ -65,7 +66,7 @@ class ObIndexBlockCachePreWarmer : public ObDataBlockCachePreWarmer
public:
ObIndexBlockCachePreWarmer();
virtual ~ObIndexBlockCachePreWarmer();
int init();
int init(const ObIArray<share::schema::ObColDesc> *col_desc_array);
protected:
virtual void calculate_base_percentage(const int64_t free_memory) override;
virtual int do_reserve_kvpair(

View File

@ -68,6 +68,7 @@ struct ObKVCachePair
: magic_(KVPAIR_MAGIC_NUM), size_(0), key_(NULL), value_(NULL)
{
}
TO_STRING_KV(K_(magic), K_(size), KP_(key), KP_(value));
};
enum ObKVCachePolicy

View File

@ -49,6 +49,7 @@ ob_set_subtarget(ob_storage blocksstable
blocksstable/ob_datum_row.cpp
blocksstable/ob_datum_rowkey.cpp
blocksstable/ob_data_store_desc.cpp
blocksstable/ob_datum_rowkey_vector.cpp
)
ob_set_subtarget(ob_storage blocksstable_encoding

View File

@ -91,7 +91,7 @@ int ObBlockRowStore::init(const ObTableAccessParam &param)
int ObBlockRowStore::apply_blockscan(
blocksstable::ObIMicroBlockRowScanner &micro_scanner,
const bool can_pushdown,
ObTableStoreStat &table_store_stat)
ObTableScanStoreStat &table_store_stat)
{
int ret = OB_SUCCESS;
int64_t access_count = micro_scanner.get_access_cnt();
@ -116,12 +116,9 @@ int ObBlockRowStore::apply_blockscan(
// Check pushdown filter successed
can_blockscan_ = true;
++table_store_stat.pushdown_micro_access_cnt_;
table_store_stat.pushdown_row_access_cnt_ += access_count;
if (!filter_applied_ || nullptr == pd_filter_info_.filter_) {
table_store_stat.pushdown_row_select_cnt_ += access_count;
} else {
int64_t select_cnt = pd_filter_info_.filter_->get_result()->popcnt();
table_store_stat.pushdown_row_select_cnt_ += select_cnt;
EVENT_ADD(ObStatEventIds::PUSHDOWN_STORAGE_FILTER_ROW_CNT, select_cnt);
}
if (iter_param_->has_lob_column_out()) {

View File

@ -15,7 +15,6 @@
#include "common/object/ob_object.h"
#include "lib/container/ob_bitmap.h"
#include "storage/ob_table_store_stat_mgr.h"
#include "sql/engine/basic/ob_pushdown_filter.h"
namespace oceanbase
@ -32,6 +31,7 @@ struct ObTableAccessContext;
struct ObTableAccessParam;
struct ObTableIterParam;
struct ObStoreRow;
struct ObTableScanStoreStat;
struct ObFilterResult
{
@ -73,7 +73,7 @@ public:
int apply_blockscan(
blocksstable::ObIMicroBlockRowScanner &micro_scanner,
const bool can_pushdown,
ObTableStoreStat &table_store_stat);
ObTableScanStoreStat &table_store_stat);
int get_filter_result(ObFilterResult &res);
OB_INLINE sql::ObPushdownFilterExecutor *get_pd_filter()
{ return pd_filter_info_.filter_; }

View File

@ -138,7 +138,9 @@ int ObIndexTreePrefetcher::init_basic_info(
LOG_WARN("Unexpected state, index_scanner_ is valid at first scan", K(ret), KPC(this), K(index_scanner_), K(iter_param), K(lbt()));
} else {
const ObTablet *cur_tablet = OB_ISNULL(iter_param_->tablet_handle_) ? nullptr : iter_param_->tablet_handle_->get_obj();
index_scanner_.switch_context(sstable, cur_tablet, *datum_utils_, *access_ctx_);
index_scanner_.switch_context(sstable, cur_tablet, *datum_utils_, *access_ctx_,
ObRowkeyVectorHelper::can_use_non_datum_rowkey_vector(sstable.is_normal_cg_sstable(), iter_param_->tablet_id_)
? iter_param_->get_rowkey_col_descs() : nullptr);
}
} else if (OB_FAIL(init_index_scanner(index_scanner_))) {
LOG_WARN("Fail to init index_scanner", K(ret));
@ -282,16 +284,18 @@ int ObIndexTreePrefetcher::lookup_in_index_tree(ObSSTableReadHandle &read_handle
int ObIndexTreePrefetcher::init_index_scanner(ObIndexBlockRowScanner &index_scanner)
{
int ret = OB_SUCCESS;
if (OB_FAIL(index_scanner.init(
if (OB_ISNULL(iter_param_)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid iter param", K(ret), KPC(iter_param_), K(lbt()));
} else if (OB_FAIL(index_scanner.init(
*datum_utils_,
*access_ctx_->stmt_allocator_,
access_ctx_->query_flag_,
sstable_->get_macro_offset(),
sstable_->is_normal_cg_sstable()))) {
sstable_->is_normal_cg_sstable(),
ObRowkeyVectorHelper::can_use_non_datum_rowkey_vector(sstable_->is_normal_cg_sstable(), iter_param_->tablet_id_)
? iter_param_->get_rowkey_col_descs() : nullptr))) {
LOG_WARN("init index scanner fail", K(ret), KPC(sstable_), KP(sstable_));
} else if (OB_ISNULL(iter_param_)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid iter param", K(ret), KPC(iter_param_), K(lbt()));
} else {
const ObTablet *cur_tablet = OB_ISNULL(iter_param_->tablet_handle_) ? nullptr : iter_param_->tablet_handle_->get_obj();
index_scanner.set_iter_param(sstable_, cur_tablet);
@ -315,7 +319,6 @@ bool ObIndexTreePrefetcher::last_handle_hit(const ObMicroIndexInfo &block_info,
}
} else if (micro_handle.match(macro_id, offset, size)) {
EVENT_INC(ObStatEventIds::INDEX_BLOCK_CACHE_HIT);
++access_ctx_->table_store_stat_.index_block_cache_hit_cnt_;
bret = true;
}
return bret;
@ -398,7 +401,8 @@ int ObIndexTreePrefetcher::prefetch_block_data(
index_block_info,
is_data,
true, /* need submit io */
micro_handle))) {
micro_handle,
cur_level_))) {
LOG_WARN("Fail to get micro block handle from handle mgr", K(ret));
} else if (is_rescan() && is_data && micro_handle.in_block_state()) {
last_micro_block_handle_ = micro_handle;
@ -630,7 +634,8 @@ int ObIndexTreeMultiPrefetcher::multi_prefetch()
cur_index_info,
cur_index_info.is_data_block(),
false, /* need submit io */
next_handle))) {
next_handle,
cur_level_))) {
//not in cache yet, stop this rowkey prefetching if it's not the rowkey to be feteched
ret = OB_SUCCESS;
if (is_rowkey_to_fetched) {
@ -954,7 +959,9 @@ int ObIndexTreeMultiPassPrefetcher<DATA_PREFETCH_DEPTH, INDEX_PREFETCH_DEPTH>::s
LOG_WARN("invalid iter param", K(ret), KPC(iter_param_), K(lbt()));
} else {
const ObTablet *cur_tablet = OB_ISNULL(iter_param_->tablet_handle_) ? nullptr : iter_param_->tablet_handle_->get_obj();
tree_handles_[level].index_scanner_.switch_context(sstable, cur_tablet, *datum_utils_, *access_ctx_);
tree_handles_[level].index_scanner_.switch_context(sstable, cur_tablet, *datum_utils_, *access_ctx_,
ObRowkeyVectorHelper::can_use_non_datum_rowkey_vector(sstable.is_normal_cg_sstable(), iter_param_->tablet_id_)
? iter_param_->get_rowkey_col_descs() : nullptr);
}
} else if (OB_FAIL(init_index_scanner(tree_handles_[level].index_scanner_))) {
LOG_WARN("Fail to init index_scanner", K(ret), K(level));
@ -1441,8 +1448,8 @@ public:
} else if (OB_UNLIKELY(!index_info.is_valid() || !rowkey.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid MicroIndexInfo", K(ret), K(index_info), K(rowkey));
} else if (OB_FAIL(index_info.endkey_->compare(rowkey, datum_utils_, cmp_ret, false))) {
LOG_WARN("fail to compare rowkey", K(ret), KPC(index_info.endkey_), K(rowkey));
} else if (OB_FAIL(index_info.endkey_.compare(rowkey, datum_utils_, cmp_ret, false))) {
LOG_WARN("fail to compare rowkey", K(ret), K(index_info.endkey_), K(rowkey));
}
return cmp_ret;
}
@ -1482,10 +1489,10 @@ OB_INLINE static int binary_check_micro_infos(
// move to left if the two rightmost endkeys is same(possible in multiple ranges)
int cmp_ret = 0;
for (; OB_SUCC(ret) && check_border_idx > start; check_border_idx--) {
const ObDatumRowkey *cur_endkey = (micro_infos + check_border_idx)->endkey_;
const ObDatumRowkey *prev_endkey = (micro_infos + check_border_idx - 1)->endkey_;
if (OB_FAIL(cur_endkey->compare(*prev_endkey, datum_utils, cmp_ret, false))) {
LOG_WARN("fail to compare rowkey", K(ret), KPC(cur_endkey), KPC(prev_endkey));
const ObCommonDatumRowkey &cur_endkey = (micro_infos + check_border_idx)->endkey_;
const ObCommonDatumRowkey &prev_endkey = (micro_infos + check_border_idx - 1)->endkey_;
if (OB_FAIL(cur_endkey.compare(prev_endkey, datum_utils, cmp_ret, false))) {
LOG_WARN("fail to compare rowkey", K(ret), K(cur_endkey), K(prev_endkey));
} else if (cmp_ret != 0) {
break;
}
@ -1520,8 +1527,8 @@ int ObIndexTreeMultiPassPrefetcher<DATA_PREFETCH_DEPTH, INDEX_PREFETCH_DEPTH>::c
int64_t start_idx = start_pos % max_micro_handle_cnt_;
int64_t end_idx = end_pos % max_micro_handle_cnt_;
if (!is_reverse) {
if (OB_FAIL(micro_data_infos_[end_idx].endkey_->compare(border_rowkey, datum_utils, cmp_ret, false))) {
LOG_WARN("Fail to compare endkey", K(ret), K(border_rowkey), K(micro_data_infos_[start_idx].endkey_));
if (OB_FAIL(micro_data_infos_[end_idx].endkey_.compare(border_rowkey, datum_utils, cmp_ret, false))) {
LOG_WARN("Fail to compare endkey", K(ret), K(border_rowkey), K(micro_data_infos_[end_idx].endkey_));
} else if (cmp_ret < 0) {
for (int64_t pos = start_pos; pos <= end_pos; pos++) {
micro_data_infos_[pos % max_micro_handle_cnt_].set_blockscan();
@ -1538,7 +1545,7 @@ int ObIndexTreeMultiPassPrefetcher<DATA_PREFETCH_DEPTH, INDEX_PREFETCH_DEPTH>::c
} else {
cmp_ret = 0;
// split to [start_idx, max_micro_handle_cnt_ - 1], [0, end_idx - 1]
if (OB_FAIL(micro_data_infos_[max_micro_handle_cnt_ - 1].endkey_->compare(border_rowkey, datum_utils, cmp_ret, false))) {
if (OB_FAIL(micro_data_infos_[max_micro_handle_cnt_ - 1].endkey_.compare(border_rowkey, datum_utils, cmp_ret, false))) {
LOG_WARN("Fail to compare endkey", K(ret), K(border_rowkey), K(micro_data_infos_[max_micro_handle_cnt_ - 1]));
} else if (cmp_ret < 0) {
for (int64_t idx = start_idx; idx < max_micro_handle_cnt_; idx++) {
@ -1567,7 +1574,7 @@ int ObIndexTreeMultiPassPrefetcher<DATA_PREFETCH_DEPTH, INDEX_PREFETCH_DEPTH>::c
} else {
cmp_ret = 0;
// split to [start_idx, max_micro_handle_cnt_ - 1], [0, end_idx]
if (OB_FAIL(micro_data_infos_[0].endkey_->compare(border_rowkey, datum_utils, cmp_ret, false))) {
if (OB_FAIL(micro_data_infos_[0].endkey_.compare(border_rowkey, datum_utils, cmp_ret, false))) {
LOG_WARN("Fail to compare endkey", K(ret), K(border_rowkey), K(micro_data_infos_[0]));
} else if (cmp_ret > 0) {
for (int64_t idx = start_idx; idx < max_micro_handle_cnt_; idx++) {

View File

@ -22,7 +22,6 @@
#include "storage/blocksstable/ob_row_cache.h"
#include "storage/blocksstable/ob_sstable.h"
#include "storage/access/ob_micro_block_handle_mgr.h"
#include "storage/ob_table_store_stat_mgr.h"
#include "storage/access/ob_rows_info.h"
namespace oceanbase {

View File

@ -13,6 +13,7 @@
#include "storage/blocksstable/ob_micro_block_info.h"
#include "storage/blocksstable/ob_storage_cache_suite.h"
#include "share/cache/ob_kvcache_pointer_swizzle.h"
#include "ob_micro_block_handle_mgr.h"
@ -49,6 +50,19 @@ ObMicroBlockDataHandle::~ObMicroBlockDataHandle()
reset();
}
void ObMicroBlockDataHandle::init(
const uint64_t tenant_id,
const MacroBlockId &macro_id,
const int64_t offset,
const int64_t size,
ObMicroBlockHandleMgr *handle_mgr)
{
tenant_id_ = tenant_id;
macro_block_id_ = macro_id;
micro_info_.set(offset, size);
handle_mgr_ = handle_mgr;
}
void ObMicroBlockDataHandle::reset()
{
if (nullptr != handle_mgr_) {
@ -105,7 +119,7 @@ int ObMicroBlockDataHandle::get_micro_block_data(
macro_reader,
loaded_block_data_,
allocator_))) {
LOG_WARN("Fail to load micro block, ", K(ret), K_(tenant_id), K_(macro_block_id), K_(micro_info));
LOG_WARN("Fail to load micro block", K(ret), K_(tenant_id), K_(macro_block_id), K_(micro_info));
try_release_loaded_block();
} else {
io_handle_.reset();
@ -128,7 +142,7 @@ int ObMicroBlockDataHandle::get_cached_index_block_data(ObMicroBlockData &index_
const ObMicroBlockData *pblock = NULL;
if (NULL == (pblock = cache_handle_.get_block_data())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Fail to get cache block, ", K(ret));
LOG_WARN("Fail to get cache block", K(ret));
} else {
index_block = *pblock;
}
@ -185,10 +199,10 @@ int ObMicroBlockDataHandle::get_loaded_block_data(ObMicroBlockData &block_data)
}
} else if (ObSSTableMicroBlockState::IN_BLOCK_IO == block_state_) {
if (OB_FAIL(io_handle_.wait())) {
LOG_WARN("Fail to wait micro block io, ", K(ret));
LOG_WARN("Fail to wait micro block io", K(ret));
} else if (NULL == (io_buf = io_handle_.get_buffer())) {
ret = OB_INVALID_IO_BUFFER;
LOG_WARN("Fail to get block data, io may be failed, ", K(ret));
LOG_WARN("Fail to get block data, io may be failed", K(ret));
} else {
if (-1 == block_index_) {
//single block io
@ -204,7 +218,7 @@ int ObMicroBlockDataHandle::get_loaded_block_data(ObMicroBlockData &block_data)
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected block state, ", K(ret), K_(block_state));
LOG_WARN("Unexpected block state", K(ret), K_(block_state));
}
return ret;
}
@ -221,6 +235,113 @@ void ObMicroBlockDataHandle::try_release_loaded_block()
is_loaded_block_ = false;
}
/**
* -------------------------------------------------------------------ObCacheMemController----------------------------------------------------------------------
*/
ObCacheMemController::ObCacheMemController()
: update_limit_count_(0),
data_block_submit_io_size_(0),
data_block_use_cache_limit_(DEFAULT_DATA_BLOCK_USE_CACHE_LIMIT),
hold_limit_(HOLD_LIMIT_BASE),
current_hold_size_(0),
use_data_block_cache_(true)
{}
ObCacheMemController::~ObCacheMemController()
{}
void ObCacheMemController::init(const bool enable_limit)
{
if (!enable_limit) {
need_sync_io_func = &ObCacheMemController::need_sync_io_nlimit;
reach_hold_limit_func = &ObCacheMemController::reach_hold_limit_nlimit;
update_data_block_io_size_func = &ObCacheMemController::update_data_block_io_size_nlimit;
} else {
need_sync_io_func = &ObCacheMemController::need_sync_io_limit;
reach_hold_limit_func = &ObCacheMemController::reach_hold_limit_limit;
update_data_block_io_size_func = &ObCacheMemController::update_data_block_io_size_limit;
LOG_INFO("Start cache memory controller", K(enable_limit), KPC(this));
}
}
void ObCacheMemController::reset()
{
update_limit_count_ = 0;
data_block_submit_io_size_ = 0;
data_block_use_cache_limit_ = DEFAULT_DATA_BLOCK_USE_CACHE_LIMIT;
hold_limit_ = HOLD_LIMIT_BASE;
current_hold_size_ = 0;
use_data_block_cache_ = true;
}
void ObCacheMemController::add_hold_size(const int64_t handle_size)
{
current_hold_size_ += handle_size;
}
void ObCacheMemController::dec_hold_size(const int64_t handle_size)
{
current_hold_size_ -= handle_size;
}
bool ObCacheMemController::need_sync_io_limit(
const ObQueryFlag &query_flag,
ObMicroBlockDataHandle &micro_block_handle,
blocksstable::ObIMicroBlockCache *cache,
ObFIFOAllocator &block_io_allocator)
{
bool bret = false;
int ret = OB_SUCCESS;
if (OB_FAIL(update_limit(query_flag))) {
LOG_WARN("Fail to update limit", K(ret));
}
if (current_hold_size_ > hold_limit_) {
LOG_DEBUG("Reach hold limit, submit sync io", K(current_hold_size_), K(hold_limit_));
micro_block_handle.block_state_ = ObSSTableMicroBlockState::NEED_SYNC_IO;
micro_block_handle.allocator_ = &block_io_allocator;
cache->cache_bypass();
bret = true;
}
return bret;
}
bool ObCacheMemController::reach_hold_limit_limit() const
{
return current_hold_size_ >= hold_limit_;
}
void ObCacheMemController::update_data_block_io_size_limit(
const int64_t block_size,
const bool is_data_block,
const bool use_cache)
{
if (is_data_block && use_cache) {
data_block_submit_io_size_ += block_size;
use_data_block_cache_ = data_block_submit_io_size_ <= data_block_use_cache_limit_;
}
}
int ObCacheMemController::update_limit(const ObQueryFlag &query_flag)
{
int ret = OB_SUCCESS;
if (0 == (++update_limit_count_ % UPDATE_INTERVAL)) {
uint64_t tenant_id = MTL_ID();
int64_t tenant_free_memory = lib::get_tenant_memory_limit(tenant_id) - lib::get_tenant_memory_hold(tenant_id);
hold_limit_ = HOLD_LIMIT_BASE + tenant_free_memory / 1024 ;
int64_t cache_washable_size = 0;
if (query_flag.is_use_block_cache()) {
if (OB_FAIL(ObKVGlobalCache::get_instance().get_washable_size(tenant_id, cache_washable_size))) {
LOG_WARN("Fail to get kvcache washable size", K(ret));
} else {
data_block_use_cache_limit_ = tenant_free_memory / 5 + cache_washable_size / 10;
use_data_block_cache_ = data_block_submit_io_size_ <= data_block_use_cache_limit_;
}
}
LOG_DEBUG("Update limit details", K(tenant_id), K(tenant_free_memory), K(cache_washable_size),
K(query_flag.is_use_block_cache()), KPC(this));
}
return ret;
}
/**
* -------------------------------------------------------------------ObMicroBlockHandleMgr----------------------------------------------------------------------
@ -231,13 +352,7 @@ ObMicroBlockHandleMgr::ObMicroBlockHandleMgr()
table_store_stat_(nullptr),
query_flag_(nullptr),
block_io_allocator_(),
update_limit_count_(0),
data_block_submit_io_size_(0),
data_block_use_cache_limit_(DEFAULT_DATA_BLOCK_USE_CACHE_LIMIT),
hold_limit_(HOLD_LIMIT_BASE),
current_hold_size_(0),
use_data_block_cache_(true),
enable_limit_(true),
cache_mem_ctrl_(),
is_inited_(false)
{
}
@ -249,22 +364,18 @@ ObMicroBlockHandleMgr::~ObMicroBlockHandleMgr()
void ObMicroBlockHandleMgr::reset()
{
is_inited_ = false;
if (is_inited_) {
is_inited_ = false;
block_io_allocator_.reset();
cache_mem_ctrl_.reset();
}
data_block_cache_ = nullptr;
index_block_cache_ = nullptr;
table_store_stat_ = nullptr;
query_flag_ = nullptr;
update_limit_count_ = 0;
data_block_submit_io_size_ = 0;
data_block_use_cache_limit_ = DEFAULT_DATA_BLOCK_USE_CACHE_LIMIT;
hold_limit_ = HOLD_LIMIT_BASE;
current_hold_size_ = 0;
use_data_block_cache_ = true;
enable_limit_ = true;
block_io_allocator_.reset();
}
int ObMicroBlockHandleMgr::init(const bool enable_prefetch_limiting, ObTableStoreStat &stat, ObQueryFlag &query_flag)
int ObMicroBlockHandleMgr::init(const bool enable_prefetch_limiting, ObTableScanStoreStat &stat, ObQueryFlag &query_flag)
{
int ret = OB_SUCCESS;
lib::ObMemAttr mem_attr(MTL_ID(), "MicroBlockIO");
@ -278,18 +389,18 @@ int ObMicroBlockHandleMgr::init(const bool enable_prefetch_limiting, ObTableStor
index_block_cache_ = &(OB_STORE_CACHE.get_index_block_cache());
table_store_stat_ = &stat;
query_flag_ = &query_flag;
enable_limit_ = enable_prefetch_limiting;
cache_mem_ctrl_.init(enable_prefetch_limiting);
is_inited_ = true;
}
LOG_DEBUG("micro block handle mgr init details", K(ret), K(is_inited_), K(enable_limit_));
return ret;
}
int ObMicroBlockHandleMgr::get_micro_block_handle(
const ObMicroIndexInfo &index_block_info,
ObMicroIndexInfo &index_block_info,
const bool is_data_block,
const bool need_submit_io,
ObMicroBlockDataHandle &micro_block_handle)
ObMicroBlockDataHandle &micro_block_handle,
int16_t cur_level)
{
int ret = OB_SUCCESS;
const uint64_t tenant_id = MTL_ID();
@ -297,8 +408,13 @@ int ObMicroBlockHandleMgr::get_micro_block_handle(
const int64_t offset = index_block_info.get_block_offset();
const int64_t size = index_block_info.get_block_size();
const ObIndexBlockRowHeader *idx_header = index_block_info.row_header_;
ObPointerSwizzleNode *ps_node = index_block_info.ps_node_;
micro_block_handle.reset();
ObIMicroBlockCache *cache = is_data_block ? data_block_cache_ : index_block_cache_;
bool is_effective = true;
#if defined(__aarch64__)
is_effective = false;
#endif
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
@ -308,135 +424,75 @@ int ObMicroBlockHandleMgr::get_micro_block_handle(
LOG_WARN("Unexpect null index header", K(ret), KP(idx_header));
} else if (OB_FAIL(idx_header->fill_micro_des_meta(true /* deep_copy_key */, micro_block_handle.des_meta_))) {
LOG_WARN("Fail to fill micro block deserialize meta", K(ret));
} else {
int tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(update_limit())) {
LOG_WARN("Fail to update limit", K(tmp_ret));
} else if (FALSE_IT(micro_block_handle.init(tenant_id, macro_id, offset, size, this))) {
} else if (OB_LIKELY(nullptr != ps_node) && is_effective
&& OB_SUCC(ps_node->access_mem_ptr(micro_block_handle.cache_handle_))) {
// get data / index block cache with direct memory pointer
micro_block_handle.block_state_ = ObSSTableMicroBlockState::IN_BLOCK_CACHE;
cache->cache_hit(table_store_stat_->block_cache_hit_cnt_);
LOG_DEBUG("Access memory pointer successfully", K(tenant_id), K(macro_id), K(offset), KPC(ps_node),
K(micro_block_handle.cache_handle_), K(cur_level));
} else if (OB_FAIL(cache->get_cache_block(tenant_id, macro_id, offset, size, micro_block_handle.cache_handle_))) {
// get data / index block cache from disk
if (!need_submit_io) {
} else if (cache_mem_ctrl_.need_sync_io(*query_flag_, micro_block_handle, cache, block_io_allocator_)) {
} else if (OB_FAIL(submit_async_io(cache, tenant_id, index_block_info, is_data_block, micro_block_handle))) {
LOG_WARN("Fail to submit async io for prefetch", K(ret), K(index_block_info), K(micro_block_handle));
}
micro_block_handle.tenant_id_ = tenant_id;
micro_block_handle.macro_block_id_ = macro_id;
micro_block_handle.micro_info_.set(offset, size);
micro_block_handle.handle_mgr_ = this;
if (enable_limit_ && need_submit_io && current_hold_size_ > hold_limit_) {
// reach hold limit, need sync io
LOG_DEBUG("reach hold limit, submit sync io", K(enable_limit_), K(current_hold_size_), K(hold_limit_),
K(macro_id), K(offset), K(size));
micro_block_handle.block_state_ = ObSSTableMicroBlockState::NEED_SYNC_IO;
micro_block_handle.allocator_ = &block_io_allocator_;
cache_bypass(is_data_block);
} else if (OB_FAIL(cache->get_cache_block(tenant_id, macro_id, offset, size, micro_block_handle.cache_handle_))) {
// submit io
LOG_DEBUG("try submit io", K(ret), K(enable_limit_), K(query_flag_->is_use_block_cache()),
K(is_data_block), K(need_submit_io), K(tenant_id), K(macro_id), K(offset), K(size));
if (need_submit_io) {
cache_miss(is_data_block);
ObMacroBlockHandle macro_handle;
bool use_cache = is_data_block ? query_flag_->is_use_block_cache() && use_data_block_cache_
: query_flag_->is_use_block_cache();
if (OB_FAIL(cache->prefetch(tenant_id, macro_id, index_block_info, use_cache,
macro_handle, &block_io_allocator_))) {
LOG_WARN("Fail to prefetch micro block", K(ret), K(index_block_info), K(macro_handle),
K(micro_block_handle));
} else {
micro_block_handle.block_state_ = ObSSTableMicroBlockState::IN_BLOCK_IO;
current_hold_size_ += micro_block_handle.get_handle_size();
micro_block_handle.io_handle_ = macro_handle;
micro_block_handle.allocator_ = &block_io_allocator_;
micro_block_handle.need_release_data_buf_ = true;
if (use_cache && is_data_block) {
update_data_block_io_size(size);
}
}
}
} else {
// hit in data / index block cache
LOG_DEBUG("block cache hit", K(enable_limit_), K(is_data_block), K(tenant_id), K(macro_id), K(offset), K(size));
micro_block_handle.block_state_ = ObSSTableMicroBlockState::IN_BLOCK_CACHE;
current_hold_size_ += micro_block_handle.get_handle_size();
cache_hit(is_data_block);
} else {
// get data / index block cache from cache
LOG_DEBUG("block cache hit", K(is_data_block), K(tenant_id), K(macro_id), K(offset), K(size), K(cur_level));
micro_block_handle.block_state_ = ObSSTableMicroBlockState::IN_BLOCK_CACHE;
cache_mem_ctrl_.add_hold_size(micro_block_handle.get_handle_size());
cache->cache_hit(table_store_stat_->block_cache_hit_cnt_);
if (nullptr == ps_node || !is_effective) {
} else if (OB_FAIL(ps_node->swizzle(micro_block_handle.cache_handle_))) {
LOG_WARN("Fail to swizzle", K(is_data_block), K(tenant_id), K(macro_id), K(offset), K(size), K(cur_level),
K(micro_block_handle), KP(ps_node), KPC(ps_node));
}
}
return ret;
}
int ObMicroBlockHandleMgr::submit_async_io(
blocksstable::ObIMicroBlockCache *cache,
const uint64_t tenant_id,
const ObMicroIndexInfo &index_block_info,
const bool is_data_block,
ObMicroBlockDataHandle &micro_block_handle)
{
int ret = OB_SUCCESS;
const MacroBlockId &macro_id = index_block_info.get_macro_id();
const int64_t size = index_block_info.get_block_size();
cache->cache_miss(table_store_stat_->block_cache_miss_cnt_);
ObMacroBlockHandle macro_handle;
bool is_use_block_cache = query_flag_->is_use_block_cache();
bool use_cache = is_data_block ? is_use_block_cache && cache_mem_ctrl_.get_cache_use_flag()
: is_use_block_cache;
if (OB_FAIL(cache->prefetch(tenant_id, macro_id, index_block_info, use_cache,
macro_handle, &block_io_allocator_))) {
LOG_WARN("Fail to prefetch micro block", K(ret), K(index_block_info), K(macro_handle),
K(micro_block_handle));
} else {
micro_block_handle.block_state_ = ObSSTableMicroBlockState::IN_BLOCK_IO;
cache_mem_ctrl_.add_hold_size(micro_block_handle.get_handle_size());
micro_block_handle.io_handle_ = macro_handle;
micro_block_handle.allocator_ = &block_io_allocator_;
micro_block_handle.need_release_data_buf_ = true;
cache_mem_ctrl_.update_data_block_io_size(size, is_data_block, use_cache);
}
return ret;
}
void ObMicroBlockHandleMgr::dec_hold_size(ObMicroBlockDataHandle &handle)
{
current_hold_size_ -= handle.get_handle_size();
cache_mem_ctrl_.dec_hold_size(handle.get_handle_size());
}
bool ObMicroBlockHandleMgr::reach_hold_limit() const
{
return enable_limit_ && current_hold_size_ >= hold_limit_;
return cache_mem_ctrl_.reach_hold_limit();
}
int ObMicroBlockHandleMgr::update_limit()
{
int ret = OB_SUCCESS;
if (enable_limit_ && 0 == (++update_limit_count_ % UPDATE_INTERVAL)) {
uint64_t tenant_id = MTL_ID();
int64_t tenant_free_memory = lib::get_tenant_memory_limit(tenant_id) - lib::get_tenant_memory_hold(tenant_id);
hold_limit_ = HOLD_LIMIT_BASE + tenant_free_memory / 1024 ;
int64_t cache_washable_size = 0;
if (query_flag_->is_use_block_cache()) {
if (OB_FAIL(ObKVGlobalCache::get_instance().get_washable_size(tenant_id, cache_washable_size))) {
LOG_WARN("Fail to get kvcache washable size", K(ret));
} else {
data_block_use_cache_limit_ = tenant_free_memory / 5 + cache_washable_size / 10;
use_data_block_cache_ = data_block_submit_io_size_ <= data_block_use_cache_limit_;
}
}
LOG_DEBUG("calculate limit", K(tenant_id), K(tenant_free_memory), K(cache_washable_size),
K(query_flag_->is_use_block_cache()));
}
LOG_DEBUG("update limit details", K(enable_limit_), K(update_limit_count_), K(current_hold_size_),
K(hold_limit_), K(data_block_submit_io_size_), K(data_block_use_cache_limit_), K(use_data_block_cache_));
return ret;
}
void ObMicroBlockHandleMgr::update_data_block_io_size(const int64_t block_size)
{
if (enable_limit_) {
data_block_submit_io_size_ += block_size;
use_data_block_cache_ = data_block_submit_io_size_ <= data_block_use_cache_limit_;
}
LOG_DEBUG("update data block io size", K(enable_limit_), K(block_size), K(use_data_block_cache_),
K(data_block_submit_io_size_), K(data_block_use_cache_limit_));
}
void ObMicroBlockHandleMgr::cache_bypass(const bool is_data_block)
{
if (is_data_block) {
EVENT_INC(ObStatEventIds::DATA_BLOCK_READ_CNT);
} else {
EVENT_INC(ObStatEventIds::INDEX_BLOCK_READ_CNT);
}
}
void ObMicroBlockHandleMgr::cache_hit(const bool is_data_block)
{
++table_store_stat_->block_cache_hit_cnt_;
if (is_data_block) {
EVENT_INC(ObStatEventIds::DATA_BLOCK_CACHE_HIT);
} else {
EVENT_INC(ObStatEventIds::INDEX_BLOCK_CACHE_HIT);
++table_store_stat_->index_block_cache_hit_cnt_;
}
}
void ObMicroBlockHandleMgr::cache_miss(const bool is_data_block)
{
++table_store_stat_->block_cache_miss_cnt_;
if (is_data_block) {
EVENT_INC(ObStatEventIds::DATA_BLOCK_READ_CNT);
} else {
EVENT_INC(ObStatEventIds::INDEX_BLOCK_READ_CNT);
++table_store_stat_->index_block_cache_miss_cnt_;
}
}
}
}
}

View File

@ -34,11 +34,17 @@ struct ObSSTableMicroBlockState {
};
};
struct ObTableScanStoreStat;
class ObMicroBlockHandleMgr;
struct ObMicroBlockDataHandle {
ObMicroBlockDataHandle();
virtual ~ObMicroBlockDataHandle();
void init(
const uint64_t tenant_id,
const blocksstable::MacroBlockId &macro_id,
const int64_t offset,
const int64_t size,
ObMicroBlockHandleMgr *handle_mgr);
void reset();
bool match(const blocksstable::MacroBlockId &macro_id,
const int32_t offset,
@ -74,53 +80,121 @@ private:
void try_release_loaded_block();
};
class ObMicroBlockHandleMgr
class ObCacheMemController final
{
public:
ObMicroBlockHandleMgr();
~ObMicroBlockHandleMgr();
void reset();
int init(const bool enable_prefetch_limiting, ObTableStoreStat &stat, ObQueryFlag &query_flag);
int get_micro_block_handle(
const blocksstable::ObMicroIndexInfo &index_block_info,
typedef bool (ObCacheMemController::*need_sync_io_func_ptr)(
const ObQueryFlag &query_flag,
ObMicroBlockDataHandle &micro_block_handle,
blocksstable::ObIMicroBlockCache *cache,
ObFIFOAllocator &block_io_allocator);
typedef bool (ObCacheMemController::*reach_hold_limit_func_ptr)() const;
typedef void (ObCacheMemController::*update_data_block_io_size_func_ptr)(
const int64_t block_size,
const bool is_data_block,
const bool need_submit_io,
ObMicroBlockDataHandle &micro_block_handle);
void dec_hold_size(ObMicroBlockDataHandle &handle);
bool reach_hold_limit() const;
OB_INLINE bool is_valid() const { return is_inited_; }
TO_STRING_KV(K_(is_inited), K_(enable_limit), K_(current_hold_size), K_(hold_limit),
K_(data_block_submit_io_size), K_(data_block_use_cache_limit), K_(update_limit_count),
KPC_(query_flag), KPC_(table_store_stat), KP_(data_block_cache), KP_(index_block_cache));
const bool use_cache);
public:
ObCacheMemController();
~ObCacheMemController();
OB_INLINE void init(const bool enable_limit);
OB_INLINE void reset();
OB_INLINE bool get_cache_use_flag() { return use_data_block_cache_; }
OB_INLINE void add_hold_size(const int64_t handle_size);
OB_INLINE void dec_hold_size(const int64_t handle_size);
public:
OB_INLINE bool need_sync_io(
const ObQueryFlag &query_flag,
ObMicroBlockDataHandle &micro_block_handle,
blocksstable::ObIMicroBlockCache *cache,
ObFIFOAllocator &block_io_allocator)
{
return (this->*need_sync_io_func)(query_flag, micro_block_handle, cache, block_io_allocator);
}
OB_INLINE bool reach_hold_limit() const { return (this->*reach_hold_limit_func)(); }
OB_INLINE void update_data_block_io_size(
const int64_t block_size,
const bool is_data_block,
const bool use_cache)
{
return (this->*update_data_block_io_size_func)(block_size, is_data_block, use_cache);
}
TO_STRING_KV(K_(update_limit_count),
K_(data_block_submit_io_size), K_(data_block_use_cache_limit),
K_(hold_limit), K_(current_hold_size), K_(use_data_block_cache));
private:
int update_limit();
void update_data_block_io_size(const int64_t block_size);
void cache_bypass(const bool is_data_block);
void cache_hit(const bool is_data_block);
void cache_miss(const bool is_data_block);
OB_INLINE bool need_sync_io_nlimit(
const ObQueryFlag &query_flag,
ObMicroBlockDataHandle &micro_block_handle,
blocksstable::ObIMicroBlockCache *cache,
ObFIFOAllocator &block_io_allocator) { return false; }
OB_INLINE bool reach_hold_limit_nlimit() const { return false; }
OB_INLINE void update_data_block_io_size_nlimit(
const int64_t block_size,
const bool is_data_block,
const bool use_cache) {}
private:
OB_INLINE bool need_sync_io_limit(
const ObQueryFlag &query_flag,
ObMicroBlockDataHandle &micro_block_handle,
blocksstable::ObIMicroBlockCache *cache,
ObFIFOAllocator &block_io_allocator);
OB_INLINE bool reach_hold_limit_limit() const;
OB_INLINE void update_data_block_io_size_limit(
const int64_t block_size,
const bool is_data_block,
const bool use_cache);
int update_limit(const ObQueryFlag &query_flag);
private:
need_sync_io_func_ptr need_sync_io_func;
reach_hold_limit_func_ptr reach_hold_limit_func;
update_data_block_io_size_func_ptr update_data_block_io_size_func;
private:
static const int64_t HOLD_LIMIT_BASE = 10L << 20; // 10M
static const int64_t UPDATE_INTERVAL = 100;
static const int64_t DEFAULT_DATA_BLOCK_USE_CACHE_LIMIT = 2L << 20; // 2M
blocksstable::ObDataMicroBlockCache *data_block_cache_;
blocksstable::ObIndexMicroBlockCache *index_block_cache_;
ObTableStoreStat *table_store_stat_;
ObQueryFlag *query_flag_;
ObFIFOAllocator block_io_allocator_;
int64_t update_limit_count_;
int64_t data_block_submit_io_size_;
int64_t data_block_use_cache_limit_;
int64_t hold_limit_;
int64_t current_hold_size_;
bool use_data_block_cache_;
bool enable_limit_;
private:
static const int64_t HOLD_LIMIT_BASE = 10L << 20; // 10M
static const int64_t UPDATE_INTERVAL = 100;
static const int64_t DEFAULT_DATA_BLOCK_USE_CACHE_LIMIT = 2L << 20; // 2M
};
class ObMicroBlockHandleMgr
{
public:
ObMicroBlockHandleMgr();
~ObMicroBlockHandleMgr();
void reset();
int init(const bool enable_prefetch_limiting, ObTableScanStoreStat &stat, ObQueryFlag &query_flag);
int get_micro_block_handle(
blocksstable::ObMicroIndexInfo &index_block_info,
const bool is_data_block,
const bool need_submit_io,
ObMicroBlockDataHandle &micro_block_handle,
int16_t cur_level);
int submit_async_io(
blocksstable::ObIMicroBlockCache *cache,
const uint64_t tenant_id,
const blocksstable::ObMicroIndexInfo &index_block_info,
const bool is_data_block,
ObMicroBlockDataHandle &micro_block_handle);
void dec_hold_size(ObMicroBlockDataHandle &handle);
bool reach_hold_limit() const;
OB_INLINE bool is_valid() const { return is_inited_; }
TO_STRING_KV(K_(is_inited), KP_(table_store_stat), KPC_(query_flag),
K_(cache_mem_ctrl), KP_(data_block_cache), KP_(index_block_cache));
private:
blocksstable::ObDataMicroBlockCache *data_block_cache_;
blocksstable::ObIndexMicroBlockCache *index_block_cache_;
ObTableScanStoreStat *table_store_stat_;
ObQueryFlag *query_flag_;
ObFIFOAllocator block_io_allocator_;
ObCacheMemController cache_mem_ctrl_;
bool is_inited_;
};
}
}
} //storage
} //oceanbase
#endif /* OB_MICRO_BLOCK_HANDLE_MGR_H_ */

View File

@ -247,15 +247,5 @@ int ObMultipleGetMerge::inner_get_next_row(ObDatumRow &row)
return ret;
}
void ObMultipleGetMerge::collect_merge_stat(ObTableStoreStat &stat) const
{
stat.multi_get_stat_.call_cnt_++;
stat.multi_get_stat_.output_row_cnt_ += access_ctx_->table_store_stat_.output_row_cnt_;
if (access_ctx_->query_flag_.is_index_back()) {
stat.index_back_stat_.call_cnt_++;
stat.index_back_stat_.output_row_cnt_ += access_ctx_->table_store_stat_.output_row_cnt_;
}
}
}
}

View File

@ -64,7 +64,6 @@ protected:
virtual int calc_scan_range() override;
virtual int construct_iters() override;
virtual int inner_get_next_row(blocksstable::ObDatumRow &row);
virtual void collect_merge_stat(ObTableStoreStat &stat) const override;
private:
const common::ObIArray<blocksstable::ObDatumRowkey> *rowkeys_;
common::ObSEArray<blocksstable::ObDatumRowkey, common::OB_DEFAULT_MULTI_GET_ROWKEY_NUM> cow_rowkeys_;

View File

@ -439,9 +439,6 @@ int ObMultipleMerge::get_next_row(ObDatumRow *&row)
}
if (OB_SUCC(ret)) {
if (nullptr != access_ctx_->table_scan_stat_) {
access_ctx_->table_scan_stat_->access_row_cnt_++;
}
if (OB_FAIL(fill_group_idx_if_need(unprojected_row_))) {
LOG_WARN("Failed to fill iter idx", K(ret), KPC(access_param_), K(unprojected_row_));
} else if (OB_FAIL(process_fuse_row(not_using_static_engine, unprojected_row_, row))) {
@ -614,9 +611,6 @@ int ObMultipleMerge::get_next_normal_rows(int64_t &count, int64_t capacity)
LOG_WARN("fail to aggregate row", K(ret));
}
}
if (nullptr != access_ctx_->table_scan_stat_) {
access_ctx_->table_scan_stat_->access_row_cnt_++;
}
}
}
}
@ -730,9 +724,6 @@ int ObMultipleMerge::get_next_aggregate_row(ObDatumRow *&row)
LOG_WARN("fail to aggregate row", K(ret));
}
}
if (nullptr != access_ctx_->table_scan_stat_) {
access_ctx_->table_scan_stat_->access_row_cnt_++;
}
}
}
}
@ -772,8 +763,8 @@ void ObMultipleMerge::report_tablet_stat()
int tmp_ret = OB_SUCCESS;
bool report_succ = false; /*placeholder*/
storage::ObTabletStat tablet_stat;
tablet_stat.ls_id_ = access_ctx_->table_store_stat_.ls_id_.id();
tablet_stat.tablet_id_ = access_ctx_->table_store_stat_.tablet_id_.id();
tablet_stat.ls_id_ = access_ctx_->ls_id_.id();
tablet_stat.tablet_id_ = access_ctx_->tablet_id_.id();
tablet_stat.query_cnt_ = 1;
tablet_stat.scan_logical_row_cnt_ = access_ctx_->table_store_stat_.logical_read_cnt_;
tablet_stat.scan_physical_row_cnt_ = access_ctx_->table_store_stat_.physical_read_cnt_;

View File

@ -20,7 +20,6 @@
#include "storage/ob_i_store.h"
#include "storage/ob_row_fuse.h"
#include "ob_store_row_iterator.h"
#include "storage/ob_table_store_stat_mgr.h"
#include "share/schema/ob_table_param.h"
#include "ob_table_scan_range.h"
#include "storage/tablet/ob_table_store_util.h"
@ -79,7 +78,6 @@ protected:
virtual int inner_get_next_row(blocksstable::ObDatumRow &row) = 0;
virtual int inner_get_next_rows() { return OB_SUCCESS; };
virtual int can_batch_scan(bool &can_batch) { can_batch = false; return OB_SUCCESS; }
virtual void collect_merge_stat(ObTableStoreStat &stat) const = 0;
int add_iterator(ObStoreRowIterator &iter); // for unit test
const ObTableIterParam * get_actual_iter_param(const ObITable *table) const;
int project_row(const blocksstable::ObDatumRow &unprojected_row,
@ -186,13 +184,12 @@ OB_INLINE int ObMultipleMerge::update_and_report_tablet_stat()
{
int ret = OB_SUCCESS;
EVENT_ADD(ObStatEventIds::STORAGE_READ_ROW_COUNT, scan_cnt_);
access_ctx_->table_store_stat_.access_row_cnt_ += row_stat_.filt_del_count_;
if (NULL != access_ctx_->table_scan_stat_) {
access_ctx_->table_scan_stat_->access_row_cnt_ += row_stat_.filt_del_count_;
access_ctx_->table_scan_stat_->rowkey_prefix_ = access_ctx_->table_store_stat_.rowkey_prefix_;
access_ctx_->table_scan_stat_->bf_filter_cnt_ += access_ctx_->table_store_stat_.bf_filter_cnt_;
access_ctx_->table_scan_stat_->bf_access_cnt_ += access_ctx_->table_store_stat_.bf_access_cnt_;
access_ctx_->table_scan_stat_->empty_read_cnt_ += access_ctx_->table_store_stat_.get_empty_read_cnt();
access_ctx_->table_scan_stat_->empty_read_cnt_ += access_ctx_->table_store_stat_.empty_read_cnt_;
access_ctx_->table_scan_stat_->fuse_row_cache_hit_cnt_ += access_ctx_->table_store_stat_.fuse_row_cache_hit_cnt_;
access_ctx_->table_scan_stat_->fuse_row_cache_miss_cnt_ += access_ctx_->table_store_stat_.fuse_row_cache_miss_cnt_;
access_ctx_->table_scan_stat_->block_cache_hit_cnt_ += access_ctx_->table_store_stat_.block_cache_hit_cnt_;
@ -200,12 +197,10 @@ OB_INLINE int ObMultipleMerge::update_and_report_tablet_stat()
access_ctx_->table_scan_stat_->row_cache_hit_cnt_ += access_ctx_->table_store_stat_.row_cache_hit_cnt_;
access_ctx_->table_scan_stat_->row_cache_miss_cnt_ += access_ctx_->table_store_stat_.row_cache_miss_cnt_;
}
if (lib::is_diagnose_info_enabled()) {
collect_merge_stat(access_ctx_->table_store_stat_);
}
if (MTL(compaction::ObTenantTabletScheduler *)->enable_adaptive_compaction()) {
report_tablet_stat();
}
access_ctx_->table_store_stat_.reset();
return ret;
}

View File

@ -142,12 +142,6 @@ int ObMultipleMultiScanMerge::is_range_valid() const
return ret;
}
void ObMultipleMultiScanMerge::collect_merge_stat(ObTableStoreStat &stat) const
{
stat.multi_scan_stat_.call_cnt_++;
stat.multi_scan_stat_.output_row_cnt_ += access_ctx_->table_store_stat_.output_row_cnt_;
}
int ObMultipleMultiScanMerge::construct_iters()
{
int ret = OB_SUCCESS;

View File

@ -33,7 +33,6 @@ protected:
virtual int construct_iters() override;
virtual int inner_get_next_row(blocksstable::ObDatumRow &row);
virtual int is_range_valid() const override;
virtual void collect_merge_stat(ObTableStoreStat &stat) const override;
private:
const ObIArray<blocksstable::ObDatumRange> *ranges_;
common::ObSEArray<blocksstable::ObDatumRange, 32> cow_ranges_;

View File

@ -164,12 +164,6 @@ int ObMultipleScanMerge::prepare()
return ret;
}
void ObMultipleScanMerge::collect_merge_stat(ObTableStoreStat &stat) const
{
stat.single_scan_stat_.call_cnt_++;
stat.single_scan_stat_.output_row_cnt_ += access_ctx_->table_store_stat_.output_row_cnt_;
}
int ObMultipleScanMerge::construct_iters()
{
int ret = OB_SUCCESS;

View File

@ -53,7 +53,6 @@ protected:
virtual int can_batch_scan(bool &can_batch) override;
virtual int is_range_valid() const override;
virtual int prepare() override;
virtual void collect_merge_stat(ObTableStoreStat &stat) const override;
virtual int supply_consume();
virtual int inner_merge_row(blocksstable::ObDatumRow &row);
int set_rows_merger(const int64_t table_cnt);

View File

@ -388,11 +388,5 @@ int ObSingleMerge::inner_get_next_row(ObDatumRow &row)
return ret;
}
void ObSingleMerge::collect_merge_stat(ObTableStoreStat &stat) const
{
stat.single_get_stat_.call_cnt_++;
stat.single_get_stat_.output_row_cnt_ += access_ctx_->table_store_stat_.output_row_cnt_;
}
} /* namespace storage */
} /* namespace oceanbase */

View File

@ -34,7 +34,6 @@ protected:
virtual int construct_iters() override;
virtual int is_range_valid() const override;
virtual int inner_get_next_row(blocksstable::ObDatumRow &row);
virtual void collect_merge_stat(ObTableStoreStat &stat) const override;
private:
virtual int get_table_row(const int64_t table_idx,
const common::ObIArray<ObITable *> &tables,

View File

@ -131,11 +131,8 @@ int ObSSTableRowExister::exist_block_row(ObSSTableReadHandle &read_handle, ObDat
iter_param_->table_id_,
read_handle.micro_handle_->macro_block_id_,
read_handle.rowkey_->get_datum_cnt());
if (read_handle.is_bf_contain_) {
++access_ctx_->table_store_stat_.bf_empty_read_cnt_;
}
}
++access_ctx_->table_store_stat_.exist_row_.empty_read_cnt_;
++access_ctx_->table_store_stat_.empty_read_cnt_;
EVENT_INC(ObStatEventIds::EXIST_ROW_EMPTY_READ);
} else {
if (exist) {
@ -143,7 +140,6 @@ int ObSSTableRowExister::exist_block_row(ObSSTableReadHandle &read_handle, ObDat
} else {
store_row.row_flag_.set_flag(ObDmlFlag::DF_DELETE);
}
++access_ctx_->table_store_stat_.exist_row_.effect_read_cnt_;
EVENT_INC(ObStatEventIds::EXIST_ROW_EFFECT_READ);
}
}

View File

@ -131,11 +131,8 @@ int ObSSTableRowMultiExister::exist_block_row(ObSSTableReadHandle &read_handle,
iter_param_->table_id_,
read_handle.micro_handle_->macro_block_id_,
read_handle.rowkey_->get_datum_cnt());
if (read_handle.is_bf_contain_) {
++access_ctx_->table_store_stat_.bf_empty_read_cnt_;
}
}
++access_ctx_->table_store_stat_.exist_row_.empty_read_cnt_;
++access_ctx_->table_store_stat_.empty_read_cnt_;
EVENT_INC(ObStatEventIds::EXIST_ROW_EMPTY_READ);
} else {
if (exist) {
@ -143,7 +140,6 @@ int ObSSTableRowMultiExister::exist_block_row(ObSSTableReadHandle &read_handle,
} else {
store_row.row_flag_.set_flag(ObDmlFlag::DF_DELETE);
}
++access_ctx_->table_store_stat_.exist_row_.effect_read_cnt_;
EVENT_INC(ObStatEventIds::EXIST_ROW_EFFECT_READ);
}
}

View File

@ -14,7 +14,6 @@
#define OB_STORAGE_OB_STORE_ROW_ITERATOR_H_
#include "storage/ob_i_store.h"
#include "storage/ob_table_store_stat_mgr.h"
#include "storage/column_store/ob_i_cg_iterator.h"
#include "ob_block_row_store.h"

View File

@ -156,7 +156,8 @@ int ObTableAccessContext::build_lob_locator_helper(const ObStoreCtx &ctx,
ret = OB_ALLOCATE_MEMORY_FAILED;
STORAGE_LOG(WARN, "Failed to alloc memory for ObLobLocatorHelper", K(ret));
} else if (FALSE_IT(lob_locator_helper_ = new (buf) ObLobLocatorHelper())) {
} else if (OB_FAIL(lob_locator_helper_->init(table_store_stat_,
} else if (OB_FAIL(lob_locator_helper_->init(tablet_id_.id(),
tablet_id_.id(),
ctx,
ls_id_,
trans_version_range.snapshot_version_))) {
@ -194,9 +195,6 @@ int ObTableAccessContext::init(ObTableScanParam &scan_param,
table_scan_stat_ = &scan_param.main_table_scan_stat_;
limit_param_ = scan_param.limit_param_.is_valid() ? &scan_param.limit_param_ : NULL;
table_scan_stat_->reset();
table_store_stat_.ls_id_ = scan_param.ls_id_;
table_store_stat_.tablet_id_ = scan_param.tablet_id_;
table_store_stat_.table_id_ = scan_param.index_id_;
trans_version_range_ = trans_version_range;
need_scn_ = scan_param.need_scn_;
range_array_pos_ = &scan_param.range_array_pos_;
@ -251,9 +249,6 @@ int ObTableAccessContext::init(const common::ObQueryFlag &query_flag,
trans_version_range_ = trans_version_range;
ls_id_ = ctx.ls_id_;
tablet_id_ = ctx.tablet_id_;
table_store_stat_.ls_id_ = ctx.ls_id_;
table_store_stat_.tablet_id_ = ctx.tablet_id_;
table_store_stat_.table_id_ = ctx.tablet_id_.id(); // TODO (yuanzhe) remove table_id in virtual table
// handle lob types without ObTableScanParam:
// 1. use lob locator instead of full lob data
// 2. without rowkey, since need not send result to dbmslob/client
@ -292,9 +287,6 @@ int ObTableAccessContext::init(const common::ObQueryFlag &query_flag,
trans_version_range_ = trans_version_range;
ls_id_ = ctx.ls_id_;
tablet_id_ = ctx.tablet_id_;
table_store_stat_.ls_id_ = ctx.ls_id_;
table_store_stat_.tablet_id_ = ctx.tablet_id_;
table_store_stat_.table_id_ = ctx.tablet_id_.id(); // TODO (yuanzhe) remove table_id in virtual table
lob_locator_helper_ = nullptr;
if (!micro_block_handle_mgr_.is_valid()
&& OB_FAIL(micro_block_handle_mgr_.init(enable_limit, table_store_stat_, query_flag_))) {

View File

@ -59,6 +59,63 @@ struct ObRowStat
TO_STRING_KV(K_(base_row_count), K_(inc_row_count), K_(merge_row_count), K_(result_row_count), K_(filt_del_count));
};
struct ObTableScanStoreStat
{
ObTableScanStoreStat() { reset(); }
~ObTableScanStoreStat() = default;
OB_INLINE void reset()
{
MEMSET(this, 0, sizeof(ObTableScanStoreStat));
}
public:
OB_INLINE bool enable_get_row_cache() const
{
return row_cache_miss_cnt_ < common::MAX_MULTI_GET_CACHE_AWARE_ROW_NUM
|| row_cache_hit_cnt_ > row_cache_miss_cnt_ / 2;
}
OB_INLINE bool enable_put_row_cache() const
{
return row_cache_put_cnt_ < common::MAX_MULTI_GET_CACHE_AWARE_ROW_NUM;
}
OB_INLINE bool enable_put_fuse_row_cache(const int64_t threshold) const
{
return fuse_row_cache_put_cnt_ < threshold;
}
OB_INLINE bool enable_get_fuse_row_cache(const int64_t threshold) const
{
return fuse_row_cache_miss_cnt_ < threshold
|| fuse_row_cache_hit_cnt_ > fuse_row_cache_miss_cnt_ / 4;
}
OB_INLINE bool enable_bf_cache() const
{
return (bf_access_cnt_ < common::MAX_MULTI_GET_CACHE_AWARE_ROW_NUM
|| bf_filter_cnt_ > (bf_access_cnt_ / 8));
}
TO_STRING_KV(K_(row_cache_hit_cnt), K_(row_cache_miss_cnt), K_(row_cache_put_cnt),
K_(bf_filter_cnt), K_(bf_access_cnt),
K_(block_cache_hit_cnt), K_(block_cache_miss_cnt),
K_(fuse_row_cache_hit_cnt), K_(fuse_row_cache_miss_cnt), K_(fuse_row_cache_put_cnt),
K_(micro_access_cnt), K_(pushdown_micro_access_cnt),
K_(empty_read_cnt), K_(rowkey_prefix),
K_(logical_read_cnt), K_(physical_read_cnt));
int64_t row_cache_hit_cnt_;
int64_t row_cache_miss_cnt_;
int64_t row_cache_put_cnt_;
int64_t bf_filter_cnt_;
int64_t bf_access_cnt_;
int64_t block_cache_hit_cnt_;
int64_t block_cache_miss_cnt_;
int64_t fuse_row_cache_hit_cnt_;
int64_t fuse_row_cache_miss_cnt_;
int64_t fuse_row_cache_put_cnt_;
int64_t micro_access_cnt_;
int64_t pushdown_micro_access_cnt_;
int64_t empty_read_cnt_;
int64_t rowkey_prefix_;
int64_t logical_read_cnt_;
int64_t physical_read_cnt_;
};
struct ObTableAccessContext
{
ObTableAccessContext();
@ -80,9 +137,6 @@ struct ObTableAccessContext
inline bool enable_bf_cache() const {
return query_flag_.is_use_bloomfilter_cache() && table_store_stat_.enable_bf_cache() && !need_scn_ && !tablet_id_.is_ls_inner_tablet();
}
inline bool enable_sstable_bf_cache() const {
return query_flag_.is_use_bloomfilter_cache() && table_store_stat_.enable_sstable_bf_cache() && !need_scn_ && !tablet_id_.is_ls_inner_tablet();
}
inline bool is_multi_version_read(const int64_t snapshot_version) {
return trans_version_range_.snapshot_version_ < snapshot_version;
}
@ -202,7 +256,7 @@ public:
common::ObIAllocator *range_allocator_;
lib::MemoryContext scan_mem_; // scan/rescan level memory entity, only for query
common::ObTableScanStatistic *table_scan_stat_;
ObTableStoreStat table_store_stat_;
ObTableScanStoreStat table_store_stat_;
int64_t out_cnt_;
common::ObVersionRange trans_version_range_;
const common::ObSEArray<int64_t, 4, common::ModulePageAllocator> *range_array_pos_;

View File

@ -97,6 +97,10 @@ public:
{
return (read_info_ != nullptr) ? &read_info_->get_columns_desc() : nullptr;
}
OB_INLINE const ObColDescIArray *get_rowkey_col_descs() const
{
return (rowkey_read_info_ != nullptr) ? &rowkey_read_info_->get_columns_desc() : nullptr;
}
OB_INLINE bool read_with_same_schema() const
{
return is_same_schema_column_;
@ -219,7 +223,7 @@ public:
OB_INLINE bool is_valid() const { return is_inited_ && iter_param_.is_valid(); }
// used for query
int init(const ObTableScanParam &scan_param, const ObTabletHandle &tablet_handle);
// used for merge
// used for merge
int init_merge_param(const uint64_t table_id,
const common::ObTabletID &tablet_id,
const ObITableReadInfo &read_info,

View File

@ -292,13 +292,13 @@ int ObDDLIndexBlockRowIterator::check_blockscan(const ObDatumRowkey &rowkey, boo
}
int ObDDLIndexBlockRowIterator::get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey)
ObCommonDatumRowkey &endkey)
{
int ret = OB_SUCCESS;
bool is_start_key = false;
bool is_end_key = false;
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("Iter not opened yet", K(ret), KPC(this));
@ -307,20 +307,20 @@ int ObDDLIndexBlockRowIterator::get_current(const ObIndexBlockRowHeader *&idx_ro
LOG_WARN("cur tree value is null", K(ret));
} else {
idx_row_header = &(cur_tree_value_->header_);
endkey = &(cur_tree_value_->block_meta_->end_key_);
endkey.set_compact_rowkey(&(cur_tree_value_->block_meta_->end_key_));
}
return ret;
}
int ObDDLIndexBlockRowIterator::inner_get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
int64_t &row_offset)
{
int ret = OB_SUCCESS;
bool is_start_key = false;
bool is_end_key = false;
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
row_offset = 0;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
@ -330,14 +330,14 @@ int ObDDLIndexBlockRowIterator::inner_get_current(const ObIndexBlockRowHeader *&
LOG_WARN("cur tree value is null", K(ret));
} else {
idx_row_header = &(cur_tree_value_->header_);
endkey = &(cur_tree_value_->block_meta_->end_key_);
endkey.set_compact_rowkey(&(cur_tree_value_->block_meta_->end_key_));
row_offset = cur_tree_value_->co_sstable_row_offset_;
}
return ret;
}
int ObDDLIndexBlockRowIterator::get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -347,7 +347,7 @@ int ObDDLIndexBlockRowIterator::get_next(const ObIndexBlockRowHeader *&idx_row_h
{
int ret = OB_SUCCESS;
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
is_scan_left_border = false;
is_scan_right_border = false;
idx_minor_info = nullptr;
@ -361,10 +361,10 @@ int ObDDLIndexBlockRowIterator::get_next(const ObIndexBlockRowHeader *&idx_row_h
ret = OB_NOT_INIT;
LOG_WARN("Iter not opened yet", K(ret), KPC(this));
} else if (OB_FAIL(inner_get_current(idx_row_header, endkey, co_sstable_row_offset))) {
LOG_WARN("read cur idx row failed", K(ret), KPC(idx_row_header), KPC(endkey), K(co_sstable_row_offset));
} else if (OB_UNLIKELY(nullptr == idx_row_header || nullptr == endkey)) {
LOG_WARN("read cur idx row failed", K(ret), KPC(idx_row_header), K(endkey), K(co_sstable_row_offset));
} else if (OB_UNLIKELY(nullptr == idx_row_header || !endkey.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KP(idx_row_header), KP(endkey));
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KP(idx_row_header), K(endkey));
} else if (OB_UNLIKELY((idx_row_header->is_data_index() && !idx_row_header->is_major_node()) ||
idx_row_header->is_pre_aggregated() ||
!idx_row_header->is_major_node())) {
@ -631,15 +631,15 @@ int ObDDLSStableAllRangeIterator::check_blockscan(const ObDatumRowkey &rowkey, b
}
int ObDDLSStableAllRangeIterator::get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey)
ObCommonDatumRowkey &endkey)
{
endkey = cur_index_info_.endkey_;
endkey.set_compact_rowkey(cur_index_info_.endkey_);
idx_row_header = cur_index_info_.idx_row_header_;
return OB_SUCCESS;
}
int ObDDLSStableAllRangeIterator::get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -649,7 +649,7 @@ int ObDDLSStableAllRangeIterator::get_next(const ObIndexBlockRowHeader *&idx_row
{
int ret = OB_SUCCESS;
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
is_scan_left_border = false;
is_scan_right_border = false;
idx_minor_info = nullptr;
@ -671,7 +671,7 @@ int ObDDLSStableAllRangeIterator::get_next(const ObIndexBlockRowHeader *&idx_row
LOG_WARN("Unexpected null index block row endkey", K(ret), K(cur_index_info_));
} else {
idx_row_header = cur_index_info_.idx_row_header_;
endkey = cur_index_info_.endkey_;
endkey.set_compact_rowkey(cur_index_info_.endkey_);
idx_minor_info = cur_index_info_.idx_minor_info_;
agg_row_buf = cur_index_info_.agg_row_buf_;
agg_buf_size = cur_index_info_.agg_buf_size_;
@ -787,15 +787,15 @@ int ObDDLMergeEmptyIterator::check_blockscan(const ObDatumRowkey &rowkey, bool &
}
int ObDDLMergeEmptyIterator::get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey)
ObCommonDatumRowkey &endkey)
{
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
return OB_SUCCESS;
}
int ObDDLMergeEmptyIterator::get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -805,7 +805,7 @@ int ObDDLMergeEmptyIterator::get_next(const ObIndexBlockRowHeader *&idx_row_head
{
int ret = OB_SUCCESS;
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
is_scan_left_border = false;
is_scan_right_border = false;
idx_minor_info = nullptr;
@ -1517,11 +1517,11 @@ int ObDDLMergeBlockRowIterator::locate_range(const ObDatumRange &range,
}
int ObDDLMergeBlockRowIterator::get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey)
ObCommonDatumRowkey &endkey)
{
int ret = OB_SUCCESS;
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("Iter not opened yet", K(ret), KPC(this));
@ -1531,7 +1531,7 @@ int ObDDLMergeBlockRowIterator::get_current(const ObIndexBlockRowHeader *&idx_ro
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid iters count or iter is nll", K(ret), K(iters_));
} else if (OB_FAIL(iters_.at(0)->get_current(idx_row_header, endkey))) {
LOG_WARN("read cur idx row failed", K(ret), KPC(idx_row_header), KPC(endkey), KPC(iters_.at(0)));
LOG_WARN("read cur idx row failed", K(ret), KPC(idx_row_header), K(endkey), KPC(iters_.at(0)));
}
} else {
// get next row from loser tree
@ -1559,7 +1559,7 @@ int ObDDLMergeBlockRowIterator::get_current(const ObIndexBlockRowHeader *&idx_ro
}
int ObDDLMergeBlockRowIterator::get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -1569,7 +1569,7 @@ int ObDDLMergeBlockRowIterator::get_next(const ObIndexBlockRowHeader *&idx_row_h
{
int ret = OB_SUCCESS;
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
is_scan_left_border = false;
is_scan_right_border = false;
idx_minor_info = nullptr;
@ -1592,7 +1592,7 @@ int ObDDLMergeBlockRowIterator::get_next(const ObIndexBlockRowHeader *&idx_row_h
agg_row_buf,
agg_buf_size,
row_offset))) {
LOG_WARN("read cur idx row failed", K(ret), KPC(idx_row_header), KPC(endkey), KPC(iters_.at(0)));
LOG_WARN("read cur idx row failed", K(ret), KPC(idx_row_header), K(endkey), KPC(iters_.at(0)));
}
} else {
// get next row from loser tree
@ -1621,7 +1621,6 @@ int ObDDLMergeBlockRowIterator::supply_consume()
for (int64_t i = 0; OB_SUCC(ret) && i < consumer_cnt_; ++i) {
const int64_t iter_idx = consumers_[i];
const ObIndexBlockRowHeader *idx_row_header = nullptr;
const ObDatumRowkey *endkey = nullptr;
ObIndexBlockRowIterator *cur_iter = iters_.at(iter_idx);
if (OB_ISNULL(cur_iter)) {
ret = OB_ERR_UNEXPECTED;
@ -1657,7 +1656,7 @@ int ObDDLMergeBlockRowIterator::supply_consume()
}
int ObDDLMergeBlockRowIterator::inner_get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -1667,7 +1666,7 @@ int ObDDLMergeBlockRowIterator::inner_get_next(const ObIndexBlockRowHeader *&idx
{
int ret = OB_SUCCESS;
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
is_scan_left_border = false;
is_scan_right_border = false;
idx_minor_info = nullptr;
@ -1682,7 +1681,7 @@ int ObDDLMergeBlockRowIterator::inner_get_next(const ObIndexBlockRowHeader *&idx
ret = OB_ITER_END;
} else {
idx_row_header = first_index_item_.idx_row_header_;
endkey = first_index_item_.rowkey_;
endkey.set_compact_rowkey(first_index_item_.rowkey_);
is_scan_left_border = first_index_item_.is_scan_left_border_;
is_scan_right_border = first_index_item_.is_scan_right_border_;
idx_minor_info = first_index_item_.idx_minor_info_;
@ -1704,7 +1703,7 @@ int ObDDLMergeBlockRowIterator::inner_get_next(const ObIndexBlockRowHeader *&idx
}
if (OB_SUCC(ret)) {
while (OB_SUCC(ret) && !endkey_merger_->empty() && nullptr == endkey) {
while (OB_SUCC(ret) && !endkey_merger_->empty() && !endkey.is_valid()) {
bool skip_iter = false;
if (OB_FAIL(endkey_merger_->top(top_item))) {
LOG_WARN("fail to get top item", K(ret));
@ -1718,9 +1717,9 @@ int ObDDLMergeBlockRowIterator::inner_get_next(const ObIndexBlockRowHeader *&idx
agg_row_buf = top_item->agg_row_buf_;
agg_buf_size = top_item->agg_buf_size_;
row_offset = top_item->row_offset_;
if (OB_UNLIKELY(nullptr == idx_row_header || nullptr == endkey || cur_iter_idx >= iters_.count())) {
if (OB_UNLIKELY(nullptr == idx_row_header || !endkey.is_valid() || cur_iter_idx >= iters_.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KP(idx_row_header), KP(endkey));
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KP(idx_row_header), K(endkey));
} else {
ObIndexBlockRowIterator *cur_iter = iters_.at(cur_iter_idx);
if (OB_ISNULL(cur_iter)) {
@ -1733,8 +1732,8 @@ int ObDDLMergeBlockRowIterator::inner_get_next(const ObIndexBlockRowHeader *&idx
if (OB_SUCC(ret) && !is_reverse_scan_) { // not_reverse_scan can quit early
int tmp_cmp_ret = 0;
if (OB_FAIL(endkey->compare(query_range_.get_end_key(), *datum_utils_, tmp_cmp_ret))) {
LOG_WARN("fail to cmp rowkey", K(ret), K(query_range_.get_end_key()), KPC(endkey), KPC(datum_utils_));
if (OB_FAIL(endkey.compare(query_range_.get_end_key(), *datum_utils_, tmp_cmp_ret))) {
LOG_WARN("fail to cmp rowkey", K(ret), K(query_range_.get_end_key()), K(endkey), KPC(datum_utils_));
} else if (tmp_cmp_ret >= 0) {
// reach endkey, stop get_next
is_iter_finish_ = true;
@ -1805,7 +1804,7 @@ void ObDDLMergeBlockRowIterator::MergeIndexItem::reset()
int ObDDLMergeBlockRowIterator::MergeIndexItem::init(ObIAllocator *allocator,
const ObIndexBlockRowHeader *idx_row_header,
const ObDatumRowkey *endkey,
const ObCommonDatumRowkey &endkey,
const bool is_scan_left_border,
const bool is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *idx_minor_info,
@ -1815,9 +1814,9 @@ int ObDDLMergeBlockRowIterator::MergeIndexItem::init(ObIAllocator *allocator,
const int64_t iter_idx)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(allocator) || OB_ISNULL(idx_row_header) || OB_ISNULL(endkey)) {
if (OB_ISNULL(allocator) || OB_ISNULL(idx_row_header) || !endkey.is_valid()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid arguemen", K(ret), KP(allocator), KP(idx_row_header), KP(endkey), KP(idx_minor_info), KP(agg_row_buf));
LOG_WARN("invalid arguemen", K(ret), KP(allocator), KP(idx_row_header), K(endkey), KP(idx_minor_info), KP(agg_row_buf));
} else {
item_allocator_ = allocator;
void *key_buf = nullptr;
@ -1828,8 +1827,8 @@ int ObDDLMergeBlockRowIterator::MergeIndexItem::init(ObIAllocator *allocator,
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret), K(sizeof(ObDatumRowkey)));
} else if (FALSE_IT(rowkey_ = new (key_buf) ObDatumRowkey())) {
} else if (OB_FAIL(endkey->deep_copy(*rowkey_, *allocator))) {
LOG_WARN("fail to deep copy rowkey", K(ret), KPC(rowkey_), KPC(endkey));
} else if (OB_FAIL(endkey.deep_copy(*rowkey_, *allocator))) {
LOG_WARN("fail to deep copy rowkey", K(ret), KPC(rowkey_), K(endkey));
}
if (OB_FAIL(ret)) {
@ -1900,9 +1899,9 @@ int ObDDLMergeBlockRowIterator::locate_first_endkey()
if (OB_FAIL(endkey_merger_->top(top_item))) {
LOG_WARN("fail to get top item", K(ret));
} else if (OB_LIKELY(endkey_merger_->is_unique_champion())) {
if (OB_UNLIKELY(nullptr == top_item->header_ || nullptr == top_item->end_key_ || top_item->iter_idx_ >= iters_.count())) {
if (OB_UNLIKELY(nullptr == top_item->header_ || !top_item->end_key_.is_valid() || top_item->iter_idx_ >= iters_.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KP(top_item->header_), KP(top_item->end_key_));
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KP(top_item->header_), K(top_item->end_key_));
} else {
ObIndexBlockRowIterator *tmp_iter = iters_.at(top_item->iter_idx_);
if (OB_ISNULL(tmp_iter)) {
@ -1946,17 +1945,17 @@ int ObDDLMergeBlockRowIterator::locate_first_endkey()
if (OB_FAIL(endkey_merger_->top(top_item))) {
LOG_WARN("fail to get top item", K(ret));
} else if (OB_LIKELY(endkey_merger_->is_unique_champion())) {
if (OB_UNLIKELY(nullptr == top_item->header_ || nullptr == top_item->end_key_ || top_item->iter_idx_ >= iters_.count())) {
if (OB_UNLIKELY(nullptr == top_item->header_ || !top_item->end_key_.is_valid() || top_item->iter_idx_ >= iters_.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KP(top_item->end_key_), KP(top_item->header_));
LOG_WARN("Unexpected null index block row header/endkey", K(ret), K(top_item->end_key_), KP(top_item->header_));
} else if (top_item->iter_idx_ == first_index_item_.iter_index_) {
// continuous item from same iter, find
find = true; //first_index_item_
} else {
int tmp_cmp_ret = 0;
// top_item->end_key_ means first_index_item_.start_key
if (OB_FAIL(top_item->end_key_->compare(query_range_.get_end_key(), *datum_utils_, tmp_cmp_ret))) {
LOG_WARN("fail to cmp rowkey", K(ret), K(query_range_.get_end_key()), KPC(top_item->end_key_), KPC(datum_utils_));
if (OB_FAIL(top_item->end_key_.compare(query_range_.get_end_key(), *datum_utils_, tmp_cmp_ret))) {
LOG_WARN("fail to cmp rowkey", K(ret), K(query_range_.get_end_key()), K(top_item->end_key_), KPC(datum_utils_));
} else if (tmp_cmp_ret < 0) {
find = true; //first_index_item_
} else {
@ -2124,7 +2123,7 @@ int ObDDLMergeBlockRowIterator::get_index_row_count(const ObDatumRange &range,
}
} else {
int ret = OB_SUCCESS;
const ObDatumRowkey *endkey = nullptr;
ObCommonDatumRowkey endkey;
const ObIndexBlockRowHeader *idx_row_header = nullptr;
const ObIndexBlockRowMinorMetaInfo *idx_minor_info = nullptr;
const char *idx_data_buf = nullptr;
@ -2135,10 +2134,10 @@ int ObDDLMergeBlockRowIterator::get_index_row_count(const ObDatumRange &range,
bool is_scan_right_border = false;
while (OB_SUCC(ret)) {
if (OB_FAIL(tmp_merge_iter->get_next(idx_row_header, endkey, is_scan_left_border, is_scan_right_border, idx_minor_info, agg_row_buf, agg_buf_size, row_offset))) {
LOG_WARN("get next idx block row failed", K(ret), KP(idx_row_header), KPC(endkey), K(is_reverse_scan_));
} else if (OB_UNLIKELY(nullptr == idx_row_header || nullptr == endkey)) {
LOG_WARN("get next idx block row failed", K(ret), KP(idx_row_header), K(endkey), K(is_reverse_scan_));
} else if (OB_UNLIKELY(nullptr == idx_row_header || !endkey.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KPC(tmp_merge_iter), KP(idx_row_header), KP(endkey));
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KPC(tmp_merge_iter), KP(idx_row_header), K(endkey));
} else {
++index_row_count;
data_row_count += idx_row_header->get_row_count();

View File

@ -37,9 +37,9 @@ public:
const bool is_reverse_scan,
const ObIndexBlockIterParam &iter_param) override;
virtual int get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey) override;
ObCommonDatumRowkey &endkey) override;
virtual int get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -76,7 +76,7 @@ public:
int get_next_meta(const ObDataMacroBlockMeta *&meta);
private:
int inner_get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
int64_t &row_offset/*for co sstable*/);
private:
bool is_iter_start_;
@ -99,9 +99,9 @@ public:
const bool is_reverse_scan,
const ObIndexBlockIterParam &iter_param) override;
virtual int get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey) override;
ObCommonDatumRowkey &endkey) override;
virtual int get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -148,9 +148,9 @@ public:
const bool is_reverse_scan,
const ObIndexBlockIterParam &iter_param) override;
virtual int get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey) override;
ObCommonDatumRowkey &endkey) override;
virtual int get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -187,9 +187,9 @@ public:
const bool is_reverse_scan,
const ObIndexBlockIterParam &iter_param) override;
virtual int get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey) override;
ObCommonDatumRowkey &endkey) override;
virtual int get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -227,7 +227,7 @@ public:
}
int init(ObIAllocator *allocator,
const ObIndexBlockRowHeader *idx_row_header,
const ObDatumRowkey *endkey,
const ObCommonDatumRowkey &endkey,
const bool is_scan_left_border,
const bool is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *idx_minor_info,
@ -270,7 +270,7 @@ private:
const ObIndexBlockIterParam &iter_param);
int init_merger();
int inner_get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,

View File

@ -44,10 +44,10 @@ int ObDDLSSTableMergeLoserTreeCompare::cmp(const ObDDLSSTableMergeLoserTreeItem
if (OB_UNLIKELY(nullptr == datum_utils_)) {
ret = OB_NOT_INIT;
LOG_WARN("ObDirectLoadSSTableScanMergeLoserTreeCompare not init", K(ret), KP(this));
} else if (OB_UNLIKELY(nullptr == lhs.end_key_ || nullptr == rhs.end_key_)) {
} else if (OB_UNLIKELY(!lhs.end_key_.is_valid() || !rhs.end_key_.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid args", K(ret), K(lhs), K(rhs));
} else if (OB_FAIL(lhs.end_key_->compare(*rhs.end_key_, *datum_utils_, tmp_cmp_ret))) {
} else if (OB_FAIL(lhs.end_key_.compare(rhs.end_key_, *datum_utils_, tmp_cmp_ret))) {
LOG_WARN("fail to compare rowkey", K(ret), K(lhs), K(rhs), KPC(datum_utils_));
} else {
cmp_ret = tmp_cmp_ret * (reverse_scan_ ? -1 : 1);

View File

@ -29,7 +29,7 @@ public:
: equal_with_next_(false),
is_scan_left_border_(false),
is_scan_right_border_(false),
end_key_(nullptr),
end_key_(),
header_(nullptr),
iter_idx_(0),
agg_buf_size_(0),
@ -41,7 +41,7 @@ public:
~ObDDLSSTableMergeLoserTreeItem() = default;
void reset()
{
end_key_ = nullptr;
end_key_.reset();
header_ = nullptr;
idx_minor_info_ = nullptr;
agg_row_buf_ = nullptr;
@ -52,13 +52,13 @@ public:
is_scan_left_border_ = false;
is_scan_right_border_ = false;
}
TO_STRING_KV(K_(equal_with_next), KPC_(end_key), KPC(header_), K_(iter_idx), K_(is_scan_left_border), K_(is_scan_right_border),
TO_STRING_KV(K_(equal_with_next), K_(end_key), KPC(header_), K_(iter_idx), K_(is_scan_left_border), K_(is_scan_right_border),
K_(agg_buf_size), K_(row_offset), KP_(idx_minor_info), KP_(agg_row_buf));
public:
bool equal_with_next_; // for simple row merger
bool is_scan_left_border_;
bool is_scan_right_border_;
const blocksstable::ObDatumRowkey *end_key_;
ObCommonDatumRowkey end_key_;
const blocksstable::ObIndexBlockRowHeader *header_;
int64_t iter_idx_;
int64_t agg_buf_size_;

View File

@ -1340,7 +1340,10 @@ int ObBaseIndexBlockBuilder::init(const ObDataStoreDesc &data_store_desc,
} else {
if (need_pre_warm() && index_store_desc.get_tablet_id().is_user_tablet()) {
int tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(index_block_pre_warmer_.init())) {
if (OB_TMP_FAIL(index_block_pre_warmer_.init(ObRowkeyVectorHelper::can_use_non_datum_rowkey_vector(
index_store_desc_->is_cg(), index_store_desc_->get_tablet_id()) ?
&index_store_desc_->get_rowkey_col_descs()
: nullptr))) {
STORAGE_LOG(WARN, "Failed to init index block prewarmer", K(tmp_ret));
}
}

View File

@ -13,6 +13,7 @@
#define USING_LOG_PREFIX STORAGE
#include "share/schema/ob_column_schema.h"
#include "share/cache/ob_kvcache_pointer_swizzle.h"
#include "ob_index_block_row_scanner.h"
#include "ob_index_block_row_struct.h"
#include "storage/access/ob_rows_info.h"
@ -25,6 +26,7 @@
namespace oceanbase
{
using namespace storage;
using namespace common;
namespace blocksstable
{
@ -36,8 +38,7 @@ int ObIndexBlockDataHeader::get_index_data(
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid row count", K(ret), K(row_idx), K_(row_cnt), KPC(this));
} else {
const int64_t obj_idx = (row_idx + 1) * col_cnt_ - 1;
const ObStorageDatum &datum = datum_array_[obj_idx];
const ObStorageDatum &datum = index_datum_array_[row_idx];
ObString index_data_buf = datum.get_string();
if (OB_UNLIKELY(index_data_buf.empty())) {
ret = OB_ERR_UNEXPECTED;
@ -47,6 +48,9 @@ int ObIndexBlockDataHeader::get_index_data(
index_len = index_data_buf.length();
}
}
if (OB_SUCC(ret)) {
LOG_DEBUG("get index data", K(ret), K(row_idx), KPC(this));
}
return ret;
}
@ -57,35 +61,35 @@ int ObIndexBlockDataHeader::deep_copy_transformed_index_block(
int64_t &pos)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!header.is_valid() || buf_size < 0 || pos >= buf_size || pos + header.data_buf_size_ > buf_size)
if (OB_UNLIKELY(!header.is_valid() || buf_size < 0 || pos >= buf_size || header.data_buf_size_ > buf_size)
|| OB_ISNULL(buf)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid argument for copy transformed index block", K(ret), KP(buf),
K(header), K(buf_size), K(pos));
} else {
char *data_buf = buf + pos;
const int64_t datum_cnt = header.row_cnt_ * header.col_cnt_;
ObDatumRowkey *rowkey_arr = new (buf + pos) ObDatumRowkey[header.row_cnt_];
pos += sizeof(ObDatumRowkey) * header.row_cnt_;
ObStorageDatum *datum_arr = new (buf + pos) ObStorageDatum[datum_cnt];
pos += sizeof(ObStorageDatum) * datum_cnt;
for (int64_t row_idx = 0; OB_SUCC(ret) && row_idx < header.row_cnt_; ++row_idx) {
if (OB_FAIL(rowkey_arr[row_idx].assign(datum_arr + row_idx * header.col_cnt_, header.col_cnt_ - 1))) {
LOG_WARN("Fail to assign datum array to rowkey", K(ret), K(header));
}
for (int64_t col_idx = 0; OB_SUCC(ret) && col_idx < header.col_cnt_; ++col_idx) {
const int64_t datum_idx = row_idx * header.col_cnt_ + col_idx;
if (OB_FAIL(datum_arr[datum_idx].deep_copy(header.datum_array_[datum_idx], buf, buf_size, pos))) {
LOG_WARN("Fail to deep copy datum", K(ret), K(datum_idx), K(header), K(header.datum_array_[datum_idx]));
ObRowkeyVector *rowkey_vector = new (buf + pos) ObRowkeyVector();
pos += sizeof(ObRowkeyVector);
ObStorageDatum *index_datum_array = new (buf + pos) ObStorageDatum [header.row_cnt_];
pos += sizeof(ObStorageDatum) * header.row_cnt_;
common::ObPointerSwizzleNode *ps_node_arr = new (buf + pos) common::ObPointerSwizzleNode[header.row_cnt_];
pos += sizeof(common::ObPointerSwizzleNode) * header.row_cnt_;
if (OB_FAIL(rowkey_vector->deep_copy(buf, pos, buf_size, *header.rowkey_vector_))) {
LOG_WARN("Failed to deep copy rowkey vector", K(ret));
} else {
for (int64_t row_idx = 0; OB_SUCC(ret) && row_idx < header.row_cnt_; ++row_idx) {
ps_node_arr[row_idx] = header.ps_node_array_[row_idx];
if (OB_FAIL(index_datum_array[row_idx].deep_copy(header.index_datum_array_[row_idx], buf, buf_size, pos))) {
LOG_WARN("Failed to deep copy storage datum to buf", K(ret), K(row_idx));
}
}
}
if (OB_SUCC(ret)) {
rowkey_vector_ = rowkey_vector;
index_datum_array_ = index_datum_array;
row_cnt_ = header.row_cnt_;
col_cnt_ = header.col_cnt_;
rowkey_array_ = rowkey_arr;
datum_array_ = datum_arr;
ps_node_array_ = ps_node_arr;
data_buf_ = data_buf;
data_buf_size_ = header.data_buf_size_;
}
@ -105,7 +109,8 @@ int ObIndexBlockDataTransformer::transform(
const ObMicroBlockData &block_data,
ObMicroBlockData &transformed_data,
ObIAllocator &allocator,
char *&allocated_buf)
char *&allocated_buf,
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs)
{
int ret = OB_SUCCESS;
ObDatumRow row;
@ -118,7 +123,10 @@ int ObIndexBlockDataTransformer::transform(
const int64_t col_cnt = micro_block_header->column_count_;
const int64_t row_cnt = micro_block_header->row_count_;
int64_t mem_limit = 0;
if (OB_UNLIKELY(!block_data.is_valid() || !micro_block_header->is_valid())) {
if (OB_UNLIKELY(nullptr != rowkey_col_descs && col_cnt - 1 > rowkey_col_descs->count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected rowkey count", K(ret), K(col_cnt), KPC(rowkey_col_descs));
} else if (OB_UNLIKELY(!block_data.is_valid() || !micro_block_header->is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid argument", K(ret), K(block_data), KPC(micro_block_header));
} else if (OB_FAIL(get_reader(block_data.get_store_type(), micro_reader))) {
@ -127,9 +135,9 @@ int ObIndexBlockDataTransformer::transform(
LOG_WARN("Fail to init micro block reader", K(ret), K(block_data));
} else if (OB_FAIL(row.init(allocator, col_cnt))) {
LOG_WARN("Failed to init datum row", K(ret), K(col_cnt));
} else if (OB_FAIL(get_transformed_upper_mem_size(block_data.get_buf(), mem_limit))) {
} else if (OB_FAIL(get_transformed_upper_mem_size(rowkey_col_descs, block_data.get_buf(), mem_limit))) {
LOG_WARN("Failed to get upper bound of transformed block size", K(ret), K(block_data));
} else if (OB_ISNULL(block_buf = static_cast<char *>(allocator.alloc(mem_limit)))) {
} else if (OB_ISNULL(block_buf = static_cast<char *>(allocator.alloc(mem_limit)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("Failed to allocate memory for transformed block buf", K(ret), K(mem_limit));
} else if (OB_FAIL(micro_block_header->deep_copy(block_buf, mem_limit, pos, new_micro_header))) {
@ -140,40 +148,52 @@ int ObIndexBlockDataTransformer::transform(
LOG_WARN("invalid copied micro block header", K(ret), KPC(new_micro_header));
} else {
const int64_t micro_header_size = pos;
const int64_t datum_cnt = row_cnt * col_cnt;
ObIndexBlockDataHeader *idx_header = new (block_buf + pos) ObIndexBlockDataHeader();
pos += sizeof(ObIndexBlockDataHeader);
ObDatumRowkey *rowkey_arr = new (block_buf + pos) ObDatumRowkey[row_cnt];
pos += sizeof(ObDatumRowkey) * row_cnt;
ObStorageDatum *datum_arr = new (block_buf + pos) ObStorageDatum[datum_cnt];
pos += sizeof(ObStorageDatum) * datum_cnt;
for (int64_t row_idx = 0; OB_SUCC(ret) && row_idx < row_cnt; ++row_idx) {
row.reuse();
if (OB_FAIL(micro_reader->get_row(row_idx, row))) {
LOG_WARN("Fail to get row", K(ret), K(row_idx));
} else {
for (int64_t col_idx = 0; OB_SUCC(ret) && col_idx < col_cnt; ++col_idx) {
const int64_t datum_idx = row_idx * col_cnt + col_idx;
if (OB_UNLIKELY(datum_idx >= datum_cnt)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected datum size overflow", K(ret), K(datum_idx), K(datum_cnt));
} else if (OB_FAIL(datum_arr[datum_idx].deep_copy(
row.storage_datums_[col_idx], block_buf, mem_limit, pos))) {
LOG_WARN("Fail to deep copy storage datum to buf", K(ret), K(row_idx), K(col_idx), K(col_cnt));
ObStorageDatum *index_datum_array = new (block_buf + pos) ObStorageDatum [row_cnt];
pos += sizeof(ObStorageDatum) * row_cnt;
common::ObPointerSwizzleNode *ps_node_arr = new (block_buf + pos) common::ObPointerSwizzleNode[row_cnt];
pos += sizeof(common::ObPointerSwizzleNode) * row_cnt;
ObRowkeyVector *rowkey_vector = nullptr;
if (OB_FAIL(ObRowkeyVector::construct_rowkey_vector(row_cnt,
col_cnt - 1,
rowkey_col_descs,
block_buf,
pos,
mem_limit,
rowkey_vector))) {
LOG_WARN("Failed to construct rowkey vector", K(ret));
} else {
for (int64_t row_idx = 0; OB_SUCC(ret) && row_idx < row_cnt; ++row_idx) {
row.reuse();
if (OB_FAIL(micro_reader->get_row(row_idx, row))) {
LOG_WARN("Failed to get row", K(ret), K(row_idx));
} else {
for (int64_t col_idx = 0; OB_SUCC(ret) && col_idx < col_cnt - 1; ++col_idx) {
if (OB_FAIL(rowkey_vector->columns_[col_idx].fill_column_datum(block_buf,
pos,
mem_limit,
row_idx,
row.storage_datums_[col_idx]))) {
LOG_WARN("Failed to fill column vector", K(ret), K(row_idx), K(col_idx));
}
}
if (FAILEDx(index_datum_array[row_idx].deep_copy(row.storage_datums_[col_cnt - 1], block_buf, mem_limit, pos))) {
LOG_WARN("Failed to deep copy storage datum to buf", K(ret), K(row_idx), K(col_cnt));
}
}
if (FAILEDx(rowkey_arr[row_idx].assign(datum_arr + row_idx * col_cnt, col_cnt - 1))) {
LOG_WARN("Fail to assign datum array to rowkey", K(ret), K(row), K(row_idx));
}
}
if (FAILEDx(rowkey_vector->set_construct_finished())) {
LOG_WARN("Failed to set construct finished", K(ret));
}
}
if (OB_SUCC(ret)) {
idx_header->row_cnt_ = row_cnt;
idx_header->col_cnt_ = col_cnt;
idx_header->rowkey_array_ = rowkey_arr;
idx_header->datum_array_ = datum_arr;
idx_header->rowkey_vector_ = rowkey_vector;
idx_header->index_datum_array_ = index_datum_array;
idx_header->ps_node_array_ = ps_node_arr;
idx_header->data_buf_size_ = pos - micro_header_size;
transformed_data.buf_ = block_buf;
transformed_data.size_ = micro_header_size;
@ -258,6 +278,7 @@ int ObIndexBlockDataTransformer::fix_micro_header_and_transform(
}
int ObIndexBlockDataTransformer::get_transformed_upper_mem_size(
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs,
const char *raw_block_data,
int64_t &mem_limit)
{
@ -269,11 +290,20 @@ int ObIndexBlockDataTransformer::get_transformed_upper_mem_size(
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid argument", K(ret), KP(raw_block_data), KPC(micro_header));
} else {
int64_t rowkey_vector_size = 0;
mem_limit += micro_header->get_serialize_size();
mem_limit += sizeof(ObIndexBlockDataHeader);
mem_limit += micro_header->row_count_ * sizeof(ObDatumRowkey);
mem_limit += micro_header->row_count_ * (sizeof(ObStorageDatum) * micro_header->column_count_);
mem_limit += micro_header->original_length_;
mem_limit += micro_header->row_count_ * sizeof(ObStorageDatum);
mem_limit += micro_header->row_count_ * sizeof(common::ObPointerSwizzleNode);
if (OB_FAIL(ObRowkeyVector::get_occupied_size(micro_header->row_count_,
micro_header->column_count_ - 1,
rowkey_col_descs,
rowkey_vector_size))) {
LOG_WARN("Failed to get occupied size of rowkey vector", K(ret));
} else {
mem_limit += rowkey_vector_size;
mem_limit += micro_header->original_length_;
}
}
return ret;
}
@ -624,11 +654,11 @@ bool ObRAWIndexBlockRowIterator::end_of_block() const
}
int ObRAWIndexBlockRowIterator::get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey)
ObCommonDatumRowkey &endkey)
{
int ret = OB_SUCCESS;
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
const int64_t rowkey_column_count = datum_utils_->get_rowkey_count();
idx_row_parser_.reset();
endkey_.reset();
@ -647,13 +677,13 @@ int ObRAWIndexBlockRowIterator::get_current(const ObIndexBlockRowHeader *&idx_ro
} else if (OB_FAIL(endkey_.assign(datum_row_->storage_datums_, rowkey_column_count))) {
LOG_WARN("Fail to assign storage datum to endkey", K(ret), KPC(datum_row_), K(rowkey_column_count));
} else {
endkey = &endkey_;
endkey.set_compact_rowkey(&endkey_);
}
return ret;
}
int ObRAWIndexBlockRowIterator::get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -663,7 +693,7 @@ int ObRAWIndexBlockRowIterator::get_next(const ObIndexBlockRowHeader *&idx_row_h
{
int ret = OB_SUCCESS;
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
idx_minor_info = nullptr;
agg_row_buf = nullptr;
agg_buf_size = 0;
@ -672,10 +702,10 @@ int ObRAWIndexBlockRowIterator::get_next(const ObIndexBlockRowHeader *&idx_row_h
ret = OB_NOT_INIT;
LOG_WARN("Iter not opened yet", K(ret), KPC(this));
} else if (OB_FAIL(get_current(idx_row_header, endkey))) {
LOG_WARN("read cur idx row failed", K(ret), KPC(idx_row_header), KPC(endkey));
} else if (OB_UNLIKELY(nullptr == idx_row_header || nullptr == endkey)) {
LOG_WARN("read cur idx row failed", K(ret), KPC(idx_row_header), K(endkey));
} else if (OB_UNLIKELY(nullptr == idx_row_header || !endkey.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KP(idx_row_header), KP(endkey));
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KP(idx_row_header), K(endkey));
} else if (idx_row_header->is_data_index() && !idx_row_header->is_major_node()) {
if (OB_FAIL(idx_row_parser_.get_minor_meta(idx_minor_info))) {
LOG_WARN("Fail to get minor meta info", K(ret));
@ -771,7 +801,8 @@ int ObRAWIndexBlockRowIterator::get_index_row_count(const ObDatumRange &range,
/****************** ObTFMIndexBlockRowIterator **********************/
ObTFMIndexBlockRowIterator::ObTFMIndexBlockRowIterator()
: idx_data_header_(nullptr)
: idx_data_header_(nullptr),
cur_node_index_(0)
{
}
@ -785,6 +816,7 @@ void ObTFMIndexBlockRowIterator::reset()
{
ObRAWIndexBlockRowIterator::reset();
idx_data_header_ = nullptr;
cur_node_index_ = 0;
}
void ObTFMIndexBlockRowIterator::reuse()
@ -826,26 +858,21 @@ int ObTFMIndexBlockRowIterator::locate_key(const ObDatumRowkey &rowkey)
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("Iter not opened yet", K(ret), KPC(this));
} else if (OB_UNLIKELY(!rowkey.is_valid() || OB_ISNULL(idx_data_header_))) {
} else if (OB_UNLIKELY(!rowkey.is_valid() || OB_ISNULL(idx_data_header_) || !idx_data_header_->is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid rowkey", K(ret), K(rowkey), KP(idx_data_header_));
} else {
ObDatumComparor<ObDatumRowkey> cmp(*datum_utils_, ret);
const ObDatumRowkey *first = idx_data_header_->rowkey_array_;
const ObDatumRowkey *last = idx_data_header_->rowkey_array_ + idx_data_header_->row_cnt_;
const ObDatumRowkey *found = std::lower_bound(first, last, rowkey, cmp);
if (OB_FAIL(ret)) {
LOG_WARN("fail to get rowkey lower_bound", K(ret), K(rowkey), KPC(idx_data_header_));
} else if (found == last) {
current_ = ObIMicroBlockReaderInfo::INVALID_ROW_INDEX;
} else {
begin_idx = found - first;
}
LOG_TRACE("Binary search rowkey in transformed block", K(ret), KP(found), KPC(first), KP(last),
K(current_), K(rowkey), KPC(idx_data_header_));
LOG_WARN("invalid rowkey", K(ret), K(rowkey), KPC(idx_data_header_));
} else if (OB_FAIL(idx_data_header_->rowkey_vector_->locate_key(0,
idx_data_header_->row_cnt_,
rowkey,
*datum_utils_,
begin_idx))) {
LOG_WARN("Failed to locate key in rowkey vector", K(ret), KPC(idx_data_header_));
} else if (begin_idx == idx_data_header_->row_cnt_) {
begin_idx = ObIMicroBlockReaderInfo::INVALID_ROW_INDEX;
}
if (OB_SUCC(ret)) {
LOG_TRACE("Binary search rowkey in transformed block with rowkey vector", K(ret), K(begin_idx),
K(rowkey), KPC(idx_data_header_));
current_ = begin_idx;
start_ = begin_idx;
end_ = begin_idx;
@ -862,76 +889,19 @@ int ObTFMIndexBlockRowIterator::locate_range(const ObDatumRange &range,
int64_t begin_idx = -1;
int64_t end_idx = -1;
current_ = ObIMicroBlockReaderInfo::INVALID_ROW_INDEX;
bool is_begin_equal = false;
ObDatumComparor<ObDatumRowkey> lower_bound_cmp(*datum_utils_, ret);
ObDatumComparor<ObDatumRowkey> upper_bound_cmp(*datum_utils_, ret, false, false);
const ObDatumRowkey *first = idx_data_header_->rowkey_array_;
const ObDatumRowkey *last = idx_data_header_->rowkey_array_ + idx_data_header_->row_cnt_;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("Iter not opened yet", K(ret), KPC(this));
} else if (OB_UNLIKELY(!range.is_valid() || OB_ISNULL(idx_data_header_))) {
} else if (OB_UNLIKELY(!range.is_valid() || OB_ISNULL(idx_data_header_) || !idx_data_header_->is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid range", K(ret), K(range), KP(idx_data_header_));
LOG_WARN("invalid range", K(ret), K(range), KPC(idx_data_header_));
} else if (OB_FAIL(locate_range_by_rowkey_vector(range, is_left_border, is_right_border, is_normal_cg, begin_idx, end_idx))) {
if (OB_UNLIKELY(OB_BEYOND_THE_RANGE != ret)) {
LOG_WARN("Failed to locate range by rowkey vector", K(ret));
}
} else {
if (!is_left_border || range.get_start_key().is_min_rowkey()) {
begin_idx = 0;
} else {
const ObDatumRowkey *start_found = std::lower_bound(first, last, range.get_start_key(), lower_bound_cmp);
if (OB_FAIL(ret)) {
LOG_WARN("fail to get rowkey lower_bound", K(ret), K(range), KPC(idx_data_header_));
} else if (start_found == last) {
ret = OB_BEYOND_THE_RANGE;
} else if (!range.get_border_flag().inclusive_start()) {
bool is_equal = false;
if (OB_FAIL(start_found->equal(range.get_start_key(), *datum_utils_, is_equal))) {
STORAGE_LOG(WARN, "Failed to check datum rowkey equal", K(ret), K(range), KPC(start_found));
} else if (is_equal) {
++start_found;
if (start_found == last) {
ret = OB_BEYOND_THE_RANGE;
}
}
}
if (OB_SUCC(ret)) {
begin_idx = start_found - first;
}
}
LOG_TRACE("Locate range start key in index block by range", K(ret),
K(range), K(begin_idx), KPC(first), K(idx_data_header_->row_cnt_));
if (OB_FAIL(ret)) {
} else if (!is_right_border || range.get_end_key().is_max_rowkey()) {
end_idx = idx_data_header_->row_cnt_ - 1;
} else {
const ObDatumRowkey *end_found = nullptr;
// TODO remove is_normal_cg_, use flag in header
// no need to use upper_bound for column store
if (!is_normal_cg && range.get_border_flag().inclusive_end()) {
end_found = std::upper_bound(first, last, range.get_end_key(), upper_bound_cmp);
} else {
end_found = std::lower_bound(first, last, range.get_end_key(), lower_bound_cmp);
}
if (OB_FAIL(ret)) {
LOG_WARN("fail to get rowkey lower_bound", K(ret), K(range), KPC(idx_data_header_));
} else if (end_found == last) {
--end_found;
}
if (OB_SUCC(ret)) {
end_idx = end_found - first;
if (OB_UNLIKELY(end_idx < begin_idx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected end of range less than start of range", K(ret), K(range), K_(end),
K_(start), K(is_begin_equal), KPC(idx_data_header_));
}
}
}
LOG_TRACE("Locate range in index block by range", K(ret), K(range), K(begin_idx), K(end_idx),
K(is_left_border), K(is_right_border), K_(current), KPC(idx_data_header_));
}
if (OB_SUCC(ret)) {
K(is_left_border), K(is_right_border), K_(current), KPC(idx_data_header_));
start_ = begin_idx;
end_ = end_idx;
current_ = is_reverse_scan_ ? end_idx : begin_idx;
@ -963,7 +933,10 @@ int ObTFMIndexBlockRowIterator::check_blockscan(const ObDatumRowkey &rowkey, boo
} else if (OB_UNLIKELY(!rowkey.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid rowkey", K(ret), K(rowkey));
} else if (OB_FAIL((idx_data_header_->rowkey_array_ + end_)->compare(rowkey, *datum_utils_, cmp_ret, false))) {
} else if (OB_UNLIKELY(!idx_data_header_->is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected index data header", K(ret), KPC(idx_data_header_));
} else if (OB_FAIL(idx_data_header_->rowkey_vector_->compare_rowkey(rowkey, end_, *datum_utils_, cmp_ret, false))) {
LOG_WARN("Fail to compare rowkey", K(ret), K(rowkey));
} else {
can_blockscan = cmp_ret < 0;
@ -972,11 +945,11 @@ int ObTFMIndexBlockRowIterator::check_blockscan(const ObDatumRowkey &rowkey, boo
}
int ObTFMIndexBlockRowIterator::get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey)
ObCommonDatumRowkey &endkey)
{
int ret = OB_SUCCESS;
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
const int64_t rowkey_column_count = datum_utils_->get_rowkey_count();
idx_row_parser_.reset();
const char *idx_data_buf = nullptr;
@ -990,14 +963,19 @@ int ObTFMIndexBlockRowIterator::get_current(const ObIndexBlockRowHeader *&idx_ro
LOG_WARN("Fail to parse index block row", K(ret), K_(current), KPC(idx_data_header_));
} else if (OB_FAIL(idx_row_parser_.get_header(idx_row_header))) {
LOG_WARN("Fail to get index block row header", K(ret));
} else if (OB_UNLIKELY(!idx_data_header_->is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid idx data header", K(ret), KPC(idx_data_header_));
} else if (OB_FAIL(idx_data_header_->rowkey_vector_->get_rowkey(current_, endkey))) {
LOG_WARN("Fail to get rowkey from vector", K(ret), K(current_), KPC(idx_data_header_));
} else {
endkey = &idx_data_header_->rowkey_array_[current_];
cur_node_index_ = current_;
}
return ret;
}
int ObTFMIndexBlockRowIterator::get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -1007,7 +985,7 @@ int ObTFMIndexBlockRowIterator::get_next(const ObIndexBlockRowHeader *&idx_row_h
{
int ret = OB_SUCCESS;
idx_row_header = nullptr;
endkey = nullptr;
endkey.reset();
idx_minor_info = nullptr;
agg_row_buf = nullptr;
agg_buf_size = 0;
@ -1015,10 +993,10 @@ int ObTFMIndexBlockRowIterator::get_next(const ObIndexBlockRowHeader *&idx_row_h
ret = OB_NOT_INIT;
LOG_WARN("Iter not opened yet", K(ret), KPC(this));
} else if (OB_FAIL(get_current(idx_row_header, endkey))) {
LOG_WARN("read cur idx row failed", K(ret), KPC(idx_row_header), KPC(endkey));
} else if (OB_UNLIKELY(nullptr == idx_row_header || nullptr == endkey)) {
LOG_WARN("read cur idx row failed", K(ret), KPC(idx_row_header), K(endkey));
} else if (OB_UNLIKELY(nullptr == idx_row_header || !endkey.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KP(idx_row_header), KP(endkey));
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KP(idx_row_header), K(endkey));
} else if (idx_row_header->is_data_index() && !idx_row_header->is_major_node()) {
if (OB_FAIL(idx_row_parser_.get_minor_meta(idx_minor_info))) {
LOG_WARN("Fail to get minor meta info", K(ret));
@ -1148,51 +1126,73 @@ int ObTFMIndexBlockRowIterator::advance_to_border(const ObDatumRowkey &rowkey,
} else if (OB_UNLIKELY(!rowkey.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid rowkey", K(ret), K(rowkey));
} else if (OB_UNLIKELY(!idx_data_header_->is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected index data header", K(ret), KPC(idx_data_header_));
} else if (OB_FAIL(advance_to_border_by_rowkey_vector(rowkey, is_left_border, is_right_border, parent_row_range, cs_range))) {
LOG_WARN("Failed to advance border", K(ret));
}
return ret;
}
int ObTFMIndexBlockRowIterator::advance_to_border_by_rowkey_vector(const ObDatumRowkey &rowkey,
const bool is_left_border,
const bool is_right_border,
const ObCSRange &parent_row_range,
ObCSRange &cs_range)
{
int ret = OB_SUCCESS;
const bool is_range_end = is_reverse_scan_ ? is_left_border : is_right_border;
const int64_t begin = is_reverse_scan_ ? start_ : current_;
const int64_t end = is_reverse_scan_ ? current_ + 1 : end_ + 1;
int64_t found_pos = ObIMicroBlockReaderInfo::INVALID_ROW_INDEX;
// do not need upper_bound for reverse scan, as only co sstable reach here.
if (OB_FAIL(idx_data_header_->rowkey_vector_->locate_key(begin,
end,
rowkey,
*datum_utils_,
found_pos))) {
LOG_WARN("Failed to locate key in rowkey vector", K(ret), KPC(idx_data_header_));
} else if (!is_reverse_scan_) {
// found_pos is safe to skip(end_key < border_rowkey).
found_pos--;
if (is_range_end && found_pos == end_) {
// if is_range_end is true, we cannot skip all rowids because only subset of rowids statisy query range.
found_pos--;
}
LOG_DEBUG("ObTFMIndexBlockRowIterator::advance_to_border_by_rowkey_vector", K(found_pos), K(is_range_end),
KPC(this), K(is_range_end));
if (found_pos >= current_) {
current_ = found_pos + 1;
if (OB_FAIL(get_cur_row_id_range(parent_row_range, cs_range))) {
LOG_WARN("Failed to get cur row id range", K(ret), K(rowkey), KPC(this), K(is_range_end));
}
}
} else {
const int64_t limit_idx = is_reverse_scan_ ? start_ : end_;
const bool is_range_end = is_reverse_scan_ ? is_left_border : is_right_border;
ObDatumComparor<ObDatumRowkey> lower_bound_cmp(*datum_utils_, ret);
const ObDatumRowkey *first = nullptr;
const ObDatumRowkey *last = nullptr;
if (!is_reverse_scan_) {
first = idx_data_header_->rowkey_array_ + current_;
last = idx_data_header_->rowkey_array_ + limit_idx + 1;
} else {
first =idx_data_header_->rowkey_array_ + limit_idx;
last = idx_data_header_->rowkey_array_ + current_ + 1;
}
const ObDatumRowkey *start_found = std::lower_bound(first, last, rowkey, lower_bound_cmp);
if (OB_FAIL(ret)) {
LOG_WARN("Failed to get rowkey lower bound", K(ret), K(rowkey), KPC(this));
} else if (!is_reverse_scan_) {
// found_pos is safe to skip(end_key < border_rowkey).
int64_t found_pos = start_found - idx_data_header_->rowkey_array_ - 1;
if (is_range_end && found_pos == limit_idx) {
// if is_range_end is true, we cannot skip all rowids because only subset of rowids statisy query range.
found_pos--;
}
LOG_DEBUG("ObTFMIndexBlockRowIterator::advance_to_border", K(found_pos), K(is_range_end),
KPC(this), K(limit_idx), K(is_range_end));
if (found_pos >= current_) {
current_ = found_pos + 1;
if (OB_FAIL(get_cur_row_id_range(parent_row_range, cs_range))) {
LOG_WARN("Failed to get cur row id range", K(ret), K(rowkey), KPC(this),
K(limit_idx), K(is_range_end));
}
}
} else {
// found_pos is safe to skip.
int64_t found_pos = start_found - idx_data_header_->rowkey_array_ + 1;
// found_pos != start_, there is no need to check is_range_end.
if (found_pos <= current_ + 1) {
current_ = found_pos - 1;
if (OB_FAIL(get_cur_row_id_range(parent_row_range, cs_range))) {
LOG_WARN("Failed to get cur row id range", K(ret), K(rowkey), KPC(this),
K(limit_idx), K(is_range_end));
}
// found_pos is safe to skip(end_key > border_rowkey).
found_pos++;
// found_pos != start_, there is no need to check is_range_end.
if (found_pos <= current_ + 1) {
current_ = found_pos - 1;
if (OB_FAIL(get_cur_row_id_range(parent_row_range, cs_range))) {
LOG_WARN("Failed to get cur row id range", K(ret), K(rowkey), KPC(this), K(is_range_end));
}
}
LOG_DEBUG("ObTFMIndexBlockRowIterator::advance_to_border", K(limit_idx), K(is_range_end), KPC(this));
}
return ret;
}
int ObTFMIndexBlockRowIterator::get_end_key(ObCommonDatumRowkey &endkey)
{
int ret = OB_SUCCESS;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("Iter not opened yet", K(ret), KPC(this));
} else if (OB_UNLIKELY(!idx_data_header_->is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected index data header", K(ret), KPC(idx_data_header_));
} else {
endkey.set_compact_rowkey(idx_data_header_->rowkey_vector_->get_last_rowkey());
}
return ret;
}
@ -1202,7 +1202,7 @@ int ObTFMIndexBlockRowIterator::get_cur_row_id_range(const ObCSRange &parent_row
{
int ret = OB_SUCCESS;
const ObIndexBlockRowHeader *idx_row_header = nullptr;
const ObDatumRowkey *endkey = nullptr;
ObCommonDatumRowkey endkey;
bool is_scan_left_border = false;
bool is_scan_right_border = false;
if (IS_NOT_INIT) {
@ -1212,7 +1212,7 @@ int ObTFMIndexBlockRowIterator::get_cur_row_id_range(const ObCSRange &parent_row
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected end of index block scanner", KPC(this));
} else if (OB_FAIL(get_current(idx_row_header, endkey))) {
LOG_WARN("get next idx block row failed", K(ret), KP(idx_row_header), KPC(endkey), K(is_reverse_scan_));
LOG_WARN("get next idx block row failed", K(ret), KP(idx_row_header), K(endkey), K(is_reverse_scan_));
} else if (OB_ISNULL(idx_row_header)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null index block row header", K(ret));
@ -1235,17 +1235,21 @@ int ObTFMIndexBlockRowIterator::skip_to_next_valid_position(const ObDatumRowkey
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("Iter not opened yet", K(ret), KPC(this));
} else if (OB_UNLIKELY(!idx_data_header_->is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid idx data header", K(ret), KPC(idx_data_header_));
} else {
ObDatumComparor<ObDatumRowkey> cmp(*datum_utils_, ret, false, true, false);
const ObDatumRowkey *first = idx_data_header_->rowkey_array_ + current_;
const ObDatumRowkey *last = idx_data_header_->rowkey_array_ + end_ + 1;
const ObDatumRowkey *found = std::lower_bound(first, last, rowkey, cmp);
if (OB_FAIL(ret)) {
LOG_WARN("Failed to get lower bound of rowkey", K(ret), K(rowkey), KPC_(idx_data_header));
} else if (found == last) {
int64_t found_idx = ObIMicroBlockReaderInfo::INVALID_ROW_INDEX;
if (OB_FAIL(idx_data_header_->rowkey_vector_->locate_key(current_,
end_ + 1,
rowkey,
*datum_utils_,
found_idx))) {
LOG_WARN("Failed to locate key in rowkey vector", K(ret), KPC(idx_data_header_));
} else if (end_ + 1 == found_idx) {
ret = OB_ITER_END;
} else {
current_ = found - idx_data_header_->rowkey_array_;
current_ = found_idx;
}
}
return ret;
@ -1257,11 +1261,10 @@ int ObTFMIndexBlockRowIterator::find_rowkeys_belong_to_same_idx_row(ObMicroIndex
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("Iter not opened yet", K(ret), KPC(this));
} else if (OB_ISNULL(rows_info)) {
} else if (OB_UNLIKELY(nullptr == rows_info || !idx_data_header_->is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid rows info", K(ret));
LOG_WARN("invalid rows info or header", K(ret), KP(rows_info), KPC(idx_data_header_));
} else {
const ObDatumRowkey *cur_rowkey = idx_data_header_->rowkey_array_ + current_;
bool is_decided = false;
for (; OB_SUCC(ret) && rowkey_begin_idx < rowkey_end_idx; ++rowkey_begin_idx) {
if (rows_info->is_row_skipped(rowkey_begin_idx)) {
@ -1269,15 +1272,9 @@ int ObTFMIndexBlockRowIterator::find_rowkeys_belong_to_same_idx_row(ObMicroIndex
}
const ObDatumRowkey &rowkey = rows_info->get_rowkey(rowkey_begin_idx);
int32_t cmp_ret = 0;
if (OB_ISNULL(cur_rowkey)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("null rowkey", K(ret), K(current_), KP(cur_rowkey));
} else if (OB_FAIL(rowkey.compare(*cur_rowkey, *datum_utils_, cmp_ret, false))) {
LOG_WARN("Failed to compare rowkey", K(ret), K(rowkey), KPC(cur_rowkey));
}
if (OB_FAIL(ret)) {
} else if (cmp_ret > 0) {
if (OB_FAIL(idx_data_header_->rowkey_vector_->compare_rowkey(rowkey, current_, *datum_utils_, cmp_ret, false))) {
LOG_WARN("Failed to compare rowkey", K(ret), K(rowkey), K(current_));
} else if (cmp_ret < 0) {
idx_block_row.rowkey_end_idx_ = rowkey_begin_idx;
is_decided = true;
break;
@ -1295,26 +1292,25 @@ int ObTFMIndexBlockRowIterator::find_rowkeys_belong_to_same_idx_row(ObMicroIndex
return ret;
}
int ObTFMIndexBlockRowIterator::find_rowkeys_belong_to_curr_idx_row(const int64_t rowkey_end_idx, int64_t &rowkey_idx, const ObRowKeysInfo *rowkeys_info)
int ObTFMIndexBlockRowIterator::find_rowkeys_belong_to_curr_idx_row(ObMicroIndexInfo &idx_block_row, const int64_t rowkey_end_idx, const ObRowKeysInfo *rowkeys_info)
{
int ret = OB_SUCCESS;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("Iter not opened yet", K(ret), KPC(this));
} else if (OB_ISNULL(rowkeys_info)) {
} else if (OB_UNLIKELY(!idx_block_row.endkey_.is_valid() ||
nullptr == rowkeys_info)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid rowkeys info", K(ret));
LOG_WARN("invalid rowkeys info", K(ret), K(idx_block_row.endkey_), KP(rowkeys_info));
} else {
const int64_t current_idx = current_ - iter_step_;
const ObDatumRowkey *cur_rowkey = idx_data_header_->rowkey_array_ + current_idx;
for (; OB_SUCC(ret) && rowkey_idx < rowkey_end_idx; ++rowkey_idx) {
if (rowkeys_info->is_rowkey_not_exist(rowkey_idx)) {
for (; OB_SUCC(ret) && idx_block_row.rowkey_end_idx_ < rowkey_end_idx; ++idx_block_row.rowkey_end_idx_) {
if (rowkeys_info->is_rowkey_not_exist(idx_block_row.rowkey_end_idx_)) {
continue;
}
const ObDatumRowkey &rowkey = rowkeys_info->get_rowkey(rowkey_idx);
const ObDatumRowkey &rowkey = rowkeys_info->get_rowkey(idx_block_row.rowkey_end_idx_);
int cmp_ret = 0;
if (OB_FAIL(rowkey.compare(*cur_rowkey, *datum_utils_, cmp_ret, false))) {
LOG_WARN("Failed to compare rowkey", K(ret), K(rowkey), KPC(cur_rowkey));
if (OB_FAIL(rowkey.compare(idx_block_row.endkey_, *datum_utils_, cmp_ret, false))) {
LOG_WARN("Failed to compare rowkey", K(ret), K(rowkey), K(idx_block_row.endkey_));
} else if (cmp_ret >= 0) {
break;
}
@ -1323,6 +1319,29 @@ int ObTFMIndexBlockRowIterator::find_rowkeys_belong_to_curr_idx_row(const int64_
return ret;
}
int ObTFMIndexBlockRowIterator::locate_range_by_rowkey_vector(
const ObDatumRange &range,
const bool is_left_border,
const bool is_right_border,
const bool is_normal_cg,
int64_t &begin_idx,
int64_t &end_idx)
{
int ret = OB_SUCCESS;
if (OB_FAIL(idx_data_header_->rowkey_vector_->locate_range(range,
is_left_border,
is_right_border,
is_normal_cg,
*datum_utils_,
begin_idx,
end_idx))) {
if (OB_UNLIKELY(OB_BEYOND_THE_RANGE != ret)) {
LOG_WARN("Failed to locate range by rowkey vector", K(ret));
}
}
return ret;
}
/****************** ObIndexBlockRowScanner **********************/
ObIndexBlockRowScanner::ObIndexBlockRowScanner()
: query_range_(nullptr), macro_id_(), allocator_(nullptr), raw_iter_(nullptr), transformed_iter_(nullptr),
@ -1331,7 +1350,7 @@ ObIndexBlockRowScanner::ObIndexBlockRowScanner()
index_format_(ObIndexFormat::INVALID), parent_row_range_(), is_get_(false), is_reverse_scan_(false),
is_left_border_(false), is_right_border_(false), is_inited_(false),
is_normal_cg_(false), is_normal_query_(true), filter_constant_type_(sql::ObBoolMaskType::PROBABILISTIC),
iter_param_()
iter_param_(), rowkey_col_descs_(nullptr)
{}
ObIndexBlockRowScanner::~ObIndexBlockRowScanner()
@ -1358,6 +1377,7 @@ void ObIndexBlockRowScanner::reuse()
is_right_border_ = false;
parent_row_range_.reset();
filter_constant_type_ = sql::ObBoolMaskType::PROBABILISTIC;
rowkey_col_descs_ = nullptr;
}
void ObIndexBlockRowScanner::reset()
@ -1409,6 +1429,7 @@ void ObIndexBlockRowScanner::reset()
iter_param_.reset();
allocator_ = nullptr;
filter_constant_type_ = sql::ObBoolMaskType::PROBABILISTIC;
rowkey_col_descs_ = nullptr;
}
int ObIndexBlockRowScanner::init(
@ -1416,7 +1437,8 @@ int ObIndexBlockRowScanner::init(
ObIAllocator &allocator,
const common::ObQueryFlag &query_flag,
const int64_t nested_offset,
const bool is_normal_cg)
const bool is_normal_cg,
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs)
{
int ret = OB_SUCCESS;
if (IS_INIT) {
@ -1432,6 +1454,7 @@ int ObIndexBlockRowScanner::init(
nested_offset_ = nested_offset;
is_normal_cg_ = is_normal_cg;
is_normal_query_ = !query_flag.is_daily_merge() && !query_flag.is_multi_version_minor_merge();
rowkey_col_descs_ = rowkey_col_descs;
is_inited_ = true;
}
return ret;
@ -1623,7 +1646,7 @@ int ObIndexBlockRowScanner::get_next(
if (OB_ISNULL(iter_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("iter is null", K(index_format_), K(ret));
} else if (OB_FAIL(iter_->find_rowkeys_belong_to_curr_idx_row(rowkey_end_idx_, idx_block_row.rowkey_end_idx_, rowkeys_info_))) {
} else if (OB_FAIL(iter_->find_rowkeys_belong_to_curr_idx_row(idx_block_row, rowkey_end_idx_, rowkeys_info_))) {
LOG_WARN("Failed to find rowkeys", K(ret));
}
}
@ -1860,25 +1883,34 @@ int ObIndexBlockRowScanner::find_out_rows_from_start_to_end(
return ret;
}
const ObDatumRowkey &ObIndexBlockRowScanner::get_end_key() const
int ObIndexBlockRowScanner::get_end_key(ObCommonDatumRowkey &endkey) const
{
const ObDatumRowkey *tmp_key = nullptr;
if (OB_NOT_NULL(iter_)) {
iter_->get_end_key(tmp_key);
int ret = OB_SUCCESS;
endkey.reset();
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("Not inited", K_(is_inited));
} else if (OB_ISNULL(iter_) || OB_UNLIKELY(index_format_ != ObIndexFormat::TRANSFORMED)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("iter is null or wrong format", KP(iter_), K(index_format_), K(ret));
} else if (OB_FAIL(iter_->get_end_key(endkey))) {
LOG_WARN("Failed to get end key", K(ret));
}
return *tmp_key;
return ret;
}
void ObIndexBlockRowScanner::switch_context(const ObSSTable &sstable,
const ObTablet *tablet,
const ObStorageDatumUtils &datum_utils,
ObTableAccessContext &access_ctx)
ObTableAccessContext &access_ctx,
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs)
{
nested_offset_ = sstable.get_macro_offset();
datum_utils_ = &datum_utils;
is_normal_cg_ = sstable.is_normal_cg_sstable();
is_reverse_scan_ = access_ctx.query_flag_.is_reverse_scan();
is_normal_query_ = !access_ctx.query_flag_.is_daily_merge() && !access_ctx.query_flag_.is_multi_version_minor_merge();
rowkey_col_descs_ = rowkey_col_descs;
iter_param_.sstable_ = &sstable;
iter_param_.tablet_ = tablet;
int ret = OB_SUCCESS;
@ -1891,7 +1923,6 @@ void ObIndexBlockRowScanner::switch_context(const ObSSTable &sstable,
int ObIndexBlockRowScanner::get_next_idx_row(ObMicroIndexInfo &idx_block_row)
{
int ret = OB_SUCCESS;
const ObDatumRowkey *endkey = nullptr;
const ObIndexBlockRowHeader *idx_row_header = nullptr;
const ObIndexBlockRowMinorMetaInfo *idx_minor_info = nullptr;
const char *idx_data_buf = nullptr;
@ -1904,18 +1935,18 @@ int ObIndexBlockRowScanner::get_next_idx_row(ObMicroIndexInfo &idx_block_row)
ret = OB_ERR_UNEXPECTED;
LOG_WARN("iter is null", K(ret), K(index_format_), KP(iter_));
} else {
if (OB_FAIL(iter_->get_next(idx_row_header, endkey, is_scan_left_border, is_scan_right_border, idx_minor_info, agg_row_buf, agg_buf_size, row_offset))) {
LOG_WARN("get next idx block row failed", K(ret), KP(idx_row_header), KPC(endkey), K(is_reverse_scan_));
} else if (OB_UNLIKELY(nullptr == idx_row_header || nullptr == endkey)) {
if (OB_FAIL(iter_->get_next(idx_row_header, idx_block_row.endkey_, is_scan_left_border, is_scan_right_border, idx_minor_info, agg_row_buf, agg_buf_size, row_offset))) {
LOG_WARN("get next idx block row failed", K(ret), KP(idx_row_header), K(is_reverse_scan_));
} else if (OB_UNLIKELY(nullptr == idx_row_header || !idx_block_row.endkey_.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null index block row header/endkey", K(ret), KPC(iter_),
K(index_format_), KP(idx_row_header), KP(endkey));
K(index_format_), KP(idx_row_header), K(idx_block_row.endkey_));
}
}
if (OB_SUCC(ret)) {
idx_block_row.ps_node_ = iter_->get_cur_ps_node();
idx_block_row.flag_ = 0;
idx_block_row.endkey_ = endkey;
idx_block_row.row_header_ = idx_row_header;
idx_block_row.minor_meta_info_ = idx_minor_info;
idx_block_row.is_get_ = is_get_;
@ -1928,20 +1959,26 @@ int ObIndexBlockRowScanner::get_next_idx_row(ObMicroIndexInfo &idx_block_row)
idx_block_row.nested_offset_ = nested_offset_;
idx_block_row.agg_row_buf_ = agg_row_buf;
idx_block_row.agg_buf_size_ = agg_buf_size;
idx_block_row.rowkey_col_descs_ = rowkey_col_descs_;
if (is_normal_cg_) {
idx_block_row.cs_row_range_.start_row_id_ = idx_block_row.endkey_->datums_[0].get_int() - idx_block_row.get_row_count() + 1;
idx_block_row.cs_row_range_.end_row_id_ = idx_block_row.endkey_->datums_[0].get_int();
idx_block_row.set_filter_constant_type(filter_constant_type_);
int64_t row_offset;
if (OB_FAIL(idx_block_row.endkey_.get_column_int(0, row_offset))) {
LOG_WARN("Failed to get datum int", K(ret));
} else {
idx_block_row.cs_row_range_.start_row_id_ =row_offset - idx_block_row.get_row_count() + 1;
idx_block_row.cs_row_range_.end_row_id_ = row_offset;
idx_block_row.set_filter_constant_type(filter_constant_type_);
}
} else {
idx_block_row.cs_row_range_.start_row_id_ = row_offset - idx_block_row.get_row_count() + 1;
idx_block_row.cs_row_range_.end_row_id_ = row_offset;
}
if (idx_block_row.is_data_block()) {
if (OB_SUCC(ret) && idx_block_row.is_data_block()) {
idx_block_row.cs_row_range_.start_row_id_ += parent_row_range_.start_row_id_;
idx_block_row.cs_row_range_.end_row_id_ += parent_row_range_.start_row_id_;
}
}
LOG_DEBUG("Get next index block row", K(ret), KPC(iter_), K(idx_block_row), KP(this), K(endkey));
LOG_DEBUG("Get next index block row", K(ret), KPC(iter_), K(idx_block_row), KP(this), K(idx_block_row.endkey_));
return ret;
}

View File

@ -18,11 +18,13 @@
#include "storage/blocksstable/ob_micro_block_reader_helper.h"
#include "storage/blocksstable/ob_datum_range.h"
#include "storage/blocksstable/index_block/ob_ddl_sstable_scan_merge.h"
#include "storage/blocksstable/ob_datum_rowkey_vector.h"
#include "storage/column_store/ob_column_store_util.h"
#include "ob_index_block_row_struct.h"
#include "storage/memtable/mvcc/ob_keybtree.h"
#include "storage/meta_mem/ob_tablet_handle.h"
#include "storage/access/ob_simple_rows_merger.h"
#include "share/cache/ob_kvcache_pointer_swizzle.h"
namespace oceanbase
{
namespace storage
@ -47,8 +49,9 @@ struct ObIndexBlockDataHeader
{
return row_cnt_ >= 0
&& col_cnt_ > 0
&& nullptr != rowkey_array_
&& nullptr != datum_array_;
&& nullptr != rowkey_vector_
&& nullptr != index_datum_array_
&& nullptr != ps_node_array_;
}
int get_index_data(const int64_t row_idx, const char *&index_ptr, int64_t &index_len) const;
@ -60,18 +63,19 @@ struct ObIndexBlockDataHeader
int64_t row_cnt_;
int64_t col_cnt_;
// Array of rowkeys in index block
const ObDatumRowkey *rowkey_array_;
// Array of deserialzed Object array
ObStorageDatum *datum_array_;
// Vectors of rowkeys
const ObRowkeyVector *rowkey_vector_;
// Array of index Object
ObStorageDatum *index_datum_array_;
ObPointerSwizzleNode *ps_node_array_;
const char *data_buf_;
int64_t data_buf_size_;
TO_STRING_KV(
K_(row_cnt), K_(col_cnt),
// "Rowkeys:", common::ObArrayWrap<ObDatumRowkey>(rowkey_array_, row_cnt_)
// output first only
KPC_(rowkey_array)
KPC_(rowkey_vector),
KP_(index_datum_array),
KP_(ps_node_array), K_(data_buf_size)
);
};
@ -84,7 +88,8 @@ public:
const ObMicroBlockData &raw_data,
ObMicroBlockData &transformed_data,
ObIAllocator &allocator,
char *&allocated_buf);
char *&allocated_buf,
const ObIArray<share::schema::ObColDesc> *col_descs = nullptr);
// For micro header bug in version before 4.3, when root block serialized in sstable meta,
// data length related fileds was lefted to be filled
@ -93,7 +98,7 @@ int fix_micro_header_and_transform(
ObMicroBlockData &transformed_data,
ObIAllocator &allocator,
char *&allocated_buf);
static int get_transformed_upper_mem_size(const char *raw_block_data, int64_t &mem_limit);
static int get_transformed_upper_mem_size(const ObIArray<share::schema::ObColDesc> *rowkey_col_descs, const char *raw_block_data, int64_t &mem_limit);
private:
int get_reader(const ObRowStoreType store_type, ObIMicroBlockReader *&micro_reader);
private:
@ -139,9 +144,9 @@ public:
const bool is_reverse_scan,
const ObIndexBlockIterParam &iter_param) = 0;
virtual int get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey) = 0;
ObCommonDatumRowkey &endkey) = 0;
virtual int get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -175,14 +180,16 @@ public:
int64_t &found_idx) { return OB_NOT_SUPPORTED; }
virtual int skip_to_next_valid_position(const ObDatumRowkey &rowkey) { return OB_NOT_SUPPORTED; }
virtual int find_rowkeys_belong_to_same_idx_row(ObMicroIndexInfo &idx_block_row, int64_t &rowkey_begin_idx, int64_t &rowkey_end_idx, const ObRowsInfo *&rows_info) { return OB_NOT_SUPPORTED; }
virtual int find_rowkeys_belong_to_curr_idx_row(const int64_t rowkey_end_idx, int64_t &rowkey_idx, const ObRowKeysInfo *rowkeys_info) { return OB_NOT_SUPPORTED; }
virtual int find_rowkeys_belong_to_curr_idx_row(ObMicroIndexInfo &idx_block_row, const int64_t rowkey_end_idx, const ObRowKeysInfo *rowkeys_info) { return OB_NOT_SUPPORTED; }
virtual int advance_to_border(const ObDatumRowkey &rowkey,
const bool is_left_border,
const bool is_right_border,
const ObCSRange &parent_row_range,
ObCSRange &cs_range) { return OB_NOT_SUPPORTED; }
virtual void get_end_key(const ObDatumRowkey *&rowkey) {}
virtual int get_end_key(ObCommonDatumRowkey &endkey) { return OB_NOT_SUPPORTED; }
virtual void set_iter_end() {}
virtual ObPointerSwizzleNode* get_cur_ps_node() { return nullptr; }
virtual int64_t get_cur_ps_node_index() { return 0; }
public:
virtual int switch_context(ObStorageDatumUtils *datum_utils)
{
@ -212,9 +219,9 @@ public:
const bool is_reverse_scan,
const ObIndexBlockIterParam &iter_param) override;
virtual int get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey) override;
ObCommonDatumRowkey &endkey) override;
virtual int get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -267,9 +274,9 @@ public:
const bool is_reverse_scan,
const ObIndexBlockIterParam &iter_param) override;
virtual int get_current(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey) override;
ObCommonDatumRowkey &endkey) override;
virtual int get_next(const ObIndexBlockRowHeader *&idx_row_header,
const ObDatumRowkey *&endkey,
ObCommonDatumRowkey &endkey,
bool &is_scan_left_border,
bool &is_scan_right_border,
const ObIndexBlockRowMinorMetaInfo *&idx_minor_info,
@ -296,7 +303,7 @@ public:
int64_t &found_idx) override;
virtual int skip_to_next_valid_position(const ObDatumRowkey &rowkey) override;
virtual int find_rowkeys_belong_to_same_idx_row(ObMicroIndexInfo &idx_block_row, int64_t &rowkey_begin_idx, int64_t &rowkey_end_idx, const ObRowsInfo *&rows_info) override;
virtual int find_rowkeys_belong_to_curr_idx_row(const int64_t rowkey_end_idx, int64_t &rowkey_idx, const ObRowKeysInfo *rowkeys_info) override;
virtual int find_rowkeys_belong_to_curr_idx_row(ObMicroIndexInfo &idx_block_row, const int64_t rowkey_end_idx, const ObRowKeysInfo *rowkeys_info) override;
virtual int get_idx_row_header_in_target_idx(const int64_t idx,
const ObIndexBlockRowHeader *&idx_row_header) override;
virtual int advance_to_border(const ObDatumRowkey &rowkey,
@ -304,15 +311,32 @@ public:
const bool is_right_border,
const ObCSRange &parent_row_range,
ObCSRange &cs_range) override;
virtual void get_end_key(const ObDatumRowkey *&rowkey) { rowkey = &(idx_data_header_->rowkey_array_[idx_data_header_->row_cnt_ - 1]); }
virtual int get_end_key(ObCommonDatumRowkey &endkey) override;
virtual int64_t get_cur_ps_node_index() { return cur_node_index_; }
virtual ObPointerSwizzleNode* get_cur_ps_node() {
return (nullptr == idx_data_header_ || nullptr == idx_data_header_->ps_node_array_) ? nullptr : (idx_data_header_->ps_node_array_ + cur_node_index_);
}
INHERIT_TO_STRING_KV("base iterator:", ObRAWIndexBlockRowIterator, "format:", "ObTFMIndexBlockRowIterator", KPC(idx_data_header_));
private:
int get_cur_row_id_range(const ObCSRange &parent_row_range,
ObCSRange &cs_range);
int locate_range_by_rowkey_vector(
const ObDatumRange &range,
const bool is_left_border,
const bool is_right_border,
const bool is_normal_cg,
int64_t &begin_idx,
int64_t &end_idx);
int advance_to_border_by_rowkey_vector(const ObDatumRowkey &rowkey,
const bool is_left_border,
const bool is_right_border,
const ObCSRange &parent_row_range,
ObCSRange &cs_range);
private:
const ObIndexBlockDataHeader *idx_data_header_;
int64_t cur_node_index_;
};
class ObIndexBlockRowScanner
@ -328,7 +352,8 @@ public:
ObIAllocator &allocator,
const common::ObQueryFlag &query_flag,
const int64_t nested_offset,
const bool is_normal_cg = false);
const bool is_normal_cg = false,
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs = nullptr);
// todo :qilu get ls_id from MTL() after ddl_kv_mgr split to tenant
int open(
const MacroBlockId &macro_id,
@ -380,23 +405,23 @@ public:
bool &is_certain,
int64_t &found_idx);
bool is_in_border();
const ObDatumRowkey &get_end_key() const;
int get_end_key(ObCommonDatumRowkey &endkey) const;
OB_INLINE bool is_valid() const { return is_inited_; }
OB_INLINE bool is_ddl_merge_scan() const { return index_format_ == ObIndexFormat::DDL_MERGE; }
void switch_context(const ObSSTable &sstable,
const ObTablet *tablet,
const ObStorageDatumUtils &datum_utils,
ObTableAccessContext &access_ctx);
ObTableAccessContext &access_ctx,
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs = nullptr);
TO_STRING_KV(K_(index_format), KP_(raw_iter), KP_(transformed_iter), KP_(ddl_iter), KP_(ddl_merge_iter),
KPC_(iter), K_(range_idx), K_(is_get), K_(is_reverse_scan), K_(is_left_border), K_(is_right_border),
K_(rowkey_begin_idx), K_(rowkey_end_idx), K_(is_inited), K_(macro_id), KPC_(datum_utils),
K_(is_normal_cg), K_(parent_row_range), K_(filter_constant_type), K_(is_normal_query),
K_(iter_param));
K_(iter_param), KP_(rowkey_col_descs));
private:
int init_by_micro_data(const ObMicroBlockData &idx_block_data);
int locate_key(const ObDatumRowkey &rowkey);
int init_datum_row();
int read_curr_idx_row(const ObIndexBlockRowHeader *&idx_row_header, const ObDatumRowkey *&endkey);
int get_cur_row_id_range(ObCSRange &cs_range);
int get_idx_row_header_in_target_idx(
const ObIndexBlockRowHeader *&idx_row_header,
@ -439,6 +464,7 @@ private:
bool is_normal_query_;
sql::ObBoolMaskType filter_constant_type_;
ObIndexBlockIterParam iter_param_; // todo qilu: refactor this after refactor ddl_kv_mgr
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs_;
};
} // namespace blocksstable

View File

@ -27,6 +27,7 @@ namespace oceanbase
namespace common
{
class ObObj;
class ObPointerSwizzleNode;
}
namespace storage
{
@ -262,7 +263,8 @@ public:
ObMicroIndexInfo()
: row_header_(nullptr),
minor_meta_info_(nullptr),
endkey_(nullptr),
endkey_(),
ps_node_(nullptr),
query_range_(nullptr),
agg_row_buf_(nullptr),
agg_buf_size_(0),
@ -271,14 +273,16 @@ public:
parent_macro_id_(),
nested_offset_(0),
cs_row_range_(),
skipping_filter_results_()
skipping_filter_results_(),
rowkey_col_descs_(nullptr)
{
}
OB_INLINE void reset()
{
row_header_ = nullptr;
minor_meta_info_ = nullptr;
endkey_ = nullptr;
endkey_.reset();
ps_node_ = nullptr;
query_range_ = nullptr;
agg_row_buf_ = nullptr;
agg_buf_size_ = 0;
@ -288,6 +292,7 @@ public:
nested_offset_ = 0;
cs_row_range_.reset();
skipping_filter_results_.reset();
rowkey_col_descs_ = nullptr;
}
OB_INLINE bool is_valid() const
{
@ -300,7 +305,7 @@ public:
|| nullptr != minor_meta_info_;
const bool parent_macro_id_valid = !row_header_->is_data_block() || parent_macro_id_.is_valid();
const bool pre_agg_valid = !row_header_->is_pre_aggregated() || nullptr != agg_row_buf_;
bret = minor_meta_info_valid && parent_macro_id_valid && pre_agg_valid && nullptr != endkey_;
bret = minor_meta_info_valid && parent_macro_id_valid && pre_agg_valid && endkey_.is_valid();
}
return bret;
}
@ -528,15 +533,20 @@ public:
}
}
}
TO_STRING_KV(KP_(query_range), KPC_(row_header), KPC_(minor_meta_info), KPC_(endkey),
OB_INLINE const ObIArray<share::schema::ObColDesc> *get_rowkey_col_descs() const
{
return rowkey_col_descs_;
}
TO_STRING_KV(KP_(query_range), KPC_(row_header), KPC_(minor_meta_info), K_(endkey), KP_(ps_node),
KP_(agg_row_buf), K_(agg_buf_size), K_(flag), K_(range_idx), K_(parent_macro_id),
K_(nested_offset), K_(rowkey_begin_idx), K_(rowkey_end_idx), K_(cs_row_range),
K_(skipping_filter_results));
K_(skipping_filter_results), KP_(rowkey_col_descs));
public:
const ObIndexBlockRowHeader *row_header_;
const ObIndexBlockRowMinorMetaInfo *minor_meta_info_;
const ObDatumRowkey *endkey_;
ObCommonDatumRowkey endkey_;
ObPointerSwizzleNode *ps_node_;
union {
const ObDatumRowkey *rowkey_;
const ObDatumRange *range_;
@ -570,6 +580,7 @@ public:
int64_t rowkey_end_idx_;
ObCSRange cs_row_range_;
ObSkippingFilterResults skipping_filter_results_;
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs_;
};

View File

@ -319,6 +319,7 @@ void ObIndexBlockTreeCursor::reset()
{
if (is_inited_) {
row_.reset();
vector_endkey_.reset();
cursor_path_.reset();
micro_reader_helper_.reset();
reader_ = nullptr;
@ -387,6 +388,8 @@ int ObIndexBlockTreeCursor::init(
} else if (FALSE_IT(curr_path_item_->row_store_type_ = root_row_store_type)) {
} else if (OB_FAIL(row_.init(allocator, rowkey_column_cnt_ + 1))) {
STORAGE_LOG(WARN, "Failed to init datum row", K(ret));
} else if (OB_FAIL(init_curr_endkey(row_, rowkey_column_cnt_ + 1))) {
STORAGE_LOG(WARN, "Failed to init curr endkey", K(ret));
} else if (nullptr != curr_path_item_->block_data_.get_extra_buf()) {
curr_path_item_->is_block_transformed_ = true;
} else if (OB_FAIL(set_reader(root_row_store_type))) {
@ -665,20 +668,18 @@ int ObIndexBlockTreeCursor::search_rowkey_in_transformed_block(
{
int ret = OB_SUCCESS;
const ObStorageDatumUtils &datum_utils = read_info_->get_datum_utils();
ObDatumComparor<ObDatumRowkey> cmp(datum_utils, ret, false, lower_bound);
const ObDatumRowkey *first = idx_data_header.rowkey_array_;
const ObDatumRowkey *last = idx_data_header.rowkey_array_ + idx_data_header.row_cnt_;
const ObDatumRowkey *found = std::lower_bound(first, last, rowkey, cmp);
if (OB_FAIL(ret)) {
LOG_WARN("Fail to binary search on transformed index block", K(ret), K(rowkey), K(idx_data_header));
} else if (found == last) {
row_idx = idx_data_header.row_cnt_;
if (OB_UNLIKELY(!idx_data_header.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid idx data header", K(ret), K(idx_data_header));
} else if (OB_FAIL(idx_data_header.rowkey_vector_->locate_key(0,
idx_data_header.row_cnt_,
rowkey,
datum_utils,
row_idx,
lower_bound))) {
LOG_WARN("Failed to locate key in rowkey vector", K(ret), K(rowkey), K(idx_data_header));
} else if (row_idx == idx_data_header.row_cnt_) {
ret = OB_BEYOND_THE_RANGE;
} else {
row_idx = found - first;
if (OB_FAIL(rowkey.equal(idx_data_header.rowkey_array_[row_idx], datum_utils, equal))) {
LOG_WARN("Fail to compare datum rowkey", K(ret));
}
}
return ret;
}
@ -963,9 +964,8 @@ int ObIndexBlockTreeCursor::get_current_endkey(ObDatumRowkey &endkey, const bool
const ObIndexBlockDataHeader *idx_data_header = nullptr;
if (OB_FAIL(get_transformed_data_header(*curr_path_item_, idx_data_header))) {
LOG_WARN("Fail to get transformed data header", K(ret));
} else if (OB_FAIL(endkey.assign(
idx_data_header->rowkey_array_[curr_path_item_->curr_row_idx_].datums_, rowkey_datum_cnt))) {
LOG_WARN("Failed to assign endkey", K(ret), K(rowkey_datum_cnt), KPC(idx_data_header));
} else if (OB_FAIL(endkey.assign(vector_endkey_.datums_, rowkey_datum_cnt))) {
LOG_WARN("Failed to assign endkey", K(ret), K(rowkey_datum_cnt), K(vector_endkey_));
}
} else if (OB_FAIL(endkey.assign(row_.storage_datums_, rowkey_datum_cnt))) {
LOG_WARN("Failed to assign endkey", K(ret), K(rowkey_datum_cnt), K(row_));
@ -1188,6 +1188,10 @@ int ObIndexBlockTreeCursor::read_next_level_row(const int64_t row_idx)
} else if (OB_FAIL(idx_row_parser_.init(idx_data_buf, idx_data_len))) {
LOG_WARN("Fail to init index row parser with transformed index data",
K(ret), K(row_idx), KPC(idx_data_header));
} else if (nullptr != idx_data_header->rowkey_vector_) {
if (OB_FAIL(idx_data_header->rowkey_vector_->get_rowkey(row_idx, vector_endkey_))) {
LOG_WARN("Failed to get rowkey", K(ret));
}
}
} else if (FALSE_IT(row_.reuse())) {
} else if (OB_FAIL(reader_->get_row(row_idx, row_))) {
@ -1276,10 +1280,18 @@ int ObIndexBlockTreeCursor::get_micro_block_endkeys(
} else if (OB_UNLIKELY(begin_idx < 0 || end_idx >= idx_data_header->row_cnt_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid range idx located", K(ret), K(begin_idx), K(end_idx));
}
for (int64_t i = begin_idx; OB_SUCC(ret) && i <= end_idx; ++i) {
if (OB_FAIL(end_keys.push_back(idx_data_header->rowkey_array_[i]))) {
LOG_WARN("Fail to push rowkey into array", K(ret), KPC(idx_data_header));
} else {
ObDatumRowkey rowkey, endkey;
for (int64_t i = begin_idx; OB_SUCC(ret) && i <= end_idx; ++i) {
if (OB_FAIL(idx_data_header->rowkey_vector_->get_rowkey(i, vector_endkey_))) {
LOG_WARN("Failed to get rowkey", K(ret));
} else if (OB_FAIL(rowkey.assign(vector_endkey_.datums_, rowkey_column_cnt_))) {
STORAGE_LOG(WARN, "Failed to assign datum rowkey", K(ret), K_(vector_endkey), K_(rowkey_column_cnt));
} else if (OB_FAIL(rowkey.deep_copy(endkey, endkey_allocator))) {
STORAGE_LOG(WARN, "Failed to deep copy endkey", K(ret), K(rowkey));
} else if (OB_FAIL(end_keys.push_back(endkey))) {
LOG_WARN("Fail to push rowkey into array", K(ret));
}
}
}
} else {
@ -1305,7 +1317,7 @@ int ObIndexBlockTreeCursor::get_micro_block_endkeys(
LOG_WARN("Unexpected endkey index overflow", K(ret),
K(it), K(end_keys.count()), K(micro_index_infos.count()));
} else {
micro_index_infos.at(it).endkey_ = &end_keys.at(it);
micro_index_infos.at(it).endkey_.set_compact_rowkey(&end_keys.at(it));
if (OB_UNLIKELY(!micro_index_infos.at(it).is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid micro index info", K(ret), K(it), K(micro_index_infos.at(it)));
@ -1350,5 +1362,19 @@ int ObIndexBlockTreeCursor::check_reach_target_depth(
return ret;
}
int ObIndexBlockTreeCursor::init_curr_endkey(ObDatumRow &row_buf, const int64_t datum_cnt)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(datum_cnt > row_buf.get_capacity())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid row buf", K(ret), K(datum_cnt), K(row_buf.get_capacity()));
} else if (!vector_endkey_.is_valid()) {
if (OB_FAIL(vector_endkey_.assign(row_buf.storage_datums_, datum_cnt))) {
LOG_WARN("Failed to assign", K(ret));
}
}
return ret;
}
} // namespace oceanbase
} // namespace oceanbase

View File

@ -232,6 +232,7 @@ private:
ObIArray<ObMicroIndexInfo> &micro_index_infos,
ObIArray<ObDatumRowkey> &end_keys);
int check_reach_target_depth(const MoveDepth target_depth, bool &reach_target_depth);
int init_curr_endkey(ObDatumRow &row_buf, const int64_t datum_cnt);
private:
static const int64_t OB_INDEX_BLOCK_MAX_COL_CNT =
@ -250,6 +251,7 @@ private:
ObIndexBlockTreePathItem *curr_path_item_;
ObDatumRow row_;
ObDatumRowkey vector_endkey_;
ObIndexBlockRowParser idx_row_parser_;
const ObITableReadInfo *read_info_;
ObSSTableMetaHandle sstable_meta_handle_;

View File

@ -91,6 +91,22 @@ struct ObDatumComparor
{
return compare<T>(left, right);
}
OB_INLINE bool is_reverse() const
{
return reverse_;
}
OB_INLINE bool is_lower_bound() const
{
return lower_bound_;
}
OB_INLINE bool is_compare_datum_cnt() const
{
return compare_datum_cnt_;
}
OB_INLINE const ObStorageDatumUtils &get_datum_utils() const
{
return datum_utils_;
}
private:
template <typename DataType>
OB_INLINE bool compare(const DataType &left, const DataType &right)

View File

@ -13,6 +13,7 @@
#include "ob_datum_rowkey.h"
#include "ob_datum_range.h"
#include "share/schema/ob_table_param.h"
#include "storage/blocksstable/ob_datum_rowkey_vector.h"
namespace oceanbase
{
@ -124,6 +125,30 @@ int ObDatumRowkey::compare(const ObDatumRowkey &rhs, const ObStorageDatumUtils &
return ret;
}
int ObDatumRowkey::compare(const ObDiscreteDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt) const
{
int ret = OB_SUCCESS;
if (OB_FAIL(rhs.compare(*this, datum_utils, cmp_ret, compare_datum_cnt))) {
STORAGE_LOG(WARN, "Failed to compare", K(ret));
} else {
cmp_ret = -cmp_ret;
}
return ret;
}
int ObDatumRowkey::compare(const ObCommonDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt) const
{
int ret = OB_SUCCESS;
if (OB_FAIL(rhs.compare(*this, datum_utils, cmp_ret, compare_datum_cnt))) {
STORAGE_LOG(WARN, "Failed to compare", K(ret));
} else {
cmp_ret = -cmp_ret;
}
return ret;
}
OB_DEF_SERIALIZE(ObDatumRowkey)
{
int ret = OB_SUCCESS;
@ -378,6 +403,169 @@ void ObDatumRowkey::reuse()
}
}
int ObDiscreteDatumRowkey::compare(const ObDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_valid() || !rhs.is_valid() || !datum_utils.is_valid())) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "Invalid argument to compare datum rowkey", K(ret), K(*this), K(rhs), K(datum_utils));
} else if (OB_FAIL(rowkey_vector_->compare_rowkey(rhs, row_idx_, datum_utils, cmp_ret, compare_datum_cnt))) {
STORAGE_LOG(WARN, "Failed to compare rowkey in rowkey vector", K(ret));
}
return ret;
}
int ObDiscreteDatumRowkey::compare(const ObDiscreteDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_valid() || !rhs.is_valid() || !datum_utils.is_valid())) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "Invalid argument to compare datum rowkey", K(ret), K(*this), K(rhs), K(datum_utils));
} else if (OB_FAIL(rowkey_vector_->compare_rowkey(rhs, row_idx_, datum_utils, cmp_ret, compare_datum_cnt))) {
STORAGE_LOG(WARN, "Failed to compare rowkey in rowkey vector", K(ret), K(*this), K(rhs));
}
return ret;
}
int ObDiscreteDatumRowkey::compare(const ObCommonDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_valid() || !rhs.is_valid() || !datum_utils.is_valid())) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "Invalid argument to compare datum rowkey", K(ret), K(*this), K(rhs), K(datum_utils));
} else if (rhs.is_compact_rowkey()) {
ret = compare(*rhs.get_compact_rowkey(), datum_utils, cmp_ret, compare_datum_cnt);
} else {
ret = compare(*rhs.get_discrete_rowkey(), datum_utils, cmp_ret, compare_datum_cnt);
}
return ret;
}
int ObDiscreteDatumRowkey::deep_copy(ObDatumRowkey &dest, common::ObIAllocator &allocator) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_valid())) {
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "Unexpected error for deep copy invalid rowkey", K(ret), K(*this));
} else {
char *buf = nullptr;
int64_t deep_copy_size = 0;
if (OB_FAIL(rowkey_vector_->get_deep_copy_rowkey_size(row_idx_, deep_copy_size))) {
STORAGE_LOG(WARN, "Failed to get deep copy rowkey size", K(ret));
} else if (OB_UNLIKELY(deep_copy_size <= 0)) {
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "Unexpected deep copy size", K(ret), KPC(rowkey_vector_));
} else if (OB_ISNULL(buf = reinterpret_cast<char *>(allocator.alloc(deep_copy_size)))) {
ret = common::OB_ALLOCATE_MEMORY_FAILED;
STORAGE_LOG(WARN, "Failed to alloc memory for datum rowkey", K(ret), K(deep_copy_size));
} else if (OB_FAIL(rowkey_vector_->deep_copy_rowkey(row_idx_, dest, buf, deep_copy_size))) {
STORAGE_LOG(WARN, "Failed to deep copy datum rowkey", K(ret));
}
if (OB_FAIL(ret) && nullptr != buf) {
dest.reset();
allocator.free(buf);
}
}
return ret;
}
int ObDiscreteDatumRowkey::get_column_int(const int64_t col_idx, int64_t &int_val) const
{
return rowkey_vector_->get_column_int(row_idx_, col_idx, int_val);
}
int ObCommonDatumRowkey::compare(const ObDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_valid() || !rhs.is_valid() || !datum_utils.is_valid())) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "Invalid argument to compare datum rowkey", K(ret), K(*this), K(rhs), K(datum_utils));
} else if (is_compact_rowkey()) {
if (OB_FAIL(rowkey_->compare(rhs, datum_utils, cmp_ret, compare_datum_cnt))) {
STORAGE_LOG(WARN, "Failed to compare compact rowkey", K(ret));
}
} else if (OB_FAIL(discrete_rowkey_->compare(rhs, datum_utils, cmp_ret, compare_datum_cnt))) {
STORAGE_LOG(WARN, "Failed to compare discrete rowkey", K(ret));
}
return ret;
}
int ObCommonDatumRowkey::compare(const ObDiscreteDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt) const
{
int ret = OB_SUCCESS;
if (OB_FAIL(rhs.compare(*this, datum_utils, cmp_ret, compare_datum_cnt))) {
STORAGE_LOG(WARN, "Failed to compare", K(ret));
} else {
cmp_ret = -cmp_ret;
}
return ret;
}
int ObCommonDatumRowkey::compare(const ObCommonDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_valid() || !rhs.is_valid() || !datum_utils.is_valid())) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "Invalid argument to compare datum rowkey", K(ret), K(*this), K(rhs), K(datum_utils));
} else if (is_compact_rowkey()) {
if (OB_FAIL(rhs.compare(*rowkey_, datum_utils, cmp_ret, compare_datum_cnt))) {
STORAGE_LOG(WARN, "Failed to compare", K(ret));
} else {
cmp_ret = -cmp_ret;
}
} else {
ret = discrete_rowkey_->compare(rhs, datum_utils, cmp_ret, compare_datum_cnt);
}
return ret;
}
int ObCommonDatumRowkey::deep_copy(ObDatumRowkey &dest, common::ObIAllocator &allocator) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_valid())) {
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "Unexpected error for deep copy invalid rowkey", K(ret), K(*this));
} else if (is_compact_rowkey()) {
ret = rowkey_->deep_copy(dest, allocator);
} else {
ret = discrete_rowkey_->deep_copy(dest, allocator);
}
return ret;
}
int ObCommonDatumRowkey::get_column_int(const int64_t col_idx, int64_t &int_val) const
{
int ret = OB_SUCCESS;
int_val = 0;
if (is_compact_rowkey()) {
int_val = rowkey_->datums_[col_idx].get_int();
} else {
ret = discrete_rowkey_->get_column_int(col_idx, int_val);
}
return ret;
}
DEF_TO_STRING(ObCommonDatumRowkey)
{
int64_t pos = 0;
J_OBJ_START();
J_KV(K_(type), K_(key_ptr));
J_COMMA();
if (is_compact_rowkey()) {
KPC_(rowkey);
} else if (is_discrete_rowkey()) {
KPC_(discrete_rowkey);
}
J_OBJ_END();
return pos;
}
int ObDatumRowkeyHelper::convert_datum_rowkey(const common::ObRowkey &rowkey, ObDatumRowkey &datum_rowkey)
{
int ret = OB_SUCCESS;

View File

@ -24,6 +24,9 @@ namespace oceanbase
namespace blocksstable
{
struct ObDatumRange;
class ObRowkeyVector;
struct ObDiscreteDatumRowkey;
struct ObCommonDatumRowkey;
struct ObDatumRowkey
{
@ -76,6 +79,10 @@ public:
int equal(const ObDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, bool &is_equal) const;
int compare(const ObDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt = true) const;
int compare(const ObDiscreteDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt = true) const;
int compare(const ObCommonDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt = true) const;
int from_rowkey(const ObRowkey &rowkey, common::ObIAllocator &allocator);
int from_rowkey(const ObRowkey &rowkey, ObStorageDatumBuffer &datum_buffer);
int to_store_rowkey(const common::ObIArray<share::schema::ObColDesc> &col_descs,
@ -223,6 +230,66 @@ private:
ObDatumRowkey rowkey_;
};
struct ObDiscreteDatumRowkey
{
ObDiscreteDatumRowkey() : row_idx_(-1), rowkey_vector_(nullptr) {}
~ObDiscreteDatumRowkey() = default;
OB_INLINE bool is_valid() const { return row_idx_ >= 0 && nullptr != rowkey_vector_; }
int compare(const ObDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt = true) const;
int compare(const ObDiscreteDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt = true) const;
int compare(const ObCommonDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt = true) const;
int deep_copy(ObDatumRowkey &dest, common::ObIAllocator &allocator) const;
int get_column_int(const int64_t col_idx, int64_t &int_val) const;
TO_STRING_KV(K_(row_idx), KP_(rowkey_vector));
int64_t row_idx_;
const ObRowkeyVector *rowkey_vector_;
};
struct ObCommonDatumRowkey
{
enum RowkeyType {
NONE,
COMPACT,
DISCRETE,
};
ObCommonDatumRowkey() : type_(NONE), key_ptr_(nullptr) {}
~ObCommonDatumRowkey() = default;
OB_INLINE void reset() { type_ = NONE; key_ptr_ = nullptr; }
OB_INLINE bool is_valid() const { return (COMPACT == type_ || DISCRETE == type_) && (nullptr != key_ptr_); }
OB_INLINE bool is_compact_rowkey() const { return COMPACT == type_; }
OB_INLINE bool is_discrete_rowkey() const { return DISCRETE == type_; }
OB_INLINE void set_compact_rowkey(const ObDatumRowkey *rowkey)
{
type_ = COMPACT;
rowkey_ = rowkey;
}
OB_INLINE const ObDatumRowkey *get_compact_rowkey() const { return rowkey_; }
OB_INLINE void set_discrete_rowkey(const ObDiscreteDatumRowkey *discrete_rowkey)
{
type_ = DISCRETE;
discrete_rowkey_ = discrete_rowkey;
}
OB_INLINE const ObDiscreteDatumRowkey *get_discrete_rowkey() const { return discrete_rowkey_; }
int compare(const ObDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt = true) const;
int compare(const ObDiscreteDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt = true) const;
int compare(const ObCommonDatumRowkey &rhs, const ObStorageDatumUtils &datum_utils, int &cmp_ret,
const bool compare_datum_cnt = true) const;
int deep_copy(ObDatumRowkey &dest, common::ObIAllocator &allocator) const;
int get_column_int(const int64_t col_idx, int64_t &int_val) const;
DECLARE_TO_STRING;
RowkeyType type_;
union {
const void *key_ptr_;
const ObDatumRowkey *rowkey_;
const ObDiscreteDatumRowkey *discrete_rowkey_;
};
};
/*
*ObDatumRowkey
*/

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,504 @@
/**
* Copyright (c) 2023 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.
*/
#ifndef OB_STORAGE_BLOCKSSTABLE_DATUM_ROWKEY_VECTOR_H_
#define OB_STORAGE_BLOCKSSTABLE_DATUM_ROWKEY_VECTOR_H_
#include "storage/blocksstable/ob_datum_range.h"
#include "storage/blocksstable/ob_datum_row.h"
#include "storage/access/ob_table_read_info.h"
namespace oceanbase
{
namespace blocksstable
{
class ObRowkeyVectorHelper
{
public:
OB_INLINE static bool can_use_non_datum_rowkey_vector(const bool is_cg, const ObTabletID &tablet_id)
{
return !is_cg && !tablet_id.is_inner_tablet();
}
};
enum class ObColumnVectorType: int8_t
{
UNKNOW_TYPE = -1,
DATUM_TYPE = 0,
SIGNED_INTEGER_TYPE = 1,
UNSIGNED_INTEGER_TYPE = 2,
MAX_TYPE = 3,
};
OB_INLINE bool is_valid_vector_type(const ObColumnVectorType type)
{
return type > ObColumnVectorType::UNKNOW_TYPE && type < ObColumnVectorType::MAX_TYPE;
}
template<typename T>
struct ObRVIntegerCell
{
ObRVIntegerCell() = default;
ObRVIntegerCell(T d, const bool null) : d_(d), null_(null) {}
~ObRVIntegerCell() = default;
TO_STRING_KV(K_(d), K_(null));
T d_;
bool null_;
};
class ObColumnVector
{
public:
ObColumnVector() : flag_(0), data_(nullptr), nulls_(nullptr)
{
type_ = ObColumnVectorType::UNKNOW_TYPE;
}
virtual ~ObColumnVector() = default;
OB_INLINE bool is_valid() const { return is_valid_vector_type(type_) && nullptr != data_ && row_cnt_ > 0 && is_filled_; }
int locate_key(
const bool need_upper_bound,
int64_t &begin,
int64_t &end,
const ObStorageDatum &key,
const ObStorageDatumCmpFunc &cmp_func,
const bool is_oracle_mode) const;
int fill_column_datum(
char *buf,
int64_t &pos,
const int64_t buf_size,
const int64_t row_idx,
const ObStorageDatum &datum);
int get_deep_copy_size(const int64_t row_idx, int64_t &size) const;
int get_column_datum(const int64_t row_idx, ObStorageDatum &dst, char *buf, const int64_t buf_size, int64_t &pos);
int get_column_int(const int64_t row_idx, int64_t &int_val) const;
int deep_copy(
char *buf,
int64_t &pos,
const int64_t buf_size,
const ObColumnVector &other);
template<typename T>
int inner_locate_key(
const bool need_upper_bound,
int64_t &begin,
int64_t &end,
const ObStorageDatum &key,
const ObStorageDatumCmpFunc &cmp_func,
const bool is_oracle_mode) const;
template<typename T>
int locate_integer_key(
const bool need_upper_bound,
int64_t &begin,
int64_t &end,
const T *data,
const T key) const;
template<typename T>
int locate_integer_key_with_null(
const bool need_upper_bound,
int64_t &begin,
int64_t &end,
T *data,
const ObRVIntegerCell<T> &cell,
const bool is_oracle_mode) const;
static int construct_column_vector(
char *buf,
int64_t &pos,
const int64_t buf_size,
const int64_t row_cnt,
const ObObjMeta *obj_meta,
ObColumnVector &vector);
static int construct_datum_vector(
char *buf,
int64_t &pos,
const int64_t buf_size,
const int64_t row_cnt,
ObColumnVector &vector);
static int construct_integer_vector(
char *buf,
int64_t &pos,
const int64_t buf_size,
const int64_t row_cnt,
const bool is_signed,
ObColumnVector &vector);
DECLARE_TO_STRING;
union {
struct {
ObColumnVectorType type_ : 8;
int64_t has_null_ : 1;
int64_t row_cnt_ : 32;
int64_t is_filled_: 1;
int64_t reserved_ : 22;
};
int64_t flag_;
};
union {
ObStorageDatum *datums_;
int64_t *signed_ints_;
uint64_t *unsigned_ints_;
void *data_;
};
bool *nulls_;
};
class ObRowkeyVector
{
public:
ObRowkeyVector()
: flag_(0),
columns_(nullptr),
last_rowkey_(nullptr),
discrete_rowkey_array_(nullptr)
{}
~ObRowkeyVector() {};
int locate_key(
const int64_t begin,
const int64_t end,
const ObDatumRowkey &rowkey,
const ObStorageDatumUtils &datum_utils,
int64_t &rowkey_idx,
const bool is_lower_bound = true) const;
int locate_range(
const ObDatumRange &range,
const bool is_left_border,
const bool is_right_border,
const bool is_normal_cg,
const ObStorageDatumUtils &datum_utils,
int64_t &endkey_begin_idx,
int64_t &endkey_end_idx) const;
int compare_rowkey(
const ObDatumRowkey &rowkey,
const int64_t row_idx,
const ObStorageDatumUtils &datum_utils,
int &cmp_ret,
const bool compare_datum_cnt) const;
int compare_rowkey(
const ObDiscreteDatumRowkey &rowkey,
const int64_t row_idx,
const ObStorageDatumUtils &datum_utils,
int &cmp_ret,
const bool compare_datum_cnt) const;
int get_rowkey(const int64_t row_idx, ObDatumRowkey &rowkey) const;
int get_rowkey(const int64_t row_idx, ObCommonDatumRowkey &common_rowkey) const;
int get_deep_copy_rowkey_size(const int64_t row_idx, int64_t &size) const;
int deep_copy_rowkey(const int64_t row_idx, ObDatumRowkey &dest, char *buf, const int64_t buf_size) const;
int get_column_int(const int64_t row_idx, const int64_t col_idx, int64_t &int_val) const;
int fill_last_rowkey();
int set_construct_finished();
OB_INLINE const ObDatumRowkey *get_last_rowkey() const
{
return last_rowkey_;
}
int deep_copy(
char *buf,
int64_t &pos,
const int64_t buf_size,
const ObRowkeyVector &other);
static int get_occupied_size(
const int64_t row_cnt,
const int64_t col_cnt,
const ObIArray<share::schema::ObColDesc> *col_descs,
int64_t &size);
static int construct_rowkey_vector(
const int64_t row_cnt,
const int64_t col_cnt,
const ObIArray<share::schema::ObColDesc> *col_descs,
char *buf,
int64_t &pos,
const int64_t buf_size,
ObRowkeyVector *&rowkey_vector);
static int prepare_rowkeys_buffer(
const int64_t row_cnt,
const int64_t col_cnt,
char *buf,
int64_t &pos,
const int64_t buf_size,
ObRowkeyVector *rowkey_vector);
static bool is_all_integer_cols(const int64_t col_cnt, const ObIArray<share::schema::ObColDesc> *col_descs);
DECLARE_TO_STRING;
private:
int compare_datum_rowkey(
const ObDatumRowkey &rowkey,
const int64_t row_idx,
const ObStorageDatumUtils &datum_utils,
const int64_t cmp_cnt,
int &cmp_ret,
const bool compare_datum_cnt) const;
public:
union {
struct {
int64_t col_cnt_ : 16;
int64_t row_cnt_ : 32;
int64_t is_datum_vectors_ : 1;
int64_t reserved_ : 15;
};
int64_t flag_;
};
ObColumnVector *columns_;
ObDatumRowkey *last_rowkey_;
ObDiscreteDatumRowkey *discrete_rowkey_array_;
};
template<typename T>
int ObColumnVector::locate_integer_key(
const bool need_upper_bound,
int64_t &begin,
int64_t &end,
const T *data,
const T key) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(begin >= end)) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "Invalid argument", K(ret), K(begin), K(end));
} else {
const T *first = data + begin;
const T *last = data + end;
const T *lb = std::lower_bound(first, last, key);
if (lb == last) {
begin = end;
} else if (FALSE_IT(begin = lb - data)) {
} else if (need_upper_bound) {
if (OB_UNLIKELY(*lb < key)) {
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "Unexpected cmp ret", K(ret), K(*lb), K(key), K(begin), K(end));
} else if (*lb > key) {
end = begin;
} else {
const T *ub = std::upper_bound(lb, last, key);
if (ub == last) {
} else {
end = ub - data;
}
}
} else {
end = begin;
}
}
return ret;
}
template<typename T>
struct ObRVIntegerIterator
{
typedef std::random_access_iterator_tag iterator_category;
typedef ObRVIntegerCell<T> value_type;
typedef int64_t difference_type;
typedef T* pointer;
typedef T& reference;
ObRVIntegerIterator(const int64_t pos, T *data, const bool *nulls)
: pos_(pos), data_(data), nulls_(nulls) {}
~ObRVIntegerIterator() = default;
OB_INLINE value_type &operator*()
{
cell_.d_ = data_[pos_];
cell_.null_ = nulls_[pos_];
return cell_;
}
OB_INLINE ObRVIntegerIterator operator--()
{
pos_--;
return *this;
}
OB_INLINE ObRVIntegerIterator operator--(int)
{
return ObRVIntegerIterator(pos_--, data_, nulls_);
}
OB_INLINE ObRVIntegerIterator operator++()
{
pos_++;
return *this;
}
OB_INLINE ObRVIntegerIterator operator++(int)
{
return ObRVIntegerIterator(pos_++, data_, nulls_);
}
OB_INLINE ObRVIntegerIterator &operator+(int64_t offset)
{
pos_ += offset;
return *this;
}
OB_INLINE ObRVIntegerIterator &operator+=(int64_t offset)
{
pos_ += offset;
return *this;
}
OB_INLINE difference_type operator-(const ObRVIntegerIterator &rhs)
{
return pos_ - rhs.pos_;
}
OB_INLINE ObRVIntegerIterator &operator-(int64_t offset)
{
pos_ -= offset;
return *this;
}
OB_INLINE bool operator==(const ObRVIntegerIterator &rhs) const
{
return pos_ == rhs.pos_;
}
OB_INLINE bool operator!=(const ObRVIntegerIterator &rhs)
{
return pos_ != rhs.pos_;
}
OB_INLINE bool operator<(const ObRVIntegerIterator &rhs)
{
return pos_ < rhs.pos_;
}
OB_INLINE bool operator<=(const ObRVIntegerIterator &rhs)
{
return pos_ <= rhs.pos_;
}
OB_INLINE bool is_null() const
{
return nulls_[pos_];
}
int64_t pos_;
T *data_;
const bool *nulls_;
value_type cell_;
};
// null first
// [left_is_null][right_is_null][left>right][left==right]
static int NULL_FIRST_CMP_RET[2][2][2][2] =
{
// left_is_null = false
{
// right_is_null = false
{
// left > right = false
{-1, 0},
// left > right = true
{1, 1}
},
// right_is_null = true
{
{1, 1},
{1, 1}
}
},
// left_is_null = true;
{
// right_is_null = false
{
{-1, -1},
{-1, -1}
},
// right_is_null = true
{
{0, 0},
{0, 0}
}
}
};
// null last
// [left_is_null][right_is_null][left>right][left==right]
static int NULL_LAST_CMP_RET[2][2][2][2] =
{
// left_is_null = false
{
// right_is_null = false
{
// left > right = false
{-1, 0},
// left > right = true
{1, 1}
},
// right_is_null = true
{
{-1, -1},
{-1, -1}
}
},
// left_is_null = true;
{
// right_is_null = false
{
{1, 1},
{1, 1}
},
// right_is_null = true
{
{0, 0},
{0, 0}
}
}
};
template<typename T>
class ObRVIntegerWithNullComparor
{
public:
ObRVIntegerWithNullComparor(const bool null_first) : null_first_(null_first) {}
~ObRVIntegerWithNullComparor() = default;
bool operator() (const ObRVIntegerCell<T> &left, const ObRVIntegerCell<T> &right)
{
return compare(left, right) < 0;
}
int compare(const ObRVIntegerCell<T> &left, const ObRVIntegerCell<T> &right)
{
int cmp_ret = 0;
if (null_first_) {
cmp_ret = NULL_FIRST_CMP_RET[left.null_][right.null_][left.d_ > right.d_][left.d_ == right.d_];
} else {
cmp_ret = NULL_LAST_CMP_RET[left.null_][right.null_][left.d_ > right.d_][left.d_ == right.d_];
}
return cmp_ret;
}
private:
bool null_first_;
};
template<typename T>
int ObColumnVector::locate_integer_key_with_null(
const bool need_upper_bound,
int64_t &begin,
int64_t &end,
T *data,
const ObRVIntegerCell<T> &cell,
const bool is_oracle_mode) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(begin >= end || (!has_null_ && !cell.null_))) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "Invalid argument", K(ret), K(begin), K(end), K(has_null_), K(cell));
} else {
ObRVIntegerWithNullComparor<T> comparor(!is_oracle_mode);
const ObRVIntegerIterator<T> first(begin, data, nulls_);
const ObRVIntegerIterator<T> last(end, data, nulls_);
ObRVIntegerIterator<T> lb = std::lower_bound(first, last, cell, comparor);
if (lb == last) {
begin = end;
} else if (FALSE_IT(begin = lb.pos_)) {
} else if (need_upper_bound) {
const int cmp_ret = comparor.compare(*lb, cell);
if (OB_UNLIKELY(cmp_ret < 0)) {
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "Unexpected cmp ret", K(ret), K(cmp_ret), K(*lb), K(cell), K(begin), K(end));
} else if (cmp_ret > 0) {
end = begin;
} else {
ObRVIntegerIterator<T> ub = std::upper_bound(lb, last, cell, comparor);
if (ub == last) {
} else {
end = ub.pos_;
}
}
} else {
end = begin;
}
}
return ret;
}
} // namespace blocksstable
} // namespace oceanbase
#endif

View File

@ -133,6 +133,8 @@ public:
int64_t &get_extra_size() { return extra_size_; }
int64_t total_size() const { return size_ + extra_size_; }
bool is_normal_block() const { return INDEX_BLOCK == type_ || DATA_BLOCK == type_; }
bool is_index_block() const { return INDEX_BLOCK == type_ || DDL_BLOCK_TREE == type_ || DDL_MERGE_INDEX_BLOCK == type_;}
void reset() { *this = ObMicroBlockData(); }

View File

@ -273,7 +273,7 @@ int ObMicroBlockBareIterator::get_next_micro_block_desc(
micro_block_desc.max_merged_trans_version_ = header->max_merged_trans_version_; // do not get from index
micro_block_desc.is_last_row_last_flag_ = header->is_last_row_last_flag_;
micro_block_desc.original_size_ = header->original_length_;
micro_index_info.endkey_ = &micro_block_desc.last_rowkey_;
micro_index_info.endkey_.set_compact_rowkey(&micro_block_desc.last_rowkey_);
micro_block_desc.has_string_out_row_ = micro_index_info.has_string_out_row();
micro_block_desc.has_lob_out_row_ = micro_index_info.has_lob_out_row();

View File

@ -996,7 +996,7 @@ int ObMacroBlockWriter::init_data_pre_warmer(const ObMacroDataSeq &start_seq)
need_pre_warm = true;
}
if (need_pre_warm) {
if (OB_TMP_FAIL(data_block_pre_warmer_.init())) {
if (OB_TMP_FAIL(data_block_pre_warmer_.init(nullptr))) {
STORAGE_LOG(WARN, "Failed to init pre warmer", K(tmp_ret));
}
}

View File

@ -312,7 +312,8 @@ ObIMicroBlockIOCallback::ObIMicroBlockIOCallback()
block_id_(),
offset_(0),
block_des_meta_(),
use_block_cache_(true)
use_block_cache_(true),
rowkey_col_descs_(nullptr)
{
MEMSET(encrypt_key_, 0, sizeof(encrypt_key_));
block_des_meta_.encrypt_key_ = encrypt_key_;
@ -326,6 +327,7 @@ ObIMicroBlockIOCallback::~ObIMicroBlockIOCallback()
data_buffer_ = nullptr;
}
allocator_ = nullptr;
rowkey_col_descs_ = nullptr;
}
void ObIMicroBlockIOCallback::set_micro_des_meta(const ObIndexBlockRowHeader *idx_row_header)
@ -401,7 +403,7 @@ int ObIMicroBlockIOCallback::process_block(
} else if (OB_UNLIKELY(OB_SUCCESS == (ret = kvcache->get(key, micro_block, cache_handle)))) {
// entry exist, no need to put
} else if (OB_FAIL(cache_->put_cache_block(
block_des_meta_, buffer, key, *reader, *allocator_, micro_block, cache_handle))) {
block_des_meta_, buffer, key, *reader, *allocator_, micro_block, cache_handle, rowkey_col_descs_))) {
LOG_WARN("Failed to put block to cache", K(ret));
}
}
@ -794,7 +796,7 @@ int ObIMicroBlockCache::prefetch(
callback = new (buf) ObAsyncSingleMicroBlockIOCallback;
callback->allocator_ = allocator;
callback->use_block_cache_ = use_cache;
if (OB_FAIL(prefetch(tenant_id, macro_id, idx_row, macro_handle, *callback))) {
if (OB_FAIL(prefetch(tenant_id, macro_id, idx_row, macro_handle, *callback))) {
LOG_WARN("Fail to prefetch data micro block", K(ret));
}
if (OB_FAIL(ret) && OB_NOT_NULL(callback->get_allocator())) { //Avoid double_free with io_handle
@ -825,6 +827,7 @@ int ObIMicroBlockCache::prefetch(
callback.tenant_id_ = tenant_id;
callback.block_id_ = macro_id;
callback.offset_ = idx_row.get_block_offset();
callback.set_rowkey_col_descs(idx_row.get_rowkey_col_descs());
callback.set_micro_des_meta(idx_row_header);
// fill read info
ObMacroBlockReadInfo read_info;
@ -1186,7 +1189,8 @@ int ObDataMicroBlockCache::put_cache_block(
ObMacroBlockReader &reader,
ObIAllocator &allocator,
const ObMicroBlockCacheValue *&micro_block,
common::ObKVCacheHandle &cache_handle)
common::ObKVCacheHandle &cache_handle,
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs)
{
UNUSED(allocator);
int ret = OB_SUCCESS;
@ -1329,6 +1333,22 @@ ObMicroBlockData::Type ObDataMicroBlockCache::get_type()
return ObMicroBlockData::DATA_BLOCK;
}
void ObDataMicroBlockCache::cache_bypass()
{
EVENT_INC(ObStatEventIds::DATA_BLOCK_READ_CNT);
}
void ObDataMicroBlockCache::cache_hit(int64_t &hit_cnt)
{
++hit_cnt;
EVENT_INC(ObStatEventIds::DATA_BLOCK_CACHE_HIT);
}
void ObDataMicroBlockCache::cache_miss(int64_t &miss_cnt)
{
++miss_cnt;
EVENT_INC(ObStatEventIds::DATA_BLOCK_READ_CNT);
}
/*-------------------------------------ObIndexMicroBlockCache-------------------------------------*/
ObIndexMicroBlockCache::ObIndexMicroBlockCache()
@ -1425,7 +1445,8 @@ int ObIndexMicroBlockCache::put_cache_block(
ObMacroBlockReader &reader,
ObIAllocator &allocator,
const ObMicroBlockCacheValue *&micro_block,
common::ObKVCacheHandle &cache_handle)
common::ObKVCacheHandle &cache_handle,
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs)
{
int ret = OB_SUCCESS;
ObMicroBlockHeader header;
@ -1464,7 +1485,7 @@ int ObIndexMicroBlockCache::put_cache_block(
ObMicroBlockData &block_data = cache_value.get_block_data();
block_data.type_ = get_type();
char *allocated_buf = nullptr;
if (OB_FAIL(idx_transformer.transform(block_data, block_data, allocator, allocated_buf))) {
if (OB_FAIL(idx_transformer.transform(block_data, block_data, allocator, allocated_buf, rowkey_col_descs))) {
LOG_WARN("Fail to transform index block to memory format", K(ret));
} else if (OB_FAIL(put_and_fetch(key, cache_value, micro_block, cache_handle, false /* overwrite */))) {
if (OB_ENTRY_EXIST != ret) {
@ -1499,6 +1520,22 @@ ObMicroBlockData::Type ObIndexMicroBlockCache::get_type()
return ObMicroBlockData::INDEX_BLOCK;
}
void ObIndexMicroBlockCache::cache_bypass()
{
EVENT_INC(ObStatEventIds::INDEX_BLOCK_READ_CNT);
}
void ObIndexMicroBlockCache::cache_hit(int64_t &hit_cnt)
{
++hit_cnt;
EVENT_INC(ObStatEventIds::INDEX_BLOCK_CACHE_HIT);
}
void ObIndexMicroBlockCache::cache_miss(int64_t &miss_cnt)
{
++miss_cnt;
EVENT_INC(ObStatEventIds::INDEX_BLOCK_READ_CNT);
}
}//end namespace blocksstable
}//end namespace oceanbase

View File

@ -96,6 +96,10 @@ public:
{ return is_valid() ? &(micro_block_->get_block_data()) : NULL; }
int64_t get_block_size() const { return is_valid() ? micro_block_->get_block_data().total_size() : 0; }
inline bool is_valid() const { return NULL != micro_block_ && handle_.is_valid(); }
inline ObKVMemBlockHandle* get_mb_handle() const { return handle_.get_mb_handle(); }
inline const ObMicroBlockCacheValue* get_micro_block() const { return micro_block_; }
inline void set_mb_handle(ObKVMemBlockHandle *mb_handle) { handle_.set_mb_handle(mb_handle); }
inline void set_micro_block(const ObMicroBlockCacheValue *micro_block) { micro_block_ = micro_block; }
TO_STRING_KV(K_(handle), KP_(micro_block));
private:
friend class ObIMicroBlockCache;
@ -160,6 +164,10 @@ public:
virtual int alloc_data_buf(const char *io_data_buffer, const int64_t data_size);
virtual ObIAllocator *get_allocator() { return allocator_; }
void set_micro_des_meta(const ObIndexBlockRowHeader *idx_row_header);
OB_INLINE void set_rowkey_col_descs(const ObIArray<share::schema::ObColDesc> *rowkey_col_descs)
{
rowkey_col_descs_ = rowkey_col_descs;
}
protected:
friend class ObIMicroBlockCache;
friend class ObDataMicroBlockCache;
@ -192,6 +200,7 @@ protected:
ObMicroBlockDesMeta block_des_meta_;
bool use_block_cache_;
char encrypt_key_[share::OB_MAX_TABLESPACE_ENCRYPT_KEY_LENGTH];
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs_;
DISALLOW_COPY_AND_ASSIGN(ObIMicroBlockIOCallback);
};
@ -310,7 +319,8 @@ public:
ObMacroBlockReader &reader,
ObIAllocator &allocator,
const ObMicroBlockCacheValue *&micro_block,
common::ObKVCacheHandle &cache_handle) = 0;
common::ObKVCacheHandle &cache_handle,
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs = nullptr) = 0;
virtual int reserve_kvpair(
const ObMicroBlockDesc &micro_block_desc,
ObKVCacheInstHandle &inst_handle,
@ -319,6 +329,9 @@ public:
int64_t &kvpair_size) = 0;
virtual ObMicroBlockData::Type get_type() = 0;
virtual int add_put_size(const int64_t put_size) override;
virtual void cache_bypass() = 0;
virtual void cache_hit(int64_t &hit_cnt) = 0;
virtual void cache_miss(int64_t &miss_cnt) = 0;
protected:
int prefetch(
@ -367,7 +380,8 @@ public:
ObMacroBlockReader &reader,
ObIAllocator &allocator,
const ObMicroBlockCacheValue *&micro_block,
common::ObKVCacheHandle &cache_handle) override;
common::ObKVCacheHandle &cache_handle,
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs = nullptr) override;
virtual int reserve_kvpair(
const ObMicroBlockDesc &micro_block_desc,
ObKVCacheInstHandle &inst_handle,
@ -375,6 +389,9 @@ public:
ObKVCachePair *&kvpair,
int64_t &kvpair_size) override;
virtual ObMicroBlockData::Type get_type() override;
virtual void cache_bypass();
virtual void cache_hit(int64_t &hit_cnt);
virtual void cache_miss(int64_t &miss_cnt);
private:
int64_t calc_value_size(const int64_t data_length, const ObRowStoreType &type, bool &need_decoder);
int write_extra_buf(
@ -407,7 +424,8 @@ public:
ObMacroBlockReader &reader,
ObIAllocator &allocator,
const ObMicroBlockCacheValue *&micro_block,
common::ObKVCacheHandle &cache_handle) override;
common::ObKVCacheHandle &cache_handle,
const ObIArray<share::schema::ObColDesc> *rowkey_col_descs = nullptr) override;
virtual int reserve_kvpair(
const ObMicroBlockDesc &micro_block_desc,
ObKVCacheInstHandle &inst_handle,
@ -419,6 +437,9 @@ public:
return OB_NOT_SUPPORTED;
}
virtual ObMicroBlockData::Type get_type() override;
virtual void cache_bypass();
virtual void cache_hit(int64_t &hit_cnt);
virtual void cache_miss(int64_t &miss_cnt);
};

View File

@ -285,7 +285,7 @@ int ObMicroBlockRowGetter::get_block_row(
}
} else {
if (store_row->row_flag_.is_not_exist()) {
++context_->table_store_stat_.get_row_.empty_read_cnt_;
++context_->table_store_stat_.empty_read_cnt_;
EVENT_INC(ObStatEventIds::GET_ROW_EMPTY_READ);
if (!context_->query_flag_.is_index_back()
&& context_->query_flag_.is_use_bloomfilter_cache()
@ -295,12 +295,8 @@ int ObMicroBlockRowGetter::get_block_row(
param_->table_id_,
read_handle.micro_handle_->macro_block_id_,
read_handle.get_rowkey().get_datum_cnt());
if (read_handle.is_bf_contain_) {
++context_->table_store_stat_.bf_empty_read_cnt_;
}
}
} else {
++context_->table_store_stat_.get_row_.effect_read_cnt_;
EVENT_INC(ObStatEventIds::GET_ROW_EFFECT_READ);
}
}

View File

@ -15,7 +15,6 @@
#include "storage/blocksstable/ob_micro_block_reader.h"
#include "storage/blocksstable/encoding/ob_micro_block_decoder.h"
#include "storage/ob_table_store_stat_mgr.h"
#include "ob_datum_row.h"
#include "ob_row_cache.h"

View File

@ -362,7 +362,7 @@ int ObIMicroBlockRowScanner::inner_get_next_row_blockscan(const ObDatumRow *&row
int ObIMicroBlockRowScanner::apply_blockscan(
storage::ObBlockRowStore *block_row_store,
storage::ObTableStoreStat &table_store_stat)
storage::ObTableScanStoreStat &table_store_stat)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(nullptr == block_row_store || !block_row_store->is_valid() || nullptr == reader_)) {

View File

@ -30,7 +30,7 @@ struct ObTableIterParam;
struct ObTableAccessContext;
struct ObRowSampleFilter;
class ObBlockRowStore;
class ObTableStoreStat;
class ObTableScanStoreStat;
class ObCGAggCells;
}
namespace blocksstable
@ -65,7 +65,7 @@ public:
virtual int get_next_rows();
virtual int apply_blockscan(
storage::ObBlockRowStore *block_row_store,
storage::ObTableStoreStat &table_store_stat);
storage::ObTableScanStoreStat &table_store_stat);
virtual int set_ignore_shadow_row() { return OB_NOT_SUPPORTED;}
int end_of_block() const;
OB_INLINE int get_access_cnt() const { return reverse_scan_ ? (current_ - last_ + 1) : (last_ - current_ + 1);}
@ -373,7 +373,7 @@ public:
void reuse() override;
virtual int apply_blockscan(
storage::ObBlockRowStore *block_row_store,
storage::ObTableStoreStat &table_store_stat) override final
storage::ObTableScanStoreStat &table_store_stat) override final
{
UNUSEDx(block_row_store, table_store_stat);
return OB_NOT_SUPPORTED;

View File

@ -620,7 +620,7 @@ int ObSSTable::exist(
ObTabletStat &stat = context.store_ctx_->tablet_stat_;
stat.ls_id_ = context.ls_id_.id();
stat.tablet_id_ = context.tablet_id_.id();
stat.query_cnt_ = context.table_store_stat_.exist_row_.empty_read_cnt_ > 0;
stat.query_cnt_ = context.table_store_stat_.empty_read_cnt_ > 0;
iter->~ObStoreRowIterator();
context.stmt_allocator_->free(iter);
@ -1844,8 +1844,9 @@ int ObSSTable::get_last_rowkey(const ObDatumRowkey *&sstable_endkey)
} else if (OB_UNLIKELY(!idx_data_header->is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid index data header", KP(idx_data_header));
} else {
sstable_endkey = &(idx_data_header->rowkey_array_[idx_data_header->row_cnt_ - 1]);
} else if (OB_ISNULL(sstable_endkey = idx_data_header->rowkey_vector_->get_last_rowkey())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid last rowkey", K(ret), KP(idx_data_header));
}
}
}

View File

@ -12,7 +12,6 @@
#define USING_LOG_PREFIX STORAGE
#include "ob_cg_scanner.h"
#include "common/sql_mode/ob_sql_mode_utils.h"
#include "storage/ob_table_store_stat_mgr.h"
namespace oceanbase
{

View File

@ -420,10 +420,17 @@ int ObCOPrefetcher::ObCOIndexTreeLevelHandle::try_advancing_fetch_idx(
const int32_t idx = prefetch_idx % INDEX_TREE_PREFETCH_DEPTH;
ObMicroIndexInfo &index_info = index_block_read_handles_[idx].index_info_;
bool is_range_end = !is_reverse && index_info.is_right_border();
const ObDatumRowkey &endkey = prefetch_idx == fetch_idx_ ? index_scanner_.get_end_key()
: *(index_info.endkey_);
ObCommonDatumRowkey endkey;
if (prefetch_idx == fetch_idx_) {
if (OB_FAIL(index_scanner_.get_end_key(endkey))) {
LOG_WARN("Failed to get end key", K(ret));
}
} else {
endkey = index_info.endkey_;
}
// For reverse scan, there is no need to judge is_range_end.
if (range_idx != index_info.range_idx()) {
if (OB_FAIL(ret)) {
} else if (range_idx != index_info.range_idx()) {
} else {
int cmp_ret = 0;
if (OB_FAIL(endkey.compare(border_rowkey, *prefetcher.datum_utils_,

View File

@ -708,15 +708,12 @@ int ObCOSSTableRowScanner::inner_filter(
} else {
int64_t select_cnt = result_bitmap->popcnt();
EVENT_ADD(ObStatEventIds::PUSHDOWN_STORAGE_FILTER_ROW_CNT, select_cnt);
access_ctx_->table_store_stat_.pushdown_row_select_cnt_ += select_cnt;
}
} else {
EVENT_ADD(ObStatEventIds::PUSHDOWN_STORAGE_FILTER_ROW_CNT, group_size);
access_ctx_->table_store_stat_.pushdown_row_select_cnt_ += group_size;
}
if (OB_SUCC(ret)) {
EVENT_ADD(ObStatEventIds::BLOCKSCAN_ROW_CNT, group_size);
access_ctx_->table_store_stat_.pushdown_row_access_cnt_ += group_size;
access_ctx_->table_store_stat_.logical_read_cnt_ += group_size;
access_ctx_->table_store_stat_.physical_read_cnt_ += group_size;
LOG_TRACE("[COLUMNSTORE] COSSTableRowScanner inner filter", K(ret), "begin", begin, "count", group_size,
@ -972,7 +969,6 @@ int ObCOSSTableRowScanner::filter_group_by_rows()
LOG_WARN("Unexpected result bitmap", K(ret), KPC(rows_filter_));
} else {
EVENT_ADD(ObStatEventIds::PUSHDOWN_STORAGE_FILTER_ROW_CNT, result_bitmap->popcnt());
access_ctx_->table_store_stat_.pushdown_row_select_cnt_ += result_bitmap->popcnt();
if (result_bitmap->is_all_false()) {
update_current(group_size_);
continue;
@ -980,7 +976,6 @@ int ObCOSSTableRowScanner::filter_group_by_rows()
}
} else {
EVENT_ADD(ObStatEventIds::PUSHDOWN_STORAGE_FILTER_ROW_CNT, group_size_);
access_ctx_->table_store_stat_.pushdown_row_select_cnt_ += group_size_;
}
if (OB_SUCC(ret)) {
break;
@ -994,7 +989,6 @@ int ObCOSSTableRowScanner::filter_group_by_rows()
} else {
is_new_group_ = true;
EVENT_ADD(ObStatEventIds::BLOCKSCAN_ROW_CNT, group_size_);
access_ctx_->table_store_stat_.pushdown_row_access_cnt_ += group_size_;
access_ctx_->table_store_stat_.logical_read_cnt_ += group_size_;
access_ctx_->table_store_stat_.physical_read_cnt_ += group_size_;
}

View File

@ -15,7 +15,6 @@
#include "lib/ob_define.h"
#include "lib/utility/ob_print_utils.h"
#include "lib/container/ob_iarray.h"
#include "storage/ob_table_store_stat_mgr.h"
#include "storage/blocksstable/ob_datum_range.h"
namespace oceanbase

View File

@ -113,7 +113,8 @@ int ObLobLocatorHelper::init(const ObTableScanParam &scan_param,
return ret;
}
int ObLobLocatorHelper::init(const ObTableStoreStat &table_store_stat,
int ObLobLocatorHelper::init(const uint64_t table_id,
const uint64_t tablet_id,
const ObStoreCtx &ctx,
const share::ObLSID &ls_id,
const int64_t snapshot_version)
@ -129,8 +130,8 @@ int ObLobLocatorHelper::init(const ObTableStoreStat &table_store_stat,
rowid_version_ = ObURowIDData::INVALID_ROWID_VERSION;
rowid_project_ = NULL;
// table id只用来判断是不是systable, 这个接口创建的locator不会构造真正的rowid
table_id_ = table_store_stat.table_id_;
tablet_id_ = table_store_stat.tablet_id_.id();
table_id_ = table_id;
tablet_id_ = tablet_id;
ls_id_ = ls_id.id();
read_snapshot_ = ctx.mvcc_acc_ctx_.snapshot_;
enable_locator_v2_ = true; // must be called en locator v2 enabled

View File

@ -36,7 +36,8 @@ public:
const ObStoreCtx &ctx,
const share::ObLSID &ls_id,
const int64_t snapshot_version);
int init(const ObTableStoreStat &table_store_stat,
int init(const uint64_t table_id,
const uint64_t tablet_id,
const ObStoreCtx &ctx,
const share::ObLSID &ls_id,
const int64_t snapshot_version);

View File

@ -3852,7 +3852,7 @@ int ObTablet::do_rowkeys_exist(
tablet_stat.exist_row_read_table_cnt_ = check_table_cnt;
tablet_stat.exist_row_total_table_cnt_ = tables_iter.count();
int tmp_ret = OB_SUCCESS;
if (0 == access_ctx.table_store_stat_.exist_row_.empty_read_cnt_) {
if (0 == access_ctx.table_store_stat_.empty_read_cnt_) {
// ROWKEY IN_ROW_CACHE / NOT EXIST
} else if (MTL(ObTenantTabletScheduler *)->enable_adaptive_compaction()) {
bool report_succ = false; /*placeholder*/

View File

@ -24,6 +24,7 @@ storage_unittest(test_agg_row_struct)
storage_unittest(test_skip_index_filter)
storage_unittest(test_sstable_index_filter)
storage_unittest(test_data_store_desc)
storage_unittest(test_datum_rowkey_vector)
add_subdirectory(encoding)
add_subdirectory(cs_encoding)

View File

@ -0,0 +1,931 @@
/**
* Copyright (c) 2023 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 <chrono>
#include <gtest/gtest.h>
#define private public
#define protected public
#include "lib/oblog/ob_log_module.h"
#include "storage/blocksstable/ob_datum_rowkey_vector.h"
namespace oceanbase
{
using namespace std::chrono;
using namespace share;
using namespace common;
using namespace storage;
namespace blocksstable
{
class ObDatumRowkeyVectorTest: public ::testing::Test
{
public:
ObDatumRowkeyVectorTest();
virtual ~ObDatumRowkeyVectorTest() = default;
virtual void SetUp() override;
virtual void TearDown() override;
static void SetUpTestCase();
static void TearDownTestCase();
private:
void prepare_datum_util(const int64_t rowkey_cnt, ObStorageDatumUtils &datum_util);
bool is_oracle_mode_;
ObArenaAllocator allocator_;
};
ObDatumRowkeyVectorTest::ObDatumRowkeyVectorTest()
{
is_oracle_mode_ = false;
}
void ObDatumRowkeyVectorTest::SetUpTestCase()
{
}
void ObDatumRowkeyVectorTest::TearDownTestCase()
{
}
void ObDatumRowkeyVectorTest::SetUp()
{
}
void ObDatumRowkeyVectorTest::TearDown()
{
}
void ObDatumRowkeyVectorTest::prepare_datum_util(const int64_t rowkey_cnt, ObStorageDatumUtils &datum_util)
{
int ret = OB_SUCCESS;
ASSERT_TRUE(rowkey_cnt < 16);
ObSEArray<share::schema::ObColDesc, 16> cols_desc;
for (int64_t i = 0; i < rowkey_cnt; ++i) {
share::schema::ObColDesc col_desc;
col_desc.col_id_ = 16 + i;
col_desc.col_type_.set_type(ObIntType);
col_desc.col_type_.set_collation_type(CS_TYPE_UTF8MB4_GENERAL_CI);
col_desc.col_type_.set_collation_level(CS_LEVEL_IMPLICIT);
ret = cols_desc.push_back(col_desc);
ASSERT_EQ(ret, OB_SUCCESS);
}
datum_util.init(cols_desc, rowkey_cnt, false, allocator_);
ASSERT_EQ(ret, OB_SUCCESS);
}
TEST_F(ObDatumRowkeyVectorTest, int_vector_locate_key)
{
int ret = 0;
ObStorageDatumUtils datum_utils;
prepare_datum_util(1, datum_utils);
const ObStorageDatumCmpFunc &cmp_func = datum_utils.get_cmp_funcs().at(0);
int64_t row_count = 10;
int64_t row_arr[10] = {1,2,3,3,3,4,5,5,6,6};
ObColumnVector int_vec;
int_vec.type_ = ObColumnVectorType::SIGNED_INTEGER_TYPE;
int_vec.signed_ints_ = row_arr;
int_vec.row_cnt_ = row_count;
int_vec.has_null_ = false;
bool need_upper_bound = false;
int64_t begin = 0;
int64_t end = row_count;
int64_t key = 3;
ObStorageDatum datum_key;
datum_key.reuse();
datum_key.set_int(key);
ret = int_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 2);
ASSERT_EQ(end, 2);
need_upper_bound = true;
begin = 0;
end = row_count;
datum_key.reuse();
datum_key.set_int(key);
ret = int_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 2);
ASSERT_EQ(end, 5);
key = 6;
begin = 0;
end = row_count;
datum_key.reuse();
datum_key.set_int(key);
ret = int_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 8);
ASSERT_EQ(end, 10);
row_count = 1;
int64_t row_arr1[1] = {0};
int_vec.signed_ints_ = row_arr1;
int_vec.row_cnt_ = row_count;
key = 0;
begin = 0;
end = row_count;
datum_key.reuse();
datum_key.set_int(key);
ret = int_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 0);
ASSERT_EQ(end, 1);
}
TEST_F(ObDatumRowkeyVectorTest, datum_vector_locate_key)
{
int ret = 0;
ObStorageDatumUtils datum_utils;
prepare_datum_util(1, datum_utils);
const ObStorageDatumCmpFunc &cmp_func = datum_utils.get_cmp_funcs().at(0);
int64_t row_count = 10;
int64_t row_arr[10] = {1,2,3,3,3,4,5,5,6,6};
ObStorageDatum datum_arr[10];
for (int64_t i = 0; i < row_count; ++i) {
datum_arr[i].set_int(row_arr[i]);
}
ObColumnVector datum_vec;
datum_vec.type_ = ObColumnVectorType::DATUM_TYPE;
datum_vec.datums_ = datum_arr;
datum_vec.row_cnt_ = row_count;
ObStorageDatum datum_key;
bool need_upper_bound = false;
int64_t key = 3;
int64_t begin = 0;
int64_t end = row_count;
datum_key.reuse();
datum_key.set_int(key);
ret = datum_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 2);
ASSERT_EQ(end, 2);
need_upper_bound = true;
begin = 0;
end = row_count;
datum_key.reuse();
datum_key.set_int(key);
ret = datum_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 2);
ASSERT_EQ(end, 5);
key = 6;
begin = 0;
end = row_count;
datum_key.reuse();
datum_key.set_int(key);
ret = datum_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 8);
ASSERT_EQ(end, 10);
int64_t int_val;
ret = datum_vec.get_column_int(5, int_val);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(int_val, 4);
row_count = 1;
ObStorageDatum datum_arr1[1];
datum_arr1[0].set_int(0);
datum_vec.datums_ = datum_arr1;
datum_vec.row_cnt_ = row_count;
key = 0;
begin = 0;
end = row_count;
datum_key.reuse();
datum_key.set_int(key);
ret = datum_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 0);
ASSERT_EQ(end, 1);
}
TEST_F(ObDatumRowkeyVectorTest, int_vector_locate_key_with_null)
{
int ret = 0;
ObStorageDatumUtils datum_utils;
prepare_datum_util(1, datum_utils);
const ObStorageDatumCmpFunc &cmp_func = datum_utils.get_cmp_funcs().at(0);
int64_t row_count = 10;
ObColumnVector int_vec;
int_vec.type_ = ObColumnVectorType::SIGNED_INTEGER_TYPE;
int64_t row_arr[10] = {1,2,3,3,3,4,5,5,6,6};
bool null_arr[10] = {false,false,false,false,false,false,false,false,false,false};
int_vec.signed_ints_ = row_arr;
int_vec.row_cnt_ = row_count;
int_vec.nulls_ = null_arr;
int_vec.has_null_ = false;
ObStorageDatum datum_key;
bool need_upper_bound = false;
int64_t begin = 0;
int64_t end = row_count;
datum_key.reuse();
datum_key.set_null();
ret = int_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 0);
ASSERT_EQ(end, 0);
for (int64_t i = 0; i < 5; ++i) {
null_arr[i] = true;
}
int_vec.has_null_ = true;
begin = 0;
end = row_count;
ret = int_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 0);
ASSERT_EQ(end, 0);
need_upper_bound = true;
begin = 0;
end = row_count;
ret = int_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 0);
ASSERT_EQ(end, 5);
int64_t key = 3;
need_upper_bound = false;
begin = 0;
end = row_count;
datum_key.reuse();
datum_key.set_int(key);
ret = int_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 5);
ASSERT_EQ(end, 5);
key = 6;
begin = 0;
end = row_count;
datum_key.reuse();
datum_key.set_int(key);
ret = int_vec.locate_key(need_upper_bound, begin, end, datum_key, cmp_func, is_oracle_mode_);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin, 8);
ASSERT_EQ(end, 8);
}
TEST_F(ObDatumRowkeyVectorTest, rowkey_vector_locate_key)
{
int ret = 0;
ObStorageDatumUtils datum_utils;
prepare_datum_util(1, datum_utils);
// 0 3 6 9 12 ... 3*(n-1) ... 2997
int64_t row_count = 1000;
int64_t interval = 3;
ObColumnVector int_vec;
int_vec.type_ = ObColumnVectorType::SIGNED_INTEGER_TYPE;
void *buf = allocator_.alloc(sizeof(int64_t) *row_count);
ASSERT_TRUE(buf != nullptr);
int_vec.signed_ints_ = static_cast<int64_t *>(buf);
int_vec.row_cnt_ = row_count;
int_vec.has_null_ = false;
for (int64_t i = 0; i < row_count; ++i) {
int_vec.signed_ints_[i] = i * interval;
}
ObColumnVector col_vectors[1];
col_vectors[0] = int_vec;
ObRowkeyVector rowkey_vector;
rowkey_vector.columns_ = col_vectors;
rowkey_vector.col_cnt_ = 1;
rowkey_vector.row_cnt_ = row_count;
ObDatumRowkey rowkey;
ObStorageDatum rowkey_datums[1];
ret = rowkey.assign(rowkey_datums, 1);
ASSERT_EQ(ret, OB_SUCCESS);
int64_t key_arr[11] = {-1,0,1,2,3,4,1000,1500,2996,2997,2998};
for (int64_t i = 0; i < 11; ++i) {
int64_t key = key_arr[i];
rowkey_datums[0].set_int(key);
int64_t rowkey_idx = -1;
ret = rowkey_vector.locate_key(0, row_count, rowkey, datum_utils, rowkey_idx);
ASSERT_EQ(ret, OB_SUCCESS);
int64_t target_idx = key < 0 ?
0 : key / interval + (key % interval == 0 ? 0 : 1);
ASSERT_EQ(rowkey_idx, target_idx) << "i="<< i << " key=" << key;
}
}
TEST_F(ObDatumRowkeyVectorTest, rowkey_vector_locate_range)
{
int ret = 0;
ObStorageDatumUtils datum_utils;
prepare_datum_util(1, datum_utils);
// 0 3 6 9 12 ... 3*(n-1) ... 2997
int64_t row_count = 1000;
int64_t interval = 3;
ObColumnVector int_vec;
int_vec.type_ = ObColumnVectorType::SIGNED_INTEGER_TYPE;
void *buf = allocator_.alloc(sizeof(int64_t) *row_count);
ASSERT_TRUE(buf != nullptr);
int_vec.signed_ints_ = static_cast<int64_t *>(buf);
for (int64_t i = 0; i < row_count; ++i) {
int_vec.signed_ints_[i] = i * interval;
}
bool *nulls = static_cast<bool*>(allocator_.alloc(sizeof(bool) * row_count));
ASSERT_TRUE(nulls != nullptr);
for (int64_t i = 0; i < row_count; ++i) {
nulls[i] = false;
}
int_vec.row_cnt_ = row_count;
int_vec.nulls_ = nulls;
int_vec.has_null_ = false;
ObColumnVector col_vectors[1];
col_vectors[0] = int_vec;
ObRowkeyVector rowkey_vector;
rowkey_vector.columns_ = col_vectors;
rowkey_vector.col_cnt_ = 1;
rowkey_vector.row_cnt_ = row_count;
ObDatumRange range;
ObDatumRowkey start_key, end_key;
ObStorageDatum start_datums[1];
ObStorageDatum end_datums[1];
ret = start_key.assign(start_datums, 1);
ASSERT_EQ(ret, OB_SUCCESS);
ret = end_key.assign(end_datums, 1);
ASSERT_EQ(ret, OB_SUCCESS);
range.set_start_key(start_key);
range.set_end_key(end_key);
bool is_left_border = true;
bool is_right_border = true;
bool is_normal_cg = false;
int64_t begin_idx, end_idx;
start_datums[0].set_min();
end_datums[0].set_max();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 0);
ASSERT_EQ(end_idx, row_count - 1);
start_datums[0].set_min();
end_datums[0].set_int(9);
range.set_right_closed();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 0);
ASSERT_EQ(end_idx, 4);
end_datums[0].set_int(9);
range.set_right_open();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 0);
ASSERT_EQ(end_idx, 3);
start_datums[0].set_int(9);
end_datums[0].set_int(2500);
range.set_left_open();
range.set_right_open();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 4);
ASSERT_EQ(end_idx, 834);
start_datums[0].set_int(2610);
end_datums[0].set_max();
range.set_left_open();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 871);
ASSERT_EQ(end_idx, row_count - 1);
range.set_left_closed();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 870);
ASSERT_EQ(end_idx, row_count - 1);
for (int64_t i = 0; i < row_count; ++i) {
nulls[i] = true;
}
rowkey_vector.columns_[0].has_null_ = true;
start_datums[0].set_null();
end_datums[0].set_null();
range.set_left_closed();
range.set_right_closed();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 0);
ASSERT_EQ(end_idx, row_count - 1);
}
TEST_F(ObDatumRowkeyVectorTest, rowkey_vector_locate_range_2int_col)
{
int ret = 0;
ObStorageDatumUtils datum_utils;
prepare_datum_util(2, datum_utils);
int64_t row_count = 13;
int64_t int_arr0[13] = {1,1,1,2,2,2,2,2,3,3,4,5,7};
bool bool_arr0[13] = {false,false,false,false,false,false,false,false,false,false,false,false,false};
int64_t int_arr1[13] = {2,3,4,1,1,1,2,3,4,5,6,6,8};
bool bool_arr1[13] = {false,false,false,false,false,false,false,false,false,false,false,false,false};
ObColumnVector int_vec0;
int_vec0.type_ = ObColumnVectorType::SIGNED_INTEGER_TYPE;
int_vec0.row_cnt_ = row_count;
int_vec0.has_null_ = false;
int_vec0.signed_ints_ = int_arr0;
int_vec0.nulls_ = bool_arr0;
ObColumnVector int_vec1;
int_vec1.type_ = ObColumnVectorType::SIGNED_INTEGER_TYPE;
int_vec1.row_cnt_ = row_count;
int_vec1.has_null_ = false;
int_vec1.signed_ints_ = int_arr1;
int_vec1.nulls_ = bool_arr1;
ObColumnVector col_vectors[2];
col_vectors[0] = int_vec0;
col_vectors[1] = int_vec1;
ObRowkeyVector rowkey_vector;
rowkey_vector.columns_ = col_vectors;
rowkey_vector.col_cnt_ = 2;
rowkey_vector.row_cnt_ = row_count;
ObDatumRange range;
ObDatumRowkey start_key, end_key;
ObStorageDatum start_datums[2];
ObStorageDatum end_datums[2];
ret = start_key.assign(start_datums, 2);
ASSERT_EQ(ret, OB_SUCCESS);
ret = end_key.assign(end_datums, 2);
ASSERT_EQ(ret, OB_SUCCESS);
range.set_start_key(start_key);
range.set_end_key(end_key);
bool is_normal_cg = false;
bool is_left_border = true;
bool is_right_border = true;
int64_t begin_idx, end_idx;
// (1,3 : 2,1) -> 2,3
start_datums[0].set_int(1);
start_datums[1].set_int(3);
end_datums[0].set_int(2);
end_datums[1].set_int(1);
range.set_left_open();
range.set_right_open();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 2);
ASSERT_EQ(end_idx, 3);
// (1,3 : 2,1] -> 2,6
range.set_right_closed();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 2);
ASSERT_EQ(end_idx, 6);
// [1,3 : 2,1] -> 1,6
range.set_left_closed();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 1);
ASSERT_EQ(end_idx, 6);
// middle block
is_left_border = false;
is_right_border = false;
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 0);
ASSERT_EQ(end_idx, 12);
is_left_border = true;
is_right_border = true;
// [2,1 : 2,1] -> 3,6
start_datums[0].set_int(2);
start_datums[1].set_int(1);
end_datums[0].set_int(2);
end_datums[1].set_int(1);
range.set_left_closed();
range.set_right_closed();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 3);
ASSERT_EQ(end_idx, 6);
// [3,6 : 6,6] -> 10,12
start_datums[0].set_int(3);
start_datums[1].set_int(6);
end_datums[0].set_int(6);
end_datums[1].set_int(6);
range.set_left_closed();
range.set_right_closed();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 10);
ASSERT_EQ(end_idx, 12);
// [2,min : 3,max] -> 3,10
start_datums[0].set_int(2);
start_datums[1].set_min();
end_datums[0].set_int(3);
end_datums[1].set_max();
range.set_left_closed();
range.set_right_closed();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 3);
ASSERT_EQ(end_idx, 10);
// [0,1 : 7,8] -> 0,12
start_datums[0].set_int(0);
start_datums[1].set_int(1);
end_datums[0].set_int(7);
end_datums[1].set_int(8);
range.set_left_closed();
range.set_right_closed();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 0);
ASSERT_EQ(end_idx, 12);
// [8,1 : 3,max] -> OB_BEYOND_THE_RANGE
start_datums[0].set_int(8);
start_datums[1].set_int(1);
end_datums[0].set_int(3);
end_datums[1].set_max();
range.set_left_closed();
range.set_right_closed();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_BEYOND_THE_RANGE);
MEMSET(bool_arr0, true, sizeof(bool_arr0));
rowkey_vector.columns_[0].has_null_ = true;
for (int64_t i = 0; i < 13; ++i) {
int_arr1[i] = i;
}
start_datums[0].set_null();
start_datums[1].set_min();
end_datums[0].set_null();
end_datums[1].set_max();
range.set_left_open();
range.set_right_open();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 0);
ASSERT_EQ(end_idx, 12);
start_datums[1].set_int(2);
end_datums[1].set_int(9);
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 3);
ASSERT_EQ(end_idx, 9);
MEMSET(bool_arr0, false, sizeof(bool_arr0));
MEMSET(bool_arr1, true, sizeof(bool_arr1));
rowkey_vector.columns_[0].has_null_ = false;
rowkey_vector.columns_[1].has_null_ = true;
for (int64_t i = 0; i < 13; ++i) {
int_arr0[i] = i;
}
start_datums[0].set_int(1);
start_datums[1].set_null();
end_datums[0].set_int(7);
end_datums[1].set_null();
range.set_left_open();
range.set_right_open();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 2);
ASSERT_EQ(end_idx, 7);
range.set_left_closed();
range.set_right_closed();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 1);
ASSERT_EQ(end_idx, 8);
}
TEST_F(ObDatumRowkeyVectorTest, rowkey_vector_locate_range_with_datum)
{
int ret = 0;
ObStorageDatumUtils datum_utils;
prepare_datum_util(2, datum_utils);
int64_t row_count = 10;
void *buf = allocator_.alloc(sizeof(ObStorageDatum) * row_count);
ASSERT_TRUE(buf != nullptr);
ObStorageDatum *datums = new (buf) ObStorageDatum[row_count];
ObColumnVector datum_vec0;
datum_vec0.type_ = ObColumnVectorType::DATUM_TYPE;
datum_vec0.row_cnt_ = row_count;
datum_vec0.datums_ = datums;
for (int64_t i = 0; i < row_count; ++i) {
datums[i].set_int(i);
}
buf = allocator_.alloc(sizeof(ObStorageDatum) *row_count);
ASSERT_TRUE(buf != nullptr);
datums = new (buf) ObStorageDatum[row_count];
ObColumnVector datum_vec1;
datum_vec1.type_ = ObColumnVectorType::DATUM_TYPE;
datum_vec1.row_cnt_ = row_count;
datum_vec1.datums_ = datums;
for (int64_t i = 0; i < row_count; ++i) {
datums[i].set_int(i);
}
ObColumnVector col_vectors[2];
col_vectors[0] = datum_vec0;
col_vectors[1] = datum_vec1;
ObRowkeyVector rowkey_vector;
rowkey_vector.columns_ = col_vectors;
rowkey_vector.col_cnt_ = 2;
rowkey_vector.row_cnt_ = row_count;
rowkey_vector.is_datum_vectors_ = 1;
buf = allocator_.alloc(sizeof(ObDiscreteDatumRowkey) *row_count);
ASSERT_TRUE(buf != nullptr);
rowkey_vector.discrete_rowkey_array_ = new (buf) ObDiscreteDatumRowkey[row_count];
for (int64_t i = 0; i < row_count; ++i) {
rowkey_vector.discrete_rowkey_array_[i].row_idx_ = i;
rowkey_vector.discrete_rowkey_array_ [i].rowkey_vector_ = &rowkey_vector;
}
ObDatumRange range;
ObDatumRowkey start_key, end_key;
ObStorageDatum start_datums[2];
ObStorageDatum end_datums[2];
ret = start_key.assign(start_datums, 2);
ASSERT_EQ(ret, OB_SUCCESS);
ret = end_key.assign(end_datums, 2);
ASSERT_EQ(ret, OB_SUCCESS);
range.set_start_key(start_key);
range.set_end_key(end_key);
bool is_left_border = true;
bool is_right_border = true;
bool is_normal_cg = false;
int64_t begin_idx, end_idx;
start_datums[0].set_min();
start_datums[1].set_min();
end_datums[0].set_max();
end_datums[1].set_max();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 0);
ASSERT_EQ(end_idx, row_count - 1);
start_datums[0].set_int(1);
start_datums[1].set_int(1);
end_datums[0].set_int(2);
end_datums[1].set_int(2);
range.set_left_open();
range.set_right_open();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 2);
ASSERT_EQ(end_idx, 2);
range.set_left_closed();
range.set_right_closed();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 1);
ASSERT_EQ(end_idx, 3);
start_datums[1].set_min();
end_datums[1].set_max();
range.set_left_open();
range.set_right_open();
ret = rowkey_vector.locate_range(range, is_left_border, is_right_border, is_normal_cg, datum_utils, begin_idx, end_idx);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(begin_idx, 1);
ASSERT_EQ(end_idx, 3);
}
TEST_F(ObDatumRowkeyVectorTest, rowkey_vector_compare_rowkey)
{
int ret = 0;
ObStorageDatumUtils datum_utils;
prepare_datum_util(2, datum_utils);
int64_t row_count = 13;
int64_t int_arr0[13] = {1,1,1,2,2,2,2,2,3,3,4,5,7};
bool bool_arr0[13] = {false,false,false,false,false,false,false,false,false,false,false,false,false};
int64_t int_arr1[13] = {2,3,4,1,1,1,2,3,4,5,6,6,8};
bool bool_arr1[13] = {false,false,false,false,false,false,false,false,false,false,false,false,false};
ObColumnVector int_vec0;
int_vec0.type_ = ObColumnVectorType::SIGNED_INTEGER_TYPE;
int_vec0.row_cnt_ = row_count;
int_vec0.has_null_ = false;
int_vec0.signed_ints_ = int_arr0;
int_vec0.nulls_ = bool_arr0;
ObColumnVector int_vec1;
int_vec1.type_ = ObColumnVectorType::SIGNED_INTEGER_TYPE;
int_vec1.row_cnt_ = row_count;
int_vec1.has_null_ = false;
int_vec1.signed_ints_ = int_arr1;
int_vec1.nulls_ = bool_arr1;
ObColumnVector col_vectors[2];
col_vectors[0] = int_vec0;
col_vectors[1] = int_vec1;
ObRowkeyVector rowkey_vector;
rowkey_vector.columns_ = col_vectors;
rowkey_vector.col_cnt_ = 2;
rowkey_vector.row_cnt_ = row_count;
ObDatumRowkey curr_key;
ObStorageDatum curr_key_datums[2];
ret = curr_key.assign(curr_key_datums, 2);
ASSERT_EQ(ret, OB_SUCCESS);
int cmp_ret = 0;
bool compare_datum_cnt = true;
curr_key_datums[0].set_int(1);
curr_key_datums[1].set_int(1);
ret = rowkey_vector.compare_rowkey(curr_key, 0, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret > 0);
curr_key_datums[0].set_int(1);
curr_key_datums[1].set_int(2);
ret = rowkey_vector.compare_rowkey(curr_key, 0, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret == 0);
curr_key_datums[0].set_int(2);
curr_key_datums[1].set_int(1);
ret = rowkey_vector.compare_rowkey(curr_key, 2, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret < 0);
ret = curr_key.assign(curr_key_datums, 1);
ASSERT_EQ(ret, OB_SUCCESS);
curr_key_datums[0].set_int(2);
ret = rowkey_vector.compare_rowkey(curr_key, 3, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret > 0);
compare_datum_cnt = false;
ret = rowkey_vector.compare_rowkey(curr_key, 3, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret == 0);
curr_key_datums[0].set_min();
ret = rowkey_vector.compare_rowkey(curr_key, 3, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret > 0);
curr_key_datums[0].set_max();
ret = rowkey_vector.compare_rowkey(curr_key, 3, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret < 0);
ObDiscreteDatumRowkey discrete_rowkey;
discrete_rowkey.row_idx_ = 0;
discrete_rowkey.rowkey_vector_ = &rowkey_vector;
ret = rowkey_vector.compare_rowkey(discrete_rowkey, 0, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret == 0);
discrete_rowkey.row_idx_ = 1;
ret = rowkey_vector.compare_rowkey(discrete_rowkey, 0, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret < 0);
discrete_rowkey.row_idx_ = 0;
ret = rowkey_vector.compare_rowkey(discrete_rowkey, 1, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret > 0);
ObRowkeyVector rowkey_vector1;
rowkey_vector1.columns_ = col_vectors;
rowkey_vector1.col_cnt_ = 2;
rowkey_vector1.row_cnt_ = row_count;
discrete_rowkey.row_idx_ = 3;
discrete_rowkey.rowkey_vector_ = &rowkey_vector1;
ret = rowkey_vector.compare_rowkey(discrete_rowkey, 1, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret < 0);
ret = rowkey_vector.compare_rowkey(discrete_rowkey, 4, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret == 0);
ret = rowkey_vector.compare_rowkey(discrete_rowkey, 6, datum_utils, cmp_ret, compare_datum_cnt);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(cmp_ret > 0);
int64_t int_val;
ret = rowkey_vector.get_column_int(5, 0, int_val);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_EQ(int_val, 2);
}
TEST_F(ObDatumRowkeyVectorTest, rowkey_vector_deep_copy)
{
int ret = 0;
ObRowkeyVector rowkey_vector;
ObColumnVector col_vectors[2];
int64_t row_count = 6;
int64_t int_arr0[6] = {1,2,3,4,5,6};
bool bool_arr0[6] = {false,false,false,false,false,false};
int64_t int_arr1[6] = {2,3,4,1,1,1};
bool bool_arr1[6] = {false,true,false,true,false,true};
ObColumnVector int_vec0;
int_vec0.type_ = ObColumnVectorType::SIGNED_INTEGER_TYPE;
int_vec0.row_cnt_ = row_count;
int_vec0.has_null_ = false;
int_vec0.signed_ints_ = int_arr0;
int_vec0.nulls_ = bool_arr0;
int_vec0.is_filled_ = true;
ObColumnVector int_vec1;
int_vec1.type_ = ObColumnVectorType::SIGNED_INTEGER_TYPE;
int_vec1.row_cnt_ = row_count;
int_vec1.has_null_ = false;
int_vec1.signed_ints_ = int_arr1;
int_vec1.nulls_ = bool_arr1;
int_vec1.is_filled_ = true;
col_vectors[0] = int_vec0;
col_vectors[1] = int_vec1;
rowkey_vector.columns_ = col_vectors;
rowkey_vector.col_cnt_ = 2;
rowkey_vector.row_cnt_ = row_count;
int64_t buf_size = 0;
ObObjMeta obj_meta;
obj_meta.set_type_simple(ObIntType);
ObColDesc col_desc;
col_desc.col_type_ = obj_meta;
ObSEArray<ObColDesc, 2> col_descs;
ASSERT_EQ(col_descs.push_back(col_desc), OB_SUCCESS);
ASSERT_EQ(col_descs.push_back(col_desc), OB_SUCCESS);
ret = ObRowkeyVector::get_occupied_size(row_count, 2, &col_descs, buf_size);
ASSERT_EQ(ret, OB_SUCCESS);
int64_t expected_size = sizeof(ObRowkeyVector) // 1 rowkey vector
+ 2 * sizeof(ObColumnVector) // 2 column vector
+ 2 * row_count * (sizeof(int64_t) + sizeof(bool)) // 2 int column vector
+ sizeof(ObDatumRowkey) + sizeof(ObStorageDatum) * 2 // last rowkey
+ sizeof(ObDiscreteDatumRowkey) * row_count; // discrete datum rowkey array
ASSERT_EQ(buf_size, expected_size);
void *buf = allocator_.alloc(buf_size);
ASSERT_TRUE(buf != nullptr);
ObRowkeyVector new_vector;
int64_t pos = 0;
ret = new_vector.deep_copy((char*)buf, pos, buf_size, rowkey_vector);
STORAGE_LOG(INFO, "deep copy", K(ret), K(new_vector));
ASSERT_EQ(ret, OB_SUCCESS);
const ObDatumRowkey *last_rowkey = new_vector.get_last_rowkey();
ASSERT_TRUE(last_rowkey != nullptr);
ASSERT_TRUE(last_rowkey->is_valid());
ASSERT_EQ(6, last_rowkey->datums_[0].get_int());
ASSERT_TRUE(last_rowkey->datums_[1].is_null());
ASSERT_TRUE(nullptr != new_vector.discrete_rowkey_array_);
ASSERT_EQ(0, new_vector.discrete_rowkey_array_[0].row_idx_);
ASSERT_EQ(&new_vector, new_vector.discrete_rowkey_array_[0].rowkey_vector_);
ASSERT_EQ(5, new_vector.discrete_rowkey_array_[5].row_idx_);
ASSERT_EQ(&new_vector, new_vector.discrete_rowkey_array_[5].rowkey_vector_);
ObCommonDatumRowkey endkey;
ret = new_vector.get_rowkey(3, endkey);
ASSERT_EQ(ret, OB_SUCCESS);
ASSERT_TRUE(endkey.is_valid());
ASSERT_TRUE(endkey.is_discrete_rowkey());
ASSERT_EQ(3, endkey.get_discrete_rowkey()->row_idx_);
ASSERT_EQ(&new_vector, endkey.get_discrete_rowkey()->rowkey_vector_);
}
}
}
int main(int argc, char **argv)
{
system("rm -f test_datum_rowkey_vector.log*");
OB_LOGGER.set_file_name("test_datum_rowkey_vector.log", true, true);
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
::testing::InitGoogleTest(&argc,argv);
return RUN_ALL_TESTS();
}