[FEAT MERGE] column store ddl

Co-authored-by: simonjoylet <simonjoylet@gmail.com>
Co-authored-by: Monk-Liu <1152761042@qq.com>
Co-authored-by: renju96 <fcbrenju@163.com>
This commit is contained in:
Charles0429
2024-02-07 23:09:01 +00:00
committed by ob-robot
parent fd359dc4a4
commit 1c20161f72
354 changed files with 29599 additions and 7741 deletions

View File

@ -298,7 +298,7 @@ ObITable::TableType ObMultiVersionSSTableTest::get_merged_table_type() const
} else if (META_MAJOR_MERGE == merge_type_) {
table_type = ObITable::TableType::META_MAJOR_SSTABLE;
} else if (DDL_KV_MERGE == merge_type_) {
table_type = ObITable::TableType::DDL_DUMP_SSTABLE;
table_type = ObITable::TableType::DDL_MERGE_CO_SSTABLE;
} else { // MINOR_MERGE
table_type = ObITable::TableType::MINOR_SSTABLE;
}

View File

@ -86,7 +86,7 @@ public:
TestTmpFileStress(ObTenantBase *tenant_ctx);
virtual ~TestTmpFileStress();
int init(const int fd, const bool is_write, const int64_t thread_cnt, ObTableSchema *table_schema,
const bool is_plain_data, const bool is_big_file);
const bool is_plain_data, const bool is_big_file, bool is_truncate = false);
virtual void run1();
private:
void prepare_data(char *buf, const int64_t macro_block_size);
@ -97,6 +97,7 @@ private:
void write_data(const int64_t macro_block_size);
void write_plain_data(char *&buf, const int64_t macro_block_size);
void read_data(const int64_t macro_block_size);
void read_and_truncate(const int64_t macro_block_size);
void read_plain_data(const char *buf, const int64_t macro_block_size);
private:
static const int64_t BUF_COUNT = 16;
@ -107,19 +108,21 @@ private:
bool is_big_file_;
ObTableSchema *table_schema_;
bool is_plain_;
bool is_truncate_;
ObTenantBase *tenant_ctx_;
};
TestTmpFileStress::TestTmpFileStress()
: thread_cnt_(0), size_(OB_SERVER_BLOCK_MGR.get_macro_block_size()), fd_(0),
is_write_(false), is_big_file_(false), table_schema_(NULL), is_plain_(false)
is_write_(false), is_big_file_(false), table_schema_(NULL), is_plain_(false),
is_truncate_(false)
{
}
TestTmpFileStress::TestTmpFileStress(ObTenantBase *tenant_ctx)
: thread_cnt_(0), size_(OB_SERVER_BLOCK_MGR.get_macro_block_size()), fd_(0),
is_write_(false), is_big_file_(false), table_schema_(NULL), is_plain_(false),
tenant_ctx_(tenant_ctx)
is_truncate_(false), tenant_ctx_(tenant_ctx)
{
}
@ -129,7 +132,7 @@ TestTmpFileStress::~TestTmpFileStress()
int TestTmpFileStress::init(const int fd, const bool is_write,
const int64_t thread_cnt, ObTableSchema *table_schema,
const bool is_plain, const bool is_big_file)
const bool is_plain, const bool is_big_file, const bool is_truncate)
{
int ret = OB_SUCCESS;
if (thread_cnt < 0) {
@ -145,6 +148,7 @@ int TestTmpFileStress::init(const int fd, const bool is_write,
if (!is_big_file_) {
size_ = 16L * 1024L;
}
is_truncate_ = is_truncate;
set_thread_count(static_cast<int32_t>(thread_cnt));
}
return ret;
@ -327,6 +331,51 @@ void TestTmpFileStress::read_data(const int64_t macro_block_size)
handle.reset();
}
void TestTmpFileStress::read_and_truncate(const int64_t macro_block_size)
{
int ret = OB_SUCCESS;
const int64_t timeout_ms = 100000;
ObTmpFileIOInfo io_info;
ObTmpFileIOHandle handle;
io_info.fd_ = fd_;
io_info.size_ = macro_block_size;
io_info.tenant_id_ = 1;
io_info.io_desc_.set_wait_event(2);
int64_t trunc_offset = 0;
char *buf = new char[macro_block_size];
char *zero_buf = new char[macro_block_size];
memset(zero_buf, 0, macro_block_size);
for (int64_t i = 0; i < BUF_COUNT; ++i) {
io_info.buf_ = buf;
ret = ObTmpFileManager::get_instance().read(io_info, handle);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(macro_block_size, handle.get_data_size());
check_data(handle.get_buffer(), handle.get_data_size());
ASSERT_EQ(OB_SUCCESS, ret);
// truncate data
// truncate trunc_offset + macro_block_size;
ret = ObTmpFileManager::get_instance().truncate(io_info.fd_, trunc_offset + macro_block_size);
ASSERT_EQ(OB_SUCCESS, ret);
ret = ObTmpFileManager::get_instance().pread(io_info, trunc_offset, handle);
ASSERT_EQ(OB_SUCCESS, ret);
check_plain_data(zero_buf, buf, macro_block_size);
trunc_offset += macro_block_size;
}
// check tuncated(0) won't reset the read_guard;
if (ret == OB_SUCCESS) {
io_info.buf_ = buf;
ret = ObTmpFileManager::get_instance().pread(io_info, 0, handle);
ASSERT_EQ(OB_SUCCESS, ret);
check_plain_data(zero_buf, buf, macro_block_size);
ret = ObTmpFileManager::get_instance().truncate(io_info.fd_, 0);
ASSERT_EQ(OB_SUCCESS, ret);
ret = ObTmpFileManager::get_instance().pread(io_info, 0, handle);
ASSERT_EQ(OB_SUCCESS, ret);
check_plain_data(zero_buf, buf, macro_block_size);
}
handle.reset();
}
void TestTmpFileStress::read_plain_data(const char *read_buf, const int64_t macro_block_size)
{
int ret = OB_SUCCESS;
@ -375,7 +424,8 @@ public:
TestMultiTmpFileStress(ObTenantBase *tenant_ctx);
virtual ~TestMultiTmpFileStress();
int init(const int64_t file_cnt, const int64_t dir_id, const int64_t thread_cnt,
ObTableSchema *table_schema, const bool is_plain_data, const bool is_big_file);
ObTableSchema *table_schema, const bool is_plain_data, const bool is_big_file,
const bool is_truncate = false);
virtual void run1();
private:
void run_plain_case();
@ -387,6 +437,7 @@ private:
ObTableSchema *table_schema_;
bool is_big_file_;
bool is_plain_data_;
bool is_truncate_;
ObTenantBase *tenant_ctx_;
};
@ -396,7 +447,8 @@ TestMultiTmpFileStress::TestMultiTmpFileStress()
thread_cnt_perf_file_(0),
table_schema_(NULL),
is_big_file_(false),
is_plain_data_(false)
is_plain_data_(false),
is_truncate_(false)
{
}
TestMultiTmpFileStress::TestMultiTmpFileStress(ObTenantBase *tenant_ctx)
@ -406,6 +458,7 @@ TestMultiTmpFileStress::TestMultiTmpFileStress(ObTenantBase *tenant_ctx)
table_schema_(NULL),
is_big_file_(false),
is_plain_data_(false),
is_truncate_(false),
tenant_ctx_(tenant_ctx)
{
}
@ -419,7 +472,8 @@ int TestMultiTmpFileStress::init(const int64_t file_cnt,
const int64_t thread_cnt,
ObTableSchema *table_schema,
const bool is_plain_data,
const bool is_big_file)
const bool is_big_file,
const bool is_truncate)
{
int ret = OB_SUCCESS;
if (file_cnt < 0 || thread_cnt < 0 || NULL == table_schema) {
@ -433,6 +487,7 @@ int TestMultiTmpFileStress::init(const int64_t file_cnt,
table_schema_ = table_schema;
is_big_file_ = is_big_file;
is_plain_data_ = is_plain_data;
is_truncate_ = is_truncate;
set_thread_count(static_cast<int32_t>(file_cnt));
}
return ret;
@ -463,9 +518,9 @@ void TestMultiTmpFileStress::run_normal_case()
ret = ObTmpFileManager::get_instance().open(fd, dir_id_);
ASSERT_EQ(OB_SUCCESS, ret);
STORAGE_LOG(INFO, "open file success", K(fd));
ret = test_write.init(fd, true, thread_cnt_perf_file_, table_schema_, is_plain_data_, is_big_file_);
ret = test_write.init(fd, true, thread_cnt_perf_file_, table_schema_, is_plain_data_, is_big_file_, is_truncate_);
ASSERT_EQ(OB_SUCCESS, ret);
ret = test_read.init(fd, false, thread_cnt_perf_file_, table_schema_, is_plain_data_, is_big_file_);
ret = test_read.init(fd, false, thread_cnt_perf_file_, table_schema_, is_plain_data_, is_big_file_, is_truncate_);
ASSERT_EQ(OB_SUCCESS, ret);
test_write.start();
test_write.wait();
@ -2016,6 +2071,190 @@ TEST_F(TestTmpFile, test_tmp_file_wash)
}
}
// test truncate, simple thread and multi thread
TEST_F(TestTmpFile, test_tmp_file_truncate)
{
int ret = OB_SUCCESS;
int64_t dir = -1;
int64_t fd = -1;
const int64_t timeout_ms = 5000;
ObTmpFileIOHandle handle;
ObTmpFileIOInfo io_info;
const int64_t macro_block_size = OB_SERVER_BLOCK_MGR.get_macro_block_size();
ret = ObTmpFileManager::get_instance().alloc_dir(dir);
ASSERT_EQ(OB_SUCCESS, ret);
ret = ObTmpFileManager::get_instance().open(fd, dir);
ASSERT_EQ(OB_SUCCESS, ret);
char *write_buf = new char [macro_block_size + 256];
for (int i = 0; i < macro_block_size + 256; ++i) {
write_buf[i] = static_cast<char>(i % 256);
}
char *read_buf = new char [macro_block_size + 256];
io_info.fd_ = fd;
io_info.tenant_id_ = 1;
io_info.io_desc_.set_wait_event(2);
io_info.buf_ = write_buf;
io_info.size_ = macro_block_size + 256;
io_info.io_desc_.set_group_id(THIS_WORKER.get_group_id());
int64_t write_time = ObTimeUtility::current_time();
ret = ObTmpFileManager::get_instance().write(io_info);
write_time = ObTimeUtility::current_time() - write_time;
ASSERT_EQ(OB_SUCCESS, ret);
io_info.buf_ = read_buf;
int64_t read_time = ObTimeUtility::current_time();
ret = ObTmpFileManager::get_instance().read(io_info, handle);
read_time = ObTimeUtility::current_time() - read_time;
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(macro_block_size + 256, handle.get_data_size());
int cmp = memcmp(handle.get_buffer(), write_buf, macro_block_size + 256);
ASSERT_EQ(0, cmp);
ret = ObTmpFileManager::get_instance().seek(fd, 0, ObTmpFile::SET_SEEK);
ASSERT_EQ(OB_SUCCESS, ret);
io_info.size_ = 200;
ret = ObTmpFileManager::get_instance().read(io_info, handle);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(200, handle.get_data_size());
cmp = memcmp(handle.get_buffer(), write_buf, 200);
ASSERT_EQ(0, cmp);
io_info.size_ = 200;
ret = ObTmpFileManager::get_instance().seek(fd, 0, ObTmpFile::SET_SEEK);
ASSERT_EQ(OB_SUCCESS, ret);
ret = ObTmpFileManager::get_instance().truncate(fd, 100);
ASSERT_EQ(OB_SUCCESS, ret);
ret = ObTmpFileManager::get_instance().read(io_info, handle);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(200, handle.get_data_size());
MEMSET(write_buf, 0, 100);
cmp = memcmp(handle.get_buffer(), write_buf, 200);
ASSERT_EQ(0, cmp);
io_info.size_ = 200;
ret = ObTmpFileManager::get_instance().truncate(fd, 300);
ASSERT_EQ(OB_SUCCESS, ret);
ret = ObTmpFileManager::get_instance().read(io_info, handle);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(200, handle.get_data_size());
MEMSET(write_buf + 100, 0, 200);
cmp = memcmp(handle.get_buffer(), write_buf + 200, 200);
ASSERT_EQ(0, cmp);
free(write_buf);
free(read_buf);
STORAGE_LOG(INFO, "test_tmp_file_truncate");
STORAGE_LOG(INFO, "io time", K(write_time), K(read_time));
ObTmpTenantFileStoreHandle store_handle;
OB_TMP_FILE_STORE.get_store(1, store_handle);
store_handle.get_tenant_store()->print_block_usage();
ObMallocAllocator::get_instance()->print_tenant_memory_usage(1);
ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1);
ObMallocAllocator::get_instance()->print_tenant_memory_usage(500);
ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500);
ObTmpFileManager::get_instance().remove(fd);
}
TEST_F(TestTmpFile, test_multi_thread_truncate)
{
int ret = OB_SUCCESS;
const int64_t thread_cnt = 4;
const int64_t file_cnt = 1;
const bool is_plain_data = false;
const bool is_big_file = true;
const bool is_truncate = true;
TestMultiTmpFileStress test(MTL_CTX());
int64_t dir = -1;
ret = ObTmpFileManager::get_instance().alloc_dir(dir);
ASSERT_EQ(OB_SUCCESS, ret);
ret = test.init(file_cnt, dir, thread_cnt, &table_schema_, is_plain_data, is_big_file, is_truncate);
ASSERT_EQ(OB_SUCCESS, ret);
int64_t io_time = ObTimeUtility::current_time();
test.start();
test.wait();
io_time = ObTimeUtility::current_time() - io_time;
STORAGE_LOG(INFO, "test_multi_thread_truncate");
STORAGE_LOG(INFO, "io time", K(io_time));
ObTmpTenantFileStoreHandle store_handle;
OB_TMP_FILE_STORE.get_store(1, store_handle);
store_handle.get_tenant_store()->print_block_usage();
ObMallocAllocator::get_instance()->print_tenant_memory_usage(1);
ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1);
ObMallocAllocator::get_instance()->print_tenant_memory_usage(500);
ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500);
}
TEST_F(TestTmpFile, test_truncate_free_block) {
int ret = OB_SUCCESS;
int count = 32;
int64_t dir = -1;
int64_t fd = -1;
ObTmpFileIOHandle handle;
ObTmpFileIOInfo io_info;
io_info.tenant_id_ = 1;
io_info.io_desc_.set_group_id(THIS_WORKER.get_group_id());
io_info.io_desc_.set_wait_event(2);
//int64_t write_size = OB_SERVER_BLOCK_MGR.get_macro_block_size();
int64_t write_size = 1024 * 1024;
char *write_buf = (char *)malloc(write_size);
for (int64_t i = 0; i < write_size; ++i) {
write_buf[i] = static_cast<char>(i % 256);
}
ret = ObTmpFileManager::get_instance().alloc_dir(dir);
ASSERT_EQ(OB_SUCCESS, ret);
ret = ObTmpFileManager::get_instance().open(fd, dir);
ASSERT_EQ(OB_SUCCESS, ret);
io_info.fd_ = fd;
io_info.buf_ = write_buf;
io_info.size_ = write_size;
for (int64_t i = 0; i < count; i++) {
ret = ObTmpFileManager::get_instance().write(io_info);
ASSERT_EQ(OB_SUCCESS, ret);
}
STORAGE_LOG(INFO, "test_truncate_free_block");
ObTmpTenantFileStoreHandle store_handle;
OB_TMP_FILE_STORE.get_store(1, store_handle);
ASSERT_EQ(count, store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size());
for (int64_t i = 0; i < count; i++) {
ret = ObTmpFileManager::get_instance().truncate(fd, (i + 1) * write_size);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(count - i - 1 , store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size());
}
ret = ObTmpFileManager::get_instance().truncate(fd, 0);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(0 , store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size());
int64_t read_size = write_size;
char *read_buf = (char *)malloc(read_size);
memset(write_buf, 0, write_size);
io_info.buf_ = read_buf;
ret = ObTmpFileManager::get_instance().read(io_info, handle);
int cmp = memcmp(read_buf, write_buf, read_size);
ASSERT_EQ(0, cmp);
free(write_buf);
free(read_buf);
store_handle.get_tenant_store()->print_block_usage();
ObMallocAllocator::get_instance()->print_tenant_memory_usage(1);
ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1);
ObMallocAllocator::get_instance()->print_tenant_memory_usage(500);
ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500);
ObTmpFileManager::get_instance().remove(fd);
}
} // end namespace unittest
} // end namespace oceanbase

