From 28b517068fcb051ecc81470d278266c72ad7da5f Mon Sep 17 00:00:00 2001 From: Yixin Luo <18810541851@163.com> Date: Mon, 27 May 2024 15:25:50 +0800 Subject: [PATCH 1/2] [Enhancement] release metadata memory after rowsets have been compacted in pk table (#46067) Signed-off-by: luohaha <18810541851@163.com> (cherry picked from commit 0471705863e9ebee531873b7bc40992614aab0c8) # Conflicts: # be/src/storage/tablet_updates.cpp --- be/src/storage/tablet_updates.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/be/src/storage/tablet_updates.cpp b/be/src/storage/tablet_updates.cpp index 933a38596f13c..844cf7d2b2fd5 100644 --- a/be/src/storage/tablet_updates.cpp +++ b/be/src/storage/tablet_updates.cpp @@ -1766,9 +1766,19 @@ Status TabletUpdates::_do_compaction(std::unique_ptr* pinfo) { // 4. commit compaction EditVersion version; RETURN_IF_ERROR(_commit_compaction(pinfo, *output_rowset, &version)); +<<<<<<< HEAD // already committed, so we can ignore timeout error here std::unique_lock ul(_lock); _wait_for_version(version, 120000, ul); +======= + { + // already committed, so we can ignore timeout error here + std::unique_lock ul(_lock); + RETURN_IF_ERROR(_wait_for_version(version, 120000, ul)); + } + // Release metadata memory after rowsets have been compacted. + Rowset::close_rowsets(input_rowsets); +>>>>>>> 0471705863 ([Enhancement] release metadata memory after rowsets have been compacted in pk table (#46067)) return Status::OK(); } From b112234203bee8d259e7b94d0fc2e1fbf6c36f6a Mon Sep 17 00:00:00 2001 From: Yixin Luo <18810541851@163.com> Date: Mon, 27 May 2024 07:41:16 +0000 Subject: [PATCH 2/2] resolve Signed-off-by: Yixin Luo <18810541851@163.com> --- be/src/storage/tablet_updates.cpp | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/be/src/storage/tablet_updates.cpp b/be/src/storage/tablet_updates.cpp index 844cf7d2b2fd5..9e4cac2ea8917 100644 --- a/be/src/storage/tablet_updates.cpp +++ b/be/src/storage/tablet_updates.cpp @@ -1766,19 +1766,13 @@ Status TabletUpdates::_do_compaction(std::unique_ptr* pinfo) { // 4. commit compaction EditVersion version; RETURN_IF_ERROR(_commit_compaction(pinfo, *output_rowset, &version)); -<<<<<<< HEAD - // already committed, so we can ignore timeout error here - std::unique_lock ul(_lock); - _wait_for_version(version, 120000, ul); -======= { // already committed, so we can ignore timeout error here std::unique_lock ul(_lock); - RETURN_IF_ERROR(_wait_for_version(version, 120000, ul)); + _wait_for_version(version, 120000, ul); } // Release metadata memory after rowsets have been compacted. Rowset::close_rowsets(input_rowsets); ->>>>>>> 0471705863 ([Enhancement] release metadata memory after rowsets have been compacted in pk table (#46067)) return Status::OK(); }