swap tablet in compaction and destroy memtabls array & fix wrong allocator in multi_source_data

This commit is contained in:
obdev
2023-01-13 03:08:11 +00:00
committed by OB-robot
parent 263a44af96
commit 0083c6d39a
13 changed files with 193 additions and 47 deletions

View File

@ -446,7 +446,8 @@ int ObMediumCompactionScheduleFunc::prepare_medium_info(
{
int ret = OB_SUCCESS;
ObTableStoreIterator table_iter;
medium_info.cluster_id_ = GCONF.cluster_id;
medium_info.cluster_id_ = GCONF.cluster_id; // set cluster id
if (medium_info.is_major_compaction()) {
// get table schema
if (OB_UNLIKELY(result.schema_version_ <= 0)) {
@ -457,14 +458,35 @@ int ObMediumCompactionScheduleFunc::prepare_medium_info(
LOG_WARN("failed to get table schema", K(ret), KPC(this), K(medium_info));
}
}
} else if (OB_FAIL(medium_info.save_storage_schema(allocator_, tablet_.get_storage_schema()))) {
LOG_WARN("failed to save storage schema", K(ret), K(tablet_.get_storage_schema()));
} else {
ObStorageSchema tmp_storage_schema;
bool use_storage_schema_on_tablet = true;
if (medium_info.medium_snapshot_ > tablet_.get_snapshot_version()) {
ObSEArray<ObITable*, MAX_MEMSTORE_CNT> memtables;
if (OB_FAIL(tablet_.get_table_store().get_memtables(memtables, true/*need_active*/))) {
LOG_WARN("failed to get memtables", K(ret), KPC(this));
} else if (OB_FAIL(ObMediumCompactionScheduleFunc::get_latest_storage_schema_from_memtable(
allocator_, memtables, tmp_storage_schema))) {
if (OB_ENTRY_NOT_EXIST == ret) {
ret = OB_SUCCESS; // clear errno
} else {
LOG_WARN("failed to get storage schema from memtable", K(ret));
}
} else {
use_storage_schema_on_tablet = false;
}
}
if (FAILEDx(medium_info.save_storage_schema(
allocator_,
use_storage_schema_on_tablet ? tablet_.get_storage_schema() : tmp_storage_schema))) {
LOG_WARN("failed to save storage schema", K(ret), K(use_storage_schema_on_tablet), K(tmp_storage_schema));
}
}
if (FAILEDx(init_parallel_range(result, medium_info))) {
LOG_WARN("failed to init parallel range", K(ret), K(medium_info));
} else {
LOG_INFO("success to init parallel range", K(ret), K(medium_info));
LOG_INFO("success to prepare medium info", K(ret), K(medium_info));
}
return ret;
}
@ -855,5 +877,38 @@ int ObMediumCompactionScheduleFunc::check_need_merge_and_schedule(
return ret;
}
int ObMediumCompactionScheduleFunc::get_latest_storage_schema_from_memtable(
ObIAllocator &allocator,
const ObIArray<ObITable *> &memtables,
ObStorageSchema &storage_schema)
{
int ret = OB_SUCCESS;
ObITable *table = nullptr;
memtable::ObMemtable * memtable = nullptr;
bool found = false;
for (int64_t i = memtables.count() - 1; OB_SUCC(ret) && i >= 0; --i) {
if (OB_ISNULL(table = memtables.at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table in tables_handle is invalid", K(ret), KPC(table));
} else if (OB_ISNULL(memtable = dynamic_cast<memtable::ObMemtable *>(table))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table pointer does not point to a ObMemtable object", KPC(table));
} else if (OB_FAIL(memtable->get_multi_source_data_unit(&storage_schema, &allocator))) {
if (OB_ENTRY_NOT_EXIST == ret) {
ret = OB_SUCCESS; // clear OB_ENTRY_NOT_EXIST
} else {
LOG_WARN("failed to get storage schema from memtable", K(ret), KPC(table));
}
} else {
found = true;
break;
}
} // end for
if (OB_SUCC(ret) && !found) {
ret = OB_ENTRY_NOT_EXIST;
}
return ret;
}
} //namespace compaction
} // namespace oceanbase

View File

