From 706d7c32811f9444507cadfb1bec92aeb421504b Mon Sep 17 00:00:00 2001 From: JaySon Date: Fri, 18 Mar 2022 13:46:30 +0800 Subject: [PATCH 1/9] Tracing: Add tracing_id to PageStorage snapshot (#4330) ref pingcap/tiflash#4287 --- .../DeltaMerge/Delta/DeltaValueSpace.cpp | 2 +- .../DeltaMerge/Delta/DeltaValueSpace.h | 8 +-- .../Storages/DeltaMerge/Delta/Snapshot.cpp | 3 +- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 38 +++++----- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 8 ++- dbms/src/Storages/DeltaMerge/StoragePool.h | 20 +++--- .../tests/gtest_dm_delta_value_space.cpp | 4 +- .../tests/gtest_dm_minmax_index.cpp | 1 + .../DeltaMerge/tools/workload/DTWorkload.cpp | 1 + dbms/src/Storages/Page/PageStorage.h | 9 +-- dbms/src/Storages/Page/Snapshot.h | 9 +++ dbms/src/Storages/Page/V2/PageStorage.cpp | 27 ++++---- dbms/src/Storages/Page/V2/PageStorage.h | 5 +- .../PageEntriesVersionSetWithDelta.cpp | 69 +++++++++++-------- .../PageEntriesVersionSetWithDelta.h | 15 ++-- .../Storages/Page/V2/gc/LegacyCompactor.cpp | 2 +- dbms/src/Storages/Page/V3/PageDirectory.cpp | 23 +++---- dbms/src/Storages/Page/V3/PageDirectory.h | 18 +++-- dbms/src/Storages/Page/V3/PageStorageImpl.cpp | 20 +++--- dbms/src/Storages/Page/V3/PageStorageImpl.h | 6 +- .../src/Storages/Page/stress/PSBackground.cpp | 19 +++-- dbms/src/Storages/Page/stress/PSRunnable.cpp | 2 +- 22 files changed, 174 insertions(+), 135 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp index 05a371f97a0..b57ce3cb50a 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp @@ -236,7 +236,7 @@ bool DeltaValueSpace::compact(DMContext & context) LOG_FMT_DEBUG(log, "{} Nothing to compact", simpleInfo()); return true; } - log_storage_snap = context.storage_pool.log()->getSnapshot(); + log_storage_snap = context.storage_pool.log()->getSnapshot(/*tracing_id*/ fmt::format("minor_compact_{}", simpleInfo())); } // do compaction task diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h index ec413eac3a6..f1f4d141953 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h @@ -242,7 +242,8 @@ class DeltaValueSpace : public std::enable_shared_from_this DeltaSnapshotPtr createSnapshot(const DMContext & context, bool for_update, CurrentMetrics::Metric type); }; -class DeltaValueSnapshot : public std::enable_shared_from_this +class DeltaValueSnapshot + : public std::enable_shared_from_this , private boost::noncopyable { friend class DeltaValueSpace; @@ -260,7 +261,7 @@ class DeltaValueSnapshot : public std::enable_shared_from_thisgetStorageSnapshot(); } const auto & getSharedDeltaIndex() { return shared_delta_index; } }; diff --git a/dbms/src/Storages/DeltaMerge/Delta/Snapshot.cpp b/dbms/src/Storages/DeltaMerge/Delta/Snapshot.cpp index 752a68c7844..af1fdfec94f 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/Snapshot.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/Snapshot.cpp @@ -38,7 +38,8 @@ DeltaSnapshotPtr DeltaValueSpace::createSnapshot(const DMContext & context, bool snap->is_update = for_update; snap->_delta = this->shared_from_this(); - auto storage_snap = std::make_shared(context.storage_pool, context.getReadLimiter(), true); + // TODO: Add tracing_id from mpp task or background tasks + auto storage_snap = std::make_shared(context.storage_pool, context.getReadLimiter(), /*tracing_id*/ "", true); snap->persisted_files_snap = persisted_file_set->createSnapshot(storage_snap); snap->shared_delta_index = delta_index; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index b163821d22d..719b63a3527 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -2260,12 +2261,14 @@ DeltaMergeStoreStat DeltaMergeStore::getStat() stat.avg_pack_rows_in_stable = static_cast(stat.total_stable_rows) / stat.total_pack_count_in_stable; stat.avg_pack_size_in_stable = static_cast(stat.total_stable_size) / stat.total_pack_count_in_stable; + static const String useless_tracing_id("DeltaMergeStore::getStat"); { - std::tie(stat.storage_stable_num_snapshots, // - stat.storage_stable_oldest_snapshot_lifetime, - stat.storage_stable_oldest_snapshot_thread_id) - = storage_pool.data()->getSnapshotsStat(); - PageStorage::SnapshotPtr stable_snapshot = storage_pool.data()->getSnapshot(); + auto snaps_stat = storage_pool.data()->getSnapshotsStat(); + stat.storage_stable_num_snapshots = snaps_stat.num_snapshots; + stat.storage_stable_oldest_snapshot_lifetime = snaps_stat.longest_living_seconds; + stat.storage_stable_oldest_snapshot_thread_id = snaps_stat.longest_living_from_thread_id; + stat.storage_stable_oldest_snapshot_tracing_id = snaps_stat.longest_living_from_tracing_id; + PageStorage::SnapshotPtr stable_snapshot = storage_pool.data()->getSnapshot(useless_tracing_id); const auto * concrete_snap = toConcreteSnapshot(stable_snapshot); if (const auto * const version = concrete_snap->version(); version != nullptr) { @@ -2279,11 +2282,12 @@ DeltaMergeStoreStat DeltaMergeStore::getStat() } } { - std::tie(stat.storage_delta_num_snapshots, // - stat.storage_delta_oldest_snapshot_lifetime, - stat.storage_delta_oldest_snapshot_thread_id) - = storage_pool.log()->getSnapshotsStat(); - PageStorage::SnapshotPtr log_snapshot = storage_pool.log()->getSnapshot(); + auto snaps_stat = storage_pool.log()->getSnapshotsStat(); + stat.storage_delta_num_snapshots = snaps_stat.num_snapshots; + stat.storage_delta_oldest_snapshot_lifetime = snaps_stat.longest_living_seconds; + stat.storage_delta_oldest_snapshot_thread_id = snaps_stat.longest_living_from_thread_id; + stat.storage_delta_oldest_snapshot_tracing_id = snaps_stat.longest_living_from_tracing_id; + PageStorage::SnapshotPtr log_snapshot = storage_pool.log()->getSnapshot(useless_tracing_id); const auto * concrete_snap = toConcreteSnapshot(log_snapshot); if (const auto * const version = concrete_snap->version(); version != nullptr) { @@ -2297,11 +2301,12 @@ DeltaMergeStoreStat DeltaMergeStore::getStat() } } { - std::tie(stat.storage_meta_num_snapshots, // - stat.storage_meta_oldest_snapshot_lifetime, - stat.storage_meta_oldest_snapshot_thread_id) - = storage_pool.meta()->getSnapshotsStat(); - PageStorage::SnapshotPtr meta_snapshot = storage_pool.meta()->getSnapshot(); + auto snaps_stat = storage_pool.meta()->getSnapshotsStat(); + stat.storage_meta_num_snapshots = snaps_stat.num_snapshots; + stat.storage_meta_oldest_snapshot_lifetime = snaps_stat.longest_living_seconds; + stat.storage_meta_oldest_snapshot_thread_id = snaps_stat.longest_living_from_thread_id; + stat.storage_meta_oldest_snapshot_tracing_id = snaps_stat.longest_living_from_tracing_id; + PageStorage::SnapshotPtr meta_snapshot = storage_pool.meta()->getSnapshot(useless_tracing_id); const auto * concrete_snap = toConcreteSnapshot(meta_snapshot); if (const auto * const version = concrete_snap->version(); version != nullptr) { @@ -2431,8 +2436,7 @@ SegmentReadTasks DeltaMergeStore::getReadTasksByRanges( LOG_FMT_DEBUG( log, - "{} [sorted_ranges: {}] [tasks before split: {}] [tasks final: {}] [ranges final: {}]", - __FUNCTION__, + "[sorted_ranges: {}] [tasks before split: {}] [tasks final: {}] [ranges final: {}]", sorted_ranges.size(), tasks.size(), result_tasks.size(), diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index d12f85836f1..5877704412a 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -32,6 +32,9 @@ namespace DB { +class LogWithPrefix; +using LogWithPrefixPtr = std::shared_ptr; + namespace DM { class Segment; @@ -118,6 +121,7 @@ struct DeltaMergeStoreStat UInt64 storage_stable_num_snapshots = 0; Float64 storage_stable_oldest_snapshot_lifetime = 0.0; UInt64 storage_stable_oldest_snapshot_thread_id = 0; + String storage_stable_oldest_snapshot_tracing_id; UInt64 storage_stable_num_pages = 0; UInt64 storage_stable_num_normal_pages = 0; UInt64 storage_stable_max_page_id = 0; @@ -125,6 +129,7 @@ struct DeltaMergeStoreStat UInt64 storage_delta_num_snapshots = 0; Float64 storage_delta_oldest_snapshot_lifetime = 0.0; UInt64 storage_delta_oldest_snapshot_thread_id = 0; + String storage_delta_oldest_snapshot_tracing_id; UInt64 storage_delta_num_pages = 0; UInt64 storage_delta_num_normal_pages = 0; UInt64 storage_delta_max_page_id = 0; @@ -132,6 +137,7 @@ struct DeltaMergeStoreStat UInt64 storage_meta_num_snapshots = 0; Float64 storage_meta_oldest_snapshot_lifetime = 0.0; UInt64 storage_meta_oldest_snapshot_thread_id = 0; + String storage_meta_oldest_snapshot_tracing_id; UInt64 storage_meta_num_pages = 0; UInt64 storage_meta_num_normal_pages = 0; UInt64 storage_meta_max_page_id = 0; @@ -369,7 +375,7 @@ class DeltaMergeStore : private boost::noncopyable /// Do merge delta for all segments. Only used for debug. void mergeDeltaAll(const Context & context); - /// Compact fregment packs into bigger one. + /// Compact fragment column files into bigger one. void compact(const Context & context, const RowKeyRange & range); /// Iterator over all segments and apply gc jobs. diff --git a/dbms/src/Storages/DeltaMerge/StoragePool.h b/dbms/src/Storages/DeltaMerge/StoragePool.h index 44b757d3897..5efc87840da 100644 --- a/dbms/src/Storages/DeltaMerge/StoragePool.h +++ b/dbms/src/Storages/DeltaMerge/StoragePool.h @@ -52,17 +52,17 @@ class StoragePool : private boost::noncopyable PageReader & dataReader() { return data_storage_reader; } PageReader & metaReader() { return meta_storage_reader; } - PageReader newLogReader(ReadLimiterPtr read_limiter, bool snapshot_read) + PageReader newLogReader(ReadLimiterPtr read_limiter, bool snapshot_read, const String & tracing_id) { - return PageReader(ns_id, log_storage, snapshot_read ? log_storage->getSnapshot() : nullptr, read_limiter); + return PageReader(ns_id, log_storage, snapshot_read ? log_storage->getSnapshot(tracing_id) : nullptr, read_limiter); } - PageReader newDataReader(ReadLimiterPtr read_limiter, bool snapshot_read) + PageReader newDataReader(ReadLimiterPtr read_limiter, bool snapshot_read, const String & tracing_id) { - return PageReader(ns_id, data_storage, snapshot_read ? data_storage->getSnapshot() : nullptr, read_limiter); + return PageReader(ns_id, data_storage, snapshot_read ? data_storage->getSnapshot(tracing_id) : nullptr, read_limiter); } - PageReader newMetaReader(ReadLimiterPtr read_limiter, bool snapshot_read) + PageReader newMetaReader(ReadLimiterPtr read_limiter, bool snapshot_read, const String & tracing_id) { - return PageReader(ns_id, meta_storage, snapshot_read ? meta_storage->getSnapshot() : nullptr, read_limiter); + return PageReader(ns_id, meta_storage, snapshot_read ? meta_storage->getSnapshot(tracing_id) : nullptr, read_limiter); } // Caller must cancel gc tasks before drop @@ -112,10 +112,10 @@ class PageIdGenerator : private boost::noncopyable struct StorageSnapshot : private boost::noncopyable { - StorageSnapshot(StoragePool & storage, ReadLimiterPtr read_limiter, bool snapshot_read = true) - : log_reader(storage.newLogReader(read_limiter, snapshot_read)) - , data_reader(storage.newDataReader(read_limiter, snapshot_read)) - , meta_reader(storage.newMetaReader(read_limiter, snapshot_read)) + StorageSnapshot(StoragePool & storage, ReadLimiterPtr read_limiter, const String & tracing_id, bool snapshot_read) + : log_reader(storage.newLogReader(read_limiter, snapshot_read, tracing_id)) + , data_reader(storage.newDataReader(read_limiter, snapshot_read, tracing_id)) + , meta_reader(storage.newMetaReader(read_limiter, snapshot_read, tracing_id)) {} PageReader log_reader; diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp index 6ce4eecf898..3aec2dea15f 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp @@ -344,7 +344,7 @@ TEST_F(DeltaValueSpaceTest, MinorCompaction) // build compaction task and finish prepare stage MinorCompactionPtr compaction_task; { - PageReader reader = dmContext().storage_pool.newLogReader(dmContext().getReadLimiter(), true); + PageReader reader = dmContext().storage_pool.newLogReader(dmContext().getReadLimiter(), true, ""); compaction_task = persisted_file_set->pickUpMinorCompaction(dmContext()); ASSERT_EQ(compaction_task->getCompactionSourceLevel(), 0); // There should be two compaction sub_tasks. @@ -392,7 +392,7 @@ TEST_F(DeltaValueSpaceTest, MinorCompaction) delta->flush(dmContext()); while (true) { - PageReader reader = dmContext().storage_pool.newLogReader(dmContext().getReadLimiter(), true); + PageReader reader = dmContext().storage_pool.newLogReader(dmContext().getReadLimiter(), true, ""); auto minor_compaction_task = persisted_file_set->pickUpMinorCompaction(dmContext()); if (!minor_compaction_task) break; diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp index 82f0735939b..11c5cf8233e 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp b/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp index 9f1ac768f41..0fe42277a99 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp +++ b/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include diff --git a/dbms/src/Storages/Page/PageStorage.h b/dbms/src/Storages/Page/PageStorage.h index 979e0ffab7a..4521ecb3140 100644 --- a/dbms/src/Storages/Page/PageStorage.h +++ b/dbms/src/Storages/Page/PageStorage.h @@ -173,13 +173,10 @@ class PageStorage : private boost::noncopyable virtual PageId getMaxId(NamespaceId ns_id) = 0; - virtual SnapshotPtr getSnapshot() = 0; + virtual SnapshotPtr getSnapshot(const String & tracing_id) = 0; // Get some statistics of all living snapshots and the oldest living snapshot. - // Return < num of snapshots, - // living time(seconds) of the oldest snapshot, - // created thread id of the oldest snapshot > - virtual std::tuple getSnapshotsStat() const = 0; + virtual SnapshotsStatistics getSnapshotsStat() const = 0; virtual void write(WriteBatch && write_batch, const WriteLimiterPtr & write_limiter = nullptr) = 0; @@ -222,9 +219,7 @@ class PageReader : private boost::noncopyable explicit PageReader(NamespaceId ns_id_, PageStoragePtr storage_, ReadLimiterPtr read_limiter_) : ns_id(ns_id_) , storage(storage_) - , snap() , read_limiter(read_limiter_) - {} /// Snapshot read. PageReader(NamespaceId ns_id_, PageStoragePtr storage_, const PageStorage::SnapshotPtr & snap_, ReadLimiterPtr read_limiter_) diff --git a/dbms/src/Storages/Page/Snapshot.h b/dbms/src/Storages/Page/Snapshot.h index 2edeb8dfbd5..348d084382f 100644 --- a/dbms/src/Storages/Page/Snapshot.h +++ b/dbms/src/Storages/Page/Snapshot.h @@ -13,10 +13,19 @@ // limitations under the License. #pragma once +#include + #include namespace DB { +struct SnapshotsStatistics +{ + size_t num_snapshots = 0; + double longest_living_seconds = 0.0; + unsigned longest_living_from_thread_id = 0; + String longest_living_from_tracing_id; +}; class PageStorageSnapshot { public: diff --git a/dbms/src/Storages/Page/V2/PageStorage.cpp b/dbms/src/Storages/Page/V2/PageStorage.cpp index 6cf454fcf27..2a61e7ec442 100644 --- a/dbms/src/Storages/Page/V2/PageStorage.cpp +++ b/dbms/src/Storages/Page/V2/PageStorage.cpp @@ -358,14 +358,14 @@ void PageStorage::restore() PageId PageStorage::getMaxId(NamespaceId /*ns_id*/) { std::lock_guard write_lock(write_mutex); - return versioned_page_entries.getSnapshot()->version()->maxId(); + return versioned_page_entries.getSnapshot("")->version()->maxId(); } PageId PageStorage::getNormalPageId(NamespaceId /*ns_id*/, PageId page_id, SnapshotPtr snapshot) { if (!snapshot) { - snapshot = this->getSnapshot(); + snapshot = this->getSnapshot(""); } auto [is_ref_id, normal_page_id] = toConcreteSnapshot(snapshot)->version()->isRefId(page_id); @@ -376,7 +376,7 @@ DB::PageEntry PageStorage::getEntry(NamespaceId /*ns_id*/, PageId page_id, Snaps { if (!snapshot) { - snapshot = this->getSnapshot(); + snapshot = this->getSnapshot(""); } try @@ -576,18 +576,18 @@ void PageStorage::write(DB::WriteBatch && wb, const WriteLimiterPtr & write_limi } } -DB::PageStorage::SnapshotPtr PageStorage::getSnapshot() +DB::PageStorage::SnapshotPtr PageStorage::getSnapshot(const String & tracing_id) { - return versioned_page_entries.getSnapshot(); + return versioned_page_entries.getSnapshot(tracing_id); } PageStorage::VersionedPageEntries::SnapshotPtr PageStorage::getConcreteSnapshot() { - return versioned_page_entries.getSnapshot(); + return versioned_page_entries.getSnapshot(/*tracing_id*/ ""); } -std::tuple PageStorage::getSnapshotsStat() const +SnapshotsStatistics PageStorage::getSnapshotsStat() const { return versioned_page_entries.getSnapshotsStat(); } @@ -596,7 +596,7 @@ DB::Page PageStorage::read(NamespaceId /*ns_id*/, PageId page_id, const ReadLimi { if (!snapshot) { - snapshot = this->getSnapshot(); + snapshot = this->getSnapshot(""); } const auto page_entry = toConcreteSnapshot(snapshot)->version()->find(page_id); @@ -612,7 +612,7 @@ PageMap PageStorage::read(NamespaceId /*ns_id*/, const std::vector & pag { if (!snapshot) { - snapshot = this->getSnapshot(); + snapshot = this->getSnapshot(""); } std::map> file_read_infos; @@ -655,7 +655,7 @@ void PageStorage::read(NamespaceId /*ns_id*/, const std::vector & page_i { if (!snapshot) { - snapshot = this->getSnapshot(); + snapshot = this->getSnapshot(""); } std::map> file_read_infos; @@ -694,8 +694,9 @@ void PageStorage::read(NamespaceId /*ns_id*/, const std::vector & page_i PageMap PageStorage::read(NamespaceId /*ns_id*/, const std::vector & page_fields, const ReadLimiterPtr & read_limiter, SnapshotPtr snapshot) { if (!snapshot) - snapshot = this->getSnapshot(); - + { + snapshot = this->getSnapshot(""); + } std::map> file_read_infos; for (const auto & [page_id, field_indices] : page_fields) @@ -737,7 +738,7 @@ void PageStorage::traverse(const std::function & ac { if (!snapshot) { - snapshot = this->getSnapshot(); + snapshot = this->getSnapshot(""); } std::map file_and_pages; diff --git a/dbms/src/Storages/Page/V2/PageStorage.h b/dbms/src/Storages/Page/V2/PageStorage.h index 9f0effeeff8..8163456e907 100644 --- a/dbms/src/Storages/Page/V2/PageStorage.h +++ b/dbms/src/Storages/Page/V2/PageStorage.h @@ -99,13 +99,13 @@ class PageStorage : public DB::PageStorage PageId getNormalPageId(NamespaceId ns_id, PageId page_id, SnapshotPtr snapshot) override; - DB::PageStorage::SnapshotPtr getSnapshot() override; + DB::PageStorage::SnapshotPtr getSnapshot(const String & tracing_id) override; using ConcreteSnapshotRawPtr = VersionedPageEntries::Snapshot *; using ConcreteSnapshotPtr = VersionedPageEntries::SnapshotPtr; ConcreteSnapshotPtr getConcreteSnapshot(); - std::tuple getSnapshotsStat() const override; + SnapshotsStatistics getSnapshotsStat() const override; void write(DB::WriteBatch && wb, const WriteLimiterPtr & write_limiter) override; @@ -193,6 +193,7 @@ class PageStorage : public DB::PageStorage #ifndef NDEBUG // Just for tests, refactor them out later + DB::PageStorage::SnapshotPtr getSnapshot() { return getSnapshot(""); } void write(DB::WriteBatch && wb) { return write(std::move(wb), nullptr); } DB::PageEntry getEntry(PageId page_id) { return getEntry(TEST_NAMESPACE_ID, page_id, nullptr); } DB::PageEntry getEntry(PageId page_id, SnapshotPtr snapshot) { return getEntry(TEST_NAMESPACE_ID, page_id, snapshot); }; diff --git a/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.cpp b/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.cpp index f4963f8466e..b71e9da339d 100644 --- a/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.cpp +++ b/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.cpp @@ -15,6 +15,8 @@ #include #include #include +#include +#include #include @@ -99,19 +101,19 @@ size_t PageEntriesVersionSetWithDelta::sizeUnlocked() const return sz; } -std::tuple PageEntriesVersionSetWithDelta::getSnapshotsStat() const +SnapshotsStatistics PageEntriesVersionSetWithDelta::getSnapshotsStat() const { // Note: this will scan and remove expired weak_ptrs from `snapshots` return removeExpiredSnapshots(); } -PageEntriesVersionSetWithDelta::SnapshotPtr PageEntriesVersionSetWithDelta::getSnapshot() +PageEntriesVersionSetWithDelta::SnapshotPtr PageEntriesVersionSetWithDelta::getSnapshot(const String & tracing_id) { // acquire for unique_lock since we need to add all snapshots to link list std::unique_lock lock(read_write_mutex); - auto s = std::make_shared(this, current); + auto s = std::make_shared(this, current, tracing_id); // Register a weak_ptr to snapshot into VersionSet so that we can get all living PageFiles // by `PageEntriesVersionSetWithDelta::listAllLiveFiles`, and it remove useless weak_ptr of snapshots. // Do not call `vset->removeExpiredSnapshots` inside `~Snapshot`, or it may cause incursive deadlock @@ -220,13 +222,12 @@ void PageEntriesVersionSetWithDelta::compactOnDeltaRelease(VersionPtr tail) } } -std::tuple PageEntriesVersionSetWithDelta::removeExpiredSnapshots() const +SnapshotsStatistics PageEntriesVersionSetWithDelta::removeExpiredSnapshots() const { // Notice: we should free those valid snapshots without locking, or it may cause // incursive deadlock on `vset->read_write_mutex`. std::vector valid_snapshots; - double longest_living_seconds = 0.0; - unsigned longest_living_from_thread_id = 0; + SnapshotsStatistics stats; DB::Int64 num_snapshots_removed = 0; { std::unique_lock lock(read_write_mutex); @@ -247,10 +248,11 @@ std::tuple PageEntriesVersionSetWithDelta::removeExpir std::this_thread::sleep_for(ms); }); const auto snapshot_lifetime = snapshot_or_invalid->elapsedSeconds(); - if (snapshot_lifetime > longest_living_seconds) + if (snapshot_lifetime > stats.longest_living_seconds) { - longest_living_seconds = snapshot_lifetime; - longest_living_from_thread_id = snapshot_or_invalid->t_id; + stats.longest_living_seconds = snapshot_lifetime; + stats.longest_living_from_thread_id = snapshot_or_invalid->create_thread; + stats.longest_living_from_tracing_id = snapshot_or_invalid->tracing_id; } valid_snapshots.emplace_back(snapshot_or_invalid); // Save valid snapshot and release them without lock later iter++; @@ -258,12 +260,12 @@ std::tuple PageEntriesVersionSetWithDelta::removeExpir } } // unlock `read_write_mutex` - const size_t num_valid_snapshots = valid_snapshots.size(); + stats.num_snapshots = valid_snapshots.size(); valid_snapshots.clear(); CurrentMetrics::sub(CurrentMetrics::PSMVCCSnapshotsList, num_snapshots_removed); // Return some statistics of the oldest living snapshot. - return {num_valid_snapshots, longest_living_seconds, longest_living_from_thread_id}; + return stats; } @@ -298,12 +300,13 @@ PageEntriesVersionSetWithDelta::gcApply( // std::pair, std::set> PageEntriesVersionSetWithDelta::listAllLiveFiles(std::unique_lock && lock, bool need_scan_page_ids) { + constexpr const double exist_stale_snapshot = 60.0; + /// Collect live files is costly, we save SnapshotPtrs and scan them without lock. // Note read_write_mutex must be hold. std::vector valid_snapshots; const size_t snapshots_size_before_clean = snapshots.size(); - double longest_living_seconds = 0.0; - unsigned longest_living_from_thread_id = 0; + SnapshotsStatistics stats; for (auto iter = snapshots.begin(); iter != snapshots.end(); /* empty */) { auto snapshot_or_invalid = iter->lock(); @@ -320,35 +323,47 @@ PageEntriesVersionSetWithDelta::listAllLiveFiles(std::unique_lockelapsedSeconds(); - if (snapshot_lifetime > longest_living_seconds) + if (snapshot_lifetime > stats.longest_living_seconds) + { + stats.longest_living_seconds = snapshot_lifetime; + stats.longest_living_from_thread_id = snapshot_or_invalid->create_thread; + stats.longest_living_from_tracing_id = snapshot_or_invalid->tracing_id; + } + if (snapshot_lifetime > exist_stale_snapshot) { - longest_living_seconds = snapshot_lifetime; - longest_living_from_thread_id = snapshot_or_invalid->t_id; + LOG_FMT_WARNING( + log, + "Suspicious stale snapshot detected lifetime {:.3f} seconds, created from thread_id {}, tracing_id {}", + snapshot_lifetime, + snapshot_or_invalid->create_thread, + snapshot_or_invalid->tracing_id); } valid_snapshots.emplace_back(snapshot_or_invalid); // Save valid snapshot and release them without lock later iter++; } } // Create a temporary latest snapshot by using `current` - valid_snapshots.emplace_back(std::make_shared(this, current)); + valid_snapshots.emplace_back(std::make_shared(this, current, "")); lock.unlock(); // Notice: unlock and we should free those valid snapshots without locking + stats.num_snapshots = valid_snapshots.size(); // Plus 1 for eliminating the counting of temporary snapshot of `current` const size_t num_invalid_snapshot_to_clean = snapshots_size_before_clean + 1 - valid_snapshots.size(); if (num_invalid_snapshot_to_clean > 0) { CurrentMetrics::sub(CurrentMetrics::PSMVCCSnapshotsList, num_invalid_snapshot_to_clean); -#define STALE_SNAPSHOT_LOG_PARAMS \ - "{} gcApply remove {} invalid snapshots, " \ - "{} snapshots left, longest lifetime {:.3f} seconds, created from thread_id {}", \ - name, \ - num_invalid_snapshot_to_clean, \ - valid_snapshots.size(), \ - longest_living_seconds, \ - longest_living_from_thread_id - constexpr const double exist_stale_snapshot = 60.0; - if (longest_living_seconds > exist_stale_snapshot) +#define STALE_SNAPSHOT_LOG_PARAMS \ + "{} gcApply remove {} invalid snapshots, " \ + "{} snapshots left, longest lifetime {:.3f} seconds, " \ + "created from thread_id {}, tracing_id {}", \ + name, \ + num_invalid_snapshot_to_clean, \ + stats.num_snapshots, \ + stats.longest_living_seconds, \ + stats.longest_living_from_thread_id, \ + stats.longest_living_from_tracing_id + if (stats.longest_living_seconds > exist_stale_snapshot) LOG_FMT_WARNING(log, STALE_SNAPSHOT_LOG_PARAMS); else LOG_FMT_DEBUG(log, STALE_SNAPSHOT_LOG_PARAMS); diff --git a/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.h b/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.h index 9162879f3bd..bc83c6fca25 100644 --- a/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.h +++ b/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.h @@ -55,7 +55,6 @@ class PageEntriesVersionSetWithDelta public: explicit PageEntriesVersionSetWithDelta(String name_, const MVCC::VersionSetConfig & config_, Poco::Logger * log_) : current(VersionType::createBase()) - , snapshots() , config(config_) , name(std::move(name_)) , log(log_) @@ -78,7 +77,7 @@ class PageEntriesVersionSetWithDelta size_t sizeUnlocked() const; - std::tuple getSnapshotsStat() const; + SnapshotsStatistics getSnapshotsStat() const; std::string toDebugString() const { @@ -121,16 +120,18 @@ class PageEntriesVersionSetWithDelta PageEntriesView view; using TimePoint = std::chrono::time_point; - const unsigned t_id; + const unsigned create_thread; + const String tracing_id; private: const TimePoint create_time; public: - Snapshot(PageEntriesVersionSetWithDelta * vset_, VersionPtr tail_) + Snapshot(PageEntriesVersionSetWithDelta * vset_, VersionPtr tail_, const String & tracing_id_) : vset(vset_) , view(std::move(tail_)) - , t_id(Poco::ThreadNumber::get()) + , create_thread(Poco::ThreadNumber::get()) + , tracing_id(tracing_id_) , create_time(std::chrono::steady_clock::now()) { CurrentMetrics::add(CurrentMetrics::PSMVCCNumSnapshots); @@ -163,7 +164,7 @@ class PageEntriesVersionSetWithDelta using SnapshotPtr = std::shared_ptr; using SnapshotWeakPtr = std::weak_ptr; - SnapshotPtr getSnapshot(); + SnapshotPtr getSnapshot(const String & tracing_id = ""); std::pair, std::set> gcApply(PageEntriesEdit & edit, bool need_scan_page_ids = true); @@ -213,7 +214,7 @@ class PageEntriesVersionSetWithDelta // Return < num of snapshots, // living time(seconds) of the oldest snapshot, // created thread id of the oldest snapshot > - std::tuple removeExpiredSnapshots() const; + SnapshotsStatistics removeExpiredSnapshots() const; static void collectLiveFilesFromVersionList( // const PageEntriesView & view, diff --git a/dbms/src/Storages/Page/V2/gc/LegacyCompactor.cpp b/dbms/src/Storages/Page/V2/gc/LegacyCompactor.cpp index ce6da1d41f2..63425d051c5 100644 --- a/dbms/src/Storages/Page/V2/gc/LegacyCompactor.cpp +++ b/dbms/src/Storages/Page/V2/gc/LegacyCompactor.cpp @@ -85,7 +85,7 @@ LegacyCompactor::tryCompact( } // Build a version_set with snapshot - auto snapshot = version_set.getSnapshot(); + auto snapshot = version_set.getSnapshot(/*tracing_id*/ ""); auto wb = prepareCheckpointWriteBatch(snapshot, checkpoint_sequence); { diff --git a/dbms/src/Storages/Page/V3/PageDirectory.cpp b/dbms/src/Storages/Page/V3/PageDirectory.cpp index b31bc25c82f..c659fdeb88d 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectory.cpp @@ -629,9 +629,9 @@ PageDirectory::PageDirectory(WALStorePtr && wal_) { } -PageDirectorySnapshotPtr PageDirectory::createSnapshot() const +PageDirectorySnapshotPtr PageDirectory::createSnapshot(const String & tracing_id) const { - auto snap = std::make_shared(sequence.load()); + auto snap = std::make_shared(sequence.load(), tracing_id); { std::lock_guard snapshots_lock(snapshots_mutex); snapshots.emplace_back(std::weak_ptr(snap)); @@ -641,12 +641,10 @@ PageDirectorySnapshotPtr PageDirectory::createSnapshot() const return snap; } -std::tuple PageDirectory::getSnapshotsStat() const +SnapshotsStatistics PageDirectory::getSnapshotsStat() const { - double longest_living_seconds = 0.0; - unsigned longest_living_from_thread_id = 0; + SnapshotsStatistics stat; DB::Int64 num_snapshots_removed = 0; - size_t num_valid_snapshots = 0; { std::lock_guard lock(snapshots_mutex); for (auto iter = snapshots.begin(); iter != snapshots.end(); /* empty */) @@ -666,12 +664,13 @@ std::tuple PageDirectory::getSnapshotsStat() const }); const auto snapshot_lifetime = snapshot_ptr->elapsedSeconds(); - if (snapshot_lifetime > longest_living_seconds) + if (snapshot_lifetime > stat.longest_living_seconds) { - longest_living_seconds = snapshot_lifetime; - longest_living_from_thread_id = snapshot_ptr->getTid(); + stat.longest_living_seconds = snapshot_lifetime; + stat.longest_living_from_thread_id = snapshot_ptr->create_thread; + stat.longest_living_from_tracing_id = snapshot_ptr->tracing_id; } - num_valid_snapshots++; + stat.num_snapshots++; ++iter; } } @@ -679,7 +678,7 @@ std::tuple PageDirectory::getSnapshotsStat() const CurrentMetrics::sub(CurrentMetrics::PSMVCCSnapshotsList, num_snapshots_removed); // Return some statistics of the oldest living snapshot. - return {num_valid_snapshots, longest_living_seconds, longest_living_from_thread_id}; + return stat; } PageIDAndEntryV3 PageDirectory::get(PageIdV3Internal page_id, const PageDirectorySnapshotPtr & snap) const @@ -1189,7 +1188,7 @@ PageEntriesEdit PageDirectory::dumpSnapshotToEdit(PageDirectorySnapshotPtr snap) { if (!snap) { - snap = createSnapshot(); + snap = createSnapshot(/*tracing_id*/ ""); } PageEntriesEdit edit; diff --git a/dbms/src/Storages/Page/V3/PageDirectory.h b/dbms/src/Storages/Page/V3/PageDirectory.h index 143c5040259..a6349687335 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.h +++ b/dbms/src/Storages/Page/V3/PageDirectory.h @@ -44,10 +44,10 @@ class PageDirectorySnapshot : public DB::PageStorageSnapshot public: using TimePoint = std::chrono::time_point; - UInt64 sequence; - explicit PageDirectorySnapshot(UInt64 seq) + explicit PageDirectorySnapshot(UInt64 seq, const String & tracing_id_) : sequence(seq) - , t_id(Poco::ThreadNumber::get()) + , create_thread(Poco::ThreadNumber::get()) + , tracing_id(tracing_id_) , create_time(std::chrono::steady_clock::now()) { CurrentMetrics::add(CurrentMetrics::PSMVCCNumSnapshots); @@ -67,11 +67,15 @@ class PageDirectorySnapshot : public DB::PageStorageSnapshot unsigned getTid() const { - return t_id; + return create_thread; } +public: + UInt64 sequence; + const unsigned create_thread; + const String tracing_id; + private: - const unsigned t_id; const TimePoint create_time; }; using PageDirectorySnapshotPtr = std::shared_ptr; @@ -286,9 +290,9 @@ class PageDirectory public: explicit PageDirectory(WALStorePtr && wal); - PageDirectorySnapshotPtr createSnapshot() const; + PageDirectorySnapshotPtr createSnapshot(const String & tracing_id = "") const; - std::tuple getSnapshotsStat() const; + SnapshotsStatistics getSnapshotsStat() const; PageIDAndEntryV3 get(PageIdV3Internal page_id, const PageDirectorySnapshotPtr & snap) const; PageIDAndEntryV3 get(PageIdV3Internal page_id, const DB::PageStorageSnapshotPtr & snap) const diff --git a/dbms/src/Storages/Page/V3/PageStorageImpl.cpp b/dbms/src/Storages/Page/V3/PageStorageImpl.cpp index 0b8357da6c3..25a3f45cec3 100644 --- a/dbms/src/Storages/Page/V3/PageStorageImpl.cpp +++ b/dbms/src/Storages/Page/V3/PageStorageImpl.cpp @@ -66,18 +66,18 @@ PageId PageStorageImpl::getNormalPageId(NamespaceId ns_id, PageId page_id, Snaps { if (!snapshot) { - snapshot = this->getSnapshot(); + snapshot = this->getSnapshot(""); } return page_directory->getNormalPageId(buildV3Id(ns_id, page_id), snapshot).low; } -DB::PageStorage::SnapshotPtr PageStorageImpl::getSnapshot() +DB::PageStorage::SnapshotPtr PageStorageImpl::getSnapshot(const String & tracing_id) { - return page_directory->createSnapshot(); + return page_directory->createSnapshot(tracing_id); } -std::tuple PageStorageImpl::getSnapshotsStat() const +SnapshotsStatistics PageStorageImpl::getSnapshotsStat() const { return page_directory->getSnapshotsStat(); } @@ -96,7 +96,7 @@ DB::PageEntry PageStorageImpl::getEntry(NamespaceId ns_id, PageId page_id, Snaps { if (!snapshot) { - snapshot = this->getSnapshot(); + snapshot = this->getSnapshot(""); } try @@ -125,7 +125,7 @@ DB::Page PageStorageImpl::read(NamespaceId ns_id, PageId page_id, const ReadLimi { if (!snapshot) { - snapshot = this->getSnapshot(); + snapshot = this->getSnapshot(""); } auto page_entry = page_directory->get(buildV3Id(ns_id, page_id), snapshot); @@ -136,7 +136,7 @@ PageMap PageStorageImpl::read(NamespaceId ns_id, const std::vector & pag { if (!snapshot) { - snapshot = this->getSnapshot(); + snapshot = this->getSnapshot(""); } PageIdV3Internals page_id_v3s; @@ -150,7 +150,7 @@ void PageStorageImpl::read(NamespaceId ns_id, const std::vector & page_i { if (!snapshot) { - snapshot = this->getSnapshot(); + snapshot = this->getSnapshot(""); } PageIdV3Internals page_id_v3s; @@ -164,7 +164,7 @@ PageMap PageStorageImpl::read(NamespaceId ns_id, const std::vectorgetSnapshot(); + snapshot = this->getSnapshot(""); } BlobStore::FieldReadInfos read_infos; @@ -183,7 +183,7 @@ void PageStorageImpl::traverse(const std::function { if (!snapshot) { - snapshot = this->getSnapshot(); + snapshot = this->getSnapshot(""); } // TODO: This could hold the read lock of `page_directory` for a long time diff --git a/dbms/src/Storages/Page/V3/PageStorageImpl.h b/dbms/src/Storages/Page/V3/PageStorageImpl.h index 119596491c4..a18d7118a28 100644 --- a/dbms/src/Storages/Page/V3/PageStorageImpl.h +++ b/dbms/src/Storages/Page/V3/PageStorageImpl.h @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -42,9 +43,9 @@ class PageStorageImpl : public DB::PageStorage PageId getNormalPageId(NamespaceId ns_id, PageId page_id, SnapshotPtr snapshot) override; - DB::PageStorage::SnapshotPtr getSnapshot() override; + DB::PageStorage::SnapshotPtr getSnapshot(const String & tracing_id) override; - std::tuple getSnapshotsStat() const override; + SnapshotsStatistics getSnapshotsStat() const override; void write(DB::WriteBatch && write_batch, const WriteLimiterPtr & write_limiter) override; @@ -67,6 +68,7 @@ class PageStorageImpl : public DB::PageStorage void clearExternalPagesCallbacks(); #ifndef NDEBUG // Just for tests, refactor them out later + DB::PageStorage::SnapshotPtr getSnapshot() { return getSnapshot(""); } void write(DB::WriteBatch && wb) { return write(std::move(wb), nullptr); } DB::PageEntry getEntry(PageId page_id) { return getEntry(TEST_NAMESPACE_ID, page_id, nullptr); } DB::Page read(PageId page_id) { return read(TEST_NAMESPACE_ID, page_id, nullptr, nullptr); } diff --git a/dbms/src/Storages/Page/stress/PSBackground.cpp b/dbms/src/Storages/Page/stress/PSBackground.cpp index 12b84ed1b48..af7329e8348 100644 --- a/dbms/src/Storages/Page/stress/PSBackground.cpp +++ b/dbms/src/Storages/Page/stress/PSBackground.cpp @@ -70,18 +70,17 @@ void PSGc::start() void PSScanner::onTime(Poco::Timer & /*timer*/) { - size_t num_snapshots = 0; - double oldest_snapshot_seconds = 0.0; - unsigned oldest_snapshot_thread = 0; try { - LOG_INFO(StressEnv::logger, "Scanner start"); - std::tie(num_snapshots, oldest_snapshot_seconds, oldest_snapshot_thread) = ps->getSnapshotsStat(); - LOG_INFO(StressEnv::logger, - fmt::format("Scanner get {} snapshots, longest lifetime: {:.3f}s longest from thread: {}", - num_snapshots, - oldest_snapshot_seconds, - oldest_snapshot_thread)); + LOG_FMT_INFO(StressEnv::logger, "Scanner start"); + auto stat = ps->getSnapshotsStat(); + LOG_FMT_INFO( + StressEnv::logger, + "Scanner get {} snapshots, longest lifetime: {:.3f}s longest from thread: {}, tracing_id: {}", + stat.num_snapshots, + stat.longest_living_seconds, + stat.longest_living_from_thread_id, + stat.longest_living_from_tracing_id); } catch (...) { diff --git a/dbms/src/Storages/Page/stress/PSRunnable.cpp b/dbms/src/Storages/Page/stress/PSRunnable.cpp index 17e948b19a6..2d8f83c89a5 100644 --- a/dbms/src/Storages/Page/stress/PSRunnable.cpp +++ b/dbms/src/Storages/Page/stress/PSRunnable.cpp @@ -390,7 +390,7 @@ DB::PageIds PSWindowReader::genRandomPageIds() bool PSSnapshotReader::runImpl() { - snapshots.emplace_back(ps->getSnapshot()); + snapshots.emplace_back(ps->getSnapshot("")); usleep(snapshot_get_interval_ms * 1000); return true; } From f48d8031650835f30c8867bb81b47fc1eabdcd4b Mon Sep 17 00:00:00 2001 From: Schrodinger ZHU Yifan Date: Fri, 18 Mar 2022 14:26:30 +0800 Subject: [PATCH 2/9] Fix problems introduced by symbolization (#4334) close pingcap/tiflash#4333 --- CMakeLists.txt | 11 +++++++---- cmake/find_tiflash_proxy.cmake | 4 ++++ cmake/sequential.cmake | 20 ++++++++++++++++++++ contrib/tiflash-proxy-cmake/CMakeLists.txt | 3 +-- libs/libdaemon/CMakeLists.txt | 2 +- libs/libdaemon/src/BaseDaemon.cpp | 7 ++++--- libs/libsymbolization/CMakeLists.txt | 10 +++++----- 7 files changed, 42 insertions(+), 15 deletions(-) create mode 100644 cmake/sequential.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index a71bf760050..e37df3a59c2 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -19,10 +19,6 @@ set(CMAKE_SKIP_INSTALL_ALL_DEPENDENCY true) set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${TiFlash_SOURCE_DIR}/cmake/Modules/") set(CMAKE_MACOSX_RPATH 1) -# Rust jobs may modify global toolchain settings. We make rust related jobs sequentially scheduled -# by putting them in a job pool with single concurrency. -set_property(GLOBAL PROPERTY JOB_POOLS rust_job_pool=1) - option(TIFLASH_ENABLE_LLVM_DEVELOPMENT "enable facilities for development with LLVM" OFF) if(CMAKE_PREFIX_PATH) @@ -442,3 +438,10 @@ add_subdirectory (dbms) include (cmake/print_include_directories.cmake) +# Building a rust project may require setting up toolchains. Rustup cannot handle parallel requests, hence it may fail to +# finish the setup if multiple projects want to install a same version. To mitigate the situation, we force cargo targets +# to be invoked sequentially by adding linear dependency relations between them. +# Another way to do so is to use JOB_POOL with single concurrency. However, it is Ninja specific and it seems to have bugs +# with a large number of threads. +include (cmake/sequential.cmake) +build_sequentially (symbolization tiflash_proxy) diff --git a/cmake/find_tiflash_proxy.cmake b/cmake/find_tiflash_proxy.cmake index 00bbf3e8841..13a58155a64 100644 --- a/cmake/find_tiflash_proxy.cmake +++ b/cmake/find_tiflash_proxy.cmake @@ -50,3 +50,7 @@ endif() set(TIFLASH_PROXY_FOUND TRUE) message(STATUS "Using tiflash proxy: ${USE_INTERNAL_TIFLASH_PROXY} : ${TIFLASH_PROXY_INCLUDE_DIR}, ${TIFLASH_PROXY_LIBRARY}") + +if (NOT USE_INTERNAL_TIFLASH_PROXY) + add_custom_target(tiflash_proxy ALL DEPENDS ${TIFLASH_PROXY_LIBRARY}) +endif() \ No newline at end of file diff --git a/cmake/sequential.cmake b/cmake/sequential.cmake new file mode 100644 index 00000000000..0a86cf8e280 --- /dev/null +++ b/cmake/sequential.cmake @@ -0,0 +1,20 @@ +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +## Mark targets to build sequentially +# Example: build_sequentially(a, b, c, d) +# a -> b -> c -> d +function(build_sequentially target1 target2) + add_dependencies(${target2} ${target1}) + if(${ARGC} GREATER 2) + build_sequentially(${target2} ${ARGN}) + endif() +endfunction() \ No newline at end of file diff --git a/contrib/tiflash-proxy-cmake/CMakeLists.txt b/contrib/tiflash-proxy-cmake/CMakeLists.txt index aca39209b73..41c2d0bc4ab 100644 --- a/contrib/tiflash-proxy-cmake/CMakeLists.txt +++ b/contrib/tiflash-proxy-cmake/CMakeLists.txt @@ -31,8 +31,7 @@ add_custom_command(OUTPUT ${_TIFLASH_PROXY_LIBRARY} DEPENDS "${_TIFLASH_PROXY_SRCS}" "${_TIFLASH_PROXY_SOURCE_DIR}/Cargo.lock" "${_TIFLASH_PROXY_SOURCE_DIR}/rust-toolchain") add_custom_target(tiflash_proxy ALL - DEPENDS ${_TIFLASH_PROXY_LIBRARY} - JOB_POOL rust_job_pool) + DEPENDS ${_TIFLASH_PROXY_LIBRARY}) add_library(libtiflash_proxy SHARED IMPORTED GLOBAL) set_target_properties(libtiflash_proxy PROPERTIES IMPORTED_LOCATION ${_TIFLASH_PROXY_LIBRARY} IMPORTED_NO_SONAME ON) diff --git a/libs/libdaemon/CMakeLists.txt b/libs/libdaemon/CMakeLists.txt index f06eeb22e65..da39500917a 100644 --- a/libs/libdaemon/CMakeLists.txt +++ b/libs/libdaemon/CMakeLists.txt @@ -35,7 +35,7 @@ endif () target_include_directories (daemon PUBLIC include) target_include_directories (daemon PRIVATE ${TiFlash_SOURCE_DIR}/libs/libpocoext/include) -target_link_libraries (daemon clickhouse_common_io clickhouse_common_config ${EXECINFO_LIBRARY} symbolization) +target_link_libraries (daemon clickhouse_common_io clickhouse_common_config ${EXECINFO_LIBRARY} libsymbolization) target_compile_definitions(daemon PRIVATE -DTIFLASH_SOURCE_PREFIX=\"${TiFlash_SOURCE_DIR}\") if (ENABLE_TESTS) add_subdirectory (src/tests EXCLUDE_FROM_ALL) diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 835d23c4304..1994df1d90c 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -543,11 +543,12 @@ class SignalListener : public Poco::Runnable if (already_printed_stack_trace) return; - static const int max_frames = 50; + static constexpr int max_frames = 50; + int frames_size = 0; void * frames[max_frames]; #if USE_UNWIND - int frames_size = backtraceLibUnwind(frames, max_frames, unw_context); + frames_size = backtraceLibUnwind(frames, max_frames, unw_context); UNUSED(caller_address); #else /// No libunwind means no backtrace, because we are in a different thread from the one where the signal happened. @@ -555,7 +556,7 @@ class SignalListener : public Poco::Runnable if (caller_address) { frames[0] = caller_address; - int frames_size = 1; + frames_size = 1; } #endif diff --git a/libs/libsymbolization/CMakeLists.txt b/libs/libsymbolization/CMakeLists.txt index ab56c83f2dd..43c9a088547 100644 --- a/libs/libsymbolization/CMakeLists.txt +++ b/libs/libsymbolization/CMakeLists.txt @@ -27,8 +27,8 @@ add_custom_command(OUTPUT ${_SYMBOLIZATION_LIBRARY} "${_SYMBOLIZATION_SOURCE_DIR}/Cargo.toml" "${_SYMBOLIZATION_SOURCE_DIR}/rust-toolchain") -add_custom_target(_symbolization ALL DEPENDS ${_SYMBOLIZATION_LIBRARY} JOB_POOL rust_job_pool) -add_library(symbolization STATIC IMPORTED GLOBAL) -set_target_properties(symbolization PROPERTIES IMPORTED_LOCATION ${_SYMBOLIZATION_LIBRARY}) -add_dependencies(symbolization _symbolization) -target_include_directories(symbolization INTERFACE ${_SYMBOLIZATION_SOURCE_DIR}/include) +add_custom_target(symbolization ALL DEPENDS ${_SYMBOLIZATION_LIBRARY}) +add_library(libsymbolization STATIC IMPORTED GLOBAL) +set_target_properties(libsymbolization PROPERTIES IMPORTED_LOCATION ${_SYMBOLIZATION_LIBRARY}) +add_dependencies(libsymbolization symbolization) +target_include_directories(libsymbolization INTERFACE ${_SYMBOLIZATION_SOURCE_DIR}/include) From 358d52843e71e322ce2aa7b8d88976059af255fe Mon Sep 17 00:00:00 2001 From: Schrodinger ZHU Yifan Date: Fri, 18 Mar 2022 17:30:31 +0800 Subject: [PATCH 3/9] fix: remove basename (#4343) close pingcap/tiflash#4342 --- libs/libdaemon/src/BaseDaemon.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 1994df1d90c..4f29fe23643 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -595,7 +595,16 @@ class SignalListener : public Poco::Runnable if (sym_info.object_name) { - output << " [" << ::basename(sym_info.object_name) << "+" << sym_info.svma << "]"; + std::string_view view(sym_info.object_name); + auto pos = view.rfind('/'); + if (pos != std::string_view::npos) + { + output << " [" << view.substr(pos + 1) << "+" << sym_info.svma << "]"; + } + else + { + output << " [" << view << "+" << sym_info.svma << "]"; + } } if (sym_info.source_filename) @@ -604,11 +613,11 @@ class SignalListener : public Poco::Runnable std::string_view view(sym_info.source_filename, sym_info.source_filename_length); if (view.find(TIFLASH_SOURCE_PREFIX) != std::string_view::npos) { - output << address << "\t" << view.substr(prefix_size) << ":" << sym_info.lineno << ""; + output << address << "\t" << view.substr(prefix_size) << ":" << sym_info.lineno; } else { - output << address << "\t" << view << ":" << sym_info.lineno << ""; + output << address << "\t" << view << ":" << sym_info.lineno; } } } From 5f163e623bf5cd810db1805b0655cd7d6b37c8f3 Mon Sep 17 00:00:00 2001 From: lidezhu <47731263+lidezhu@users.noreply.github.com> Date: Sat, 19 Mar 2022 12:28:30 +0800 Subject: [PATCH 4/9] share PageStorage V3 inside tiflash instance (#4220) ref pingcap/tiflash#3594 --- dbms/src/Interpreters/Context.cpp | 70 +++++++ dbms/src/Interpreters/Context.h | 6 + dbms/src/Interpreters/Settings.h | 4 +- dbms/src/Server/DTTool/DTToolBench.cpp | 2 - dbms/src/Server/Server.cpp | 7 + dbms/src/Server/StorageConfigParser.cpp | 8 +- dbms/src/Server/StorageConfigParser.h | 1 + dbms/src/Server/tests/gtest_dttool.cpp | 2 - .../DeltaMerge/ColumnFile/ColumnFileTiny.cpp | 2 +- dbms/src/Storages/DeltaMerge/DMContext.h | 4 - .../Delta/ColumnFilePersistedSet.cpp | 4 +- .../DeltaMerge/Delta/DeltaValueSpace.h | 7 +- .../Storages/DeltaMerge/Delta/MemTableSet.cpp | 4 +- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 135 ++++++++------ .../src/Storages/DeltaMerge/DeltaMergeStore.h | 9 +- dbms/src/Storages/DeltaMerge/Segment.cpp | 22 +-- dbms/src/Storages/DeltaMerge/StoragePool.cpp | 174 +++++++++++++++--- dbms/src/Storages/DeltaMerge/StoragePool.h | 98 +++++++--- .../tests/gtest_dm_delta_value_space.cpp | 6 +- .../DeltaMerge/tests/gtest_dm_file.cpp | 6 - .../DeltaMerge/tests/gtest_dm_segment.cpp | 8 +- .../tests/gtest_dm_segment_common_handle.cpp | 6 +- .../DeltaMerge/tools/workload/DTWorkload.cpp | 91 +++++---- .../DeltaMerge/tools/workload/MainEntry.cpp | 31 ++-- .../DeltaMerge/tools/workload/Options.cpp | 5 +- .../DeltaMerge/tools/workload/Options.h | 4 +- dbms/src/Storages/Page/PageStorage.cpp | 9 +- dbms/src/Storages/Page/PageStorage.h | 6 +- dbms/src/Storages/Page/V3/BlobStore.cpp | 14 +- dbms/src/Storages/Page/V3/PageDirectory.cpp | 17 +- dbms/src/Storages/Page/V3/PageDirectory.h | 6 +- .../Storages/Page/V3/PageDirectoryFactory.cpp | 1 + dbms/src/Storages/Page/V3/PageStorageImpl.cpp | 31 +++- dbms/src/Storages/Page/V3/PageStorageImpl.h | 11 +- dbms/src/Storages/Page/V3/WAL/serialize.cpp | 2 + dbms/src/Storages/Page/V3/WALStore.cpp | 10 +- .../Page/V3/tests/gtest_page_directory.cpp | 36 ++++ .../Page/V3/tests/gtest_page_storage.cpp | 9 +- .../Page/V3/tests/gtest_wal_store.cpp | 2 + dbms/src/Storages/Page/WriteBatch.h | 16 +- dbms/src/Storages/PathPool.cpp | 110 ++++++++++- dbms/src/Storages/PathPool.h | 73 ++++++++ dbms/src/Storages/StorageDeltaMerge.cpp | 6 +- .../Storages/Transaction/RegionPersister.cpp | 41 ++++- .../Storages/Transaction/RegionPersister.h | 7 +- dbms/src/TestUtils/TiFlashTestEnv.cpp | 5 +- dbms/src/TestUtils/TiFlashTestEnv.h | 11 +- 47 files changed, 847 insertions(+), 292 deletions(-) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 41a0adcb43f..efaf3e833d4 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -53,8 +53,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -161,6 +163,7 @@ struct ContextShared PathCapacityMetricsPtr path_capacity_ptr; /// Path capacity metrics FileProviderPtr file_provider; /// File provider. IORateLimiter io_rate_limiter; + DM::GlobalStoragePoolPtr global_storage_pool; /// Named sessions. The user could specify session identifier to reuse settings and temporary tables in subsequent requests. class SessionKeyHash @@ -1571,6 +1574,73 @@ ReadLimiterPtr Context::getReadLimiter() const return getIORateLimiter().getReadLimiter(); } +static bool isUsingPageStorageV3(const PathPool & path_pool, bool enable_ps_v3) +{ + // Check whether v3 is already enabled + for (const auto & path : path_pool.listGlobalPagePaths()) + { + if (PS::V3::PageStorageImpl::isManifestsFileExists(path)) + { + return true; + } + } + + // Check whether v3 on new node is enabled in the config, if not, no need to check anymore + if (!enable_ps_v3) + return false; + + // Check whether there are any files in kvstore path, if exists, then this is not a new node. + // If it's a new node, then we enable v3. Otherwise, we use v2. + for (const auto & path : path_pool.listKVStorePaths()) + { + Poco::File dir(path); + if (!dir.exists()) + continue; + + std::vector files; + dir.list(files); + if (!files.empty()) + { + return false; + } + } + return true; +} + +bool Context::initializeGlobalStoragePoolIfNeed(const PathPool & path_pool, bool enable_ps_v3) +{ + auto lock = getLock(); + if (isUsingPageStorageV3(path_pool, enable_ps_v3)) + { + try + { + // create manifests file before initialize GlobalStoragePool + for (const auto & path : path_pool.listGlobalPagePaths()) + PS::V3::PageStorageImpl::createManifestsFileIfNeed(path); + + shared->global_storage_pool = std::make_shared(path_pool, *this, settings); + shared->global_storage_pool->restore(); + return true; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + throw; + } + } + else + { + shared->global_storage_pool = nullptr; + return false; + } +} + +DM::GlobalStoragePoolPtr Context::getGlobalStoragePool() const +{ + auto lock = getLock(); + return shared->global_storage_pool; +} + UInt16 Context::getTCPPort() const { auto lock = getLock(); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 9ba7231298c..a3de8daf68e 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -104,6 +104,8 @@ namespace DM { class MinMaxIndexCache; class DeltaIndexManager; +class GlobalStoragePool; +using GlobalStoragePoolPtr = std::shared_ptr; } // namespace DM /// (database name, table name) @@ -408,6 +410,10 @@ class Context ReadLimiterPtr getReadLimiter() const; IORateLimiter & getIORateLimiter() const; + bool initializeGlobalStoragePoolIfNeed(const PathPool & path_pool, bool enable_ps_v3); + + DM::GlobalStoragePoolPtr getGlobalStoragePool() const; + Compiler & getCompiler(); /// Call after initialization before using system logs. Call for global context. diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 2c54f741842..b44990758eb 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -49,7 +49,7 @@ struct Settings M(SettingBool, group_by_collation_sensitive, false, "do group by with collation info.") \ M(SettingUInt64, read_tso, DEFAULT_MAX_READ_TSO, "tmt read tso.") \ M(SettingInt64, dag_records_per_chunk, DEFAULT_DAG_RECORDS_PER_CHUNK, "default chunk size of a DAG response.") \ - M(SettingInt64, batch_send_min_limit, DEFAULT_BATCH_SEND_MIN_LIMIT, "default minial chunk size of exchanging data among TiFlash.") \ + M(SettingInt64, batch_send_min_limit, DEFAULT_BATCH_SEND_MIN_LIMIT, "default minimal chunk size of exchanging data among TiFlash.") \ M(SettingInt64, schema_version, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, "tmt schema version.") \ M(SettingUInt64, mpp_task_timeout, DEFAULT_MPP_TASK_TIMEOUT, "mpp task max endurable time.") \ M(SettingUInt64, mpp_task_running_timeout, DEFAULT_MPP_TASK_RUNNING_TIMEOUT, "mpp task max time that running without any progress.") \ @@ -282,7 +282,7 @@ struct Settings M(SettingBool, dt_raw_filter_range, true, "Do range filter or not when read data in raw mode in DeltaTree Engine.") \ M(SettingBool, dt_read_delta_only, false, "Only read delta data in DeltaTree Engine.") \ M(SettingBool, dt_read_stable_only, false, "Only read stable data in DeltaTree Engine.") \ - M(SettingBool, dt_enable_logical_split, false, "Enable logical split or not in DeltaTree Engine.") \ + M(SettingBool, dt_enable_logical_split, false, "Enable logical split or not in DeltaTree Engine.") \ M(SettingBool, dt_flush_after_write, false, "Flush cache or not after write in DeltaTree Engine.") \ M(SettingBool, dt_enable_relevant_place, false, "Enable relevant place or not in DeltaTree Engine.") \ M(SettingBool, dt_enable_skippable_place, true, "Enable skippable place or not in DeltaTree Engine.") \ diff --git a/dbms/src/Server/DTTool/DTToolBench.cpp b/dbms/src/Server/DTTool/DTToolBench.cpp index abdf4a6a033..b67a3512ba9 100644 --- a/dbms/src/Server/DTTool/DTToolBench.cpp +++ b/dbms/src/Server/DTTool/DTToolBench.cpp @@ -337,13 +337,11 @@ int benchEntry(const std::vector & opts) auto db_context = env.getContext(); auto path_pool = std::make_unique(db_context->getPathPool().withTable("test", "t1", false)); auto storage_pool = std::make_unique("test.t1", /*table_id*/ 1, *path_pool, *db_context, db_context->getSettingsRef()); - auto page_id_generator = std::make_unique(); auto dm_settings = DB::DM::DeltaMergeStore::Settings{}; auto dm_context = std::make_unique( // *db_context, *path_pool, *storage_pool, - *page_id_generator, /*hash_salt*/ 0, 0, dm_settings.not_compress_columns, diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 36e7c7461d1..84afee9af58 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -1101,6 +1101,13 @@ int Server::main(const std::vector & /*args*/) raft_config.enable_compatible_mode, // global_context->getPathCapacity(), global_context->getFileProvider()); + // must initialize before the following operation: + // 1. load data from disk(because this process may depend on the initialization of global StoragePool) + // 2. initialize KVStore service + // 1) because we need to check whether this is the first startup of this node, and we judge it based on whether there are any files in kvstore directory + // 2) KVStore service also choose its data format based on whether the GlobalStoragePool is initialized + if (global_context->initializeGlobalStoragePoolIfNeed(global_context->getPathPool(), storage_config.enable_ps_v3)) + LOG_FMT_INFO(log, "PageStorage V3 enabled."); // Use pd address to define which default_database we use by default. // For mock test, we use "default". For deployed with pd/tidb/tikv use "system", which is always exist in TiFlash. diff --git a/dbms/src/Server/StorageConfigParser.cpp b/dbms/src/Server/StorageConfigParser.cpp index ab1aecb6ab0..a3c2d7aa9e9 100644 --- a/dbms/src/Server/StorageConfigParser.cpp +++ b/dbms/src/Server/StorageConfigParser.cpp @@ -202,7 +202,13 @@ void TiFlashStorageConfig::parseMisc(const String & storage_section, Poco::Logge lazily_init_store = (*lazily_init != 0); } - LOG_FMT_INFO(log, "format_version {} lazily_init_store {}", format_version, lazily_init_store); + // config for experimental feature, may remove later + if (auto enable_v3 = table->get_qualified_as("enable_ps_v3"); enable_v3) + { + enable_ps_v3 = (*enable_v3 != 0); + } + + LOG_FMT_INFO(log, "format_version {} lazily_init_store {} enable_ps_v3 {}", format_version, lazily_init_store, enable_ps_v3); } Strings TiFlashStorageConfig::getAllNormalPaths() const diff --git a/dbms/src/Server/StorageConfigParser.h b/dbms/src/Server/StorageConfigParser.h index 4efc5637634..6f42b4a454b 100644 --- a/dbms/src/Server/StorageConfigParser.h +++ b/dbms/src/Server/StorageConfigParser.h @@ -103,6 +103,7 @@ struct TiFlashStorageConfig UInt64 format_version = 0; bool lazily_init_store = true; + bool enable_ps_v3 = false; public: TiFlashStorageConfig() = default; diff --git a/dbms/src/Server/tests/gtest_dttool.cpp b/dbms/src/Server/tests/gtest_dttool.cpp index 2357b0d9b4a..8499f2f863c 100644 --- a/dbms/src/Server/tests/gtest_dttool.cpp +++ b/dbms/src/Server/tests/gtest_dttool.cpp @@ -72,13 +72,11 @@ struct DTToolTest : public DB::base::TiFlashStorageTestBasic } auto path_pool = std::make_unique(db_context->getPathPool().withTable("test", "t1", false)); auto storage_pool = std::make_unique("test.t1", /*table_id*/ 1, *path_pool, *db_context, db_context->getSettingsRef()); - auto page_id_generator = std::make_unique(); auto dm_settings = DB::DM::DeltaMergeStore::Settings{}; auto dm_context = std::make_unique( // *db_context, *path_pool, *storage_pool, - *page_id_generator, /*hash_salt*/ 0, 0, dm_settings.not_compress_columns, diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileTiny.cpp b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileTiny.cpp index b0a5393e3fd..519bec134bc 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileTiny.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileTiny.cpp @@ -201,7 +201,7 @@ ColumnTinyFilePtr ColumnFileTiny::writeColumnFile(DMContext & context, const Blo PageId ColumnFileTiny::writeColumnFileData(DMContext & context, const Block & block, size_t offset, size_t limit, WriteBatches & wbs) { - auto page_id = context.page_id_generator.newLogPageId(); + auto page_id = context.storage_pool.newLogPageId(); MemoryWriteBuffer write_buf; PageFieldSizes col_data_sizes; diff --git a/dbms/src/Storages/DeltaMerge/DMContext.h b/dbms/src/Storages/DeltaMerge/DMContext.h index 669c89a7b75..30ae4c723d5 100644 --- a/dbms/src/Storages/DeltaMerge/DMContext.h +++ b/dbms/src/Storages/DeltaMerge/DMContext.h @@ -27,7 +27,6 @@ class StoragePathPool; namespace DM { class StoragePool; -class PageIdGenerator; using NotCompress = std::unordered_set; struct DMContext; using DMContextPtr = std::shared_ptr; @@ -41,7 +40,6 @@ struct DMContext : private boost::noncopyable StoragePathPool & path_pool; StoragePool & storage_pool; - PageIdGenerator & page_id_generator; const UInt64 hash_salt; // gc safe-point, maybe update. @@ -89,7 +87,6 @@ struct DMContext : private boost::noncopyable DMContext(const Context & db_context_, StoragePathPool & path_pool_, StoragePool & storage_pool_, - PageIdGenerator & page_id_generator_, const UInt64 hash_salt_, const DB::Timestamp min_version_, const NotCompress & not_compress_, @@ -100,7 +97,6 @@ struct DMContext : private boost::noncopyable : db_context(db_context_) , path_pool(path_pool_) , storage_pool(storage_pool_) - , page_id_generator(page_id_generator_) , hash_salt(hash_salt_) , min_version(min_version_) , not_compress(not_compress_) diff --git a/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp b/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp index a52fde00a4a..49db09e0a55 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp @@ -207,7 +207,7 @@ ColumnFilePersisteds ColumnFilePersistedSet::checkHeadAndCloneTail(DMContext & c else if (auto * t_file = column_file->tryToTinyFile(); t_file) { // Use a newly created page_id to reference the data page_id of current column file. - PageId new_data_page_id = context.page_id_generator.newLogPageId(); + PageId new_data_page_id = context.storage_pool.newLogPageId(); wbs.log.putRefPage(new_data_page_id, t_file->getDataPageId()); auto new_column_file = t_file->cloneWith(new_data_page_id); cloned_tail.push_back(new_column_file); @@ -215,7 +215,7 @@ ColumnFilePersisteds ColumnFilePersistedSet::checkHeadAndCloneTail(DMContext & c else if (auto * b_file = column_file->tryToBigFile(); b_file) { auto delegator = context.path_pool.getStableDiskDelegator(); - auto new_ref_id = context.page_id_generator.newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); + auto new_ref_id = context.storage_pool.newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); auto file_id = b_file->getFile()->fileId(); wbs.data.putRefPage(new_ref_id, file_id); auto file_parent_path = delegator.getDTFilePath(file_id); diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h index f1f4d141953..0e84a28526f 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h @@ -55,7 +55,8 @@ struct DMContext; struct WriteBatches; class StoragePool; -class DeltaValueSpace : public std::enable_shared_from_this +class DeltaValueSpace + : public std::enable_shared_from_this , private boost::noncopyable { public: @@ -166,7 +167,7 @@ class DeltaValueSpace : public std::enable_shared_from_this // Other thread is doing structure update, just return. if (!is_updating.compare_exchange_strong(v, true)) { - LOG_DEBUG(log, simpleInfo() << " Stop create snapshot because updating"); + LOG_FMT_DEBUG(log, "{} Stop create snapshot because updating", simpleInfo()); return false; } return true; @@ -177,7 +178,7 @@ class DeltaValueSpace : public std::enable_shared_from_this bool v = true; if (!is_updating.compare_exchange_strong(v, false)) { - LOG_ERROR(log, "!!!=========================delta [" << getId() << "] is expected to be updating=========================!!!"); + LOG_FMT_ERROR(log, "!!!=========================delta [ {}] is expected to be updating=========================!!!", getId()); return false; } else diff --git a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp index a2bc2ba482c..4d02c14d345 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/MemTableSet.cpp @@ -92,7 +92,7 @@ ColumnFiles MemTableSet::cloneColumnFiles(DMContext & context, const RowKeyRange else if (auto * t = column_file->tryToTinyFile(); t) { // Use a newly created page_id to reference the data page_id of current column file. - PageId new_data_page_id = context.page_id_generator.newLogPageId(); + PageId new_data_page_id = context.storage_pool.newLogPageId(); wbs.log.putRefPage(new_data_page_id, t->getDataPageId()); auto new_column_file = t->cloneWith(new_data_page_id); @@ -101,7 +101,7 @@ ColumnFiles MemTableSet::cloneColumnFiles(DMContext & context, const RowKeyRange else if (auto * f = column_file->tryToBigFile(); f) { auto delegator = context.path_pool.getStableDiskDelegator(); - auto new_ref_id = context.page_id_generator.newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); + auto new_ref_id = context.storage_pool.newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); auto file_id = f->getFile()->fileId(); wbs.data.putRefPage(new_ref_id, file_id); auto file_parent_path = delegator.getDTFilePath(file_id); diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 719b63a3527..2a2bbd299dd 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -189,21 +189,18 @@ DeltaMergeStore::DeltaMergeStore(Context & db_context, bool data_path_contains_database_name, const String & db_name_, const String & table_name_, - TableID table_id_, + TableID physical_table_id_, const ColumnDefines & columns, const ColumnDefine & handle, bool is_common_handle_, size_t rowkey_column_size_, - const Settings & settings_, - const TableID physical_table_id) + const Settings & settings_) : global_context(db_context.getGlobalContext()) , path_pool(global_context.getPathPool().withTable(db_name_, table_name_, data_path_contains_database_name)) , settings(settings_) - // for mock test, table_id_ should be DB::InvalidTableID - , storage_pool(db_name_ + "." + table_name_, table_id_ == DB::InvalidTableID ? TEST_NAMESPACE_ID : table_id_, path_pool, global_context, db_context.getSettingsRef()) , db_name(db_name_) , table_name(table_name_) - , physical_table_id(physical_table_id) + , physical_table_id(physical_table_id_) , is_common_handle(is_common_handle_) , rowkey_column_size(rowkey_column_size_) , original_table_handle_define(handle) @@ -215,6 +212,17 @@ DeltaMergeStore::DeltaMergeStore(Context & db_context, { LOG_FMT_INFO(log, "Restore DeltaMerge Store start [{}.{}]", db_name, table_name); + // for mock test, table_id_ should be DB::InvalidTableID + NamespaceId ns_id = physical_table_id == DB::InvalidTableID ? TEST_NAMESPACE_ID : physical_table_id; + if (auto global_storage_pool = global_context.getGlobalStoragePool(); global_storage_pool) + { + storage_pool = std::make_shared(ns_id, *global_storage_pool, global_context); + } + else + { + storage_pool = std::make_shared(db_name_ + "." + table_name_, ns_id, path_pool, global_context, db_context.getSettingsRef()); + } + // Restore existing dm files and set capacity for path_pool. // Should be done before any background task setup. restoreStableFiles(); @@ -235,12 +243,11 @@ DeltaMergeStore::DeltaMergeStore(Context & db_context, try { - storage_pool.restore(); // restore from disk - page_id_generator.restore(storage_pool); - if (!page_id_generator.maxMetaPageId()) + storage_pool->restore(); // restore from disk + if (!storage_pool->maxMetaPageId()) { // Create the first segment. - auto segment_id = page_id_generator.newMetaPageId(); + auto segment_id = storage_pool->newMetaPageId(); if (segment_id != DELTA_MERGE_FIRST_SEGMENT_ID) throw Exception(fmt::format("The first segment id should be {}", DELTA_MERGE_FIRST_SEGMENT_ID), ErrorCodes::LOGICAL_ERROR); auto first_segment @@ -321,10 +328,11 @@ void DeltaMergeStore::setUpBackgroundTask(const DMContextPtr & dm_context) } } }; - callbacks.ns_id = storage_pool.getNamespaceId(); - storage_pool.data()->registerExternalPagesCallbacks(callbacks); + callbacks.ns_id = storage_pool->getNamespaceId(); + // remember to unregister it when shutdown + storage_pool->data()->registerExternalPagesCallbacks(callbacks); + storage_pool->enableGC(); - gc_handle = background_pool.addTask([this] { return storage_pool.gc(global_context.getSettingsRef()); }); background_task_handle = background_pool.addTask([this] { return handleBackgroundTask(false); }); blockable_background_pool_handle = blockable_background_pool.addTask([this] { return handleBackgroundTask(true); }); @@ -367,7 +375,7 @@ void DeltaMergeStore::drop() shutdown(); LOG_FMT_INFO(log, "Drop DeltaMerge removing data from filesystem [{}.{}]", db_name, table_name); - storage_pool.drop(); + storage_pool->drop(); for (auto & [end, segment] : segments) { (void)end; @@ -390,8 +398,10 @@ void DeltaMergeStore::shutdown() return; LOG_FMT_TRACE(log, "Shutdown DeltaMerge start [{}.{}]", db_name, table_name); - background_pool.removeTask(gc_handle); - gc_handle = nullptr; + // shutdown before unregister to avoid conflict between this thread and background gc thread on the `ExternalPagesCallbacks` + // because PageStorage V2 doesn't have any lock protection on the `ExternalPagesCallbacks`.(The order doesn't matter for V3) + storage_pool->shutdown(); + storage_pool->data()->unregisterExternalPagesCallbacks(storage_pool->getNamespaceId()); background_pool.removeTask(background_task_handle); blockable_background_pool.removeTask(blockable_background_pool_handle); @@ -409,8 +419,7 @@ DMContextPtr DeltaMergeStore::newDMContext(const Context & db_context, const DB: // Like the context created by InterpreterSelectWithUnionQuery. auto * ctx = new DMContext(db_context.getGlobalContext(), path_pool, - storage_pool, - page_id_generator, + *storage_pool, hash_salt, latest_gc_safe_point.load(std::memory_order_acquire), settings.not_compress_columns, @@ -512,7 +521,7 @@ void DeltaMergeStore::write(const Context & db_context, const DB::Settings & db_ while (offset != rows) { RowKeyValueRef start_key = rowkey_column.getRowKeyValue(offset); - WriteBatches wbs(storage_pool, db_context.getWriteLimiter()); + WriteBatches wbs(*storage_pool, db_context.getWriteLimiter()); ColumnFilePtr write_column_file; RowKeyRange write_range; @@ -611,7 +620,7 @@ std::tuple DeltaMergeStore::preAllocateIngestFile() auto delegator = path_pool.getStableDiskDelegator(); auto parent_path = delegator.choosePath(); - auto new_id = page_id_generator.newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); + auto new_id = storage_pool->newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); return {parent_path, new_id}; } @@ -680,7 +689,7 @@ void DeltaMergeStore::ingestFiles( // Check https://github.com/pingcap/tics/issues/2040 for more details. // TODO: If tiflash crash during the middle of ingesting, we may leave some DTFiles on disk and // they can not be deleted. We should find a way to cleanup those files. - WriteBatches ingest_wbs(storage_pool, dm_context->getWriteLimiter()); + WriteBatches ingest_wbs(*storage_pool, dm_context->getWriteLimiter()); if (!files.empty()) { for (const auto & file : files) @@ -721,14 +730,14 @@ void DeltaMergeStore::ingestFiles( // Write could fail, because other threads could already updated the instance. Like split/merge, merge delta. ColumnFiles column_files; - WriteBatches wbs(storage_pool, dm_context->getWriteLimiter()); + WriteBatches wbs(*storage_pool, dm_context->getWriteLimiter()); for (const auto & file : files) { /// Generate DMFile instance with a new ref_id pointed to the file_id. auto file_id = file->fileId(); const auto & file_parent_path = file->parentPath(); - auto ref_id = page_id_generator.newDataPageIdForDTFile(delegate, __PRETTY_FUNCTION__); + auto ref_id = storage_pool->newDataPageIdForDTFile(delegate, __PRETTY_FUNCTION__); auto ref_file = DMFile::restore(file_provider, file_id, ref_id, file_parent_path, DMFile::ReadMetaMode::all()); auto column_file = std::make_shared(*dm_context, ref_file, segment_range); @@ -751,7 +760,7 @@ void DeltaMergeStore::ingestFiles( fiu_do_on(FailPoints::segment_merge_after_ingest_packs, { segment->flushCache(*dm_context); segmentMergeDelta(*dm_context, segment, TaskRunThread::BackgroundThreadPool); - storage_pool.gc(global_context.getSettingsRef(), StoragePool::Seconds(0)); + storage_pool->gc(global_context.getSettingsRef(), StoragePool::Seconds(0)); }); break; } @@ -1256,6 +1265,7 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const auto it = segments.find(segment->getRowKeyRange().getEnd()); // check legality if (it == segments.end()) + return {}; auto & cur_segment = it->second; if (cur_segment.get() != segment.get()) @@ -1709,7 +1719,7 @@ SegmentPair DeltaMergeStore::segmentSplit(DMContext & dm_context, const SegmentP Stopwatch watch_seg_split; SCOPE_EXIT({ GET_METRIC(tiflash_storage_subtask_duration_seconds, type_seg_split).Observe(watch_seg_split.elapsedSeconds()); }); - WriteBatches wbs(storage_pool, dm_context.getWriteLimiter()); + WriteBatches wbs(*storage_pool, dm_context.getWriteLimiter()); auto range = segment->getRowKeyRange(); auto split_info_opt = segment->prepareSplit(dm_context, schema_snap, segment_snap, wbs); @@ -1841,7 +1851,7 @@ void DeltaMergeStore::segmentMerge(DMContext & dm_context, const SegmentPtr & le auto left_range = left->getRowKeyRange(); auto right_range = right->getRowKeyRange(); - WriteBatches wbs(storage_pool, dm_context.getWriteLimiter()); + WriteBatches wbs(*storage_pool, dm_context.getWriteLimiter()); auto merged_stable = Segment::prepareMerge(dm_context, schema_snap, left, left_snap, right, right_snap, wbs); wbs.writeLogAndData(); merged_stable->enableDMFilesGC(); @@ -1964,7 +1974,7 @@ SegmentPtr DeltaMergeStore::segmentMergeDelta( } }); - WriteBatches wbs(storage_pool, dm_context.getWriteLimiter()); + WriteBatches wbs(*storage_pool, dm_context.getWriteLimiter()); auto new_stable = segment->prepareMergeDelta(dm_context, schema_snap, segment_snap, wbs); wbs.writeLogAndData(); @@ -2175,7 +2185,7 @@ void DeltaMergeStore::restoreStableFiles() static inline DB::PS::V2::PageEntriesVersionSetWithDelta::Snapshot * toConcreteSnapshot(const DB::PageStorage::SnapshotPtr & ptr) { - return assert_cast(ptr.get()); + return dynamic_cast(ptr.get()); } DeltaMergeStoreStat DeltaMergeStore::getStat() @@ -2263,60 +2273,69 @@ DeltaMergeStoreStat DeltaMergeStore::getStat() static const String useless_tracing_id("DeltaMergeStore::getStat"); { - auto snaps_stat = storage_pool.data()->getSnapshotsStat(); + auto snaps_stat = storage_pool->data()->getSnapshotsStat(); stat.storage_stable_num_snapshots = snaps_stat.num_snapshots; stat.storage_stable_oldest_snapshot_lifetime = snaps_stat.longest_living_seconds; stat.storage_stable_oldest_snapshot_thread_id = snaps_stat.longest_living_from_thread_id; stat.storage_stable_oldest_snapshot_tracing_id = snaps_stat.longest_living_from_tracing_id; - PageStorage::SnapshotPtr stable_snapshot = storage_pool.data()->getSnapshot(useless_tracing_id); + PageStorage::SnapshotPtr stable_snapshot = storage_pool->data()->getSnapshot(useless_tracing_id); const auto * concrete_snap = toConcreteSnapshot(stable_snapshot); - if (const auto * const version = concrete_snap->version(); version != nullptr) + if (concrete_snap) { - stat.storage_stable_num_pages = version->numPages(); - stat.storage_stable_num_normal_pages = version->numNormalPages(); - stat.storage_stable_max_page_id = version->maxId(); - } - else - { - LOG_FMT_ERROR(log, "Can't get any version from current snapshot.[type=data] [database={}] [table={}]", db_name, table_name); + if (const auto * const version = concrete_snap->version(); version != nullptr) + { + stat.storage_stable_num_pages = version->numPages(); + stat.storage_stable_num_normal_pages = version->numNormalPages(); + stat.storage_stable_max_page_id = version->maxId(); + } + else + { + LOG_FMT_ERROR(log, "Can't get any version from current snapshot.[type=data] [database={}] [table={}]", db_name, table_name); + } } } { - auto snaps_stat = storage_pool.log()->getSnapshotsStat(); + auto snaps_stat = storage_pool->log()->getSnapshotsStat(); stat.storage_delta_num_snapshots = snaps_stat.num_snapshots; stat.storage_delta_oldest_snapshot_lifetime = snaps_stat.longest_living_seconds; stat.storage_delta_oldest_snapshot_thread_id = snaps_stat.longest_living_from_thread_id; stat.storage_delta_oldest_snapshot_tracing_id = snaps_stat.longest_living_from_tracing_id; - PageStorage::SnapshotPtr log_snapshot = storage_pool.log()->getSnapshot(useless_tracing_id); + PageStorage::SnapshotPtr log_snapshot = storage_pool->log()->getSnapshot(useless_tracing_id); const auto * concrete_snap = toConcreteSnapshot(log_snapshot); - if (const auto * const version = concrete_snap->version(); version != nullptr) + if (concrete_snap) { - stat.storage_delta_num_pages = version->numPages(); - stat.storage_delta_num_normal_pages = version->numNormalPages(); - stat.storage_delta_max_page_id = version->maxId(); - } - else - { - LOG_FMT_ERROR(log, "Can't get any version from current snapshot.[type=log] [database={}] [table={}]", db_name, table_name); + if (const auto * const version = concrete_snap->version(); version != nullptr) + { + stat.storage_delta_num_pages = version->numPages(); + stat.storage_delta_num_normal_pages = version->numNormalPages(); + stat.storage_delta_max_page_id = version->maxId(); + } + else + { + LOG_FMT_ERROR(log, "Can't get any version from current snapshot.[type=log] [database={}] [table={}]", db_name, table_name); + } } } { - auto snaps_stat = storage_pool.meta()->getSnapshotsStat(); + auto snaps_stat = storage_pool->meta()->getSnapshotsStat(); stat.storage_meta_num_snapshots = snaps_stat.num_snapshots; stat.storage_meta_oldest_snapshot_lifetime = snaps_stat.longest_living_seconds; stat.storage_meta_oldest_snapshot_thread_id = snaps_stat.longest_living_from_thread_id; stat.storage_meta_oldest_snapshot_tracing_id = snaps_stat.longest_living_from_tracing_id; - PageStorage::SnapshotPtr meta_snapshot = storage_pool.meta()->getSnapshot(useless_tracing_id); + PageStorage::SnapshotPtr meta_snapshot = storage_pool->meta()->getSnapshot(useless_tracing_id); const auto * concrete_snap = toConcreteSnapshot(meta_snapshot); - if (const auto * const version = concrete_snap->version(); version != nullptr) + if (concrete_snap) { - stat.storage_meta_num_pages = version->numPages(); - stat.storage_meta_num_normal_pages = version->numNormalPages(); - stat.storage_meta_max_page_id = version->maxId(); - } - else - { - LOG_FMT_ERROR(log, "Can't get any version from current snapshot.[type=meta] [database={}] [table={}]", db_name, table_name); + if (const auto * const version = concrete_snap->version(); version != nullptr) + { + stat.storage_meta_num_pages = version->numPages(); + stat.storage_meta_num_normal_pages = version->numNormalPages(); + stat.storage_meta_max_page_id = version->maxId(); + } + else + { + LOG_FMT_ERROR(log, "Can't get any version from current snapshot.[type=meta] [database={}] [table={}]", db_name, table_name); + } } } diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 5877704412a..5a37b98ea6b 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -296,13 +296,12 @@ class DeltaMergeStore : private boost::noncopyable bool data_path_contains_database_name, const String & db_name, const String & table_name_, - TableID table_id_, + TableID physical_table_id_, const ColumnDefines & columns, const ColumnDefine & handle, bool is_common_handle_, size_t rowkey_column_size_, - const Settings & settings_ = EMPTY_SETTINGS, - const TableID physical_table_id = 0); + const Settings & settings_ = EMPTY_SETTINGS); ~DeltaMergeStore(); void setUpBackgroundTask(const DMContextPtr & dm_context); @@ -455,8 +454,7 @@ class DeltaMergeStore : private boost::noncopyable Context & global_context; StoragePathPool path_pool; Settings settings; - StoragePool storage_pool; - PageIdGenerator page_id_generator; + StoragePoolPtr storage_pool; String db_name; String table_name; @@ -477,7 +475,6 @@ class DeltaMergeStore : private boost::noncopyable std::atomic shutdown_called{false}; BackgroundProcessingPool & background_pool; - BackgroundProcessingPool::TaskHandle gc_handle; BackgroundProcessingPool::TaskHandle background_task_handle; BackgroundProcessingPool & blockable_background_pool; diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index ade6222f6c6..2a8d1d51c6e 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -159,7 +159,7 @@ StableValueSpacePtr createNewStable(DMContext & context, DMFileBlockOutputStream::Flags flags; flags.setSingleFile(context.db_context.getSettingsRef().dt_enable_single_file_mode_dmfile); - PageId dtfile_id = context.page_id_generator.newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); + PageId dtfile_id = context.storage_pool.newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); auto dtfile = writeIntoNewDMFile(context, schema_snap, input_stream, dtfile_id, store_path, flags); auto stable = std::make_shared(stable_id); stable->setFiles({dtfile}, RowKeyRange::newAll(context.is_common_handle, context.rowkey_column_size)); @@ -229,8 +229,8 @@ SegmentPtr Segment::newSegment( rowkey_range, segment_id, next_segment_id, - context.page_id_generator.newMetaPageId(), - context.page_id_generator.newMetaPageId()); + context.storage_pool.newMetaPageId(), + context.storage_pool.newMetaPageId()); } SegmentPtr Segment::restoreSegment(DMContext & context, PageId segment_id) @@ -896,7 +896,7 @@ std::optional Segment::prepareSplitLogical(DMContext & dm_co EventRecorder recorder(ProfileEvents::DMSegmentSplit, ProfileEvents::DMSegmentSplitNS); - auto & page_id_generator = dm_context.page_id_generator; + auto & storage_pool = dm_context.storage_pool; RowKeyRange my_range(rowkey_range.start, split_point, is_common_handle, rowkey_column_size); RowKeyRange other_range(split_point, rowkey_range.end, is_common_handle, rowkey_column_size); @@ -913,7 +913,7 @@ std::optional Segment::prepareSplitLogical(DMContext & dm_co } GenPageId log_gen_page_id = [&]() { - return page_id_generator.newLogPageId(); + return storage_pool.newLogPageId(); }; DMFiles my_stable_files; @@ -926,8 +926,8 @@ std::optional Segment::prepareSplitLogical(DMContext & dm_co auto file_id = dmfile->fileId(); auto file_parent_path = delegate.getDTFilePath(file_id); - auto my_dmfile_id = page_id_generator.newDataPageIdForDTFile(delegate, __PRETTY_FUNCTION__); - auto other_dmfile_id = page_id_generator.newDataPageIdForDTFile(delegate, __PRETTY_FUNCTION__); + auto my_dmfile_id = storage_pool.newDataPageIdForDTFile(delegate, __PRETTY_FUNCTION__); + auto other_dmfile_id = storage_pool.newDataPageIdForDTFile(delegate, __PRETTY_FUNCTION__); wbs.data.putRefPage(my_dmfile_id, file_id); wbs.data.putRefPage(other_dmfile_id, file_id); @@ -950,7 +950,7 @@ std::optional Segment::prepareSplitLogical(DMContext & dm_co other_stable_files.push_back(other_dmfile); } - auto other_stable_id = page_id_generator.newMetaPageId(); + auto other_stable_id = storage_pool.newMetaPageId(); auto my_stable = std::make_shared(segment_snap->stable->getId()); auto other_stable = std::make_shared(other_stable_id); @@ -1052,7 +1052,7 @@ std::optional Segment::prepareSplitPhysical(DMContext & dm_c *read_info.read_columns, dm_context.min_version, is_common_handle); - auto other_stable_id = dm_context.page_id_generator.newMetaPageId(); + auto other_stable_id = dm_context.storage_pool.newMetaPageId(); other_stable = createNewStable(dm_context, schema_snap, other_data, other_stable_id, wbs); } @@ -1089,8 +1089,8 @@ SegmentPair Segment::applySplit(DMContext & dm_context, // // Created references to tail pages' pages in "log" storage, we need to write them down. wbs.writeLogAndData(); - auto other_segment_id = dm_context.page_id_generator.newMetaPageId(); - auto other_delta_id = dm_context.page_id_generator.newMetaPageId(); + auto other_segment_id = dm_context.storage_pool.newMetaPageId(); + auto other_delta_id = dm_context.storage_pool.newMetaPageId(); auto my_delta = std::make_shared(delta->getId(), my_persisted_files, my_in_memory_files); auto other_delta = std::make_shared(other_delta_id, other_persisted_files, other_in_memory_files); diff --git a/dbms/src/Storages/DeltaMerge/StoragePool.cpp b/dbms/src/Storages/DeltaMerge/StoragePool.cpp index 9ae3d37f20a..3d6fb73344b 100644 --- a/dbms/src/Storages/DeltaMerge/StoragePool.cpp +++ b/dbms/src/Storages/DeltaMerge/StoragePool.cpp @@ -67,8 +67,92 @@ PageStorage::Config extractConfig(const Settings & settings, StorageType subtype return config; } -StoragePool::StoragePool(const String & name, NamespaceId ns_id_, StoragePathPool & path_pool, const Context & global_ctx, const Settings & settings) - : ns_id(ns_id_) +template +static bool doStoragePoolGC(const Context & global_context, const Settings & settings, const T & storage_pool) +{ + bool done_anything = false; + auto write_limiter = global_context.getWriteLimiter(); + auto read_limiter = global_context.getReadLimiter(); + auto config = extractConfig(settings, StorageType::Meta); + storage_pool.meta()->reloadSettings(config); + done_anything |= storage_pool.meta()->gc(/*not_skip*/ false, write_limiter, read_limiter); + + config = extractConfig(settings, StorageType::Data); + storage_pool.data()->reloadSettings(config); + done_anything |= storage_pool.data()->gc(/*not_skip*/ false, write_limiter, read_limiter); + + config = extractConfig(settings, StorageType::Log); + storage_pool.log()->reloadSettings(config); + done_anything |= storage_pool.log()->gc(/*not_skip*/ false, write_limiter, read_limiter); + + return done_anything; +} + +GlobalStoragePool::GlobalStoragePool(const PathPool & path_pool, Context & global_ctx, const Settings & settings) + : // The iops and bandwidth in log_storage are relatively high, use multi-disks if possible + log_storage(PageStorage::create("__global__.log", + path_pool.getPSDiskDelegatorGlobalMulti("log"), + extractConfig(settings, StorageType::Log), + global_ctx.getFileProvider(), + true)) + , + // The iops in data_storage is low, only use the first disk for storing data + data_storage(PageStorage::create("__global__.data", + path_pool.getPSDiskDelegatorGlobalSingle("data"), + extractConfig(settings, StorageType::Data), + global_ctx.getFileProvider(), + true)) + , + // The iops in meta_storage is relatively high, use multi-disks if possible + meta_storage(PageStorage::create("__global__.meta", + path_pool.getPSDiskDelegatorGlobalMulti("meta"), + extractConfig(settings, StorageType::Meta), + global_ctx.getFileProvider(), + true)) + , global_context(global_ctx) +{} + +void GlobalStoragePool::restore() +{ + log_storage->restore(); + data_storage->restore(); + meta_storage->restore(); + + gc_handle = global_context.getBackgroundPool().addTask( + [this] { + return this->gc(global_context.getSettingsRef()); + }, + false); +} + +GlobalStoragePool::~GlobalStoragePool() +{ + if (gc_handle) + { + global_context.getBackgroundPool().removeTask(gc_handle); + gc_handle = nullptr; + } +} + +bool GlobalStoragePool::gc(const Settings & settings, const Seconds & try_gc_period) +{ + { + std::lock_guard lock(mutex); + + Timepoint now = Clock::now(); + if (now < (last_try_gc_time.load() + try_gc_period)) + return false; + + last_try_gc_time = now; + } + + return doStoragePoolGC(global_context, settings, *this); +} + + +StoragePool::StoragePool(const String & name, NamespaceId ns_id_, StoragePathPool & path_pool, Context & global_ctx, const Settings & settings) + : owned_storage(true) + , ns_id(ns_id_) , // The iops and bandwidth in log_storage are relatively high, use multi-disks if possible log_storage(PageStorage::create(name + ".log", @@ -93,24 +177,52 @@ StoragePool::StoragePool(const String & name, NamespaceId ns_id_, StoragePathPoo , global_context(global_ctx) {} +StoragePool::StoragePool(NamespaceId ns_id_, const GlobalStoragePool & global_storage_pool, Context & global_ctx) + : owned_storage(false) + , ns_id(ns_id_) + , log_storage(global_storage_pool.log()) + , data_storage(global_storage_pool.data()) + , meta_storage(global_storage_pool.meta()) + , log_storage_reader(ns_id, log_storage, nullptr) + , data_storage_reader(ns_id, data_storage, nullptr) + , meta_storage_reader(ns_id, meta_storage, nullptr) + , global_context(global_ctx) +{} + void StoragePool::restore() { - log_storage->restore(); - data_storage->restore(); - meta_storage->restore(); + // If the storage instances is not global, we need to initialize it by ourselves and add a gc task. + if (owned_storage) + { + log_storage->restore(); + data_storage->restore(); + meta_storage->restore(); + } + + max_log_page_id = log_storage->getMaxId(ns_id); + max_data_page_id = data_storage->getMaxId(ns_id); + max_meta_page_id = meta_storage->getMaxId(ns_id); } -void StoragePool::drop() +StoragePool::~StoragePool() +{ + shutdown(); +} + +void StoragePool::enableGC() { - meta_storage->drop(); - data_storage->drop(); - log_storage->drop(); + if (owned_storage) + gc_handle = global_context.getBackgroundPool().addTask([this] { return this->gc(global_context.getSettingsRef()); }); } bool StoragePool::gc(const Settings & settings, const Seconds & try_gc_period) { + if (!owned_storage) + return false; + { std::lock_guard lock(mutex); + // Just do gc for owned storage, otherwise the gc will be handled globally Timepoint now = Clock::now(); if (now < (last_try_gc_time.load() + try_gc_period)) @@ -119,32 +231,35 @@ bool StoragePool::gc(const Settings & settings, const Seconds & try_gc_period) last_try_gc_time = now; } - bool done_anything = false; - auto write_limiter = global_context.getWriteLimiter(); - auto read_limiter = global_context.getReadLimiter(); - auto config = extractConfig(settings, StorageType::Meta); - meta_storage->reloadSettings(config); - done_anything |= meta_storage->gc(/*not_skip*/ false, write_limiter, read_limiter); - - config = extractConfig(settings, StorageType::Data); - data_storage->reloadSettings(config); - done_anything |= data_storage->gc(/*not_skip*/ false, write_limiter, read_limiter); - - config = extractConfig(settings, StorageType::Log); - log_storage->reloadSettings(config); - done_anything |= log_storage->gc(/*not_skip*/ false, write_limiter, read_limiter); + return doStoragePoolGC(global_context, settings, *this); +} - return done_anything; +void StoragePool::shutdown() +{ + if (gc_handle) + { + global_context.getBackgroundPool().removeTask(gc_handle); + gc_handle = nullptr; + } } -void PageIdGenerator::restore(const StoragePool & storage_pool) +void StoragePool::drop() { - max_log_page_id = storage_pool.log_storage_reader.getMaxId(); - max_data_page_id = storage_pool.data_storage_reader.getMaxId(); - max_meta_page_id = storage_pool.meta_storage_reader.getMaxId(); + shutdown(); + + if (owned_storage) + { + meta_storage->drop(); + data_storage->drop(); + log_storage->drop(); + } + else + { + // FIXME: drop data for this table + } } -PageId PageIdGenerator::newDataPageIdForDTFile(StableDiskDelegator & delegator, const char * who) +PageId StoragePool::newDataPageIdForDTFile(StableDiskDelegator & delegator, const char * who) { // In case that there is a DTFile created on disk but TiFlash crashes without persisting the ID. // After TiFlash process restored, the ID will be inserted into the stable delegator, but we may @@ -175,5 +290,6 @@ PageId PageIdGenerator::newDataPageIdForDTFile(StableDiskDelegator & delegator, } while (true); return dtfile_id; } + } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/StoragePool.h b/dbms/src/Storages/DeltaMerge/StoragePool.h index 5efc87840da..859be76dbe5 100644 --- a/dbms/src/Storages/DeltaMerge/StoragePool.h +++ b/dbms/src/Storages/DeltaMerge/StoragePool.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include @@ -28,25 +29,67 @@ class StableDiskDelegator; namespace DM { +class StoragePool; +using StoragePoolPtr = std::shared_ptr; +class GlobalStoragePool; +using GlobalStoragePoolPtr = std::shared_ptr; + static const std::chrono::seconds DELTA_MERGE_GC_PERIOD(60); +class GlobalStoragePool : private boost::noncopyable +{ +public: + using Clock = std::chrono::system_clock; + using Timepoint = Clock::time_point; + using Seconds = std::chrono::seconds; + + GlobalStoragePool(const PathPool & path_pool, Context & global_ctx, const Settings & settings); + + void restore(); + + ~GlobalStoragePool(); + + PageStoragePtr log() const { return log_storage; } + PageStoragePtr data() const { return data_storage; } + PageStoragePtr meta() const { return meta_storage; } + +private: + // TODO: maybe more frequent gc for GlobalStoragePool? + bool gc(const Settings & settings, const Seconds & try_gc_period = DELTA_MERGE_GC_PERIOD); + +private: + PageStoragePtr log_storage; + PageStoragePtr data_storage; + PageStoragePtr meta_storage; + + std::atomic last_try_gc_time = Clock::now(); + + std::mutex mutex; + + Context & global_context; + BackgroundProcessingPool::TaskHandle gc_handle; +}; + class StoragePool : private boost::noncopyable { public: using Clock = std::chrono::system_clock; using Timepoint = Clock::time_point; - using Duration = Clock::duration; using Seconds = std::chrono::seconds; - StoragePool(const String & name, NamespaceId ns_id_, StoragePathPool & path_pool, const Context & global_ctx, const Settings & settings); + StoragePool(const String & name, NamespaceId ns_id_, StoragePathPool & path_pool, Context & global_ctx, const Settings & settings); + + StoragePool(NamespaceId ns_id_, const GlobalStoragePool & global_storage_pool, Context & global_ctx); void restore(); + ~StoragePool(); + NamespaceId getNamespaceId() const { return ns_id; } - PageStoragePtr log() { return log_storage; } - PageStoragePtr data() { return data_storage; } - PageStoragePtr meta() { return meta_storage; } + PageStoragePtr log() const { return log_storage; } + PageStoragePtr data() const { return data_storage; } + PageStoragePtr meta() const { return meta_storage; } PageReader & logReader() { return log_storage_reader; } PageReader & dataReader() { return data_storage_reader; } @@ -65,17 +108,30 @@ class StoragePool : private boost::noncopyable return PageReader(ns_id, meta_storage, snapshot_read ? meta_storage->getSnapshot(tracing_id) : nullptr, read_limiter); } + void enableGC(); + + bool gc(const Settings & settings, const Seconds & try_gc_period = DELTA_MERGE_GC_PERIOD); + + void shutdown(); + // Caller must cancel gc tasks before drop void drop(); - bool gc(const Settings & settings, const Seconds & try_gc_period = DELTA_MERGE_GC_PERIOD); + PageId newDataPageIdForDTFile(StableDiskDelegator & delegator, const char * who); + + PageId maxMetaPageId() { return max_meta_page_id; } + + PageId newLogPageId() { return ++max_log_page_id; } + PageId newMetaPageId() { return ++max_meta_page_id; } private: - NamespaceId ns_id; + // whether the three storage instance is owned by this StoragePool + const bool owned_storage = false; + const NamespaceId ns_id; - PageStoragePtr log_storage; - PageStoragePtr data_storage; - PageStoragePtr meta_storage; + const PageStoragePtr log_storage; + const PageStoragePtr data_storage; + const PageStoragePtr meta_storage; PageReader log_storage_reader; PageReader data_storage_reader; @@ -85,29 +141,13 @@ class StoragePool : private boost::noncopyable std::mutex mutex; - const Context & global_context; - - friend class PageIdGenerator; -}; - -class PageIdGenerator : private boost::noncopyable -{ -public: - PageIdGenerator() = default; - - void restore(const StoragePool & storage_pool); - - PageId newDataPageIdForDTFile(StableDiskDelegator & delegator, const char * who); + Context & global_context; - PageId maxMetaPageId() { return max_meta_page_id; } - - PageId newLogPageId() { return ++max_log_page_id; } - PageId newMetaPageId() { return ++max_meta_page_id; } - -private: std::atomic max_log_page_id = 0; std::atomic max_data_page_id = 0; std::atomic max_meta_page_id = 0; + + BackgroundProcessingPool::TaskHandle gc_handle = nullptr; }; struct StorageSnapshot : private boost::noncopyable diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp index 3aec2dea15f..93b366f3b15 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp @@ -85,9 +85,7 @@ class DeltaValueSpaceTest : public DB::base::TiFlashStorageTestBasic TiFlashStorageTestBasic::reload(std::move(db_settings)); storage_path_pool = std::make_unique(db_context->getPathPool().withTable("test", "t1", false)); storage_pool = std::make_unique("test.t1", table_id, *storage_path_pool, *db_context, db_context->getSettingsRef()); - page_id_generator = std::make_unique(); storage_pool->restore(); - page_id_generator->restore(*storage_pool); ColumnDefinesPtr cols = (!pre_define_columns) ? DMTestEnv::getDefaultColumns() : pre_define_columns; setColumns(cols); @@ -102,7 +100,6 @@ class DeltaValueSpaceTest : public DB::base::TiFlashStorageTestBasic dm_context = std::make_unique(*db_context, *storage_path_pool, *storage_pool, - *page_id_generator, 0, /*min_version_*/ 0, settings.not_compress_columns, @@ -119,7 +116,6 @@ class DeltaValueSpaceTest : public DB::base::TiFlashStorageTestBasic /// all these var lives as ref in dm_context std::unique_ptr storage_path_pool; std::unique_ptr storage_pool; - std::unique_ptr page_id_generator; ColumnDefinesPtr table_columns; DM::DeltaMergeStore::Settings settings; /// dm_context @@ -153,7 +149,7 @@ Block appendColumnFileBigToDeltaValueSpace(DMContext & context, ColumnDefinesPtr { Block block = DMTestEnv::prepareSimpleWriteBlock(rows_start, rows_start + rows_num, false, tso); auto delegator = context.path_pool.getStableDiskDelegator(); - auto file_id = context.page_id_generator.newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); + auto file_id = context.storage_pool.newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); auto input_stream = std::make_shared(block); auto store_path = delegator.choosePath(); auto dmfile diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp index 9438a934045..c07ee59433b 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp @@ -105,7 +105,6 @@ class DMFile_Test parent_path = TiFlashStorageTestBasic::getTemporaryPath(); path_pool = std::make_unique(db_context->getPathPool().withTable("test", "DMFile_Test", false)); storage_pool = std::make_unique("test.t1", /*table_id*/ 100, *path_pool, *db_context, db_context->getSettingsRef()); - page_id_generator = std::make_unique(); dm_file = DMFile::create(1, parent_path, single_file_mode, std::move(configuration)); table_columns_ = std::make_shared(); column_cache_ = std::make_shared(); @@ -124,7 +123,6 @@ class DMFile_Test *db_context, *path_pool, *storage_pool, - *page_id_generator, /*hash_salt*/ 0, 0, settings.not_compress_columns, @@ -152,7 +150,6 @@ class DMFile_Test /// all these var live as ref in dm_context std::unique_ptr path_pool; std::unique_ptr storage_pool; - std::unique_ptr page_id_generator; ColumnDefinesPtr table_columns_; DeltaMergeStore::Settings settings; @@ -1134,7 +1131,6 @@ class DMFile_Clustered_Index_Test : public DB::base::TiFlashStorageTestBasic path_pool = std::make_unique(db_context->getPathPool().withTable("test", "t", false)); storage_pool = std::make_unique("test.t1", table_id, *path_pool, *db_context, DB::Settings()); - page_id_generator = std::make_unique(); dm_file = DMFile::create(0, path, single_file_mode, std::move(configuration)); table_columns_ = std::make_shared(); column_cache_ = std::make_shared(); @@ -1155,7 +1151,6 @@ class DMFile_Clustered_Index_Test : public DB::base::TiFlashStorageTestBasic *db_context, *path_pool, *storage_pool, - *page_id_generator, /*hash_salt*/ 0, 0, settings.not_compress_columns, @@ -1175,7 +1170,6 @@ class DMFile_Clustered_Index_Test : public DB::base::TiFlashStorageTestBasic /// all these var live as ref in dm_context std::unique_ptr path_pool; std::unique_ptr storage_pool; - std::unique_ptr page_id_generator; ColumnDefinesPtr table_columns_; DeltaMergeStore::Settings settings; diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp index 24ee0ba5916..a77629f72f4 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp @@ -75,13 +75,11 @@ class Segment_test : public DB::base::TiFlashStorageTestBasic TiFlashStorageTestBasic::reload(std::move(db_settings)); storage_path_pool = std::make_unique(db_context->getPathPool().withTable("test", "t1", false)); storage_pool = std::make_unique("test.t1", /*table_id*/ 100, *storage_path_pool, *db_context, db_context->getSettingsRef()); - page_id_generator = std::make_unique(); storage_pool->restore(); - page_id_generator->restore(*storage_pool); ColumnDefinesPtr cols = (!pre_define_columns) ? DMTestEnv::getDefaultColumns() : pre_define_columns; setColumns(cols); - return Segment::newSegment(*dm_context_, table_columns_, RowKeyRange::newAll(false, 1), page_id_generator->newMetaPageId(), 0); + return Segment::newSegment(*dm_context_, table_columns_, RowKeyRange::newAll(false, 1), storage_pool->newMetaPageId(), 0); } // setColumns should update dm_context at the same time @@ -92,7 +90,6 @@ class Segment_test : public DB::base::TiFlashStorageTestBasic dm_context_ = std::make_unique(*db_context, *storage_path_pool, *storage_pool, - *page_id_generator, 0, /*min_version_*/ 0, settings.not_compress_columns, @@ -109,7 +106,6 @@ class Segment_test : public DB::base::TiFlashStorageTestBasic /// all these var lives as ref in dm_context std::unique_ptr storage_path_pool; std::unique_ptr storage_pool; - std::unique_ptr page_id_generator; ColumnDefinesPtr table_columns_; DM::DeltaMergeStore::Settings settings; /// dm_context @@ -1250,7 +1246,7 @@ class Segment_test_2 : public Segment_test std::pair> genDMFile(DMContext & context, const Block & block) { auto delegator = context.path_pool.getStableDiskDelegator(); - auto file_id = context.page_id_generator.newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); + auto file_id = context.storage_pool.newDataPageIdForDTFile(delegator, __PRETTY_FUNCTION__); auto input_stream = std::make_shared(block); auto store_path = delegator.choosePath(); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment_common_handle.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment_common_handle.cpp index 2ce8ab1166a..8fc5c145f81 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment_common_handle.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment_common_handle.cpp @@ -54,14 +54,12 @@ class Segment_Common_Handle_test : public DB::base::TiFlashStorageTestBasic TiFlashStorageTestBasic::reload(std::move(db_settings)); path_pool = std::make_unique(db_context->getPathPool().withTable("test", "t", false)); storage_pool = std::make_unique("test.t1", /*table_id*/ 100, *path_pool, *db_context, db_context->getSettingsRef()); - page_id_generator = std::make_unique(); storage_pool->restore(); - page_id_generator->restore(*storage_pool); if (!cols) cols = DMTestEnv::getDefaultColumns(is_common_handle ? DMTestEnv::PkType::CommonHandle : DMTestEnv::PkType::HiddenTiDBRowID); setColumns(cols); - auto segment_id = page_id_generator->newMetaPageId(); + auto segment_id = storage_pool->newMetaPageId(); return Segment::newSegment(*dm_context_, table_columns_, RowKeyRange::newAll(is_common_handle, rowkey_column_size), segment_id, 0); } @@ -73,7 +71,6 @@ class Segment_Common_Handle_test : public DB::base::TiFlashStorageTestBasic dm_context_ = std::make_unique(*db_context, *path_pool, *storage_pool, - *page_id_generator, 0, /*min_version_*/ 0, settings.not_compress_columns, @@ -90,7 +87,6 @@ class Segment_Common_Handle_test : public DB::base::TiFlashStorageTestBasic /// all these var lives as ref in dm_context std::unique_ptr path_pool; std::unique_ptr storage_pool; - std::unique_ptr page_id_generator; ColumnDefinesPtr table_columns_; DM::DeltaMergeStore::Settings settings; /// dm_context diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp b/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp index 0fe42277a99..9de98ab62bf 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp +++ b/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -66,16 +67,20 @@ DTWorkload::DTWorkload(const WorkloadOptions & opts_, std::shared_ptr(); Stopwatch sw; - store = std::make_unique(*context, true, table_info->db_name, table_info->table_name, table_info->table_id, // - *table_info->columns, - table_info->handle, - table_info->is_common_handle, - table_info->rowkey_column_indexes.size(), - DeltaMergeStore::Settings()); + store = std::make_unique( + *context, + true, + table_info->db_name, + table_info->table_name, + table_info->table_id, + *table_info->columns, + table_info->handle, + table_info->is_common_handle, + table_info->rowkey_column_indexes.size(), + DeltaMergeStore::Settings()); stat.init_store_sec = sw.elapsedSeconds(); LOG_FMT_INFO(log, "Init store {} seconds", stat.init_store_sec); - if (opts_.verification) { handle_lock = std::make_unique(); @@ -131,22 +136,12 @@ void DTWorkload::write(ThreadWriteStat & write_stat) write_stat.write_count = write_count; write_stat.max_do_write_us = max_do_write_us; - LOG_INFO(log, write_stat.toString()); + LOG_FMT_INFO(log, "{}", write_stat.toString()); writing_threads.fetch_sub(1, std::memory_order_relaxed); } - catch (const DB::Exception & e) - { - LOG_ERROR(log, e.message()); - std::abort(); - } - catch (const std::exception & e) - { - LOG_ERROR(log, e.what()); - std::abort(); - } catch (...) { - LOG_FMT_INFO(log, "Unknow exception."); + tryLogCurrentException("exception thrown in DTWorkload::write"); std::abort(); } } @@ -244,31 +239,55 @@ void DTWorkload::verifyHandle(uint64_t r) throw std::logic_error(s); } stat.verify_count = handle_count; - LOG_INFO(log, fmt::format("verifyHandle: round {} columns {} streams {} read_count {} read_sec {} handle_count {}", r, columns.size(), stream_count, read_count.load(std::memory_order_relaxed), stat.verify_sec, handle_count)); + LOG_FMT_INFO( + log, + "verifyHandle: round {} columns {} streams {} read_count {} read_sec {} handle_count {}", + r, + columns.size(), + stream_count, + read_count.load(std::memory_order_relaxed), + stat.verify_sec, + handle_count); } void DTWorkload::scanAll(uint64_t i) { - while (writing_threads.load(std::memory_order_relaxed) > 0) + try { - const auto & columns = store->getTableColumns(); - int stream_count = opts->read_stream_count; + while (writing_threads.load(std::memory_order_relaxed) > 0) + { + const auto & columns = store->getTableColumns(); + int stream_count = opts->read_stream_count; - std::atomic read_count = 0; - auto count_row = [&read_count](BlockInputStreamPtr in, [[maybe_unused]] uint64_t read_ts) { - while (Block block = in->read()) - { - read_count.fetch_add(block.rows(), std::memory_order_relaxed); - } - }; - Stopwatch sw; - read(columns, stream_count, count_row); - double read_sec = sw.elapsedSeconds(); + std::atomic read_count = 0; + auto count_row = [&read_count](BlockInputStreamPtr in, [[maybe_unused]] uint64_t read_ts) { + while (Block block = in->read()) + { + read_count.fetch_add(block.rows(), std::memory_order_relaxed); + } + }; + Stopwatch sw; + read(columns, stream_count, count_row); + double read_sec = sw.elapsedSeconds(); - stat.total_read_usec.fetch_add(read_sec * 1000000, std::memory_order_relaxed); - stat.total_read_count.fetch_add(read_count.load(std::memory_order_relaxed), std::memory_order_relaxed); + stat.total_read_usec.fetch_add(read_sec * 1000000, std::memory_order_relaxed); + stat.total_read_count.fetch_add(read_count.load(std::memory_order_relaxed), std::memory_order_relaxed); - LOG_FMT_INFO(log, "scanAll[{}]: columns {} streams {} read_count {} read_sec {} handle_count {}", i, columns.size(), stream_count, read_count.load(std::memory_order_relaxed), read_sec, handle_table->count()); + LOG_FMT_INFO( + log, + "scanAll[{}]: columns {} streams {} read_count {} read_sec {} handle_count {}", + i, + columns.size(), + stream_count, + read_count.load(std::memory_order_relaxed), + read_sec, + handle_table->count()); + } + } + catch (...) + { + tryLogCurrentException("exception thrown in scanAll"); + throw; } } diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/MainEntry.cpp b/dbms/src/Storages/DeltaMerge/tools/workload/MainEntry.cpp index f6606bf737e..1c650fbb3a5 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/MainEntry.cpp +++ b/dbms/src/Storages/DeltaMerge/tools/workload/MainEntry.cpp @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include #include #include #include @@ -50,9 +52,9 @@ void removeData(Poco::Logger * log, const std::vector & data_dirs) { for (const auto & dir : data_dirs) { - auto cmd = fmt::format("rm -rf {}", dir); - LOG_ERROR(log, cmd); - [[maybe_unused]] int ret = system(cmd.c_str()); + LOG_FMT_ERROR(log, "rm -rf {}", dir); + Poco::File d(dir); + d.remove(true); } } @@ -64,7 +66,7 @@ struct BasicStatistics uint64_t read_count = 0; double read_sec = 0.0; - BasicStatistics(const DTWorkload::Statistics & stat) + explicit BasicStatistics(const DTWorkload::Statistics & stat) { init_sec = stat.init_store_sec; for (const auto & t : stat.write_stats) @@ -76,7 +78,7 @@ struct BasicStatistics read_sec = stat.total_read_usec.load(std::memory_order_relaxed) / 1000000.0; } - BasicStatistics() {} + BasicStatistics() = default; }; void print(Poco::Logger * log, uint64_t i, const BasicStatistics & stat, WorkloadOptions & opts) @@ -121,9 +123,8 @@ std::shared_ptr createHandleTable(WorkloadOptions & opts) void run(WorkloadOptions & opts) { - init(opts); auto * log = &Poco::Logger::get("DTWorkload_main"); - LOG_INFO(log, opts.toString()); + LOG_FMT_INFO(log, "{}", opts.toString()); auto data_dirs = DB::tests::TiFlashTestEnv::getGlobalContext().getPathPool().listPaths(); std::vector basic_stats; try @@ -143,17 +144,9 @@ void run(WorkloadOptions & opts) } removeData(log, data_dirs); } - catch (const DB::Exception & e) - { - LOG_ERROR(log, e.message()); - } - catch (const std::exception & e) - { - LOG_ERROR(log, e.what()); - } catch (...) { - LOG_ERROR(log, "Unknow Exception"); + DB::tryLogCurrentException("exception thrown"); } outputResult(log, basic_stats, opts); @@ -262,7 +255,11 @@ int DTWorkload::mainEntry(int argc, char ** argv) return -1; } - TiFlashTestEnv::initializeGlobalContext(opts.work_dirs); + // need to init logger before creating global context, + // or the logging in global context won't be output to + // the log file + init(opts); + TiFlashTestEnv::initializeGlobalContext(opts.work_dirs, opts.enable_ps_v3); if (opts.testing_type == "daily_perf") { diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/Options.cpp b/dbms/src/Storages/DeltaMerge/tools/workload/Options.cpp index c3fb7a673ba..bddd796fe73 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/Options.cpp +++ b/dbms/src/Storages/DeltaMerge/tools/workload/Options.cpp @@ -83,6 +83,8 @@ std::pair WorkloadOptions::parseOptions(int argc, char * argv ("testing_type", value()->default_value(""), "daily_perf/daily_random") // // ("log_write_request", value()->default_value(false), "") // + // + ("enable_ps_v3", value()->default_value(false), "") // ; boost::program_options::variables_map vm; @@ -145,6 +147,7 @@ std::pair WorkloadOptions::parseOptions(int argc, char * argv testing_type = vm["testing_type"].as(); log_write_request = vm["log_write_request"].as(); + enable_ps_v3 = vm["enable_ps_v3"].as(); return {true, toString()}; } @@ -159,4 +162,4 @@ void WorkloadOptions::initFailpoints() const #endif } -} // namespace DB::DM::tests \ No newline at end of file +} // namespace DB::DM::tests diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/Options.h b/dbms/src/Storages/DeltaMerge/tools/workload/Options.h index e9f3bad51db..175246df01d 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/Options.h +++ b/dbms/src/Storages/DeltaMerge/tools/workload/Options.h @@ -53,8 +53,10 @@ struct WorkloadOptions bool log_write_request; + bool enable_ps_v3; + std::string toString(std::string seperator = "\n") const; std::pair parseOptions(int argc, char * argv[]); void initFailpoints() const; }; -} // namespace DB::DM::tests \ No newline at end of file +} // namespace DB::DM::tests diff --git a/dbms/src/Storages/Page/PageStorage.cpp b/dbms/src/Storages/Page/PageStorage.cpp index 825f10da373..7f14d905858 100644 --- a/dbms/src/Storages/Page/PageStorage.cpp +++ b/dbms/src/Storages/Page/PageStorage.cpp @@ -14,6 +14,7 @@ #include #include +#include namespace DB { @@ -21,9 +22,13 @@ PageStoragePtr PageStorage::create( String name, PSDiskDelegatorPtr delegator, const PageStorage::Config & config, - const FileProviderPtr & file_provider) + const FileProviderPtr & file_provider, + bool use_v3) { - return std::make_shared(name, delegator, config, file_provider); + if (use_v3) + return std::make_shared(name, delegator, config, file_provider); + else + return std::make_shared(name, delegator, config, file_provider); } } // namespace DB diff --git a/dbms/src/Storages/Page/PageStorage.h b/dbms/src/Storages/Page/PageStorage.h index 4521ecb3140..6d97e21acd9 100644 --- a/dbms/src/Storages/Page/PageStorage.h +++ b/dbms/src/Storages/Page/PageStorage.h @@ -151,7 +151,8 @@ class PageStorage : private boost::noncopyable String name, PSDiskDelegatorPtr delegator, const PageStorage::Config & config, - const FileProviderPtr & file_provider); + const FileProviderPtr & file_provider, + bool use_v3 = false); PageStorage( String name, @@ -199,8 +200,9 @@ class PageStorage : private boost::noncopyable // We may skip the GC to reduce useless reading by default. virtual bool gc(bool not_skip = false, const WriteLimiterPtr & write_limiter = nullptr, const ReadLimiterPtr & read_limiter = nullptr) = 0; - // Register external pages GC callbacks + // Register and unregister external pages GC callbacks virtual void registerExternalPagesCallbacks(const ExternalPageCallbacks & callbacks) = 0; + virtual void unregisterExternalPagesCallbacks(NamespaceId /*ns_id*/){}; #ifndef DBMS_PUBLIC_GTEST protected: diff --git a/dbms/src/Storages/Page/V3/BlobStore.cpp b/dbms/src/Storages/Page/V3/BlobStore.cpp index 23173914010..5e35eeda8fb 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.cpp +++ b/dbms/src/Storages/Page/V3/BlobStore.cpp @@ -212,11 +212,17 @@ void BlobStore::remove(const PageEntriesV3 & del_entries) if (entry.size == 0) { continue; - // throw Exception(fmt::format("Invaild entry. entry size 0. [id={}] [offset={}]", - // entry.file_id, - // entry.offset)); } - removePosFromStats(entry.file_id, entry.offset, entry.size); + + try + { + removePosFromStats(entry.file_id, entry.offset, entry.size); + } + catch (DB::Exception & e) + { + e.addMessage(fmt::format("while removing entry [entry={}]", toDebugString(entry))); + e.rethrow(); + } } } diff --git a/dbms/src/Storages/Page/V3/PageDirectory.cpp b/dbms/src/Storages/Page/V3/PageDirectory.cpp index c659fdeb88d..74aa0dce2d5 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectory.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -681,7 +682,7 @@ SnapshotsStatistics PageDirectory::getSnapshotsStat() const return stat; } -PageIDAndEntryV3 PageDirectory::get(PageIdV3Internal page_id, const PageDirectorySnapshotPtr & snap) const +PageIDAndEntryV3 PageDirectory::get(PageIdV3Internal page_id, const PageDirectorySnapshotPtr & snap, bool throw_on_not_exist) const { PageEntryV3 entry_got; @@ -696,7 +697,14 @@ PageIDAndEntryV3 PageDirectory::get(PageIdV3Internal page_id, const PageDirector iter = mvcc_table_directory.find(id_to_resolve); if (iter == mvcc_table_directory.end()) { - throw Exception(fmt::format("Invalid page id, entry not exist [page_id={}] [resolve_id={}]", page_id, id_to_resolve), ErrorCodes::PS_ENTRY_NOT_EXISTS); + if (throw_on_not_exist) + { + throw Exception(fmt::format("Invalid page id, entry not exist [page_id={}] [resolve_id={}]", page_id, id_to_resolve), ErrorCodes::PS_ENTRY_NOT_EXISTS); + } + else + { + return PageIDAndEntryV3{page_id, PageEntryV3{.file_id = INVALID_BLOBFILE_ID}}; + } } } auto [need_collapse, next_id_to_resolve, next_ver_to_resolve] = iter->second->resolveToPageId(ver_to_resolve.sequence, id_to_resolve != page_id, &entry_got); @@ -834,7 +842,10 @@ PageId PageDirectory::getMaxId(NamespaceId ns_id) const // iter is not at the beginning and mvcc_table_directory is not empty, // so iter-- must be a valid iterator, and it's the largest page id which is smaller than the target page id. iter--; - return iter->first.low; + if (iter->first.high == ns_id) + return iter->first.low; + else + return 0; } } diff --git a/dbms/src/Storages/Page/V3/PageDirectory.h b/dbms/src/Storages/Page/V3/PageDirectory.h index a6349687335..5436094209e 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.h +++ b/dbms/src/Storages/Page/V3/PageDirectory.h @@ -294,11 +294,15 @@ class PageDirectory SnapshotsStatistics getSnapshotsStat() const; - PageIDAndEntryV3 get(PageIdV3Internal page_id, const PageDirectorySnapshotPtr & snap) const; + PageIDAndEntryV3 get(PageIdV3Internal page_id, const PageDirectorySnapshotPtr & snap, bool throw_on_not_exist = true) const; PageIDAndEntryV3 get(PageIdV3Internal page_id, const DB::PageStorageSnapshotPtr & snap) const { return get(page_id, toConcreteSnapshot(snap)); } + PageIDAndEntryV3 getOrNull(PageIdV3Internal page_id, const DB::PageStorageSnapshotPtr & snap) const + { + return get(page_id, toConcreteSnapshot(snap), /*throw_on_not_exist=*/false); + } PageIDAndEntriesV3 get(const PageIdV3Internals & page_ids, const PageDirectorySnapshotPtr & snap) const; PageIDAndEntriesV3 get(const PageIdV3Internals & page_ids, const DB::PageStorageSnapshotPtr & snap) const diff --git a/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp b/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp index d0c6b6f2078..afa8761b6aa 100644 --- a/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include diff --git a/dbms/src/Storages/Page/V3/PageStorageImpl.cpp b/dbms/src/Storages/Page/V3/PageStorageImpl.cpp index 25a3f45cec3..2244472c5a2 100644 --- a/dbms/src/Storages/Page/V3/PageStorageImpl.cpp +++ b/dbms/src/Storages/Page/V3/PageStorageImpl.cpp @@ -101,13 +101,14 @@ DB::PageEntry PageStorageImpl::getEntry(NamespaceId ns_id, PageId page_id, Snaps try { - const auto & [id, entry] = page_directory->get(buildV3Id(ns_id, page_id), snapshot); + const auto & [id, entry] = page_directory->getOrNull(buildV3Id(ns_id, page_id), snapshot); (void)id; // TODO : after `PageEntry` in page.h been moved to v2. // Then we don't copy from V3 to V2 format PageEntry entry_ret; entry_ret.file_id = entry.file_id; entry_ret.offset = entry.offset; + entry_ret.tag = entry.tag; entry_ret.size = entry.size; entry_ret.field_offsets = entry.field_offsets; entry_ret.checksum = entry.checksum; @@ -211,10 +212,10 @@ bool PageStorageImpl::gc(bool /*not_skip*/, const WriteLimiterPtr & write_limite std::scoped_lock lock{callbacks_mutex}; if (!callbacks_container.empty()) { - for (const auto & callbacks : callbacks_container) + for (const auto & [ns_id, callbacks] : callbacks_container) { auto pending_external_pages = callbacks.scanner(); - auto alive_external_ids = page_directory->getAliveExternalIds(callbacks.ns_id); + auto alive_external_ids = page_directory->getAliveExternalIds(ns_id); callbacks.remover(pending_external_pages, alive_external_ids); } } @@ -225,6 +226,7 @@ bool PageStorageImpl::gc(bool /*not_skip*/, const WriteLimiterPtr & write_limite // And get the expired entries. [[maybe_unused]] bool is_snapshot_dumped = page_directory->tryDumpSnapshot(write_limiter); const auto & del_entries = page_directory->gcInMemEntries(); + LOG_FMT_DEBUG(log, "Remove entries from memory [num_entries={}]", del_entries.size()); // 2. Remove the expired entries in BlobStore. // It won't delete the data on the disk. @@ -279,13 +281,30 @@ void PageStorageImpl::registerExternalPagesCallbacks(const ExternalPageCallbacks assert(callbacks.scanner != nullptr); assert(callbacks.remover != nullptr); assert(callbacks.ns_id != MAX_NAMESPACE_ID); - callbacks_container.push_back(callbacks); + assert(callbacks_container.count(callbacks.ns_id) == 0); + callbacks_container.emplace(callbacks.ns_id, callbacks); } -void PageStorageImpl::clearExternalPagesCallbacks() +void PageStorageImpl::unregisterExternalPagesCallbacks(NamespaceId ns_id) { std::scoped_lock lock{callbacks_mutex}; - callbacks_container.clear(); + callbacks_container.erase(ns_id); +} + +const String PageStorageImpl::manifests_file_name = "manifests"; + +bool PageStorageImpl::isManifestsFileExists(const String & path) +{ + Poco::File file(fmt::format("{}/{}", path, manifests_file_name)); + return file.exists(); +} + +void PageStorageImpl::createManifestsFileIfNeed(const String & path) +{ + Poco::File dir(path); + dir.createDirectories(); + Poco::File file(fmt::format("{}/{}", path, manifests_file_name)); + file.createFile(); } } // namespace PS::V3 diff --git a/dbms/src/Storages/Page/V3/PageStorageImpl.h b/dbms/src/Storages/Page/V3/PageStorageImpl.h index a18d7118a28..e7f2ca62b66 100644 --- a/dbms/src/Storages/Page/V3/PageStorageImpl.h +++ b/dbms/src/Storages/Page/V3/PageStorageImpl.h @@ -65,7 +65,12 @@ class PageStorageImpl : public DB::PageStorage void registerExternalPagesCallbacks(const ExternalPageCallbacks & callbacks) override; - void clearExternalPagesCallbacks(); + void unregisterExternalPagesCallbacks(NamespaceId ns_id) override; + + static bool isManifestsFileExists(const String & path); + + static void createManifestsFileIfNeed(const String & path); + #ifndef NDEBUG // Just for tests, refactor them out later DB::PageStorage::SnapshotPtr getSnapshot() { return getSnapshot(""); } @@ -93,8 +98,10 @@ class PageStorageImpl : public DB::PageStorage std::atomic gc_is_running = false; + const static String manifests_file_name; + std::mutex callbacks_mutex; - using ExternalPageCallbacksContainer = std::vector; + using ExternalPageCallbacksContainer = std::unordered_map; ExternalPageCallbacksContainer callbacks_container; }; diff --git a/dbms/src/Storages/Page/V3/WAL/serialize.cpp b/dbms/src/Storages/Page/V3/WAL/serialize.cpp index f3bd86c03da..a6afbb357f6 100644 --- a/dbms/src/Storages/Page/V3/WAL/serialize.cpp +++ b/dbms/src/Storages/Page/V3/WAL/serialize.cpp @@ -41,6 +41,7 @@ inline void serializeEntryTo(const PageEntryV3 & entry, WriteBuffer & buf) writeIntBinary(entry.offset, buf); writeIntBinary(entry.size, buf); writeIntBinary(entry.checksum, buf); + writeIntBinary(entry.tag, buf); // fieldsOffset TODO: compression on `fieldsOffset` writeIntBinary(entry.field_offsets.size(), buf); for (const auto & [off, checksum] : entry.field_offsets) @@ -56,6 +57,7 @@ inline void deserializeEntryFrom(ReadBuffer & buf, PageEntryV3 & entry) readIntBinary(entry.offset, buf); readIntBinary(entry.size, buf); readIntBinary(entry.checksum, buf); + readIntBinary(entry.tag, buf); // fieldsOffset PageFieldOffsetChecksums field_offsets; UInt64 size_field_offsets = 0; diff --git a/dbms/src/Storages/Page/V3/WALStore.cpp b/dbms/src/Storages/Page/V3/WALStore.cpp index 215bd03b84a..19c08b9500f 100644 --- a/dbms/src/Storages/Page/V3/WALStore.cpp +++ b/dbms/src/Storages/Page/V3/WALStore.cpp @@ -13,16 +13,7 @@ // limitations under the License. #include -#include -#include #include -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -185,6 +176,7 @@ bool WALStore::saveSnapshot(FilesSnapshot && files_snap, PageEntriesEdit && dire if (files_snap.persisted_log_files.empty()) return false; + LOG_FMT_INFO(logger, "Saving directory snapshot"); { // Use {largest_log_num + 1, 1} to save the `edit` const auto log_num = files_snap.persisted_log_files.rbegin()->log_num; 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 46a5f6fc3ff..1be4a01dc11 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp @@ -1951,6 +1951,42 @@ try } CATCH +TEST_F(PageDirectoryTest, GetMaxId) +try +{ + NamespaceId small = 20; + NamespaceId medium = 50; + NamespaceId large = 100; + ASSERT_EQ(dir->getMaxId(small), 0); + ASSERT_EQ(dir->getMaxId(medium), 0); + ASSERT_EQ(dir->getMaxId(large), 0); + + PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry2{.file_id = 2, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + { + PageEntriesEdit edit; + edit.put(buildV3Id(small, 1), entry1); + edit.put(buildV3Id(large, 2), entry2); + dir->apply(std::move(edit)); + ASSERT_EQ(dir->getMaxId(small), 1); + ASSERT_EQ(dir->getMaxId(medium), 0); + ASSERT_EQ(dir->getMaxId(large), 2); + } + + PageEntryV3 entry3{.file_id = 3, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry4{.file_id = 4, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + { + PageEntriesEdit edit; + edit.put(buildV3Id(medium, 300), entry1); + edit.put(buildV3Id(medium, 320), entry2); + dir->apply(std::move(edit)); + ASSERT_EQ(dir->getMaxId(small), 1); + ASSERT_EQ(dir->getMaxId(medium), 320); + ASSERT_EQ(dir->getMaxId(large), 2); + } +} +CATCH + #undef INSERT_ENTRY_TO #undef INSERT_ENTRY #undef INSERT_ENTRY_ACQ_SNAP 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 6bb9c7b0ce9..58dbdd438ee 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp @@ -25,6 +25,7 @@ #include #include #include +#include namespace DB { @@ -388,6 +389,11 @@ TEST_F(PageStorageTest, IngestFile) page_storage->registerExternalPagesCallbacks(callbacks); page_storage->gc(); ASSERT_EQ(times_remover_called, 1); + page_storage->gc(); + ASSERT_EQ(times_remover_called, 2); + page_storage->unregisterExternalPagesCallbacks(callbacks.ns_id); + page_storage->gc(); + ASSERT_EQ(times_remover_called, 2); } // TBD : enable after wal apply and restore @@ -848,7 +854,7 @@ try EXPECT_EQ(living_page_ids.size(), 1); EXPECT_GT(living_page_ids.count(0), 0); }; - page_storage->clearExternalPagesCallbacks(); + page_storage->unregisterExternalPagesCallbacks(callbacks.ns_id); page_storage->registerExternalPagesCallbacks(callbacks); { SCOPED_TRACE("gc with snapshot released"); @@ -899,6 +905,5 @@ try } CATCH - } // namespace PS::V3::tests } // namespace DB diff --git a/dbms/src/Storages/Page/V3/tests/gtest_wal_store.cpp b/dbms/src/Storages/Page/V3/tests/gtest_wal_store.cpp index 21ca60ea089..5d9e8d001b7 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_wal_store.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_wal_store.cpp @@ -15,6 +15,8 @@ #include #include #include +#include +#include #include #include #include diff --git a/dbms/src/Storages/Page/WriteBatch.h b/dbms/src/Storages/Page/WriteBatch.h index 14e429e2e2d..69da8d96cef 100644 --- a/dbms/src/Storages/Page/WriteBatch.h +++ b/dbms/src/Storages/Page/WriteBatch.h @@ -99,10 +99,16 @@ class WriteBatch : private boost::noncopyable off += data_sz; } if (unlikely(!data_sizes.empty() && off != size)) - throw Exception("Try to put Page" + DB::toString(page_id) + " with " + DB::toString(data_sizes.size()) - + " fields, but page size and filelds total size not match, page_size: " + DB::toString(size) - + ", all fields size: " + DB::toString(off), + { + throw Exception(fmt::format( + "Try to put Page with fields, but page size and fields total size not match " + "[page_id={}] [num_fields={}] [page_size={}] [all_fields_size={}]", + page_id, + data_sizes.size(), + size, + off), ErrorCodes::LOGICAL_ERROR); + } Write w{WriteType::PUT, page_id, tag, read_buffer, size, 0, std::move(offsets), 0, 0, {}}; total_data_size += size; @@ -130,8 +136,8 @@ class WriteBatch : private boost::noncopyable total_data_size += size; } - // Upsering a page{page_id} to PageFile{file_id}. This type of upsert is a simple mark and - // only used for checkpoint. That page will be overwriten by WriteBatch with larger sequence, + // Upserting a page{page_id} to PageFile{file_id}. This type of upsert is a simple mark and + // only used for checkpoint. That page will be overwritten by WriteBatch with larger sequence, // so we don't need to write page's data. void upsertPage(PageId page_id, UInt64 tag, diff --git a/dbms/src/Storages/PathPool.cpp b/dbms/src/Storages/PathPool.cpp index 277a1799efe..2dda5ed30f7 100644 --- a/dbms/src/Storages/PathPool.cpp +++ b/dbms/src/Storages/PathPool.cpp @@ -76,6 +76,12 @@ PathPool::PathPool( kvstore_paths.emplace_back(std::move(p)); } } + for (const auto & s : latest_data_paths) + { + // Get a normalized path without trailing '/' + auto p = getNormalizedPath(s + "/page"); + global_page_paths.emplace_back(std::move(p)); + } } StoragePathPool PathPool::withTable(const String & database_, const String & table_, bool path_need_database_name_) const @@ -101,6 +107,16 @@ PSDiskDelegatorPtr PathPool::getPSDiskDelegatorRaft() return std::make_shared(*this); } +PSDiskDelegatorPtr PathPool::getPSDiskDelegatorGlobalMulti(const String & prefix) const +{ + return std::make_shared(*this, prefix); +} + +PSDiskDelegatorPtr PathPool::getPSDiskDelegatorGlobalSingle(const String & prefix) const +{ + return std::make_shared(*this, prefix); +} + //========================================================================================== // StoragePathPool //========================================================================================== @@ -329,7 +345,7 @@ String genericChoosePath(const std::vector & paths, const PathCapacityMetrics } } - double rand_number = static_cast(rand()) / RAND_MAX; + double rand_number = static_cast(rand()) / RAND_MAX; // NOLINT(cert-msc50-cpp) double ratio_sum = 0.0; for (size_t i = 0; i < ratio.size(); i++) { @@ -736,4 +752,96 @@ void PSDiskDelegatorRaft::removePageFile(const PageFileIdAndLevel & id_lvl, size } } +//========================================================================================== +// Global page data +//========================================================================================== + +size_t PSDiskDelegatorGlobalMulti::numPaths() const +{ + return pool.listGlobalPagePaths().size(); +} + +String PSDiskDelegatorGlobalMulti::defaultPath() const +{ + return fmt::format("{}/{}", pool.listGlobalPagePaths()[default_path_index], path_prefix); +} + +Strings PSDiskDelegatorGlobalMulti::listPaths() const +{ + // The delta data could be stored in all direcotries. + std::vector paths; + for (const auto & path : pool.listGlobalPagePaths()) + { + paths.push_back(fmt::format("{}/{}", path, path_prefix)); + } + return paths; +} + +String PSDiskDelegatorGlobalMulti::choosePath(const PageFileIdAndLevel & /*id_lvl*/) +{ + throw Exception("Not Implemented", ErrorCodes::NOT_IMPLEMENTED); +} + +size_t PSDiskDelegatorGlobalMulti::addPageFileUsedSize( + const PageFileIdAndLevel & /*id_lvl*/, + size_t /*size_to_add*/, + const String & /*pf_parent_path*/, + bool /*need_insert_location*/) +{ + throw Exception("Not Implemented", ErrorCodes::NOT_IMPLEMENTED); +} + +String PSDiskDelegatorGlobalMulti::getPageFilePath(const PageFileIdAndLevel & /*id_lvl*/) const +{ + throw Exception("Not Implemented", ErrorCodes::NOT_IMPLEMENTED); +} + +void PSDiskDelegatorGlobalMulti::removePageFile(const PageFileIdAndLevel & /*id_lvl*/, size_t /*file_size*/, bool /*meta_left*/, bool /*remove_from_default_path*/) +{ + throw Exception("Not Implemented", ErrorCodes::NOT_IMPLEMENTED); +} + + +size_t PSDiskDelegatorGlobalSingle::numPaths() const +{ + return 1; +} + +String PSDiskDelegatorGlobalSingle::defaultPath() const +{ + return fmt::format("{}/{}", pool.listGlobalPagePaths()[0], path_prefix); +} + +Strings PSDiskDelegatorGlobalSingle::listPaths() const +{ + // only stored in the first path. + std::vector paths; + paths.push_back(fmt::format("{}/{}", pool.listGlobalPagePaths()[0], path_prefix)); + return paths; +} + +String PSDiskDelegatorGlobalSingle::choosePath(const PageFileIdAndLevel & /*id_lvl*/) +{ + throw Exception("Not Implemented", ErrorCodes::NOT_IMPLEMENTED); +} + +size_t PSDiskDelegatorGlobalSingle::addPageFileUsedSize( + const PageFileIdAndLevel & /*id_lvl*/, + size_t /*size_to_add*/, + const String & /*pf_parent_path*/, + bool /*need_insert_location*/) +{ + throw Exception("Not Implemented", ErrorCodes::NOT_IMPLEMENTED); +} + +String PSDiskDelegatorGlobalSingle::getPageFilePath(const PageFileIdAndLevel & /*id_lvl*/) const +{ + throw Exception("Not Implemented", ErrorCodes::NOT_IMPLEMENTED); +} + +void PSDiskDelegatorGlobalSingle::removePageFile(const PageFileIdAndLevel & /*id_lvl*/, size_t /*file_size*/, bool /*meta_left*/, bool /*remove_from_default_path*/) +{ + throw Exception("Not Implemented", ErrorCodes::NOT_IMPLEMENTED); +} + } // namespace DB diff --git a/dbms/src/Storages/PathPool.h b/dbms/src/Storages/PathPool.h index ccc3f900bc1..399551fa241 100644 --- a/dbms/src/Storages/PathPool.h +++ b/dbms/src/Storages/PathPool.h @@ -72,10 +72,17 @@ class PathPool // User should keep the pointer to track the PageFileID -> path index mapping. PSDiskDelegatorPtr getPSDiskDelegatorRaft(); + PSDiskDelegatorPtr getPSDiskDelegatorGlobalMulti(const String & prefix) const; + PSDiskDelegatorPtr getPSDiskDelegatorGlobalSingle(const String & prefix) const; + public: /// Methods for the root PathPool /// Strings listPaths() const; + const Strings & listKVStorePaths() const { return kvstore_paths; } + + const Strings & listGlobalPagePaths() const { return global_page_paths; } + public: struct PageFileIdLvlHasher { @@ -92,6 +99,7 @@ class PathPool Strings main_data_paths; Strings latest_data_paths; Strings kvstore_paths; + Strings global_page_paths; bool enable_raft_compatible_mode; @@ -253,6 +261,71 @@ class PSDiskDelegatorRaft : public PSDiskDelegator const UInt32 default_path_index = 0; }; +class PSDiskDelegatorGlobalMulti : public PSDiskDelegator +{ +public: + PSDiskDelegatorGlobalMulti(const PathPool & pool_, String prefix) + : pool(pool_) + , path_prefix(std::move(prefix)) + {} + + size_t numPaths() const override; + + String defaultPath() const override; + + Strings listPaths() const override; + + String choosePath(const PageFileIdAndLevel & id_lvl) override; + + size_t addPageFileUsedSize( + const PageFileIdAndLevel & id_lvl, + size_t size_to_add, + const String & pf_parent_path, + bool need_insert_location) override; + + String getPageFilePath(const PageFileIdAndLevel & id_lvl) const override; + + void removePageFile(const PageFileIdAndLevel & id_lvl, size_t file_size, bool meta_left, bool remove_from_default_path) override; + +private: + const PathPool & pool; + const String path_prefix; + // PageFileID -> path index + PathPool::PageFilePathMap page_path_map; + const UInt32 default_path_index = 0; +}; + +class PSDiskDelegatorGlobalSingle : public PSDiskDelegator +{ +public: + PSDiskDelegatorGlobalSingle(const PathPool & pool_, String prefix) + : pool(pool_) + , path_prefix(std::move(prefix)) + {} + + size_t numPaths() const override; + + String defaultPath() const override; + + Strings listPaths() const override; + + String choosePath(const PageFileIdAndLevel & id_lvl) override; + + size_t addPageFileUsedSize( + const PageFileIdAndLevel & id_lvl, + size_t size_to_add, + const String & pf_parent_path, + bool need_insert_location) override; + + String getPageFilePath(const PageFileIdAndLevel & id_lvl) const override; + + void removePageFile(const PageFileIdAndLevel & id_lvl, size_t file_size, bool meta_left, bool remove_from_default_path) override; + +private: + const PathPool & pool; + const String path_prefix; +}; + /// A class to manage paths for the specified storage. class StoragePathPool { diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 8d9bbd64eb5..93d20baaec7 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -1202,8 +1202,7 @@ void StorageDeltaMerge::rename( std::move(handle_column_define), is_common_handle, rowkey_column_size, - settings, - tidb_table_info.id); + settings); } String StorageDeltaMerge::getTableName() const @@ -1520,8 +1519,7 @@ DeltaMergeStorePtr & StorageDeltaMerge::getAndMaybeInitStore() std::move(table_column_info->handle_column_define), is_common_handle, rowkey_column_size, - DeltaMergeStore::Settings(), - tidb_table_info.id); + DeltaMergeStore::Settings()); table_column_info.reset(nullptr); store_inited.store(true, std::memory_order_release); } diff --git a/dbms/src/Storages/Transaction/RegionPersister.cpp b/dbms/src/Storages/Transaction/RegionPersister.cpp index 686594b9698..6e9d4215e94 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.cpp +++ b/dbms/src/Storages/Transaction/RegionPersister.cpp @@ -15,14 +15,18 @@ #include #include #include +#include #include #include #include +#include #include #include #include #include +#include + namespace DB { namespace ErrorCodes @@ -60,9 +64,12 @@ void RegionPersister::computeRegionWriteBuffer(const Region & region, RegionCach std::tie(region_size, applied_index) = region.serialize(buffer); if (unlikely(region_size > static_cast(std::numeric_limits::max()))) { - LOG_WARNING(&Poco::Logger::get("RegionPersister"), - "Persisting big region: " << region.toString() << " with data info: " << region.dataInfo() << ", serialized size " - << region_size); + LOG_FMT_WARNING( + &Poco::Logger::get("RegionPersister"), + "Persisting big region: {} with data info: {}, serialized size {}", + region.toString(), + region.dataInfo(), + region_size); } } @@ -152,11 +159,13 @@ PS::V1::PageStorage::Config getV1PSConfig(const PS::V2::PageStorage::Config & co return c; } -RegionMap RegionPersister::restore(const TiFlashRaftProxyHelper * proxy_helper, PS::V2::PageStorage::Config config) +RegionMap RegionPersister::restore(const TiFlashRaftProxyHelper * proxy_helper, PageStorage::Config config) { { auto & path_pool = global_context.getPathPool(); auto delegator = path_pool.getPSDiskDelegatorRaft(); + // If the GlobalStoragePool is initialized, then use v3 format + bool use_v3_format = global_context.getGlobalStoragePool() != nullptr; // If there is no PageFile with basic version binary format, use the latest version of PageStorage. auto detect_binary_version = DB::PS::V2::PageStorage::getMaxDataVersion(global_context.getFileProvider(), delegator); bool run_in_compatible_mode = path_pool.isRaftCompatibleModeEnabled() && (detect_binary_version == PageFormat::V1); @@ -164,13 +173,25 @@ RegionMap RegionPersister::restore(const TiFlashRaftProxyHelper * proxy_helper, fiu_do_on(FailPoints::force_enable_region_persister_compatible_mode, { run_in_compatible_mode = true; }); fiu_do_on(FailPoints::force_disable_region_persister_compatible_mode, { run_in_compatible_mode = false; }); - if (!run_in_compatible_mode) + if (use_v3_format) + { + mergeConfigFromSettings(global_context.getSettingsRef(), config); + + LOG_FMT_INFO(log, "RegionPersister running in v3 mode"); + page_storage = std::make_unique( // + "RegionPersister", + delegator, + config, + global_context.getFileProvider()); + page_storage->restore(); + } + else if (!run_in_compatible_mode) { mergeConfigFromSettings(global_context.getSettingsRef(), config); config.num_write_slots = 4; // extend write slots to 4 at least - LOG_INFO(log, "RegionPersister running in normal mode"); - page_storage = std::make_unique( // + LOG_FMT_INFO(log, "RegionPersister running in v2 mode"); + page_storage = std::make_unique( "RegionPersister", delegator, config, @@ -179,9 +200,9 @@ RegionMap RegionPersister::restore(const TiFlashRaftProxyHelper * proxy_helper, } else { - LOG_INFO(log, "RegionPersister running in compatible mode"); + LOG_FMT_INFO(log, "RegionPersister running in v1 mode"); auto c = getV1PSConfig(config); - stable_page_storage = std::make_unique( // + stable_page_storage = std::make_unique( "RegionPersister", delegator->defaultPath(), c, @@ -220,7 +241,7 @@ bool RegionPersister::gc() { if (page_storage) { - PS::V2::PageStorage::Config config = getConfigFromSettings(global_context.getSettingsRef()); + PageStorage::Config config = getConfigFromSettings(global_context.getSettingsRef()); page_storage->reloadSettings(config); return page_storage->gc(false, nullptr, nullptr); } diff --git a/dbms/src/Storages/Transaction/RegionPersister.h b/dbms/src/Storages/Transaction/RegionPersister.h index bad7800198f..feb4353a0d0 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.h +++ b/dbms/src/Storages/Transaction/RegionPersister.h @@ -38,11 +38,8 @@ namespace V1 { class PageStorage; } -namespace V2 -{ -class PageStorage; -} } // namespace PS +class PageStorage; class RegionPersister final : private boost::noncopyable { @@ -70,7 +67,7 @@ class RegionPersister final : private boost::noncopyable #endif Context & global_context; - std::shared_ptr page_storage; + PageStoragePtr page_storage; std::shared_ptr stable_page_storage; // RegionPersister stores it's data individually, so the `ns_id` value doesn't matter diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index c33afbcc9cb..f60f27c8269 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -27,7 +27,7 @@ namespace DB::tests { std::unique_ptr TiFlashTestEnv::global_context = nullptr; -void TiFlashTestEnv::initializeGlobalContext(Strings testdata_path) +void TiFlashTestEnv::initializeGlobalContext(Strings testdata_path, bool enable_ps_v3) { // set itself as global context global_context = std::make_unique(DB::Context::createGlobal()); @@ -64,6 +64,9 @@ void TiFlashTestEnv::initializeGlobalContext(Strings testdata_path) raft_config.disable_bg_flush = true; global_context->createTMTContext(raft_config, pingcap::ClusterConfig()); + if (global_context->initializeGlobalStoragePoolIfNeed(global_context->getPathPool(), enable_ps_v3)) + LOG_FMT_INFO(&Poco::Logger::get("TiFlashTestEnv"), "PageStorage V3 enabled."); + global_context->setDeltaIndexManager(1024 * 1024 * 100 /*100MB*/); global_context->getTMTContext().restore(); diff --git a/dbms/src/TestUtils/TiFlashTestEnv.h b/dbms/src/TestUtils/TiFlashTestEnv.h index 7ca9f16fea1..65dad63d937 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.h +++ b/dbms/src/TestUtils/TiFlashTestEnv.h @@ -71,7 +71,7 @@ class TiFlashTestEnv static Strings findTestDataPath(const String & name) { const static std::vector SEARCH_PATH = {"../tests/testdata/", "/tests/testdata/"}; - for (auto & prefix : SEARCH_PATH) + for (const auto & prefix : SEARCH_PATH) { String path = prefix + name; if (auto f = Poco::File(path); f.exists() && f.isDirectory()) @@ -88,14 +88,13 @@ class TiFlashTestEnv static Context getContext(const DB::Settings & settings = DB::Settings(), Strings testdata_path = {}); - static void initializeGlobalContext(Strings testdata_path = {}); + static void initializeGlobalContext(Strings testdata_path = {}, bool enable_ps_v3 = false); static Context & getGlobalContext() { return *global_context; } static void shutdown(); -private: - static std::unique_ptr global_context; + TiFlashTestEnv() = delete; private: - TiFlashTestEnv() = delete; + static std::unique_ptr global_context; }; -} // namespace DB::tests \ No newline at end of file +} // namespace DB::tests From a48949344bb74d1a34ee3813eb41ff60aa174f10 Mon Sep 17 00:00:00 2001 From: Fu Zhe Date: Mon, 21 Mar 2022 11:36:32 +0800 Subject: [PATCH 5/9] *: Introduce Logger for replacing LogWithPrefix (#4354) ref pingcap/tiflash#4355 --- dbms/src/Common/Logger.h | 131 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 131 insertions(+) create mode 100644 dbms/src/Common/Logger.h diff --git a/dbms/src/Common/Logger.h b/dbms/src/Common/Logger.h new file mode 100644 index 00000000000..9791413cb7a --- /dev/null +++ b/dbms/src/Common/Logger.h @@ -0,0 +1,131 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include + +#include + +namespace DB +{ +class Logger; +using LoggerPtr = std::shared_ptr; + +/** + * Logger is to support identifiers based on Poco::Logger. + * + * Identifiers could be request_id, session_id, etc. They can be used in `LogSearch` when we want to + * glob all logs related to one request/session/query. + * + * Logger will print all identifiers at the front of each log record (and after the `source`). + * + * Interfaces in Logger are definitely the same with the Poco::Logger, so that they could use the same + * macro such as LOG_INFO() etc. + */ +class Logger : private boost::noncopyable +{ +public: + static LoggerPtr get(const std::string & source) + { + return std::make_shared(source, ""); + } + + static LoggerPtr get(const std::string & source, const std::string & identifier) + { + return std::make_shared(source, identifier); + } + + template + static LoggerPtr get(const std::string & source, Args &&... args) + { + FmtBuffer buf; + return getInternal(source, buf, std::forward(args)...); + } + + Logger(const std::string & source, const std::string & identifier) + : logger(&Poco::Logger::get(source)) + , id(identifier) + { + } + +#define M(level) \ + bool level() const { return logger->level(); } \ + void level(const std::string & msg) const \ + { \ + if (id.empty()) \ + logger->level(msg); \ + else \ + logger->level(wrapMsg(msg)); \ + } + + M(trace) + M(debug) + M(information) + M(warning) + M(error) + M(fatal) +#undef M + + void log(const Poco::Message & msg) const + { + if (id.empty()) + return logger->log(msg); + else + return logger->log(Poco::Message(msg, wrapMsg(msg.getText()))); + } + + void log(Poco::Message & msg) const + { + if (!id.empty()) + msg.setText(wrapMsg(msg.getText())); + return logger->log(msg); + } + + bool is(int level) const { return logger->is(level); } + + Poco::Channel * getChannel() const { return logger->getChannel(); } + + const std::string & name() const { return logger->name(); } + + const std::string & identifier() const { return id; } + + Poco::Logger * getLog() const { return logger; } + +private: + template + static LoggerPtr getInternal(const std::string & source, FmtBuffer & buf, T && first, Args &&... args) + { + buf.fmtAppend("{} ", std::forward(first)); + return getInternal(source, buf, std::forward(args)...); + } + + template + static LoggerPtr getInternal(const std::string & source, FmtBuffer & buf, T && identifier) + { + buf.fmtAppend("{}", std::forward(identifier)); + return get(source, buf.toString()); + } + + std::string wrapMsg(const std::string & msg) const + { + return fmt::format("{} {}", id, msg); + } + + Poco::Logger * logger; + const std::string id; +}; + +} // namespace DB From 0f38afb5c0e470624353112ede1d23ca0f729fa9 Mon Sep 17 00:00:00 2001 From: lidezhu <47731263+lidezhu@users.noreply.github.com> Date: Mon, 21 Mar 2022 12:22:31 +0800 Subject: [PATCH 6/9] fix cannot find path error when pass relative data path to dt_workload (#4349) close pingcap/tiflash#4347 --- dbms/src/TestUtils/TiFlashTestEnv.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index f60f27c8269..870db4217c4 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -47,6 +47,15 @@ void TiFlashTestEnv::initializeGlobalContext(Strings testdata_path, bool enable_ { testdata_path = {getTemporaryPath()}; } + else + { + Strings absolute_testdata_path; + for (const auto & path : testdata_path) + { + absolute_testdata_path.push_back(Poco::Path(path).absolute().toString()); + } + testdata_path.swap(absolute_testdata_path); + } global_context->initializePathCapacityMetric(0, testdata_path, {}, {}, {}); auto paths = getPathPool(testdata_path); From f2d36c1881565f77aa9e898ee08ffac68745f010 Mon Sep 17 00:00:00 2001 From: jiaqizho Date: Mon, 21 Mar 2022 14:24:32 +0800 Subject: [PATCH 7/9] Fix `max_caps` may not been update after GC (#4323) ref pingcap/tiflash#3594 --- dbms/src/Storages/Page/V3/BlobStore.cpp | 50 ++++++++++++--- dbms/src/Storages/Page/V3/BlobStore.h | 10 ++- dbms/src/Storages/Page/V3/spacemap/SpaceMap.h | 4 ++ .../Page/V3/spacemap/SpaceMapRBTree.cpp | 61 ++++++++++++++++--- .../Page/V3/spacemap/SpaceMapRBTree.h | 2 + .../Page/V3/spacemap/SpaceMapSTDMap.h | 48 ++++++++++++--- .../Page/V3/tests/gtest_blob_store.cpp | 32 +++++++--- .../Storages/Page/V3/tests/gtest_free_map.cpp | 19 ++++++ 8 files changed, 190 insertions(+), 36 deletions(-) diff --git a/dbms/src/Storages/Page/V3/BlobStore.cpp b/dbms/src/Storages/Page/V3/BlobStore.cpp index 5e35eeda8fb..97c4fe12704 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.cpp +++ b/dbms/src/Storages/Page/V3/BlobStore.cpp @@ -206,8 +206,10 @@ PageEntriesEdit BlobStore::write(DB::WriteBatch & wb, const WriteLimiterPtr & wr void BlobStore::remove(const PageEntriesV3 & del_entries) { + std::set blob_updated; for (const auto & entry : del_entries) { + blob_updated.insert(entry.file_id); // External page size is 0 if (entry.size == 0) { @@ -224,6 +226,25 @@ void BlobStore::remove(const PageEntriesV3 & del_entries) e.rethrow(); } } + + // After we remove postion of blob, we need recalculate the blob. + for (const auto & blob_id : blob_updated) + { + const auto & stat = blob_stats.blobIdToStat(blob_id, + /*restore_if_not_exist*/ false, + /*ignore_not_exist*/ true); + + // Some of blob may been removed. + // So if we can't use id find blob, just ignore it. + if (stat) + { + LOG_FMT_TRACE(log, "Blob begin to recalculate capability [blob_id={}]", blob_id); + { + auto lock = stat->lock(); + stat->recalculateCapacity(); + } + } + } } std::pair BlobStore::getPosFromStats(size_t size) @@ -233,7 +254,7 @@ std::pair BlobStore::getPosFromStats(size_t size) auto lock_stat = [size, this, &stat]() -> std::lock_guard { auto lock_stats = blob_stats.lock(); BlobFileId blob_file_id = INVALID_BLOBFILE_ID; - std::tie(stat, blob_file_id) = blob_stats.chooseStat(size, config.file_limit_size, lock_stats); + std::tie(stat, blob_file_id) = blob_stats.chooseStat(size, lock_stats); // No valid stat for puting data with `size`, create a new one if (stat == nullptr) @@ -763,13 +784,12 @@ void BlobStore::BlobStats::restoreByEntry(const PageEntryV3 & entry) void BlobStore::BlobStats::restore() { BlobFileId max_restored_file_id = 0; - std::set existing_file_ids; for (const auto & stat : stats_map) { stat->recalculateSpaceMap(); max_restored_file_id = std::max(stat->id, max_restored_file_id); - existing_file_ids.insert(stat->id); } + // restore `roll_id` roll_id = max_restored_file_id + 1; } @@ -852,7 +872,7 @@ void BlobStore::BlobStats::eraseStat(BlobFileId blob_file_id, const std::lock_gu eraseStat(std::move(stat), lock); } -std::pair BlobStore::BlobStats::chooseStat(size_t buf_size, UInt64 file_limit_size, const std::lock_guard &) +std::pair BlobStore::BlobStats::chooseStat(size_t buf_size, const std::lock_guard &) { BlobStatPtr stat_ptr = nullptr; double smallest_valid_rate = 2; @@ -867,10 +887,9 @@ std::pair BlobStore::BlobStats::chooseStat(size_t buf_s { if (!stat->isReadOnly() && stat->sm_max_caps >= buf_size - && stat->sm_total_size + buf_size < file_limit_size && stat->sm_valid_rate < smallest_valid_rate) { - smallest_valid_rate = stat->sm_valid_size; + smallest_valid_rate = stat->sm_valid_rate; stat_ptr = stat; } } @@ -883,7 +902,7 @@ std::pair BlobStore::BlobStats::chooseStat(size_t buf_s return std::make_pair(stat_ptr, INVALID_BLOBFILE_ID); } -BlobStatPtr BlobStore::BlobStats::blobIdToStat(BlobFileId file_id, bool restore_if_not_exist) +BlobStatPtr BlobStore::BlobStats::blobIdToStat(BlobFileId file_id, bool restore_if_not_exist, bool ignore_not_exist) { auto guard = lock(); for (auto & stat : stats_map) @@ -900,9 +919,14 @@ BlobStatPtr BlobStore::BlobStats::blobIdToStat(BlobFileId file_id, bool restore_ return createStatNotCheckingRoll(file_id, guard); } - throw Exception(fmt::format("Can't find BlobStat with [blob_id={}]", - file_id), - ErrorCodes::LOGICAL_ERROR); + if (!ignore_not_exist) + { + throw Exception(fmt::format("Can't find BlobStat with [blob_id={}]", + file_id), + ErrorCodes::LOGICAL_ERROR); + } + + return nullptr; } /********************* @@ -980,6 +1004,12 @@ void BlobStore::BlobStats::BlobStat::recalculateSpaceMap() sm_total_size = total_size; sm_valid_size = valid_size; sm_valid_rate = valid_size * 1.0 / total_size; + recalculateCapacity(); +} + +void BlobStore::BlobStats::BlobStat::recalculateCapacity() +{ + sm_max_caps = smap->updateAccurateMaxCapacity(); } } // namespace PS::V3 diff --git a/dbms/src/Storages/Page/V3/BlobStore.h b/dbms/src/Storages/Page/V3/BlobStore.h index caa68661bb3..ca65b2ac6c7 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.h +++ b/dbms/src/Storages/Page/V3/BlobStore.h @@ -124,6 +124,12 @@ class BlobStore : private Allocator * We still need to recalculate a `sm_total_size`/`sm_valid_size`/`sm_valid_rate`. */ void recalculateSpaceMap(); + + /** + * The `sm_max_cap` is not accurate after GC removes out-of-date data, or after restoring from disk. + * Caller should call this function to update the `sm_max_cap` so that we can reuse the space in this BlobStat. + */ + void recalculateCapacity(); }; using BlobStatPtr = std::shared_ptr; @@ -155,9 +161,9 @@ class BlobStore : private Allocator * The `INVALID_BLOBFILE_ID` means that you don't need create a new `BlobFile`. * */ - std::pair chooseStat(size_t buf_size, UInt64 file_limit_size, const std::lock_guard &); + std::pair chooseStat(size_t buf_size, const std::lock_guard &); - BlobStatPtr blobIdToStat(BlobFileId file_id, bool restore_if_not_exist = false); + BlobStatPtr blobIdToStat(BlobFileId file_id, bool restore_if_not_exist = false, bool ignore_not_exist = false); std::list getStats() const { diff --git a/dbms/src/Storages/Page/V3/spacemap/SpaceMap.h b/dbms/src/Storages/Page/V3/spacemap/SpaceMap.h index a52ae867427..f50eeee580a 100644 --- a/dbms/src/Storages/Page/V3/spacemap/SpaceMap.h +++ b/dbms/src/Storages/Page/V3/spacemap/SpaceMap.h @@ -96,6 +96,10 @@ class SpaceMap */ virtual UInt64 getRightMargin() = 0; + /** + * Get the accurate max capacity of the space map. + */ + virtual UInt64 updateAccurateMaxCapacity() = 0; /** * Return the size of file and the size contains valid data. diff --git a/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.cpp b/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.cpp index c19745845fb..33bb1304a19 100644 --- a/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.cpp +++ b/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.cpp @@ -664,6 +664,34 @@ std::pair RBTreeSpaceMap::searchInsertOffset(size_t size) return std::make_pair(offset, max_cap); } +UInt64 RBTreeSpaceMap::updateAccurateMaxCapacity() +{ + struct rb_node * node = nullptr; + struct SmapRbEntry * entry; + UInt64 max_offset = 0; + UInt64 max_cap = 0; + + node = rb_tree_first(&rb_tree->root); + if (node == nullptr) + { + return max_cap; + } + + for (; node != nullptr; node = rb_tree_next(node)) + { + entry = node_to_entry(node); + if (entry->count > max_cap) + { + max_offset = entry->start; + max_cap = entry->count; + } + } + + biggest_range = max_offset; + biggest_cap = max_cap; + return max_cap; +} + std::pair RBTreeSpaceMap::getSizes() const { struct rb_node * node = rb_tree_last(&rb_tree->root); @@ -674,18 +702,33 @@ std::pair RBTreeSpaceMap::getSizes() const } auto * entry = node_to_entry(node); - UInt64 total_size = entry->start - start; - UInt64 last_node_size = entry->count; - UInt64 valid_size = 0; - - for (node = rb_tree_first(&rb_tree->root); node != nullptr; node = rb_tree_next(node)) + if (entry->start + entry->count != end) { - entry = node_to_entry(node); - valid_size += entry->count; + UInt64 total_size = end - start; + UInt64 valid_size = total_size; + for (node = rb_tree_first(&rb_tree->root); node != nullptr; node = rb_tree_next(node)) + { + entry = node_to_entry(node); + valid_size -= entry->count; + } + + return std::make_pair(total_size, valid_size); } - valid_size = total_size - (valid_size - last_node_size); + else + { + UInt64 total_size = entry->start - start; + UInt64 last_node_size = entry->count; + UInt64 valid_size = 0; - return std::make_pair(total_size, valid_size); + for (node = rb_tree_first(&rb_tree->root); node != nullptr; node = rb_tree_next(node)) + { + entry = node_to_entry(node); + valid_size += entry->count; + } + valid_size = total_size - (valid_size - last_node_size); + + return std::make_pair(total_size, valid_size); + } } UInt64 RBTreeSpaceMap::getRightMargin() diff --git a/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.h b/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.h index 7b85a4fca0d..baeb6ef20b9 100644 --- a/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.h +++ b/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.h @@ -42,6 +42,8 @@ class RBTreeSpaceMap std::pair searchInsertOffset(size_t size) override; + UInt64 updateAccurateMaxCapacity() override; + std::pair getSizes() const override; UInt64 getRightMargin() override; diff --git a/dbms/src/Storages/Page/V3/spacemap/SpaceMapSTDMap.h b/dbms/src/Storages/Page/V3/spacemap/SpaceMapSTDMap.h index fe01d339bdb..3dd4396fbd6 100644 --- a/dbms/src/Storages/Page/V3/spacemap/SpaceMapSTDMap.h +++ b/dbms/src/Storages/Page/V3/spacemap/SpaceMapSTDMap.h @@ -78,17 +78,32 @@ class STDMapSpaceMap } const auto & last_free_block = free_map.rbegin(); - UInt64 total_size = last_free_block->first - start; - UInt64 last_free_block_size = last_free_block->second; - UInt64 valid_size = 0; - for (const auto & free_block : free_map) + if (last_free_block->first + last_free_block->second != end) { - valid_size += free_block.second; + UInt64 total_size = end - start; + UInt64 valid_size = total_size; + for (const auto & free_block : free_map) + { + valid_size -= free_block.second; + } + + return std::make_pair(total_size, valid_size); } - valid_size = total_size - (valid_size - last_free_block_size); + else + { + UInt64 total_size = last_free_block->first - start; + UInt64 last_free_block_size = last_free_block->second; - return std::make_pair(total_size, valid_size); + UInt64 valid_size = 0; + for (const auto & free_block : free_map) + { + valid_size += free_block.second; + } + valid_size = total_size - (valid_size - last_free_block_size); + + return std::make_pair(total_size, valid_size); + } } UInt64 getRightMargin() override @@ -257,6 +272,25 @@ class STDMapSpaceMap return std::make_pair(offset, hint_biggest_cap); } + UInt64 updateAccurateMaxCapacity() override + { + UInt64 max_offset = 0; + UInt64 max_cap = 0; + + for (const auto & [start, size] : free_map) + { + if (size > max_cap) + { + max_cap = size; + max_offset = start; + } + } + hint_biggest_offset = max_offset; + hint_biggest_cap = max_cap; + + return max_cap; + } + bool markFreeImpl(UInt64 offset, size_t length) override { auto it = free_map.find(offset); 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 770475f0ff1..b0d35cd4287 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp @@ -137,17 +137,17 @@ TEST_F(BlobStoreStatsTest, testStat) BlobStats stats(logger, config); - std::tie(stat, blob_file_id) = stats.chooseStat(10, BLOBFILE_LIMIT_SIZE, stats.lock()); + std::tie(stat, blob_file_id) = stats.chooseStat(10, stats.lock()); ASSERT_EQ(blob_file_id, 1); ASSERT_FALSE(stat); // still 0 - std::tie(stat, blob_file_id) = stats.chooseStat(10, BLOBFILE_LIMIT_SIZE, stats.lock()); + std::tie(stat, blob_file_id) = stats.chooseStat(10, stats.lock()); ASSERT_EQ(blob_file_id, 1); ASSERT_FALSE(stat); stats.createStat(0, stats.lock()); - std::tie(stat, blob_file_id) = stats.chooseStat(10, BLOBFILE_LIMIT_SIZE, stats.lock()); + std::tie(stat, blob_file_id) = stats.chooseStat(10, stats.lock()); ASSERT_EQ(blob_file_id, INVALID_BLOBFILE_ID); ASSERT_TRUE(stat); @@ -224,7 +224,7 @@ TEST_F(BlobStoreStatsTest, testFullStats) ASSERT_LE(stat->sm_valid_rate, 1); // Won't choose full one - std::tie(stat, blob_file_id) = stats.chooseStat(100, BLOBFILE_LIMIT_SIZE, stats.lock()); + std::tie(stat, blob_file_id) = stats.chooseStat(100, stats.lock()); ASSERT_EQ(blob_file_id, 2); ASSERT_FALSE(stat); @@ -242,7 +242,7 @@ TEST_F(BlobStoreStatsTest, testFullStats) // Then choose stat , it should return the stat id 3 // Stat which id is 2 is full. - std::tie(stat, blob_file_id) = stats.chooseStat(100, BLOBFILE_LIMIT_SIZE, stats.lock()); + std::tie(stat, blob_file_id) = stats.chooseStat(100, stats.lock()); ASSERT_EQ(blob_file_id, 3); ASSERT_FALSE(stat); } @@ -271,6 +271,7 @@ TEST_F(BlobStoreTest, Restore) try { const auto file_provider = DB::tests::TiFlashTestEnv::getContext().getFileProvider(); + config.file_limit_size = 2560; auto blob_store = BlobStore(file_provider, path, config); BlobFileId file_id1 = 10; @@ -301,9 +302,24 @@ try blob_store.blob_stats.restore(); } - auto blob_need_gc = blob_store.getGCStats(); - ASSERT_EQ(blob_need_gc.size(), 1); - EXPECT_EQ(blob_need_gc[0], 12); + // check spacemap updated + { + for (const auto & stat : blob_store.blob_stats.getStats()) + { + if (stat->id == file_id1) + { + ASSERT_EQ(stat->sm_total_size, 2560); + ASSERT_EQ(stat->sm_valid_size, 640); + ASSERT_EQ(stat->sm_max_caps, 1024); + } + else if (stat->id == file_id2) + { + ASSERT_EQ(stat->sm_total_size, 2560); + ASSERT_EQ(stat->sm_valid_size, 512); + ASSERT_EQ(stat->sm_max_caps, 2048); + } + } + } } CATCH 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 b28e5639ef2..a4f3fdbe948 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_free_map.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_free_map.cpp @@ -370,6 +370,25 @@ TEST_P(SpaceMapTest, TestGetSizes) } } + +TEST_P(SpaceMapTest, TestGetMaxCap) +{ + { + auto smap = SpaceMap::createSpaceMap(test_type, 0, 100); + ASSERT_TRUE(smap->markUsed(50, 10)); + ASSERT_TRUE(smap->markUsed(80, 10)); + + ASSERT_EQ(smap->updateAccurateMaxCapacity(), 50); + } + + { + auto smap = SpaceMap::createSpaceMap(test_type, 0, 100); + ASSERT_TRUE(smap->markUsed(0, 100)); + + ASSERT_EQ(smap->updateAccurateMaxCapacity(), 0); + } +} + INSTANTIATE_TEST_CASE_P( Type, SpaceMapTest, From e8d59623e03d4c9b6e9999675e7d797497cb1d43 Mon Sep 17 00:00:00 2001 From: Zhuhe Fang Date: Mon, 21 Mar 2022 14:58:32 +0800 Subject: [PATCH 8/9] Grafana: correct figure name (#4341) close pingcap/tiflash#4340 --- metrics/grafana/tiflash_summary.json | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 8d6d1ea8982..fcb72ac914a 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -2723,7 +2723,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "hard_limit_exceeded_count", + "title": "Hard Limit Exceeded Count", "tooltip": { "shared": true, "sort": 0, @@ -2842,7 +2842,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Waiting Duration", + "title": "Task Waiting Duration", "tooltip": { "shared": true, "sort": 0, From 55395525881f6f04e1e3a4af1dc939d47c4a9b3e Mon Sep 17 00:00:00 2001 From: JaySon Date: Mon, 21 Mar 2022 15:49:33 +0800 Subject: [PATCH 9/9] Update dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp --- .../Storages/Page/V3/PageDirectoryFactory.cpp | 31 ++----------------- 1 file changed, 3 insertions(+), 28 deletions(-) diff --git a/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp b/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp index ac76e67bb05..a32af387d31 100644 --- a/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp @@ -42,34 +42,9 @@ PageDirectoryPtr PageDirectoryFactory::create(FileProviderPtr & file_provider, P { (void)page_id; - // We can't use getEntry(max_seq) to get the entry. - // Otherwise, It is likely to cause data loss. - // for example: - // - // page id 4927 - // {type:5, create_ver: <0,0>, is_deleted: false, delete_ver: <0,0>, ori_page_id: 0.0, being_ref_count: 1, num_entries: 2} - // entry 0 - // sequence: 1802 - // epoch: 0 - // is del: false - // blob id: 5 - // offset: 77661628 - // size: 2381165 - // crc: 0x1D1BEF504F12D3A0 - // field offset size: 0 - // entry 1 - // sequence: 2121 - // epoch: 0 - // is del: true - // blob id: 0 - // offset: 0 - // size: 0 - // crc: 0x0 - // field offset size: 0 - // page id 5819 - // {type:6, create_ver: <2090,0>, is_deleted: false, delete_ver: <0,0>, ori_page_id: 0.4927, being_ref_count: 1, num_entries: 0} - // - // After getEntry, page id `4927` won't be restore by BlobStore. + // We should restore the entry to `blob_stats` even if it is marked as "deleted", + // or we will mistakenly reuse the space to write other blobs down into that space. + // So we need to use `getLastEntry` instead of `getEntry(version)` here. if (auto entry = entries->getLastEntry(); entry) { blob_stats->restoreByEntry(*entry);