View File

@ -1 +1,2 @@
#storage_unittest(test_ddl_kv)
storage_unittest(test_chunk_compact_store)

View File

@ -0,0 +1,802 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX STORAGE
#define ASSERT_OK(x) ASSERT_EQ(OB_SUCCESS, (x))
#include <gtest/gtest.h>
#define private public
#include "storage/meta_mem/ob_tenant_meta_mem_mgr.h"
#include "storage/blocksstable/ob_row_generate.h"
#include "storage/blocksstable/ob_data_file_prepare.h"
#include "src/sql/engine/basic/ob_chunk_datum_store.h"
#include "unittest/storage/blocksstable/ob_data_file_prepare.h"
#include "src/sql/engine/basic/chunk_store/ob_compact_store.h"
#include "src/sql/engine/basic/ob_temp_block_store.h"
#undef private
namespace oceanbase
{
using namespace common;
using namespace lib;
using namespace share;
using namespace sql;
//const int64_t COLUMN_CNT = 64;
const int64_t COLUMN_CNT = 64;
const int64_t BATCH_SIZE = 10000;
const int64_t ROUND[6] = {2,8,32,128,512, 1024};
int64_t RESULT_ADD[6] = {0,0,0,0,0,0};
int64_t RESULT_BUILD[6] = {0,0,0,0,0,0};
static ObSimpleMemLimitGetter getter;
typedef ObChunkDatumStore::StoredRow StoredRow;
//typedef ObChunkDatumStore::Block Block;
typedef ObTempBlockStore::Block Block;
class ObStoredRowGenerate {
public:
int get_stored_row(StoredRow **&sr);
int get_stored_row_irregular(StoredRow **&sr);
common::ObArenaAllocator allocator_;
};
int ObStoredRowGenerate::get_stored_row(StoredRow **&sr)
{
int ret = OB_SUCCESS;
int64_t data_size = ((sizeof(ObDatum) + 8) * COLUMN_CNT + 8) * BATCH_SIZE;
int32_t row_size = (sizeof(ObDatum) + 8) * COLUMN_CNT + 8;
allocator_.reuse();
void *buf = allocator_.alloc(data_size);
if (OB_ISNULL(buf)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to alloc buff", K(ret));
} else {
MEMSET(buf, 0, data_size);
for (int64_t i = 0; i < BATCH_SIZE; i++)
{
StoredRow * cur_sr = (StoredRow*) ((char*)buf + i * row_size);
if (i == BATCH_SIZE) {
cur_sr->row_size_ = 8 + 1042*COLUMN_CNT;
} else {
cur_sr->row_size_ = row_size;
}
cur_sr->cnt_ = COLUMN_CNT;
for (int64_t j = 0; j < COLUMN_CNT; j++) {
if (i != BATCH_SIZE) {
int64_t datum_offset = sizeof(ObDatum) * j;
int64_t data_offset = COLUMN_CNT * sizeof(ObDatum) + 8 * j + sizeof(StoredRow);
ObDatum *datum_ptr = (ObDatum *)(cur_sr->payload_ + datum_offset);
int64_t *data_ptr = (int64_t *)((char*)cur_sr + data_offset);
datum_ptr->len_ = 8;
//MEMCPY((void*)&datum_ptr->ptr_, &data_offset, 8);
MEMCPY((void*)&datum_ptr->ptr_, &data_ptr, 8);
*data_ptr = 1;
} else {
// wont't go here
// generate var data
int64_t datum_offset = sizeof(ObDatum) * j;
int64_t data_offset = COLUMN_CNT * sizeof(ObDatum) + 8 * j + sizeof(StoredRow);
ObDatum *datum_ptr = (ObDatum *)(cur_sr->payload_ + datum_offset);
int64_t *data_ptr = (int64_t *)((char*)cur_sr + data_offset);
datum_ptr->len_ = 1030;
//MEMCPY((void*)&datum_ptr->ptr_, &data_offset, 8);
MEMCPY((void*)&datum_ptr->ptr_, &data_ptr, 8);
*data_ptr = 1;
}
}
}
sr = (StoredRow**)buf;
}
return ret;
}
int ObStoredRowGenerate::get_stored_row_irregular(StoredRow **&sr)
{
int ret = OB_SUCCESS;
int64_t data_size = ((sizeof(ObDatum) + 8) * COLUMN_CNT + 8) * BATCH_SIZE;
int32_t row_size = (sizeof(ObDatum) + 8) * COLUMN_CNT + 8;
allocator_.reuse();
void *buf = allocator_.alloc(data_size);
if (OB_ISNULL(buf)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to alloc buff", K(ret));
} else {
MEMSET(buf, 0, data_size);
for (int64_t i = 0; i < BATCH_SIZE; i++)
{
StoredRow * cur_sr = (StoredRow*) ((char*)buf + i * row_size);
if (i == BATCH_SIZE) {
cur_sr->row_size_ = 8 + 1042*COLUMN_CNT;
} else {
cur_sr->row_size_ = row_size;
}
cur_sr->cnt_ = COLUMN_CNT;
for (int64_t j = 0; j < COLUMN_CNT; j++) {
if (i != BATCH_SIZE) {
int64_t datum_offset = sizeof(ObDatum) * j;
int64_t data_offset = COLUMN_CNT * sizeof(ObDatum) + 8 * j + sizeof(StoredRow);
ObDatum *datum_ptr = (ObDatum *)(cur_sr->payload_ + datum_offset);
int64_t *data_ptr = (int64_t *)((char*)cur_sr + data_offset);
datum_ptr->len_ = 8;
//MEMCPY((void*)&datum_ptr->ptr_, &data_offset, 8);
MEMCPY((void*)&datum_ptr->ptr_, &data_ptr, 8);
*data_ptr = i * 1024 + j;
} else {
// wont't go here
// generate var data
int64_t datum_offset = sizeof(ObDatum) * j;
int64_t data_offset = COLUMN_CNT * sizeof(ObDatum) + 8 * j + sizeof(StoredRow);
ObDatum *datum_ptr = (ObDatum *)(cur_sr->payload_ + datum_offset);
int64_t *data_ptr = (int64_t *)((char*)cur_sr + data_offset);
datum_ptr->len_ = 1030;
//MEMCPY((void*)&datum_ptr->ptr_, &data_offset, 8);
MEMCPY((void*)&datum_ptr->ptr_, &data_ptr, 8);
*data_ptr = 1;
}
}
}
sr = (StoredRow**)buf;
}
return ret;
}
class TestCompactChunk : public TestDataFilePrepare
{
public:
TestCompactChunk() :
TestDataFilePrepare(&getter, "TestTmpFile", 2 * 1024 * 1024, 2048) {};
void SetUp();
void TearDown();
protected:
ObStoredRowGenerate row_generate_;
ObArenaAllocator allocator_;
};
void TestCompactChunk::SetUp()
{
int ret = OB_SUCCESS;
const int64_t bucket_num = 1024;
const int64_t max_cache_size = 1024 * 1024 * 1024;
const int64_t block_size = common::OB_MALLOC_BIG_BLOCK_SIZE;
TestDataFilePrepare::SetUp();
ret = getter.add_tenant(1,
8L * 1024L * 1024L, 2L * 1024L * 1024L * 1024L);
ASSERT_EQ(OB_SUCCESS, ret);
ret = ObKVGlobalCache::get_instance().init(&getter, bucket_num, max_cache_size, block_size);
if (OB_INIT_TWICE == ret) {
ret = OB_SUCCESS;
} else {
ASSERT_EQ(OB_SUCCESS, ret);
}
// set observer memory limit
CHUNK_MGR.set_limit(8L * 1024L * 1024L * 1024L);
ret = ObTmpFileManager::get_instance().init();
ASSERT_EQ(OB_SUCCESS, ret);
static ObTenantBase tenant_ctx(1);
ObTenantEnv::set_tenant(&tenant_ctx);
ObTenantIOManager *io_service = nullptr;
EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service));
}
void TestCompactChunk::TearDown()
{
ObTmpFileManager::get_instance().destroy();
ObKVGlobalCache::get_instance().destroy();
ObTmpFileStore::get_instance().destroy();
allocator_.reuse();
row_generate_.allocator_.reuse();
TestDataFilePrepare::TearDown();
}
TEST_F(TestCompactChunk, test_read_writer)
{
int ret = OB_SUCCESS;
ObCompactStore cs_chunk;
cs_chunk.init(1, 1,
ObCtxIds::DEFAULT_CTX_ID, "SORT_CACHE_CTX", true, 0, true, share::SORT_DEFAULT_LEVEL);
StoredRow **sr;
ret = row_generate_.get_stored_row(sr);
ASSERT_EQ(ret, OB_SUCCESS);
char *buf = reinterpret_cast<char*>(sr);
int64_t pos = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
StoredRow *tmp_sr = (StoredRow *)(buf + pos);
ret = cs_chunk.add_row(*tmp_sr);
ASSERT_EQ(ret, OB_SUCCESS);
pos += tmp_sr->row_size_;
}
ret = cs_chunk.finish_add_row();
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
int64_t result = 0;
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_next_row(cur_sr);
if (ret == OB_ITER_END) {
ret = OB_SUCCESS;
}
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
result += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(result == 64);
}
}
TEST_F(TestCompactChunk, test_batch)
{
int ret = OB_SUCCESS;
ObCompactStore cs_chunk;
cs_chunk.init(1, 1,
ObCtxIds::DEFAULT_CTX_ID, "SORT_CACHE_CTX", true, 0, true, share::SORT_DEFAULT_LEVEL);
StoredRow **sr;
ret = row_generate_.get_stored_row(sr);
ASSERT_EQ(ret, OB_SUCCESS);
char *buf = reinterpret_cast<char*>(sr);
int64_t pos = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
StoredRow *tmp_sr = (StoredRow *)(buf + pos);
ret = cs_chunk.add_row(*tmp_sr);
ASSERT_EQ(ret, OB_SUCCESS);
pos += tmp_sr->row_size_;
}
ret = cs_chunk.finish_add_row();
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
int64_t result = 0;
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_next_row(cur_sr);
if (ret == OB_ITER_END) {
ret = OB_SUCCESS;
}
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
result += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(result == 64);
}
}
TEST_F(TestCompactChunk, test_read_writer_compact)
{
int ret = OB_SUCCESS;
ObCompactStore cs_chunk;
cs_chunk.init(1, 1,
ObCtxIds::DEFAULT_CTX_ID, "SORT_CACHE_CTX", true, 0, true, share::SORT_COMPACT_LEVEL);
RowMeta row_meta(allocator_);
row_meta.col_cnt_ = COLUMN_CNT;
row_meta.fixed_cnt_ = COLUMN_CNT;
row_meta.var_data_off_ = 8 * row_meta.fixed_cnt_;
row_meta.column_length_.prepare_allocate(COLUMN_CNT);
row_meta.column_offset_.prepare_allocate(COLUMN_CNT);
for (int64_t i = 0; i < COLUMN_CNT; i++) {
if (i != COLUMN_CNT) {
row_meta.column_length_[i] = 8;
row_meta.column_offset_[i] = 8 * i;
} else {
row_meta.column_length_[i] = 0;
row_meta.column_offset_[i] = 0;
}
}
cs_chunk.set_meta(&row_meta);
StoredRow **sr;
ret = row_generate_.get_stored_row(sr);
ASSERT_EQ(ret, OB_SUCCESS);
char *buf = reinterpret_cast<char*>(sr);
int64_t pos = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
StoredRow *tmp_sr = (StoredRow *)(buf + pos);
ret = cs_chunk.add_row(*tmp_sr);
ASSERT_EQ(ret, OB_SUCCESS);
pos += tmp_sr->row_size_;
}
ret = cs_chunk.finish_add_row();
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
int64_t result = 0;
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_next_row(cur_sr);
if (ret == OB_ITER_END) {
ret = OB_SUCCESS;
}
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
result += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(result == 64);
}
}
TEST_F(TestCompactChunk, test_read_writer_compact_vardata)
{
int ret = OB_SUCCESS;
ObCompactStore cs_chunk;
cs_chunk.init(1, 1,
ObCtxIds::DEFAULT_CTX_ID, "SORT_CACHE_CTX", true, 0, true, share::SORT_COMPACT_LEVEL);
RowMeta row_meta(allocator_);
row_meta.col_cnt_ = COLUMN_CNT;
row_meta.fixed_cnt_ = 0;
row_meta.var_data_off_ = 0;
row_meta.column_length_.prepare_allocate(COLUMN_CNT);
row_meta.column_offset_.prepare_allocate(COLUMN_CNT);
for (int64_t i = 0; i < COLUMN_CNT; i++) {
if (i != COLUMN_CNT) {
row_meta.column_length_[i] = 0;
row_meta.column_offset_[i] = 0;
} else {
row_meta.column_length_[i] = 0;
row_meta.column_offset_[i] = 0;
}
}
cs_chunk.set_meta(&row_meta);
StoredRow **sr;
ret = row_generate_.get_stored_row(sr);
ASSERT_EQ(ret, OB_SUCCESS);
char *buf = reinterpret_cast<char*>(sr);
int64_t pos = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
StoredRow *tmp_sr = (StoredRow *)(buf + pos);
ret = cs_chunk.add_row(*tmp_sr);
ASSERT_EQ(ret, OB_SUCCESS);
pos += tmp_sr->row_size_;
}
ret = cs_chunk.finish_add_row();
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
int64_t result = 0;
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_next_row(cur_sr);
if (ret == OB_ITER_END) {
ret = OB_SUCCESS;
}
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
result += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(result == 64);
}
}
TEST_F(TestCompactChunk, test_read_writer_compression)
{
int ret = OB_SUCCESS;
ObCompactStore cs_chunk;
cs_chunk.init(1, 1,
ObCtxIds::DEFAULT_CTX_ID, "SORT_CACHE_CTX", true, 0, true, share::SORT_COMPRESSION_LEVEL, ZSTD_COMPRESSOR);
StoredRow **sr;
ret = row_generate_.get_stored_row(sr);
ASSERT_EQ(ret, OB_SUCCESS);
char *buf = reinterpret_cast<char*>(sr);
int64_t pos = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
StoredRow *tmp_sr = (StoredRow *)(buf + pos);
ret = cs_chunk.add_row(*tmp_sr);
ASSERT_EQ(ret, OB_SUCCESS);
pos += tmp_sr->row_size_;
}
ret = cs_chunk.finish_add_row();
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
int64_t result = 0;
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_next_row(cur_sr);
if (ret == OB_ITER_END) {
ret = OB_SUCCESS;
}
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
result += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(result == 64);
}
}
TEST_F(TestCompactChunk, test_irregular)
{
int ret = OB_SUCCESS;
ObCompactStore cs_chunk;
cs_chunk.init(1, 1,
ObCtxIds::DEFAULT_CTX_ID, "SORT_CACHE_CTX", true, 0, false/*disable trunc*/, share::SORT_DEFAULT_LEVEL);
StoredRow **sr;
ret = row_generate_.get_stored_row_irregular(sr);
ASSERT_EQ(ret, OB_SUCCESS);
char *buf = reinterpret_cast<char*>(sr);
int64_t pos = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
StoredRow *tmp_sr = (StoredRow *)(buf + pos);
ret = cs_chunk.add_row(*tmp_sr);
ASSERT_EQ(ret, OB_SUCCESS);
pos += tmp_sr->row_size_;
}
ret = cs_chunk.finish_add_row();
ASSERT_EQ(ret, OB_SUCCESS);
for (int j = 0; OB_SUCC(ret) && j < 1; j++ ) {
int64_t total_res = 0;
//cs_chunk.rescan();
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
int64_t result = 0;
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_next_row(cur_sr);
if (ret == OB_ITER_END) {
ret = OB_SUCCESS;
}
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
result += *(int64_t *)(cur_cell.ptr_);
total_res += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(result == ((1024 * i * COLUMN_CNT) + ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
OB_ASSERT(total_res == ((1024 * (BATCH_SIZE-1) * BATCH_SIZE * COLUMN_CNT / 2) + BATCH_SIZE * ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
}
TEST_F(TestCompactChunk, test_rescan_irregular_1)
{
int ret = OB_SUCCESS;
ObCompactStore cs_chunk;
cs_chunk.init(1, 1,
ObCtxIds::DEFAULT_CTX_ID, "SORT_CACHE_CTX", true, 0, false/*disable trunc*/, share::SORT_DEFAULT_LEVEL);
StoredRow **sr;
ret = row_generate_.get_stored_row_irregular(sr);
ASSERT_EQ(ret, OB_SUCCESS);
char *buf = reinterpret_cast<char*>(sr);
int64_t pos = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
StoredRow *tmp_sr = (StoredRow *)(buf + pos);
ret = cs_chunk.add_row(*tmp_sr);
ASSERT_EQ(ret, OB_SUCCESS);
pos += tmp_sr->row_size_;
}
ret = cs_chunk.finish_add_row();
ASSERT_EQ(ret, OB_SUCCESS);
for (int j = 0; OB_SUCC(ret) && j < 1; j++ ) {
int64_t total_res = 0;
cs_chunk.rescan();
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
int64_t result = 0;
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_next_row(cur_sr);
if (ret == OB_ITER_END) {
ret = OB_SUCCESS;
}
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
result += *(int64_t *)(cur_cell.ptr_);
total_res += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(result == ((1024 * i * COLUMN_CNT) + ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
OB_ASSERT(total_res == ((1024 * (BATCH_SIZE-1) * BATCH_SIZE * COLUMN_CNT / 2) + BATCH_SIZE * ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
}
TEST_F(TestCompactChunk, test_rescan)
{
int ret = OB_SUCCESS;
ObCompactStore cs_chunk;
cs_chunk.init(1, 1,
ObCtxIds::DEFAULT_CTX_ID, "SORT_CACHE_CTX", true, 0, false/*disable trunc*/, share::SORT_DEFAULT_LEVEL);
StoredRow **sr;
ret = row_generate_.get_stored_row(sr);
ASSERT_EQ(ret, OB_SUCCESS);
char *buf = reinterpret_cast<char*>(sr);
int64_t pos = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
StoredRow *tmp_sr = (StoredRow *)(buf + pos);
ret = cs_chunk.add_row(*tmp_sr);
ASSERT_EQ(ret, OB_SUCCESS);
pos += tmp_sr->row_size_;
}
ret = cs_chunk.finish_add_row();
ASSERT_EQ(ret, OB_SUCCESS);
for (int j = 0; OB_SUCC(ret) && j < 10; j++ ) {
cs_chunk.rescan();
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
int64_t result = 0;
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_next_row(cur_sr);
if (ret == OB_ITER_END) {
ret = OB_SUCCESS;
}
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
result += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(result == 64);
}
}
}
TEST_F(TestCompactChunk, test_rescan_irregular)
{
int ret = OB_SUCCESS;
ObCompactStore cs_chunk;
cs_chunk.init(1, 1,
ObCtxIds::DEFAULT_CTX_ID, "SORT_CACHE_CTX", true, 0, false/*disable trunc*/, share::SORT_DEFAULT_LEVEL);
StoredRow **sr;
ret = row_generate_.get_stored_row_irregular(sr);
ASSERT_EQ(ret, OB_SUCCESS);
char *buf = reinterpret_cast<char*>(sr);
int64_t pos = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
StoredRow *tmp_sr = (StoredRow *)(buf + pos);
ret = cs_chunk.add_row(*tmp_sr);
ASSERT_EQ(ret, OB_SUCCESS);
pos += tmp_sr->row_size_;
}
ret = cs_chunk.finish_add_row();
ASSERT_EQ(ret, OB_SUCCESS);
for (int j = 0; OB_SUCC(ret) && j < 2; j++ ) {
int64_t total_res = 0;
cs_chunk.rescan();
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
int64_t result = 0;
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_next_row(cur_sr);
if (ret == OB_ITER_END) {
ret = OB_SUCCESS;
}
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
result += *(int64_t *)(cur_cell.ptr_);
total_res += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(result == ((1024 * i * COLUMN_CNT) + ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
OB_ASSERT(total_res == ((1024 * (BATCH_SIZE-1) * BATCH_SIZE * COLUMN_CNT / 2) + BATCH_SIZE * ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
}
TEST_F(TestCompactChunk, test_rescan_get_last_row)
{
int ret = OB_SUCCESS;
ObCompactStore cs_chunk;
cs_chunk.init(1, 1,
ObCtxIds::DEFAULT_CTX_ID, "SORT_CACHE_CTX", true, 0, false/*disable trunc*/, share::SORT_DEFAULT_LEVEL);
StoredRow **sr;
ret = row_generate_.get_stored_row_irregular(sr);
ASSERT_EQ(ret, OB_SUCCESS);
char *buf = reinterpret_cast<char*>(sr);
int64_t pos = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
StoredRow *tmp_sr = (StoredRow *)(buf + pos);
ret = cs_chunk.add_row(*tmp_sr);
ASSERT_EQ(ret, OB_SUCCESS);
pos += tmp_sr->row_size_;
// get last row
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_last_stored_row(cur_sr);
ASSERT_EQ(ret, OB_SUCCESS);
int64_t res = 0;
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
res += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(res == ((1024 * i * COLUMN_CNT) + ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
ret = cs_chunk.finish_add_row();
ASSERT_EQ(ret, OB_SUCCESS);
for (int j = 0; OB_SUCC(ret) && j < 2; j++ ) {
int64_t total_res = 0;
cs_chunk.rescan();
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
int64_t result = 0;
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_next_row(cur_sr);
if (ret == OB_ITER_END) {
ret = OB_SUCCESS;
}
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
result += *(int64_t *)(cur_cell.ptr_);
total_res += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(result == ((1024 * i * COLUMN_CNT) + ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
OB_ASSERT(total_res == ((1024 * (BATCH_SIZE-1) * BATCH_SIZE * COLUMN_CNT / 2) + BATCH_SIZE * ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
}
TEST_F(TestCompactChunk, test_rescan_get_last_row_compact)
{
int ret = OB_SUCCESS;
ObCompactStore cs_chunk;
cs_chunk.init(1, 1,
ObCtxIds::DEFAULT_CTX_ID, "SORT_CACHE_CTX", true, 0, false/*disable trunc*/, share::SORT_COMPACT_LEVEL);
RowMeta row_meta(allocator_);
row_meta.col_cnt_ = COLUMN_CNT;
row_meta.fixed_cnt_ = 0;
row_meta.var_data_off_ = 0;
row_meta.column_length_.prepare_allocate(COLUMN_CNT);
row_meta.column_offset_.prepare_allocate(COLUMN_CNT);
for (int64_t i = 0; i < COLUMN_CNT; i++) {
if (i != COLUMN_CNT) {
row_meta.column_length_[i] = 0;
row_meta.column_offset_[i] = 0;
} else {
row_meta.column_length_[i] = 0;
row_meta.column_offset_[i] = 0;
}
}
cs_chunk.set_meta(&row_meta);
StoredRow **sr;
ret = row_generate_.get_stored_row_irregular(sr);
ASSERT_EQ(ret, OB_SUCCESS);
char *buf = reinterpret_cast<char*>(sr);
int64_t pos = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
StoredRow *tmp_sr = (StoredRow *)(buf + pos);
ret = cs_chunk.add_row(*tmp_sr);
ASSERT_EQ(ret, OB_SUCCESS);
pos += tmp_sr->row_size_;
// get last row
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_last_stored_row(cur_sr);
ASSERT_EQ(ret, OB_SUCCESS);
int64_t res = 0;
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
res += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(res == ((1024 * i * COLUMN_CNT) + ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
ret = cs_chunk.finish_add_row();
ASSERT_EQ(ret, OB_SUCCESS);
for (int j = 0; OB_SUCC(ret) && j < 2; j++ ) {
int64_t total_res = 0;
cs_chunk.rescan();
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
int64_t result = 0;
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_next_row(cur_sr);
if (ret == OB_ITER_END) {
ret = OB_SUCCESS;
}
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
result += *(int64_t *)(cur_cell.ptr_);
total_res += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(result == ((1024 * i * COLUMN_CNT) + ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
OB_ASSERT(total_res == ((1024 * (BATCH_SIZE-1) * BATCH_SIZE * COLUMN_CNT / 2) + BATCH_SIZE * ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
}
TEST_F(TestCompactChunk, test_rescan_add_storagedatum)
{
int ret = OB_SUCCESS;
ObCompactStore cs_chunk;
cs_chunk.init(1, 1,
ObCtxIds::DEFAULT_CTX_ID, "SORT_CACHE_CTX", true, 0, false/*disable trunc*/, share::SORT_COMPACT_LEVEL);
RowMeta row_meta(allocator_);
row_meta.col_cnt_ = COLUMN_CNT;
row_meta.fixed_cnt_ = 0;
row_meta.var_data_off_ = 0;
row_meta.column_length_.prepare_allocate(COLUMN_CNT);
row_meta.column_offset_.prepare_allocate(COLUMN_CNT);
for (int64_t i = 0; i < COLUMN_CNT; i++) {
if (i != COLUMN_CNT) {
row_meta.column_length_[i] = 0;
row_meta.column_offset_[i] = 0;
} else {
row_meta.column_length_[i] = 0;
row_meta.column_offset_[i] = 0;
}
}
cs_chunk.set_meta(&row_meta);
StoredRow **sr;
ret = row_generate_.get_stored_row_irregular(sr);
ASSERT_EQ(ret, OB_SUCCESS);
char *buf = reinterpret_cast<char*>(sr);
int64_t pos = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
StoredRow *tmp_sr = (StoredRow *)(buf + pos);
ObStorageDatum ssr[COLUMN_CNT];
for (int64_t k = 0; OB_SUCC(ret) && k < COLUMN_CNT; k++) {
ssr[k].shallow_copy_from_datum(tmp_sr->cells()[k]);
}
ret = cs_chunk.add_row(ssr, COLUMN_CNT, 0);
ASSERT_EQ(ret, OB_SUCCESS);
pos += tmp_sr->row_size_;
// get last row
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_last_stored_row(cur_sr);
ASSERT_EQ(ret, OB_SUCCESS);
int64_t res = 0;
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
res += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(res == ((1024 * i * COLUMN_CNT) + ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
ret = cs_chunk.finish_add_row();
ASSERT_EQ(ret, OB_SUCCESS);
for (int j = 0; OB_SUCC(ret) && j < 2; j++ ) {
int64_t total_res = 0;
cs_chunk.rescan();
for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_SIZE; i++) {
int64_t result = 0;
const StoredRow *cur_sr = nullptr;
ret = cs_chunk.get_next_row(cur_sr);
if (ret == OB_ITER_END) {
ret = OB_SUCCESS;
}
ASSERT_EQ(ret, OB_SUCCESS);
for (int64_t k = 0; k < cur_sr->cnt_; k++) {
ObDatum cur_cell = cur_sr->cells()[k];
result += *(int64_t *)(cur_cell.ptr_);
total_res += *(int64_t *)(cur_cell.ptr_);
}
OB_ASSERT(result == ((1024 * i * COLUMN_CNT) + ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
OB_ASSERT(total_res == ((1024 * (BATCH_SIZE-1) * BATCH_SIZE * COLUMN_CNT / 2) + BATCH_SIZE * ((COLUMN_CNT - 1) * COLUMN_CNT / 2)));
}
}
}
int main(int argc, char **argv)
{
::testing::InitGoogleTest(&argc, argv);
system("rm -rf test_ddl_compact_store.log*");
OB_LOGGER.set_log_level("INFO");
OB_LOGGER.set_file_name("test_ddl_compact_store.log", true);
//testing::FLAGS_gtest_filter = "TestCompactChunk.test_dump_one_block";
return RUN_ALL_TESTS();
}

View File

@ -22,6 +22,7 @@
#include "storage/meta_mem/ob_tenant_meta_mem_mgr.h"
#include "storage/blocksstable/ob_row_generate.h"
#include "storage/blocksstable/ob_data_file_prepare.h"
#include "storage/ddl/ob_direct_insert_sstable_ctx_new.h"
#undef private
namespace oceanbase
@ -154,7 +155,8 @@ TEST_F(TestBlockMetaTree, random_keybtree)
for (int64_t i = 0; i < 10; ++i) {
ASSERT_OK(meta_tree.block_tree_.init());
meta_tree.is_inited_ = true;
ASSERT_OK(meta_tree.data_desc_.assign(data_desc_.get_desc()));
ASSERT_OK(meta_tree.data_desc_.assign(data_desc_));
meta_tree.datum_utils_ = &meta_tree.data_desc_.get_desc().get_datum_utils();
for (int64_t j = 0; j < 10000; ++j) {
void *buf = allocator_.alloc(sizeof(ObDatumRow));
ASSERT_TRUE(nullptr != buf);
@ -195,13 +197,13 @@ TEST_F(TestBlockMetaTree, random_keybtree)
//for (int64_t i = 1; i <= 5; ++i) {
//void *buf = arena.alloc(sizeof(ObSSTable));
//ObSSTable *tmp_sstable = new (buf) ObSSTable();
//tmp_sstable->key_.table_type_ = ObITable::DDL_DUMP_SSTABLE;
//tmp_sstable->key_.table_type_ = ObITable::DDL_MERGE_CO_SSTABLE;
//tmp_sstable->key_.scn_range_.start_scn_ = SCN::plus(SCN::min_scn(), 10 * i);
//tmp_sstable->key_.scn_range_.end_scn_ = SCN::plus(SCN::min_scn(), 10 * (i + 1));
//ASSERT_OK(ddl_sstables.push_back(tmp_sstable));
//}
//ObSSTable compact_sstable;
//compact_sstable.key_.table_type_ = ObITable::DDL_DUMP_SSTABLE;
//compact_sstable.key_.table_type_ = ObITable::DDL_MERGE_CO_SSTABLE;
//compact_sstable.key_.scn_range_.start_scn_ = SCN::plus(SCN::min_scn(), 10);
//compact_sstable.key_.scn_range_.end_scn_ = SCN::plus(SCN::min_scn(), 60);

View File

@ -390,7 +390,7 @@ public:
virtual ~TestTablet();
virtual void SetUp();
virtual void TearDown();
void pull_ddl_memtables(ObIArray<ObITable *> &ddl_kvs)
void pull_ddl_memtables(ObIArray<ObDDLKV *> &ddl_kvs)
{
for (int64_t i = 0; i < ddl_kv_count_; ++i) {
ASSERT_EQ(OB_SUCCESS, ddl_kvs.push_back(ddl_kvs_[i]));
@ -400,7 +400,7 @@ public:
void reproducing_bug();
private:
ObArenaAllocator allocator_;
ObITable **ddl_kvs_;
ObDDLKV **ddl_kvs_;
volatile int64_t ddl_kv_count_;
};
@ -544,7 +544,7 @@ public:
int init(ObArenaAllocator &allocator, TestTablet &tablet)
{
int ret = OB_SUCCESS;
ObArray<ObITable *> ddl_kvs;
ObArray<ObDDLKV *> ddl_kvs;
tablet.pull_ddl_memtables(ddl_kvs);
ret = ddl_kvs_.init(allocator, ddl_kvs);
const int64_t count = ddl_kvs_.count();
@ -554,10 +554,9 @@ public:
}
void reproducing_bug(ObArenaAllocator &allocator)
{
ObArray<ObITable *> ddl_kvs;
ObArray<ObDDLKV *> ddl_kvs;
for (int64_t i = 0; i < 3; ++i) {
ObITable *ddl_kv = new ObDDLKV();
ddl_kv->key_.table_type_ = ObITable::TableType::DDL_MEM_SSTABLE;
ObDDLKV *ddl_kv = new ObDDLKV();
ddl_kvs.push_back(ddl_kv);
}
ddl_kvs_.init(allocator, ddl_kvs);
@ -573,14 +572,11 @@ void TestTablet::reproducing_bug()
{
int ret = OB_SUCCESS;
ObTabletComplexAddr<TestTableStore> table_store_addr;
ddl_kvs_ = static_cast<ObITable**>(allocator_.alloc(sizeof(ObITable*) * ObTablet::DDL_KV_ARRAY_SIZE));
ddl_kvs_ = static_cast<ObDDLKV**>(allocator_.alloc(sizeof(ObDDLKV*) * ObTablet::DDL_KV_ARRAY_SIZE));
ASSERT_TRUE(nullptr != ddl_kvs_);
ddl_kvs_[0] = new ObDDLKV();
ddl_kvs_[0]->key_.table_type_ = ObITable::TableType::DDL_MEM_SSTABLE;
ddl_kvs_[1] = new ObDDLKV();
ddl_kvs_[1]->key_.table_type_ = ObITable::TableType::DDL_MEM_SSTABLE;
ddl_kvs_[2] = new ObDDLKV();
ddl_kvs_[2]->key_.table_type_ = ObITable::TableType::DDL_MEM_SSTABLE;
std::cout<< "reproducing_bug 1:" << ddl_kv_count_ << std::endl;
ddl_kv_count_ = 3;
std::cout<< "reproducing_bug 2:" << ddl_kv_count_ << std::endl;

View File

@ -396,12 +396,12 @@ int TestCompactionPolicy::mock_tablet(
ObTablet *tablet = nullptr;
ObTableHandleV2 table_handle;
bool need_empty_major_table = false;
ObLSHandle ls_handle;
ObLSService *ls_svr = nullptr;
ObArenaAllocator arena_allocator;
ObCreateTabletSchema create_tablet_schema;
bool need_empty_major_table = false;
if (OB_ISNULL(t3m)) {
ret = OB_ERR_UNEXPECTED;
@ -416,10 +416,11 @@ int TestCompactionPolicy::mock_tablet(
LOG_WARN("failed to acquire tablet", K(ret), K(key));
} else if (FALSE_IT(tablet = tablet_handle.get_obj())) {
} else if (OB_FAIL(create_tablet_schema.init(arena_allocator, table_schema, compat_mode,
false/*skip_column_info*/, ObCreateTabletSchema::STORAGE_SCHEMA_VERSION_V3))) {
false/*skip_column_info*/, ObCreateTabletSchema::STORAGE_SCHEMA_VERSION_V3,
0/*tenant_data_version, default val*/, need_empty_major_table/*need_create_empty_major*/))) {
LOG_WARN("failed to init storage schema", KR(ret), K(table_schema));
} else if (OB_FAIL(tablet->init_for_first_time_creation(allocator, ls_id, tablet_id, tablet_id,
SCN::min_scn(), snapshot_version, create_tablet_schema, need_empty_major_table, ls_handle.get_ls()->get_freezer()))) {
SCN::min_scn(), snapshot_version, create_tablet_schema, ls_handle.get_ls()->get_freezer()))) {
LOG_WARN("failed to init tablet", K(ret), K(ls_id), K(tablet_id), K(snapshot_version),
K(table_schema), K(compat_mode));
} else {

View File

@ -332,7 +332,8 @@ int TestLobCommon::build_lob_tablet_arg(
} else if (OB_FAIL(tablet_schema_index_array.push_back(2))) {
STORAGE_LOG(WARN, "failed to push index into array", K(ret));
} else if (OB_FAIL(tablet_info.init(tablet_id_array, data_tablet_id, tablet_schema_index_array,
lib::get_compat_mode(), false/*is_create_bind_hidden_tablets*/))) {
lib::get_compat_mode(), false/*is_create_bind_hidden_tablets*/,
true /*need_create_empty_major_sstable*/))) {
STORAGE_LOG(WARN, "failed to init tablet info", K(ret), K(tablet_id_array),
K(data_tablet_id), K(tablet_schema_index_array));
} else if (OB_FAIL(arg.init_create_tablet(ls_id, share::SCN::min_scn(), false/*need_check_tablet_cnt*/))) {

View File

@ -118,7 +118,8 @@ inline int TestTabletHelper::create_tablet(
prepare_sstable_param(tablet_id, table_schema, param);
void *buff = nullptr;
if (OB_FAIL(create_tablet_schema.init(schema_allocator, table_schema, compat_mode,
false/*skip_column_info*/, ObCreateTabletSchema::STORAGE_SCHEMA_VERSION_V3))) {
false/*skip_column_info*/, ObCreateTabletSchema::STORAGE_SCHEMA_VERSION_V3,
0/*tenant_data_version, default val*/, true/*need_create_empty_major*/))) {
STORAGE_LOG(WARN, "failed to init storage schema", KR(ret), K(table_schema));
} else if (OB_FAIL(ObSSTableMergeRes::fill_column_checksum_for_empty_major(param.column_cnt_, param.column_checksums_))) {
STORAGE_LOG(WARN, "fill column checksum failed", K(ret), K(param));
@ -133,7 +134,7 @@ inline int TestTabletHelper::create_tablet(
} else if (OB_FAIL(tablet_handle.get_obj()->init_for_first_time_creation(
*tablet_handle.get_allocator(),
ls_id, tablet_id, tablet_id, share::SCN::base_scn(),
snapshot_version, create_tablet_schema, true, freezer))){
snapshot_version, create_tablet_schema, freezer))){
STORAGE_LOG(WARN, "failed to init tablet", K(ret), K(ls_id), K(tablet_id));
} else if (ObTabletStatus::Status::MAX != tablet_status) {
ObTabletCreateDeleteMdsUserData data;