@ -36,6 +36,10 @@ public:
ObLS &ls,
ObTablet &tablet,
const int64_t major_frozen_scn = 0);
static int get_latest_storage_schema_from_memtable(
ObIAllocator &allocator,
const ObIArray<ObITable *> &memtables,
ObStorageSchema &storage_schema);
static int get_palf_role(const share::ObLSID &ls_id, ObRole &role);
int schedule_next_medium_for_leader(const int64_t major_snapshot);

View File

@ -24,6 +24,7 @@
#include "share/scheduler/ob_dag_warning_history_mgr.h"
#include "storage/compaction/ob_medium_compaction_mgr.h"
#include "storage/compaction/ob_medium_compaction_func.h"
#include "src/storage/meta_mem/ob_tenant_meta_mem_mgr.h"
namespace oceanbase
{
@ -723,7 +724,7 @@ int ObTabletMergeCtx::init_get_medium_compaction_info(
return ret;
}
int ObTabletMergeCtx::inner_init_for_minor(bool &skip_rest_operation)
int ObTabletMergeCtx::inner_init_for_mini(bool &skip_rest_operation)
{
int ret = OB_SUCCESS;
skip_rest_operation = false;
@ -735,7 +736,10 @@ int ObTabletMergeCtx::inner_init_for_minor(bool &skip_rest_operation)
get_merge_table_param.merge_version_ = param_.merge_version_;
ObTablet *tablet = tablet_handle_.get_obj();
if (OB_FAIL(ObPartitionMergePolicy::get_merge_tables[param_.merge_type_](
if (OB_UNLIKELY(!is_mini_merge(param_.merge_type_))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid merge type", K(ret), K_(param));
} else if (OB_FAIL(ObPartitionMergePolicy::get_merge_tables[param_.merge_type_](
get_merge_table_param,
*ls_handle_.get_ls(),
*tablet,
@ -743,7 +747,7 @@ int ObTabletMergeCtx::inner_init_for_minor(bool &skip_rest_operation)
// TODO(@DanLin) optimize this interface
if (OB_NO_NEED_MERGE != ret) {
LOG_WARN("failed to get merge tables", K(ret), KPC(this), K(get_merge_table_result));
} else if (is_mini_merge(param_.merge_type_)) { // OB_NO_NEED_MERGE && mini merge
} else { // OB_NO_NEED_MERGE
int tmp_ret = OB_SUCCESS;
// then release memtable
if (OB_TMP_FAIL(tablet->release_memtables(tablet->get_tablet_meta().clog_checkpoint_scn_))) {
@ -1054,6 +1058,8 @@ int ObTabletMergeCtx::get_storage_schema_to_merge(
bool get_storage_schema_flag = true;
if (is_mini_merge(merge_type) && get_schema_on_memtable) {
void *buf = nullptr;
ObSEArray<ObITable*, MAX_MEMSTORE_CNT> memtables;
if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObStorageSchema)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("failed to alloc storage schema", K(ret));
@ -1061,26 +1067,18 @@ int ObTabletMergeCtx::get_storage_schema_to_merge(
storage_schema = new(buf) ObStorageSchema();
}
ObITable *table = nullptr;
memtable::ObMemtable * memtable = nullptr;
for (int i = merge_tables_handle.get_count() - 1; OB_SUCC(ret) && i >= 0; --i) {
if (OB_UNLIKELY(nullptr == (table = merge_tables_handle.get_table(i)) || !table->is_frozen_memtable())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table in tables_handle is invalid", K(ret), KPC(table));
} else if (OB_ISNULL(memtable = dynamic_cast<memtable::ObMemtable *>(table))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table pointer does not point to a ObMemtable object", KPC(table));
} else if (OB_FAIL(memtable->get_multi_source_data_unit(storage_schema, &allocator_))) {
if (OB_ENTRY_NOT_EXIST != ret) {
LOG_WARN("failed to get storage schema from memtable", K(ret), KPC(table));
if (FAILEDx(merge_tables_handle.get_tables(memtables))) {
LOG_WARN("failed to get tables", K(ret), K(memtables));
} else if (OB_FAIL(ObMediumCompactionScheduleFunc::get_latest_storage_schema_from_memtable(
allocator_, memtables, *storage_schema))) {
if (OB_ENTRY_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else {
ret = OB_SUCCESS; // clear OB_ENTRY_NOT_EXIST
LOG_WARN("failed to get storage schema on memtable", K(ret), K_(param));
}
} else {
get_storage_schema_flag = false;
break;
}
} // end for
// free alloced storage schema
if ((OB_FAIL(ret) || get_storage_schema_flag) && nullptr != storage_schema) {
@ -1171,6 +1169,43 @@ int ObTabletMergeCtx::prepare_merge_progress()
return ret;
}
int ObTabletMergeCtx::try_swap_tablet_handle(const ObTablesHandleArray &tables_handle)
{
int ret = OB_SUCCESS;
// check need swap tablet when compaction
if (OB_UNLIKELY(is_mini_merge(param_.merge_type_))) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("mini merge not support swap tablet", K(ret), K_(param));
} else {
int64_t row_count = 0;
int64_t macro_count = 0;
const ObSSTable *table = nullptr;
for (int64_t i = 0; i < tables_handle_.get_count(); ++i) {
table = static_cast<const ObSSTable*>(tables_handle_.get_table(i));
row_count += table->get_meta().get_row_count();
macro_count += table->get_meta().get_basic_meta().get_data_macro_block_count();
}
if (tablet_handle_.get_obj()->get_table_store().get_memtables_count() > 0
&& (row_count >= LARGE_VOLUME_DATA_ROW_COUNT_THREASHOLD
|| macro_count >= LARGE_VOLUME_DATA_MACRO_COUNT_THREASHOLD)) {
ObTabletHandle alloc_handle;
const ObTabletMapKey key(param_.ls_id_, param_.tablet_id_);
if (OB_FAIL(MTL(ObTenantMetaMemMgr*)->get_tablet_with_allocator(
WashTabletPriority::WTP_HIGH, key, allocator_, alloc_handle, true/*force_alloc_new*/))) {
LOG_WARN("failed to get alloc tablet handle", K(ret), K(key));
} else {
tablet_handle_ = alloc_handle;
if (OB_FAIL(alloc_handle.get_obj()->clear_memtables_on_table_store())) {
LOG_WARN("failed to clear memtables on table_store", K(ret), K(param_));
} else {
LOG_INFO("success to swap tablet handle", K(ret), K(macro_count), K(row_count), K(tablet_handle_.get_obj()->get_table_store()));
}
}
}
}
return ret;
}
int ObTabletMergeCtx::generate_participant_table_info(char *buf, const int64_t buf_len) const
{
int ret = OB_SUCCESS;

View File

@ -165,12 +165,14 @@ struct ObTabletMergeCtx
int init_merge_progress(bool is_major);
int get_merge_range(int64_t parallel_idx, blocksstable::ObDatumRange &merge_range);
int inner_init_for_minor(bool &skip_rest_operation);
int inner_init_for_mini(bool &skip_rest_operation);
int inner_init_for_medium();
int init_get_medium_compaction_info(const int64_t medium_snapshot, const ObMediumCompactionInfo *&medium_info);
int get_schema_and_gene_from_result(const ObGetMergeTablesResult &get_merge_table_result);
int get_storage_schema_and_gene_from_result(const ObGetMergeTablesResult &get_merge_table_result);
int get_storage_schema_to_merge(const ObTablesHandleArray &merge_tables_handle, const bool get_schema_on_memtable = true);
int try_swap_tablet_handle(const ObTablesHandleArray &tables_handle);
public:
int get_medium_compaction_info_to_store();
@ -197,7 +199,8 @@ public:
}
typedef common::ObSEArray<ObGetMergeTablesResult, ObPartitionMergePolicy::OB_MINOR_PARALLEL_INFO_ARRAY_SIZE> MinorParallelResultArray;
static const int64_t LARGE_VOLUME_DATA_ROW_COUNT_THREASHOLD = 1000L * 1000L; // 100w
static const int64_t LARGE_VOLUME_DATA_MACRO_COUNT_THREASHOLD = 300L;
// 1. init in dag
ObTabletMergeDagParam &param_;
common::ObIAllocator &allocator_;

View File

@ -642,6 +642,8 @@ int ObTabletMergeExecutePrepareTask::process()
} else if (OB_ISNULL(ctx_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ctx is unexpected null", K(ret), K(ctx_));
} else if (OB_FAIL(ctx_->try_swap_tablet_handle(result_.handle_))) { // swap tablet before get schema ptr from tablet
LOG_WARN("failed to try swap tablet handle", K(ret));
} else if (OB_FAIL(ctx_->get_schema_and_gene_from_result(result_))) {
LOG_WARN("failed to get schema and generage from result", K(ret), K_(result));
} else if (OB_FAIL(ctx_->init_merge_info())) {
@ -817,6 +819,9 @@ int ObTabletMergePrepareTask::process()
}
if (OB_FAIL(ret) || skip_rest_operation) {
} else if (!is_mini_merge(ctx->param_.merge_type_)
&& OB_FAIL(ctx->try_swap_tablet_handle(ctx->tables_handle_))) {
LOG_WARN("failed to try swap tablet handle", K(ret));
} else if (OB_FAIL(ObBasicTabletMergeDag::generate_merge_task(
*merge_dag_, *ctx, this))) {
LOG_WARN("Failed to generate_merge_sstable_task", K(ret));
@ -933,7 +938,7 @@ int ObTabletMiniPrepareTask::inner_init_ctx(ObTabletMergeCtx &ctx, bool &skip_me
{
int ret = OB_SUCCESS;
skip_merge_task_flag = false;
if (OB_FAIL(ctx.inner_init_for_minor(skip_merge_task_flag))) {
if (OB_FAIL(ctx.inner_init_for_mini(skip_merge_task_flag))) {
LOG_WARN("failed to inner init for mini", K(ret));
}
return ret;

View File

@ -164,22 +164,22 @@ private:
};
template<class T>
int ObMultiSourceData::deep_copy_data_unit(const T *const src, T *&dst, ObIAllocator &allocator)
int ObMultiSourceData::deep_copy_data_unit(const T *const src, T *&dst, ObIAllocator &input_allocator)
{
int ret = OB_SUCCESS;
dst = nullptr;
void *buf = nullptr;
if (OB_ISNULL(buf = allocator.alloc(src->get_data_size()))) {
if (OB_ISNULL(buf = input_allocator.alloc(src->get_data_size()))) {
ret = common::OB_ALLOCATE_MEMORY_FAILED;
TRANS_LOG(WARN, "fail to alloc memory", K(ret));
} else if (FALSE_IT(dst = new (buf) T())) {
} else if (OB_FAIL(dst->deep_copy_unit(src, &allocator_))) {
} else if (OB_FAIL(dst->deep_copy_unit(src, &input_allocator))) {
TRANS_LOG(WARN, "fail to deep copy", K(ret), KP(dst), KP(src));
}
if (OB_FAIL(ret)) {
if (nullptr != buf) {
dst->~ObIMultiSourceDataUnit();
allocator.free(buf);
input_allocator.free(buf);
dst = nullptr;
}
}
@ -268,7 +268,7 @@ template<class T>
int ObMultiSourceData::get_multi_source_data_unit_list(
const T * const useless_unit,
ObIMultiSourceDataUnitList &dst_list,
ObIAllocator *allocator)
ObIAllocator *input_allocator)
{
int ret = OB_SUCCESS;
int64_t type = 0;
@ -277,9 +277,9 @@ int ObMultiSourceData::get_multi_source_data_unit_list(
if (OB_UNLIKELY(nullptr == useless_unit
|| FALSE_IT(type = (int64_t)useless_unit->type())
|| type < 0 || type >= type_count
|| nullptr == allocator)) {
|| nullptr == input_allocator)) {
ret = OB_INVALID_ARGUMENT;
TRANS_LOG(WARN, "invalid argument", K(ret), KPC(useless_unit), KP(allocator));
TRANS_LOG(WARN, "invalid argument", K(ret), KPC(useless_unit), KP(input_allocator));
} else if (!ObIMultiSourceDataUnit::is_unit_list(static_cast<MultiSourceDataUnitType>(type))) {
ret = OB_NOT_SUPPORTED;
TRANS_LOG(WARN, "not supported for cur data unit", K(ret), K(type));
@ -294,7 +294,7 @@ int ObMultiSourceData::get_multi_source_data_unit_list(
ret = OB_ERR_UNEXPECTED;
TRANS_LOG(WARN, "data unit is invalid", K(ret), KPC(item));
} else if (item->is_sync_finish()) {
if (OB_FAIL(deep_copy_data_unit(static_cast<const T *>(item), dst, *allocator))) {
if (OB_FAIL(deep_copy_data_unit(static_cast<const T *>(item), dst, *input_allocator))) {
TRANS_LOG(WARN, "failed to deep copy unit", K(ret), KPC(item));
} else if (!dst_list.add_last(dst)) {
ret = OB_ERR_UNEXPECTED;
@ -302,7 +302,7 @@ int ObMultiSourceData::get_multi_source_data_unit_list(
}
if (OB_FAIL(ret) && nullptr != dst) {
dst->~ObIMultiSourceDataUnit();
allocator->free(dst);
input_allocator->free(dst);
}
}
}

View File

@ -36,7 +36,8 @@ public:
int get_meta_obj_with_external_memory(
const Key &key,
common::ObIAllocator &allocator,
ObMetaObjGuard<T> &guard);
ObMetaObjGuard<T> &guard,
const bool force_alloc_new = false);
int try_get_in_memory_meta_obj(const Key &key, bool &success, ObMetaObjGuard<T> &guard);
int try_get_in_memory_meta_obj_and_addr(
const Key &key,
@ -517,7 +518,8 @@ template <typename Key, typename T>
int ObMetaPointerMap<Key, T>::get_meta_obj_with_external_memory(
const Key &key,
common::ObIAllocator &allocator,
ObMetaObjGuard<T> &guard)
ObMetaObjGuard<T> &guard,
const bool force_alloc_new)
{
int ret = common::OB_SUCCESS;
ObMetaPointerHandle<Key, T> ptr_hdl(*this);
@ -527,13 +529,21 @@ int ObMetaPointerMap<Key, T>::get_meta_obj_with_external_memory(
if (OB_UNLIKELY(!key.is_valid())) {
ret = common::OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "invalid argument", K(ret), K(key));
} else if (force_alloc_new) {
common::ObBucketHashRLockGuard lock_guard(ResourceMap::bucket_lock_, ResourceMap::hash_func_(key));
if (OB_FAIL(ResourceMap::get_without_lock(key, ptr_hdl))) {
if (common::OB_ENTRY_NOT_EXIST != ret) {
STORAGE_LOG(WARN, "fail to get pointer handle", K(ret));
}
}
} else if (OB_FAIL(try_get_in_memory_meta_obj(key, ptr_hdl, guard, is_in_memory))) {
if (OB_ENTRY_NOT_EXIST == ret) {
STORAGE_LOG(DEBUG, "meta obj does not exist", K(ret), K(key));
} else {
STORAGE_LOG(WARN, "fail to try get in memory meta obj", K(ret), K(key));
}
} else if (!is_in_memory) {
}
if (OB_SUCC(ret) && !is_in_memory) {
t_ptr = ptr_hdl.get_resource_ptr();
ObMetaDiskAddr disk_addr;
void *buf = allocator.alloc(sizeof(T));
@ -555,7 +565,7 @@ int ObMetaPointerMap<Key, T>::get_meta_obj_with_external_memory(
if (OB_ENTRY_NOT_EXIST != ret) {
STORAGE_LOG(WARN, "fail to get pointer handle", K(ret));
}
} else if (t_ptr->is_in_memory()) {
} else if (!force_alloc_new && t_ptr->is_in_memory()) {
if (OB_FAIL(t_ptr->get_in_memory_obj(guard))) {
STORAGE_LOG(ERROR, "fail to get meta object", K(ret), KP(t_ptr));
} else {

View File

@ -948,7 +948,8 @@ int ObTenantMetaMemMgr::get_tablet_with_allocator(
const WashTabletPriority &priority,
const ObTabletMapKey &key,
common::ObIAllocator &allocator,
ObTabletHandle &handle)
ObTabletHandle &handle,
const bool force_alloc_new)
{
int ret = OB_SUCCESS;
handle.reset();
@ -958,7 +959,7 @@ int ObTenantMetaMemMgr::get_tablet_with_allocator(
} else if (OB_UNLIKELY(!key.is_valid() || is_used_obj_pool(&allocator))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(key), KP(&allocator), KP(&allocator_));
} else if (OB_FAIL(tablet_map_.get_meta_obj_with_external_memory(key, allocator, handle))) {
} else if (OB_FAIL(tablet_map_.get_meta_obj_with_external_memory(key, allocator, handle, force_alloc_new))) {
if (OB_ENTRY_NOT_EXIST != ret) {
LOG_WARN("fail to get tablet", K(ret), K(key));
}

View File

@ -161,7 +161,8 @@ public:
const WashTabletPriority &priority,
const ObTabletMapKey &key,
common::ObIAllocator &allocator,
ObTabletHandle &handle);
ObTabletHandle &handle,
const bool force_alloc_new = false);
int get_tablet_addr(const ObTabletMapKey &key, ObMetaDiskAddr &addr);
int has_tablet(const ObTabletMapKey &key, bool &is_exist);
int del_tablet(const ObTabletMapKey &key);

View File

@ -3262,7 +3262,7 @@ int ObTablet::check_max_sync_schema_version() const
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("schema recorder is invalid", K(ret), K_(tablet_meta), KPC(data_memtable_mgr));
} else if (OB_FAIL(data_memtable_mgr->get_multi_source_data_unit(&storage_schema, &tmp_allocator))) {
LOG_ERROR("failed to storage schema from memtable, max_sync_schema_version is invalid", K(ret),
LOG_ERROR("failed to get storage schema from memtable, max_sync_schema_version is invalid", K(ret),
K(max_sync_schema_version), KPC(data_memtable_mgr));
} else if (OB_UNLIKELY(storage_schema.schema_version_ < max_sync_schema_version)) {
ret = OB_ERR_UNEXPECTED;
@ -3313,5 +3313,18 @@ int ObTablet::set_memtable_clog_checkpoint_scn(
return ret;
}
int ObTablet::clear_memtables_on_table_store() // be careful to call this func
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not inited", K(ret), K_(is_inited));
} else {
table_store_.clear_memtables();
}
return ret;
}
} // namespace storage
} // namespace oceanbase

View File

@ -388,6 +388,7 @@ public:
int set_redefined_schema_version_in_tablet_pointer(const int64_t schema_version);
int set_memtable_clog_checkpoint_scn(
const ObMigrationTabletParam *tablet_meta);
int clear_memtables_on_table_store(); // be careful to call this func, will destroy memtables array on table_store
TO_STRING_KV(KP(this), K_(wash_score), K_(ref_cnt), K_(tablet_meta), K_(table_store), K_(storage_schema),
K_(medium_info_list));
private:

View File

@ -431,6 +431,19 @@ int ObTabletTableStore::update_memtables()
return ret;
}
int ObTabletTableStore::clear_memtables()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table store is unexpected invalid", K(ret), KPC(this));
} else {
memtables_.destroy();
read_cache_.reset();
}
return ret;
}
int ObTabletTableStore::init_read_cache()
{
int ret = OB_SUCCESS;
@ -1200,7 +1213,7 @@ int64_t ObTabletTableStore::to_string(char *buf, const int64_t buf_len) const
J_OBJ_START();
J_NAME("ObTabletTableStore");
J_COLON();
J_KV(KP(this), KP_(tablet_ptr), K_(major_tables), K_(minor_tables), K_(is_ready_for_read));
J_KV(KP(this), KP_(tablet_ptr), K_(major_tables), K_(minor_tables), K_(memtables), K_(is_ready_for_read));
J_COMMA();
J_ARRAY_START();
for (int64_t i = 0; i < major_tables_.count_; ++i) {

View File

@ -96,8 +96,13 @@ public:
ObTableStoreIterator &iterator);
int get_memtables(common::ObIArray<storage::ObITable *> &memtables, const bool need_active = false) const;
int64_t get_memtables_count() const
{
return memtables_.count();
}
int prepare_memtables();
int update_memtables();
int clear_memtables();
int get_first_frozen_memtable(ObITable *&table);
int get_ddl_sstable_handles(ObTablesHandleArray &ddl_sstable_handles) const;