[CP] [CP] [TMP.FILE] support single read huge tmp file.

This commit is contained in:
Tyshawn
2022-07-20 11:31:16 +08:00
committed by wangzelin.wzl
parent cdb3f87700
commit 18019f2bf2
3 changed files with 228 additions and 84 deletions

View File

@ -43,17 +43,27 @@ bool ObTmpFileIOInfo::is_valid() const
} }
ObTmpFileIOHandle::ObTmpFileIOHandle() ObTmpFileIOHandle::ObTmpFileIOHandle()
: tmp_file_(NULL), io_handles_(), page_cache_handles_(), block_cache_handles_(), buf_(NULL), : tmp_file_(NULL),
size_(0), is_read_(false), has_wait_(false) io_handles_(),
{ page_cache_handles_(),
} block_cache_handles_(),
buf_(NULL),
size_(0),
is_read_(false),
has_wait_(false),
expect_read_size_(0),
last_read_offset_(-1),
io_flag_(),
update_offset_in_file_(false)
{}
ObTmpFileIOHandle::~ObTmpFileIOHandle() ObTmpFileIOHandle::~ObTmpFileIOHandle()
{ {
reset(); reset();
} }
int ObTmpFileIOHandle::prepare_read(char* read_buf, ObTmpFile* file) int ObTmpFileIOHandle::prepare_read(const int64_t read_size, const int64_t read_offset, const common::ObIODesc &io_flag,
char *read_buf, ObTmpFile *file)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
if (NULL == read_buf || NULL == file) { if (NULL == read_buf || NULL == file) {
@ -65,6 +75,9 @@ int ObTmpFileIOHandle::prepare_read(char* read_buf, ObTmpFile* file)
tmp_file_ = file; tmp_file_ = file;
is_read_ = true; is_read_ = true;
has_wait_ = false; has_wait_ = false;
expect_read_size_ = read_size;
last_read_offset_ = read_offset;
io_flag_ = io_flag;
} }
return ret; return ret;
} }
@ -81,6 +94,8 @@ int ObTmpFileIOHandle::prepare_write(char* write_buf, const int64_t write_size,
tmp_file_ = file; tmp_file_ = file;
is_read_ = false; is_read_ = false;
has_wait_ = false; has_wait_ = false;
expect_read_size_ = 0;
last_read_offset_ = -1;
} }
return ret; return ret;
} }
@ -91,33 +106,68 @@ int ObTmpFileIOHandle::wait(const int64_t timeout_ms)
if (OB_UNLIKELY(has_wait_ && is_read_)) { if (OB_UNLIKELY(has_wait_ && is_read_)) {
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(ERROR, "read wait() isn't reentrant interface, shouldn't call again", K(ret)); STORAGE_LOG(ERROR, "read wait() isn't reentrant interface, shouldn't call again", K(ret));
} else { } else if (OB_FAIL(do_wait(timeout_ms))) {
for (int32_t i = 0; OB_SUCC(ret) && i < block_cache_handles_.count(); i++) { STORAGE_LOG(WARN, "fail to wait tmp file io", K(ret), K(timeout_ms));
ObBlockCacheHandle &tmp = block_cache_handles_.at(i); } else if (is_read_ && !has_wait_) {
MEMCPY(tmp.buf_, tmp.block_handle_.value_->get_buffer() + tmp.offset_, tmp.size_); if (size_ == expect_read_size_) {
tmp.block_handle_.reset(); // do nothing
} } else if (OB_UNLIKELY(size_ > expect_read_size_)) {
block_cache_handles_.reset(); ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "read size more than expected size", K(ret), K(timeout_ms));
for (int32_t i = 0; OB_SUCC(ret) && i < page_cache_handles_.count(); i++) { } else {
ObPageCacheHandle &tmp = page_cache_handles_.at(i); ObTmpFileIOInfo io_info;
MEMCPY(tmp.buf_, tmp.page_handle_.value_->get_buffer() + tmp.offset_, tmp.size_); io_info.fd_ = tmp_file_->get_fd();
tmp.page_handle_.reset(); io_info.dir_id_ = tmp_file_->get_dir_id();
} io_info.tenant_id_ = tmp_file_->get_tenant_id();
page_cache_handles_.reset(); io_info.size_ = expect_read_size_;
io_info.buf_ = buf_;
for (int32_t i = 0; OB_SUCC(ret) && i < io_handles_.count(); i++) { io_info.io_desc_ = io_flag_;
ObIOReadHandle &tmp = io_handles_.at(i); while (OB_SUCC(ret) && size_ < expect_read_size_) {
if (OB_FAIL(tmp.macro_handle_.wait(timeout_ms))) { if (OB_FAIL(tmp_file_->once_aio_read_batch(io_info, update_offset_in_file_, last_read_offset_, *this))) {
STORAGE_LOG(WARN, "fail to wait tmp read io", K(ret)); STORAGE_LOG(WARN, "fail to read once batch", K(ret), K(timeout_ms), K(io_info), K(*this));
} else { } else if (OB_FAIL(do_wait(timeout_ms))) {
MEMCPY(tmp.buf_, tmp.macro_handle_.get_buffer() + tmp.offset_, tmp.size_); STORAGE_LOG(WARN, "fail to wait tmp file io", K(ret), K(timeout_ms));
tmp.macro_handle_.reset(); }
} }
} }
io_handles_.reset();
has_wait_ = true;
} }
if (OB_SUCC(ret) || OB_ITER_END == ret) {
has_wait_ = true;
expect_read_size_ = 0;
last_read_offset_ = -1;
update_offset_in_file_ = false;
}
return ret;
}
int ObTmpFileIOHandle::do_wait(const int64_t timeout_ms)
{
int ret = OB_SUCCESS;
for (int32_t i = 0; OB_SUCC(ret) && i < block_cache_handles_.count(); i++) {
ObBlockCacheHandle &tmp = block_cache_handles_.at(i);
MEMCPY(tmp.buf_, tmp.block_handle_.value_->get_buffer() + tmp.offset_, tmp.size_);
tmp.block_handle_.reset();
}
block_cache_handles_.reset();
for (int32_t i = 0; OB_SUCC(ret) && i < page_cache_handles_.count(); i++) {
ObPageCacheHandle &tmp = page_cache_handles_.at(i);
MEMCPY(tmp.buf_, tmp.page_handle_.value_->get_buffer() + tmp.offset_, tmp.size_);
tmp.page_handle_.reset();
}
page_cache_handles_.reset();
for (int32_t i = 0; OB_SUCC(ret) && i < io_handles_.count(); i++) {
ObIOReadHandle &tmp = io_handles_.at(i);
if (OB_FAIL(tmp.macro_handle_.wait(timeout_ms))) {
STORAGE_LOG(WARN, "fail to wait tmp read io", K(ret));
} else {
MEMCPY(tmp.buf_, tmp.macro_handle_.get_buffer() + tmp.offset_, tmp.size_);
tmp.macro_handle_.reset();
}
}
io_handles_.reset();
return ret; return ret;
} }
@ -140,6 +190,9 @@ void ObTmpFileIOHandle::reset()
tmp_file_ = NULL; tmp_file_ = NULL;
is_read_ = false; is_read_ = false;
has_wait_ = false; has_wait_ = false;
expect_read_size_ = 0;
last_read_offset_ = -1;
update_offset_in_file_ = false;
} }
bool ObTmpFileIOHandle::is_valid() bool ObTmpFileIOHandle::is_valid()
@ -556,51 +609,98 @@ int64_t ObTmpFile::find_first_extent(const int64_t offset)
return first_extent; return first_extent;
} }
int ObTmpFile::aio_pread_without_lock(const ObTmpFileIOInfo &io_info, int ObTmpFile::aio_read_without_lock(const ObTmpFileIOInfo &io_info, int64_t &offset, ObTmpFileIOHandle &handle)
int64_t &offset, ObTmpFileIOHandle &handle)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
char* buf = io_info.buf_; ObTmpFileExtent *tmp = nullptr;
int64_t size = io_info.size_;
int64_t read_size = 0; if (OB_ISNULL(tmp = file_meta_.get_last_extent())) {
ObTmpFileExtent* tmp = file_meta_.get_last_extent();
if (NULL == tmp) {
ret = OB_BAD_NULL_ERROR; ret = OB_BAD_NULL_ERROR;
STORAGE_LOG(WARN, "fail to read, because this temporary file is empty", K(ret), K(io_info)); STORAGE_LOG(WARN, "fail to read, because the tmp file is empty", K(ret), KP(tmp), K(io_info));
} else if (size > 0 && offset >= tmp->get_global_end()) { } else if (OB_UNLIKELY(io_info.size_ > 0 && offset >= tmp->get_global_end())) {
ret = OB_ITER_END; ret = OB_ITER_END;
} else if (OB_FAIL(handle.prepare_read(io_info.buf_, this))) { } else if (OB_FAIL(handle.prepare_read(io_info.size_, offset, io_info.io_desc_, io_info.buf_, this))) {
STORAGE_LOG(WARN, "fail to prepare read io handle", K(ret)); STORAGE_LOG(WARN, "fail to prepare read io handle", K(ret), K(io_info), K(offset));
} else if (once_aio_read_batch_without_lock(io_info, offset, handle)) {
STORAGE_LOG(WARN, "fail to read one batch", K(ret), K(offset), K(handle));
} else { } else {
int64_t ith_extent = 0; handle.set_last_read_offset(offset);
common::ObIArray<ObTmpFileExtent *> &extents = file_meta_.get_extents(); }
if (offset >= last_extent_min_offset_ && offset_ <= last_extent_max_offset_) { return ret;
ith_extent = last_extent_id_; }
} else {
ith_extent = find_first_extent(offset); int ObTmpFile::once_aio_read_batch(
} const ObTmpFileIOInfo &io_info, const bool need_update_offset, int64_t &offset, ObTmpFileIOHandle &handle)
for (; OB_SUCC(ret) && ith_extent < extents.count() && size > 0; ++ith_extent) { {
tmp = extents.at(ith_extent); int ret = OB_SUCCESS;
if (tmp->get_global_start() <= offset && offset < tmp->get_global_end()) { ObTmpFileExtent *tmp = nullptr;
if (offset + size > tmp->get_global_end()) { const int64_t remain_size = io_info.size_ - handle.get_data_size();
read_size = tmp->get_global_end() - offset;
} else { SpinWLockGuard guard(lock_);
read_size = size; if (OB_UNLIKELY(!is_inited_)) {
} ret = OB_NOT_INIT;
// read from the extent. STORAGE_LOG(WARN, "ObTmpFile has not been initialized", K(ret));
if (OB_FAIL(tmp->read(io_info, offset - tmp->get_global_start(), read_size, buf, handle))) { } else if (OB_UNLIKELY(offset < 0 || remain_size < 0) || OB_ISNULL(io_info.buf_)) {
STORAGE_LOG(WARN, "fail to read the extent", K(ret), K(io_info), K(buf), KP_(io_info.buf)); ret = OB_INVALID_ARGUMENT;
} else { STORAGE_LOG(WARN, "invalid argument", K(ret), K(offset), K(remain_size), KP(io_info.buf_));
offset += read_size; } else if (OB_ISNULL(tmp = file_meta_.get_last_extent())) {
size -= read_size; ret = OB_ERR_UNEXPECTED;
buf += read_size; STORAGE_LOG(WARN, "unexpected error, null tmp file extent", K(ret), KP(tmp), K(io_info));
handle.add_data_size(read_size); } else if (OB_UNLIKELY(remain_size > 0 && offset >= tmp->get_global_end())) {
last_extent_id_ = ith_extent; ret = OB_ITER_END;
last_extent_min_offset_ = tmp->get_global_start(); } else if (once_aio_read_batch_without_lock(io_info, offset, handle)) {
last_extent_max_offset_ = tmp->get_global_end(); STORAGE_LOG(WARN, "fail to read one batch", K(ret), K(offset), K(handle));
} } else {
handle.set_last_read_offset(offset);
}
if (need_update_offset) {
offset_ = offset;
}
return ret;
}
int ObTmpFile::once_aio_read_batch_without_lock(
const ObTmpFileIOInfo &io_info, int64_t &offset, ObTmpFileIOHandle &handle)
{
int ret = OB_SUCCESS;
int64_t one_batch_read_size = 0;
char *buf = io_info.buf_ + handle.get_data_size();
int64_t remain_size = io_info.size_ - handle.get_data_size();
int64_t read_size = 0;
int64_t ith_extent = 0;
ObTmpFileExtent *tmp = nullptr;
common::ObIArray<ObTmpFileExtent *> &extents = file_meta_.get_extents();
if (offset >= last_extent_min_offset_ && offset_ <= last_extent_max_offset_) {
ith_extent = last_extent_id_;
} else {
ith_extent = find_first_extent(offset);
}
while (OB_SUCC(ret) && ith_extent < extents.count() && remain_size > 0 && one_batch_read_size < READ_SIZE_PER_BATCH) {
tmp = extents.at(ith_extent);
if (tmp->get_global_start() <= offset && offset < tmp->get_global_end()) {
if (offset + remain_size > tmp->get_global_end()) {
read_size = tmp->get_global_end() - offset;
} else {
read_size = remain_size;
}
// read from the extent.
if (OB_FAIL(tmp->read(io_info, offset - tmp->get_global_start(), read_size, buf, handle))) {
STORAGE_LOG(WARN, "fail to read the extent", K(ret), K(io_info), K(buf), KP_(io_info.buf));
} else {
offset += read_size;
remain_size -= read_size;
buf += read_size;
one_batch_read_size += read_size;
handle.add_data_size(read_size);
last_extent_id_ = ith_extent;
last_extent_min_offset_ = tmp->get_global_start();
last_extent_max_offset_ = tmp->get_global_end();
} }
} }
++ith_extent;
} }
return ret; return ret;
} }
@ -614,10 +714,12 @@ int ObTmpFile::aio_read(const ObTmpFileIOInfo& io_info, ObTmpFileIOHandle& handl
} else { } else {
tenant_id_ = io_info.tenant_id_; tenant_id_ = io_info.tenant_id_;
SpinWLockGuard guard(lock_); SpinWLockGuard guard(lock_);
if (OB_FAIL(aio_pread_without_lock(io_info, offset_, handle))) { if (OB_FAIL(aio_read_without_lock(io_info, offset_, handle))) {
if (OB_ITER_END != ret) { if (OB_ITER_END != ret) {
STORAGE_LOG(WARN, "fail to do aio pread without lock", K(ret)); STORAGE_LOG(WARN, "fail to do aio read without lock", K(ret));
} }
} else {
handle.set_update_offset_in_file();
} }
} }
return ret; return ret;
@ -631,9 +733,9 @@ int ObTmpFile::aio_pread(const ObTmpFileIOInfo& io_info, const int64_t offset, O
} else { } else {
int64_t tmp_offset = offset; int64_t tmp_offset = offset;
SpinRLockGuard guard(lock_); SpinRLockGuard guard(lock_);
if (OB_FAIL(aio_pread_without_lock(io_info, tmp_offset, handle))) { if (OB_FAIL(aio_read_without_lock(io_info, tmp_offset, handle))) {
if (OB_ITER_END != ret) { if (OB_ITER_END != ret) {
STORAGE_LOG(WARN, "fail to do aio pread without lock", K(ret)); STORAGE_LOG(WARN, "fail to do aio read without lock", K(ret));
} }
} }
} }
@ -677,7 +779,9 @@ int ObTmpFile::read(const ObTmpFileIOInfo& io_info, const int64_t timeout_ms, Ob
} }
} }
} else if (OB_FAIL(handle.wait(timeout_ms))) { } else if (OB_FAIL(handle.wait(timeout_ms))) {
STORAGE_LOG(WARN, "fail to wait io finish", K(ret), K(timeout_ms)); if (OB_ITER_END != ret) {
STORAGE_LOG(WARN, "fail to wait io finish", K(ret), K(timeout_ms));
}
} }
return ret; return ret;
} }
@ -700,7 +804,9 @@ int ObTmpFile::pread(
} }
} }
} else if (OB_FAIL(handle.wait(timeout_ms))) { } else if (OB_FAIL(handle.wait(timeout_ms))) {
STORAGE_LOG(WARN, "fail to wait io finish", K(ret), K(timeout_ms)); if (OB_ITER_END != ret) {
STORAGE_LOG(WARN, "fail to wait io finish", K(ret), K(timeout_ms));
}
} }
return ret; return ret;
} }
@ -811,7 +917,7 @@ uint64_t ObTmpFile::get_tenant_id() const
return tenant_id_; return tenant_id_;
} }
int64_t ObTmpFile::get_fd() int64_t ObTmpFile::get_fd() const
{ {
return file_meta_.get_fd(); return file_meta_.get_fd();
} }

