diff --git a/dbms/src/Common/UniThreadPool.h b/dbms/src/Common/UniThreadPool.h index 0e6857fae2f..d61d7a2f35c 100644 --- a/dbms/src/Common/UniThreadPool.h +++ b/dbms/src/Common/UniThreadPool.h @@ -29,7 +29,6 @@ #include #include #include -#include #include namespace DB diff --git a/dbms/src/Storages/Page/V3/Blob/BlobStat.cpp b/dbms/src/Storages/Page/V3/Blob/BlobStat.cpp index c47bf595099..2d18b53cb56 100644 --- a/dbms/src/Storages/Page/V3/Blob/BlobStat.cpp +++ b/dbms/src/Storages/Page/V3/Blob/BlobStat.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -216,8 +217,6 @@ std::pair BlobStats::chooseStat( PageType page_type, const std::lock_guard &) { - BlobStatPtr stat_ptr = nullptr; - // No stats exist if (stats_map.empty()) { @@ -301,6 +300,12 @@ BlobStats::StatsMap BlobStats::getStats() const NO_THREAD_SAFETY_ANALYSIS BlobFileOffset BlobStats::BlobStat::getPosFromStat(size_t buf_size, const std::unique_lock &) { + // A shortcut for empty page. All empty pages will be stored + // at the beginning of the BlobFile. It should not affects the + // sm_max_caps or other fields by adding these empty pages. + if (unlikely(buf_size == 0)) + return 0; + BlobFileOffset offset = 0; UInt64 max_cap = 0; bool expansion = true; diff --git a/dbms/src/Storages/Page/V3/BlobStore.cpp b/dbms/src/Storages/Page/V3/BlobStore.cpp index 15f92f8d6ac..7a40a4ddf47 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.cpp +++ b/dbms/src/Storages/Page/V3/BlobStore.cpp @@ -36,7 +36,6 @@ #include #include -#include #include #include @@ -730,6 +729,7 @@ std::pair BlobStore::getPosFromStats(size_t s Stopwatch watch; BlobStatPtr stat; + // TODO: make this lambda as a function of BlobStats to simplify code auto lock_stat = [size, this, &stat, &page_type]() NO_THREAD_SAFETY_ANALYSIS { auto lock_stats = blob_stats.lock(); BlobFileId blob_file_id = INVALID_BLOBFILE_ID; @@ -911,8 +911,7 @@ typename BlobStore::PageMap BlobStore::read(FieldReadInfos & to_re // TODO: Continuously fields can read by one system call. const auto [beg_offset, end_offset] = entry.getFieldOffsets(field_index); const auto size_to_read = end_offset - beg_offset; - auto blob_file - = read(page_id_v3, entry.file_id, entry.offset + beg_offset, write_offset, size_to_read, read_limiter); + read(page_id_v3, entry.file_id, entry.offset + beg_offset, write_offset, size_to_read, read_limiter); fields_offset_in_page.emplace(field_index, read_size_this_entry); if constexpr (BLOBSTORE_CHECKSUM_ON_READ) @@ -926,17 +925,22 @@ typename BlobStore::PageMap BlobStore::read(FieldReadInfos & to_re throw Exception( ErrorCodes::CHECKSUM_DOESNT_MATCH, "Reading with fields meet checksum not match " +<<<<<<< HEAD "[page_id={}] [expected=0x{:X}] [actual=0x{:X}] " "[field_index={}] [field_offset={}] [field_size={}] " "[entry={}] [file={}]", +======= + "page_id={} expected=0x{:X} actual=0x{:X} " + "field_index={} field_offset={} field_size={} " + "entry={}", +>>>>>>> dc20fe919f (PageStorage: Fix empty page cause TiFlash failed to start (#9283)) page_id_v3, expect_checksum, field_checksum, field_index, beg_offset, size_to_read, - entry, - blob_file->getPath()); + entry); } } @@ -1008,8 +1012,8 @@ typename BlobStore::PageMap BlobStore::read( for (const auto & [page_id_v3, entry] : entries) { // Unexpected behavior but do no harm - LOG_INFO(log, "Read entry without entry size, page_id={} entry={}", page_id_v3, entry); Page page(Trait::PageIdTrait::getU64ID(page_id_v3)); + page.data = std::string_view(nullptr, 0); page_map.emplace(Trait::PageIdTrait::getPageMapKey(page_id_v3), page); } return page_map; @@ -1022,7 +1026,7 @@ typename BlobStore::PageMap BlobStore::read( PageMap page_map; for (const auto & [page_id_v3, entry] : entries) { - auto blob_file = read(page_id_v3, entry.file_id, entry.offset, pos, entry.size, read_limiter); + read(page_id_v3, entry.file_id, entry.offset, pos, entry.size, read_limiter); if constexpr (BLOBSTORE_CHECKSUM_ON_READ) { @@ -1032,6 +1036,7 @@ typename BlobStore::PageMap BlobStore::read( if (unlikely(entry.size != 0 && checksum != entry.checksum)) { throw Exception( +<<<<<<< HEAD fmt::format( "Reading with entries meet checksum not match [page_id={}] [expected=0x{:X}] [actual=0x{:X}] " "[entry={}] [file={}]", @@ -1041,6 +1046,15 @@ typename BlobStore::PageMap BlobStore::read( entry, blob_file->getPath()), ErrorCodes::CHECKSUM_DOESNT_MATCH); +======= + ErrorCodes::CHECKSUM_DOESNT_MATCH, + "Reading with entries meet checksum not match page_id={} expected=0x{:X} actual=0x{:X} " + "entry={}", + page_id_v3, + entry.checksum, + checksum, + entry); +>>>>>>> dc20fe919f (PageStorage: Fix empty page cause TiFlash failed to start (#9283)) } } @@ -1098,15 +1112,15 @@ Page BlobStore::read(const PageIdAndEntry & id_entry, const ReadLimiterPt if (buf_size == 0) { // Unexpected behavior but do no harm - LOG_INFO(log, "Read entry without entry size, page_id={} entry={}", page_id_v3, entry); Page page(Trait::PageIdTrait::getU64ID(page_id_v3)); + page.data = std::string_view(nullptr, 0); return page; } char * data_buf = static_cast(alloc(buf_size)); MemHolder mem_holder = createMemHolder(data_buf, [&, buf_size](char * p) { free(p, buf_size); }); - auto blob_file = read(page_id_v3, entry.file_id, entry.offset, data_buf, buf_size, read_limiter); + read(page_id_v3, entry.file_id, entry.offset, data_buf, buf_size, read_limiter); if constexpr (BLOBSTORE_CHECKSUM_ON_READ) { ChecksumClass digest; @@ -1115,15 +1129,13 @@ Page BlobStore::read(const PageIdAndEntry & id_entry, const ReadLimiterPt if (unlikely(entry.size != 0 && checksum != entry.checksum)) { throw Exception( - fmt::format( - "Reading with entries meet checksum not match [page_id={}] [expected=0x{:X}] [actual=0x{:X}] " - "[entry={}] [file={}]", - page_id_v3, - entry.checksum, - checksum, - entry, - blob_file->getPath()), - ErrorCodes::CHECKSUM_DOESNT_MATCH); + ErrorCodes::CHECKSUM_DOESNT_MATCH, + "Reading with entries meet checksum not match page_id={} expected=0x{:X} actual=0x{:X} " + "entry={}", + page_id_v3, + entry.checksum, + checksum, + entry); } } @@ -1142,7 +1154,7 @@ Page BlobStore::read(const PageIdAndEntry & id_entry, const ReadLimiterPt } template -BlobFilePtr BlobStore::read( +void BlobStore::read( const typename BlobStore::PageId & page_id_v3, BlobFileId blob_id, BlobFileOffset offset, @@ -1152,6 +1164,12 @@ BlobFilePtr BlobStore::read( bool background) { GET_METRIC(tiflash_storage_page_command_count, type_read_blob).Increment(); + + // A shortcut to avoid unnecessary locks / system call when "reading an empty page" + // Reading an empty page should not create a BlobFile if it has already removed. + if (unlikely(size == 0)) + return; + assert(buffers != nullptr); BlobFilePtr blob_file = getBlobFile(blob_id); try @@ -1170,7 +1188,6 @@ BlobFilePtr BlobStore::read( background)); e.rethrow(); } - return blob_file; } diff --git a/dbms/src/Storages/Page/V3/BlobStore.h b/dbms/src/Storages/Page/V3/BlobStore.h index 6aaf783b172..b887d9165fa 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.h +++ b/dbms/src/Storages/Page/V3/BlobStore.h @@ -126,7 +126,7 @@ class BlobStore : private Allocator PageType page_type, const WriteLimiterPtr & write_limiter = nullptr); - BlobFilePtr read( + void read( const PageId & page_id_v3, BlobFileId blob_id, BlobFileOffset offset, diff --git a/dbms/src/Storages/Page/V3/spacemap/SpaceMap.cpp b/dbms/src/Storages/Page/V3/spacemap/SpaceMap.cpp index a2d52664172..3622e1b3129 100644 --- a/dbms/src/Storages/Page/V3/spacemap/SpaceMap.cpp +++ b/dbms/src/Storages/Page/V3/spacemap/SpaceMap.cpp @@ -17,9 +17,6 @@ #include #include #include -#include -#include -#include namespace DB { @@ -52,14 +49,17 @@ SpaceMapPtr SpaceMap::createSpaceMap(SpaceMapType type, UInt64 start, UInt64 end return smap; } -bool SpaceMap::checkSpace(UInt64 offset, size_t size) const +bool SpaceMap::isInvalidRange(UInt64 offset, size_t size) const { - return (offset < start) || (offset > end) || (offset + size - 1 > end); + return (offset < start) + || (offset > end) + // check whether it can be changed to `(offset + size > end)` + || ((size != 0) && (offset + size - 1 > end)); } bool SpaceMap::markFree(UInt64 offset, size_t length) { - if (checkSpace(offset, length)) + if (isInvalidRange(offset, length)) { throw Exception( fmt::format( @@ -75,7 +75,7 @@ bool SpaceMap::markFree(UInt64 offset, size_t length) bool SpaceMap::markUsed(UInt64 offset, size_t length) { - if (checkSpace(offset, length)) + if (isInvalidRange(offset, length)) { throw Exception( fmt::format( @@ -91,7 +91,7 @@ bool SpaceMap::markUsed(UInt64 offset, size_t length) bool SpaceMap::isMarkUsed(UInt64 offset, size_t length) { - if (checkSpace(offset, length)) + if (isInvalidRange(offset, length)) { throw Exception( fmt::format( diff --git a/dbms/src/Storages/Page/V3/spacemap/SpaceMap.h b/dbms/src/Storages/Page/V3/spacemap/SpaceMap.h index e110dd3d4f8..23e0a0f9160 100644 --- a/dbms/src/Storages/Page/V3/spacemap/SpaceMap.h +++ b/dbms/src/Storages/Page/V3/spacemap/SpaceMap.h @@ -82,7 +82,7 @@ class SpaceMap * If such span is found. * It will mark that span to be used and also return a hint of the max capacity available in this SpaceMap. * - * return value is : + * return value is : * insert_offset: start offset for the inserted space * max_cap: A hint of the largest available space this SpaceMap can hold. * is_expansion: Whether it is an expansion span @@ -145,7 +145,7 @@ class SpaceMap private: /* Check the range */ - bool checkSpace(UInt64 offset, size_t size) const; + bool isInvalidRange(UInt64 offset, size_t size) const; #ifndef DBMS_PUBLIC_GTEST protected: diff --git a/dbms/src/Storages/Page/V3/spacemap/SpaceMapSTDMap.h b/dbms/src/Storages/Page/V3/spacemap/SpaceMapSTDMap.h index d07955e17f9..e0b92cf65ab 100644 --- a/dbms/src/Storages/Page/V3/spacemap/SpaceMapSTDMap.h +++ b/dbms/src/Storages/Page/V3/spacemap/SpaceMapSTDMap.h @@ -20,6 +20,7 @@ #include #include +#include namespace DB { @@ -153,6 +154,11 @@ class STDMapSpaceMap bool markUsedImpl(UInt64 offset, size_t length) override { + // An empty data, we can simply consider it is stored and return true + // Do not let it split the space into smaller pieces. + if (length == 0) + return true; + auto it = MapUtils::findLessEQ(free_map, offset); // first free block <= `offset` if (it == free_map.end()) { @@ -217,8 +223,15 @@ class STDMapSpaceMap return true; } + // return value is std::tuple searchInsertOffset(size_t size) override { + if (unlikely(size == 0)) + { + // The returned `max_cap` is 0 under this case, user should not use it. + return std::make_tuple(0, 0, false); + } + if (unlikely(free_map.empty())) { LOG_ERROR(Logger::get(), "Current space map is full"); @@ -255,24 +268,23 @@ class STDMapSpaceMap bool markFreeImpl(UInt64 offset, size_t length) override { - auto it = free_map.find(offset); + // for an empty blob, no new free block is created, just skip + if (length == 0) + { + return true; + } /** * already unmarked. * The `offset` won't be mid of free space. * Because we alloc space from left to right. */ + auto it = free_map.find(offset); if (it != free_map.end()) { return true; } - // for an empty blob, no new free block is created, just skip - if (length == 0) - { - return true; - } - bool meanless = false; std::tie(it, meanless) = free_map.insert({offset, length}); insertIntoInvertIndex(length, offset); diff --git a/dbms/src/Storages/Page/V3/tests/gtest_blob_stat.cpp b/dbms/src/Storages/Page/V3/tests/gtest_blob_stat.cpp index 4c4492e8eac..d6a6326ee9d 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_blob_stat.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_blob_stat.cpp @@ -165,7 +165,7 @@ try } CATCH -TEST_F(BlobStoreStatsTest, RestoreWithEmptyPage) +TEST_F(BlobStoreStatsTest, RestoreWithEmptyPageSamePosition) try { BlobStats stats(logger, delegator, config); @@ -221,6 +221,54 @@ try } CATCH +TEST_F(BlobStoreStatsTest, RestoreWithEmptyPageSplitSpace) +try +{ + BlobStats stats(logger, delegator, config); + + BlobFileId file_id1 = 11; + + { + std::lock_guard lock(stats.lock_stats); + stats.createStatNotChecking(file_id1, config.file_limit_size, lock); + } + + { + // an entry at offset=0x15376, size=0 + stats.restoreByEntry(PageEntryV3{ + .file_id = file_id1, + .size = 0, + .padded_size = 0, + .tag = 0, + .offset = 0x15376, + .checksum = 0x4567, + }); + // an entry at offset=0x15373, size=15. offset+size > 0x15376, but should be able to insert + stats.restoreByEntry(PageEntryV3{ + .file_id = file_id1, + .size = 15, + .padded_size = 0, + .tag = 0, + .offset = 0x15373, + .checksum = 0x4567, + }); + stats.restore(); + } + + auto stats_copy = stats.getStats(); + + ASSERT_EQ(stats_copy.size(), std::min(getTotalStatsNum(stats_copy), path_num)); + ASSERT_EQ(getTotalStatsNum(stats_copy), 1); + EXPECT_EQ(stats.cur_max_id, file_id1); + + auto stat = stats.blobIdToStat(file_id1); + EXPECT_EQ(stat->sm_total_size, 0x15373 + 15); + EXPECT_EQ(stat->sm_valid_size, 15); + + EXPECT_ANY_THROW({ stats.createStat(file_id1, config.file_limit_size, stats.lock()); }); +} +CATCH + TEST_F(BlobStoreStatsTest, testStats) { BlobStats stats(logger, delegator, config); @@ -343,7 +391,11 @@ TEST_F(BlobStoreStatsTest, StatWithEmptyBlob) ASSERT_EQ(offset, 0); offset = stat->getPosFromStat(0, stat->lock()); // empty - ASSERT_EQ(offset, 10); + ASSERT_EQ(offset, 0); // empty page always "stored" to the beginning of the space + offset = stat->getPosFromStat(0, stat->lock()); // empty + ASSERT_EQ(offset, 0); // empty page always "stored" to the beginning of the space + offset = stat->getPosFromStat(0, stat->lock()); // empty + ASSERT_EQ(offset, 0); // empty page always "stored" to the beginning of the space offset = stat->getPosFromStat(20, stat->lock()); ASSERT_EQ(offset, 10); @@ -351,6 +403,9 @@ TEST_F(BlobStoreStatsTest, StatWithEmptyBlob) offset = stat->getPosFromStat(100, stat->lock()); ASSERT_EQ(offset, 30); + offset = stat->getPosFromStat(0, stat->lock()); // empty + ASSERT_EQ(offset, 0); // empty page always "stored" to the beginning of the space + ASSERT_EQ(stat->sm_total_size, 10 + 20 + 100); ASSERT_EQ(stat->sm_valid_size, 10 + 20 + 100); ASSERT_EQ(stat->sm_valid_rate, 1); @@ -371,9 +426,8 @@ TEST_F(BlobStoreStatsTest, testFullStats) BlobStats stats(logger, delegator, config); { - std::lock_guard lock(stats.lock_stats); BlobFileId file_id = 10; - BlobStats::BlobStatPtr stat = stats.createStat(file_id, config.file_limit_size, lock); + BlobStats::BlobStatPtr stat = stats.createStat(file_id, config.file_limit_size, stats.lock()); auto offset = stat->getPosFromStat(BLOBFILE_LIMIT_SIZE - 1, stat->lock()); ASSERT_EQ(offset, 0); stats.cur_max_id = file_id; diff --git a/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp b/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp index 0b688a96bc4..9d4a9727593 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp @@ -31,9 +31,12 @@ #include #include +#include + namespace DB::FailPoints { extern const char exception_after_large_write_exceed[]; +extern const char force_pick_all_blobs_to_full_gc[]; } // namespace DB::FailPoints namespace DB::PS::V3::tests @@ -1645,7 +1648,7 @@ try page_type_and_config); { - size_t size_500 = 500; + const size_t size_500 = 500; char c_buff[size_500]; WriteBatch wb; @@ -1656,6 +1659,7 @@ try const auto & gc_info = blob_store.getGCStats(); ASSERT_TRUE(gc_info.empty()); + // only one blob_file for the large write ASSERT_EQ(getTotalStatsNum(blob_store.blob_stats.getStats()), 1); const auto & records = edit.getRecords(); ASSERT_EQ(records.size(), 1); @@ -1665,6 +1669,58 @@ try } CATCH +TEST_F(BlobStoreTest, ReadEmptyPageAfterAllValidEntriesRmoved) +try +{ + const auto file_provider = DB::tests::TiFlashTestEnv::getMockFileProvider(); + + BlobConfig config_with_small_file_limit_size; + config_with_small_file_limit_size.file_limit_size = 400; + auto blob_store = BlobStore( + getCurrentTestName(), + file_provider, + delegator, + config_with_small_file_limit_size, + page_type_and_config); + + { + const size_t size_500 = 500; + char c_buff[size_500]; + + WriteBatch wb; + ReadBufferPtr buff = std::make_shared(c_buff, size_500); + wb.putPage(50, /* tag */ 0, buff, size_500); + wb.putPage(51, 0, std::make_shared(c_buff, 0), 0); + wb.putPage(52, 0, std::make_shared(c_buff, 20), 20); + wb.putPage(53, 0, std::make_shared(c_buff, 0), 0); + PageEntriesEdit edit = blob_store.write(std::move(wb)); + + // make the blob_file to be READ_ONLY + FailPointHelper::enableFailPoint(FailPoints::force_pick_all_blobs_to_full_gc); + SCOPE_EXIT({ FailPointHelper::disableFailPoint(FailPoints::force_pick_all_blobs_to_full_gc); }); + const auto & gc_info = blob_store.getGCStats(); + ASSERT_FALSE(gc_info.empty()); + + // one blob_file for the large write, another for the other pages + ASSERT_EQ(getTotalStatsNum(blob_store.blob_stats.getStats()), 2); + const auto & records = edit.getRecords(); + ASSERT_EQ(records.size(), 4); + // remove the non-empty entries + blob_store.removeEntries({records[0].entry, records[2].entry}); + + /// try to read the empty pages after the blob_file get removed, no exception should happen + // read-1-entry + blob_store.read(PageIDAndEntryV3(51, records[1].entry), nullptr); + blob_store.read(PageIDAndEntryV3(53, records[3].entry), nullptr); + // read multiple entry + PageIDAndEntriesV3 entries{PageIDAndEntryV3(51, records[1].entry), PageIDAndEntryV3(53, records[3].entry)}; + blob_store.read(entries, nullptr); + + ASSERT_EQ(getTotalStatsNum(blob_store.blob_stats.getStats()), 0); + } +} +CATCH + TEST_F(BlobStoreTest, LargeWriteRegisterPath) try { diff --git a/dbms/src/Storages/Page/V3/tests/gtest_free_map.cpp b/dbms/src/Storages/Page/V3/tests/gtest_free_map.cpp index 4c7d4bd89df..8ff108285b9 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_free_map.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_free_map.cpp @@ -43,7 +43,7 @@ class SpaceMapTest : public testing::TestWithParam return [ranges, range_size](size_t idx, UInt64 start, UInt64 end) -> bool { return idx < range_size && ranges[idx].start == start && ranges[idx].end == end; }; - }; + } }; TEST_P(SpaceMapTest, InitAndDestory) @@ -60,6 +60,13 @@ TEST_P(SpaceMapTest, MarkUnmark) Range ranges[] = {{.start = 0, .end = 100}}; ASSERT_TRUE(smap->check(genChecker(ranges, 1), 1)); + ASSERT_TRUE(smap->markUsed(0, 100)); + ASSERT_TRUE(smap->markUsed(0, 0)); + ASSERT_TRUE(smap->markUsed(100, 0)); + ASSERT_TRUE(smap->markFree(0, 100)); + // Now off-by-1 will return false but no exception + ASSERT_FALSE(smap->markUsed(0, 101)); + ASSERT_TRUE(smap->markUsed(50, 1)); ASSERT_FALSE(smap->markUsed(50, 1)); @@ -440,6 +447,23 @@ TEST_P(SpaceMapTest, EmptyBlob) ASSERT_EQ(sizes.second, 10); } +TEST_P(SpaceMapTest, Fragmentation) +{ + auto smap = SpaceMap::createSpaceMap(SpaceMap::SMAP64_STD_MAP, 0, 100); + // add an empty page + ASSERT_TRUE(smap->markUsed(60, 0)); + auto sizes = smap->getSizes(); + ASSERT_EQ(sizes.first, 0); + ASSERT_EQ(sizes.second, 0); + ASSERT_EQ(smap->getUsedBoundary(), 0); // used boundary won't contain the empty page + + // add [50, 70), should success + ASSERT_TRUE(smap->markUsed(50, 20)); + sizes = smap->getSizes(); + ASSERT_EQ(sizes.first, 70); + ASSERT_EQ(sizes.second, 20); + ASSERT_EQ(smap->getUsedBoundary(), 70); +} INSTANTIATE_TEST_CASE_P(Type, SpaceMapTest, testing::Values(SpaceMap::SMAP64_STD_MAP)); diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp index e910307ffea..1a835d376a0 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp @@ -1120,6 +1120,53 @@ try } CATCH +TEST_F(PageDirectoryTest, EmptyPage) +try +{ + { + PageEntriesEdit edit; + edit.put(buildV3Id(TEST_NAMESPACE_ID, 9), PageEntryV3{.file_id = 551, .size = 0, .offset = 0x15376}); + edit.put(buildV3Id(TEST_NAMESPACE_ID, 10), PageEntryV3{.file_id = 551, .size = 15, .offset = 0x15373}); + edit.put(buildV3Id(TEST_NAMESPACE_ID, 100), PageEntryV3{.file_id = 551, .size = 0, .offset = 0x0}); + edit.put( + buildV3Id(TEST_NAMESPACE_ID, 101), + PageEntryV3{.file_id = 552, .size = BLOBFILE_LIMIT_SIZE, .offset = 0x0}); + dir->apply(std::move(edit)); + } + + auto s0 = dir->createSnapshot(); + auto edit = dir->dumpSnapshotToEdit(s0); + auto restore_from_edit = [](const PageEntriesEdit & edit, BlobStats & stats) { + auto deseri_edit = u128::Serializer::deserializeFrom(u128::Serializer::serializeTo(edit), nullptr); + auto provider = DB::tests::TiFlashTestEnv::getDefaultFileProvider(); + auto path = getTemporaryPath(); + PSDiskDelegatorPtr delegator = std::make_shared(path); + PageDirectoryFactory factory; + auto d + = factory.setBlobStats(stats).createFromEditForTest(getCurrentTestName(), provider, delegator, deseri_edit); + return d; + }; + + { + auto path = getTemporaryPath(); + PSDiskDelegatorPtr delegator = std::make_shared(path); + auto config = BlobConfig{}; + BlobStats stats(log, delegator, config); + { + std::lock_guard lock(stats.lock_stats); + stats.createStatNotChecking(551, BLOBFILE_LIMIT_SIZE, lock); + stats.createStatNotChecking(552, BLOBFILE_LIMIT_SIZE, lock); + } + auto restored_dir = restore_from_edit(edit, stats); + auto snap = restored_dir->createSnapshot(); + ASSERT_EQ(getEntry(restored_dir, 9, snap).offset, 0x15376); + ASSERT_EQ(getEntry(restored_dir, 10, snap).offset, 0x15373); + ASSERT_EQ(getEntry(restored_dir, 100, snap).offset, 0x0); + ASSERT_EQ(getEntry(restored_dir, 101, snap).offset, 0x0); + } +} +CATCH + class PageDirectoryGCTest : public PageDirectoryTest { }; diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp index ab4b517fe1e..26f2cc28434 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp @@ -1950,6 +1950,7 @@ try } FailPointHelper::enableFailPoint(FailPoints::force_pick_all_blobs_to_full_gc); + SCOPE_EXIT({ FailPointHelper::disableFailPoint(FailPoints::force_pick_all_blobs_to_full_gc); }); auto done_full_gc = page_storage->gc(); EXPECT_TRUE(done_full_gc);