diff --git a/src/include/homestore/homestore.hpp b/src/include/homestore/homestore.hpp index 69d766092..72f6120c7 100644 --- a/src/include/homestore/homestore.hpp +++ b/src/include/homestore/homestore.hpp @@ -195,6 +195,14 @@ class HomeStore { unique< CrashSimulator > m_crash_simulator; #endif +public: + /*** for metacli tool ***/ + bool start_tool(const hs_input_params& input, const std::string& scan_type, + std::optional< uint16_t > debug_chunk_id = std::nullopt, + std::optional< blk_num_t > debug_blk_num = std::nullopt); + void stop_tool(); + shared< VirtualDev > create_vdev_cb_for_tool(const vdev_info& vinfo, bool load_existing); + private: void init_cache(); shared< VirtualDev > create_vdev_cb(const vdev_info& vinfo, bool load_existing); diff --git a/src/include/homestore/meta_service.hpp b/src/include/homestore/meta_service.hpp index 747ad998e..92db7f455 100644 --- a/src/include/homestore/meta_service.hpp +++ b/src/include/homestore/meta_service.hpp @@ -213,6 +213,14 @@ class MetaBlkService { nlohmann::json get_status(const int log_level); +public: + /************************* metacli function **********************/ + void scan(const std::string& scan_type, std::optional< uint16_t > debug_chunk_id = std::nullopt, + std::optional< uint32_t > debug_blk_num = std::nullopt); + void scan_blks_on_all_chunks(std::optional< uint16_t > debug_chunk_id = std::nullopt, + std::optional< uint32_t > debug_blk_num = std::nullopt) const; + void scan_blks_by_chain() const; + public: /*********************** static public function **********************/ uint64_t ovf_blk_max_num_data_blk() const; diff --git a/src/lib/device/device.h b/src/lib/device/device.h index 4682e07e5..33f1acf36 100644 --- a/src/lib/device/device.h +++ b/src/lib/device/device.h @@ -156,7 +156,7 @@ class DeviceManager { void format_devices(); uint32_t format_single_device(dev_info& dinfo); void commit_formatting(); - void load_devices(); + void load_devices(bool ignore_unkown_vdevs=false); void close_devices(); bool is_boot_in_degraded_mode() const { return m_boot_in_degraded_mode; } @@ -195,7 +195,7 @@ class DeviceManager { void remove_chunk_locked(shared< Chunk > chunk); private: - void load_vdevs(); + void load_vdevs(bool ignore_unkown_vdevs); int device_open_flags(const std::string& devname) const; std::vector< vdev_info > read_vdev_infos(const std::vector< PhysicalDev* >& pdevs); diff --git a/src/lib/device/device_manager.cpp b/src/lib/device/device_manager.cpp index f4b3cf19a..278a530d6 100644 --- a/src/lib/device/device_manager.cpp +++ b/src/lib/device/device_manager.cpp @@ -115,7 +115,7 @@ void DeviceManager::format_devices() { // Get common iomgr_attributes for (auto& dinfo : m_dev_infos) { format_single_device(dinfo); - } + } // Verify the first blocks to see if the devs are unique HS_REL_ASSERT(verify_unique_devs(), "Found duplicate physical devices in the system"); @@ -186,7 +186,7 @@ bool DeviceManager::verify_unique_devs() const { return ret; } -void DeviceManager::load_devices() { +void DeviceManager::load_devices(bool ignore_unkown_vdevs) { RELEASE_ASSERT_EQ(m_first_blk_hdr.version, first_block_header::CURRENT_SUPERBLOCK_VERSION, "We don't support superblock version upgrade yet"); @@ -241,7 +241,7 @@ void DeviceManager::load_devices() { } // 3. Recover vdevs from the physical devices. - load_vdevs(); + load_vdevs(ignore_unkown_vdevs); if (pdevs_to_format.empty() && !stale_first_blk_found) return; @@ -565,7 +565,7 @@ void DeviceManager::add_pdev_to_vdev(shared< VirtualDev > vdev, PhysicalDev* pde } } -void DeviceManager::load_vdevs() { +void DeviceManager::load_vdevs(bool ignore_unkown_vdevs) { std::unique_lock lg{m_vdev_mutex}; for (auto& [dtype, pdevs] : m_pdevs_by_type) { @@ -583,9 +583,14 @@ void DeviceManager::load_vdevs() { // we might have some missing pdevs in the sparse_vector m_all_pdevs, so skip them if (!pdev) continue; // Empty device will skip this callback. - pdev->load_chunks([this](cshared< Chunk >& chunk) -> bool { + pdev->load_chunks([this, &ignore_unkown_vdevs](cshared< Chunk >& chunk) -> bool { // Found a chunk for which vdev information is missing if (m_vdevs[chunk->vdev_id()] == nullptr) { + if (ignore_unkown_vdevs) { + LOGWARN("Ignoring chunk id={} belonging to unknown vdev_id={}", chunk->chunk_id(), + chunk->vdev_id()); + return true; + } LOGWARN("Found a chunk id={}, which is expected to be part of vdev_id={}, but that vdev " "information is missing, may be before vdev is created, system crashed. Need upper layer " "to retry vdev create", @@ -603,6 +608,10 @@ void DeviceManager::load_vdevs() { // Run initialization of all vdevs. for (auto& vdev : m_vdevs) { + if (ignore_unkown_vdevs && vdev == nullptr) { + LOGWARN("Ignoring initialization of vdev of unknown device type"); + continue; + } vdev->init(); } } diff --git a/src/lib/device/virtual_dev.cpp b/src/lib/device/virtual_dev.cpp index b06859b0b..7f981c938 100644 --- a/src/lib/device/virtual_dev.cpp +++ b/src/lib/device/virtual_dev.cpp @@ -755,6 +755,13 @@ void VirtualDev::recovery_completed() { } } +void VirtualDev::foreach_chunks(std::function< void(cshared< Chunk >&) >&& cb) { + std::unique_lock lg{m_mgmt_mutex}; + for (auto& [_, chunk] : m_all_chunks) { + cb(chunk); + } +} + ///////////////////////// VirtualDev Private Methods ///////////////////////////// uint64_t VirtualDev::to_dev_offset(BlkId const& b, Chunk** chunk) const { *chunk = m_dmgr.get_chunk_mutable(b.chunk_num()); diff --git a/src/lib/device/virtual_dev.hpp b/src/lib/device/virtual_dev.hpp index ef3cb886a..b371c6d0a 100644 --- a/src/lib/device/virtual_dev.hpp +++ b/src/lib/device/virtual_dev.hpp @@ -313,6 +313,8 @@ class VirtualDev { ///////////////////////// Meta operations on vdev //////////////////////// void update_vdev_private(const sisl::blob& data); + void foreach_chunks(std::function< void(cshared< Chunk >&) >&& cb); + private: uint64_t to_dev_offset(BlkId const& b, Chunk** chunk) const; bool is_chunk_available(cshared< Chunk >& chunk) const; diff --git a/src/lib/homestore.cpp b/src/lib/homestore.cpp index 3dd23bcc5..92ed65114 100644 --- a/src/lib/homestore.cpp +++ b/src/lib/homestore.cpp @@ -201,6 +201,65 @@ bool HomeStore::start(const hs_input_params& input, hs_before_services_starting_ } } +// TODO: change to tool file with well structure and tool related parameters +bool HomeStore::start_tool(const hs_input_params& input, const std::string& scan_type, + std::optional< uint16_t > debug_chunk_id, std::optional< blk_num_t > debug_blk_num) { + auto& hs_config = HomeStoreStaticConfig::instance(); + hs_config.input = input; + + if (input.devices.empty()) { + LOGERROR("No devices provided to start tool"); + throw std::invalid_argument("null device list"); + } + + ///////////// Startup resource status and other manager outside core services ///////////////////////// + sisl::ObjCounterRegistry::enable_metrics_reporting(); + sisl::MallocMetrics::enable(); + + LOGINFO("HomeStore version: {}", version); + static std::once_flag flag1; + std::call_once(flag1, [this]() { + sisl::VersionMgr::addVersion(PACKAGE_NAME, version::Semver200_version(PACKAGE_VERSION)); + m_periodic_logger = + sisl::logging::CreateCustomLogger("homestore", "_periodic", false, true /* tee_to_stdout_stderr */); + sisl::logging::SetLogPattern("[%D %T.%f] [%^%L%$] [%t] %v", m_periodic_logger); + }); + + HomeStoreDynamicConfig::init_settings_default(); + + LOGINFO("HS tool is loading with meta service", m_services.list()); + m_meta_service = std::make_unique< MetaBlkService >(); + + m_dev_mgr = std::make_unique< DeviceManager >(input.devices, bind_this(HomeStore::create_vdev_cb_for_tool, 2)); + if (m_dev_mgr->is_first_time_boot()) { + LOGERROR("HS tool requires existing homestore setup on the devices"); + return false; + } + + m_dev_mgr->load_devices(true /* ignore unknown vdevs */); + if (input.has_fast_dev()) { + hs_utils::set_btree_mempool_size(m_dev_mgr->atomic_page_size({HSDevType::Fast})); + } else { + hs_utils::set_btree_mempool_size(m_dev_mgr->atomic_page_size({HSDevType::Data})); + } + + LOGINFO("HS tool loaded devices successfully"); + LOGINFO("HS tool starting scan meta blks, dynamic_config_version={}, static_config: {}", HS_DYNAMIC_CONFIG(version), + HomeStoreStaticConfig::instance().to_json().dump(4)); + m_meta_service->scan(scan_type, debug_chunk_id, debug_blk_num); + return true; +} + +void HomeStore::stop_tool() { + LOGINFO("HS tool stopping services"); + m_meta_service->stop(); + m_meta_service.reset(); + + m_dev_mgr->close_devices(); + m_dev_mgr.reset(); + LOGINFO("HS tool stopped all services"); +} + void HomeStore::format_and_start(std::map< uint32_t, hs_format_params >&& format_opts) { std::map< HSDevType, float > total_pct_by_type = {{HSDevType::Fast, 0.0f}, {HSDevType::Data, 0.0f}}; // Accumulate total percentage of services on each device type @@ -465,6 +524,27 @@ shared< VirtualDev > HomeStore::create_vdev_cb(const vdev_info& vinfo, bool load return ret_vdev; } +shared< VirtualDev > HomeStore::create_vdev_cb_for_tool(const vdev_info& vinfo, bool load_existing) { + shared< VirtualDev > ret_vdev; + auto& hs_config = HomeStoreStaticConfig::instance(); + auto vdev_context = r_cast< const hs_vdev_context* >(vinfo.get_user_private()); + + switch (vdev_context->type) { + case hs_vdev_type_t::META_VDEV: + if (has_meta_service()) { + ret_vdev = m_meta_service->open_vdev(vinfo, load_existing); + } else { + LOGERROR("HS tool requires meta service to be enabled"); + } + break; + default: + LOGINFO("[HS_TOOL] skip vdev_type {}", vdev_context->type); + break; + } + + return ret_vdev; +} + uint64_t HomeStore::pct_to_size(float pct, HSDevType dev_type) const { uint64_t sz = uint64_cast((pct * static_cast< double >(m_dev_mgr->total_capacity(dev_type))) / 100); return sisl::round_up(sz, m_dev_mgr->optimal_page_size(dev_type)); diff --git a/src/lib/meta/meta_blk_service.cpp b/src/lib/meta/meta_blk_service.cpp index 557f23417..86f7df234 100644 --- a/src/lib/meta/meta_blk_service.cpp +++ b/src/lib/meta/meta_blk_service.cpp @@ -104,6 +104,19 @@ void MetaBlkService::start(bool need_format) { recover(); } +void MetaBlkService::scan(const std::string& scan_type, std::optional< uint16_t > debug_chunk_id, + std::optional< blk_num_t > debug_blk_num) { + if (scan_type == "chunk") { + HS_LOG(INFO, metablk, "Scanning meta blks on all chunk"); + scan_blks_on_all_chunks(debug_chunk_id, debug_blk_num); + } else if (scan_type == "chain") { + HS_LOG(INFO, metablk, "Scanning meta blks by chain"); + scan_blks_by_chain(); + } else { + HS_LOG(ERROR, metablk, "Unsupported scan type: {}", scan_type); + } +} + void MetaBlkService::stop() { { std::lock_guard< decltype(m_shutdown_mtx) > lg_shutdown{m_shutdown_mtx}; @@ -222,6 +235,316 @@ void MetaBlkService::scan_meta_blks() { if (self_recover) { set_self_recover(); } } +ENUM(PGState, uint8_t, ALIVE = 0, DESTROYED); + +struct ShardInfo { + enum class State : uint8_t { + OPEN = 0, + SEALED = 1, + DELETED = 2, + }; + + uint64_t id; + uint16_t placement_group; + State state; + uint64_t lsn; // created_lsn + uint64_t created_time; + uint64_t last_modified_time; + uint64_t available_capacity_bytes; + uint64_t total_capacity_bytes; + std::optional< boost::uuids::uuid > current_leader{std::nullopt}; + + auto operator<=>(ShardInfo const& rhs) const { return id <=> rhs.id; } + auto operator==(ShardInfo const& rhs) const { return id == rhs.id; } + bool is_open() const { return state == State::OPEN; } + + // to_string method for printing shard info + std::string to_string() const { + return fmt::format("ShardInfo: id={}, placement_group={}, state={}, lsn={}, " + "created_time={}, last_modified_time={}, available_capacity_bytes={}, " + "total_capacity_bytes={}, current_leader={}", + id, placement_group, static_cast< uint8_t >(state), lsn, created_time, last_modified_time, + available_capacity_bytes, total_capacity_bytes, + current_leader.has_value() ? boost::uuids::to_string(current_leader.value()) : "null"); + } +}; + +#pragma pack(push, 1) +struct pg_info_superblk { + uint16_t id; + PGState state; + // num_expected_members means how many members a pg should have, while num_dynamic_members is the actual + // members count, as members might float during member replacement. + uint32_t num_expected_members; + uint32_t num_dynamic_members; + uint32_t num_chunks; + uint64_t chunk_size; + boost::uuids::uuid replica_set_uuid; + uint64_t pg_size; + boost::uuids::uuid index_table_uuid; + uint64_t blob_sequence_num; + uint64_t active_blob_count; // Total number of active blobs + uint64_t tombstone_blob_count; // Total number of tombstones + uint64_t total_occupied_blk_count; // Total number of occupied blocks + uint64_t total_reclaimed_blk_count; // Total number of reclaimed blocks + char data[1]; // ISO C++ forbids zero-size array + + pg_info_superblk() = default; + pg_info_superblk(pg_info_superblk const& rhs) { *this = rhs; } +}; + +struct DataHeader { + static constexpr uint8_t data_header_version = 0x01; + static constexpr uint64_t data_header_magic = 0x21fdffdba8d68fc6; // echo "BlobHeader" | md5sum + + enum class data_type_t : uint32_t { SHARD_INFO = 1, BLOB_INFO = 2 }; + + bool valid() const { return ((magic == data_header_magic) && (version <= data_header_version)); } + + uint64_t magic{data_header_magic}; + uint8_t version{data_header_version}; + data_type_t type{data_type_t::BLOB_INFO}; +}; + +struct shard_info_superblk : DataHeader { + ShardInfo info; + uint16_t p_chunk_id; + uint16_t v_chunk_id; + + // to_string method for printing shard_info_superblk with DataHeader and chunk_ids + std::string to_string() const { + return fmt::format( + "shard_info_superblk: [DataHeader: magic=0x{:x}, version={}, type={}], [{}], p_chunk_id={}, v_chunk_id={}", + magic, version, static_cast< uint32_t >(type), info.to_string(), p_chunk_id, v_chunk_id); + } + + // Helper method to print chunk ids + std::string chunk_ids_to_string() const { + return fmt::format("chunk_ids: [p_chunk_id={}, v_chunk_id={}]", p_chunk_id, v_chunk_id); + } +}; + +#pragma pack(pop) + +void MetaBlkService::scan_blks_on_all_chunks(std::optional< uint16_t > debug_chunk_id, + std::optional< blk_num_t > debug_blk_num) const { + constexpr blk_num_t BATCH_SIZE = 1024; + const auto blk_sz = block_size(); + const auto batch_size_bytes = BATCH_SIZE * blk_sz; + + HS_LOG(INFO, metablk, "Starting batch scan with batch_size={} blocks ({} bytes)", BATCH_SIZE, + in_bytes(batch_size_bytes)); + + m_sb_vdev->foreach_chunks([this, blk_sz, batch_size_bytes, BATCH_SIZE, debug_chunk_id, debug_blk_num](cshared< Chunk >& chunk) { + // Skip chunks that don't match debug_chunk_id if specified + if (debug_chunk_id.has_value() && chunk->chunk_id() != debug_chunk_id.value()) { return; } + + HS_LOG(INFO, metablk, "Scanning chunk: {}, size: {}, start_offset: {}", chunk->chunk_id(), + in_bytes(chunk->size()), chunk->start_offset()); + + const blk_num_t total_blks = chunk->size() / blk_sz; + HS_LOG(INFO, metablk, "Number of blocks in chunk {}: {}, chunk_size={} block_size={}", chunk->chunk_id(), + total_blks, chunk->size(), blk_sz); + + int valid_cnt{0}; + int invalid_cnt{0}; + int ssb_cnt{0}; + int meta_blk_cnt{0}; + int ovf_blk_cnt{0}; + + auto batch_buf = hs_utils::iobuf_alloc(batch_size_bytes, sisl::buftag::metablk, align_size()); + for (blk_num_t batch_start = 0; batch_start < total_blks; batch_start += BATCH_SIZE) { + // If debug_blk_num is specified, set batch_start to debug_blk_num + if (debug_blk_num.has_value()) { + batch_start = debug_blk_num.value(); + HS_REL_ASSERT(batch_start < total_blks, "debug_blk_num {} exceeds total_blks {}", batch_start, + total_blks); + } + + const blk_num_t blks_in_batch = std::min(BATCH_SIZE, total_blks - batch_start); + const size_t read_size = blks_in_batch * blk_sz; + const auto offset_in_chunk = batch_start * blk_sz; + + // Read a batch of blocks + auto ec = m_sb_vdev->sync_read(r_cast< char* >(batch_buf), read_size, chunk, offset_in_chunk); + if (ec.value()) { + HS_LOG(ERROR, metablk, "Failed to read batch at chunk: {}, offset: {}, size: {}, error: {}", + chunk->chunk_id(), offset_in_chunk, read_size, ec.message()); + hs_utils::iobuf_free(batch_buf, sisl::buftag::metablk); + return; + } + + // Process each block in the batch + for (blk_num_t i = 0; i < blks_in_batch; ++i) { + const auto blk_offset_in_batch = i * blk_sz; + auto* blk_ptr = r_cast< uint8_t* >(batch_buf) + blk_offset_in_batch; + const uint32_t magic = *r_cast< uint32_t* >(blk_ptr); + const blk_num_t global_blk_num = batch_start + i; + + switch (magic) { + case META_BLK_SB_MAGIC: { + ssb_cnt++; + auto* ssb = r_cast< meta_blk_sb* >(blk_ptr); + HS_LOG(INFO, metablk, "[SSB] found at blk#{}@c{}, blkid=[{}], blk={}", global_blk_num, + chunk->chunk_id(), ssb->bid.to_string(), ssb->to_string()); + valid_cnt++; + break; + } + case META_BLK_MAGIC: { + meta_blk_cnt++; + auto* mblk = r_cast< meta_blk* >(blk_ptr); + HS_LOG(INFO, metablk, "[MetaBlk] found at blk#{}@c{} blk_content=[{}]", global_blk_num, + chunk->chunk_id(), mblk->to_string()); + + // Check if this is a PGManager meta block + if (std::string(mblk->hdr.h.type) == "PGManager") { + // Get the context data (pg_info_superblk) + const auto* pg_info = r_cast< const pg_info_superblk* >(mblk->get_context_data()); + HS_LOG(INFO, metablk, + "[PGManager] chunk={} blk={} pg_info: id={}, state={}, num_expected_members={}, " + "num_dynamic_members={}, num_chunks={}, chunk_size={}, pg_size={}, " + "blob_sequence_num={}, active_blob_count={}, tombstone_blob_count={}, " + "total_occupied_blk_count={}, total_reclaimed_blk_count={}", + chunk->chunk_id(), global_blk_num, pg_info->id, static_cast< uint8_t >(pg_info->state), + pg_info->num_expected_members, pg_info->num_dynamic_members, pg_info->num_chunks, pg_info->chunk_size, pg_info->pg_size, + pg_info->blob_sequence_num, pg_info->active_blob_count, pg_info->tombstone_blob_count, + pg_info->total_occupied_blk_count, pg_info->total_reclaimed_blk_count); + } else if (std::string(mblk->hdr.h.type) == "ShardManager") { + const auto* shard_info_blk = r_cast< const shard_info_superblk* >(mblk->get_context_data()); + HS_LOG(INFO, metablk, "[ShardManager] chunk={} blk={} shard_info: {}", chunk->chunk_id(), + global_blk_num, shard_info_blk->to_string()); + } + + valid_cnt++; + break; + } + case META_BLK_OVF_MAGIC: { + ovf_blk_cnt++; + auto* ovf_blk = r_cast< meta_blk_ovf_hdr* >(blk_ptr); + HS_LOG(INFO, metablk, "[OvfBlk] found at blk#{}@c{} {}", global_blk_num, chunk->chunk_id(), + ovf_blk->to_string()); + valid_cnt++; + break; + } + default: { + invalid_cnt++; + HS_LOG(DEBUG, metablk, "Invalid blk at chunk: {}, blk_num={}, offset_in_chunk: {}, magic: {}", + chunk->chunk_id(), global_blk_num, global_blk_num * blk_sz, magic); + + // Debug specific block if parameters are provided + if (debug_chunk_id.has_value() && debug_blk_num.has_value() && + chunk->chunk_id() == debug_chunk_id.value() && global_blk_num == debug_blk_num.value()) { + HS_LOG(INFO, metablk, "try to read blk#{}@c{}", debug_blk_num.value(), debug_chunk_id.value()); + auto* mblk = r_cast< meta_blk* >(blk_ptr); + HS_LOG(INFO, metablk, "blk#{}@c{} hdr is {}", debug_blk_num.value(), debug_chunk_id.value(), + mblk->to_string()); + + // Print whole 4k data + HS_LOG(INFO, metablk, "blk#{}@c{} whole 4k data:", debug_blk_num.value(), debug_chunk_id.value()); + const uint8_t* data_ptr = blk_ptr; + for (size_t i = 0; i < 4096; i += 16) { + std::ostringstream oss; + for (size_t j = 0; j < 16 && (i + j) < 4096; ++j) { + oss << std::hex << std::setw(2) << std::setfill('0') + << static_cast< int >(data_ptr[i + j]) << " "; + } + HS_LOG(INFO, metablk, "{}", oss.str()); + } + } + } + } + } + + // If debug_blk_num is specified, break after processing this batch + if (debug_blk_num.has_value()) { break; } + } + + hs_utils::iobuf_free(batch_buf, sisl::buftag::metablk); + HS_LOG(INFO, metablk, + "scanned chunk: {}, valid_cnt: {}, invalid_cnt: {}, ssb_cnt: {}, meta_blk_cnt: {}, ovf_blk_cnt: {}", + chunk->chunk_id(), valid_cnt, invalid_cnt, ssb_cnt, meta_blk_cnt, ovf_blk_cnt); + }); +} + +void MetaBlkService::scan_blks_by_chain() const { + // load ssb at first + auto bid = m_meta_vdev_context->first_blkid; + HS_LOG(INFO, metablk, "Loading meta ssb blkid: {}", bid.to_string()); + auto* ssb = r_cast< meta_blk_sb* >(hs_utils::iobuf_alloc(block_size(), sisl::buftag::metablk, align_size())); + std::memset(uintptr_cast(ssb), 0, block_size()); + read(bid, uintptr_cast(ssb), block_size()); + LOGINFO("Successfully loaded meta ssb from disk: {}", ssb->to_string()); + + // scan meta blk chain + bid = ssb->next_bid; + auto prev_meta_bid = ssb->bid; + std::map< meta_sub_type, int > type_cnt; + + while (bid.is_valid()) { + auto* mblk = r_cast< meta_blk* >(hs_utils::iobuf_alloc(block_size(), sisl::buftag::metablk, align_size())); + read(bid, uintptr_cast(mblk), block_size()); + HS_LOG(INFO, metablk, "Scanned meta blk: {}", mblk->to_string()); + type_cnt[mblk->hdr.h.type] += 1; + + // Check if this is a PGManager meta block + if (std::string(mblk->hdr.h.type) == "PGManager") { + // Get the context data (pg_info_superblk) + const auto* pg_info = r_cast< const pg_info_superblk* >(mblk->get_context_data()); + HS_LOG(INFO, metablk, + "[PGManager] pg_info: id={}, state={}, num_expected_members={}, " + "num_dynamic_members={}, num_chunks={}, chunk_size={}, pg_size={}, " + "blob_sequence_num={}, active_blob_count={}, tombstone_blob_count={}, " + "total_occupied_blk_count={}, total_reclaimed_blk_count={}", + pg_info->id, static_cast< uint8_t >(pg_info->state), pg_info->num_expected_members, + pg_info->num_dynamic_members, pg_info->num_chunks, pg_info->chunk_size, pg_info->pg_size, + pg_info->blob_sequence_num, pg_info->active_blob_count, pg_info->tombstone_blob_count, + pg_info->total_occupied_blk_count, pg_info->total_reclaimed_blk_count); + } + + if (prev_meta_bid.to_integer() != mblk->hdr.h.prev_bid.to_integer()) { + HS_LOG(INFO, metablk, "[type={}], meta blk has wrong prev meta bid. expected={}, actual={}", + mblk->hdr.h.type, prev_meta_bid.to_string(), mblk->hdr.h.prev_bid.to_string()); + } + prev_meta_bid = bid; + + auto obid = mblk->hdr.h.ovf_bid; + uint64_t read_sz = mblk->hdr.h.context_sz > meta_blk_context_sz() ? 0 : mblk->hdr.h.context_sz; + while (obid.is_valid()) { + // ovf blk header occupies whole blk; + auto* ovf_hdr = + r_cast< meta_blk_ovf_hdr* >(hs_utils::iobuf_alloc(block_size(), sisl::buftag::metablk, align_size())); + read(obid, uintptr_cast(ovf_hdr), block_size()); + + HS_LOG(INFO, metablk, "Scanned ovf blk: {}", ovf_hdr->to_string(true)); + + read_sz += ovf_hdr->h.context_sz; + obid = ovf_hdr->h.next_bid; + hs_utils::iobuf_free(uintptr_cast(ovf_hdr), sisl::buftag::metablk); + } + + if (read_sz != static_cast< uint64_t >(mblk->hdr.h.context_sz)) { + LOGERROR("[type={}], total size read: {} mismatch from meta blk context_sz: {}", mblk->hdr.h.type, read_sz, + mblk->hdr.h.context_sz); + } else { + LOGDEBUG("[type={}], meta blk scan completed", mblk->hdr.h.type); + } + + // move on to next meta blk; + bid = mblk->hdr.h.next_bid; + hs_utils::iobuf_free(uintptr_cast(mblk), sisl::buftag::metablk); + } + + HS_LOG(INFO, metablk, "Meta blk scan summary:"); + auto total_valid_count = 0; + for (const auto& [type, cnt] : type_cnt) { + HS_LOG(INFO, metablk, "type: {}, count: {}", type, cnt); + total_valid_count += cnt; + } + HS_LOG(INFO, metablk, "Total valid meta blk count: {}", total_valid_count); + + hs_utils::iobuf_free(uintptr_cast(ssb), sisl::buftag::metablk); +} + bool MetaBlkService::scan_and_load_meta_blks(meta_blk_map_t& meta_blks, ovf_hdr_map_t& ovf_blk_hdrs, BlkId* last_mblk_id, client_info_map_t& sub_info) { // take a look so that before scan is complete, no add/remove/update operations will be allowed; diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt index 76a94c1dd..8a4db1adb 100644 --- a/src/tests/CMakeLists.txt +++ b/src/tests/CMakeLists.txt @@ -55,6 +55,10 @@ if (${build_nonio_tests}) target_link_libraries(test_device_manager homestore ${COMMON_TEST_DEPS} GTest::gmock) add_test(NAME DeviceManager COMMAND test_device_manager) + set(TEST_SCAN_TOOL_SOURCES scan_tool.cpp) + add_executable(scan_tool ${TEST_SCAN_TOOL_SOURCES}) + target_link_libraries(scan_tool homestore ${COMMON_TEST_DEPS} GTest::gmock) + add_executable(test_append_blk_allocator) target_sources(test_append_blk_allocator PRIVATE test_append_blkalloc.cpp) target_link_libraries(test_append_blk_allocator homestore ${COMMON_TEST_DEPS} GTest::gmock) diff --git a/src/tests/scan_tool.cpp b/src/tests/scan_tool.cpp new file mode 100755 index 000000000..c4e861c12 --- /dev/null +++ b/src/tests/scan_tool.cpp @@ -0,0 +1,114 @@ +/********************************************************************************* + * Modifications Copyright 2017-2019 eBay Inc. + * + * + * 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 + * https://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. + * + *********************************************************************************/ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +using namespace homestore; +SISL_LOGGING_INIT(HOMESTORE_LOG_MODS) +SISL_OPTIONS_ENABLE(logging, test_scan_tool, iomgr) + +SISL_OPTION_GROUP(test_scan_tool, + (device_list, "", "device_list", "device list", ::cxxopts::value< std::vector< std::string > >(), + "[dev_path:dev_type,..]"), + (scan_type, "", "scan_type", "scan_type", ::cxxopts::value< std::string >()->default_value("chunk"), + "chunk/chain"), + (debug_chunk_id, "", "debug_chunk_id", "chunk id for debug printing", + ::cxxopts::value< uint16_t >(), "chunk_id"), + (debug_blk_num, "", "debug_blk_num", "block number for debug printing", + ::cxxopts::value< uint64_t >(), "blk_num")); + +class ScanTool : public ::testing::Test { +public: + virtual void SetUp() override { + LOGINFO("Starting iomgr with 2 threads, spdk=false"); + ioenvironment.with_iomgr(iomgr::iomgr_params{.num_threads = 2, .is_spdk = false}); + const uint64_t app_mem_size = 2147483648; + const int max_data_size = 67108864; + const int max_snapshot_batch_size_in_bytes = 128 * 1024 * 1024; + LOGINFO("Initialize and start HS tool with app_mem_size = {}", homestore::in_bytes(app_mem_size)); + + auto devices = SISL_OPTIONS["device_list"].as< std::vector< std::string > >(); + ASSERT_FALSE(devices.empty()) << "Device list is empty. Please provide a valid device list."; + + std::vector< homestore::dev_info > device_info; + for (auto const& dev : devices) { + auto delimiter_pos = dev.find(':'); + ASSERT_TRUE(delimiter_pos != std::string::npos && delimiter_pos != 0 && delimiter_pos != dev.size() - 1) + << "Invalid device format: " << dev << ". Expected format is 'path:type'."; + + std::string dev_path = dev.substr(0, delimiter_pos); + std::string dev_type_str = dev.substr(delimiter_pos + 1); + ASSERT_TRUE(dev_type_str == "HDD" || dev_type_str == "SSD" || dev_type_str == "NVME") + << "Unknown device type: " << dev_type_str << ". Expected types are 'HDD', 'SSD', or 'NVME'."; + LOGINFO("Adding device {} of type {}", dev_path, dev_type_str); + auto hs_type = (dev_type_str == "HDD") ? homestore::HSDevType::Data : homestore::HSDevType::Fast; + device_info.emplace_back(dev_path, hs_type); + } + + auto scan_type = SISL_OPTIONS["scan_type"].as< std::string >(); + + // Parse optional debug parameters + std::optional< uint16_t > debug_chunk_id = std::nullopt; + std::optional< blk_num_t > debug_blk_num = std::nullopt; + + if (SISL_OPTIONS.count("debug_chunk_id")) { + debug_chunk_id = SISL_OPTIONS["debug_chunk_id"].as< uint16_t >(); + LOGINFO("Debug chunk_id set to: {}", debug_chunk_id.value()); + } + + if (SISL_OPTIONS.count("debug_blk_num")) { + debug_blk_num = SISL_OPTIONS["debug_blk_num"].as< uint64_t >(); + LOGINFO("Debug blk_num set to: {}", debug_blk_num.value()); + } + + bool success = HomeStore::instance()->start_tool( + hs_input_params{.devices = device_info, + .app_mem_size = app_mem_size, + .max_data_size = max_data_size, + .max_snapshot_batch_size = max_snapshot_batch_size_in_bytes}, + scan_type, debug_chunk_id, debug_blk_num); + LOGINFO("HS tool started with status: {}", success ? "success" : "failure"); + } + + virtual void TearDown() override { + HomeStore::instance()->stop_tool(); + iomanager.stop(); + } +}; + +TEST_F(ScanTool, SimpleTool) { LOGINFO("HS scan tool test completed"); } + +int main(int argc, char* argv[]) { + SISL_OPTIONS_LOAD(argc, argv, logging, test_scan_tool, iomgr); + ::testing::InitGoogleTest(&argc, argv); + sisl::logging::SetLogger("test_scan_tool"); + spdlog::set_pattern("[%D %T%z] [%^%l%$] [%n] [%t] %v"); + + return RUN_ALL_TESTS(); +} diff --git a/src/tests/test_scripts/compare_metablk_chains.py b/src/tests/test_scripts/compare_metablk_chains.py new file mode 100644 index 000000000..f05201fa4 --- /dev/null +++ b/src/tests/test_scripts/compare_metablk_chains.py @@ -0,0 +1,492 @@ +#!/usr/bin/env python3 +""" +Script to reconstruct and compare metablk chains from two different scanning methods. +- First file: "chunk" entries from chunk traversal (may have orphaned blocks due to deletion) +- Second file: "chain" entries from chain traversal (complete chain) +Both chains start from SSB (Super Block) +""" + +import re +import sys +from collections import defaultdict + +class MetaBlk: + """Represents a meta block entry.""" + + def __init__(self, self_bid, next_bid, prev_bid, type_name, line, pg_shard_id=None): + self.self_bid = self_bid + self.next_bid = next_bid + self.prev_bid = prev_bid + self.type_name = type_name + self.line = line + self.pg_shard_id = pg_shard_id # Store PG ID or Shard ID + + def __repr__(self): + return f"MetaBlk({self.self_bid}, type={self.type_name})" + +def parse_bid(line, field_name): + """Extract bid field (self_bid, next_bid, prev_bid) from a log line.""" + pattern = rf'{field_name}:\s*(\[?blk#=\d+\s+count=\d+\s+chunk=\d+\]?|Invalid_Blkid)' + match = re.search(pattern, line) + if match: + bid_str = match.group(1).strip('[]') + return bid_str if bid_str != 'Invalid_Blkid' else None + return None + +def parse_type(line): + """Extract type from a log line.""" + match = re.search(r'type:\s*(\w+)', line) + return match.group(1) if match else 'unknown' + + +def parse_pg_shard_id(line): + """Extract PG ID or Shard ID from log line.""" + # For PGManager: [PGManager] chunk=X blk=Y pg_info: id=Z + if '[PGManager]' in line: + match = re.search(r'pg_info:\s*id=(\d+)', line) + if match: + return f"PG:{match.group(1)}" + + # For ShardManager: [ShardManager] chunk=X blk=Y shard_info: ... [ShardInfo: id=Z + if '[ShardManager]' in line: + match = re.search(r'\[ShardInfo:\s*id=(\d+)', line) + if match: + return f"Shard:{match.group(1)}" + + return None + +def parse_found_at_bid(line): + """Extract the 'found at' bid from the log line (e.g., 'found at blk#229376@c2').""" + match = re.search(r'found at (?:blkid=)?blk#(\d+)@c(\d+)', line) + if match: + blk_num, chunk = match.groups() + return f"blk#={blk_num} count=1 chunk={chunk}" + return None + +def parse_chunk_entries(log_file, target_ssb_bid): + """Parse chunk traversal entries and find SSB matching the target.""" + blocks = {} + inconsistent_blocks = [] + ssb_bid = None + + with open(log_file, 'r') as f: + for line in f: + # Find SSB - look for "[SSB] found at blkid=" matching target + if '[SSB] found at blkid=' in line or '[SSB] found at blk#' in line: + # Extract the found_at_bid from the log line + found_at_bid = parse_found_at_bid(line) + self_bid = parse_bid(line, 'self_bid') + + # Check consistency + if found_at_bid and self_bid: + if found_at_bid != self_bid: + inconsistent_blocks.append((line.strip(), found_at_bid, self_bid)) + continue + + if self_bid == target_ssb_bid: + ssb_bid = self_bid + next_bid = parse_bid(line, 'next_bid') + prev_bid = parse_bid(line, 'prev_bid') + blocks[self_bid] = MetaBlk(self_bid, next_bid, prev_bid, 'SSB', line.strip()) + continue + + # Parse MetaBlk entries - new format: "[MetaBlk] found at blk#X@cY" + if '[MetaBlk] found at' in line: + # Extract the found_at_bid from the log line + found_at_bid = parse_found_at_bid(line) + self_bid = parse_bid(line, 'self_bid') + + # Check consistency: self_bid should match found_at_bid + if found_at_bid and self_bid: + if found_at_bid != self_bid: + # Inconsistent block - self_bid doesn't match the location where it was found + inconsistent_blocks.append((line.strip(), found_at_bid, self_bid)) + continue + + # Consistent block - add it + next_bid = parse_bid(line, 'next_bid') + prev_bid = parse_bid(line, 'prev_bid') + type_name = parse_type(line) + blocks[self_bid] = MetaBlk(self_bid, next_bid, prev_bid, type_name, line.strip()) + + # Parse PGManager and ShardManager entries + if '[PGManager]' in line or '[ShardManager]' in line: + # Extract chunk and blk from log + chunk_match = re.search(r'chunk=(\d+)', line) + blk_match = re.search(r'blk=(\d+)', line) + if chunk_match and blk_match: + # Construct bid in the format used by the script + chunk = chunk_match.group(1) + blk = blk_match.group(1) + # The bid format is "blk#=X count=1 chunk=Y" + self_bid = f"blk#={blk} count=1 chunk={chunk}" + + # Check if this block already exists + if self_bid in blocks: + # Update with PG/Shard ID + pg_shard_id = parse_pg_shard_id(line) + blocks[self_bid].pg_shard_id = pg_shard_id + + return blocks, ssb_bid, inconsistent_blocks + + +def parse_chain_entries(log_file): + """Parse chain traversal entries and find SSB start.""" + blocks = {} + ssb_bid = None + + with open(log_file, 'r') as f: + for line in f: + # Find SSB - "Successfully loaded meta ssb from disk:" + if 'Successfully loaded meta ssb from disk:' in line: + self_bid = parse_bid(line, 'self_bid') + if self_bid: + ssb_bid = self_bid + next_bid = parse_bid(line, 'next_bid') + prev_bid = parse_bid(line, 'prev_bid') + blocks[self_bid] = MetaBlk(self_bid, next_bid, prev_bid, 'SSB', line.strip()) + continue + + # Parse Scanned meta blk entries + if 'Scanned meta blk:' in line: + self_bid = parse_bid(line, 'self_bid') + if self_bid: + next_bid = parse_bid(line, 'next_bid') + prev_bid = parse_bid(line, 'prev_bid') + type_name = parse_type(line) + blocks[self_bid] = MetaBlk(self_bid, next_bid, prev_bid, type_name, line.strip()) + + # Parse PGManager and ShardManager entries + if '[PGManager]' in line or '[ShardManager]' in line: + # Extract chunk and blk from log + chunk_match = re.search(r'chunk=(\d+)', line) + blk_match = re.search(r'blk=(\d+)', line) + if chunk_match and blk_match: + # Construct bid in the format used by the script + chunk = chunk_match.group(1) + blk = blk_match.group(1) + # The bid format is "blk#=X count=1 chunk=Y" + self_bid = f"blk#={blk} count=1 chunk={chunk}" + + # Check if this block already exists + if self_bid in blocks: + # Update with PG/Shard ID + pg_shard_id = parse_pg_shard_id(line) + blocks[self_bid].pg_shard_id = pg_shard_id + + return blocks, ssb_bid + +def build_chain(blocks, start_bid): + """Build chain from start_bid following next pointers, validating prev pointers.""" + chain = [] + visited = set() + current_bid = start_bid + prev_bid_expected = None + + while current_bid and current_bid not in visited: + if current_bid in blocks: + block = blocks[current_bid] + + # Validate prev pointer consistency + if prev_bid_expected is not None: + if block.prev_bid != prev_bid_expected: + # Note the inconsistency but continue + chain.append(f"WARNING: prev_bid mismatch at {current_bid}: expected={prev_bid_expected}, actual={block.prev_bid}") + + chain.append(block) + visited.add(current_bid) + prev_bid_expected = current_bid + current_bid = block.next_bid + else: + # Broken chain - next_bid points to non-existent block + chain.append(f"BROKEN_LINK -> {current_bid}") + break + + return chain, visited + +def find_orphaned_blocks(blocks, visited): + """Find blocks that are not part of the main chain.""" + all_bids = set(blocks.keys()) + orphaned = all_bids - visited + return orphaned + +def format_bid(bid): + """Format bid for compact display.""" + if not bid: + return "NULL" + # Extract blk#, chunk info for compact display + match = re.match(r'blk#=(\d+)\s+count=(\d+)\s+chunk=(\d+)', bid) + if match: + blk_num, count, chunk = match.groups() + if count == "1": + return f"blk#{blk_num}@c{chunk}" + else: + return f"blk#{blk_num}({count})@c{chunk}" + return bid + +def print_chain(chain, title): + """Print a chain with compact, table-like formatting.""" + print("\n" + "=" * 120) + print(title) + print("=" * 120) + print(f"{'Idx':<5} {'Self BID':<20} {'Type':<25} {'Prev BID':<20} {'Next BID':<20} {'PGID/SHARDID':<15}") + print("-" * 120) + + for i, item in enumerate(chain): + if isinstance(item, MetaBlk): + self_short = format_bid(item.self_bid) + prev_short = format_bid(item.prev_bid) + next_short = format_bid(item.next_bid) + pg_shard_display = item.pg_shard_id if item.pg_shard_id else "-" + + print( + f"{i:<5} {self_short:<20} {item.type_name:<25} {prev_short:<20} {next_short:<20} {pg_shard_display:<15}") + else: + print(f"{i:<5} {str(item)}") + print("=" * 120 + "\n") + +def check_bidirectional(block, blocks): + """Check if prev and next pointers are bidirectional.""" + prev_bidir = False + next_bidir = False + + # Check prev pointer + if block.prev_bid and block.prev_bid in blocks: + prev_block = blocks[block.prev_bid] + if prev_block.next_bid == block.self_bid: + prev_bidir = True + + # Check next pointer + if block.next_bid and block.next_bid in blocks: + next_block = blocks[block.next_bid] + if next_block.prev_bid == block.self_bid: + next_bidir = True + + return prev_bidir, next_bidir + +def print_orphaned(blocks, orphaned_bids, title): + """Print orphaned blocks (blocks not in the main chain) with bidirectional indicators.""" + if not orphaned_bids: + return + + print("\n" + "=" * 140) + print(title) + print("=" * 140) + print(f"{'ID':<5} {'Self BID':<20} {'Type':<25} {'Prev BID':<27} {'Next BID':<27} {'PGID/SHARDID':<15}") + print("-" * 140) + + for idx, bid in enumerate(sorted(orphaned_bids)): + block = blocks[bid] + self_short = format_bid(block.self_bid) + prev_short = format_bid(block.prev_bid) + next_short = format_bid(block.next_bid) + pg_shard_display = block.pg_shard_id if block.pg_shard_id else "-" + + # Check bidirectional links + prev_bidir, next_bidir = check_bidirectional(block, blocks) + + # Add symbols: ✓ for bidirectional, ✗ for broken/not found + if block.prev_bid: + prev_symbol = "✓" if prev_bidir else "✗" + prev_display = f"{prev_symbol} {prev_short}" + else: + prev_display = " NULL" + + if block.next_bid: + next_symbol = "✓" if next_bidir else "✗" + next_display = f"{next_symbol} {next_short}" + else: + next_display = " NULL" + + print( + f"{idx:<5} {self_short:<20} {block.type_name:<25} {prev_display:<27} {next_display:<27} {pg_shard_display:<15}") + + print("\n" + "=" * 140 + "\n") + +def print_inconsistent_blocks(inconsistent_blocks): + """Print blocks where self_bid doesn't match the location where they were found.""" + if not inconsistent_blocks: + return + + print("\n" + "=" * 140) + print(f"INCONSISTENT BLOCKS ({len(inconsistent_blocks)} blocks)") + print("=" * 140) + print("These blocks have self_bid that doesn't match the location where they were found.") + print("This indicates corrupted or invalid metadata.") + print("-" * 140) + print(f"{'ID':<5} {'Found At':<20} {'Self BID':<20} {'Type':<25}") + print("-" * 140) + + for idx, (line, found_at_bid, self_bid) in enumerate(inconsistent_blocks): + # Extract type from line + type_match = re.search(r'type:\s*(\w+)', line) + type_name = type_match.group(1) if type_match else 'unknown' + + found_at_short = format_bid(found_at_bid) + self_short = format_bid(self_bid) + + print(f"{idx:<5} {found_at_short:<20} {self_short:<20} {type_name:<25}") + + print("\n" + "=" * 140 + "\n") + +def compare_chains(chain1, chain2): + """Compare two chains and report differences.""" + print("\n" + "=" * 100) + print("CHAIN COMPARISON") + print("=" * 100) + + # Extract self_bids from chains + chain1_bids = [b.self_bid for b in chain1 if isinstance(b, MetaBlk)] + chain2_bids = [b.self_bid for b in chain2 if isinstance(b, MetaBlk)] + + # Find common prefix + common_length = 0 + for i in range(min(len(chain1_bids), len(chain2_bids))): + if chain1_bids[i] == chain2_bids[i]: + common_length += 1 + else: + break + + # Show differences + if chain1_bids == chain2_bids: + print("\n ✓ Chains are IDENTICAL!\n") + print(f" Chain length: {len(chain1_bids)}") + else: + print("\n ✗ Chains are DIFFERENT\n") + print(f" Chain 1 length: {len(chain1_bids)}") + print(f" Chain 2 length: {len(chain2_bids)}") + print(f" Common prefix: {common_length} blocks") + + # Show where they diverge + if common_length < min(len(chain1_bids), len(chain2_bids)): + print(f"\n Chains diverge at position {common_length}:") + if common_length < len(chain1): + blk1 = chain1[common_length] + if isinstance(blk1, MetaBlk): + print(f" Chain 1: {format_bid(blk1.self_bid):<20} type={blk1.type_name}") + if common_length < len(chain2): + blk2 = chain2[common_length] + if isinstance(blk2, MetaBlk): + print(f" Chain 2: {format_bid(blk2.self_bid):<20} type={blk2.type_name}") + + # Show blocks only in each chain + set1 = set(chain1_bids) + set2 = set(chain2_bids) + only_in_1 = set1 - set2 + only_in_2 = set2 - set1 + + if only_in_1: + print(f"\n Blocks only in Chain 1 ({len(only_in_1)}):") + for bid in only_in_1: + idx = chain1_bids.index(bid) + blk = chain1[idx] + print(f" Position {idx}: {format_bid(blk.self_bid):<20} type={blk.type_name}") + + if only_in_2: + print(f"\n Blocks only in Chain 2 ({len(only_in_2)}):") + for bid in only_in_2: + idx = chain2_bids.index(bid) + blk = chain2[idx] + print(f" Position {idx}: {format_bid(blk.self_bid):<20} type={blk.type_name}") + + print("\n" + "=" * 100 + "\n") + +def main(): + if len(sys.argv) != 3: + print("Usage: python3 compare_metablk_chains.py ") + print() + print(" chunk_log_file: Log file from chunk traversal (may have orphaned blocks)") + print(" chain_log_file: Log file from chain traversal (complete chain)") + sys.exit(1) + + chunk_log_file = sys.argv[1] + chain_log_file = sys.argv[2] + + print() + print("=" * 100) + print("MetaBlk Chain Reconstruction and Comparison") + print("=" * 100) + print(f"Chunk traversal source: {chunk_log_file}") + print(f"Chain traversal source: {chain_log_file}") + print("=" * 100) + print() + + # Parse chain entries first to get the SSB + print("Parsing chain traversal entries...") + chain_blocks, chain_ssb = parse_chain_entries(chain_log_file) + print(f" Found {len(chain_blocks)} blocks") + print(f" SSB start: {chain_ssb}") + print() + + if not chain_ssb: + print("ERROR: Could not find SSB in chain traversal entries") + sys.exit(1) + + # Parse chunk entries using the SSB from chain + print(f"Parsing chunk traversal entries using SSB from chain: {chain_ssb}...") + chunk_blocks, chunk_ssb, inconsistent_blocks = parse_chunk_entries(chunk_log_file, chain_ssb) + print(f" Found {len(chunk_blocks)} blocks") + print(f" Found {len(inconsistent_blocks)} inconsistent blocks") + print(f" SSB start: {chunk_ssb}") + print() + + if not chunk_ssb: + print("ERROR: Could not find matching SSB in chunk traversal entries") + sys.exit(1) + + # Build chains + print("Building chain from chunk traversal...") + chain1, visited1 = build_chain(chunk_blocks, chunk_ssb) + print(f" Chain length: {len([b for b in chain1 if isinstance(b, MetaBlk)])}") + print() + + print("Building chain from chain traversal...") + chain2, visited2 = build_chain(chain_blocks, chain_ssb) + print(f" Chain length: {len([b for b in chain2 if isinstance(b, MetaBlk)])}") + print() + + # Find orphaned blocks in chunk traversal + orphaned1 = find_orphaned_blocks(chunk_blocks, visited1) + if orphaned1: + print(f"Found {len(orphaned1)} orphaned blocks in chunk traversal (due to deletion)") + print() + + # Print chains + print_chain(chain1, "CHAIN FROM CHUNK TRAVERSAL") + print_chain(chain2, "CHAIN FROM CHAIN TRAVERSAL") + + # Print orphaned blocks + if orphaned1: + print_orphaned(chunk_blocks, orphaned1, f"ORPHANED BLOCKS IN CHUNK TRAVERSAL ({len(orphaned1)} blocks)") + + # Print inconsistent blocks + print_inconsistent_blocks(inconsistent_blocks) + + # Compare chains + compare_chains(chain1, chain2) + + # Summary + print("\n" + "=" * 100) + print("SUMMARY") + print("=" * 100) + chain1_count = len([b for b in chain1 if isinstance(b, MetaBlk)]) + chain2_count = len([b for b in chain2 if isinstance(b, MetaBlk)]) + + print(f"\nChunk traversal:") + print(f" Blocks in chain: {chain1_count}") + print(f" Orphaned blocks: {len(orphaned1)}") + print(f" Inconsistent blocks: {len(inconsistent_blocks)}") + print(f" Total blocks: {chain1_count + len(orphaned1)}") + + print(f"\nChain traversal:") + print(f" Blocks in chain: {chain2_count}") + + print(f"\nComparison:") + print(f" SSB match: {'✓ YES' if chunk_ssb == chain_ssb else '✗ NO'}") + chain1_bids = [b.self_bid for b in chain1 if isinstance(b, MetaBlk)] + chain2_bids = [b.self_bid for b in chain2 if isinstance(b, MetaBlk)] + print(f" Chains identical: {'✓ YES' if chain1_bids == chain2_bids else '✗ NO'}") + print("\n" + "=" * 100 + "\n") + +if __name__ == "__main__": + main()