View File

@ -91,8 +91,9 @@ public:
{ {
return size_; return size_;
} }
int prepare_read(char* read_buf, ObTmpFile* file); int prepare_read(const int64_t read_size, const int64_t read_offset, const common::ObIODesc &io_flag, char *read_buf,
int prepare_write(char* write_buf, const int64_t write_size, ObTmpFile* file); ObTmpFile *file);
int prepare_write(char *write_buf, const int64_t write_size, ObTmpFile *file);
OB_INLINE void add_data_size(const int64_t size) OB_INLINE void add_data_size(const int64_t size)
{ {
size_ += size; size_ += size;
@ -101,6 +102,14 @@ public:
{ {
size_ -= size; size_ -= size;
} }
OB_INLINE void set_update_offset_in_file()
{
update_offset_in_file_ = true;
}
OB_INLINE void set_last_read_offset(const int64_t last_read_offset)
{
last_read_offset_ = last_read_offset;
}
int wait(const int64_t timeout_ms); int wait(const int64_t timeout_ms);
void reset(); void reset();
bool is_valid(); bool is_valid();
@ -116,7 +125,16 @@ public:
{ {
return block_cache_handles_; return block_cache_handles_;
} }
TO_STRING_KV(KP_(buf), K_(size), K_(is_read)); OB_INLINE int64_t get_last_read_offset() const
{
return last_read_offset_;
}
TO_STRING_KV(KP_(buf), K_(size), K_(is_read), K_(has_wait), K_(expect_read_size), K_(last_read_offset), K_(io_flag),
K_(update_offset_in_file));
private:
int do_wait(const int64_t timeout_ms);
private: private:
ObTmpFile* tmp_file_; ObTmpFile* tmp_file_;
@ -127,6 +145,10 @@ private:
int64_t size_; // has read or to write size. int64_t size_; // has read or to write size.
bool is_read_; bool is_read_;
bool has_wait_; bool has_wait_;
int64_t expect_read_size_;
int64_t last_read_offset_; // only for more than 8MB read.
common::ObIODesc io_flag_;
bool update_offset_in_file_;
DISALLOW_COPY_AND_ASSIGN(ObTmpFileIOHandle); DISALLOW_COPY_AND_ASSIGN(ObTmpFileIOHandle);
}; };
@ -280,15 +302,21 @@ public:
int clear(); int clear();
int64_t get_dir_id() const; int64_t get_dir_id() const;
uint64_t get_tenant_id() const; uint64_t get_tenant_id() const;
int64_t get_fd(); int64_t get_fd() const;
int sync(const int64_t timeout_ms); int sync(const int64_t timeout_ms);
int deep_copy(char* buf, const int64_t buf_len, ObTmpFile*& value) const; int deep_copy(char* buf, const int64_t buf_len, ObTmpFile*& value) const;
inline int64_t get_deep_copy_size() const; inline int64_t get_deep_copy_size() const;
void get_file_size(int64_t &file_size);
// only for ObTmpFileIOHandle, once more than READ_SIZE_PER_BATCH read.
int once_aio_read_batch(
const ObTmpFileIOInfo &io_info, const bool need_update_offset, int64_t &offset, ObTmpFileIOHandle &handle);
TO_STRING_KV(K_(file_meta), K_(is_big), K_(tenant_id), K_(is_inited)); TO_STRING_KV(K_(file_meta), K_(is_big), K_(tenant_id), K_(is_inited));
private: private:
int write_file_extent(const ObTmpFileIOInfo& io_info, ObTmpFileExtent* file_extent, int64_t& size, char*& buf); int write_file_extent(const ObTmpFileIOInfo &io_info, ObTmpFileExtent *file_extent, int64_t &size, char *&buf);
int aio_pread_without_lock(const ObTmpFileIOInfo& io_info, int64_t& offset, ObTmpFileIOHandle& handle); int aio_read_without_lock(const ObTmpFileIOInfo &io_info, int64_t &offset, ObTmpFileIOHandle &handle);
int once_aio_read_batch_without_lock(const ObTmpFileIOInfo &io_info, int64_t &offset, ObTmpFileIOHandle &handle);
int64_t small_file_prealloc_size(); int64_t small_file_prealloc_size();
int64_t big_file_prealloc_size(); int64_t big_file_prealloc_size();
int64_t find_first_extent(const int64_t offset); int64_t find_first_extent(const int64_t offset);
@ -299,6 +327,7 @@ private:
// SMALL_FILE_MAX_THRESHOLD < BIG_FILE_PREALLOC_EXTENT_SIZE < block size // SMALL_FILE_MAX_THRESHOLD < BIG_FILE_PREALLOC_EXTENT_SIZE < block size
static const int64_t SMALL_FILE_MAX_THRESHOLD = 4; static const int64_t SMALL_FILE_MAX_THRESHOLD = 4;
static const int64_t BIG_FILE_PREALLOC_EXTENT_SIZE = 8; static const int64_t BIG_FILE_PREALLOC_EXTENT_SIZE = 8;
static const int64_t READ_SIZE_PER_BATCH = 8 * 1024 * 1024; // 8MB
ObTmpFileMeta file_meta_; ObTmpFileMeta file_meta_;
bool is_big_; bool is_big_;

