Merge branch 'column_store'

Co-authored-by: wangt1xiuyi <13547954130@163.com>
Co-authored-by: yangqise7en <877793735@qq.com>
Co-authored-by: Zach41 <zach_41@163.com>
This commit is contained in:
chaser-ch
2023-10-31 15:39:20 +00:00
committed by ob-robot
parent 4057fbc4ae
commit 566e920620
1375 changed files with 239147 additions and 56014 deletions

View File

@ -15,6 +15,7 @@
#include <gtest/gtest.h>
#include <gmock/gmock.h>
#define private public
#include "share/io/ob_io_define.h"
#include "share/io/ob_io_manager.h"
#include "share/io/ob_io_calibration.h"
#include "share/io/io_schedule/ob_io_mclock.h"
@ -24,6 +25,10 @@
#include "lib/thread/thread_pool.h"
#include "lib/file/file_directory_utils.h"
#include "common/ob_clock_generator.h"
#include "storage/blocksstable/ob_micro_block_cache.h"
#include "storage/blocksstable/ob_tmp_file_cache.h"
#include "storage/blocksstable/ob_tmp_file_store.h"
#include "storage/meta_mem/ob_storage_meta_cache.h"
#define ASSERT_SUCC(ret) ASSERT_EQ((ret), ::oceanbase::common::OB_SUCCESS)
#define ASSERT_FAIL(ret) ASSERT_NE((ret), ::oceanbase::common::OB_SUCCESS)
@ -31,6 +36,7 @@
using namespace oceanbase::lib;
using namespace oceanbase::common;
using namespace oceanbase::share;
using namespace oceanbase::blocksstable;
#define TEST_ROOT_DIR "io_test"
#define TEST_DATA_DIR TEST_ROOT_DIR "/data_dir"
@ -109,9 +115,11 @@ static ObIOInfo get_random_io_info()
io_info.flag_.set_mode(static_cast<ObIOMode>(ObRandom::rand(0, (int)ObIOMode::MAX_MODE - 1)));
io_info.flag_.set_group_id(0); // 0 means default
io_info.flag_.set_wait_event(ObRandom::rand(1, 9999));
io_info.timeout_us_ = DEFAULT_IO_WAIT_TIME_US;
io_info.offset_ = ObRandom::rand(1, 1000L * 1000L * 1000L);
io_info.size_ = ObRandom::rand(1, 1000L * 10L);
io_info.flag_.set_read();
io_info.user_data_buf_ = static_cast<char *>(ob_malloc(io_info.size_, ObNewModIds::TEST));
return io_info;
}
@ -139,10 +147,11 @@ public:
: number_(nullptr), allocator_(nullptr), help_buf_(nullptr)
{}
virtual ~TestIOCallback();
virtual const char *get_data() override { return help_buf_; }
virtual const char *get_data() override { return (char *)help_buf_; }
virtual int64_t size() const override { return sizeof(TestIOCallback); }
virtual int inner_deep_copy(char *buf, const int64_t buf_len, ObIOCallback *&callback) const override;
virtual int alloc_data_buf(const char *io_data_buffer, const int64_t data_size) override;
virtual int inner_process(const char *data_buffer, const int64_t size) override;
virtual ObIAllocator *get_allocator() override { return allocator_; }
TO_STRING_KV(KP(number_), KP(allocator_), KP(help_buf_));
public:
@ -208,8 +217,11 @@ TEST_F(TestIOStruct, IOInfo)
info.flag_.set_mode(ObIOMode::READ);
info.flag_.set_group_id(0);
info.flag_.set_wait_event(1);
info.timeout_us_ = DEFAULT_IO_WAIT_TIME_US;
info.offset_ = 80;
info.size_ = 1;
char user_buf[1] = { 0 };
info.user_data_buf_ = user_buf;
ASSERT_TRUE(info.is_valid());
// write io info require write buf
@ -226,29 +238,35 @@ TEST_F(TestIOStruct, IOInfo)
TEST_F(TestIOStruct, IOHandle)
{
ObIOResult result;
ObIORequest req;
req.inc_ref(); // prevent from free
result.inc_ref();
req.set_result(result);
ASSERT_EQ(1, req.ref_cnt_);
ASSERT_EQ(2, result.result_ref_cnt_);
ASSERT_EQ(0, result.out_ref_cnt_);
// default invalid
ObIOHandle handle;
ASSERT_FALSE(handle.is_valid());
ASSERT_TRUE(handle.is_empty());
// normal usage
handle.set_request(req);
handle.set_result(result);
ASSERT_TRUE(handle.is_valid());
ASSERT_FALSE(handle.is_empty());
ASSERT_EQ(2, req.ref_cnt_);
ASSERT_EQ(1, req.out_ref_cnt_);
ASSERT_EQ(3, result.result_ref_cnt_);
ASSERT_EQ(1, result.out_ref_cnt_);
// copy assign
ObIOHandle handle2 = handle;
ASSERT_EQ(3, req.ref_cnt_);
ASSERT_EQ(2, req.out_ref_cnt_);
ASSERT_EQ(4, result.result_ref_cnt_);
ASSERT_EQ(2, result.out_ref_cnt_);
// reset
handle2.reset();
ASSERT_EQ(2, req.ref_cnt_);
ASSERT_EQ(1, req.out_ref_cnt_);
ASSERT_EQ(3, result.result_ref_cnt_);
ASSERT_EQ(1, result.out_ref_cnt_);
ASSERT_FALSE(handle2.is_valid());
ASSERT_TRUE(handle2.is_empty());
@ -293,51 +311,93 @@ TEST_F(TestIOStruct, IORequest)
fd.second_id_ = 1;
// default invalid
ObIOResult result;
ASSERT_FALSE(result.is_inited_);
ASSERT_FALSE(result.is_valid());
ObIORequest req;
ASSERT_FALSE(req.is_inited_);
ASSERT_FALSE(req.is_valid());
// prepare read request
req.destroy();
result.destroy();
req.tenant_io_mgr_.hold(&tenant_io_mgr);
result.tenant_io_mgr_.hold(&tenant_io_mgr);
result.inc_ref();
req.inc_ref();
ObIOInfo read_info;
read_info.tenant_id_ = OB_SERVER_TENANT_ID;
read_info.fd_ = fd;
read_info.flag_.set_mode(ObIOMode::READ);
read_info.flag_.set_group_id(0);
read_info.flag_.set_wait_event(1);
read_info.timeout_us_ = DEFAULT_IO_WAIT_TIME_US;
read_info.offset_ = 89;
read_info.size_ = 1;
ASSERT_EQ(req.get_group_id(), 0);
char user_buf[1] = { 0 };
read_info.user_data_buf_ = user_buf;
ASSERT_TRUE(read_info.is_valid());
ASSERT_SUCC(req.tenant_io_mgr_.get_ptr()->io_usage_.init(0));
ASSERT_SUCC(req.init(read_info));
ASSERT_FAIL(result.init(read_info)); //not init cond yet
ASSERT_FALSE(result.is_inited_);
ASSERT_SUCC(result.basic_init());
ASSERT_SUCC(result.init(read_info));
ASSERT_SUCC(req.init(read_info ,&result));
ASSERT_TRUE(req.is_inited_);
ASSERT_EQ(req.io_buf_, nullptr); // read buf allocation is delayed
ASSERT_TRUE(result.is_inited_);
ASSERT_TRUE(req.is_valid());
ASSERT_TRUE(result.is_valid());
ASSERT_EQ(req.raw_buf_, nullptr); // read buf allocation is delayed
ASSERT_SUCC(req.prepare());
ASSERT_NE(req.io_buf_, nullptr);
ASSERT_NE(req.raw_buf_, nullptr);
ASSERT_EQ(2, result.result_ref_cnt_);
// prepare write request
req.destroy();
req.tenant_io_mgr_.hold(&tenant_io_mgr);
ObIOInfo write_info = read_info;
write_info.flag_.set_mode(ObIOMode::WRITE);
ASSERT_FAIL(req.init(write_info)); // not aligned
req.destroy();
ASSERT_EQ(1, result.result_ref_cnt_);
result.reset();
ASSERT_EQ(0, result.result_ref_cnt_);
req.tenant_io_mgr_.hold(&tenant_io_mgr);
result.tenant_io_mgr_.hold(&tenant_io_mgr);
ASSERT_FAIL(result.init(write_info)); // not aligned
ASSERT_TRUE(req.init(write_info ,&result));
ASSERT_EQ(1, result.result_ref_cnt_); //inc ref even fail
write_info.offset_ = DIO_READ_ALIGN_SIZE * 2;
ASSERT_FAIL(req.init(write_info)); // only offset aligned, size not aligned
req.reset();
result.reset();
req.tenant_io_mgr_.hold(&tenant_io_mgr);
result.tenant_io_mgr_.hold(&tenant_io_mgr);
ASSERT_FAIL(result.init(write_info)); // only offset aligned, size not aligned
ASSERT_TRUE(req.init(write_info ,&result));
write_info.size_ = DIO_READ_ALIGN_SIZE * 4;
ASSERT_FAIL(req.init(write_info)); // offset and size aligned, but write buf is null
req.reset();
result.reset();
req.tenant_io_mgr_.hold(&tenant_io_mgr);
result.tenant_io_mgr_.hold(&tenant_io_mgr);
ASSERT_FAIL(result.init(write_info)); // offset and size aligned, but write buf is null
ASSERT_TRUE(req.init(write_info ,&result));
write_info.buf_ = "test_write";
ASSERT_SUCC(req.init(write_info)); // normal usage
req.reset();
result.reset();
req.tenant_io_mgr_.hold(&tenant_io_mgr);
result.tenant_io_mgr_.hold(&tenant_io_mgr);
ASSERT_SUCC(result.init(write_info));
ASSERT_SUCC(req.init(write_info ,&result)); // normal usage
ASSERT_TRUE(req.is_inited_);
ASSERT_NE(req.io_buf_, nullptr); // write buf need copy immediately
ASSERT_TRUE(result.is_inited_);
ASSERT_NE(req.raw_buf_, nullptr); // write buf need copy immediately
// test finish
req.finish(OB_CANCELED);
ASSERT_EQ(req.ret_code_.io_ret_, OB_CANCELED);
req.finish(OB_IO_ERROR);
ASSERT_NE(req.ret_code_.io_ret_, OB_IO_ERROR); // finish only once
req.reset();
}
TEST_F(TestIOStruct, IOAbility)
@ -490,10 +550,14 @@ TEST_F(TestIOStruct, IOScheduler)
ASSERT_TRUE(scheduler.is_inited_);
// test schedule
ObIOResult result;
ObIORequest req;
result.inc_ref();
ObIOInfo io_info = get_random_io_info();
ASSERT_TRUE(io_info.is_valid());
ASSERT_SUCC(req.init(io_info));
ASSERT_SUCC(result.basic_init());
ASSERT_SUCC(result.init(io_info));
ASSERT_SUCC(req.init(io_info, &result));
ObTenantIOConfig tenant_config = default_tenant_io_config();
ObTenantIOClock io_clock;
ObIOUsage io_usage;
@ -559,6 +623,109 @@ TEST_F(TestIOStruct, MClockQueue)
}
}
TEST_F(TestIOStruct, Test_Size)
{
//callback
int64_t size1 = sizeof(ObAsyncSingleMicroBlockIOCallback);
int64_t size2 = sizeof(ObMultiDataBlockIOCallback);
int64_t size3 = sizeof(ObSyncSingleMicroBLockIOCallback);
int64_t size4 = sizeof(ObTmpPageCache::ObTmpPageIOCallback);
int64_t size5 = sizeof(ObTmpPageCache::ObTmpMultiPageIOCallback);
int64_t size6 = sizeof(oceanbase::ObStorageMetaCache::ObStorageMetaIOCallback);
int64_t max_callback_size = std::max({size1, size2, size3, size4, size5, size6});
int64_t size_request = sizeof(ObIORequest);
int64_t size_result = sizeof(ObIOResult);
int64_t size_info = sizeof(ObIOInfo);
int64_t size_thread_cond = sizeof(ObThreadCond);
int64_t size_flag = sizeof(ObIOFlag);
ObRefHolder<ObTenantIOManager> tenant_io_mgr_;
int64_t ref_size = sizeof(tenant_io_mgr_);
int64_t time_size = sizeof(ObIOTimeLog);
int64_t return_size = sizeof(ObIORetCode);
int64_t trace_size = sizeof(ObCurTraceId::TraceId);
int64_t fd_size = sizeof(ObIOFd);
ASSERT_LT(max_callback_size, 256);
LOG_INFO("qilu :check size", K(size1), K(size2), K(size3), K(size4), K(size5), K(size6), K(max_callback_size));
LOG_INFO("qilu :check size", K(size_request), K(size_result), K(size_info), K(size_thread_cond), K(size_flag),
K(ref_size), K(time_size), K(return_size), K(fd_size), K(trace_size));
//mark: max_callback_size=208(ObMultiDataBlockIOCallback、ObStorageMetaIOCallback)
}
TEST_F(TestIOStruct, IOResult)
{
ObTenantIOManager tenant_io_mgr;
tenant_io_mgr.inc_ref();
ASSERT_SUCC(tenant_io_mgr.io_allocator_.init(TEST_TENANT_ID, IO_MEMORY_LIMIT));
ObRefHolder<ObTenantIOManager> holder(&tenant_io_mgr);
ObIOFd fd;
fd.first_id_ = 0;
fd.second_id_ = 1;
void *result_buf = tenant_io_mgr.io_allocator_.alloc(sizeof(ObIOResult));
ObIOResult *result = new (result_buf) ObIOResult;
void *req_buf = tenant_io_mgr.io_allocator_.alloc(sizeof(ObIORequest));
ObIORequest *req = new (req_buf) ObIORequest;
// default invalid
ASSERT_FALSE(result->is_inited_);
ASSERT_FALSE(result->is_valid());
ASSERT_FALSE(req->is_inited_);
ASSERT_FALSE(req->is_valid());
// prepare test read request
req->destroy();
result->destroy();
req->tenant_io_mgr_.hold(&tenant_io_mgr);
result->tenant_io_mgr_.hold(&tenant_io_mgr);
result->inc_ref();
req->inc_ref();
ObIOInfo read_info;
read_info.tenant_id_ = OB_SERVER_TENANT_ID;
read_info.fd_ = fd;
read_info.flag_.set_mode(ObIOMode::READ);
read_info.flag_.set_group_id(10005);
read_info.flag_.set_wait_event(1);
read_info.timeout_us_ = DEFAULT_IO_WAIT_TIME_US;
read_info.offset_ = 89;
read_info.size_ = 1;
char user_buf[1] = { 0 };
read_info.user_data_buf_ = user_buf;
ASSERT_TRUE(read_info.is_valid());
ASSERT_SUCC(req->tenant_io_mgr_.get_ptr()->io_usage_.init(0));
ASSERT_FAIL(result->init(read_info));
ASSERT_FALSE(result->is_inited_);
ASSERT_SUCC(result->basic_init());
ASSERT_SUCC(result->init(read_info));
ASSERT_TRUE(result->is_inited_);
ASSERT_TRUE(result->is_valid());
ASSERT_SUCC(req->init(read_info, result));
ASSERT_TRUE(req->is_inited_);
ASSERT_EQ(req->raw_buf_, nullptr); // read buf allocation is delayed
ASSERT_SUCC(req->prepare());
ASSERT_NE(req->raw_buf_, nullptr);
ASSERT_EQ(result->get_group_id(), 10005);
ASSERT_EQ(req->get_group_id(), 10005);
// test finish
result->finish_without_accumulate(OB_CANCELED);
ASSERT_EQ(result->ret_code_.io_ret_, OB_CANCELED);
result->finish_without_accumulate(OB_IO_ERROR);
ASSERT_NE(result->ret_code_.io_ret_, OB_IO_ERROR); // finish only once
//test free
req->io_result_->dec_ref();
ASSERT_EQ(1, req->io_result_->result_ref_cnt_);
req->destroy();
ASSERT_EQ(nullptr, req->io_result_);
}
TEST_F(TestIOStruct, IOCallbackManager)
{
// test init
@ -571,23 +738,27 @@ TEST_F(TestIOStruct, IOCallbackManager)
ASSERT_TRUE(callback_mgr.is_inited_);
// test enqueue and dequeue
ObIOResult result;
ObIORequest req;
req.inc_ref();
result.inc_ref();
ObIOInfo io_info = get_random_io_info();
ASSERT_TRUE(io_info.is_valid());
ASSERT_SUCC(req.init(io_info));
ASSERT_SUCC(result.basic_init());
ASSERT_SUCC(result.init(io_info));
ASSERT_SUCC(req.init(io_info, &result));
ASSERT_FAIL(callback_mgr.enqueue_callback(req));
char buf[32] = "test";
req.io_buf_ = buf;
req.raw_buf_ = buf;
char callback_buf_[ObIOCallback::CALLBACK_BUF_SIZE] __attribute__ ((aligned (16)));
req.copied_callback_ = new (callback_buf_) TestIOCallback();
TestIOCallback *test_callback = new (callback_buf_) TestIOCallback();
result.io_callback_ = test_callback;
// ObIOManager::get_instance().io_config_ = ObIOConfig::default_config();
ASSERT_SUCC(callback_mgr.enqueue_callback(req));
// test destroy
callback_mgr.destroy();
ASSERT_FALSE(callback_mgr.is_inited_);
req.copied_callback_ = nullptr;
}
TEST_F(TestIOStruct, IOFaultDetector)
@ -612,18 +783,22 @@ TEST_F(TestIOStruct, IOFaultDetector)
// test read failure detection
ObIOInfo io_info = get_random_io_info();
ObIOResult result;
ObIORequest req;
req.inc_ref();
ASSERT_SUCC(req.init(io_info));
result.inc_ref();
ASSERT_SUCC(result.basic_init());
ASSERT_SUCC(result.init(io_info));
ASSERT_SUCC(req.init(io_info, &result));
detector.reset_device_health();
ASSERT_SUCC(detector.get_device_health_status(dhs, disk_abnormal_time));
ASSERT_TRUE(DEVICE_HEALTH_NORMAL == dhs);
ASSERT_TRUE(0 == disk_abnormal_time);
req.io_info_.flag_.set_mode(ObIOMode::READ);
result.flag_.set_mode(ObIOMode::READ);
io_config.data_storage_warning_tolerance_time_ = 1000L * 1000L;
io_config.data_storage_error_tolerance_time_ = 3000L * 1000L;
// io manager not init, skip this test
// detector.record_failure(req);
// detector.record_io_err_failure(req);
// usleep(2000L * 1000L);
// ASSERT_SUCC(detector.get_device_health(is_device_warning, is_device_error));
// ASSERT_TRUE(is_device_warning);
@ -705,6 +880,42 @@ public:
}
};
TEST_F(TestIOManager, memory_pool)
{
ObIOManager &io_mgr = ObIOManager::get_instance();
ObRefHolder<ObTenantIOManager> tenant_holder;
ASSERT_SUCC(io_mgr.get_tenant_io_manager(500, tenant_holder));
ASSERT_NE(nullptr, tenant_holder.get_ptr());
ObIORequest *io_request = nullptr;
ASSERT_SUCC(tenant_holder.get_ptr()->io_request_pool_.alloc(io_request));
ASSERT_NE(nullptr, io_request);
io_request->tenant_io_mgr_.hold(tenant_holder.get_ptr());
ASSERT_TRUE(tenant_holder.get_ptr()->io_request_pool_.contain(io_request));
ASSERT_SUCC(tenant_holder.get_ptr()->io_request_pool_.recycle(io_request));
ObIOResult *io_result = nullptr;
ASSERT_SUCC(tenant_holder.get_ptr()->io_result_pool_.alloc(io_result));
ASSERT_NE(nullptr, io_result);
io_result->tenant_io_mgr_.hold(tenant_holder.get_ptr());
ASSERT_TRUE(tenant_holder.get_ptr()->io_result_pool_.contain(io_result));
ASSERT_SUCC(tenant_holder.get_ptr()->io_result_pool_.recycle(io_result));
void *result_buf = tenant_holder.get_ptr()->io_allocator_.alloc(sizeof(ObIOResult));
ObIOResult *result1 = new (result_buf) ObIOResult;
result1->tenant_io_mgr_.hold(tenant_holder.get_ptr());
ASSERT_FALSE(tenant_holder.get_ptr()->io_result_pool_.contain(result1));
ASSERT_FAIL(tenant_holder.get_ptr()->io_result_pool_.recycle(result1));
tenant_holder.get_ptr()->io_allocator_.free(result1);
void *req_buf = tenant_holder.get_ptr()->io_allocator_.alloc(sizeof(ObIORequest));
ObIORequest *req1 = new (req_buf) ObIORequest;
req1->tenant_io_mgr_.hold(tenant_holder.get_ptr());
ASSERT_FALSE(tenant_holder.get_ptr()->io_request_pool_.contain(req1));
ASSERT_FAIL(tenant_holder.get_ptr()->io_request_pool_.recycle(req1));
tenant_holder.get_ptr()->io_allocator_.free(req1);
}
TEST_F(TestIOManager, simple)
{
ObIOFd fd;
@ -725,19 +936,21 @@ TEST_F(TestIOManager, simple)
const int64_t io_timeout_ms = 1000L * 5L;
const int64_t write_io_size = DIO_READ_ALIGN_SIZE * 2;
ObIOInfo io_info;
io_info.tenant_id_ = 500;
io_info.tenant_id_ = OB_SERVER_TENANT_ID;
io_info.fd_ = fd;
io_info.flag_.set_write();
io_info.flag_.set_group_id(0);
io_info.flag_.set_wait_event(100);
io_info.offset_ = 0;
io_info.size_ = write_io_size;
io_info.timeout_us_ = DEFAULT_IO_WAIT_TIME_US;
char buf[write_io_size] = { 0 };
const int64_t user_offset = write_io_size / 2;
memset(buf, 'a', user_offset);
memset(buf + user_offset, 'b', write_io_size - user_offset);
io_info.buf_ = buf;
ASSERT_SUCC(io_mgr.write(io_info, io_timeout_ms));
io_info.user_data_buf_ = buf;
ASSERT_SUCC(io_mgr.write(io_info));
// check size
{
@ -749,7 +962,7 @@ TEST_F(TestIOManager, simple)
// sync read and compare data buffer
io_info.flag_.set_read();
ObIOHandle io_handle;
ASSERT_SUCC(io_mgr.read(io_info, io_handle, io_timeout_ms));
ASSERT_SUCC(io_mgr.read(io_info, io_handle));
ASSERT_NE(nullptr, io_handle.get_buffer());
ASSERT_EQ(io_info.size_, io_handle.get_data_size());
ASSERT_EQ(0, memcmp(buf, io_handle.get_buffer(), write_io_size));
@ -758,21 +971,21 @@ TEST_F(TestIOManager, simple)
io_handle.reset();
io_info.offset_ = FILE_SIZE;
io_info.size_ = DIO_READ_ALIGN_SIZE;
ASSERT_NE(OB_SUCCESS, io_mgr.read(io_info, io_handle, io_timeout_ms));
ASSERT_NE(OB_SUCCESS, io_mgr.read(io_info, io_handle));
ASSERT_EQ(0, io_handle.get_data_size());
// read tail
io_handle.reset();
io_info.offset_ = FILE_SIZE - DIO_READ_ALIGN_SIZE;
io_info.size_ = DIO_READ_ALIGN_SIZE;
ASSERT_SUCC(io_mgr.read(io_info, io_handle, io_timeout_ms));
ASSERT_SUCC(io_mgr.read(io_info, io_handle));
ASSERT_EQ(DIO_READ_ALIGN_SIZE, io_handle.get_data_size());
// read tail part
io_handle.reset();
io_info.offset_ = FILE_SIZE - DIO_READ_ALIGN_SIZE;
io_info.size_ = DIO_READ_ALIGN_SIZE * 2;
ASSERT_NE(OB_SUCCESS, io_mgr.read(io_info, io_handle, io_timeout_ms));
ASSERT_NE(OB_SUCCESS, io_mgr.read(io_info, io_handle));
ASSERT_EQ(DIO_READ_ALIGN_SIZE, io_handle.get_data_size());
// read with callback
@ -785,7 +998,7 @@ TEST_F(TestIOManager, simple)
callback.number_ = &tmp_number;
callback.allocator_ = &allocator;
io_info.callback_ = &callback;
ASSERT_SUCC(io_mgr.read(io_info, io_handle, io_timeout_ms));
ASSERT_SUCC(io_mgr.read(io_info, io_handle));
ASSERT_NE(nullptr, io_handle.get_buffer());
ASSERT_EQ(io_info.size_, io_handle.get_data_size());
ASSERT_EQ(0, memcmp(buf + user_offset, io_handle.get_buffer(), write_io_size - user_offset));
@ -897,7 +1110,7 @@ class IOPerfRunner : public ThreadPool
{
public:
IOPerfRunner()
: abs_ts_(0), last_offset_(0), write_buf_(nullptr), io_count_(0), report_count_(0), total_io_count_(0)
: abs_ts_(0), last_offset_(0), write_buf_(nullptr), user_buf_(nullptr), io_count_(0), report_count_(0), total_io_count_(0)
{}
int init(const int64_t absolute_ts, const IOPerfLoad &load);
void destroy();
@ -908,13 +1121,14 @@ public:
int wait_and_count(ObIOHandle &io_handle);
int wait_handles(ObFixedQueue<ObIOHandle> &handles);
int print_result();
TO_STRING_KV(K(load_), K(result_), K(last_offset_), KP(write_buf_), K(fd_));
TO_STRING_KV(K(load_), K(result_), K(last_offset_), KP(write_buf_), KP(user_buf_), K(fd_));
public:
int64_t abs_ts_;
IOPerfLoad load_;
IOPerfResult result_;
int64_t last_offset_;
const char *write_buf_;
char *user_buf_;
ObIOFd fd_;
ObConcurrentFIFOAllocator allocator_;
ObFixedQueue<ObIOHandle> handle_queue_;
@ -946,6 +1160,27 @@ public:
ObIOFd fd_;
};
class IOCallbackModifier : public ThreadPool
{
public:
IOCallbackModifier()
: modify_init_ts_(0)
{}
int init(int64_t modify_init_ts, int64_t modify_delay_ts, const IOPerfTenant &curr_tenant);
void destroy();
virtual void run1() override;
int modify_callback_num(const int64_t thread_num,
IOPerfTenant &curr_tenant);
TO_STRING_KV(K(load_), K(modify_delay_ts_), K(fd_), K(curr_tenant_));
public:
int64_t modify_init_ts_;
int64_t modify_delay_ts_;
IOPerfTenant curr_tenant_;
ObConcurrentFIFOAllocator allocator_;
IOPerfLoad load_;
ObIOFd fd_;
};
class IOTracerSwitch : public ThreadPool
{
public:
@ -1129,6 +1364,77 @@ TEST_F(TestIOManager, perf)
LOG_INFO("wenqu: perf finished");
}
TEST_F(TestIOManager, alloc_memory)
{
// use multi thread to do some io stress, maybe use test_io_performance
bool is_perf_config_exist = false;
ASSERT_SUCC(FileDirectoryUtils::is_exists(GROUP_PERF_CONFIG_FILE, is_perf_config_exist));
if (!is_perf_config_exist) {
write_group_perf_config();
}
// parse configs
IOPerfScheduler scheduler_config;
ObArray<IOPerfDevice> perf_devices;
ObArray<IOPerfTenant> perf_tenants;
ObArray<IOPerfLoad> perf_loads;
ASSERT_SUCC(parse_group_perf_config(GROUP_PERF_CONFIG_FILE, scheduler_config, perf_devices, perf_tenants, perf_loads));
ASSERT_TRUE(perf_devices.count() > 0);
ASSERT_TRUE(perf_tenants.count() > 0);
ASSERT_TRUE(perf_loads.count() > 0);
ObIOManager::get_instance().destroy();
const int64_t memory_limit = 30L * 1024L * 1024L * 1024L; // 30GB
const int64_t queue_depth = 100L;
ASSERT_SUCC(ObIOManager::get_instance().init(memory_limit, queue_depth, scheduler_config.sender_count_, scheduler_config.schedule_media_id_));
ASSERT_SUCC(ObIOManager::get_instance().start());
// prepare devices and files
char *device_buf = (char *)malloc(sizeof(ObLocalDevice) * perf_devices.count());
ASSERT_TRUE(nullptr != device_buf);
for (int64_t i = 0; i < perf_devices.count(); ++i) {
IOPerfDevice &curr_config = perf_devices.at(i);
ASSERT_SUCC(prepare_file(curr_config.file_path_, curr_config.file_size_, curr_config.fd_));
ObLocalDevice *device = new (device_buf + sizeof(ObLocalDevice) * i) ObLocalDevice;
ASSERT_SUCC(init_device(curr_config.media_id_, *device));
ASSERT_SUCC(OB_IO_MANAGER.add_device_channel(device, curr_config.async_channel_count_, curr_config.sync_channel_count_, curr_config.max_io_depth_));
curr_config.device_handle_ = device;
}
// prepare tenant io manager
for (int64_t i = 0; i < perf_tenants.count(); ++i) {
IOPerfTenant &curr_config = perf_tenants.at(i);
curr_config.config_.memory_limit_ = 16L* 1024L * 1024L; //16MB
ASSERT_SUCC(OB_IO_MANAGER.add_tenant_io_manager(curr_config.tenant_id_, curr_config.config_));
ObRefHolder<ObTenantIOManager> tenant_holder;
ASSERT_SUCC(OB_IO_MANAGER.get_tenant_io_manager(curr_config.tenant_id_, tenant_holder));
ASSERT_SUCC(tenant_holder.get_ptr()->refresh_group_io_config());
}
// prepare perf runners
char *runner_buf = (char *)malloc(perf_loads.count() * sizeof(IOPerfRunner));
ObArray<IOPerfRunner *> runners;
const int64_t start_ts = ObTimeUtility::current_time() + 10000L;
for (int64_t i = 0; i < perf_loads.count(); ++i) {
IOPerfRunner *runner = new (runner_buf + i * sizeof(IOPerfRunner)) IOPerfRunner();
const IOPerfLoad &cur_load = perf_loads.at(i);
ASSERT_SUCC(runner->init(start_ts, cur_load));
ASSERT_SUCC(runners.push_back(runner));
}
// wait perf finished
for (int64_t i = 0; i < runners.count(); ++i) {
IOPerfRunner *runner = runners.at(i);
runner->wait();
ASSERT_SUCC(runner->print_result());
runner->destroy();
}
free(runner_buf);
ObIOManager::get_instance().stop();
ObIOManager::get_instance().destroy();
for (int64_t i = 0; i < perf_devices.count(); ++i) {
ObLocalDevice *device_handle = perf_devices.at(i).device_handle_;
device_handle->destroy();
}
free(device_buf);}
TEST_F(TestIOManager, IOTracer)
{
// use multi thread to do modify group_io_config
@ -1303,6 +1609,92 @@ TEST_F(TestIOManager, ModifyIOPS)
LOG_INFO("wenqu: modify finished");
}
TEST_F(TestIOManager, ModifyCallbackThread)
{
// use multi thread to do modify group_io_config
bool is_perf_config_exist = false;
ASSERT_SUCC(FileDirectoryUtils::is_exists(GROUP_PERF_CONFIG_FILE, is_perf_config_exist));
if (!is_perf_config_exist) {
write_group_perf_config();
}
// parse configs
IOPerfScheduler scheduler_config;
ObArray<IOPerfDevice> perf_devices;
ObArray<IOPerfTenant> perf_tenants;
ObArray<IOPerfLoad> perf_loads;
ASSERT_SUCC(parse_group_perf_config(GROUP_PERF_CONFIG_FILE, scheduler_config, perf_devices, perf_tenants, perf_loads));
ASSERT_TRUE(perf_devices.count() > 0);
ASSERT_TRUE(perf_tenants.count() > 0);
ASSERT_TRUE(perf_loads.count() > 0);
ObIOManager::get_instance().destroy();
const int64_t memory_limit = 30L * 1024L * 1024L * 1024L; // 30GB
const int64_t queue_depth = 100L;
ASSERT_SUCC(ObIOManager::get_instance().init(memory_limit, queue_depth, scheduler_config.sender_count_, scheduler_config.schedule_media_id_));
ASSERT_SUCC(ObIOManager::get_instance().start());
// prepare devices and files
char *device_buf = (char *)malloc(sizeof(ObLocalDevice) * perf_devices.count());
ASSERT_TRUE(nullptr != device_buf);
for (int64_t i = 0; i < perf_devices.count(); ++i) {
IOPerfDevice &curr_config = perf_devices.at(i);
ASSERT_SUCC(prepare_file(curr_config.file_path_, curr_config.file_size_, curr_config.fd_));
ObLocalDevice *device = new (device_buf + sizeof(ObLocalDevice) * i) ObLocalDevice;
ASSERT_SUCC(init_device(curr_config.media_id_, *device));
ASSERT_SUCC(OB_IO_MANAGER.add_device_channel(device, curr_config.async_channel_count_, curr_config.sync_channel_count_, curr_config.max_io_depth_));
curr_config.device_handle_ = device;
}
// prepare tenant io manager
for (int64_t i = 0; i < perf_tenants.count(); ++i) {
IOPerfTenant &curr_config = perf_tenants.at(i);
LOG_INFO("wenqu: tenant config", K(curr_config), K(i));
ASSERT_SUCC(OB_IO_MANAGER.add_tenant_io_manager(curr_config.tenant_id_, curr_config.config_));
ObRefHolder<ObTenantIOManager> tenant_holder;
ASSERT_SUCC(OB_IO_MANAGER.get_tenant_io_manager(curr_config.tenant_id_, tenant_holder));
ASSERT_SUCC(tenant_holder.get_ptr()->refresh_group_io_config());
}
// prepare perf runners
char *runner_buf = (char *)malloc(perf_loads.count() * sizeof(IOPerfRunner));
char *modifier_buf = (char *)malloc(perf_loads.count() * sizeof(IOConfModify));
ObArray<IOPerfRunner *> runners;
ObArray<IOCallbackModifier *> modifiers;
const int64_t start_ts = ObTimeUtility::current_time() + 10000L;
for (int64_t i = 0; i < perf_loads.count(); ++i) {
IOPerfRunner *runner = new (runner_buf + i * sizeof(IOPerfRunner)) IOPerfRunner();
const IOPerfLoad &cur_load = perf_loads.at(i);
ASSERT_SUCC(runner->init(start_ts, cur_load));
ASSERT_SUCC(runners.push_back(runner));
LOG_INFO("runner start now");
}
//prepare modifier
for (int64_t i = 0; i < perf_tenants.count(); ++i) {
IOCallbackModifier *modifier=new (modifier_buf + i * sizeof(IOCallbackModifier)) IOCallbackModifier();
IOPerfTenant &curr_tenant = perf_tenants.at(i);
int64_t modify_init_ts = start_ts;
int64_t modify_delay_ts = 2000000L; //2s后开始修改
ASSERT_SUCC(modifier->init(modify_init_ts, modify_delay_ts, curr_tenant));
ASSERT_SUCC(modifiers.push_back(modifier));
}
// wait perf finished
for (int64_t i = 0; i < runners.count(); ++i) {
IOPerfRunner *runner = runners.at(i);
runner->wait();
ASSERT_SUCC(runner->print_result());
runner->destroy();
}
free(runner_buf);
free(modifier_buf);
ObIOManager::get_instance().stop();
ObIOManager::get_instance().destroy();
for (int64_t i = 0; i < perf_devices.count(); ++i) {
ObLocalDevice *device_handle = perf_devices.at(i).device_handle_;
device_handle->destroy();
}
free(device_buf);
LOG_INFO("modify callback thread finished");
}
TEST_F(TestIOManager, abnormal)
{
// simulate submit failure
@ -1327,33 +1719,37 @@ TestIOCallback::~TestIOCallback()
{
if (nullptr != number_) {
*number_ -= 90;
number_ = nullptr;
}
number_ = nullptr;
allocator_ = nullptr;
if (nullptr != help_buf_ && nullptr != allocator_) {
allocator_->free(help_buf_);
help_buf_ = nullptr;
if (nullptr != allocator_) {
if (nullptr != help_buf_) {
allocator_->free(help_buf_);
help_buf_ = nullptr;
}
allocator_->free(this);
LOG_INFO("success reset callback when out_rec_cnt = 0");
}
}
int TestIOCallback::inner_deep_copy(char *buf, const int64_t buf_len, ObIOCallback *&callback) const
int TestIOCallback::alloc_data_buf(const char *io_data_buffer, const int64_t data_size)
{
int ret = OB_SUCCESS;
callback = nullptr;
if (OB_UNLIKELY(nullptr == buf || buf_len <= 0)) {
if (OB_ISNULL(allocator_)) {
ret = OB_INVALID_DATA;
LOG_WARN("Invalid data, the allocator is NULL, ", K(ret));
} else if (OB_UNLIKELY(data_size <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), KP(buf), K(buf_len));
} else if (buf_len < size()) {
ret = OB_BUF_NOT_ENOUGH;
LOG_WARN("buf_len not enough", K(ret), K(buf_len), K(size()));
STORAGE_LOG(WARN, "invalid data buffer size", K(ret), K(data_size));
} else if (OB_ISNULL(help_buf_ = static_cast<char *>(allocator_->alloc(data_size)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("Failed to allocate help buf", K(ret), K(data_size), KP(help_buf_));
} else {
TestIOCallback *tmp_callback = new (buf) TestIOCallback();
tmp_callback->number_ = number_;
tmp_callback->allocator_ = allocator_;
callback = tmp_callback;
memset(help_buf_, 0, data_size);
MEMCPY(help_buf_, io_data_buffer, data_size);
}
return ret;
return OB_SUCCESS;
}
int TestIOCallback::inner_process(const char *data_buffer, const int64_t size)
@ -1361,12 +1757,8 @@ int TestIOCallback::inner_process(const char *data_buffer, const int64_t size)
int ret = OB_SUCCESS;
if (OB_ISNULL(allocator_)) {
// for test, ignore
} else if (OB_ISNULL(help_buf_ = static_cast<char *>(allocator_->alloc(size)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("Failed to allocate help buf", K(ret), K(size), KP(data_buffer));
} else {
memset(help_buf_, 0, size);
MEMCPY(help_buf_, data_buffer, size);
} else if (OB_FAIL(alloc_data_buf(data_buffer, size))) {
LOG_WARN("Fail to allocate memory, ", K(ret), K(size));
}
if (nullptr != number_) {
*number_ += 100;
@ -1396,8 +1788,8 @@ void write_group_perf_config()
"tenant_id device_id group io_mode io_size_byte io_depth perf_mode target_iops thread_count is_sequence start_s stop_s\n"
"1001 1 0 r 16384 10 rolling 0 16 0 0 8\n"
"1001 1 10001 r 16384 10 rolling 0 16 0 2 7\n"
"1001 1 20001 r 16384 10 rolling 0 16 0 0 6\n"
"1002 1 0 r 16384 10 rolling 0 16 0 0 5\n"
"1001 1 10002 r 16384 10 rolling 0 16 0 0 6\n"
"1002 1 0 r 16384 10 rolling 0 16 0 0 8\n"
;
const int64_t file_len = strlen(file_buf);
int write_ret = ::write(fd, file_buf, file_len);
@ -1484,7 +1876,7 @@ int parse_group_perf_config(const char *config_file_path,
LOG_WARN("scan config file failed", K(ret), K(scan_ret));
} else {
item.config_.memory_limit_ = IO_MEMORY_LIMIT;
item.config_.callback_thread_count_ = 1;
item.config_.callback_thread_count_ = 0;
// parse group config
if (OB_FAIL(item.config_.parse_group_config(group_config))) {
LOG_WARN("parse group config failed", K(ret), K(group_config));
@ -1587,6 +1979,19 @@ int IOPerfRunner::init(const int64_t absolute_ts, const IOPerfLoad &load)
write_buf_ = reinterpret_cast<const char *>(tmp_buf);
}
}
// prepare read buffer
if (OB_SUCC(ret) && ObIOMode::READ == load_.mode_ && nullptr == user_buf_) {
const int64_t buf_size = load_.size_;
void *tmp_buf = ob_malloc(buf_size, ObNewModIds::TEST);
if (nullptr == tmp_buf) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret), K(buf_size));
} else {
memset(tmp_buf, '0', buf_size);
user_buf_ = reinterpret_cast<char *>(tmp_buf);
}
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(set_thread_count(load_.thread_count_ + 2))) {
@ -1612,6 +2017,10 @@ void IOPerfRunner::destroy()
ob_free((void *)write_buf_);
write_buf_ = nullptr;
}
if (nullptr != user_buf_) {
ob_free((void *)user_buf_);
user_buf_ = nullptr;
}
fd_.reset();
}
@ -1683,8 +2092,11 @@ int IOPerfRunner::do_perf_rolling()
info.flag_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ);
info.fd_ = fd_;
info.size_ = load_.size_;
info.timeout_us_ = DEFAULT_IO_WAIT_TIME_US;
if (ObIOMode::WRITE == load_.mode_) {
info.buf_ = reinterpret_cast<const char *>(upper_align(reinterpret_cast<int64_t>(write_buf_), DIO_READ_ALIGN_SIZE));
} else if (ObIOMode::READ == load_.mode_) {
info.user_data_buf_ = user_buf_;
}
ObArray<ObIOHandle *> handles;
RLOCAL(int64_t, local_io_count);
@ -1776,8 +2188,11 @@ int IOPerfRunner::do_batch_io()
info.flag_.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ);
info.fd_ = fd_;
info.size_ = load_.size_;
info.timeout_us_ = DEFAULT_IO_WAIT_TIME_US;
if (ObIOMode::WRITE == load_.mode_) {
info.buf_ = reinterpret_cast<const char *>(upper_align(reinterpret_cast<int64_t>(write_buf_), DIO_READ_ALIGN_SIZE));
} else if (ObIOMode::READ == load_.mode_) {
info.user_data_buf_ = user_buf_;
}
ObFixedQueue<ObIOHandle> local_handles;
if (OB_FAIL(local_handles.init(load_.depth_ * 2))) {
@ -1821,16 +2236,14 @@ int IOPerfRunner::do_batch_io()
int IOPerfRunner::wait_and_count(ObIOHandle &io_handle)
{
int ret = OB_SUCCESS;
if (OB_FAIL(io_handle.wait(DEFAULT_IO_WAIT_TIME_MS))) {
if (OB_FAIL(io_handle.wait())) {
LOG_WARN("fail to wait read io", K(ret));
ATOMIC_INC(&result_.fail_count_);
} else {
ATOMIC_INC(&result_.succ_count_);
}
const int64_t delay = max(0, io_handle.get_rt());
const int64_t disk_delay = max(0, io_handle.req_->time_log_.return_ts_ - io_handle.req_->time_log_.submit_ts_);
ATOMIC_FAA(&result_.sum_rt_, delay);
ATOMIC_FAA(&result_.disk_rt_, disk_delay);
return ret;
}
@ -1871,7 +2284,7 @@ int IOConfModify::init(int64_t modify_init_ts, int64_t modify_delay_ts, const IO
if (OB_UNLIKELY(!curr_tenant.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(curr_tenant));
} else if (OB_FAIL(allocator_.init(OB_MALLOC_BIG_BLOCK_SIZE, "perf runner", OB_SERVER_TENANT_ID, 1024L * 1024L * 1024L * 10L))) {
} else if (OB_FAIL(allocator_.init(OB_MALLOC_BIG_BLOCK_SIZE, "Modifier runner", OB_SERVER_TENANT_ID, 1024L * 1024L * 1024L * 10L))) {
LOG_WARN("init allocator failed", K(ret));
} else {
curr_tenant_ = curr_tenant;
@ -1937,7 +2350,7 @@ int IOTracerSwitch::init(int64_t switch_init_ts, int64_t switch_delay_ts, const
if (OB_UNLIKELY(!curr_tenant.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(curr_tenant));
} else if (OB_FAIL(allocator_.init(OB_MALLOC_BIG_BLOCK_SIZE, "perf runner", OB_SERVER_TENANT_ID, 1024L * 1024L * 1024L * 10L))) {
} else if (OB_FAIL(allocator_.init(OB_MALLOC_BIG_BLOCK_SIZE, "Switch runner", OB_SERVER_TENANT_ID, 1024L * 1024L * 1024L * 10L))) {
LOG_WARN("init allocator failed", K(ret));
} else {
curr_tenant_ = curr_tenant;
@ -1987,3 +2400,64 @@ int IOTracerSwitch::modify_tenant_io(IOPerfTenant &curr_tenant)
}
return ret;
}
int IOCallbackModifier::init(int64_t modify_init_ts, int64_t modify_delay_ts, const IOPerfTenant &curr_tenant)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!curr_tenant.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(curr_tenant));
} else if (OB_FAIL(allocator_.init(OB_MALLOC_BIG_BLOCK_SIZE, "Modifier runner", OB_SERVER_TENANT_ID, 1024L * 1024L * 1024L * 10L))) {
LOG_WARN("init allocator failed", K(ret));
} else {
curr_tenant_ = curr_tenant;
modify_init_ts_ = modify_init_ts;
modify_delay_ts_ = modify_delay_ts;
}
if (OB_SUCC(ret)) {
if (OB_FAIL(set_thread_count(load_.thread_count_ + 1))) {
LOG_WARN("set thread count failed", K(ret), K(modify_init_ts_), K(curr_tenant_));
} else if (OB_FAIL(start())) {
LOG_WARN("start thread failed", K(ret), K(modify_init_ts_), K(curr_tenant_));
}
}
if (OB_FAIL(ret)) {
destroy();
}
return ret;
}
void IOCallbackModifier::destroy()
{
stop();
wait();
curr_tenant_ = IOPerfTenant();
}
void IOCallbackModifier::run1()
{
int ret = OB_SUCCESS;
const int64_t thread_idx = get_thread_idx();
LOG_INFO("modify thread start");
const int64_t current_ts = ObTimeUtility::current_time();
if (modify_init_ts_ + modify_delay_ts_ > current_ts) {
usleep(modify_init_ts_ + modify_delay_ts_ - current_ts);
}
int64_t new_callback_num = 16;
if (OB_FAIL(modify_callback_num(new_callback_num, curr_tenant_))) {
LOG_WARN("modify config failed", K(ret), K(curr_tenant_));
} else {
LOG_INFO("modify callback thread num success", K(curr_tenant_));
}
}
int IOCallbackModifier::modify_callback_num(const int64_t thread_num,
IOPerfTenant &curr_tenant)
{
int ret = OB_SUCCESS;
curr_tenant.config_.callback_thread_count_ = thread_num;
if (OB_FAIL(OB_IO_MANAGER.refresh_tenant_io_config(curr_tenant.tenant_id_, curr_tenant.config_))) {
LOG_WARN("refresh tenant io config failed", K(ret), K(curr_tenant.tenant_id_), K(curr_tenant.config_));
}
return ret;
}