From b3d7e283b931c67cd1e0e294067cdf04bcb08300 Mon Sep 17 00:00:00 2001 From: Mehdi Hosseini <116847813+shosseinimotlagh@users.noreply.github.com> Date: Mon, 4 Mar 2024 14:38:01 -0800 Subject: [PATCH 1/5] merge stable/v3.x to thin branch (#341) * Fast forward API to truncate upto a log index (#334) * add truncate_sync api to persist log store meta immediately during truncation Co-authored-by: Ravi Nagarjun Akella * fix the check in UT which is only valid in prerelease build (#338) Co-authored-by: Ravi Nagarjun Akella * skip non zero request buffers * merge stable/v3.x to thin branch --------- Co-authored-by: raakella1 <114193113+raakella1@users.noreply.github.com> Co-authored-by: Ravi Nagarjun Akella --- .github/workflows/merge_conan_build.yml | 1 + .github/workflows/pr_conan_build.yml | 1 + conanfile.py | 2 +- src/homelogstore/log_dev.cpp | 7 ++ src/homelogstore/log_store.cpp | 57 +++++++++++- src/homelogstore/log_store.hpp | 14 ++- src/homelogstore/tests/test_log_store.cpp | 106 +++++++++++++++++++++- 7 files changed, 179 insertions(+), 9 deletions(-) diff --git a/.github/workflows/merge_conan_build.yml b/.github/workflows/merge_conan_build.yml index 4b056c95f..a9ab6f033 100644 --- a/.github/workflows/merge_conan_build.yml +++ b/.github/workflows/merge_conan_build.yml @@ -6,6 +6,7 @@ on: branches: - master - stable/v3.x + - thin_provisioning jobs: Build: diff --git a/.github/workflows/pr_conan_build.yml b/.github/workflows/pr_conan_build.yml index a16faae34..cf6366c31 100644 --- a/.github/workflows/pr_conan_build.yml +++ b/.github/workflows/pr_conan_build.yml @@ -5,6 +5,7 @@ on: branches: - master - stable/v3.x + - thin_provisioning jobs: Build: diff --git a/conanfile.py b/conanfile.py index d013c21b1..61d77e40d 100644 --- a/conanfile.py +++ b/conanfile.py @@ -2,7 +2,7 @@ class HomestoreConan(ConanFile): name = "homestore" - version = "3.6.14" + version = "3.7.1" homepage = "https://github.corp.ebay.com/SDS/homestore" description = "HomeStore" diff --git a/src/homelogstore/log_dev.cpp b/src/homelogstore/log_dev.cpp index 8cf51cd2b..3d3f24d92 100644 --- a/src/homelogstore/log_dev.cpp +++ b/src/homelogstore/log_dev.cpp @@ -682,6 +682,13 @@ std::vector< std::pair< logstore_id_t, logstore_superblk > > LogDevMetadata::loa } void LogDevMetadata::persist() { +#ifdef _PRERELEASE + if (homestore_flip->test_flip("logstore_test_skip_persist")) { + LOGINFO("skipping persist of logdev metadata"); + return; + } +#endif + if (m_meta_mgr_cookie) { MetaBlkMgrSI()->update_sub_sb(static_cast< const void* >(m_raw_buf->bytes), m_raw_buf->size, m_meta_mgr_cookie); } else { diff --git a/src/homelogstore/log_store.cpp b/src/homelogstore/log_store.cpp index 111e01c57..acb0e704c 100644 --- a/src/homelogstore/log_store.cpp +++ b/src/homelogstore/log_store.cpp @@ -228,7 +228,7 @@ void HomeLogStore::truncate(const logstore_seq_num_t upto_seq_num, const bool in // First try to block the flushing of logdevice and if we are successfully able to do, then auto shared_this{shared_from_this()}; const bool locked_now{m_logdev.try_lock_flush([shared_this, upto_seq_num, in_memory_truncate_only]() { - shared_this->do_truncate(upto_seq_num); + shared_this->do_truncate(upto_seq_num, false /*persist meta now*/); if (!in_memory_truncate_only) { [[maybe_unused]] const auto key{shared_this->get_family().do_device_truncate()}; } @@ -237,13 +237,62 @@ void HomeLogStore::truncate(const logstore_seq_num_t upto_seq_num, const bool in if (locked_now) { m_logdev.unlock_flush(); } } +void HomeLogStore::sync_truncate(const logstore_seq_num_t upto_seq_num, const bool in_memory_truncate_only) { + // Check if we need to fill any gaps in the logstore + auto const last_idx{get_contiguous_issued_seq_num(std::max(0l, truncated_upto()))}; + HS_REL_ASSERT_GE(last_idx, 0l, "Negative sequence number: {} [Logstore id ={}]", last_idx, m_store_id); + auto const next_slot{last_idx + 1}; + for (auto curr_idx = next_slot; upto_seq_num >= curr_idx; ++curr_idx) { + fill_gap(curr_idx); + } + +#ifndef NDEBUG + const auto s{m_safe_truncation_boundary.seq_num.load(std::memory_order_acquire)}; + // Don't check this if we don't know our truncation boundary. The call is made to inform us about + // correct truncation point. + if (s != -1) { + HS_DBG_ASSERT_LE(upto_seq_num, get_contiguous_completed_seq_num(s), + "Logstore {} expects truncation to be contiguously completed", m_store_id); + } +#endif + + struct Context { + std::mutex truncate_mutex; + std::condition_variable truncate_cv; + bool truncate_done{false}; + }; + auto ctx{std::make_shared< Context >()}; + + auto shared_this{shared_from_this()}; + const bool locked_now{m_logdev.try_lock_flush([shared_this, upto_seq_num, in_memory_truncate_only, ctx]() { + shared_this->do_truncate(upto_seq_num, true /*persist meta now*/); + if (!in_memory_truncate_only) { + [[maybe_unused]] const auto key{shared_this->get_family().do_device_truncate()}; + } + { + std::unique_lock< std::mutex > lk{ctx->truncate_mutex}; + ctx->truncate_done = true; + } + ctx->truncate_cv.notify_one(); + })}; + + if (locked_now) { + m_logdev.unlock_flush(); + } else { + { + std::unique_lock< std::mutex > lk{ctx->truncate_mutex}; + ctx->truncate_cv.wait(lk, [&ctx] { return ctx->truncate_done; }); + } + } +} + // NOTE: This method assumes the flush lock is already acquired by the caller -void HomeLogStore::do_truncate(const logstore_seq_num_t upto_seq_num) { +void HomeLogStore::do_truncate(const logstore_seq_num_t upto_seq_num, const bool persist_now) { m_records.truncate(upto_seq_num); m_safe_truncation_boundary.seq_num.store(upto_seq_num, std::memory_order_release); - // Need to update the superblock with meta, we don't persist yet, will be done as part of log dev truncation - m_logdev.update_store_superblk(m_store_id, logstore_superblk{upto_seq_num + 1}, false /* persist_now */); + // Need to update the superblock with meta, in case we don't persist yet, will be done as part of log dev truncation + m_logdev.update_store_superblk(m_store_id, logstore_superblk{upto_seq_num + 1}, persist_now /* persist_now */); const int ind{search_max_le(upto_seq_num)}; if (ind < 0) { diff --git a/src/homelogstore/log_store.hpp b/src/homelogstore/log_store.hpp index 89f69ba83..9a9650b20 100644 --- a/src/homelogstore/log_store.hpp +++ b/src/homelogstore/log_store.hpp @@ -45,7 +45,7 @@ struct log_dump_req { std::shared_ptr< HomeLogStore > log_store; // if null all log stores are dumped logstore_seq_num_t start_seq_num; // empty_key if from start of log file logstore_seq_num_t end_seq_num; // empty_key if till last log entry - logstore_seq_num_t batch_size = 0; // Size of the output batch. + logstore_seq_num_t batch_size = 0; // Size of the output batch. }; struct logstore_record { @@ -94,7 +94,7 @@ struct logstore_req { if (req->is_internal_req) { sisl::ObjectAllocator< logstore_req >::deallocate(req); } } - friend class sisl::ObjectAllocator; + friend class sisl::ObjectAllocator< logstore_req >; private: logstore_req() = default; @@ -396,6 +396,14 @@ class HomeLogStore : public std::enable_shared_from_this< HomeLogStore > { */ void truncate(const logstore_seq_num_t upto_seq_num, const bool in_memory_truncate_only = true); + /** + * @brief Truncate the logs for this log store upto the seq_num provided (inclusive) and persist the metablock + * before returning. If the next available slot is less than upto_seq_num, it will fill the gaps. + * This API makes sure the log idx is durable after the truncation. + * See above truncate method for more details on parameters + */ + void sync_truncate(const logstore_seq_num_t upto_seq_num, const bool in_memory_truncate_only = true); + /** * @brief Fill the gap in the seq_num with a dummy value. This ensures that get_contiguous_issued and completed * seq_num methods move forward. The filled data is not readable and any attempt to read this seq_num will @@ -513,7 +521,7 @@ class HomeLogStore : public std::enable_shared_from_this< HomeLogStore > { void on_log_found(const logstore_seq_num_t seq_num, const logdev_key ld_key, const logdev_key flush_ld_key, const log_buffer buf); void on_batch_completion(const logdev_key& flush_batch_ld_key); - void do_truncate(const logstore_seq_num_t upto_seq_num); + void do_truncate(const logstore_seq_num_t upto_seq_num, const bool persist_now); [[nodiscard]] int search_max_le(const logstore_seq_num_t input_sn); private: diff --git a/src/homelogstore/tests/test_log_store.cpp b/src/homelogstore/tests/test_log_store.cpp index 9dccaf185..cc4f5e00a 100644 --- a/src/homelogstore/tests/test_log_store.cpp +++ b/src/homelogstore/tests/test_log_store.cpp @@ -46,6 +46,7 @@ #include "api/vol_interface.hpp" #include "test_common/homestore_test_common.hpp" +#include "engine/common/homestore_flip.hpp" #include "../log_store.hpp" @@ -114,6 +115,9 @@ class SampleLogStoreClient { m_log_store->register_log_found_cb(bind_this(SampleLogStoreClient::on_log_found, 3)); } + auto get_log_store() { return m_log_store; } + static void set_validate_on_log_found(const bool flag) { validate_on_log_found = flag; } + void reset_recovery() { m_n_recovered_lsns = 0; m_n_recovered_truncated_lsns = 0; @@ -305,6 +309,7 @@ class SampleLogStoreClient { } void on_log_found(const logstore_seq_num_t lsn, const log_buffer buf, void* const ctx) { + if (!validate_on_log_found) { return; } LOGDEBUG("Recovered lsn {}:{} with log data of size {}", m_log_store->get_store_id(), lsn, buf.size()) EXPECT_LE(lsn, m_cur_lsn.load()) << "Recovered incorrect lsn " << m_log_store->get_store_id() << ":" << lsn << "Expected less than cur_lsn " << m_cur_lsn.load(); @@ -370,6 +375,7 @@ class SampleLogStoreClient { private: static constexpr uint32_t max_data_size = 1024; static uint64_t s_max_flush_multiple; + inline static bool validate_on_log_found = true; logstore_id_t m_store_id; test_log_store_comp_cb_t m_comp_cb; @@ -482,7 +488,7 @@ class SampleDB { if (SISL_OPTIONS.count("http_port")) { test_common::set_fixed_http_port(SISL_OPTIONS["http_port"].as< uint32_t >()); - }else { + } else { test_common::set_random_http_port(); } VolInterface::init(params, restart); @@ -554,6 +560,8 @@ class SampleDB { return m_highest_log_idx[fid].load(); } + std::vector< std::unique_ptr< SampleLogStoreClient > >& log_store_clients() { return m_log_store_clients; } + private: const static std::string s_fpath_root; std::vector< std::string > m_dev_names; @@ -897,6 +905,102 @@ class LogStoreTest : public ::testing::Test { uint32_t m_holes_per_batch{0}; }; +static uint64_t curr_trunc_start(std::shared_ptr< HomeLogStore >& _log_store) { + return std::max(0l, _log_store->truncated_upto()); +} + +TEST_F(LogStoreTest, TruncateDurability) { + auto _hs_log_store = + SampleDB::instance() + .log_store_clients() + .emplace_back(std::make_unique< SampleLogStoreClient >( + HomeLogStoreMgr::CTRL_LOG_FAMILY_IDX, [](logstore_family_id_t, logstore_seq_num_t, logdev_key) {})) + .get() + ->get_log_store(); + + auto const start_index = [](std::shared_ptr< HomeLogStore >& _log_store) { + return curr_trunc_start(_log_store) + 1; + }; + auto const next_slot = [](std::shared_ptr< HomeLogStore >& _log_store) { + auto const last_idx = _log_store->get_contiguous_issued_seq_num(curr_trunc_start(_log_store)); + EXPECT_TRUE(last_idx >= 0l); + return static_cast< uint64_t >(last_idx) + 1; + }; + + uint64_t last_idx{100}; + for (auto i = 0u; i < last_idx; ++i) { + bool io_memory{false}; + auto* const d{SampleLogStoreClient::prepare_data(i, io_memory)}; + + EXPECT_TRUE(_hs_log_store->write_sync(next_slot(_hs_log_store), + {reinterpret_cast< uint8_t* >(d), d->total_size(), false})); + + if (io_memory) { + iomanager.iobuf_free(reinterpret_cast< uint8_t* >(d)); + } else { + std::free(static_cast< void* >(d)); + } + } + // ls range should be [1, 100] + EXPECT_EQ(start_index(_hs_log_store), 1ul); + EXPECT_EQ(next_slot(_hs_log_store), last_idx + 1); + +// set flip to avoid logdev metablk persistance +#ifdef _PRERELEASE + flip::FlipClient* fc{HomeStoreFlip::client_instance()}; + flip::FlipFrequency freq; + freq.set_count(1); + freq.set_percent(100); + fc->inject_noreturn_flip("logstore_test_skip_persist", {}, freq); +#endif + + // fill gaps and truncate upto 150. ls is now [151, 151] + uint64_t truncate_upto{150}; + for (auto curr_idx = next_slot(_hs_log_store); truncate_upto >= curr_idx; ++curr_idx) { + _hs_log_store->fill_gap(curr_idx); + } + _hs_log_store->truncate(truncate_upto); + // validate the satrt and end of the ls + EXPECT_EQ(start_index(_hs_log_store), truncate_upto + 1); + EXPECT_EQ(next_slot(_hs_log_store), truncate_upto + 1); + + // restart homestore + const auto num_devs{SISL_OPTIONS["num_devs"].as< uint32_t >()}; // num devices + const auto dev_size_bytes{SISL_OPTIONS["dev_size_mb"].as< uint64_t >() * 1024 * 1024}; + const auto num_threads{SISL_OPTIONS["num_threads"].as< uint32_t >()}; + const auto num_logstores{SISL_OPTIONS["num_logstores"].as< uint32_t >() + 1}; + SampleLogStoreClient::set_validate_on_log_found(false); + SampleDB::instance().start_homestore(num_devs, dev_size_bytes, num_threads, num_logstores, true /* restart */); + + _hs_log_store = SampleDB::instance().log_store_clients().back()->get_log_store(); +#ifdef _PRERELEASE + // We are simulating a crash by setting the logstore_test_skip_persist flip which does not persist metablk + // This will invalidate truncate call above and set logstore to [1 100] + EXPECT_EQ(1ul, start_index(_hs_log_store)); + EXPECT_EQ(last_idx + 1, next_slot(_hs_log_store)); +#endif + + // fast_forward should be resilient to crashe and should be able to recover + + uint64_t fast_forward_upto{350}; + _hs_log_store->sync_truncate(fast_forward_upto); +#ifdef _PRERELEASE + flip::FlipFrequency freq1; + freq1.set_count(1); + freq1.set_percent(100); + fc->inject_noreturn_flip("logstore_test_skip_persist", {}, freq1); +#endif + EXPECT_EQ(start_index(_hs_log_store), fast_forward_upto + 1); + EXPECT_EQ(next_slot(_hs_log_store), fast_forward_upto + 1); + + SampleDB::instance().start_homestore(num_devs, dev_size_bytes, num_threads, num_logstores, true /* restart */); + EXPECT_EQ(start_index(_hs_log_store), fast_forward_upto + 1); + EXPECT_EQ(next_slot(_hs_log_store), fast_forward_upto + 1); + + EXPECT_TRUE(SampleDB::instance().delete_log_store(_hs_log_store->get_store_id())); + SampleLogStoreClient::set_validate_on_log_found(true); +} + TEST_F(LogStoreTest, BurstRandInsertThenTruncate) { const auto num_records{SISL_OPTIONS["num_records"].as< uint32_t >()}; const auto iterations{SISL_OPTIONS["iterations"].as< uint32_t >()}; From ed3ffae013e7ad917c19e296cae7a103f5b6b077 Mon Sep 17 00:00:00 2001 From: Mehdi Hosseini <116847813+shosseinimotlagh@users.noreply.github.com> Date: Tue, 12 Mar 2024 17:16:03 -0700 Subject: [PATCH 2/5] Thin provisioning : Enable zero detecting for requests (#349) * Enable zero detecting for requests Added a manual io job test --- conanfile.py | 2 +- src/.clang-format | 1 - src/api/vol_interface.hpp | 18 +- src/engine/common/homestore_config.fbs | 2 + src/engine/common/homestore_utils.cpp | 53 ++++- src/engine/common/homestore_utils.hpp | 2 + src/homeblks/volume/tests/vol_gtest.cpp | 275 +++++++++++++++++++++++- src/homeblks/volume/volume.cpp | 70 +++++- src/homeblks/volume/volume.hpp | 3 +- 9 files changed, 406 insertions(+), 20 deletions(-) diff --git a/conanfile.py b/conanfile.py index 61d77e40d..0fb9488de 100644 --- a/conanfile.py +++ b/conanfile.py @@ -2,7 +2,7 @@ class HomestoreConan(ConanFile): name = "homestore" - version = "3.7.1" + version = "3.7.2" homepage = "https://github.corp.ebay.com/SDS/homestore" description = "HomeStore" diff --git a/src/.clang-format b/src/.clang-format index 2f7712008..fdfa11f5e 100644 --- a/src/.clang-format +++ b/src/.clang-format @@ -18,7 +18,6 @@ AlignOperands: false AlignTrailingComments: true AllowShortBlocksOnASingleLine: true AllowShortIfStatementsOnASingleLine: true -AllowShortBlocksOnASingleLine: true AllowShortCaseLabelsOnASingleLine: false # AllowShortFunctionsOnASingleLine: InlineOnly # AllowShortLoopsOnASingleLine: false diff --git a/src/api/vol_interface.hpp b/src/api/vol_interface.hpp index 53627911b..7436ee5ea 100644 --- a/src/api/vol_interface.hpp +++ b/src/api/vol_interface.hpp @@ -114,6 +114,22 @@ struct vol_interface_req : public sisl::ObjLifeCounter< vol_interface_req > { bool is_write() const { return op_type == Op_type::WRITE; } bool is_unmap() const { return op_type == Op_type::UNMAP; } + bool is_zero_request(const uint64_t page_size) { + if (iovecs.empty()) { + return !buffer || hs_utils::is_buf_zero(static_cast< uint8_t* >(buffer), nlbas * page_size); + } + return is_iovec_zero(); + } + + bool is_iovec_zero() { + for (const auto& iovec : iovecs) { + auto data = static_cast< uint8_t* >(iovec.iov_base); + const size_t size = iovec.iov_len; + if (!hs_utils::is_buf_zero(data, size)) { return false; } + } + return true; + } + friend void intrusive_ptr_add_ref(vol_interface_req* req) { req->refcount.increment(1); } friend void intrusive_ptr_release(vol_interface_req* req) { @@ -316,7 +332,7 @@ class VolInterface { virtual const char* get_name(const VolumePtr& vol) = 0; virtual uint64_t get_size(const VolumePtr& vol) = 0; - virtual std::map get_used_size(const VolumePtr& vol) = 0; + virtual std::map< boost::uuids::uuid, uint64_t > get_used_size(const VolumePtr& vol) = 0; virtual uint64_t get_page_size(const VolumePtr& vol) = 0; virtual boost::uuids::uuid get_uuid(std::shared_ptr< Volume > vol) = 0; virtual sisl::blob at_offset(const boost::intrusive_ptr< BlkBuffer >& buf, uint32_t offset) = 0; diff --git a/src/engine/common/homestore_config.fbs b/src/engine/common/homestore_config.fbs index e6dfee051..9b219abe4 100644 --- a/src/engine/common/homestore_config.fbs +++ b/src/engine/common/homestore_config.fbs @@ -142,6 +142,8 @@ table Generic { // percentage of cache used to create indx mempool. It should be more than 100 to // take into account some floating buffers in writeback cache. indx_mempool_percent : uint32 = 110; + + boot_thin_provisioning: bool = false; } table ResourceLimits { diff --git a/src/engine/common/homestore_utils.cpp b/src/engine/common/homestore_utils.cpp index fa60cbb3a..c161af0fa 100644 --- a/src/engine/common/homestore_utils.cpp +++ b/src/engine/common/homestore_utils.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace homestore { uint8_t* hs_utils::iobuf_alloc(const size_t size, const sisl::buftag tag, const size_t alignment) { @@ -76,6 +77,42 @@ sisl::byte_array hs_utils::extract_byte_array(const sisl::byte_view& b, const bo return (is_aligned_needed) ? b.extract(alignment) : b.extract(0); }; +constexpr unsigned long long operator"" _KB(unsigned long long x) { return x * 1024; } + +constexpr std::array< size_t, 7 > predefined_sizes = {4_KB, 8_KB, 16_KB, 32_KB, 64_KB, 128_KB, 256_KB}; + +// Function to initialize the CRC map with predefined sizes +void initialize_crc_map(std::map< size_t, uint16_t >& crc_map) { + std::vector< uint8_t > zero_buf; + for (auto s : predefined_sizes) { + zero_buf.resize(s, 0); // Resize buffer to the required size, filling with zeros + crc_map[s] = crc16_t10dif(init_crc_16, zero_buf.data(), s); + } +} + +uint16_t hs_utils::crc_zero(const size_t size) { + static std::map< size_t, uint16_t > crc_map; + static std::once_flag init_flag; + + // Thread-safe initialization of the CRC map + std::call_once(init_flag, initialize_crc_map, std::ref(crc_map)); + + // Check if the size is already in the map + if (auto it = crc_map.find(size); it != crc_map.end()) { return it->second; } + + std::vector< uint8_t > zero_buf(size, 0); + return crc16_t10dif(init_crc_16, zero_buf.data(), size); +} + +bool hs_utils::is_buf_zero(const uint8_t* buf, size_t size) { + // TODO: subsample the buffer to detect zero request instead of working on the whole buffer to achieve constant + // processing time for large buffer size requests. Needs to investigate the performance impact of this change + // in end2end testing. + auto zero_crc = crc_zero(size); + const auto crc = crc16_t10dif(init_crc_16, buf, size); + return (crc == zero_crc) ? (buf[0] == 0 && !std::memcmp(buf, buf + 1, size - 1)) : false; +} + std::string hs_utils::encodeBase64(const uint8_t* first, std::size_t size) { using Base64FromBinary = boost::archive::iterators::base64_from_binary< boost::archive::iterators::transform_width< const char*, // sequence of chars @@ -90,15 +127,12 @@ std::string hs_utils::encodeBase64(const uint8_t* first, std::size_t size) { return encoded.append(bytes_to_pad, '='); } -std::string hs_utils::encodeBase64(const sisl::byte_view& b){ - return encodeBase64(b.bytes(), b.size()); -} +std::string hs_utils::encodeBase64(const sisl::byte_view& b) { return encodeBase64(b.bytes(), b.size()); } -template -void hs_utils::decodeBase64(const std::string &encoded_data, T out) -{ +template < typename T > +void hs_utils::decodeBase64(const std::string& encoded_data, T out) { using BinaryFromBase64 = boost::archive::iterators::transform_width< - boost::archive::iterators::binary_from_base64, + boost::archive::iterators::binary_from_base64< std::string::const_iterator >, 8, // get a view of 8 bit 6 // from a sequence of 6 bit >; @@ -107,14 +141,13 @@ void hs_utils::decodeBase64(const std::string &encoded_data, T out) std::replace(begin(unpadded_data), end(unpadded_data), '=', 'A'); // A_64 == \0 std::string decoded_data{BinaryFromBase64{begin(unpadded_data)}, - BinaryFromBase64{begin(unpadded_data) + unpadded_data.length()}}; + BinaryFromBase64{begin(unpadded_data) + unpadded_data.length()}}; decoded_data.erase(end(decoded_data) - bytes_to_pad, end(decoded_data)); std::copy(begin(decoded_data), end(decoded_data), out); } -std::string hs_utils::decodeBase64(const std::string &encoded_data) -{ +std::string hs_utils::decodeBase64(const std::string& encoded_data) { std::string rv; decodeBase64(encoded_data, std::back_inserter(rv)); return rv; diff --git a/src/engine/common/homestore_utils.hpp b/src/engine/common/homestore_utils.hpp index b1313df96..de081d7f8 100644 --- a/src/engine/common/homestore_utils.hpp +++ b/src/engine/common/homestore_utils.hpp @@ -38,6 +38,8 @@ class hs_utils { static sisl::byte_array make_byte_array(const uint64_t size, const bool is_aligned_needed, const sisl::buftag tag, const size_t alignment); static hs_uuid_t gen_system_uuid(); + static uint16_t crc_zero(const size_t size); + static bool is_buf_zero(const uint8_t* buf, size_t size); static std::string encodeBase64(const uint8_t* first, std::size_t size); static std::string encodeBase64(const sisl::byte_view& b); template static void decodeBase64(const std::string &encoded_data, T out); diff --git a/src/homeblks/volume/tests/vol_gtest.cpp b/src/homeblks/volume/tests/vol_gtest.cpp index d8aa4ef17..8f31a501e 100644 --- a/src/homeblks/volume/tests/vol_gtest.cpp +++ b/src/homeblks/volume/tests/vol_gtest.cpp @@ -174,6 +174,9 @@ struct TestCfg { uint32_t p_vol_files_space; std::string flip_name; std::string vol_copy_file_path; + uint32_t p_zero_buffer; + uint32_t zero_buffer_period; + bool thin_provision_enable{false}; bool verify_csum() { return verify_type == verify_type_t::csum; } bool verify_data() { return verify_type == verify_type_t::data; } @@ -575,6 +578,7 @@ class VolTest : public ::testing::Test { friend class VolCreateDeleteJob; friend class IOTestJob; friend class VolVerifyJob; + friend class IOManualTestJob; protected: std::atomic< size_t > outstanding_ios; @@ -620,12 +624,20 @@ class VolTest : public ::testing::Test { // vol_create_del_test = false; // move_verify_to_done = false; print_startTime = Clock::now(); + if (tcfg.thin_provision_enable) { + HS_SETTINGS_FACTORY().modifiable_settings([](auto& s) { s.generic.boot_thin_provisioning = true; }); + HS_SETTINGS_FACTORY().save(); + } // outstanding_ios = 0; } virtual ~VolTest() override { if (init_buf) { iomanager.iobuf_free(static_cast< uint8_t* >(init_buf)); } + if (tcfg.thin_provision_enable) { + HS_SETTINGS_FACTORY().modifiable_settings([](auto& s) { s.generic.boot_thin_provisioning = false; }); + HS_SETTINGS_FACTORY().save(); + } } VolTest(const VolTest&) = delete; @@ -1815,13 +1827,18 @@ class IOTestJob : public TestJob { const uint64_t page_size{VolInterface::get_instance()->get_page_size(vol)}; const uint64_t size{nlbas * page_size}; + static std::atomic< uint32_t > remaining_period{tcfg.zero_buffer_period}; + uint32_t zero_counts_per_period = tcfg.p_zero_buffer * tcfg.zero_buffer_period / 100; boost::intrusive_ptr< io_req_t > vreq{}; if (tcfg.write_cache) { uint8_t* const wbuf{iomanager.iobuf_alloc(512, size)}; HS_REL_ASSERT_NOTNULL(wbuf); populate_buf(wbuf, size, lba, vinfo.get()); - + if (HS_DYNAMIC_CONFIG(generic->boot_thin_provisioning) && + remaining_period.fetch_sub(1) < zero_counts_per_period) { + populate_zero_buf(wbuf, size); + } vreq = boost::intrusive_ptr< io_req_t >( new io_req_t(vinfo, Op_type::WRITE, wbuf, lba, nlbas, tcfg.verify_csum(), tcfg.write_cache)); } else { @@ -1833,20 +1850,32 @@ class IOTestJob : public TestJob { HS_REL_ASSERT_NOTNULL(wbuf); iovec iov{static_cast< void* >(wbuf), static_cast< size_t >(page_size)}; iovecs.emplace_back(std::move(iov)); - populate_buf(wbuf, page_size, lba + lba_num, vinfo.get()); } + if (HS_DYNAMIC_CONFIG(generic->boot_thin_provisioning) && + remaining_period.fetch_sub(1) < zero_counts_per_period) { + for (const auto& iovec : iovecs) { + auto data = static_cast< uint8_t* >(iovec.iov_base); + const size_t size = iovec.iov_len; + populate_zero_buf(data, size); + } + } vreq = boost::intrusive_ptr< io_req_t >(new io_req_t(vinfo, Op_type::WRITE, std::move(iovecs), lba, nlbas, tcfg.verify_csum(), tcfg.write_cache)); } else { uint8_t* const wbuf{iomanager.iobuf_alloc(512, size)}; populate_buf(wbuf, size, lba, vinfo.get()); + if (HS_DYNAMIC_CONFIG(generic->boot_thin_provisioning) && + remaining_period.fetch_sub(1) < zero_counts_per_period) { + populate_zero_buf(wbuf, size); + } HS_REL_ASSERT_NOTNULL(wbuf); vreq = boost::intrusive_ptr< io_req_t >{ new io_req_t(vinfo, Op_type::WRITE, wbuf, lba, nlbas, tcfg.verify_csum(), tcfg.write_cache)}; } + if (remaining_period.load() == 0) { remaining_period.store(tcfg.zero_buffer_period); } send_iovec = !send_iovec; } vreq->cookie = static_cast< void* >(this); @@ -1880,6 +1909,8 @@ class IOTestJob : public TestJob { } } + void populate_zero_buf(uint8_t* buf, const uint64_t size) { std::fill_n(buf, size, 0); } + bool read_vol(const uint32_t cur, const uint64_t lba, const uint32_t nlbas) { const auto vinfo{m_voltest->m_vol_info[cur]}; const auto vol{vinfo->vol}; @@ -2066,6 +2097,199 @@ class IOTestJob : public TestJob { } }; +// This test job is used to test the IOs with manual requests. For sake of simplicity, we will use the same volume for +// all requests. The caller needs to load the requests before starting the job. The requests are loaded in the form of +// Write with three or four parameters and Read with three parameters. The value is optional and is used only for write +// requests. +class IOManualTestJob : public TestJob { +public: + using TupleVariant = std::variant< std::tuple< std::string, uint64_t, uint32_t >, + std::tuple< std::string, uint64_t, uint32_t, uint8_t > >; + using RequestVector = std::vector< IOManualTestJob::TupleVariant >; + IOManualTestJob(VolTest* const test) : TestJob(test, 1, true) { + vol = m_voltest->m_vol_info[0]->vol; + vinfo = m_voltest->m_vol_info[0]; + page_size = VolInterface::get_instance()->get_page_size(vol); + const auto vol_size = VolInterface::get_instance()->get_size(vol); + const auto max_lbas = vol_size / page_size; + m_validate_buf.resize(max_lbas); + std::fill(m_validate_buf.begin(), m_validate_buf.end(), 0); + LOGINFO("Manual volume size {} max_lbas {}", vol_size, max_lbas); + } + virtual ~IOManualTestJob() override = default; + IOManualTestJob(const IOManualTestJob&) = delete; + IOManualTestJob(IOManualTestJob&&) noexcept = delete; + IOManualTestJob& operator=(const IOManualTestJob&) = delete; + IOManualTestJob& operator=(IOManualTestJob&&) noexcept = delete; + + virtual void run_one_iteration() override { + if (m_outstanding_ios.load() == 0 && m_current_request < m_requests.size()) { + const auto& request = m_requests[m_current_request]; + if (std::holds_alternative< std::tuple< std::string, uint64_t, uint32_t > >(request)) { + auto& tuple = std::get< std::tuple< std::string, uint64_t, uint32_t > >(request); + auto start_lba = std::get< 1 >(tuple); + auto nlbas = std::get< 2 >(tuple); + if (std::get< 0 >(tuple) == "write") { + write_vol(start_lba, nlbas); + auto it = m_validate_buf.begin() + start_lba; + std::fill(it, it + nlbas, 0); + } else { + read_vol(start_lba, nlbas); + } + } else if (std::holds_alternative< std::tuple< std::string, uint64_t, uint32_t, uint8_t > >(request)) { + auto& tuple = std::get< std::tuple< std::string, uint64_t, uint32_t, uint8_t > >(request); + auto start_lba = std::get< 1 >(tuple); + auto nlbas = std::get< 2 >(tuple); + auto value = std::get< 3 >(tuple); + if (std::get< 0 >(tuple) == "write") { + write_vol(start_lba, nlbas, value); + auto it = m_validate_buf.begin() + start_lba; + std::fill(it, it + nlbas, value); + } else { + // in case, the caller mistakenly added a value for a read request, we will ignore the value + read_vol(start_lba, nlbas); + } + } + } + } + + void on_one_iteration_completed(const boost::intrusive_ptr< io_req_t >& req) override { + --m_outstanding_ios; + if (req->op_type == Op_type::READ) { verify_request(req); } + req->vol_info->ref_cnt.decrement_testz(1); + } + uint64_t read_buffer(std::vector< iovec >& iovecs, uint8_t* buf) { + uint8_t* current_position = buf; + for (const auto& iov : iovecs) { + std::memcpy(current_position, iov.iov_base, iov.iov_len); + current_position += iov.iov_len; + } + return static_cast< uint64_t >(current_position - buf); + } + void verify_request(const boost::intrusive_ptr< io_req_t >& req) { + std::shared_ptr< uint8_t > buf(new uint8_t[req->nlbas * page_size]); + std::fill_n(buf.get(), req->nlbas * page_size, 0); + auto total_size_read = read_buffer(req->iovecs, buf.get()); + HS_REL_ASSERT_EQ(req->nlbas * page_size, total_size_read); + auto raw_buf = buf.get(); + for (size_t i = 0; i < req->nlbas; i++) { + HS_REL_ASSERT_EQ(raw_buf[i * page_size], m_validate_buf[req->lba + i]); + } + } + bool time_to_stop() const override { return m_current_request == m_requests.size(); } + + virtual bool is_job_done() const override { return (m_outstanding_ios == 0); } + bool is_async_job() const override { return true; } + std::string job_name() const { return "IO Manual Job"; } + void load_requests(RequestVector& requests) { m_requests = requests; } + +protected: + VolumePtr vol; + std::shared_ptr< vol_info_t > vinfo; + uint64_t page_size; + std::atomic< uint64_t > m_outstanding_ios{0}; + std::atomic< uint64_t > m_current_request{0}; + std::vector< uint8_t > m_validate_buf; + RequestVector m_requests; + + bool write_vol(const uint64_t lba, const uint32_t nlbas, const uint8_t value = 0) { + ++m_current_request; + ++m_outstanding_ios; + const uint64_t size{nlbas * page_size}; + boost::intrusive_ptr< io_req_t > vreq{}; + if (tcfg.write_cache) { + uint8_t* const wbuf{iomanager.iobuf_alloc(512, size)}; + populate_buf(wbuf, size, value); + vreq = boost::intrusive_ptr< io_req_t >( + new io_req_t(vinfo, Op_type::WRITE, wbuf, lba, nlbas, tcfg.verify_csum(), tcfg.write_cache)); + } else { + static bool send_iovec{true}; + std::vector< iovec > iovecs{}; + if (send_iovec) { + for (uint32_t lba_num{0}; lba_num < nlbas; ++lba_num) { + uint8_t* const wbuf{iomanager.iobuf_alloc(512, page_size)}; + iovec iov{static_cast< void* >(wbuf), static_cast< size_t >(page_size)}; + iovecs.emplace_back(std::move(iov)); + populate_buf(wbuf, page_size, value); + } + vreq = boost::intrusive_ptr< io_req_t >(new io_req_t(vinfo, Op_type::WRITE, std::move(iovecs), lba, + nlbas, tcfg.verify_csum(), tcfg.write_cache)); + } else { + uint8_t* const wbuf{iomanager.iobuf_alloc(512, size)}; + populate_buf(wbuf, size, value); + vreq = boost::intrusive_ptr< io_req_t >{ + new io_req_t(vinfo, Op_type::WRITE, wbuf, lba, nlbas, tcfg.verify_csum(), tcfg.write_cache)}; + } + send_iovec = !send_iovec; + } + vreq->cookie = static_cast< void* >(this); + + ++m_voltest->output.write_cnt; + vinfo->ref_cnt.increment(1); + const auto ret_io{VolInterface::get_instance()->write(vol, vreq)}; + LOGDEBUG("Wrote lba: {}, nlbas: {} outstanding_ios={}, iovec(s)={}, cache={}", lba, nlbas, + m_outstanding_ios.load(), (tcfg.write_iovec != 0 ? true : false), + (tcfg.write_cache != 0 ? true : false)); + if (ret_io != no_error) { return false; } + return true; + } + + void populate_buf(uint8_t* buf, const uint64_t size, const uint8_t value = 0) { std::fill_n(buf, size, value); } + + bool read_vol(const uint64_t lba, const uint32_t nlbas) { + ++m_current_request; + if (read_vol_internal(vinfo, vol, lba, nlbas, false)) { return true; } + return false; + } + + boost::intrusive_ptr< io_req_t > read_vol_internal(std::shared_ptr< vol_info_t > vinfo, VolumePtr vol, + const uint64_t lba, const uint32_t nlbas, + const bool sync = false) { + boost::intrusive_ptr< io_req_t > vreq{}; + if (tcfg.read_cache) { + vreq = boost::intrusive_ptr< io_req_t >{ + new io_req_t{vinfo, Op_type::READ, nullptr, lba, nlbas, tcfg.verify_csum(), tcfg.read_cache, sync}}; + } else { + static bool send_iovec{true}; + if (send_iovec) { + std::vector< iovec > iovecs{}; + for (uint32_t lba_num{0}; lba_num < nlbas; ++lba_num) { + uint8_t* const rbuf{iomanager.iobuf_alloc(512, page_size)}; + std::memset(static_cast< void* >(rbuf), 0, page_size); + + HS_REL_ASSERT_NOTNULL(rbuf); + iovec iov{static_cast< void* >(rbuf), static_cast< size_t >(page_size)}; + iovecs.emplace_back(std::move(iov)); + } + + vreq = boost::intrusive_ptr< io_req_t >{new io_req_t{vinfo, Op_type::READ, std::move(iovecs), lba, + nlbas, tcfg.verify_csum(), tcfg.read_cache, sync}}; + } else { + uint8_t* const rbuf{iomanager.iobuf_alloc(512, nlbas * page_size)}; + std::memset(static_cast< void* >(rbuf), 0, nlbas * page_size); + vreq = boost::intrusive_ptr< io_req_t >{ + new io_req_t{vinfo, Op_type::READ, rbuf, lba, nlbas, tcfg.verify_csum(), tcfg.read_cache, sync}}; + } + send_iovec = !send_iovec; + } + vreq->cookie = static_cast< void* >(this); + + ++m_voltest->output.read_cnt; + ++m_outstanding_ios; + vinfo->ref_cnt.increment(1); + const auto ret_io{VolInterface::get_instance()->read(vol, vreq)}; + LOGDEBUG("Read lba: {}, nlbas: {} outstanding_ios={}, iovec(s)={}, cache={}", lba, nlbas, + m_outstanding_ios.load(), (tcfg.read_iovec != 0 ? true : false), + (tcfg.read_cache != 0 ? true : false)); + if (sync) { + --m_outstanding_ios; + vinfo->ref_cnt.decrement(1); + } + if (ret_io != no_error) { return nullptr; } + return vreq; + } +}; + class VolVerifyJob : public IOTestJob { public: VolVerifyJob(VolTest* test) : IOTestJob(test, load_type_t::sequential) { @@ -2223,6 +2447,43 @@ TEST_F(VolTest, init_io_test) { this->shutdown(); if (tcfg.remove_file_on_shutdown) { this->remove_files(); } } +TEST_F(VolTest, thin_test) { + HS_SETTINGS_FACTORY().modifiable_settings([](auto& s) { s.generic.boot_thin_provisioning = true; }); + HS_SETTINGS_FACTORY().save(); + tcfg.max_vols = 1; + tcfg.verify_type = static_cast< verify_type_t >(3); + tcfg.max_disk_capacity = 1 * (1ul << 30); // 1GB + tcfg.p_volume_size = 1; // 1% of 2 (devices) * 1G = 20 MB volume + output.print("thin_test"); + + this->start_homestore(); + + std::unique_ptr< IOManualTestJob > job; + job = std::make_unique< IOManualTestJob >(this); + // request = op=[write|read], lba, nlbas [value], value is optional and is used only for write requests and If not + // provided, it defaults to 0. + IOManualTestJob::RequestVector reqs = { + // Case one: normal read (no zero padding) + std::make_tuple("write", 0, 100, 4), std::make_tuple("read", 5, 20), + // Case two: zero padding, read after write + std::make_tuple("write", 1, 10), std::make_tuple("read", 1, 20), std::make_tuple("read", 5, 3), + // Case three: zero padding, overlapping for read + std::make_tuple("write", 100, 200), std::make_tuple("read", 150, 250), + // Case four: no write + std::make_tuple("read", 800, 5)}; + job->load_requests(reqs); + + this->start_job(job.get(), wait_type::for_completion); + + LOGINFO("All volumes are deleted, do a shutdown of homestore"); + this->shutdown(); + + LOGINFO("Shutdown of homestore is completed, removing files"); + this->remove_files(); + + HS_SETTINGS_FACTORY().modifiable_settings([](auto& s) { s.generic.boot_thin_provisioning = false; }); + HS_SETTINGS_FACTORY().save(); +} /*! @test recovery_io_test @@ -2682,6 +2943,13 @@ SISL_OPTION_GROUP( (io_size, "", "io_size", "io size in KB", ::cxxopts::value< uint32_t >()->default_value("4"), "io_size"), (vol_copy_file_path, "", "vol_copy_file_path", "file path for copied volume", ::cxxopts::value< std::string >()->default_value(""), "path [...]"), + (p_zero_buffer, "", "p_zero_buffer", + "percentage of zero buffer occurrence for testing thin provisioning within period", + ::cxxopts::value< uint32_t >()->default_value("70"), "0 to 100"), + (zero_buffer_period, "", "zero_buffer_period", " the period of consecutive zero buffer occurrence", + ::cxxopts::value< uint32_t >()->default_value("100"), "0 to 100"), + (thin_provision_enable, "", "thin_provision_enable", " enable thin provisioning", + ::cxxopts::value< uint32_t >()->default_value("0"), "flag"), (unmap_frequency, "", "unmap_frequency", "do unmap for every N", ::cxxopts::value< uint64_t >()->default_value("100"), "unmap_frequency")) @@ -2758,6 +3026,9 @@ int main(int argc, char* argv[]) { gcfg.app_mem_size_in_gb = SISL_OPTIONS["app_mem_size_in_gb"].as< uint32_t >(); gcfg.vol_copy_file_path = SISL_OPTIONS["vol_copy_file_path"].as< std::string >(); const auto io_size_in_kb = SISL_OPTIONS["io_size"].as< uint32_t >(); + gcfg.p_zero_buffer = SISL_OPTIONS["p_zero_buffer"].as< uint32_t >(); + gcfg.zero_buffer_period = SISL_OPTIONS["zero_buffer_period"].as< uint32_t >(); + gcfg.thin_provision_enable = SISL_OPTIONS["thin_provision_enable"].as< uint32_t >() != 0 ? true : false; gcfg.io_size = io_size_in_kb * 1024; HS_REL_ASSERT(io_size_in_kb && (io_size_in_kb % 4 == 0), diff --git a/src/homeblks/volume/volume.cpp b/src/homeblks/volume/volume.cpp index 77ff6fee7..ec2bcef49 100644 --- a/src/homeblks/volume/volume.cpp +++ b/src/homeblks/volume/volume.cpp @@ -171,7 +171,7 @@ Volume::Volume(const vol_params& params) : throw std::runtime_error("shutdown in progress"); } m_sobject = m_hb->sobject_mgr()->create_object("volume", params.vol_name, - std::bind(&Volume::get_status, this, std::placeholders::_1)); + std::bind(&Volume::get_status, this, std::placeholders::_1)); m_state = vol_state::UNINITED; } @@ -190,7 +190,7 @@ Volume::Volume(meta_blk* mblk_cookie, sisl::byte_view sb_buf) : HS_REL_ASSERT_EQ(sb->magic, vol_sb_magic, "magic mismatch"); m_hb = HomeBlks::safe_instance(); m_sobject = m_hb->sobject_mgr()->create_object("volume", sb->vol_name, - std::bind(&Volume::get_status, this, std::placeholders::_1)); + std::bind(&Volume::get_status, this, std::placeholders::_1)); } void Volume::init() { @@ -335,7 +335,69 @@ indx_tbl* Volume::recover_indx_tbl(btree_super_block& sb, btree_cp_sb& cp_info) return static_cast< indx_tbl* >(tbl); } +#if 0 +// TODO: use these functions for near future optimization of write path for thin provisioning volumes to enable skipping +// writing empty blocks in subrange intervals for requested buffer instead of detecting the all-zero-buffer requests. +static std::vector< std::pair< int, int > > compute_range_intervals(const uint8_t* buf, size_t page_size, + uint32_t nlbas, bool empty_blocks = false) { + std::vector< std::pair< int, int > > intervals; + bool in_empty_region = false; + int current_range_start = -1; + int current_range_length = 1; + for (uint32_t i = 0; i < nlbas; i++) { + const uint8_t* page_start = buf + (i * page_size); + bool is_page_empty = (empty_blocks == is_buf_zero(page_start, page_size)); + if (is_page_empty) { + if (!in_empty_region) { + current_range_start = i; + current_range_length = 1; + in_empty_region = true; + } else { + current_range_length++; + } + } else { + if (in_empty_region) { intervals.push_back(std::make_pair(current_range_start, current_range_length)); } + in_empty_region = false; + } + } + if (in_empty_region) { intervals.push_back(std::make_pair(current_range_start, current_range_length)); } + return intervals; +} + +static std::string print_ranges(lba_t start_lba, const std::vector< std::pair< int, int > >& intervals) { + auto intervals_to_string = [start_lba](const std::vector< std::pair< int, int > >& intervals) -> std::string { + std::vector< std::string > result_strings; + std::transform(intervals.begin(), intervals.end(), std::back_inserter(result_strings), + [start_lba](const std::pair< int, int >& p) -> std::string { + // Use a static buffer to hold the formatted string + static char buffer[32]; + std::snprintf(buffer, sizeof(buffer), "<%ld,%d>", p.first + start_lba, p.second); + return buffer; + }); + return std::accumulate(result_strings.begin(), result_strings.end(), std::string("")); + }; + return intervals_to_string(intervals); +} +#endif + std::error_condition Volume::write(const vol_interface_req_ptr& iface_req) { + std::error_condition ret{no_error}; + if (!HS_DYNAMIC_CONFIG(generic->boot_thin_provisioning)) { + return write_internal(iface_req); + } else { + if (iface_req->is_zero_request(get_page_size())) { + THIS_VOL_LOG(TRACE, volume, iface_req, "zero request <{}, {}>", iface_req->lba, iface_req->nlbas); + iface_req->op_type = Op_type::UNMAP; + ret = unmap(iface_req); + } else { + ret = write_internal(iface_req); + } + } + iface_req->op_type = Op_type::WRITE; + return ret; +} + +std::error_condition Volume::write_internal(const vol_interface_req_ptr& iface_req) { static thread_local std::vector< BlkId > bid{}; std::error_condition ret{no_error}; @@ -924,11 +986,11 @@ sisl::status_response Volume::get_status(const sisl::status_request& request) { auto active_indx_json = get_active_indx()->sobject()->run_callback(request).json; if (!active_indx_json.empty()) { response.json["index"] = active_indx_json; } - response.json["name"] = sobject()->name(); + response.json["name"] = sobject()->name(); response.json["type"] = sobject()->type(); response.json["uuid"] = boost::lexical_cast< std::string >(get_uuid()); response.json["state"] = is_offline() ? "Offline" : "Online"; - response.json["size"]= get_size(); + response.json["size"] = get_size(); return response; } diff --git a/src/homeblks/volume/volume.hpp b/src/homeblks/volume/volume.hpp index 10c2fbbf5..2617dddc7 100644 --- a/src/homeblks/volume/volume.hpp +++ b/src/homeblks/volume/volume.hpp @@ -464,6 +464,7 @@ class Volume : public std::enable_shared_from_this< Volume > { * @return :- no_error if there is no error. It doesn't throw any exception */ std::error_condition write(const vol_interface_req_ptr& hb_req); + std::error_condition write_internal(const vol_interface_req_ptr& hb_req); /* Read from lba * @param hb_req :- it expects this request to be created @@ -729,7 +730,7 @@ struct volume_req : indx_req { csum_t* j_csum = (csum_t*)mem; if (!is_unmap() && active_nlbas_written != nlbas()) { - VOL_ERROR_LOG(vol()->get_name(), "all lbas are not written. lba written {}, lba supposed to write{}", + VOL_ERROR_LOG(vol()->get_name(), "all lbas are not written. lba written {}, lba supposed to write: {}", active_nlbas_written, nlbas()); } for (lba_count_t i{0}; !is_unmap() && i < active_nlbas_written; ++i) { From 63bc941a0a8497c4ab586cb4b84a0b4a5a873fc9 Mon Sep 17 00:00:00 2001 From: Mehdi Hosseini <116847813+shosseinimotlagh@users.noreply.github.com> Date: Wed, 13 Mar 2024 15:08:32 -0700 Subject: [PATCH 3/5] Update conanfile.py --- conanfile.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conanfile.py b/conanfile.py index 0fb9488de..9a83306c5 100644 --- a/conanfile.py +++ b/conanfile.py @@ -2,7 +2,7 @@ class HomestoreConan(ConanFile): name = "homestore" - version = "3.7.2" + version = "3.thin.1" homepage = "https://github.corp.ebay.com/SDS/homestore" description = "HomeStore" From 44519c83870de65e4ea547de2b70a817cee75a3f Mon Sep 17 00:00:00 2001 From: Mehdi Hosseini <116847813+shosseinimotlagh@users.noreply.github.com> Date: Wed, 13 Mar 2024 15:18:58 -0700 Subject: [PATCH 4/5] Update conanfile.py --- conanfile.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conanfile.py b/conanfile.py index 9a83306c5..048f46f9b 100644 --- a/conanfile.py +++ b/conanfile.py @@ -2,7 +2,7 @@ class HomestoreConan(ConanFile): name = "homestore" - version = "3.thin.1" + version = "3.8.1" homepage = "https://github.corp.ebay.com/SDS/homestore" description = "HomeStore" From c6630a2a97f24168faa26c7445215c788f8e3395 Mon Sep 17 00:00:00 2001 From: Mehdi Hosseini <116847813+shosseinimotlagh@users.noreply.github.com> Date: Tue, 26 Mar 2024 10:49:02 -0700 Subject: [PATCH 5/5] SDSTOR-13146 : Update AM's /api/v1/utilization endpoint (#362) --- conanfile.py | 2 +- src/homeblks/homeblks_http_server.cpp | 17 +++++++++++------ 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/conanfile.py b/conanfile.py index 048f46f9b..384e38a22 100644 --- a/conanfile.py +++ b/conanfile.py @@ -2,7 +2,7 @@ class HomestoreConan(ConanFile): name = "homestore" - version = "3.8.1" + version = "3.8.2" homepage = "https://github.corp.ebay.com/SDS/homestore" description = "HomeStore" diff --git a/src/homeblks/homeblks_http_server.cpp b/src/homeblks/homeblks_http_server.cpp index f7bffcd92..208fc0f29 100644 --- a/src/homeblks/homeblks_http_server.cpp +++ b/src/homeblks/homeblks_http_server.cpp @@ -155,12 +155,13 @@ void HomeBlksHttpServer::set_log_level(const Pistache::Rest::Request& request, response.send(Pistache::Http::Code::Ok, resp); } -void HomeBlksHttpServer::get_utilization(const Pistache::Rest::Request& request, Pistache::Http::ResponseWriter response) -{ - const std::string vol_uuid = request.hasParam(":volumeUUID") ? request.param(":volumeUUID").as():""; +void HomeBlksHttpServer::get_utilization(const Pistache::Rest::Request& request, + Pistache::Http::ResponseWriter response) { + const std::string vol_uuid = + request.hasParam(":volumeUUID") ? request.param(":volumeUUID").as< std::string >() : ""; VolumePtr vol = nullptr; - if (vol_uuid.length() != 0) { + if (vol_uuid.length()) { boost::uuids::string_generator gen; boost::uuids::uuid uuid = gen(vol_uuid); vol = VolInterface::get_instance()->lookup_volume(uuid); @@ -170,10 +171,14 @@ void HomeBlksHttpServer::get_utilization(const Pistache::Rest::Request& request, } } nlohmann::json resp; - const auto total_data_size = VolInterface::get_instance()->get_system_capacity().initial_total_data_meta_size; + nlohmann::json partitions = nlohmann::json::array(); for (auto [uuid, vol_used] : VolInterface::get_instance()->get_used_size(vol)) { - resp[boost::uuids::to_string(uuid)] = std::to_string(static_cast (vol_used)/ total_data_size); + nlohmann::json partition; + partition["id"] = boost::uuids::to_string(uuid); + partition["usedCapacity"] = vol_used; + partitions.push_back(partition); } + resp["partitions"] = partitions; response.send(Pistache::Http::Code::Ok, resp.dump()); } void HomeBlksHttpServer::get_log_level(const Pistache::Rest::Request& request,