View File

@ -585,11 +585,20 @@ TEST_F(TestTmpFile, test_big_file)
ASSERT_EQ(OB_SUCCESS, ret); ASSERT_EQ(OB_SUCCESS, ret);
io_info.buf_ = read_buf; io_info.buf_ = read_buf;
io_info.size_ = write_size;
ret = ObTmpFileManager::get_instance().aio_read(io_info, handle);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_TRUE(handle.size_ < handle.expect_read_size_);
ASSERT_EQ(OB_SUCCESS, handle.wait(timeout_ms));
ASSERT_EQ(write_size, handle.get_data_size());
int cmp = memcmp(handle.get_buffer(), write_buf, handle.get_data_size());
ASSERT_EQ(0, cmp);
io_info.size_ = macro_block_size; io_info.size_ = macro_block_size;
ret = ObTmpFileManager::get_instance().pread(io_info, 100, timeout_ms, handle); ret = ObTmpFileManager::get_instance().pread(io_info, 100, timeout_ms, handle);
ASSERT_EQ(OB_SUCCESS, ret); ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(macro_block_size, handle.get_data_size()); ASSERT_EQ(macro_block_size, handle.get_data_size());
int cmp = memcmp(handle.get_buffer(), write_buf + 100, handle.get_data_size()); cmp = memcmp(handle.get_buffer(), write_buf + 100, handle.get_data_size());
ASSERT_EQ(0, cmp); ASSERT_EQ(0, cmp);
io_info.size_ = write_size; io_info.size_ = write_size;
@ -1068,7 +1077,7 @@ TEST_F(TestTmpFile, test_write_less_than_macro_block_size)
ASSERT_EQ(0, cmp); ASSERT_EQ(0, cmp);
ret = ObTmpFileManager::get_instance().pread(io_info, 20, timeout_ms, handle); ret = ObTmpFileManager::get_instance().pread(io_info, 20, timeout_ms, handle);
ASSERT_EQ(OB_SUCCESS, ret); ASSERT_EQ(OB_ITER_END, ret);
ASSERT_EQ(256 - 20, handle.get_data_size()); ASSERT_EQ(256 - 20, handle.get_data_size());
cmp = memcmp(handle.get_buffer(), write_buf + 20, 256 - 20); cmp = memcmp(handle.get_buffer(), write_buf + 20, 256 - 20);
ASSERT_EQ(0, cmp); ASSERT_EQ(0, cmp);
@ -1157,7 +1166,7 @@ TEST_F(TestTmpFile, test_write_more_than_one_macro_block)
io_info.size_ = macro_block_size; io_info.size_ = macro_block_size;
ret = ObTmpFileManager::get_instance().pread(io_info, 400, timeout_ms, handle); ret = ObTmpFileManager::get_instance().pread(io_info, 400, timeout_ms, handle);
ASSERT_EQ(OB_SUCCESS, ret); ASSERT_EQ(OB_ITER_END, ret);
ASSERT_EQ(macro_block_size + 256 - 400, handle.get_data_size()); ASSERT_EQ(macro_block_size + 256 - 400, handle.get_data_size());
cmp = memcmp(handle.get_buffer(), write_buf + 400, macro_block_size + 256 - 400); cmp = memcmp(handle.get_buffer(), write_buf + 400, macro_block_size + 256 - 400);
ASSERT_EQ(0, cmp); ASSERT_EQ(0, cmp);
@ -1178,7 +1187,7 @@ TEST_F(TestTmpFile, test_write_more_than_one_macro_block)
io_info.size_ = 200; io_info.size_ = 200;
ret = ObTmpFileManager::get_instance().pread(io_info, macro_block_size + 100, timeout_ms, handle); ret = ObTmpFileManager::get_instance().pread(io_info, macro_block_size + 100, timeout_ms, handle);
ASSERT_EQ(OB_SUCCESS, ret); ASSERT_EQ(OB_ITER_END, ret);
ASSERT_EQ(156, handle.get_data_size()); ASSERT_EQ(156, handle.get_data_size());
cmp = memcmp(handle.get_buffer(), write_buf + macro_block_size + 100, handle.get_data_size()); cmp = memcmp(handle.get_buffer(), write_buf + macro_block_size + 100, handle.get_data_size());
ASSERT_EQ(0, cmp); ASSERT_EQ(0, cmp);