From 165118855b479b4d4b8a8bcef0634e576e7d355e Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Sun, 14 Dec 2025 18:08:05 +0200 Subject: [PATCH 01/34] mt add vector --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 531 ++++++++++++++++-- .../algorithms/hnsw/hnsw_disk_serializer.h | 3 + src/VecSim/vec_sim_common.h | 2 + .../bm_hnsw_disk_initialize_fp32.h | 144 +++-- tests/benchmark/bm_vecsim_index.h | 10 + .../run_files/bm_hnsw_disk_single_fp32.cpp | 2 +- tests/unit/test_hnsw_disk.cpp | 110 ++++ 7 files changed, 704 insertions(+), 98 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 39dee75d6..4ee2ef9fe 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -54,6 +54,10 @@ // #include // #include #include +#include + +// Forward declaration for AsyncJob +#include "VecSim/vec_sim_tiered_index.h" using std::pair; @@ -119,6 +123,48 @@ struct GraphKey { #pragma pack() +////////////////////////////////////// HNSW Disk Job Structures ////////////////////////////////////// + +/** + * Forward declaration of HNSWDiskIndex for job structures. + */ +template +class HNSWDiskIndex; + +/** + * Definition of a job that inserts a single vector into the HNSW disk graph. + * Each job handles one vector independently and can run in parallel with other insert jobs. + */ +struct HNSWDiskInsertJob : public AsyncJob { + idType vectorId; + size_t elementMaxLevel; + + // Snapshot of entry point state at job creation time + idType entryPointSnapshot; + size_t maxLevelSnapshot; + + // Retry counter for error handling + std::atomic retryCount{0}; + static constexpr int MAX_RETRIES = 3; + + HNSWDiskInsertJob(std::shared_ptr allocator, idType vectorId_, + size_t elementMaxLevel_, idType entryPoint_, size_t maxLevel_, + JobCallback insertCb, VecSimIndex *index_) + : AsyncJob(allocator, HNSW_DISK_INSERT_VECTOR_JOB, insertCb, index_), vectorId(vectorId_), + elementMaxLevel(elementMaxLevel_), entryPointSnapshot(entryPoint_), + maxLevelSnapshot(maxLevel_) {} +}; + +/** + * Definition of a job that flushes staged graph updates to RocksDB. + * This job must execute after all insert jobs complete. + */ +struct HNSWDiskFlushJob : public AsyncJob { + HNSWDiskFlushJob(std::shared_ptr allocator, JobCallback flushCb, + VecSimIndex *index_) + : AsyncJob(allocator, HNSW_DISK_FLUSH_UPDATES_JOB, flushCb, index_) {} +}; + //////////////////////////////////// HNSW index implementation //////////////////////////////////// template @@ -242,6 +288,49 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Cache for raw vectors retrieved from disk (to avoid redundant reads) std::unordered_map rawVectorsCache; + /********************************** Multi-threading Support **********************************/ + + // Job queue parameters (similar to tiered index) + void *jobQueue; + void *jobQueueCtx; + SubmitCB SubmitJobsToQueue; + + // Reader-writer lock for RocksDB operations + // Multiple threads can read concurrently, but writes must be exclusive + mutable std::shared_mutex rocksDbGuard; + + // Lock for protecting staged updates during merge from parallel insert jobs + mutable std::mutex stagedUpdatesGuard; + + // Atomic counter for pending insert jobs (for synchronization with flush job) + std::atomic pendingInsertJobsCounter{0}; + + // Flag indicating if a batch is currently being processed + std::atomic batchInProgress{false}; + + // Reference to the flush job that will be submitted after all insert jobs complete + HNSWDiskFlushJob *pendingFlushJob = nullptr; + + // Double-buffering for pending vectors + // processingBatch contains vectors currently being inserted by worker threads + struct ProcessingBatch { + vecsim_stl::vector vectorIds; + std::unordered_map rawVectors; + size_t count = 0; + + ProcessingBatch(std::shared_ptr allocator) : vectorIds(allocator) {} + + void clear() { + vectorIds.clear(); + rawVectors.clear(); + count = 0; + } + }; + std::unique_ptr processingBatch; + + // Lock for swapping between pending and processing batches + mutable std::mutex batchSwapGuard; + protected: HNSWDiskIndex() = delete; // default constructor is disabled. // default (shallow) copy constructor is disabled. @@ -281,6 +370,20 @@ class HNSWDiskIndex : public VecSimIndexAbstract void flushDeleteBatch(); // Force flush current delete batch void setBatchThreshold(size_t threshold); // Set batch threshold + // Multi-threaded job execution methods + static void executeInsertJobWrapper(AsyncJob *job); + static void executeFlushJobWrapper(AsyncJob *job); + void executeInsertJob(HNSWDiskInsertJob *job); + void executeFlushJob(HNSWDiskFlushJob *job); + + // Job submission helpers + void submitSingleJob(AsyncJob *job); + void submitJobs(vecsim_stl::vector &jobs); + + // Thread-safe staging merge + void mergeLocalStagedUpdates(vecsim_stl::vector &localGraphUpdates, + vecsim_stl::vector &localNeighborUpdates); + // Helper methods void emplaceHeap(vecsim_stl::abstract_priority_queue &heap, DistType dist, idType id) const; @@ -353,6 +456,8 @@ class HNSWDiskIndex : public VecSimIndexAbstract bool getRawVector(idType id, void* output_buffer) const; protected: + // Internal version that assumes caller already holds the lock (or is inside a locked section) + bool getRawVectorInternal(idType id, void* output_buffer) const; idType searchBottomLayerEP(const void *query_data, void *timeoutCtx = nullptr, VecSimQueryReply_Code *rc = nullptr) const; @@ -361,7 +466,8 @@ class HNSWDiskIndex : public VecSimIndexAbstract HNSWDiskIndex(const HNSWParams *params, const AbstractIndexInitParams &abstractInitParams, const IndexComponents &components, rocksdb::DB *db, rocksdb::ColumnFamilyHandle *cf, const std::string &dbPath = "", - size_t random_seed = 100); + size_t random_seed = 100, void *jobQueue = nullptr, void *jobQueueCtx = nullptr, + SubmitCB submitCb = nullptr); virtual ~HNSWDiskIndex(); /*************************** Index API ***************************/ @@ -430,6 +536,13 @@ class HNSWDiskIndex : public VecSimIndexAbstract size_t getDeleteBatchThreshold() const { return deleteBatchThreshold; } size_t getPendingDeleteCount() const { return pendingDeleteIds.size(); } + // Job queue configuration (for multi-threaded processing) + void setJobQueue(void *jobQueue_, void *jobQueueCtx_, SubmitCB submitCb_) { + jobQueue = jobQueue_; + jobQueueCtx = jobQueueCtx_; + SubmitJobsToQueue = submitCb_; + } + // Debug methods to inspect graph structure void debugPrintGraphStructure() const; void debugPrintNodeNeighbors(idType node_id) const; @@ -483,7 +596,8 @@ template HNSWDiskIndex::HNSWDiskIndex( const HNSWParams *params, const AbstractIndexInitParams &abstractInitParams, const IndexComponents &components, rocksdb::DB *db, - rocksdb::ColumnFamilyHandle *cf, const std::string &dbPath, size_t random_seed) + rocksdb::ColumnFamilyHandle *cf, const std::string &dbPath, size_t random_seed, + void *jobQueue_, void *jobQueueCtx_, SubmitCB submitCb_) : VecSimIndexAbstract(abstractInitParams, components), idToMetaData(INITIAL_CAPACITY, this->allocator), labelToIdMap(this->allocator), db(db), cf(cf), dbPath(dbPath), indexDataGuard(), @@ -493,7 +607,8 @@ HNSWDiskIndex::HNSWDiskIndex( pendingDeleteIds(this->allocator), stagedInsertUpdates(this->allocator), stagedDeleteUpdates(this->allocator), stagedRepairUpdates(this->allocator), - stagedInsertNeighborUpdates(this->allocator) { + stagedInsertNeighborUpdates(this->allocator), + jobQueue(jobQueue_), jobQueueCtx(jobQueueCtx_), SubmitJobsToQueue(submitCb_) { M = params->M ? params->M : HNSW_DEFAULT_M; M0 = M * 2; @@ -516,10 +631,19 @@ HNSWDiskIndex::HNSWDiskIndex( throw std::runtime_error("HNSW index parameter M cannot be 1"); mult = 1 / log(1.0 * M); levelGenerator.seed(random_seed); + + // Initialize processing batch for double-buffering + processingBatch = std::make_unique(this->allocator); } template HNSWDiskIndex::~HNSWDiskIndex() { + // Clean up any pending flush job + if (pendingFlushJob) { + delete pendingFlushJob; + pendingFlushJob = nullptr; + } + // Clear any staged updates before destruction stagedInsertUpdates.clear(); stagedInsertMap.clear(); @@ -849,10 +973,13 @@ idType HNSWDiskIndex::mutuallyConnectNewElement( neighbor_ids.push_back(top_candidates_list[i].second); } - // Add to staged graph updates (for insertions) - uint64_t insert_key = makeRepairKey(new_node_id, level); - stagedInsertMap[insert_key] = stagedInsertUpdates.size(); - stagedInsertUpdates.emplace_back(new_node_id, level, neighbor_ids, this->allocator); + // Add to staged graph updates (for insertions) - protected by stagedUpdatesGuard + { + std::lock_guard lock(stagedUpdatesGuard); + uint64_t insert_key = makeRepairKey(new_node_id, level); + stagedInsertMap[insert_key] = stagedInsertUpdates.size(); + stagedInsertUpdates.emplace_back(new_node_id, level, neighbor_ids, this->allocator); + } // Stage updates to existing nodes to include the new node in their neighbor lists for (const auto &neighbor_data : top_candidates_list) { @@ -866,8 +993,12 @@ idType HNSWDiskIndex::mutuallyConnectNewElement( // Read the neighbor's current neighbor count from disk to check capacity GraphKey neighborKey(selected_neighbor, level); std::string existing_neighbors_data; - rocksdb::Status status = - db->Get(rocksdb::ReadOptions(), cf, neighborKey.asSlice(), &existing_neighbors_data); + rocksdb::Status status; + { + std::shared_lock lock(rocksDbGuard); + status = + db->Get(rocksdb::ReadOptions(), cf, neighborKey.asSlice(), &existing_neighbors_data); + } size_t current_neighbor_count = 0; if (status.ok()) { @@ -876,6 +1007,7 @@ idType HNSWDiskIndex::mutuallyConnectNewElement( if (current_neighbor_count < max_M_cur) { // Neighbor has capacity, just add the new node + std::lock_guard lock(stagedUpdatesGuard); stagedInsertNeighborUpdates.emplace_back(selected_neighbor, level, new_node_id); } else { // Neighbor is full, need to re-evaluate connections using revisitNeighborConnections @@ -913,9 +1045,9 @@ void HNSWDiskIndex::flushStagedGraphUpdates( continue; } - // Get raw vector data - - if (!getRawVector(update.node_id, raw_vector_buffer.data())) { + // Get raw vector data (use internal version since we already hold the lock or + // are being called from a context where locking is managed by the caller) + if (!getRawVectorInternal(update.node_id, raw_vector_buffer.data())) { this->log(VecSimCommonStrings::LOG_WARNING_STRING, "WARNING: Skipping graph update for node %u at level %zu - no raw vector data available", update.node_id, update.level); @@ -982,7 +1114,7 @@ void HNSWDiskIndex::flushStagedGraphUpdates( } } - getRawVector(node_id, raw_vector_buffer.data()); + getRawVectorInternal(node_id, raw_vector_buffer.data()); // Serialize with new format and add to batch std::string graph_value = serializeGraphValue(raw_vector_buffer.data(), updated_neighbors); @@ -1011,7 +1143,11 @@ void HNSWDiskIndex::stageRevisitNeighborConnections(idType n // TODO: perhaps cache the neigbhors for stage update GraphKey neighborKey(selected_neighbor, level); std::string graph_value; - rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, neighborKey.asSlice(), &graph_value); + rocksdb::Status status; + { + std::shared_lock lock(rocksDbGuard); + status = db->Get(rocksdb::ReadOptions(), cf, neighborKey.asSlice(), &graph_value); + } if (!status.ok()) { this->log(VecSimCommonStrings::LOG_WARNING_STRING, @@ -1071,10 +1207,12 @@ void HNSWDiskIndex::stageRevisitNeighborConnections(idType n // Stage this update - the neighbor's neighbor list will be completely replaced // We'll need to handle this specially in flushStagedGraphUpdates + // Protected by stagedUpdatesGuard for thread safety + std::lock_guard lock(stagedUpdatesGuard); uint64_t insert_key = makeRepairKey(selected_neighbor, level); stagedInsertMap[insert_key] = stagedInsertUpdates.size(); stagedInsertUpdates.emplace_back(selected_neighbor, level, selected_neighbor_ids, - this->allocator); + this->allocator); // Also stage the bidirectional connection from new node to selected neighbor stagedInsertNeighborUpdates.emplace_back(new_node_id, level, selected_neighbor); @@ -1082,6 +1220,7 @@ void HNSWDiskIndex::stageRevisitNeighborConnections(idType n } else { // The new node was not selected, so we only need to stage the unidirectional connection // from new node to selected neighbor + std::lock_guard lock(stagedUpdatesGuard); stagedInsertNeighborUpdates.emplace_back(new_node_id, level, selected_neighbor); } } @@ -1206,13 +1345,17 @@ bool HNSWDiskIndex::getRawVector(idType id, void* output_buf const char* data_ptr = it->second.data(); std::memcpy(output_buffer, data_ptr, this->inputBlobSize); return true; - + } - // If not in RAM or cache, retrieve from disk + // If not in RAM or cache, retrieve from disk with shared lock for thread safety GraphKey graphKey(id, 0); std::string level0_graph_value; - rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &level0_graph_value); + rocksdb::Status status; + { + std::shared_lock lock(rocksDbGuard); + status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &level0_graph_value); + } if (status.ok()) { // Extract vector data @@ -1235,6 +1378,58 @@ bool HNSWDiskIndex::getRawVector(idType id, void* output_buf } +// Internal version that assumes caller already holds the lock +template +bool HNSWDiskIndex::getRawVectorInternal(idType id, void* output_buffer) const { + + if (id >= curElementCount) { + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "WARNING: getRawVectorInternal called with invalid id %u (current count: %zu)", + id, curElementCount); + return false; + } + + // First check RAM (for vectors not yet flushed) + auto it = rawVectorsInRAM.find(id); + if (it != rawVectorsInRAM.end()) { + const char* data_ptr = it->second.data(); + std::memcpy(output_buffer, data_ptr, this->inputBlobSize); + return true; + } + + // Also check processingBatch (for vectors being processed by worker threads) + if (processingBatch) { + auto pit = processingBatch->rawVectors.find(id); + if (pit != processingBatch->rawVectors.end()) { + std::memcpy(output_buffer, pit->second.data(), this->inputBlobSize); + return true; + } + } + + // If not in RAM, retrieve from disk (NO LOCK - caller is expected to hold lock) + GraphKey graphKey(id, 0); + std::string level0_graph_value; + rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &level0_graph_value); + + if (status.ok()) { + // Extract vector data + const void* vector_data = getVectorFromGraphValue(level0_graph_value); + if (vector_data != nullptr) { + std::memcpy(output_buffer, vector_data, this->inputBlobSize); + return true; + } + } else if (status.IsNotFound()) { + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "WARNING: Raw vector not found in RAM or on disk for id %u (isMarkedDeleted: %d)", + id, isMarkedDeleted(id)); + } else { + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "WARNING: Failed to retrieve raw vector for id %u: %s", id, + status.ToString().c_str()); + } + return false; +} + template vecsim_stl::updatable_max_heap HNSWDiskIndex::searchLayer(idType ep_id, const void *data_point_raw, const void *data_point, size_t layer, @@ -1341,10 +1536,14 @@ void HNSWDiskIndex::greedySearchLevel(const void *data_point do { changed = false; - // Read neighbors from RocksDB for the current node at this level + // Read neighbors from RocksDB for the current node at this level with shared lock GraphKey graphKey(bestCand, level); std::string graph_value; - rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + rocksdb::Status status; + { + std::shared_lock lock(rocksDbGuard); + status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + } if (!status.ok()) { // No neighbors found for this node at this level, stop searching @@ -1611,11 +1810,15 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level return; } - // If not found in staged updates, check disk + // If not found in staged updates, check disk with shared lock for thread safety GraphKey graphKey(nodeId, level); std::string graph_value; - rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + rocksdb::Status status; + { + std::shared_lock lock(rocksDbGuard); + status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + } if (status.ok()) { deserializeGraphValue(graph_value, result); @@ -1623,6 +1826,8 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level if (filterDeletedNodes(result)) { // Lazy repair: if we filtered any deleted nodes, stage for cleanup // Use hash map for O(1) duplicate detection + // Note: stagedRepairMap and stagedRepairUpdates are protected by stagedUpdatesGuard + std::lock_guard repairLock(stagedUpdatesGuard); uint64_t repair_key = makeRepairKey(nodeId, level); if (stagedRepairMap.find(repair_key) == stagedRepairMap.end()) { stagedRepairMap[repair_key] = stagedRepairUpdates.size(); @@ -1653,11 +1858,15 @@ void HNSWDiskIndex::getNeighborsAndVector(idType nodeId, siz if (!result.empty() && it != rawVectorsInRAM.end()) { return; } - // If not found in staged updates, check disk + // If not found in staged updates, check disk with shared lock for thread safety GraphKey graphKey(nodeId, level); std::string graph_value; - rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + rocksdb::Status status; + { + std::shared_lock lock(rocksDbGuard); + status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + } if (status.ok()) { deserializeGraphValue(graph_value, result); @@ -1711,54 +1920,278 @@ void HNSWDiskIndex::searchPendingVectors( template void HNSWDiskIndex::processBatch() { + // Lock to swap batches atomically + std::lock_guard batchLock(batchSwapGuard); + if (pendingVectorCount == 0) { return; } - // Clear any previous staged updates (for insertions) + // Check if job queue is available for multi-threaded processing + if (SubmitJobsToQueue == nullptr) { + // Fall back to single-threaded processing + // Clear any previous staged updates (for insertions) + stagedInsertUpdates.clear(); + stagedInsertMap.clear(); + stagedInsertNeighborUpdates.clear(); + + // Process each pending vector ID (vectors are already stored in memory) + for (size_t i = 0; i < pendingVectorCount; i++) { + idType vectorId = pendingVectorIds[i]; + if (isMarkedDeleted(vectorId)) { + continue; + } + + const void *vector_data = this->vectors->getElement(vectorId); + const void *raw_vector_data = rawVectorsInRAM.find(vectorId)->second.data(); + DiskElementMetaData &metadata = idToMetaData[vectorId]; + size_t elementMaxLevel = metadata.topLevel; + + if (entrypointNode != INVALID_ID) { + insertElementToGraph(vectorId, elementMaxLevel, entrypointNode, maxLevel, + raw_vector_data, vector_data); + } else { + entrypointNode = vectorId; + maxLevel = elementMaxLevel; + } + } + + flushStagedGraphUpdates(stagedInsertUpdates, stagedInsertNeighborUpdates); + stagedInsertMap.clear(); + pendingVectorIds.clear(); + rawVectorsInRAM.clear(); + pendingMetadata.clear(); + pendingVectorCount = 0; + return; + } + + // Check if previous batch is still processing + if (batchInProgress.load()) { + // Previous batch still running, return - caller can retry later + return; + } + + batchInProgress.store(true); + + // Move pending data to processing batch (double-buffering) + processingBatch->vectorIds = std::move(pendingVectorIds); + processingBatch->rawVectors = std::move(rawVectorsInRAM); + processingBatch->count = pendingVectorCount; + + // Clear pending structures for new vectors + pendingVectorIds = vecsim_stl::vector(this->allocator); + rawVectorsInRAM.clear(); + pendingVectorCount = 0; + + // Clear any previous staged updates stagedInsertUpdates.clear(); stagedInsertMap.clear(); stagedInsertNeighborUpdates.clear(); - // Process each pending vector ID (vectors are already stored in memory) - for (size_t i = 0; i < pendingVectorCount; i++) { - idType vectorId = pendingVectorIds[i]; - if (isMarkedDeleted(vectorId)) { - // Skip deleted vectors + // Snapshot current entry point state + idType currentEntryPoint = entrypointNode; + size_t currentMaxLevel = maxLevel; + + // First pass: Update entry point for any vectors with higher levels + for (size_t i = 0; i < processingBatch->count; i++) { + idType vectorId = processingBatch->vectorIds[i]; + if (isMarkedDeleted(vectorId)) + continue; + + DiskElementMetaData &metadata = idToMetaData[vectorId]; + if (currentEntryPoint == INVALID_ID || metadata.topLevel > currentMaxLevel) { + currentEntryPoint = vectorId; + currentMaxLevel = metadata.topLevel; + } + } + + // Update global entry point + entrypointNode = currentEntryPoint; + maxLevel = currentMaxLevel; + + // Count valid vectors and create jobs + size_t validVectorCount = 0; + vecsim_stl::vector jobs(this->allocator); + jobs.reserve(processingBatch->count); + + for (size_t i = 0; i < processingBatch->count; i++) { + idType vectorId = processingBatch->vectorIds[i]; + if (isMarkedDeleted(vectorId)) + continue; + + // Skip if this is the entry point (already set, no connections needed) + if (vectorId == currentEntryPoint && validVectorCount == 0) { + validVectorCount++; continue; } - // Get the vector data from memory - const void* vector_data = this->vectors->getElement(vectorId); - const void* raw_vector_data = rawVectorsInRAM.find(vectorId)->second.data(); - // Get metadata for this vector + validVectorCount++; DiskElementMetaData &metadata = idToMetaData[vectorId]; - size_t elementMaxLevel = metadata.topLevel; - // Insert into graph if not the first element - if (entrypointNode != INVALID_ID) { - insertElementToGraph(vectorId, elementMaxLevel, entrypointNode, maxLevel, raw_vector_data, vector_data); + HNSWDiskInsertJob *job = + new (this->allocator) HNSWDiskInsertJob(this->allocator, vectorId, metadata.topLevel, + currentEntryPoint, currentMaxLevel, + executeInsertJobWrapper, this); + jobs.push_back(job); + } + + if (jobs.empty()) { + // No jobs to submit (e.g., only entry point was set) + batchInProgress.store(false); + processingBatch->clear(); + return; + } + + // Set up counter for synchronization + pendingInsertJobsCounter.store(jobs.size()); + + // Create flush job (will be submitted by last insert job) + pendingFlushJob = + new (this->allocator) HNSWDiskFlushJob(this->allocator, executeFlushJobWrapper, this); + + // Submit all insert jobs + submitJobs(jobs); +} + +template +void HNSWDiskIndex::flushBatch() { + processBatch(); +} + +/********************************** Multi-threaded Job Execution **********************************/ + +template +void HNSWDiskIndex::submitSingleJob(AsyncJob *job) { + this->SubmitJobsToQueue(this->jobQueue, this->jobQueueCtx, &job, &job->Execute, 1); +} + +template +void HNSWDiskIndex::submitJobs(vecsim_stl::vector &jobs) { + vecsim_stl::vector callbacks(jobs.size(), this->allocator); + for (size_t i = 0; i < jobs.size(); i++) { + callbacks[i] = jobs[i]->Execute; + } + this->SubmitJobsToQueue(this->jobQueue, this->jobQueueCtx, jobs.data(), callbacks.data(), + jobs.size()); +} + +template +void HNSWDiskIndex::executeInsertJobWrapper(AsyncJob *job) { + auto *insertJob = static_cast(job); + auto *index = static_cast *>(job->index); + index->executeInsertJob(insertJob); +} + +template +void HNSWDiskIndex::executeFlushJobWrapper(AsyncJob *job) { + auto *flushJob = static_cast(job); + auto *index = static_cast *>(job->index); + index->executeFlushJob(flushJob); +} + +template +void HNSWDiskIndex::executeInsertJob(HNSWDiskInsertJob *job) { + if (!job->isValid) { + // Job was invalidated, decrement counter and check for flush + if (pendingInsertJobsCounter.fetch_sub(1) == 1) { + submitSingleJob(pendingFlushJob); + } + delete job; + return; + } + + // Thread-local staging for this job + vecsim_stl::vector localGraphUpdates(this->allocator); + vecsim_stl::vector localNeighborUpdates(this->allocator); + + try { + // Get vector data from processing batch (read-only, thread-safe) + const void *vector_data = this->vectors->getElement(job->vectorId); + auto it = processingBatch->rawVectors.find(job->vectorId); + if (it == processingBatch->rawVectors.end()) { + throw std::runtime_error("Vector not found in processing batch"); + } + const void *raw_vector_data = it->second.data(); + + // Skip if this is the entry point (no connections to make) + if (job->vectorId != job->entryPointSnapshot) { + // Insert into graph - reads from RocksDB are protected by shared lock + // The insertElementToGraph method will stage updates + insertElementToGraph(job->vectorId, job->elementMaxLevel, job->entryPointSnapshot, + job->maxLevelSnapshot, raw_vector_data, vector_data); + } + + } catch (const std::exception &e) { + // Handle error with retry mechanism + if (job->retryCount.fetch_add(1) < HNSWDiskInsertJob::MAX_RETRIES) { + // Resubmit job for retry + submitSingleJob(job); + return; // Don't decrement counter yet } else { - // First element becomes the entry point - entrypointNode = vectorId; - maxLevel = elementMaxLevel; + // Max retries exceeded, mark as failed + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "Insert job failed after %d retries: %s", HNSWDiskInsertJob::MAX_RETRIES, + e.what()); + job->isValid = false; } } - // Now flush all staged graph updates to disk in a single batch operation + // Decrement counter and submit flush job if this is the last insert job + if (pendingInsertJobsCounter.fetch_sub(1) == 1) { + submitSingleJob(pendingFlushJob); + } + + delete job; +} + +template +void HNSWDiskIndex::executeFlushJob(HNSWDiskFlushJob *job) { + // Acquire exclusive lock for RocksDB writes + std::unique_lock lock(rocksDbGuard); + + // Flush all staged updates to RocksDB flushStagedGraphUpdates(stagedInsertUpdates, stagedInsertNeighborUpdates); + + // Clear staging stagedInsertMap.clear(); - // Clear the pending vector IDs - pendingVectorIds.clear(); - rawVectorsInRAM.clear(); - pendingMetadata.clear(); - pendingVectorCount = 0; + lock.unlock(); + + // Clear processing batch (raw vectors now persisted to disk) + processingBatch->clear(); + + // Mark batch as complete + batchInProgress.store(false); + + // Clean up flush job + pendingFlushJob = nullptr; + delete job; } template -void HNSWDiskIndex::flushBatch() { - processBatch(); +void HNSWDiskIndex::mergeLocalStagedUpdates( + vecsim_stl::vector &localGraphUpdates, + vecsim_stl::vector &localNeighborUpdates) { + std::lock_guard lock(stagedUpdatesGuard); + + // Merge graph updates + for (auto &update : localGraphUpdates) { + uint64_t key = makeRepairKey(update.node_id, update.level); + auto it = stagedInsertMap.find(key); + if (it != stagedInsertMap.end()) { + // Update existing entry + stagedInsertUpdates[it->second] = std::move(update); + } else { + // Add new entry + stagedInsertMap[key] = stagedInsertUpdates.size(); + stagedInsertUpdates.push_back(std::move(update)); + } + } + + // Merge neighbor updates + for (auto &update : localNeighborUpdates) { + stagedInsertNeighborUpdates.push_back(std::move(update)); + } } template diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h index c1cb5b540..9bf696fe2 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h @@ -74,6 +74,9 @@ HNSWDiskIndex::HNSWDiskIndex( // Add a constant offset to avoid seed=0 for empty indexes this->levelGenerator.seed(200 + this->curElementCount); + // Initialize processing batch for double-buffering (needed for async operations) + this->processingBatch = std::make_unique(this->allocator); + // Restore graph and vectors from file this->restoreGraph(input, version); this->restoreVectors(input, version); diff --git a/src/VecSim/vec_sim_common.h b/src/VecSim/vec_sim_common.h index 97cb6ae6f..30fed7652 100644 --- a/src/VecSim/vec_sim_common.h +++ b/src/VecSim/vec_sim_common.h @@ -256,6 +256,8 @@ typedef enum { HNSW_SEARCH_JOB, HNSW_SWAP_JOB, SVS_BATCH_UPDATE_JOB, + HNSW_DISK_INSERT_VECTOR_JOB, + HNSW_DISK_FLUSH_UPDATES_JOB, INVALID_JOB // to indicate that finding a JobType >= INVALID_JOB is an error } JobType; diff --git a/tests/benchmark/bm_initialization/bm_hnsw_disk_initialize_fp32.h b/tests/benchmark/bm_initialization/bm_hnsw_disk_initialize_fp32.h index a268e0fa9..a98a0cdfd 100644 --- a/tests/benchmark/bm_initialization/bm_hnsw_disk_initialize_fp32.h +++ b/tests/benchmark/bm_initialization/bm_hnsw_disk_initialize_fp32.h @@ -29,39 +29,39 @@ BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(Disk, HNSWDisk), fp32_ (benchmark::State &st) { Disk(st, INDEX_HNSW_DISK); } BENCHMARK_REGISTER_F(BM_VecSimCommon, BM_FUNC_NAME(Disk, HNSWDisk))->Iterations(1); -// AddLabel benchmarks -BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_ADD_LABEL, fp32_index_t) -(benchmark::State &st) { AddLabel(st); } -REGISTER_AddLabel(BM_ADD_LABEL, INDEX_HNSW_DISK); - -// TopK benchmark (single-threaded) -BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK, HNSWDisk), fp32_index_t) -(benchmark::State &st) { TopK_HNSW_DISK(st); } -REGISTER_TopK_HNSW_DISK(BM_VecSimCommon, BM_FUNC_NAME(TopK, HNSWDisk)); - -// TopK benchmark (parallel) -BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopKParallel, HNSWDisk), fp32_index_t) -(benchmark::State &st) { TopK_HNSW_DISK_Parallel(st); } -REGISTER_TopK_HNSW_DISK_PARALLEL(BM_VecSimCommon, BM_FUNC_NAME(TopKParallel, HNSWDisk)); - -// TopK benchmark with marked deleted vectors -BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK_MarkDeleted, HNSWDisk), fp32_index_t) -(benchmark::State &st) { TopK_HNSW_DISK_MarkDeleted(st); } -REGISTER_TopK_HNSW_DISK_MarkDeleted(BM_VecSimCommon, BM_FUNC_NAME(TopK_MarkDeleted, HNSWDisk)); - -// TopK benchmark after deleting vectors (with graph repair) -BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel, HNSWDisk), fp32_index_t) -(benchmark::State &st) { TopK_HNSW_DISK_DeleteLabel(st); } -REGISTER_TopK_HNSW_DISK_DeleteLabel(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel, HNSWDisk)); -// AddLabel benchmarks -BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_FLUSH_BATCH_DISK, fp32_index_t) -(benchmark::State &st) { FlushBatchDisk(st); } -REGISTER_FlushBatchDisk(BM_FLUSH_BATCH_DISK); -// -// TopK benchmark after deleting vectors (with graph repair), protecting GT vectors for stable recall -BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel_ProtectGT, HNSWDisk), fp32_index_t) -(benchmark::State &st) { TopK_HNSW_DISK_DeleteLabel_ProtectGT(st); } -REGISTER_TopK_HNSW_DISK_DeleteLabel_ProtectGT(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel_ProtectGT, HNSWDisk)); +// // AddLabel benchmarks +// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_ADD_LABEL, fp32_index_t) +// (benchmark::State &st) { AddLabel(st); } +// REGISTER_AddLabel(BM_ADD_LABEL, INDEX_HNSW_DISK); + +// // TopK benchmark (single-threaded) +// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK, HNSWDisk), fp32_index_t) +// (benchmark::State &st) { TopK_HNSW_DISK(st); } +// REGISTER_TopK_HNSW_DISK(BM_VecSimCommon, BM_FUNC_NAME(TopK, HNSWDisk)); + +// // TopK benchmark (parallel) +// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopKParallel, HNSWDisk), fp32_index_t) +// (benchmark::State &st) { TopK_HNSW_DISK_Parallel(st); } +// REGISTER_TopK_HNSW_DISK_PARALLEL(BM_VecSimCommon, BM_FUNC_NAME(TopKParallel, HNSWDisk)); + +// // TopK benchmark with marked deleted vectors +// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK_MarkDeleted, HNSWDisk), fp32_index_t) +// (benchmark::State &st) { TopK_HNSW_DISK_MarkDeleted(st); } +// REGISTER_TopK_HNSW_DISK_MarkDeleted(BM_VecSimCommon, BM_FUNC_NAME(TopK_MarkDeleted, HNSWDisk)); + +// // TopK benchmark after deleting vectors (with graph repair) +// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel, HNSWDisk), fp32_index_t) +// (benchmark::State &st) { TopK_HNSW_DISK_DeleteLabel(st); } +// REGISTER_TopK_HNSW_DISK_DeleteLabel(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel, HNSWDisk)); +// // AddLabel benchmarks +// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_FLUSH_BATCH_DISK, fp32_index_t) +// (benchmark::State &st) { FlushBatchDisk(st); } +// REGISTER_FlushBatchDisk(BM_FLUSH_BATCH_DISK); +// // +// // TopK benchmark after deleting vectors (with graph repair), protecting GT vectors for stable recall +// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel_ProtectGT, HNSWDisk), fp32_index_t) +// (benchmark::State &st) { TopK_HNSW_DISK_DeleteLabel_ProtectGT(st); } +// REGISTER_TopK_HNSW_DISK_DeleteLabel_ProtectGT(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel_ProtectGT, HNSWDisk)); // Range benchmarks // BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_FUNC_NAME(Range, BF), fp32_index_t) @@ -73,18 +73,66 @@ REGISTER_TopK_HNSW_DISK_DeleteLabel_ProtectGT(BM_VecSimCommon, BM_FUNC_NAME(TopK // (benchmark::State &st) { Range_HNSW(st); } // REGISTER_Range_HNSW(BM_FUNC_NAME(Range, HNSW), fp32_index_t); -// Special disk-based HNSW benchmarks for batch processing -// RE-ENABLED: Async AddLabel and DeleteLabel benchmarks for HNSW disk index now work with populated -// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_ADD_LABEL_ASYNC, fp32_index_t) -// (benchmark::State &st) { AddLabel_AsyncIngest(st); } -// BENCHMARK_REGISTER_F(BM_VecSimBasics, BM_ADD_LABEL_ASYNC) -// ->UNIT_AND_ITERATIONS->Arg(INDEX_HNSW_DISK) -// ->ArgName("INDEX_HNSW_DISK"); - -// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_DELETE_LABEL_ASYNC, fp32_index_t) -// (benchmark::State &st) { DeleteLabel_AsyncRepair(st); } -// BENCHMARK_REGISTER_F(BM_VecSimBasics, BM_DELETE_LABEL_ASYNC) -// ->UNIT_AND_ITERATIONS->Arg(1) -// ->Arg(100) -// ->Arg(BM_VecSimGeneral::block_size) -// ->ArgName("SwapJobsThreshold"); +// Special disk-based HNSW benchmarks for batch processing with multi-threaded async ingest +// Args: {INDEX_HNSW_DISK, thread_count} +// This benchmark reloads the disk index for each run since async operations modify the index +BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_ADD_LABEL_ASYNC_DISK, fp32_index_t) +(benchmark::State &st) { + // Reload the disk index fresh for each benchmark run + std::string folder_path = AttachRootPath(hnsw_index_file); + + // Clean up existing thread pool and index + if (BM_VecSimGeneral::mock_thread_pool) { + BM_VecSimGeneral::mock_thread_pool->thread_pool_join(); + delete BM_VecSimGeneral::mock_thread_pool; + BM_VecSimGeneral::mock_thread_pool = nullptr; + } + indices[INDEX_HNSW_DISK] = IndexPtr(nullptr); + + // Reload the index from the checkpoint + indices[INDEX_HNSW_DISK] = IndexPtr(HNSWDiskFactory::NewIndex(folder_path)); + + // Create new mock thread pool + BM_VecSimGeneral::mock_thread_pool = new tieredIndexMock(); + auto &mock_thread_pool = *BM_VecSimGeneral::mock_thread_pool; + mock_thread_pool.ctx->index_strong_ref = indices[INDEX_HNSW_DISK].get_shared(); + + // Set up job queue for async operations on the disk index + auto *disk_index = dynamic_cast *>(indices[INDEX_HNSW_DISK].get()); + if (disk_index) { + disk_index->setJobQueue(&mock_thread_pool.jobQ, mock_thread_pool.ctx, + tieredIndexMock::submit_callback); + } + + // Configure thread pool size from benchmark argument and start threads + size_t thread_count = st.range(1); + mock_thread_pool.init_threads(); + mock_thread_pool.reconfigure_threads(thread_count); + + // Get initial state + auto *index = indices[INDEX_HNSW_DISK].get(); + size_t initial_index_size = VecSimIndex_IndexSize(index); + + // Measure the AddLabel_AsyncIngest benchmark + auto start_time = std::chrono::high_resolution_clock::now(); + AddLabel_AsyncIngest(st); + auto end_time = std::chrono::high_resolution_clock::now(); + + // Calculate stats + size_t final_index_size = VecSimIndex_IndexSize(index); + size_t vectors_added = final_index_size - initial_index_size; + double total_time_ns = std::chrono::duration(end_time - start_time).count(); + double avg_time_per_label_ns = vectors_added > 0 ? total_time_ns / vectors_added : 0; + + // Add custom counters + st.counters["vectors_added"] = vectors_added; + st.counters["total_time_ns"] = total_time_ns; + st.counters["avg_ns_per_label"] = avg_time_per_label_ns; +} +BENCHMARK_REGISTER_F(BM_VecSimBasics, BM_ADD_LABEL_ASYNC_DISK) + ->Unit(benchmark::kNanosecond) + ->Iterations(BM_VecSimGeneral::block_size) + ->Args({INDEX_HNSW_DISK, 1}) + ->Args({INDEX_HNSW_DISK, 4}) + ->Args({INDEX_HNSW_DISK, 8}) + ->ArgNames({"IndexType", "Threads"}); diff --git a/tests/benchmark/bm_vecsim_index.h b/tests/benchmark/bm_vecsim_index.h index 829e10de5..753fa8d80 100644 --- a/tests/benchmark/bm_vecsim_index.h +++ b/tests/benchmark/bm_vecsim_index.h @@ -192,6 +192,16 @@ void BM_VecSimIndex::Initialize() { mock_thread_pool.ctx->index_strong_ref = indices[INDEX_HNSW_DISK].get_shared(); // Threads will be started on-demand by the benchmark via reconfigure_threads(). } + + // Set up job queue for async operations on the disk index + auto *disk_index = dynamic_cast *>(indices[INDEX_HNSW_DISK].get()); + if (disk_index && BM_VecSimGeneral::mock_thread_pool) { + auto &mock_thread_pool = *BM_VecSimGeneral::mock_thread_pool; + disk_index->setJobQueue(&mock_thread_pool.jobQ, mock_thread_pool.ctx, + tieredIndexMock::submit_callback); + // Initialize threads for async processing + mock_thread_pool.init_threads(); + } } if (enabled_index_types & IndexTypeFlags::INDEX_MASK_BF) { diff --git a/tests/benchmark/run_files/bm_hnsw_disk_single_fp32.cpp b/tests/benchmark/run_files/bm_hnsw_disk_single_fp32.cpp index 28fd8fd6f..c9855e993 100644 --- a/tests/benchmark/run_files/bm_hnsw_disk_single_fp32.cpp +++ b/tests/benchmark/run_files/bm_hnsw_disk_single_fp32.cpp @@ -22,7 +22,7 @@ size_t BM_VecSimGeneral::EF_C = 256; // Dataset file paths - using deep-1M dataset // For HNSW disk, hnsw_index_file points to the folder containing index.hnsw_disk_v1 and rocksdb/ const char *BM_VecSimGeneral::hnsw_index_file = - "tests/benchmark/data/deep-1M-L2-dim96-M32-efc200-disk-vectors"; + "tests/benchmark/data/deep-1M-L2-dim96-M32-efc200-disk-vectors.zip"; const char *BM_VecSimGeneral::test_queries_file = "tests/benchmark/data/deep.query.public.10K.fbin"; const char *BM_VecSimGeneral::ground_truth_file = "tests/benchmark/data/deep.groundtruth.1M.10K.ibin"; // defined only for this benchmark diff --git a/tests/unit/test_hnsw_disk.cpp b/tests/unit/test_hnsw_disk.cpp index 300ffb065..86fb79b4e 100644 --- a/tests/unit/test_hnsw_disk.cpp +++ b/tests/unit/test_hnsw_disk.cpp @@ -15,6 +15,7 @@ #include "VecSim/memory/vecsim_malloc.h" #include "unit_test_utils.h" +#include "mock_thread_pool.h" using namespace std; @@ -1761,3 +1762,112 @@ TEST_F(HNSWDiskIndexTest, GraphRepairWithHeuristic) { ASSERT_GT(successful_queries, (n - deleted_labels.size()) / 2) << "Too many queries failed - graph may be disconnected"; } + +TEST_F(HNSWDiskIndexTest, MultiThreadedInsertTest) { + // Test multi-threaded insertion using mock thread pool (similar to tiered index tests) + const size_t dim = 64; + const size_t n = 100; + + // Create HNSW parameters + HNSWParams params; + params.dim = dim; + params.type = VecSimType_FLOAT32; + params.metric = VecSimMetric_L2; + params.multi = false; + params.M = 16; + params.efConstruction = 100; + params.efRuntime = 50; + params.epsilon = 0.01; + + // Create abstract init parameters + AbstractIndexInitParams abstractInitParams; + abstractInitParams.dim = dim; + abstractInitParams.vecType = params.type; + abstractInitParams.dataSize = dim * sizeof(int8_t); + abstractInitParams.blockSize = 1; + abstractInitParams.multi = false; + abstractInitParams.allocator = VecSimAllocator::newVecsimAllocator(); + + // Create index components + IndexComponents components = CreateQuantizedIndexComponents( + abstractInitParams.allocator, VecSimMetric_L2, dim, false); + + // Create mock thread pool (similar to tiered index tests) + auto mock_thread_pool = tieredIndexMock(); + mock_thread_pool.thread_pool_size = 4; + + // Create HNSWDiskIndex with job queue parameters + rocksdb::ColumnFamilyHandle *default_cf = db->DefaultColumnFamily(); + auto *index = new (abstractInitParams.allocator) HNSWDiskIndex( + ¶ms, abstractInitParams, components, db.get(), default_cf, "", 100, + &mock_thread_pool.jobQ, mock_thread_pool.ctx, tieredIndexMock::submit_callback); + + // Set the index in the mock thread pool context (required for job execution) + // This is similar to how TieredFactory::NewIndex sets up the index + mock_thread_pool.ctx->index_strong_ref.reset(index); + + // Set batch threshold high enough so we control when batch is processed + index->setBatchThreshold(n + 10); + + // Create test vectors and add them + std::mt19937 rng(42); + std::vector> vectors; + for (size_t i = 0; i < n; i++) { + vectors.push_back(createRandomVector(dim, rng)); + normalizeVector(vectors.back()); + } + + // Add all vectors (no batch processing triggered due to high threshold) + for (size_t i = 0; i < n; i++) { + int result = index->addVector(vectors[i].data(), i); + EXPECT_EQ(result, 1); + } + + // Verify vectors are pending + EXPECT_EQ(index->indexSize(), n); + + // Launch the BG threads loop + mock_thread_pool.init_threads(); + + // Now trigger batch processing + index->flushBatch(); + + // Wait for all jobs to complete + mock_thread_pool.thread_pool_join(); + + // Verify all vectors are in the index + EXPECT_EQ(index->indexSize(), n); + EXPECT_EQ(index->indexLabelCount(), n); + EXPECT_EQ(mock_thread_pool.jobQ.size(), 0); + + // Verify search works correctly + VecSimQueryParams queryParams; + queryParams.hnswRuntimeParams.efRuntime = 50; + + size_t successful_queries = 0; + for (size_t i = 0; i < n; i++) { + auto results = index->topKQuery(vectors[i].data(), 5, &queryParams); + ASSERT_TRUE(results != nullptr); + ASSERT_EQ(results->code, VecSim_OK); + + if (results->results.size() > 0) { + successful_queries++; + // The query vector should be in the results (exact match) + bool found_self = false; + for (const auto &result : results->results) { + if (result.id == i) { + found_self = true; + break; + } + } + EXPECT_TRUE(found_self) << "Vector " << i << " not found in its own query results"; + } + + delete results; + } + + // Most queries should succeed + EXPECT_GT(successful_queries, n * 0.9) << "Too many queries failed"; + + // Note: The mock_thread_pool destructor will clean up the index via index_strong_ref +} From 4b4807fe75c43bd6f2d97fedc07d7f109ef35fbc Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Sun, 14 Dec 2025 19:00:19 +0200 Subject: [PATCH 02/34] improve serialzor --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 43 +++++--- .../bm_hnsw_disk_initialize_fp32.h | 6 +- tests/benchmark/data/scripts/CMakeLists.txt | 10 +- .../data/scripts/hnsw_disk_serializer.cpp | 102 +++++++++++++++++- 4 files changed, 140 insertions(+), 21 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 4ee2ef9fe..feec6d30d 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -331,6 +331,10 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Lock for swapping between pending and processing batches mutable std::mutex batchSwapGuard; + // Lock for protecting vectors container during concurrent access + // Needed because addElement can resize the container, invalidating pointers + mutable std::shared_mutex vectorsGuard; + protected: HNSWDiskIndex() = delete; // default constructor is disabled. // default (shallow) copy constructor is disabled. @@ -363,6 +367,7 @@ class HNSWDiskIndex : public VecSimIndexAbstract vecsim_stl::updatable_max_heap &top_candidates, size_t level); // Batch processing methods + void singleThreadProcessBatch(); void processBatch(); void flushBatch(); // Force flush current batch @@ -867,9 +872,12 @@ int HNSWDiskIndex::addVector( // Preprocess the vector ProcessedBlobs processedBlobs = this->preprocess(vector); - // Store the processed vector in memory - size_t containerId = this->vectors->size(); - this->vectors->addElement(processedBlobs.getStorageBlob(), containerId); + // Store the processed vector in memory (protected by vectorsGuard) + { + std::unique_lock lock(vectorsGuard); + size_t containerId = this->vectors->size(); + this->vectors->addElement(processedBlobs.getStorageBlob(), containerId); + } // Create new element ID and metadata size_t elementMaxLevel = getRandomLevel(mult); @@ -1321,6 +1329,8 @@ template const void *HNSWDiskIndex::getDataByInternalId(idType id) const { assert(id < curElementCount); + // Acquire shared lock to prevent concurrent resize of vectors container + std::shared_lock lock(vectorsGuard); const void* result = this->vectors->getElement(id); if (result != nullptr) { return result; @@ -1919,17 +1929,7 @@ void HNSWDiskIndex::searchPendingVectors( /********************************** Batch Processing Methods **********************************/ template -void HNSWDiskIndex::processBatch() { - // Lock to swap batches atomically - std::lock_guard batchLock(batchSwapGuard); - - if (pendingVectorCount == 0) { - return; - } - - // Check if job queue is available for multi-threaded processing - if (SubmitJobsToQueue == nullptr) { - // Fall back to single-threaded processing +void HNSWDiskIndex:: singleThreadProcessBatch(){ // Clear any previous staged updates (for insertions) stagedInsertUpdates.clear(); stagedInsertMap.clear(); @@ -1962,6 +1962,21 @@ void HNSWDiskIndex::processBatch() { rawVectorsInRAM.clear(); pendingMetadata.clear(); pendingVectorCount = 0; +} + +template +void HNSWDiskIndex::processBatch() { + // Lock to swap batches atomically + std::lock_guard batchLock(batchSwapGuard); + + if (pendingVectorCount == 0) { + return; + } + + // Check if job queue is available for multi-threaded processing + if (SubmitJobsToQueue == nullptr) { + // Fall back to single-threaded processing + singleThreadProcessBatch(); return; } diff --git a/tests/benchmark/bm_initialization/bm_hnsw_disk_initialize_fp32.h b/tests/benchmark/bm_initialization/bm_hnsw_disk_initialize_fp32.h index a98a0cdfd..32461b41e 100644 --- a/tests/benchmark/bm_initialization/bm_hnsw_disk_initialize_fp32.h +++ b/tests/benchmark/bm_initialization/bm_hnsw_disk_initialize_fp32.h @@ -106,8 +106,8 @@ BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_ADD_LABEL_ASYNC_DISK, fp32_index // Configure thread pool size from benchmark argument and start threads size_t thread_count = st.range(1); + mock_thread_pool.thread_pool_size = thread_count; mock_thread_pool.init_threads(); - mock_thread_pool.reconfigure_threads(thread_count); // Get initial state auto *index = indices[INDEX_HNSW_DISK].get(); @@ -116,6 +116,8 @@ BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_ADD_LABEL_ASYNC_DISK, fp32_index // Measure the AddLabel_AsyncIngest benchmark auto start_time = std::chrono::high_resolution_clock::now(); AddLabel_AsyncIngest(st); + // Wait for all jobs to complete before measuring end time + mock_thread_pool.thread_pool_wait(); auto end_time = std::chrono::high_resolution_clock::now(); // Calculate stats @@ -131,7 +133,7 @@ BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_ADD_LABEL_ASYNC_DISK, fp32_index } BENCHMARK_REGISTER_F(BM_VecSimBasics, BM_ADD_LABEL_ASYNC_DISK) ->Unit(benchmark::kNanosecond) - ->Iterations(BM_VecSimGeneral::block_size) + ->Iterations(10000) ->Args({INDEX_HNSW_DISK, 1}) ->Args({INDEX_HNSW_DISK, 4}) ->Args({INDEX_HNSW_DISK, 8}) diff --git a/tests/benchmark/data/scripts/CMakeLists.txt b/tests/benchmark/data/scripts/CMakeLists.txt index d19e0bc1e..18984fc77 100644 --- a/tests/benchmark/data/scripts/CMakeLists.txt +++ b/tests/benchmark/data/scripts/CMakeLists.txt @@ -21,7 +21,15 @@ find_package(PkgConfig REQUIRED) pkg_check_modules(ROCKSDB REQUIRED IMPORTED_TARGET rocksdb) # Create the executable -add_executable(hnsw_disk_serializer hnsw_disk_serializer.cpp) +add_executable(hnsw_disk_serializer + hnsw_disk_serializer.cpp + ${CMAKE_SOURCE_DIR}/tests/utils/mock_thread_pool.cpp +) + +# Include the mock_thread_pool header +target_include_directories(hnsw_disk_serializer PRIVATE + ${CMAKE_SOURCE_DIR}/tests/utils +) # Link against VecSim and RocksDB (using CMake targets from parent project) target_link_libraries(hnsw_disk_serializer diff --git a/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp b/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp index 1af29add1..4df1a179d 100644 --- a/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp +++ b/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp @@ -5,7 +5,7 @@ * It supports both .raw files (no header) and .fbin files (with header). * * Usage: - * ./hnsw_disk_serializer [M] [efConstruction] + * ./hnsw_disk_serializer [M] [efConstruction] [threads] * * Arguments: * input_file - Binary file containing vectors (.raw or .fbin) @@ -17,14 +17,15 @@ * type - Data type: FLOAT32, FLOAT64, BFLOAT16, FLOAT16, INT8, UINT8 * M - HNSW M parameter (default: 64) * efConstruction - HNSW efConstruction parameter (default: 512) + * threads - Number of threads for parallel indexing (default: 4, use 0 for single-threaded) * * Examples: * # Using .raw file (dimension required) * ./hnsw_disk_serializer dbpedia-cosine-dim768-test_vectors.raw \ * dbpedia-cosine-dim768-M64-efc512 768 Cosine FLOAT32 64 512 * - * # Using .fbin file (auto-detect dimension) - * ./hnsw_disk_serializer vectors.fbin output 0 Cosine FLOAT32 64 512 + * # Using .fbin file (auto-detect dimension) with 8 threads + * ./hnsw_disk_serializer vectors.fbin output 0 Cosine FLOAT32 64 512 8 * * # Using .fbin file (verify dimension) * ./hnsw_disk_serializer vectors.fbin output 768 Cosine FLOAT32 64 512 @@ -35,6 +36,7 @@ #include "VecSim/algorithms/hnsw/hnsw_disk.h" #include "VecSim/types/bfloat16.h" #include "VecSim/types/float16.h" +#include "mock_thread_pool.h" #include #include #include @@ -43,6 +45,7 @@ #include #include #include +#include using bfloat16 = vecsim_types::bfloat16; using float16 = vecsim_types::float16; @@ -264,7 +267,7 @@ void saveIndexByType(VecSimIndex *index, const std::string &output_file) { int main(int argc, char *argv[]) { if (argc < 6) { - std::cerr << "Usage: " << argv[0] << " [M] [efConstruction]\n"; + std::cerr << "Usage: " << argv[0] << " [M] [efConstruction] [threads]\n"; std::cerr << "\nArguments:\n"; std::cerr << " input_file - Binary file (.raw or .fbin)\n"; std::cerr << " output_name - Base name for output files\n"; @@ -273,6 +276,7 @@ int main(int argc, char *argv[]) { std::cerr << " type - Data type: FLOAT32, FLOAT64, BFLOAT16, FLOAT16, INT8, UINT8\n"; std::cerr << " M - HNSW M parameter (default: 64)\n"; std::cerr << " efConstruction - HNSW efConstruction parameter (default: 512)\n"; + std::cerr << " threads - Number of threads for parallel indexing (default: 4, use 0 for single-threaded)\n"; return 1; } @@ -283,6 +287,7 @@ int main(int argc, char *argv[]) { VecSimType type = parseType(argv[5]); size_t M = (argc > 6) ? std::stoull(argv[6]) : 64; size_t efConstruction = (argc > 7) ? std::stoull(argv[7]) : 512; + size_t num_threads = (argc > 8) ? std::stoull(argv[8]) : 4; // Check if input file exists if (!std::filesystem::exists(input_file)) { @@ -340,6 +345,7 @@ int main(int argc, char *argv[]) { std::cout << "Type: " << getTypeName(type) << "\n"; std::cout << "M: " << M << "\n"; std::cout << "efConstruction: " << efConstruction << "\n"; + std::cout << "Threads: " << (num_threads > 0 ? std::to_string(num_threads) : "single-threaded") << "\n"; std::cout << "Number of vectors: " << num_vectors << "\n"; std::cout << "==================================\n\n"; @@ -376,6 +382,57 @@ int main(int argc, char *argv[]) { return 1; } + // Set up multi-threaded processing if requested + std::unique_ptr mock_thread_pool; + if (num_threads > 0) { + mock_thread_pool = std::make_unique(); + mock_thread_pool->ctx->index_strong_ref.reset(index, [](VecSimIndex*) { + // Custom deleter that does nothing - we manage index lifetime separately + }); + mock_thread_pool->thread_pool_size = num_threads; + mock_thread_pool->init_threads(); + + // Configure the disk index to use the job queue + if (type == VecSimType_FLOAT32) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, + tieredIndexMock::submit_callback); + } + } else if (type == VecSimType_FLOAT64) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, + tieredIndexMock::submit_callback); + } + } else if (type == VecSimType_BFLOAT16) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, + tieredIndexMock::submit_callback); + } + } else if (type == VecSimType_FLOAT16) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, + tieredIndexMock::submit_callback); + } + } else if (type == VecSimType_INT8) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, + tieredIndexMock::submit_callback); + } + } else if (type == VecSimType_UINT8) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, + tieredIndexMock::submit_callback); + } + } + std::cout << "Multi-threaded indexing enabled with " << num_threads << " threads\n"; + } + std::cout << "Index created successfully\n"; std::cout << "Loading vectors from file...\n"; @@ -399,6 +456,40 @@ int main(int argc, char *argv[]) { std::cerr << "Warning: Expected " << num_vectors << " vectors but added " << vectors_added << "\n"; } + // Wait for all background jobs to complete if using multi-threaded indexing + if (mock_thread_pool) { + std::cout << "Waiting for background indexing jobs to complete...\n"; + mock_thread_pool->thread_pool_wait(); + + // Flush any remaining pending vectors and wait for those jobs too + if (type == VecSimType_FLOAT32) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) disk_index->flushBatch(); + } else if (type == VecSimType_FLOAT64) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) disk_index->flushBatch(); + } else if (type == VecSimType_BFLOAT16) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) disk_index->flushBatch(); + } else if (type == VecSimType_FLOAT16) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) disk_index->flushBatch(); + } else if (type == VecSimType_INT8) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) disk_index->flushBatch(); + } else if (type == VecSimType_UINT8) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) disk_index->flushBatch(); + } + + // Wait again for the flush batch jobs to complete + mock_thread_pool->thread_pool_wait(); + std::cout << "All background jobs completed.\n"; + + // Stop the thread pool before saving + mock_thread_pool->thread_pool_join(); + } + auto index_time = std::chrono::high_resolution_clock::now(); auto indexing_duration = std::chrono::duration_cast(index_time - start_time).count(); std::cout << "Indexing completed in " << indexing_duration << " ms\n"; @@ -443,6 +534,9 @@ int main(int argc, char *argv[]) { std::cout << "Total size: " << (metadata_size + checkpoint_size) / (1024.0 * 1024.0) << " MB\n"; // Cleanup + if (mock_thread_pool) { + mock_thread_pool.reset(); + } VecSimIndex_Free(index); std::filesystem::remove_all(rocksdb_path); From 2bd542338e657daebbc9525c1816e8673652faab Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Mon, 15 Dec 2025 10:45:40 +0200 Subject: [PATCH 03/34] Changes --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 455 +++++++++++------- .../algorithms/hnsw/hnsw_disk_serializer.h | 49 +- 2 files changed, 319 insertions(+), 185 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index feec6d30d..85d2d0fe6 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -195,7 +195,7 @@ class HNSWDiskIndex : public VecSimIndexAbstract mutable std::default_random_engine levelGenerator; // multithreaded scenario. - size_t curElementCount; + std::atomic curElementCount; size_t numMarkedDeleted; idType entrypointNode; size_t maxLevel; // this is the top level of the entry point's element @@ -300,7 +300,8 @@ class HNSWDiskIndex : public VecSimIndexAbstract mutable std::shared_mutex rocksDbGuard; // Lock for protecting staged updates during merge from parallel insert jobs - mutable std::mutex stagedUpdatesGuard; + // Uses shared_mutex for better read concurrency - multiple readers can access simultaneously + mutable std::shared_mutex stagedUpdatesGuard; // Atomic counter for pending insert jobs (for synchronization with flush job) std::atomic pendingInsertJobsCounter{0}; @@ -315,7 +316,8 @@ class HNSWDiskIndex : public VecSimIndexAbstract // processingBatch contains vectors currently being inserted by worker threads struct ProcessingBatch { vecsim_stl::vector vectorIds; - std::unordered_map rawVectors; + std::unordered_map rawVectors; // Original float vectors + std::unordered_map processedVectors; // Processed/quantized vectors size_t count = 0; ProcessingBatch(std::shared_ptr allocator) : vectorIds(allocator) {} @@ -323,6 +325,7 @@ class HNSWDiskIndex : public VecSimIndexAbstract void clear() { vectorIds.clear(); rawVectors.clear(); + processedVectors.clear(); count = 0; } }; @@ -335,6 +338,13 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Needed because addElement can resize the container, invalidating pointers mutable std::shared_mutex vectorsGuard; + // Note: metadataGuard was consolidated into indexDataGuard + // indexDataGuard now protects: entrypointNode, maxLevel, idToMetaData, labelToIdMap + + // Lock for protecting rawVectorsInRAM during concurrent access + // Needed because unordered_map can rehash during insert, invalidating iterators + mutable std::shared_mutex rawVectorsGuard; + protected: HNSWDiskIndex() = delete; // default constructor is disabled. // default (shallow) copy constructor is disabled. @@ -408,8 +418,9 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Filters out: nodes marked as deleted, and nodes with invalid IDs (>= curElementCount) inline bool filterDeletedNodes(vecsim_stl::vector& neighbors) const { size_t original_size = neighbors.size(); + size_t elementCount = curElementCount.load(std::memory_order_acquire); auto new_end = std::remove_if(neighbors.begin(), neighbors.end(), - [this](idType id) { return id >= curElementCount || isMarkedDeleted(id); }); + [this, elementCount](idType id) { return id >= elementCount || isMarkedDeleted(id); }); neighbors.erase(new_end, neighbors.end()); return neighbors.size() < original_size; } @@ -521,14 +532,17 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Flagging API template void markAs(idType internalId) { + std::shared_lock lock(indexDataGuard); __atomic_fetch_or(&idToMetaData[internalId].flags, FLAG, 0); } template void unmarkAs(idType internalId) { + std::shared_lock lock(indexDataGuard); __atomic_fetch_and(&idToMetaData[internalId].flags, ~FLAG, 0); } template bool isMarkedAs(idType internalId) const { + std::shared_lock lock(indexDataGuard); return __atomic_load_n(&idToMetaData[internalId].flags, 0) & FLAG; } @@ -690,7 +704,13 @@ HNSWDiskIndex::topKQuery(const void *query_data, size_t k, auto rep = new VecSimQueryReply(this->allocator); this->lastMode = STANDARD_KNN; - if ((curElementCount == 0 && pendingVectorCount == 0) || k == 0) { + // Check if index is empty - need to lock to read pendingVectorCount safely + size_t pendingCount; + { + std::lock_guard lock(batchSwapGuard); + pendingCount = pendingVectorCount; + } + if ((curElementCount.load(std::memory_order_acquire) == 0 && pendingCount == 0) || k == 0) { return rep; } @@ -724,12 +744,9 @@ HNSWDiskIndex::topKQuery(const void *query_data, size_t k, } auto results = searchLayerLabels(bottom_layer_ep, query_data, processed_query , 0, query_ef); - - if (pendingVectorCount > 0) { - // Search pending vectors using the helper method - searchPendingVectors(query_data, results, k); - } + // Search pending vectors - the method handles locking and checking if there are pending vectors + searchPendingVectors(query_data, results, k); while (results.size() > k) { results.pop(); } @@ -864,11 +881,17 @@ int HNSWDiskIndex::addVector( const void *vector, labelType label ) { + // Atomically get a unique element ID - this is the critical fix for Race #1 + // fetch_add returns the OLD value before incrementing, giving us a unique ID + idType newElementId = static_cast(curElementCount.fetch_add(1, std::memory_order_acq_rel)); + // Store raw vector in RAM first (until flush batch) // We need to store the original vector before preprocessing - idType newElementId = curElementCount; const char* raw_data = reinterpret_cast(vector); - rawVectorsInRAM[newElementId] = std::string(raw_data, this->inputBlobSize); + { + std::unique_lock lock(rawVectorsGuard); + rawVectorsInRAM[newElementId] = std::string(raw_data, this->inputBlobSize); + } // Preprocess the vector ProcessedBlobs processedBlobs = this->preprocess(vector); @@ -883,31 +906,37 @@ int HNSWDiskIndex::addVector( size_t elementMaxLevel = getRandomLevel(mult); DiskElementMetaData new_element(label, elementMaxLevel); - // Ensure capacity for the new element ID - if (newElementId >= indexCapacity()) { - size_t new_cap = ((newElementId + this->blockSize) / this->blockSize) * this->blockSize; - visitedNodesHandlerPool.resize(new_cap); - idToMetaData.resize(new_cap); - labelToIdMap.reserve(new_cap); - } - - // Store metadata immediately - idToMetaData[newElementId] = new_element; - labelToIdMap[label] = newElementId; - - // Increment vector count immediately - curElementCount++; + // Ensure capacity for the new element ID (protected by indexDataGuard) + { + std::unique_lock lock(indexDataGuard); + if (newElementId >= indexCapacity()) { + size_t new_cap = ((newElementId + this->blockSize) / this->blockSize) * this->blockSize; + visitedNodesHandlerPool.resize(new_cap); + idToMetaData.resize(new_cap); + labelToIdMap.reserve(new_cap); + } + // Store metadata immediately + idToMetaData[newElementId] = new_element; + labelToIdMap[label] = newElementId; + } // Resize visited nodes handler pool to accommodate new elements - visitedNodesHandlerPool.resize(curElementCount); + // Use load() to read atomic value + visitedNodesHandlerPool.resize(curElementCount.load(std::memory_order_acquire)); // Add only the vector ID to pending vectors for indexing - pendingVectorIds.push_back(newElementId); - pendingVectorCount++; + // Protected by batchSwapGuard to synchronize with query threads and processBatch + bool shouldProcessBatch = false; + { + std::lock_guard lock(batchSwapGuard); + pendingVectorIds.push_back(newElementId); + pendingVectorCount++; + shouldProcessBatch = (pendingVectorCount >= batchThreshold); + } - // Process batch if threshold reached - if (pendingVectorCount >= batchThreshold) { + // Process batch if threshold reached (outside lock to avoid holding lock during batch processing) + if (shouldProcessBatch) { processBatch(); } @@ -983,7 +1012,7 @@ idType HNSWDiskIndex::mutuallyConnectNewElement( // Add to staged graph updates (for insertions) - protected by stagedUpdatesGuard { - std::lock_guard lock(stagedUpdatesGuard); + std::unique_lock lock(stagedUpdatesGuard); uint64_t insert_key = makeRepairKey(new_node_id, level); stagedInsertMap[insert_key] = stagedInsertUpdates.size(); stagedInsertUpdates.emplace_back(new_node_id, level, neighbor_ids, this->allocator); @@ -1015,7 +1044,7 @@ idType HNSWDiskIndex::mutuallyConnectNewElement( if (current_neighbor_count < max_M_cur) { // Neighbor has capacity, just add the new node - std::lock_guard lock(stagedUpdatesGuard); + std::unique_lock lock(stagedUpdatesGuard); stagedInsertNeighborUpdates.emplace_back(selected_neighbor, level, new_node_id); } else { // Neighbor is full, need to re-evaluate connections using revisitNeighborConnections @@ -1162,6 +1191,7 @@ void HNSWDiskIndex::stageRevisitNeighborConnections(idType n " WARNING: Could not read existing neighbors for node %u at level %zu", selected_neighbor, level); // Fall back to simple neighbor update + std::unique_lock lock(stagedUpdatesGuard); stagedInsertNeighborUpdates.emplace_back(selected_neighbor, level, new_node_id); return; } @@ -1216,7 +1246,7 @@ void HNSWDiskIndex::stageRevisitNeighborConnections(idType n // Stage this update - the neighbor's neighbor list will be completely replaced // We'll need to handle this specially in flushStagedGraphUpdates // Protected by stagedUpdatesGuard for thread safety - std::lock_guard lock(stagedUpdatesGuard); + std::unique_lock lock(stagedUpdatesGuard); uint64_t insert_key = makeRepairKey(selected_neighbor, level); stagedInsertMap[insert_key] = stagedInsertUpdates.size(); stagedInsertUpdates.emplace_back(selected_neighbor, level, selected_neighbor_ids, @@ -1228,7 +1258,7 @@ void HNSWDiskIndex::stageRevisitNeighborConnections(idType n } else { // The new node was not selected, so we only need to stage the unidirectional connection // from new node to selected neighbor - std::lock_guard lock(stagedUpdatesGuard); + std::unique_lock lock(stagedUpdatesGuard); stagedInsertNeighborUpdates.emplace_back(new_node_id, level, selected_neighbor); } } @@ -1238,10 +1268,9 @@ idType HNSWDiskIndex::searchBottomLayerEP(const void *query_ void *timeoutCtx, VecSimQueryReply_Code *rc) const { if (rc) *rc = VecSim_QueryReply_OK; - - // auto [curr_element, max_level] = safeGetEntryPointState(); - auto curr_element = entrypointNode; - auto max_level = maxLevel; + + // Use safeGetEntryPointState to read entry point with proper locking + auto [curr_element, max_level] = safeGetEntryPointState(); if (curr_element == INVALID_ID) return curr_element; // index is empty. @@ -1327,7 +1356,7 @@ const void* HNSWDiskIndex::getVectorFromGraphValue(const std template const void *HNSWDiskIndex::getDataByInternalId(idType id) const { - assert(id < curElementCount); + assert(id < curElementCount.load(std::memory_order_acquire)); // Acquire shared lock to prevent concurrent resize of vectors container std::shared_lock lock(vectorsGuard); @@ -1341,21 +1370,23 @@ const void *HNSWDiskIndex::getDataByInternalId(idType id) co template bool HNSWDiskIndex::getRawVector(idType id, void* output_buffer) const { - - if (id >= curElementCount) { + size_t elementCount = curElementCount.load(std::memory_order_acquire); + if (id >= elementCount) { this->log(VecSimCommonStrings::LOG_WARNING_STRING, "WARNING: getRawVector called with invalid id %u (current count: %zu)", - id, curElementCount); + id, elementCount); return false; } - // First check RAM (for vectors not yet flushed) - auto it = rawVectorsInRAM.find(id); - if (it != rawVectorsInRAM.end()) { - const char* data_ptr = it->second.data(); - std::memcpy(output_buffer, data_ptr, this->inputBlobSize); - return true; - + // First check RAM (for vectors not yet flushed) - protected by shared lock + { + std::shared_lock lock(rawVectorsGuard); + auto it = rawVectorsInRAM.find(id); + if (it != rawVectorsInRAM.end()) { + const char* data_ptr = it->second.data(); + std::memcpy(output_buffer, data_ptr, this->inputBlobSize); + return true; + } } // If not in RAM or cache, retrieve from disk with shared lock for thread safety @@ -1388,23 +1419,26 @@ bool HNSWDiskIndex::getRawVector(idType id, void* output_buf } -// Internal version that assumes caller already holds the lock +// Internal version for use during flush operations template bool HNSWDiskIndex::getRawVectorInternal(idType id, void* output_buffer) const { - - if (id >= curElementCount) { + size_t elementCount = curElementCount.load(std::memory_order_acquire); + if (id >= elementCount) { this->log(VecSimCommonStrings::LOG_WARNING_STRING, "WARNING: getRawVectorInternal called with invalid id %u (current count: %zu)", - id, curElementCount); + id, elementCount); return false; } - // First check RAM (for vectors not yet flushed) - auto it = rawVectorsInRAM.find(id); - if (it != rawVectorsInRAM.end()) { - const char* data_ptr = it->second.data(); - std::memcpy(output_buffer, data_ptr, this->inputBlobSize); - return true; + // First check RAM (for vectors not yet flushed) - protected by shared lock + { + std::shared_lock lock(rawVectorsGuard); + auto it = rawVectorsInRAM.find(id); + if (it != rawVectorsInRAM.end()) { + const char* data_ptr = it->second.data(); + std::memcpy(output_buffer, data_ptr, this->inputBlobSize); + return true; + } } // Also check processingBatch (for vectors being processed by worker threads) @@ -1568,7 +1602,7 @@ void HNSWDiskIndex::greedySearchLevel(const void *data_point for (size_t i = 0; i < neighbors.size(); i++) { idType candidate = neighbors[i]; - assert (candidate < curElementCount && "candidate error: out of index range"); + assert (candidate < curElementCount.load(std::memory_order_acquire) && "candidate error: out of index range"); // const int8_t* q_data = reinterpret_cast(data_point); // std::cout << "q_data[0]: " << static_cast(q_data[0]) << std::endl; // std::cout << "q_data[n]: " << static_cast(q_data[this->dim - 1]) << std::endl; @@ -1663,7 +1697,7 @@ void HNSWDiskIndex::processCandidate( if (!neighbors.empty()) { for (idType candidate_id : neighbors) { // Skip invalid neighbors - assert(candidate_id < curElementCount); + assert(candidate_id < curElementCount.load(std::memory_order_acquire)); if (visited_set->find(candidate_id) != visited_set->end()) { continue; @@ -1745,7 +1779,7 @@ size_t HNSWDiskIndex::indexCapacity() const { template size_t HNSWDiskIndex::indexSize() const { - return this->curElementCount; + return this->curElementCount.load(std::memory_order_acquire); } template @@ -1778,46 +1812,52 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level // First check staged graph updates using hash maps for O(1) lookup uint64_t lookup_key = makeRepairKey(nodeId, level); - // Check insert staging area - auto insert_it = stagedInsertMap.find(lookup_key); - if (insert_it != stagedInsertMap.end()) { - const auto &update = stagedInsertUpdates[insert_it->second]; - result.reserve(update.neighbors.size()); - for (size_t i = 0; i < update.neighbors.size(); i++) { - result.push_back(update.neighbors[i]); - } - // Filter out deleted nodes using helper - filterDeletedNodes(result); - return; - } + // Check staged updates under lock - use shared_lock for read access + { + std::shared_lock lock(stagedUpdatesGuard); - // Check delete staging area - auto delete_it = stagedDeleteMap.find(lookup_key); - if (delete_it != stagedDeleteMap.end()) { - const auto &update = stagedDeleteUpdates[delete_it->second]; - result.reserve(update.neighbors.size()); - for (size_t i = 0; i < update.neighbors.size(); i++) { - result.push_back(update.neighbors[i]); + // Check insert staging area + auto insert_it = stagedInsertMap.find(lookup_key); + if (insert_it != stagedInsertMap.end()) { + const auto &update = stagedInsertUpdates[insert_it->second]; + result.reserve(update.neighbors.size()); + for (size_t i = 0; i < update.neighbors.size(); i++) { + result.push_back(update.neighbors[i]); + } + // Release lock before calling filterDeletedNodes which may take other locks + lock.unlock(); + filterDeletedNodes(result); + return; } - // Filter out deleted nodes using helper - filterDeletedNodes(result); - return; - } - // Also check staged repair updates (already cleaned neighbors waiting to be flushed) - auto repair_it = stagedRepairMap.find(lookup_key); - if (repair_it != stagedRepairMap.end()) { - auto &update = stagedRepairUpdates[repair_it->second]; - result.reserve(update.neighbors.size()); - for (size_t i = 0; i < update.neighbors.size(); i++) { - result.push_back(update.neighbors[i]); + // Check delete staging area + auto delete_it = stagedDeleteMap.find(lookup_key); + if (delete_it != stagedDeleteMap.end()) { + const auto &update = stagedDeleteUpdates[delete_it->second]; + result.reserve(update.neighbors.size()); + for (size_t i = 0; i < update.neighbors.size(); i++) { + result.push_back(update.neighbors[i]); + } + // Release lock before calling filterDeletedNodes + lock.unlock(); + filterDeletedNodes(result); + return; } - // Filter in case nodes were deleted after this repair was staged - if (filterDeletedNodes(result)) { - // Update the existing repair entry with the more up-to-date cleaned list - update.neighbors = result; + + // Also check staged repair updates (already cleaned neighbors waiting to be flushed) + auto repair_it = stagedRepairMap.find(lookup_key); + if (repair_it != stagedRepairMap.end()) { + const auto &update = stagedRepairUpdates[repair_it->second]; + result.reserve(update.neighbors.size()); + for (size_t i = 0; i < update.neighbors.size(); i++) { + result.push_back(update.neighbors[i]); + } + // Note: We can't update the repair entry here since we have a shared lock + // Just filter and return; the repair will be re-evaluated if needed + lock.unlock(); + filterDeletedNodes(result); + return; } - return; } // If not found in staged updates, check disk with shared lock for thread safety @@ -1837,7 +1877,7 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level // Lazy repair: if we filtered any deleted nodes, stage for cleanup // Use hash map for O(1) duplicate detection // Note: stagedRepairMap and stagedRepairUpdates are protected by stagedUpdatesGuard - std::lock_guard repairLock(stagedUpdatesGuard); + std::unique_lock repairLock(stagedUpdatesGuard); uint64_t repair_key = makeRepairKey(nodeId, level); if (stagedRepairMap.find(repair_key) == stagedRepairMap.end()) { stagedRepairMap[repair_key] = stagedRepairUpdates.size(); @@ -1852,20 +1892,31 @@ void HNSWDiskIndex::getNeighborsAndVector(idType nodeId, siz // Clear the result vector first result.clear(); - // First check staged graph updates - for (const auto& update : stagedInsertUpdates) { - if (update.node_id == nodeId && update.level == level) { + // First check staged graph updates (protected by stagedUpdatesGuard) + // Use shared_lock for read-only access and hash map for O(1) lookup + { + std::shared_lock lock(stagedUpdatesGuard); + uint64_t lookup_key = makeRepairKey(nodeId, level); + auto it = stagedInsertMap.find(lookup_key); + if (it != stagedInsertMap.end()) { + const auto& update = stagedInsertUpdates[it->second]; result.reserve(update.neighbors.size()); for (size_t i = 0; i < update.neighbors.size(); i++) { result.push_back(update.neighbors[i]); } } } - auto it = rawVectorsInRAM.find(nodeId); - if (it != rawVectorsInRAM.end()) { - std::memcpy(vector_data, it->second.data(), this->inputBlobSize); + // Check rawVectorsInRAM with shared lock + bool foundInRAM = false; + { + std::shared_lock lock(rawVectorsGuard); + auto it = rawVectorsInRAM.find(nodeId); + if (it != rawVectorsInRAM.end()) { + std::memcpy(vector_data, it->second.data(), this->inputBlobSize); + foundInRAM = true; + } } - if (!result.empty() && it != rawVectorsInRAM.end()) { + if (!result.empty() && foundInRAM) { return; } // If not found in staged updates, check disk with shared lock for thread safety @@ -1899,24 +1950,46 @@ void HNSWDiskIndex::getNeighborsAndVector(labelType nodeId, template void HNSWDiskIndex::searchPendingVectors( const void *query_data, candidatesLabelsMaxHeap &top_candidates, size_t k) const { - for (size_t i = 0; i < pendingVectorCount; i++) { - idType vectorId = pendingVectorIds[i]; - if (isMarkedDeleted(vectorId)) { - // Skip deleted vectors - continue; + // Copy pending vector data under lock to avoid holding lock during expensive distance calculations + vecsim_stl::vector> pendingData(this->allocator); + std::vector vectorDataCopies; + + { + std::lock_guard batchLock(batchSwapGuard); + std::shared_lock lock(rawVectorsGuard); + + if (pendingVectorCount == 0) { + return; } - // Get the vector data from memory - const void* vector_data = rawVectorsInRAM.find(vectorId)->second.data(); + pendingData.reserve(pendingVectorCount); + vectorDataCopies.reserve(pendingVectorCount); - // Get metadata for this vector - const DiskElementMetaData &metadata = idToMetaData[vectorId]; - labelType label = metadata.label; + for (size_t i = 0; i < pendingVectorCount; i++) { + idType vectorId = pendingVectorIds[i]; + // Use direct flag check to avoid taking indexDataGuard while holding other locks + if (__atomic_load_n(&idToMetaData[vectorId].flags, 0) & DELETE_MARK) { + continue; + } + + auto it = rawVectorsInRAM.find(vectorId); + if (it == rawVectorsInRAM.end()) { + continue; + } + + const DiskElementMetaData &metadata = idToMetaData[vectorId]; + pendingData.emplace_back(vectorId, metadata.label); + vectorDataCopies.push_back(it->second); // Copy the vector data + } + } + // Locks released - now do expensive distance calculations without holding locks + + for (size_t i = 0; i < pendingData.size(); i++) { + labelType label = pendingData[i].second; + const void* vector_data = vectorDataCopies[i].data(); - // Calculate distance DistType dist = this->calcDistanceRaw(query_data, vector_data); - // Add to candidates if it's good enough if (top_candidates.size() < k) { top_candidates.emplace(dist, label); } else if (dist < top_candidates.top().first) { @@ -1993,9 +2066,24 @@ void HNSWDiskIndex::processBatch() { processingBatch->rawVectors = std::move(rawVectorsInRAM); processingBatch->count = pendingVectorCount; + // Copy processed vectors to processing batch (prevents race with main thread resizing vectors) + { + std::shared_lock lock(vectorsGuard); + for (idType vectorId : processingBatch->vectorIds) { + const void *processed = this->vectors->getElement(vectorId); + if (processed) { + processingBatch->processedVectors[vectorId] = + std::string(static_cast(processed), this->dataSize); + } + } + } + // Clear pending structures for new vectors pendingVectorIds = vecsim_stl::vector(this->allocator); - rawVectorsInRAM.clear(); + { + std::unique_lock lock(rawVectorsGuard); + rawVectorsInRAM.clear(); + } pendingVectorCount = 0; // Clear any previous staged updates @@ -2003,26 +2091,32 @@ void HNSWDiskIndex::processBatch() { stagedInsertMap.clear(); stagedInsertNeighborUpdates.clear(); - // Snapshot current entry point state - idType currentEntryPoint = entrypointNode; - size_t currentMaxLevel = maxLevel; - - // First pass: Update entry point for any vectors with higher levels - for (size_t i = 0; i < processingBatch->count; i++) { - idType vectorId = processingBatch->vectorIds[i]; - if (isMarkedDeleted(vectorId)) - continue; + // Snapshot and update entry point state - protected by indexDataGuard + idType currentEntryPoint; + size_t currentMaxLevel; + { + std::unique_lock lock(indexDataGuard); + currentEntryPoint = entrypointNode; + currentMaxLevel = maxLevel; + + // First pass: Update entry point for any vectors with higher levels + for (size_t i = 0; i < processingBatch->count; i++) { + idType vectorId = processingBatch->vectorIds[i]; + // Use direct flag check to avoid recursive lock (isMarkedDeleted takes shared lock) + if (__atomic_load_n(&idToMetaData[vectorId].flags, 0) & DELETE_MARK) + continue; - DiskElementMetaData &metadata = idToMetaData[vectorId]; - if (currentEntryPoint == INVALID_ID || metadata.topLevel > currentMaxLevel) { - currentEntryPoint = vectorId; - currentMaxLevel = metadata.topLevel; + DiskElementMetaData &metadata = idToMetaData[vectorId]; + if (currentEntryPoint == INVALID_ID || metadata.topLevel > currentMaxLevel) { + currentEntryPoint = vectorId; + currentMaxLevel = metadata.topLevel; + } } - } - // Update global entry point - entrypointNode = currentEntryPoint; - maxLevel = currentMaxLevel; + // Update global entry point + entrypointNode = currentEntryPoint; + maxLevel = currentMaxLevel; + } // Count valid vectors and create jobs size_t validVectorCount = 0; @@ -2121,12 +2215,18 @@ void HNSWDiskIndex::executeInsertJob(HNSWDiskInsertJob *job) try { // Get vector data from processing batch (read-only, thread-safe) - const void *vector_data = this->vectors->getElement(job->vectorId); - auto it = processingBatch->rawVectors.find(job->vectorId); - if (it == processingBatch->rawVectors.end()) { - throw std::runtime_error("Vector not found in processing batch"); + // Use local copies to avoid race with main thread resizing vectors container + auto rawIt = processingBatch->rawVectors.find(job->vectorId); + if (rawIt == processingBatch->rawVectors.end()) { + throw std::runtime_error("Raw vector not found in processing batch"); + } + const void *raw_vector_data = rawIt->second.data(); + + auto procIt = processingBatch->processedVectors.find(job->vectorId); + if (procIt == processingBatch->processedVectors.end()) { + throw std::runtime_error("Processed vector not found in processing batch"); } - const void *raw_vector_data = it->second.data(); + const void *vector_data = procIt->second.data(); // Skip if this is the entry point (no connections to make) if (job->vectorId != job->entryPointSnapshot) { @@ -2167,8 +2267,13 @@ void HNSWDiskIndex::executeFlushJob(HNSWDiskFlushJob *job) { // Flush all staged updates to RocksDB flushStagedGraphUpdates(stagedInsertUpdates, stagedInsertNeighborUpdates); - // Clear staging - stagedInsertMap.clear(); + // Clear staging - must hold stagedUpdatesGuard to prevent race with getNeighbors + { + std::unique_lock stagingLock(stagedUpdatesGuard); + stagedInsertMap.clear(); + stagedInsertUpdates.clear(); + stagedInsertNeighborUpdates.clear(); + } lock.unlock(); @@ -2187,7 +2292,7 @@ template void HNSWDiskIndex::mergeLocalStagedUpdates( vecsim_stl::vector &localGraphUpdates, vecsim_stl::vector &localNeighborUpdates) { - std::lock_guard lock(stagedUpdatesGuard); + std::unique_lock lock(stagedUpdatesGuard); // Merge graph updates for (auto &update : localGraphUpdates) { @@ -2235,10 +2340,11 @@ void HNSWDiskIndex::repairNeighborConnections( // Use a hash set to track candidate IDs for O(1) duplicate detection std::unordered_set candidate_ids; + size_t elementCount = curElementCount.load(std::memory_order_acquire); // Add existing neighbors (excluding the deleted node) with their distances for (idType nn : neighbor_neighbors) { - if (nn != deleted_id && nn < curElementCount && !isMarkedDeleted(nn)) { + if (nn != deleted_id && nn < elementCount && !isMarkedDeleted(nn)) { const void *nn_data = getDataByInternalId(nn); DistType dist = this->calcDistance(nn_data, neighbor_data); candidates.emplace_back(dist, nn); @@ -2248,7 +2354,7 @@ void HNSWDiskIndex::repairNeighborConnections( // Add deleted node's neighbors (excluding current neighbor) as repair candidates for (idType candidate_id : deleted_node_neighbors) { - if (candidate_id != neighbor_id && candidate_id < curElementCount && + if (candidate_id != neighbor_id && candidate_id < elementCount && !isMarkedDeleted(candidate_id)) { // Check if already in candidates to avoid duplicates using O(1) hash set lookup if (candidate_ids.find(candidate_id) == candidate_ids.end()) { @@ -2264,7 +2370,7 @@ void HNSWDiskIndex::repairNeighborConnections( vecsim_stl::unordered_set original_neighbors_set(this->allocator); original_neighbors_set.reserve(neighbor_neighbors.size()); for (idType nn : neighbor_neighbors) { - if (nn != deleted_id && nn < curElementCount) { + if (nn != deleted_id && nn < elementCount) { original_neighbors_set.insert(nn); } } @@ -2296,11 +2402,12 @@ void HNSWDiskIndex::processDeleteBatch() { // Create a set of IDs being deleted in this batch for quick lookup std::unordered_set deletingIds(pendingDeleteIds.begin(), pendingDeleteIds.end()); + size_t elementCount = curElementCount.load(std::memory_order_acquire); // Process each deleted node for (idType deleted_id : pendingDeleteIds) { // Skip if already processed or invalid - if (deleted_id >= curElementCount || deleted_id >= idToMetaData.size()) { + if (deleted_id >= elementCount || deleted_id >= idToMetaData.size()) { continue; } @@ -2320,7 +2427,7 @@ void HNSWDiskIndex::processDeleteBatch() { // For each neighbor of the deleted node for (idType neighbor_id : deleted_node_neighbors) { // Skip if neighbor is also deleted, invalid, or in the current deletion batch - if (neighbor_id >= curElementCount || isMarkedDeleted(neighbor_id) || + if (neighbor_id >= elementCount || isMarkedDeleted(neighbor_id) || deletingIds.find(neighbor_id) != deletingIds.end()) { continue; } @@ -2358,16 +2465,19 @@ void HNSWDiskIndex::processDeleteBatch() { // Mark metadata as invalid and clean up raw vectors AFTER processing all nodes // This ensures getNeighbors() and other methods work correctly during graph repair for (idType deleted_id : pendingDeleteIds) { - if (deleted_id >= curElementCount || deleted_id >= idToMetaData.size()) { + if (deleted_id >= elementCount || deleted_id >= idToMetaData.size()) { continue; } // Mark the metadata as invalid idToMetaData[deleted_id].label = INVALID_LABEL; - // Remove raw vector from RAM if it exists - auto ram_it = rawVectorsInRAM.find(deleted_id); - if (ram_it != rawVectorsInRAM.end()) { - rawVectorsInRAM.erase(ram_it); + // Remove raw vector from RAM if it exists - requires exclusive lock + { + std::unique_lock lock(rawVectorsGuard); + auto ram_it = rawVectorsInRAM.find(deleted_id); + if (ram_it != rawVectorsInRAM.end()) { + rawVectorsInRAM.erase(ram_it); + } } } @@ -2413,8 +2523,9 @@ void HNSWDiskIndex::setBatchThreshold(size_t threshold) { template void HNSWDiskIndex::debugPrintGraphStructure() const { + size_t elementCount = curElementCount.load(std::memory_order_acquire); this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "=== HNSW Disk Index Graph Structure ==="); - this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Total elements: %zu", curElementCount); + this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Total elements: %zu", elementCount); this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Entry point: %u", entrypointNode); this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Max level: %zu", maxLevel); this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "M: %zu, M0: %zu", M, M0); @@ -2425,7 +2536,7 @@ void HNSWDiskIndex::debugPrintGraphStructure() const { // Print metadata for each element this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Element metadata:"); - for (size_t i = 0; i < std::min(curElementCount, idToMetaData.size()); ++i) { + for (size_t i = 0; i < std::min(elementCount, idToMetaData.size()); ++i) { if (idToMetaData[i].label != INVALID_LABEL) { this->log(VecSimCommonStrings::LOG_DEBUG_STRING, " Element %zu: label=%u, topLevel=%zu", i, idToMetaData[i].label, @@ -2439,9 +2550,10 @@ void HNSWDiskIndex::debugPrintGraphStructure() const { template void HNSWDiskIndex::debugPrintNodeNeighbors(idType node_id) const { - if (node_id >= curElementCount) { + size_t elementCount = curElementCount.load(std::memory_order_acquire); + if (node_id >= elementCount) { this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Node %u is out of range (max: %zu)", - node_id, (curElementCount - 1)); + node_id, (elementCount - 1)); return; } @@ -2551,8 +2663,9 @@ size_t HNSWDiskIndex::debugCountGraphEdges() const { template void HNSWDiskIndex::debugValidateGraphConnectivity() const { this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "=== Graph Connectivity Validation ==="); + size_t elementCount = curElementCount.load(std::memory_order_acquire); - if (curElementCount == 0) { + if (elementCount == 0) { this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Index is empty, nothing to validate"); return; } @@ -2566,7 +2679,7 @@ void HNSWDiskIndex::debugValidateGraphConnectivity() const { } // Check connectivity for first few elements - size_t elements_to_check = std::min(curElementCount, size_t(5)); + size_t elements_to_check = std::min(elementCount, size_t(5)); this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Checking connectivity for first %zu elements:", elements_to_check); @@ -2581,7 +2694,7 @@ void HNSWDiskIndex::debugValidateGraphConnectivity() const { this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Checking for isolated nodes..."); size_t isolated_count = 0; - for (size_t i = 0; i < curElementCount; ++i) { + for (size_t i = 0; i < elementCount; ++i) { if (idToMetaData[i].label == INVALID_LABEL) continue; @@ -2812,6 +2925,9 @@ template vecsim_stl::vector HNSWDiskIndex::markDelete(labelType label) { vecsim_stl::vector internal_ids(this->allocator); + // Protect all accesses to labelToIdMap, idToMetaData, entrypointNode, maxLevel + std::unique_lock lock(indexDataGuard); + // Find the internal ID for this label auto it = labelToIdMap.find(label); if (it == labelToIdMap.end()) { @@ -2821,18 +2937,19 @@ vecsim_stl::vector HNSWDiskIndex::markDelete(labelTy const idType internalId = it->second; - // Check if already marked deleted - if (isMarkedDeleted(internalId)) { + // Check if already marked deleted (use unlocked version since we hold lock) + if (__atomic_load_n(&idToMetaData[internalId].flags, 0) & DELETE_MARK) { // Already deleted, return empty vector return internal_ids; } // Mark as deleted (but don't clean up raw vectors yet - they're needed for graph repair // in processDeleteBatch. Cleanup happens there after repair is complete.) - markAs(internalId); + __atomic_fetch_or(&idToMetaData[internalId].flags, DELETE_MARK, 0); this->numMarkedDeleted++; // If this is the entrypoint, we need to replace it + // Note: replaceEntryPoint is called while holding indexDataGuard if (internalId == entrypointNode) { replaceEntryPoint(); } @@ -2847,6 +2964,7 @@ vecsim_stl::vector HNSWDiskIndex::markDelete(labelTy template void HNSWDiskIndex::replaceEntryPoint() { + // PRECONDITION: Caller must hold indexDataGuard (exclusive lock) // This method is called when the current entrypoint is marked as deleted // We need to find a new entrypoint from the remaining non-deleted nodes idType old_entry_point_id = entrypointNode; @@ -2865,8 +2983,9 @@ void HNSWDiskIndex::replaceEntryPoint() { deserializeGraphValue(graph_value, neighbors); // Try to find a non-deleted neighbor + // Use direct flag check to avoid recursive lock (caller holds indexDataGuard) for (size_t i = 0; i < neighbors.size(); i++) { - if (!isMarkedDeleted(neighbors[i])) { + if (!(__atomic_load_n(&idToMetaData[neighbors[i]].flags, 0) & DELETE_MARK)) { entrypointNode = neighbors[i]; return; } @@ -2874,10 +2993,12 @@ void HNSWDiskIndex::replaceEntryPoint() { } // If no suitable neighbor found, search for any non-deleted node at this level - for (idType id = 0; id < curElementCount; id++) { + // Use direct flag check to avoid recursive lock (caller holds indexDataGuard) + size_t elementCount = curElementCount.load(std::memory_order_acquire); + for (idType id = 0; id < elementCount; id++) { if (id != old_entry_point_id && id < idToMetaData.size() && idToMetaData[id].label != INVALID_LABEL && idToMetaData[id].topLevel == maxLevel && - !isMarkedDeleted(id)) { + !(__atomic_load_n(&idToMetaData[id].flags, 0) & DELETE_MARK)) { entrypointNode = id; return; } diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h index 9bf696fe2..91e4f5e0e 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h @@ -72,7 +72,7 @@ HNSWDiskIndex::HNSWDiskIndex( // Initialize level generator with seed based on curElementCount for better distribution // Using curElementCount ensures different sequences for indexes with different sizes // Add a constant offset to avoid seed=0 for empty indexes - this->levelGenerator.seed(200 + this->curElementCount); + this->levelGenerator.seed(200 + this->curElementCount.load(std::memory_order_acquire)); // Initialize processing batch for double-buffering (needed for async operations) this->processingBatch = std::make_unique(this->allocator); @@ -104,10 +104,11 @@ template void HNSWDiskIndex::restoreVectorsFromFile(std::ifstream &input, EncodingVersion version) { auto start_time = std::chrono::steady_clock::now(); + size_t elementCount = this->curElementCount.load(std::memory_order_acquire); // Read vectors directly from file // The vectors are stored as processed blobs (storage format) - for (idType id = 0; id < this->curElementCount; id++) { + for (idType id = 0; id < elementCount; id++) { // Allocate memory for the processed vector auto vector_blob = this->allocator->allocate_unique(this->dataSize); @@ -132,7 +133,7 @@ void HNSWDiskIndex::restoreVectorsFromFile(std::ifstream &in double elapsed_seconds = std::chrono::duration(end_time - start_time).count(); this->log(VecSimCommonStrings::LOG_VERBOSE_STRING, "Restored %zu processed vectors from metadata file in %f seconds", - this->curElementCount, elapsed_seconds); + elementCount, elapsed_seconds); } /** @@ -157,7 +158,8 @@ template void HNSWDiskIndex::restoreVectorsFromRocksDB(EncodingVersion version) { // Iterate through all elements and restore their processed vectors auto start_time = std::chrono::steady_clock::now(); - for (idType id = 0; id < this->curElementCount; id++) { + size_t elementCount = this->curElementCount.load(std::memory_order_acquire); + for (idType id = 0; id < elementCount; id++) { // Retrieve the raw vector from RocksDB (stored in level-0 graph value) GraphKey graphKey(id, 0); std::string level0_graph_value; @@ -198,7 +200,7 @@ void HNSWDiskIndex::restoreVectorsFromRocksDB(EncodingVersio double elapsed_seconds = std::chrono::duration(end_time - start_time).count(); this->log(VecSimCommonStrings::LOG_VERBOSE_STRING, "Restored %zu processed vectors from RocksDB checkpoint in %f seconds", - this->curElementCount, elapsed_seconds); + this->curElementCount.load(std::memory_order_acquire), elapsed_seconds); } /** @@ -434,9 +436,10 @@ HNSWIndexMetaData HNSWDiskIndex::checkIntegrity() const { // Store all edges for efficient bidirectional checking: (node_id, level) -> set of neighbors std::unordered_map>> all_edges; + size_t elementCount = this->curElementCount.load(std::memory_order_acquire); // First pass: count deleted and max level - for (idType id = 0; id < this->curElementCount; id++) { + for (idType id = 0; id < elementCount; id++) { if (this->isMarkedDeleted(id)) { num_deleted++; } @@ -454,7 +457,7 @@ HNSWIndexMetaData HNSWDiskIndex::checkIntegrity() const { } // Validate entry point - if (this->curElementCount > 0 && this->entrypointNode == INVALID_ID) { + if (elementCount > 0 && this->entrypointNode == INVALID_ID) { this->log(VecSimCommonStrings::LOG_WARNING_STRING, "checkIntegrity failed: no entry point set for non-empty index"); return res; @@ -489,7 +492,7 @@ HNSWIndexMetaData HNSWDiskIndex::checkIntegrity() const { idType neighborId = neighbors[i]; // Check for invalid neighbor - if (neighborId >= this->curElementCount || neighborId == gk->id) { + if (neighborId >= elementCount || neighborId == gk->id) { this->log(VecSimCommonStrings::LOG_WARNING_STRING, "checkIntegrity failed: invalid neighbor %u for node %u at level %zu", neighborId, gk->id, gk->level); @@ -541,7 +544,7 @@ HNSWIndexMetaData HNSWDiskIndex::checkIntegrity() const { // Check for isolated nodes (non-deleted nodes with no neighbors at any level) size_t isolated_count = 0; - for (idType id = 0; id < this->curElementCount; id++) { + for (idType id = 0; id < elementCount; id++) { // Skip deleted nodes if (this->isMarkedDeleted(id)) { continue; @@ -606,7 +609,10 @@ void HNSWDiskIndex::restoreIndexFields(std::ifstream &input) Serializer::readBinaryPOD(input, this->mult); // Restore index state - Serializer::readBinaryPOD(input, this->curElementCount); + // Note: curElementCount is atomic, so we read into a temporary first + size_t tempElementCount; + Serializer::readBinaryPOD(input, tempElementCount); + this->curElementCount.store(tempElementCount, std::memory_order_release); Serializer::readBinaryPOD(input, this->numMarkedDeleted); Serializer::readBinaryPOD(input, this->maxLevel); Serializer::readBinaryPOD(input, this->entrypointNode); @@ -651,10 +657,12 @@ void HNSWDiskIndex::restoreIndexFields(std::ifstream &input) template void HNSWDiskIndex::restoreGraph(std::ifstream &input, EncodingVersion version) { + size_t elementCount = this->curElementCount.load(std::memory_order_acquire); + // Phase 1: Restore metadata for all elements - this->idToMetaData.resize(this->curElementCount); + this->idToMetaData.resize(elementCount); - for (idType id = 0; id < this->curElementCount; id++) { + for (idType id = 0; id < elementCount; id++) { labelType label; size_t topLevel; elementFlags flags; @@ -705,11 +713,13 @@ void HNSWDiskIndex::restoreGraph(std::ifstream &input, this->stagedInsertNeighborUpdates.clear(); // Resize visited nodes handler pool - this->visitedNodesHandlerPool.resize(this->curElementCount); + this->visitedNodesHandlerPool.resize(elementCount); } template void HNSWDiskIndex::saveIndexFields(std::ofstream &output) const { + size_t elementCount = this->curElementCount.load(std::memory_order_acquire); + // Save index type Serializer::writeBinaryPOD(output, VecSimAlgo_HNSWLIB_DISK); @@ -719,7 +729,7 @@ void HNSWDiskIndex::saveIndexFields(std::ofstream &output) c Serializer::writeBinaryPOD(output, this->metric); Serializer::writeBinaryPOD(output, this->blockSize); Serializer::writeBinaryPOD(output, this->isMulti); - Serializer::writeBinaryPOD(output, this->curElementCount); // Use curElementCount as initial capacity + Serializer::writeBinaryPOD(output, elementCount); // Use curElementCount as initial capacity // Save HNSW build parameters Serializer::writeBinaryPOD(output, this->M); @@ -734,7 +744,7 @@ void HNSWDiskIndex::saveIndexFields(std::ofstream &output) c Serializer::writeBinaryPOD(output, this->mult); // Save index state - Serializer::writeBinaryPOD(output, this->curElementCount); + Serializer::writeBinaryPOD(output, elementCount); Serializer::writeBinaryPOD(output, this->numMarkedDeleted); Serializer::writeBinaryPOD(output, this->maxLevel); Serializer::writeBinaryPOD(output, this->entrypointNode); @@ -750,8 +760,10 @@ void HNSWDiskIndex::saveIndexFields(std::ofstream &output) c template void HNSWDiskIndex::saveGraph(std::ofstream &output) const { + size_t elementCount = this->curElementCount.load(std::memory_order_acquire); + // Phase 1: Save metadata for all elements - for (idType id = 0; id < this->curElementCount; id++) { + for (idType id = 0; id < elementCount; id++) { labelType label = this->idToMetaData[id].label; size_t topLevel = this->idToMetaData[id].topLevel; elementFlags flags = this->idToMetaData[id].flags; @@ -801,9 +813,10 @@ void HNSWDiskIndex::saveGraph(std::ofstream &output) const { template void HNSWDiskIndex::saveVectorsToFile(std::ofstream &output) const { auto start_time = std::chrono::steady_clock::now(); + size_t elementCount = this->curElementCount.load(std::memory_order_acquire); // Save all processed vectors - for (idType id = 0; id < this->curElementCount; id++) { + for (idType id = 0; id < elementCount; id++) { // Get the processed vector from the vectors container const void *vector_data = this->vectors->getElement(id); @@ -825,5 +838,5 @@ void HNSWDiskIndex::saveVectorsToFile(std::ofstream &output) double elapsed_seconds = std::chrono::duration(end_time - start_time).count(); this->log(VecSimCommonStrings::LOG_VERBOSE_STRING, "Saved %zu processed vectors to metadata file in %f seconds", - this->curElementCount, elapsed_seconds); + elementCount, elapsed_seconds); } From 1720d541680628715e14c5259f56461f1b0cf343 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Mon, 15 Dec 2025 15:12:35 +0200 Subject: [PATCH 04/34] batch process --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 53 +++++++++++++++---- .../data/scripts/hnsw_disk_serializer.cpp | 49 ++++++++++++++++- tests/utils/mock_thread_pool.h | 5 ++ 3 files changed, 95 insertions(+), 12 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 85d2d0fe6..879b6a31e 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -554,6 +554,8 @@ class HNSWDiskIndex : public VecSimIndexAbstract void setDeleteBatchThreshold(size_t threshold) { deleteBatchThreshold = threshold; } size_t getDeleteBatchThreshold() const { return deleteBatchThreshold; } size_t getPendingDeleteCount() const { return pendingDeleteIds.size(); } + size_t getPendingInsertCount() const { return pendingVectorCount; } + size_t getProcessingBatchCount() const { return processingBatch ? processingBatch->count : 0; } // Job queue configuration (for multi-threaded processing) void setJobQueue(void *jobQueue_, void *jobQueueCtx_, SubmitCB submitCb_) { @@ -2061,10 +2063,28 @@ void HNSWDiskIndex::processBatch() { batchInProgress.store(true); - // Move pending data to processing batch (double-buffering) - processingBatch->vectorIds = std::move(pendingVectorIds); - processingBatch->rawVectors = std::move(rawVectorsInRAM); - processingBatch->count = pendingVectorCount; + // Bound the batch size to prevent large batches when vectors accumulate + // while a previous batch is processing + size_t vectorsToMove = std::min(pendingVectorCount, batchThreshold); + + // Move only up to batchThreshold vectors to processing batch (bounded double-buffering) + processingBatch->vectorIds.assign( + pendingVectorIds.begin(), + pendingVectorIds.begin() + vectorsToMove + ); + processingBatch->count = vectorsToMove; + + // Move corresponding raw vectors to processing batch + { + std::unique_lock lock(rawVectorsGuard); + for (idType id : processingBatch->vectorIds) { + auto it = rawVectorsInRAM.find(id); + if (it != rawVectorsInRAM.end()) { + processingBatch->rawVectors[id] = std::move(it->second); + rawVectorsInRAM.erase(it); + } + } + } // Copy processed vectors to processing batch (prevents race with main thread resizing vectors) { @@ -2078,13 +2098,12 @@ void HNSWDiskIndex::processBatch() { } } - // Clear pending structures for new vectors - pendingVectorIds = vecsim_stl::vector(this->allocator); - { - std::unique_lock lock(rawVectorsGuard); - rawVectorsInRAM.clear(); - } - pendingVectorCount = 0; + // Remove moved vectors from pending structures, keep remaining for next batch + pendingVectorIds.erase( + pendingVectorIds.begin(), + pendingVectorIds.begin() + vectorsToMove + ); + pendingVectorCount -= vectorsToMove; // Clear any previous staged updates stagedInsertUpdates.clear(); @@ -2286,6 +2305,18 @@ void HNSWDiskIndex::executeFlushJob(HNSWDiskFlushJob *job) { // Clean up flush job pendingFlushJob = nullptr; delete job; + + // Check if there are more pending vectors to process + // This ensures we process accumulated vectors in bounded batches + bool shouldTriggerNextBatch = false; + { + std::lock_guard lock(batchSwapGuard); + shouldTriggerNextBatch = (pendingVectorCount >= batchThreshold); + } + + if (shouldTriggerNextBatch) { + processBatch(); + } } template diff --git a/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp b/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp index 4df1a179d..954221f7e 100644 --- a/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp +++ b/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp @@ -393,44 +393,53 @@ int main(int argc, char *argv[]) { mock_thread_pool->init_threads(); // Configure the disk index to use the job queue + // Use a larger batch threshold for bulk loading (default 10 is too small) + const size_t bulk_batch_threshold = 1000; if (type == VecSimType_FLOAT32) { auto *disk_index = dynamic_cast *>(index); if (disk_index) { disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, tieredIndexMock::submit_callback); + disk_index->setBatchThreshold(bulk_batch_threshold); } } else if (type == VecSimType_FLOAT64) { auto *disk_index = dynamic_cast *>(index); if (disk_index) { disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, tieredIndexMock::submit_callback); + disk_index->setBatchThreshold(bulk_batch_threshold); } } else if (type == VecSimType_BFLOAT16) { auto *disk_index = dynamic_cast *>(index); if (disk_index) { disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, tieredIndexMock::submit_callback); + disk_index->setBatchThreshold(bulk_batch_threshold); } } else if (type == VecSimType_FLOAT16) { auto *disk_index = dynamic_cast *>(index); if (disk_index) { disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, tieredIndexMock::submit_callback); + disk_index->setBatchThreshold(bulk_batch_threshold); } } else if (type == VecSimType_INT8) { auto *disk_index = dynamic_cast *>(index); if (disk_index) { disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, tieredIndexMock::submit_callback); + disk_index->setBatchThreshold(bulk_batch_threshold); } } else if (type == VecSimType_UINT8) { auto *disk_index = dynamic_cast *>(index); if (disk_index) { disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, tieredIndexMock::submit_callback); + disk_index->setBatchThreshold(bulk_batch_threshold); } } std::cout << "Multi-threaded indexing enabled with " << num_threads << " threads\n"; + std::cout << "Batch threshold set to " << bulk_batch_threshold << "\n"; } std::cout << "Index created successfully\n"; @@ -459,7 +468,45 @@ int main(int argc, char *argv[]) { // Wait for all background jobs to complete if using multi-threaded indexing if (mock_thread_pool) { std::cout << "Waiting for background indexing jobs to complete...\n"; - mock_thread_pool->thread_pool_wait(); + + // Custom wait loop with progress reporting + auto start_time = std::chrono::steady_clock::now(); + size_t last_indexed = 0; + while (true) { + // Check if queue is empty and all jobs done + if (mock_thread_pool->isIdle()) { + break; + } + + // Print progress every second + size_t current_indexed = VecSimIndex_IndexSize(index); + size_t pending = 0; + size_t processing = 0; + size_t queue_size = mock_thread_pool->jobQ.size(); + + if (type == VecSimType_FLOAT32) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + pending = disk_index->getPendingInsertCount(); + processing = disk_index->getProcessingBatchCount(); + } + } + + auto elapsed = std::chrono::duration_cast( + std::chrono::steady_clock::now() - start_time).count(); + + if (current_indexed != last_indexed || elapsed % 5 == 0) { + std::cout << "\rIndexed: " << current_indexed << "/" << num_vectors + << " | Pending: " << pending + << " | Processing: " << processing + << " | Queue: " << queue_size + << " | Time: " << elapsed << "s " << std::flush; + last_indexed = current_indexed; + } + + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + std::cout << "\n"; // Flush any remaining pending vectors and wait for those jobs too if (type == VecSimType_FLOAT32) { diff --git a/tests/utils/mock_thread_pool.h b/tests/utils/mock_thread_pool.h index 67fc31111..2ca8607f9 100644 --- a/tests/utils/mock_thread_pool.h +++ b/tests/utils/mock_thread_pool.h @@ -128,6 +128,11 @@ class tieredIndexMock { void thread_pool_join(); void thread_pool_wait(size_t waiting_duration = 10); + // Check if all jobs are complete (queue empty and no in-flight jobs) + bool isIdle() const { + return jobQ.empty() && executions_status.AllDone(); + } + // Reconfigure the thread pool to have exactly `new_size` workers. void reconfigure_threads(size_t new_size); }; From 5e1ddc785788da41fdabc9796e26071156f7cdb5 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Mon, 15 Dec 2025 16:37:25 +0200 Subject: [PATCH 05/34] Remove rocksdb lock --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 78 +++++++++----------------- 1 file changed, 25 insertions(+), 53 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 879b6a31e..29029a2c5 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -295,10 +295,6 @@ class HNSWDiskIndex : public VecSimIndexAbstract void *jobQueueCtx; SubmitCB SubmitJobsToQueue; - // Reader-writer lock for RocksDB operations - // Multiple threads can read concurrently, but writes must be exclusive - mutable std::shared_mutex rocksDbGuard; - // Lock for protecting staged updates during merge from parallel insert jobs // Uses shared_mutex for better read concurrency - multiple readers can access simultaneously mutable std::shared_mutex stagedUpdatesGuard; @@ -891,7 +887,7 @@ int HNSWDiskIndex::addVector( // We need to store the original vector before preprocessing const char* raw_data = reinterpret_cast(vector); { - std::unique_lock lock(rawVectorsGuard); + std::lock_guard lock(rawVectorsGuard); rawVectorsInRAM[newElementId] = std::string(raw_data, this->inputBlobSize); } // Preprocess the vector @@ -899,7 +895,7 @@ int HNSWDiskIndex::addVector( // Store the processed vector in memory (protected by vectorsGuard) { - std::unique_lock lock(vectorsGuard); + std::lock_guard lock(vectorsGuard); size_t containerId = this->vectors->size(); this->vectors->addElement(processedBlobs.getStorageBlob(), containerId); } @@ -910,7 +906,7 @@ int HNSWDiskIndex::addVector( // Ensure capacity for the new element ID (protected by indexDataGuard) { - std::unique_lock lock(indexDataGuard); + std::lock_guard lock(indexDataGuard); if (newElementId >= indexCapacity()) { size_t new_cap = ((newElementId + this->blockSize) / this->blockSize) * this->blockSize; visitedNodesHandlerPool.resize(new_cap); @@ -1032,12 +1028,8 @@ idType HNSWDiskIndex::mutuallyConnectNewElement( // Read the neighbor's current neighbor count from disk to check capacity GraphKey neighborKey(selected_neighbor, level); std::string existing_neighbors_data; - rocksdb::Status status; - { - std::shared_lock lock(rocksDbGuard); - status = - db->Get(rocksdb::ReadOptions(), cf, neighborKey.asSlice(), &existing_neighbors_data); - } + rocksdb::Status status = + db->Get(rocksdb::ReadOptions(), cf, neighborKey.asSlice(), &existing_neighbors_data); size_t current_neighbor_count = 0; if (status.ok()) { @@ -1182,11 +1174,8 @@ void HNSWDiskIndex::stageRevisitNeighborConnections(idType n // TODO: perhaps cache the neigbhors for stage update GraphKey neighborKey(selected_neighbor, level); std::string graph_value; - rocksdb::Status status; - { - std::shared_lock lock(rocksDbGuard); - status = db->Get(rocksdb::ReadOptions(), cf, neighborKey.asSlice(), &graph_value); - } + rocksdb::Status status = + db->Get(rocksdb::ReadOptions(), cf, neighborKey.asSlice(), &graph_value); if (!status.ok()) { this->log(VecSimCommonStrings::LOG_WARNING_STRING, @@ -1391,14 +1380,11 @@ bool HNSWDiskIndex::getRawVector(idType id, void* output_buf } } - // If not in RAM or cache, retrieve from disk with shared lock for thread safety + // If not in RAM or cache, retrieve from disk (RocksDB is thread-safe) GraphKey graphKey(id, 0); std::string level0_graph_value; - rocksdb::Status status; - { - std::shared_lock lock(rocksDbGuard); - status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &level0_graph_value); - } + rocksdb::Status status = + db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &level0_graph_value); if (status.ok()) { // Extract vector data @@ -1582,14 +1568,11 @@ void HNSWDiskIndex::greedySearchLevel(const void *data_point do { changed = false; - // Read neighbors from RocksDB for the current node at this level with shared lock + // Read neighbors from RocksDB for the current node at this level (RocksDB is thread-safe) GraphKey graphKey(bestCand, level); std::string graph_value; - rocksdb::Status status; - { - std::shared_lock lock(rocksDbGuard); - status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); - } + rocksdb::Status status = + db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); if (!status.ok()) { // No neighbors found for this node at this level, stop searching @@ -1866,11 +1849,8 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level GraphKey graphKey(nodeId, level); std::string graph_value; - rocksdb::Status status; - { - std::shared_lock lock(rocksDbGuard); - status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); - } + rocksdb::Status status = + db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); if (status.ok()) { deserializeGraphValue(graph_value, result); @@ -1879,7 +1859,7 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level // Lazy repair: if we filtered any deleted nodes, stage for cleanup // Use hash map for O(1) duplicate detection // Note: stagedRepairMap and stagedRepairUpdates are protected by stagedUpdatesGuard - std::unique_lock repairLock(stagedUpdatesGuard); + std::lock_guard repairLock(stagedUpdatesGuard); uint64_t repair_key = makeRepairKey(nodeId, level); if (stagedRepairMap.find(repair_key) == stagedRepairMap.end()) { stagedRepairMap[repair_key] = stagedRepairUpdates.size(); @@ -1925,11 +1905,8 @@ void HNSWDiskIndex::getNeighborsAndVector(idType nodeId, siz GraphKey graphKey(nodeId, level); std::string graph_value; - rocksdb::Status status; - { - std::shared_lock lock(rocksDbGuard); - status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); - } + rocksdb::Status status = + db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); if (status.ok()) { deserializeGraphValue(graph_value, result); @@ -2076,7 +2053,7 @@ void HNSWDiskIndex::processBatch() { // Move corresponding raw vectors to processing batch { - std::unique_lock lock(rawVectorsGuard); + std::lock_guard lock(rawVectorsGuard); for (idType id : processingBatch->vectorIds) { auto it = rawVectorsInRAM.find(id); if (it != rawVectorsInRAM.end()) { @@ -2114,7 +2091,7 @@ void HNSWDiskIndex::processBatch() { idType currentEntryPoint; size_t currentMaxLevel; { - std::unique_lock lock(indexDataGuard); + std::lock_guard lock(indexDataGuard); currentEntryPoint = entrypointNode; currentMaxLevel = maxLevel; @@ -2280,22 +2257,17 @@ void HNSWDiskIndex::executeInsertJob(HNSWDiskInsertJob *job) template void HNSWDiskIndex::executeFlushJob(HNSWDiskFlushJob *job) { - // Acquire exclusive lock for RocksDB writes - std::unique_lock lock(rocksDbGuard); - - // Flush all staged updates to RocksDB + // Flush all staged updates to RocksDB (RocksDB is thread-safe for writes) flushStagedGraphUpdates(stagedInsertUpdates, stagedInsertNeighborUpdates); // Clear staging - must hold stagedUpdatesGuard to prevent race with getNeighbors { - std::unique_lock stagingLock(stagedUpdatesGuard); + std::lock_guard stagingLock(stagedUpdatesGuard); stagedInsertMap.clear(); stagedInsertUpdates.clear(); stagedInsertNeighborUpdates.clear(); } - lock.unlock(); - // Clear processing batch (raw vectors now persisted to disk) processingBatch->clear(); @@ -2323,7 +2295,7 @@ template void HNSWDiskIndex::mergeLocalStagedUpdates( vecsim_stl::vector &localGraphUpdates, vecsim_stl::vector &localNeighborUpdates) { - std::unique_lock lock(stagedUpdatesGuard); + std::lock_guard lock(stagedUpdatesGuard); // Merge graph updates for (auto &update : localGraphUpdates) { @@ -2504,7 +2476,7 @@ void HNSWDiskIndex::processDeleteBatch() { // Remove raw vector from RAM if it exists - requires exclusive lock { - std::unique_lock lock(rawVectorsGuard); + std::lock_guard lock(rawVectorsGuard); auto ram_it = rawVectorsInRAM.find(deleted_id); if (ram_it != rawVectorsInRAM.end()) { rawVectorsInRAM.erase(ram_it); @@ -2957,7 +2929,7 @@ vecsim_stl::vector HNSWDiskIndex::markDelete(labelTy vecsim_stl::vector internal_ids(this->allocator); // Protect all accesses to labelToIdMap, idToMetaData, entrypointNode, maxLevel - std::unique_lock lock(indexDataGuard); + std::lock_guard lock(indexDataGuard); // Find the internal ID for this label auto it = labelToIdMap.find(label); From 9603aeb910baabbb7752591b50ca8bdcd125a439 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Tue, 16 Dec 2025 10:29:24 +0200 Subject: [PATCH 06/34] entry point --- .../algorithms/hnsw/hnsw_disk_serializer.h | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h index 91e4f5e0e..8fd522be9 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h @@ -457,7 +457,8 @@ HNSWIndexMetaData HNSWDiskIndex::checkIntegrity() const { } // Validate entry point - if (elementCount > 0 && this->entrypointNode == INVALID_ID) { + EntryPointState state = this->entryPointState.load(std::memory_order_acquire); + if (elementCount > 0 && state.entrypointNode == INVALID_ID) { this->log(VecSimCommonStrings::LOG_WARNING_STRING, "checkIntegrity failed: no entry point set for non-empty index"); return res; @@ -614,8 +615,13 @@ void HNSWDiskIndex::restoreIndexFields(std::ifstream &input) Serializer::readBinaryPOD(input, tempElementCount); this->curElementCount.store(tempElementCount, std::memory_order_release); Serializer::readBinaryPOD(input, this->numMarkedDeleted); - Serializer::readBinaryPOD(input, this->maxLevel); - Serializer::readBinaryPOD(input, this->entrypointNode); + // Read entry point state (maxLevel and entrypointNode are now atomic) + size_t tempMaxLevel; + idType tempEntrypointNode; + Serializer::readBinaryPOD(input, tempMaxLevel); + Serializer::readBinaryPOD(input, tempEntrypointNode); + this->entryPointState.store(EntryPointState(tempEntrypointNode, tempMaxLevel), + std::memory_order_release); // Restore batch processing configuration Serializer::readBinaryPOD(input, this->batchThreshold); @@ -746,8 +752,11 @@ void HNSWDiskIndex::saveIndexFields(std::ofstream &output) c // Save index state Serializer::writeBinaryPOD(output, elementCount); Serializer::writeBinaryPOD(output, this->numMarkedDeleted); - Serializer::writeBinaryPOD(output, this->maxLevel); - Serializer::writeBinaryPOD(output, this->entrypointNode); + // Write entry point state (maxLevel and entrypointNode are now atomic) + EntryPointState state = this->entryPointState.load(std::memory_order_acquire); + size_t maxLevelToSave = static_cast(state.maxLevel); + Serializer::writeBinaryPOD(output, maxLevelToSave); + Serializer::writeBinaryPOD(output, state.entrypointNode); // Save batch processing configuration Serializer::writeBinaryPOD(output, this->batchThreshold); From e5747dbac2b3b0e81e3adef9ac14d529048a396a Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Tue, 16 Dec 2025 10:30:11 +0200 Subject: [PATCH 07/34] entrypoint declare --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 29029a2c5..a433d5427 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -197,8 +197,21 @@ class HNSWDiskIndex : public VecSimIndexAbstract // multithreaded scenario. std::atomic curElementCount; size_t numMarkedDeleted; - idType entrypointNode; - size_t maxLevel; // this is the top level of the entry point's element + + // Packed entry point state for lock-free atomic access + // Layout: [entrypointNode (32 bits) | maxLevel (32 bits)] + // This allows atomic read/write of both values together without locking + static constexpr uint32_t INVALID_MAX_LEVEL = UINT32_MAX; + struct alignas(8) EntryPointState { + idType entrypointNode; + uint32_t maxLevel; + + EntryPointState() : entrypointNode(INVALID_ID), maxLevel(INVALID_MAX_LEVEL) {} + EntryPointState(idType ep, size_t level) + : entrypointNode(ep), maxLevel(static_cast(level)) {} + }; + static_assert(sizeof(EntryPointState) == 8, "EntryPointState must be 8 bytes for atomic ops"); + std::atomic entryPointState; // Index data // vecsim_stl::vector graphDataBlocks; From fa6e5f3770e71e341e3f041a39b09ab9598490ec Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Tue, 16 Dec 2025 10:32:57 +0200 Subject: [PATCH 08/34] really add it --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 86 ++++++++++++++------------ 1 file changed, 48 insertions(+), 38 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index a433d5427..13a881910 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -653,9 +653,8 @@ HNSWDiskIndex::HNSWDiskIndex( curElementCount = 0; numMarkedDeleted = 0; - // initializations for special treatment of the first node - entrypointNode = INVALID_ID; - maxLevel = HNSW_INVALID_LEVEL; + // Initialize entry point state atomically (default constructor sets INVALID_ID, INVALID_MAX_LEVEL) + entryPointState.store(EntryPointState(), std::memory_order_relaxed); if (M <= 1) throw std::runtime_error("HNSW index parameter M cannot be 1"); @@ -781,8 +780,9 @@ auto HNSWDiskIndex::getNeighborhoods(const vecsim_stl::vecto // Create a vector of slices to store the keys std::vector graphKeys; std::vector keys; - graphKeys.reserve(ids.size() * maxLevel); - keys.reserve(ids.size() * maxLevel); + size_t currentMaxLevel = entryPointState.load(std::memory_order_acquire).maxLevel; + graphKeys.reserve(ids.size() * currentMaxLevel); + keys.reserve(ids.size() * currentMaxLevel); // Iterate over the ids and create the keys for (const auto &id : ids) { const size_t curMaxLevel = idToMetaData[id].topLevel; @@ -1565,8 +1565,9 @@ bool HNSWDiskIndex::isMarkedDeleted(idType id) const { template std::pair HNSWDiskIndex::safeGetEntryPointState() const { - std::shared_lock lock(indexDataGuard); - return std::make_pair(entrypointNode, maxLevel); + // Lock-free atomic read of entry point state + EntryPointState state = entryPointState.load(std::memory_order_acquire); + return std::make_pair(state.entrypointNode, static_cast(state.maxLevel)); } template @@ -2001,6 +2002,8 @@ void HNSWDiskIndex:: singleThreadProcessBatch(){ stagedInsertNeighborUpdates.clear(); // Process each pending vector ID (vectors are already stored in memory) + // Load entry point state once at the start + EntryPointState currentState = entryPointState.load(std::memory_order_acquire); for (size_t i = 0; i < pendingVectorCount; i++) { idType vectorId = pendingVectorIds[i]; if (isMarkedDeleted(vectorId)) { @@ -2012,12 +2015,12 @@ void HNSWDiskIndex:: singleThreadProcessBatch(){ DiskElementMetaData &metadata = idToMetaData[vectorId]; size_t elementMaxLevel = metadata.topLevel; - if (entrypointNode != INVALID_ID) { - insertElementToGraph(vectorId, elementMaxLevel, entrypointNode, maxLevel, - raw_vector_data, vector_data); + if (currentState.entrypointNode != INVALID_ID) { + insertElementToGraph(vectorId, elementMaxLevel, currentState.entrypointNode, + currentState.maxLevel, raw_vector_data, vector_data); } else { - entrypointNode = vectorId; - maxLevel = elementMaxLevel; + currentState = EntryPointState(vectorId, elementMaxLevel); + entryPointState.store(currentState, std::memory_order_release); } } @@ -2100,15 +2103,15 @@ void HNSWDiskIndex::processBatch() { stagedInsertMap.clear(); stagedInsertNeighborUpdates.clear(); - // Snapshot and update entry point state - protected by indexDataGuard - idType currentEntryPoint; - size_t currentMaxLevel; - { - std::lock_guard lock(indexDataGuard); - currentEntryPoint = entrypointNode; - currentMaxLevel = maxLevel; + // Snapshot and update entry point state - lock-free with atomic operations + EntryPointState currentState = entryPointState.load(std::memory_order_acquire); + idType currentEntryPoint = currentState.entrypointNode; + size_t currentMaxLevel = currentState.maxLevel; - // First pass: Update entry point for any vectors with higher levels + // First pass: Update entry point for any vectors with higher levels + // Note: idToMetaData access still needs indexDataGuard for resize protection + { + std::shared_lock lock(indexDataGuard); for (size_t i = 0; i < processingBatch->count; i++) { idType vectorId = processingBatch->vectorIds[i]; // Use direct flag check to avoid recursive lock (isMarkedDeleted takes shared lock) @@ -2121,10 +2124,11 @@ void HNSWDiskIndex::processBatch() { currentMaxLevel = metadata.topLevel; } } + } - // Update global entry point - entrypointNode = currentEntryPoint; - maxLevel = currentMaxLevel; + // Update global entry point atomically if changed + if (currentEntryPoint != currentState.entrypointNode || currentMaxLevel != currentState.maxLevel) { + entryPointState.store(EntryPointState(currentEntryPoint, currentMaxLevel), std::memory_order_release); } // Count valid vectors and create jobs @@ -2540,10 +2544,11 @@ void HNSWDiskIndex::setBatchThreshold(size_t threshold) { template void HNSWDiskIndex::debugPrintGraphStructure() const { size_t elementCount = curElementCount.load(std::memory_order_acquire); + EntryPointState state = entryPointState.load(std::memory_order_acquire); this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "=== HNSW Disk Index Graph Structure ==="); this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Total elements: %zu", elementCount); - this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Entry point: %u", entrypointNode); - this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Max level: %zu", maxLevel); + this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Entry point: %u", state.entrypointNode); + this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Max level: %u", state.maxLevel); this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "M: %zu, M0: %zu", M, M0); // Count total edges @@ -2687,9 +2692,10 @@ void HNSWDiskIndex::debugValidateGraphConnectivity() const { } // Check if entry point exists and has neighbors - if (entrypointNode != INVALID_ID) { - this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Entry point %u exists", entrypointNode); - debugPrintNodeNeighbors(entrypointNode); + EntryPointState state = entryPointState.load(std::memory_order_acquire); + if (state.entrypointNode != INVALID_ID) { + this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Entry point %u exists", state.entrypointNode); + debugPrintNodeNeighbors(state.entrypointNode); } else { this->log(VecSimCommonStrings::LOG_WARNING_STRING, "WARNING: No entry point set!"); } @@ -2980,15 +2986,17 @@ vecsim_stl::vector HNSWDiskIndex::markDelete(labelTy template void HNSWDiskIndex::replaceEntryPoint() { - // PRECONDITION: Caller must hold indexDataGuard (exclusive lock) + // PRECONDITION: Caller must hold indexDataGuard (for idToMetaData access) // This method is called when the current entrypoint is marked as deleted // We need to find a new entrypoint from the remaining non-deleted nodes - idType old_entry_point_id = entrypointNode; + EntryPointState currentState = entryPointState.load(std::memory_order_acquire); + idType old_entry_point_id = currentState.entrypointNode; + size_t currentMaxLevel = currentState.maxLevel; // Try to find a new entrypoint at the current max level - while (maxLevel != HNSW_INVALID_LEVEL) { + while (currentMaxLevel != INVALID_MAX_LEVEL) { // First, try to find a neighbor of the old entrypoint at the top level - GraphKey graphKey(old_entry_point_id, maxLevel); + GraphKey graphKey(old_entry_point_id, currentMaxLevel); std::string graph_value; rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); @@ -3002,7 +3010,8 @@ void HNSWDiskIndex::replaceEntryPoint() { // Use direct flag check to avoid recursive lock (caller holds indexDataGuard) for (size_t i = 0; i < neighbors.size(); i++) { if (!(__atomic_load_n(&idToMetaData[neighbors[i]].flags, 0) & DELETE_MARK)) { - entrypointNode = neighbors[i]; + entryPointState.store(EntryPointState(neighbors[i], currentMaxLevel), + std::memory_order_release); return; } } @@ -3013,20 +3022,21 @@ void HNSWDiskIndex::replaceEntryPoint() { size_t elementCount = curElementCount.load(std::memory_order_acquire); for (idType id = 0; id < elementCount; id++) { if (id != old_entry_point_id && id < idToMetaData.size() && - idToMetaData[id].label != INVALID_LABEL && idToMetaData[id].topLevel == maxLevel && + idToMetaData[id].label != INVALID_LABEL && + idToMetaData[id].topLevel == currentMaxLevel && !(__atomic_load_n(&idToMetaData[id].flags, 0) & DELETE_MARK)) { - entrypointNode = id; + entryPointState.store(EntryPointState(id, currentMaxLevel), + std::memory_order_release); return; } } // No non-deleted nodes at this level, decrease maxLevel and try again - maxLevel--; + currentMaxLevel--; } // If we get here, the index is empty or all nodes are deleted - entrypointNode = INVALID_ID; - maxLevel = HNSW_INVALID_LEVEL; + entryPointState.store(EntryPointState(), std::memory_order_release); } #ifdef BUILD_TESTS From e4ff62f60c6c233f37726523ae0b03948f070a14 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Tue, 16 Dec 2025 10:35:05 +0200 Subject: [PATCH 09/34] decalre for real --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 13a881910..972cd055f 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -2971,8 +2971,8 @@ vecsim_stl::vector HNSWDiskIndex::markDelete(labelTy this->numMarkedDeleted++; // If this is the entrypoint, we need to replace it - // Note: replaceEntryPoint is called while holding indexDataGuard - if (internalId == entrypointNode) { + EntryPointState currentState = entryPointState.load(std::memory_order_acquire); + if (internalId == currentState.entrypointNode) { replaceEntryPoint(); } From ca07b4a2ffff32058300d1b796becb1da13f1d70 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Wed, 17 Dec 2025 12:22:25 +0200 Subject: [PATCH 10/34] fix --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 390 ++++++++++++------------- 1 file changed, 186 insertions(+), 204 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 972cd055f..d39f41fe9 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -198,20 +198,8 @@ class HNSWDiskIndex : public VecSimIndexAbstract std::atomic curElementCount; size_t numMarkedDeleted; - // Packed entry point state for lock-free atomic access - // Layout: [entrypointNode (32 bits) | maxLevel (32 bits)] - // This allows atomic read/write of both values together without locking - static constexpr uint32_t INVALID_MAX_LEVEL = UINT32_MAX; - struct alignas(8) EntryPointState { - idType entrypointNode; - uint32_t maxLevel; - - EntryPointState() : entrypointNode(INVALID_ID), maxLevel(INVALID_MAX_LEVEL) {} - EntryPointState(idType ep, size_t level) - : entrypointNode(ep), maxLevel(static_cast(level)) {} - }; - static_assert(sizeof(EntryPointState) == 8, "EntryPointState must be 8 bytes for atomic ops"); - std::atomic entryPointState; + idType entrypointNode; + size_t maxLevel; // this is the top level of the entry point's element // Index data // vecsim_stl::vector graphDataBlocks; @@ -304,9 +292,9 @@ class HNSWDiskIndex : public VecSimIndexAbstract /********************************** Multi-threading Support **********************************/ // Job queue parameters (similar to tiered index) - void *jobQueue; - void *jobQueueCtx; - SubmitCB SubmitJobsToQueue; + void *jobQueue = nullptr; + void *jobQueueCtx = nullptr; + SubmitCB SubmitJobsToQueue = nullptr; // Lock for protecting staged updates during merge from parallel insert jobs // Uses shared_mutex for better read concurrency - multiple readers can access simultaneously @@ -652,9 +640,8 @@ HNSWDiskIndex::HNSWDiskIndex( dbOptions = db->GetOptions(); curElementCount = 0; numMarkedDeleted = 0; - - // Initialize entry point state atomically (default constructor sets INVALID_ID, INVALID_MAX_LEVEL) - entryPointState.store(EntryPointState(), std::memory_order_relaxed); + entrypointNode = INVALID_ID; + maxLevel = 0; if (M <= 1) throw std::runtime_error("HNSW index parameter M cannot be 1"); @@ -780,9 +767,8 @@ auto HNSWDiskIndex::getNeighborhoods(const vecsim_stl::vecto // Create a vector of slices to store the keys std::vector graphKeys; std::vector keys; - size_t currentMaxLevel = entryPointState.load(std::memory_order_acquire).maxLevel; - graphKeys.reserve(ids.size() * currentMaxLevel); - keys.reserve(ids.size() * currentMaxLevel); + graphKeys.reserve(ids.size() * maxLevel); + keys.reserve(ids.size() * maxLevel); // Iterate over the ids and create the keys for (const auto &id : ids) { const size_t curMaxLevel = idToMetaData[id].topLevel; @@ -1039,15 +1025,10 @@ idType HNSWDiskIndex::mutuallyConnectNewElement( // neighbor's connections // Read the neighbor's current neighbor count from disk to check capacity - GraphKey neighborKey(selected_neighbor, level); - std::string existing_neighbors_data; - rocksdb::Status status = - db->Get(rocksdb::ReadOptions(), cf, neighborKey.asSlice(), &existing_neighbors_data); - - size_t current_neighbor_count = 0; - if (status.ok()) { - current_neighbor_count = existing_neighbors_data.size() / sizeof(idType); - } + // First check staged updates, then fall back to disk + vecsim_stl::vector existing_neighbors(this->allocator); + getNeighbors(selected_neighbor, level, existing_neighbors); + size_t current_neighbor_count = existing_neighbors.size(); if (current_neighbor_count < max_M_cur) { // Neighbor has capacity, just add the new node @@ -1183,26 +1164,9 @@ void HNSWDiskIndex::stageRevisitNeighborConnections(idType n idType selected_neighbor, size_t level, DistType distance) { - // Read the neighbor's current neighbor list from disk - // TODO: perhaps cache the neigbhors for stage update - GraphKey neighborKey(selected_neighbor, level); - std::string graph_value; - rocksdb::Status status = - db->Get(rocksdb::ReadOptions(), cf, neighborKey.asSlice(), &graph_value); - - if (!status.ok()) { - this->log(VecSimCommonStrings::LOG_WARNING_STRING, - " WARNING: Could not read existing neighbors for node %u at level %zu", - selected_neighbor, level); - // Fall back to simple neighbor update - std::unique_lock lock(stagedUpdatesGuard); - stagedInsertNeighborUpdates.emplace_back(selected_neighbor, level, new_node_id); - return; - } - - // Parse existing neighbors using new format + // Read the neighbor's current neighbor list (checks staged updates first, then disk) vecsim_stl::vector existing_neighbors(this->allocator); - deserializeGraphValue(graph_value, existing_neighbors); + getNeighbors(selected_neighbor, level, existing_neighbors); // Collect all candidates: existing neighbors + new node candidatesList candidates(this->allocator); @@ -1565,9 +1529,7 @@ bool HNSWDiskIndex::isMarkedDeleted(idType id) const { template std::pair HNSWDiskIndex::safeGetEntryPointState() const { - // Lock-free atomic read of entry point state - EntryPointState state = entryPointState.load(std::memory_order_acquire); - return std::make_pair(state.entrypointNode, static_cast(state.maxLevel)); + return std::make_pair(entrypointNode, maxLevel); } template @@ -1582,21 +1544,15 @@ void HNSWDiskIndex::greedySearchLevel(const void *data_point do { changed = false; - // Read neighbors from RocksDB for the current node at this level (RocksDB is thread-safe) - GraphKey graphKey(bestCand, level); - std::string graph_value; - rocksdb::Status status = - db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + // Read neighbors (checks staged updates first, then RocksDB) + vecsim_stl::vector neighbors(this->allocator); + getNeighbors(bestCand, level, neighbors); - if (!status.ok()) { + if (neighbors.empty()) { // No neighbors found for this node at this level, stop searching break; } - // Parse the neighbors using new format - vecsim_stl::vector neighbors(this->allocator); - deserializeGraphValue(graph_value, neighbors); - // Check each neighbor to find a better candidate for (size_t i = 0; i < neighbors.size(); i++) { idType candidate = neighbors[i]; @@ -1810,12 +1766,16 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level // First check staged graph updates using hash maps for O(1) lookup uint64_t lookup_key = makeRepairKey(nodeId, level); + bool foundInStaged = false; + + // Collect incremental neighbor updates for this node/level + vecsim_stl::vector incrementalNeighbors(this->allocator); // Check staged updates under lock - use shared_lock for read access { std::shared_lock lock(stagedUpdatesGuard); - // Check insert staging area + // Check insert staging area (complete neighbor list replacements) auto insert_it = stagedInsertMap.find(lookup_key); if (insert_it != stagedInsertMap.end()) { const auto &update = stagedInsertUpdates[insert_it->second]; @@ -1823,64 +1783,80 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level for (size_t i = 0; i < update.neighbors.size(); i++) { result.push_back(update.neighbors[i]); } - // Release lock before calling filterDeletedNodes which may take other locks - lock.unlock(); - filterDeletedNodes(result); - return; + foundInStaged = true; } // Check delete staging area - auto delete_it = stagedDeleteMap.find(lookup_key); - if (delete_it != stagedDeleteMap.end()) { - const auto &update = stagedDeleteUpdates[delete_it->second]; - result.reserve(update.neighbors.size()); - for (size_t i = 0; i < update.neighbors.size(); i++) { - result.push_back(update.neighbors[i]); + if (!foundInStaged) { + auto delete_it = stagedDeleteMap.find(lookup_key); + if (delete_it != stagedDeleteMap.end()) { + const auto &update = stagedDeleteUpdates[delete_it->second]; + result.reserve(update.neighbors.size()); + for (size_t i = 0; i < update.neighbors.size(); i++) { + result.push_back(update.neighbors[i]); + } + foundInStaged = true; } - // Release lock before calling filterDeletedNodes - lock.unlock(); - filterDeletedNodes(result); - return; } // Also check staged repair updates (already cleaned neighbors waiting to be flushed) - auto repair_it = stagedRepairMap.find(lookup_key); - if (repair_it != stagedRepairMap.end()) { - const auto &update = stagedRepairUpdates[repair_it->second]; - result.reserve(update.neighbors.size()); - for (size_t i = 0; i < update.neighbors.size(); i++) { - result.push_back(update.neighbors[i]); + if (!foundInStaged) { + auto repair_it = stagedRepairMap.find(lookup_key); + if (repair_it != stagedRepairMap.end()) { + const auto &update = stagedRepairUpdates[repair_it->second]; + result.reserve(update.neighbors.size()); + for (size_t i = 0; i < update.neighbors.size(); i++) { + result.push_back(update.neighbors[i]); + } + foundInStaged = true; + } + } + + // Collect incremental neighbor updates (these are additions, not replacements) + // These need to be applied on top of the base neighbor list + for (const auto &update : stagedInsertNeighborUpdates) { + if (update.node_id == nodeId && update.level == level) { + incrementalNeighbors.push_back(update.new_neighbor_id); } - // Note: We can't update the repair entry here since we have a shared lock - // Just filter and return; the repair will be re-evaluated if needed - lock.unlock(); - filterDeletedNodes(result); - return; } } - // If not found in staged updates, check disk with shared lock for thread safety - GraphKey graphKey(nodeId, level); + // If not found in staged updates, check disk + if (!foundInStaged) { + GraphKey graphKey(nodeId, level); - std::string graph_value; - rocksdb::Status status = - db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + std::string graph_value; + rocksdb::Status status = + db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); - if (status.ok()) { - deserializeGraphValue(graph_value, result); - // Filter out deleted nodes and check if any were filtered - if (filterDeletedNodes(result)) { - // Lazy repair: if we filtered any deleted nodes, stage for cleanup - // Use hash map for O(1) duplicate detection - // Note: stagedRepairMap and stagedRepairUpdates are protected by stagedUpdatesGuard - std::lock_guard repairLock(stagedUpdatesGuard); - uint64_t repair_key = makeRepairKey(nodeId, level); - if (stagedRepairMap.find(repair_key) == stagedRepairMap.end()) { - stagedRepairMap[repair_key] = stagedRepairUpdates.size(); - stagedRepairUpdates.emplace_back(nodeId, level, result, this->allocator); + if (status.ok()) { + deserializeGraphValue(graph_value, result); + } + } + + // Apply incremental neighbor updates (add new neighbors that aren't already in the list) + if (!incrementalNeighbors.empty()) { + std::unordered_set existingNeighbors(result.begin(), result.end()); + for (idType neighbor : incrementalNeighbors) { + if (existingNeighbors.find(neighbor) == existingNeighbors.end()) { + result.push_back(neighbor); + existingNeighbors.insert(neighbor); } } } + + // Filter out deleted nodes and check if any were filtered + if (filterDeletedNodes(result) && !foundInStaged) { + // Lazy repair: if we filtered any deleted nodes from disk data, stage for cleanup + // Use hash map for O(1) duplicate detection + // Note: stagedRepairMap and stagedRepairUpdates are protected by stagedUpdatesGuard + std::lock_guard repairLock(stagedUpdatesGuard); + uint64_t repair_key = makeRepairKey(nodeId, level); + if (stagedRepairMap.find(repair_key) == stagedRepairMap.end()) { + stagedRepairMap[repair_key] = stagedRepairUpdates.size(); + stagedRepairUpdates.emplace_back(nodeId, level, result, this->allocator); + } + } } template @@ -1890,6 +1866,8 @@ void HNSWDiskIndex::getNeighborsAndVector(idType nodeId, siz // First check staged graph updates (protected by stagedUpdatesGuard) // Use shared_lock for read-only access and hash map for O(1) lookup + bool foundNeighborsInStaged = false; + vecsim_stl::vector incrementalNeighbors(this->allocator); { std::shared_lock lock(stagedUpdatesGuard); uint64_t lookup_key = makeRepairKey(nodeId, level); @@ -1900,19 +1878,35 @@ void HNSWDiskIndex::getNeighborsAndVector(idType nodeId, siz for (size_t i = 0; i < update.neighbors.size(); i++) { result.push_back(update.neighbors[i]); } + foundNeighborsInStaged = true; + } + + // Collect incremental neighbor updates + for (const auto &update : stagedInsertNeighborUpdates) { + if (update.node_id == nodeId && update.level == level) { + incrementalNeighbors.push_back(update.new_neighbor_id); + } } } // Check rawVectorsInRAM with shared lock - bool foundInRAM = false; + bool foundVectorInRAM = false; { std::shared_lock lock(rawVectorsGuard); auto it = rawVectorsInRAM.find(nodeId); if (it != rawVectorsInRAM.end()) { std::memcpy(vector_data, it->second.data(), this->inputBlobSize); - foundInRAM = true; + foundVectorInRAM = true; } } - if (!result.empty() && foundInRAM) { + // Check processingBatch if not found in RAM + if (!foundVectorInRAM && processingBatch) { + auto it = processingBatch->rawVectors.find(nodeId); + if (it != processingBatch->rawVectors.end()) { + std::memcpy(vector_data, it->second.data(), this->inputBlobSize); + foundVectorInRAM = true; + } + } + if (foundNeighborsInStaged && foundVectorInRAM && incrementalNeighbors.empty()) { return; } // If not found in staged updates, check disk with shared lock for thread safety @@ -1923,8 +1917,25 @@ void HNSWDiskIndex::getNeighborsAndVector(idType nodeId, siz db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); if (status.ok()) { - deserializeGraphValue(graph_value, result); - std::memcpy(vector_data, graph_value.data(), this->inputBlobSize); + // Only update neighbors if we didn't find them in staged updates + if (!foundNeighborsInStaged) { + deserializeGraphValue(graph_value, result); + } + // Only update vector if we didn't find it in RAM + if (!foundVectorInRAM) { + std::memcpy(vector_data, graph_value.data(), this->inputBlobSize); + } + } + + // Apply incremental neighbor updates + if (!incrementalNeighbors.empty()) { + std::unordered_set existingNeighbors(result.begin(), result.end()); + for (idType neighbor : incrementalNeighbors) { + if (existingNeighbors.find(neighbor) == existingNeighbors.end()) { + result.push_back(neighbor); + existingNeighbors.insert(neighbor); + } + } } } @@ -2002,8 +2013,6 @@ void HNSWDiskIndex:: singleThreadProcessBatch(){ stagedInsertNeighborUpdates.clear(); // Process each pending vector ID (vectors are already stored in memory) - // Load entry point state once at the start - EntryPointState currentState = entryPointState.load(std::memory_order_acquire); for (size_t i = 0; i < pendingVectorCount; i++) { idType vectorId = pendingVectorIds[i]; if (isMarkedDeleted(vectorId)) { @@ -2015,12 +2024,12 @@ void HNSWDiskIndex:: singleThreadProcessBatch(){ DiskElementMetaData &metadata = idToMetaData[vectorId]; size_t elementMaxLevel = metadata.topLevel; - if (currentState.entrypointNode != INVALID_ID) { - insertElementToGraph(vectorId, elementMaxLevel, currentState.entrypointNode, - currentState.maxLevel, raw_vector_data, vector_data); + if (entrypointNode != INVALID_ID) { + insertElementToGraph(vectorId, elementMaxLevel, entrypointNode, + maxLevel, raw_vector_data, vector_data); } else { - currentState = EntryPointState(vectorId, elementMaxLevel); - entryPointState.store(currentState, std::memory_order_release); + entrypointNode = vectorId; + maxLevel = elementMaxLevel; } } @@ -2049,7 +2058,7 @@ void HNSWDiskIndex::processBatch() { } // Check if previous batch is still processing - if (batchInProgress.load()) { + if (batchInProgress.load()) { // Previous batch still running, return - caller can retry later return; } @@ -2103,10 +2112,9 @@ void HNSWDiskIndex::processBatch() { stagedInsertMap.clear(); stagedInsertNeighborUpdates.clear(); - // Snapshot and update entry point state - lock-free with atomic operations - EntryPointState currentState = entryPointState.load(std::memory_order_acquire); - idType currentEntryPoint = currentState.entrypointNode; - size_t currentMaxLevel = currentState.maxLevel; + // Snapshot entry point state + idType currentEntryPoint = entrypointNode; + size_t currentMaxLevel = maxLevel; // First pass: Update entry point for any vectors with higher levels // Note: idToMetaData access still needs indexDataGuard for resize protection @@ -2126,9 +2134,10 @@ void HNSWDiskIndex::processBatch() { } } - // Update global entry point atomically if changed - if (currentEntryPoint != currentState.entrypointNode || currentMaxLevel != currentState.maxLevel) { - entryPointState.store(EntryPointState(currentEntryPoint, currentMaxLevel), std::memory_order_release); + // Update global entry point if changed + if (currentEntryPoint != entrypointNode || currentMaxLevel != maxLevel) { + entrypointNode = currentEntryPoint; + maxLevel = currentMaxLevel; } // Count valid vectors and create jobs @@ -2223,46 +2232,32 @@ void HNSWDiskIndex::executeInsertJob(HNSWDiskInsertJob *job) } // Thread-local staging for this job - vecsim_stl::vector localGraphUpdates(this->allocator); - vecsim_stl::vector localNeighborUpdates(this->allocator); - - try { - // Get vector data from processing batch (read-only, thread-safe) - // Use local copies to avoid race with main thread resizing vectors container - auto rawIt = processingBatch->rawVectors.find(job->vectorId); - if (rawIt == processingBatch->rawVectors.end()) { - throw std::runtime_error("Raw vector not found in processing batch"); - } - const void *raw_vector_data = rawIt->second.data(); - - auto procIt = processingBatch->processedVectors.find(job->vectorId); - if (procIt == processingBatch->processedVectors.end()) { - throw std::runtime_error("Processed vector not found in processing batch"); - } - const void *vector_data = procIt->second.data(); - - // Skip if this is the entry point (no connections to make) - if (job->vectorId != job->entryPointSnapshot) { - // Insert into graph - reads from RocksDB are protected by shared lock - // The insertElementToGraph method will stage updates - insertElementToGraph(job->vectorId, job->elementMaxLevel, job->entryPointSnapshot, - job->maxLevelSnapshot, raw_vector_data, vector_data); - } - - } catch (const std::exception &e) { - // Handle error with retry mechanism - if (job->retryCount.fetch_add(1) < HNSWDiskInsertJob::MAX_RETRIES) { - // Resubmit job for retry - submitSingleJob(job); - return; // Don't decrement counter yet - } else { - // Max retries exceeded, mark as failed - this->log(VecSimCommonStrings::LOG_WARNING_STRING, - "Insert job failed after %d retries: %s", HNSWDiskInsertJob::MAX_RETRIES, - e.what()); - job->isValid = false; - } + // vecsim_stl::vector localGraphUpdates(this->allocator); + // vecsim_stl::vector localNeighborUpdates(this->allocator); + + + // Get vector data from processing batch (read-only, thread-safe) + // Use local copies to avoid race with main thread resizing vectors container + auto rawIt = processingBatch->rawVectors.find(job->vectorId); + if (rawIt == processingBatch->rawVectors.end()) { + throw std::runtime_error("Raw vector not found in processing batch"); } + const void *raw_vector_data = rawIt->second.data(); + + auto procIt = processingBatch->processedVectors.find(job->vectorId); + if (procIt == processingBatch->processedVectors.end()) { + throw std::runtime_error("Processed vector not found in processing batch"); + } + const void *vector_data = procIt->second.data(); + + // Skip if this is the entry point (no connections to make) + if (job->vectorId != job->entryPointSnapshot) { + // Insert into graph - reads from RocksDB are protected by shared lock + // The insertElementToGraph method will stage updates + insertElementToGraph(job->vectorId, job->elementMaxLevel, job->entryPointSnapshot, + job->maxLevelSnapshot, raw_vector_data, vector_data); + } + // Decrement counter and submit flush job if this is the last insert job if (pendingInsertJobsCounter.fetch_sub(1) == 1) { @@ -2491,15 +2486,11 @@ void HNSWDiskIndex::processDeleteBatch() { // Mark the metadata as invalid idToMetaData[deleted_id].label = INVALID_LABEL; - // Remove raw vector from RAM if it exists - requires exclusive lock - { - std::lock_guard lock(rawVectorsGuard); - auto ram_it = rawVectorsInRAM.find(deleted_id); - if (ram_it != rawVectorsInRAM.end()) { - rawVectorsInRAM.erase(ram_it); - } + // Remove raw vector from RAM if it exists + auto ram_it = rawVectorsInRAM.find(deleted_id); + if (ram_it != rawVectorsInRAM.end()) { + rawVectorsInRAM.erase(ram_it); } - } // Flush all staged graph updates to disk in a single batch operation @@ -2544,11 +2535,10 @@ void HNSWDiskIndex::setBatchThreshold(size_t threshold) { template void HNSWDiskIndex::debugPrintGraphStructure() const { size_t elementCount = curElementCount.load(std::memory_order_acquire); - EntryPointState state = entryPointState.load(std::memory_order_acquire); this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "=== HNSW Disk Index Graph Structure ==="); this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Total elements: %zu", elementCount); - this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Entry point: %u", state.entrypointNode); - this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Max level: %u", state.maxLevel); + this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Entry point: %u", entrypointNode); + this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Max level: %zu", maxLevel); this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "M: %zu, M0: %zu", M, M0); // Count total edges @@ -2692,10 +2682,9 @@ void HNSWDiskIndex::debugValidateGraphConnectivity() const { } // Check if entry point exists and has neighbors - EntryPointState state = entryPointState.load(std::memory_order_acquire); - if (state.entrypointNode != INVALID_ID) { - this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Entry point %u exists", state.entrypointNode); - debugPrintNodeNeighbors(state.entrypointNode); + if (entrypointNode != INVALID_ID) { + this->log(VecSimCommonStrings::LOG_DEBUG_STRING, "Entry point %u exists", entrypointNode); + debugPrintNodeNeighbors(entrypointNode); } else { this->log(VecSimCommonStrings::LOG_WARNING_STRING, "WARNING: No entry point set!"); } @@ -2947,9 +2936,6 @@ template vecsim_stl::vector HNSWDiskIndex::markDelete(labelType label) { vecsim_stl::vector internal_ids(this->allocator); - // Protect all accesses to labelToIdMap, idToMetaData, entrypointNode, maxLevel - std::lock_guard lock(indexDataGuard); - // Find the internal ID for this label auto it = labelToIdMap.find(label); if (it == labelToIdMap.end()) { @@ -2959,20 +2945,19 @@ vecsim_stl::vector HNSWDiskIndex::markDelete(labelTy const idType internalId = it->second; - // Check if already marked deleted (use unlocked version since we hold lock) - if (__atomic_load_n(&idToMetaData[internalId].flags, 0) & DELETE_MARK) { + // Check if already marked deleted + if (idToMetaData[internalId].flags & DELETE_MARK) { // Already deleted, return empty vector return internal_ids; } // Mark as deleted (but don't clean up raw vectors yet - they're needed for graph repair // in processDeleteBatch. Cleanup happens there after repair is complete.) - __atomic_fetch_or(&idToMetaData[internalId].flags, DELETE_MARK, 0); + idToMetaData[internalId].flags |= DELETE_MARK; this->numMarkedDeleted++; // If this is the entrypoint, we need to replace it - EntryPointState currentState = entryPointState.load(std::memory_order_acquire); - if (internalId == currentState.entrypointNode) { + if (internalId == entrypointNode) { replaceEntryPoint(); } @@ -2986,15 +2971,13 @@ vecsim_stl::vector HNSWDiskIndex::markDelete(labelTy template void HNSWDiskIndex::replaceEntryPoint() { - // PRECONDITION: Caller must hold indexDataGuard (for idToMetaData access) // This method is called when the current entrypoint is marked as deleted // We need to find a new entrypoint from the remaining non-deleted nodes - EntryPointState currentState = entryPointState.load(std::memory_order_acquire); - idType old_entry_point_id = currentState.entrypointNode; - size_t currentMaxLevel = currentState.maxLevel; + idType old_entry_point_id = entrypointNode; + size_t currentMaxLevel = maxLevel; // Try to find a new entrypoint at the current max level - while (currentMaxLevel != INVALID_MAX_LEVEL) { + while (currentMaxLevel > 0) { // First, try to find a neighbor of the old entrypoint at the top level GraphKey graphKey(old_entry_point_id, currentMaxLevel); std::string graph_value; @@ -3007,26 +2990,24 @@ void HNSWDiskIndex::replaceEntryPoint() { deserializeGraphValue(graph_value, neighbors); // Try to find a non-deleted neighbor - // Use direct flag check to avoid recursive lock (caller holds indexDataGuard) for (size_t i = 0; i < neighbors.size(); i++) { - if (!(__atomic_load_n(&idToMetaData[neighbors[i]].flags, 0) & DELETE_MARK)) { - entryPointState.store(EntryPointState(neighbors[i], currentMaxLevel), - std::memory_order_release); + if (!(idToMetaData[neighbors[i]].flags & DELETE_MARK)) { + entrypointNode = neighbors[i]; + maxLevel = currentMaxLevel; return; } } } // If no suitable neighbor found, search for any non-deleted node at this level - // Use direct flag check to avoid recursive lock (caller holds indexDataGuard) - size_t elementCount = curElementCount.load(std::memory_order_acquire); + size_t elementCount = curElementCount.load(); for (idType id = 0; id < elementCount; id++) { if (id != old_entry_point_id && id < idToMetaData.size() && idToMetaData[id].label != INVALID_LABEL && idToMetaData[id].topLevel == currentMaxLevel && - !(__atomic_load_n(&idToMetaData[id].flags, 0) & DELETE_MARK)) { - entryPointState.store(EntryPointState(id, currentMaxLevel), - std::memory_order_release); + !(idToMetaData[id].flags & DELETE_MARK)) { + entrypointNode = id; + maxLevel = currentMaxLevel; return; } } @@ -3036,7 +3017,8 @@ void HNSWDiskIndex::replaceEntryPoint() { } // If we get here, the index is empty or all nodes are deleted - entryPointState.store(EntryPointState(), std::memory_order_release); + entrypointNode = INVALID_ID; + maxLevel = 0; } #ifdef BUILD_TESTS From 1f42ade766195f3448e41b4c938947e063148366 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Wed, 17 Dec 2025 13:53:55 +0200 Subject: [PATCH 11/34] Remove entry point from job --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 89 +++++++++++--------------- 1 file changed, 37 insertions(+), 52 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index d39f41fe9..a10532d1d 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -139,20 +139,10 @@ struct HNSWDiskInsertJob : public AsyncJob { idType vectorId; size_t elementMaxLevel; - // Snapshot of entry point state at job creation time - idType entryPointSnapshot; - size_t maxLevelSnapshot; - - // Retry counter for error handling - std::atomic retryCount{0}; - static constexpr int MAX_RETRIES = 3; - HNSWDiskInsertJob(std::shared_ptr allocator, idType vectorId_, - size_t elementMaxLevel_, idType entryPoint_, size_t maxLevel_, - JobCallback insertCb, VecSimIndex *index_) + size_t elementMaxLevel_, JobCallback insertCb, VecSimIndex *index_) : AsyncJob(allocator, HNSW_DISK_INSERT_VECTOR_JOB, insertCb, index_), vectorId(vectorId_), - elementMaxLevel(elementMaxLevel_), entryPointSnapshot(entryPoint_), - maxLevelSnapshot(maxLevel_) {} + elementMaxLevel(elementMaxLevel_) {} }; /** @@ -2112,14 +2102,10 @@ void HNSWDiskIndex::processBatch() { stagedInsertMap.clear(); stagedInsertNeighborUpdates.clear(); - // Snapshot entry point state - idType currentEntryPoint = entrypointNode; - size_t currentMaxLevel = maxLevel; - - // First pass: Update entry point for any vectors with higher levels - // Note: idToMetaData access still needs indexDataGuard for resize protection + // First pass: Set entry point if not set, or update if we have a higher level vector + // This mirrors the original disk-poc algorithm where entry point is updated during processing { - std::shared_lock lock(indexDataGuard); + std::unique_lock lock(indexDataGuard); for (size_t i = 0; i < processingBatch->count; i++) { idType vectorId = processingBatch->vectorIds[i]; // Use direct flag check to avoid recursive lock (isMarkedDeleted takes shared lock) @@ -2127,21 +2113,14 @@ void HNSWDiskIndex::processBatch() { continue; DiskElementMetaData &metadata = idToMetaData[vectorId]; - if (currentEntryPoint == INVALID_ID || metadata.topLevel > currentMaxLevel) { - currentEntryPoint = vectorId; - currentMaxLevel = metadata.topLevel; + if (entrypointNode == INVALID_ID || metadata.topLevel > maxLevel) { + entrypointNode = vectorId; + maxLevel = metadata.topLevel; } } } - // Update global entry point if changed - if (currentEntryPoint != entrypointNode || currentMaxLevel != maxLevel) { - entrypointNode = currentEntryPoint; - maxLevel = currentMaxLevel; - } - - // Count valid vectors and create jobs - size_t validVectorCount = 0; + // Create jobs for all vectors (entry point check is done in executeInsertJob) vecsim_stl::vector jobs(this->allocator); jobs.reserve(processingBatch->count); @@ -2150,18 +2129,10 @@ void HNSWDiskIndex::processBatch() { if (isMarkedDeleted(vectorId)) continue; - // Skip if this is the entry point (already set, no connections needed) - if (vectorId == currentEntryPoint && validVectorCount == 0) { - validVectorCount++; - continue; - } - - validVectorCount++; DiskElementMetaData &metadata = idToMetaData[vectorId]; HNSWDiskInsertJob *job = new (this->allocator) HNSWDiskInsertJob(this->allocator, vectorId, metadata.topLevel, - currentEntryPoint, currentMaxLevel, executeInsertJobWrapper, this); jobs.push_back(job); } @@ -2231,33 +2202,47 @@ void HNSWDiskIndex::executeInsertJob(HNSWDiskInsertJob *job) return; } - // Thread-local staging for this job - // vecsim_stl::vector localGraphUpdates(this->allocator); - // vecsim_stl::vector localNeighborUpdates(this->allocator); - - // Get vector data from processing batch (read-only, thread-safe) - // Use local copies to avoid race with main thread resizing vectors container auto rawIt = processingBatch->rawVectors.find(job->vectorId); if (rawIt == processingBatch->rawVectors.end()) { - throw std::runtime_error("Raw vector not found in processing batch"); + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "Raw vector not found in processing batch for vectorId %u", job->vectorId); + if (pendingInsertJobsCounter.fetch_sub(1) == 1) { + submitSingleJob(pendingFlushJob); + } + delete job; + return; } const void *raw_vector_data = rawIt->second.data(); auto procIt = processingBatch->processedVectors.find(job->vectorId); if (procIt == processingBatch->processedVectors.end()) { - throw std::runtime_error("Processed vector not found in processing batch"); + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "Processed vector not found in processing batch for vectorId %u", job->vectorId); + if (pendingInsertJobsCounter.fetch_sub(1) == 1) { + submitSingleJob(pendingFlushJob); + } + delete job; + return; } const void *vector_data = procIt->second.data(); - // Skip if this is the entry point (no connections to make) - if (job->vectorId != job->entryPointSnapshot) { - // Insert into graph - reads from RocksDB are protected by shared lock - // The insertElementToGraph method will stage updates - insertElementToGraph(job->vectorId, job->elementMaxLevel, job->entryPointSnapshot, - job->maxLevelSnapshot, raw_vector_data, vector_data); + // Read current entry point state dynamically (like the original disk-poc algorithm) + // This allows jobs to benefit from entry point updates made by previously completed jobs + idType currentEntryPoint; + size_t currentMaxLevel; + { + std::shared_lock lock(indexDataGuard); + currentEntryPoint = entrypointNode; + currentMaxLevel = maxLevel; } + // Skip if this is the entry point (no connections to make for the first element) + if (job->vectorId != currentEntryPoint && currentEntryPoint != INVALID_ID) { + // Insert into graph using the original algorithm + insertElementToGraph(job->vectorId, job->elementMaxLevel, currentEntryPoint, + currentMaxLevel, raw_vector_data, vector_data); + } // Decrement counter and submit flush job if this is the last insert job if (pendingInsertJobsCounter.fetch_sub(1) == 1) { From 013b25806814560108c427c288df2ad6c9073fdd Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Wed, 17 Dec 2025 15:52:10 +0200 Subject: [PATCH 12/34] entry point --- .../algorithms/hnsw/hnsw_disk_serializer.h | 24 +++++++------------ 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h index 39fbb870a..12830e895 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h @@ -61,7 +61,8 @@ HNSWDiskIndex::HNSWDiskIndex( pendingVectorCount(0), pendingDeleteIds(this->allocator), stagedInsertUpdates(this->allocator), stagedDeleteUpdates(this->allocator), stagedRepairUpdates(this->allocator), - stagedInsertNeighborUpdates(this->allocator) { + stagedInsertNeighborUpdates(this->allocator), + jobQueue(nullptr), jobQueueCtx(nullptr), SubmitJobsToQueue(nullptr) { // Restore index fields from file (including batchThreshold) this->restoreIndexFields(input); @@ -449,8 +450,7 @@ HNSWIndexMetaData HNSWDiskIndex::checkIntegrity() const { } // Validate entry point - EntryPointState state = this->entryPointState.load(std::memory_order_acquire); - if (elementCount > 0 && state.entrypointNode == INVALID_ID) { + if (elementCount > 0 && this->entrypointNode == INVALID_ID) { this->log(VecSimCommonStrings::LOG_WARNING_STRING, "checkIntegrity failed: no entry point set for non-empty index"); return res; @@ -607,13 +607,9 @@ void HNSWDiskIndex::restoreIndexFields(std::ifstream &input) Serializer::readBinaryPOD(input, tempElementCount); this->curElementCount.store(tempElementCount, std::memory_order_release); Serializer::readBinaryPOD(input, this->numMarkedDeleted); - // Read entry point state (maxLevel and entrypointNode are now atomic) - size_t tempMaxLevel; - idType tempEntrypointNode; - Serializer::readBinaryPOD(input, tempMaxLevel); - Serializer::readBinaryPOD(input, tempEntrypointNode); - this->entryPointState.store(EntryPointState(tempEntrypointNode, tempMaxLevel), - std::memory_order_release); + // Read entry point state + Serializer::readBinaryPOD(input, this->maxLevel); + Serializer::readBinaryPOD(input, this->entrypointNode); // Restore batch processing configuration Serializer::readBinaryPOD(input, this->batchThreshold); @@ -744,11 +740,9 @@ void HNSWDiskIndex::saveIndexFields(std::ofstream &output) c // Save index state Serializer::writeBinaryPOD(output, elementCount); Serializer::writeBinaryPOD(output, this->numMarkedDeleted); - // Write entry point state (maxLevel and entrypointNode are now atomic) - EntryPointState state = this->entryPointState.load(std::memory_order_acquire); - size_t maxLevelToSave = static_cast(state.maxLevel); - Serializer::writeBinaryPOD(output, maxLevelToSave); - Serializer::writeBinaryPOD(output, state.entrypointNode); + // Write entry point state + Serializer::writeBinaryPOD(output, this->maxLevel); + Serializer::writeBinaryPOD(output, this->entrypointNode); // Save batch processing configuration Serializer::writeBinaryPOD(output, this->batchThreshold); From 1cfd8af2dc62954a2b544d85d660a0cd8ec77630 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Thu, 18 Dec 2025 09:15:24 +0200 Subject: [PATCH 13/34] fix the bad recall --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 145 ++++++++++++++++++++----- tests/benchmark/bm_vecsim_index.h | 13 +-- 2 files changed, 119 insertions(+), 39 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 204a69c0d..7e88a42b8 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -396,6 +396,9 @@ class HNSWDiskIndex : public VecSimIndexAbstract void getNeighborsAndVector(idType nodeId, size_t level, vecsim_stl::vector& result, void* vector_data) const; void getNeighborsAndVector(labelType nodeId, size_t level, vecsim_stl::vector& result, void* vector_data) const; size_t getNeighborsCount(idType nodeId, size_t level) const; + // Thread-safe atomic check-and-add for neighbor updates + // Returns true if neighbor was added (had capacity), false if full (needs re-evaluation) + bool tryAddNeighborIfCapacity(idType nodeId, size_t level, idType newNeighborId, size_t maxCapacity); void searchPendingVectors(const void* query_data, candidatesLabelsMaxHeap& top_candidates, size_t k) const; // Manual control of staged updates @@ -1032,15 +1035,9 @@ idType HNSWDiskIndex::mutuallyConnectNewElement( idType selected_neighbor = neighbor_data.second; DistType distance = neighbor_data.first; - // Check if the neighbor's neighbor list has capacity - // Use the helper function to get the current neighbor count (checks staged updates too) - size_t current_neighbor_count = getNeighborsCount(selected_neighbor, level); - - if (current_neighbor_count < max_M_cur) { - // Neighbor has capacity, just add the new node - std::unique_lock lock(stagedUpdatesGuard); - stagedInsertNeighborUpdates.emplace_back(selected_neighbor, level, new_node_id); - } else { + // Use atomic check-and-add to prevent race conditions where multiple threads + // think there's capacity and all add neighbors, exceeding max_M_cur + if (!tryAddNeighborIfCapacity(selected_neighbor, level, new_node_id, max_M_cur)) { // Neighbor is full, need to re-evaluate connections using revisitNeighborConnections // logic stageRevisitNeighborConnections(new_node_id, selected_neighbor, level, distance); @@ -2009,15 +2006,35 @@ void HNSWDiskIndex::getNeighborsAndVector(labelType nodeId, template size_t HNSWDiskIndex::getNeighborsCount(idType nodeId, size_t level) const { - // First check staged graph updates using hash maps for O(1) lookup uint64_t lookup_key = makeRepairKey(nodeId, level); + size_t base_count = 0; + bool foundInStaged = false; + size_t incrementalCount = 0; - // Check insert staging area - auto insert_it = stagedInsertMap.find(lookup_key); - if (insert_it != stagedInsertMap.end()) { - const auto &update = stagedInsertUpdates[insert_it->second]; - - return update.neighbors.size(); + // Check staged updates under lock for thread safety + { + std::shared_lock lock(stagedUpdatesGuard); + + // Check insert staging area (complete neighbor list replacements) + auto insert_it = stagedInsertMap.find(lookup_key); + if (insert_it != stagedInsertMap.end()) { + const auto &update = stagedInsertUpdates[insert_it->second]; + base_count = update.neighbors.size(); + foundInStaged = true; + } + + // Count incremental neighbor updates for this node/level + // These are additions on top of the base neighbor list + for (const auto &update : stagedInsertNeighborUpdates) { + if (update.node_id == nodeId && update.level == level) { + incrementalCount++; + } + } + } + + if (foundInStaged) { + // Return staged count + incremental updates + return base_count + incrementalCount; } // If not found in staged updates, check disk @@ -2026,15 +2043,82 @@ size_t HNSWDiskIndex::getNeighborsCount(idType nodeId, size_ rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); if (status.ok()) { - const char* ptr = graph_value.data(); ptr += this->inputBlobSize; size_t neighbor_count = *reinterpret_cast(ptr); - - return neighbor_count; + + // Return disk count + incremental updates from staging + return neighbor_count + incrementalCount; } - return 0; // Not found + return incrementalCount; // Only incremental updates if not found elsewhere +} + +template +bool HNSWDiskIndex::tryAddNeighborIfCapacity( + idType nodeId, size_t level, idType newNeighborId, size_t maxCapacity) { + // Atomic check-and-add: under a single lock, check capacity and add if there's room + // This prevents race conditions where multiple threads think there's capacity + + uint64_t lookup_key = makeRepairKey(nodeId, level); + + std::unique_lock lock(stagedUpdatesGuard); + + size_t base_count = 0; + bool foundInStaged = false; + size_t incrementalCount = 0; + + // Check insert staging area (complete neighbor list replacements) + auto insert_it = stagedInsertMap.find(lookup_key); + if (insert_it != stagedInsertMap.end()) { + const auto &update = stagedInsertUpdates[insert_it->second]; + base_count = update.neighbors.size(); + foundInStaged = true; + } + + // Count incremental neighbor updates for this node/level + for (const auto &update : stagedInsertNeighborUpdates) { + if (update.node_id == nodeId && update.level == level) { + incrementalCount++; + } + } + + size_t current_count; + if (foundInStaged) { + current_count = base_count + incrementalCount; + } else { + // Need to check disk (release lock temporarily for I/O, then re-acquire and re-check) + lock.unlock(); + + GraphKey graphKey(nodeId, level); + std::string graph_value; + rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + + size_t disk_count = 0; + if (status.ok()) { + const char* ptr = graph_value.data(); + ptr += this->inputBlobSize; + disk_count = *reinterpret_cast(ptr); + } + + // Re-acquire lock and re-count incremental updates (they may have changed) + lock.lock(); + incrementalCount = 0; + for (const auto &update : stagedInsertNeighborUpdates) { + if (update.node_id == nodeId && update.level == level) { + incrementalCount++; + } + } + current_count = disk_count + incrementalCount; + } + + // Now atomically decide and add + if (current_count < maxCapacity) { + stagedInsertNeighborUpdates.emplace_back(nodeId, level, newNeighborId); + return true; // Successfully added + } + + return false; // Neighbor is full, needs re-evaluation } template @@ -2198,20 +2282,23 @@ void HNSWDiskIndex::processBatch() { stagedInsertMap.clear(); stagedInsertNeighborUpdates.clear(); - // First pass: Set entry point if not set, or update if we have a higher level vector - // This mirrors the original disk-poc algorithm where entry point is updated during processing + // Set entry point only if it doesn't exist (matches original disk-poc algorithm) + // In the original algorithm, only the first vector becomes entry point when graph is empty. + // Subsequent vectors always insert using the existing entry point. { std::unique_lock lock(indexDataGuard); - for (size_t i = 0; i < processingBatch->count; i++) { - idType vectorId = processingBatch->vectorIds[i]; - // Use direct flag check to avoid recursive lock (isMarkedDeleted takes shared lock) - if (__atomic_load_n(&idToMetaData[vectorId].flags, 0) & DELETE_MARK) - continue; + if (entrypointNode == INVALID_ID) { + // Find first non-deleted vector to be the entry point + for (size_t i = 0; i < processingBatch->count; i++) { + idType vectorId = processingBatch->vectorIds[i]; + // Use direct flag check to avoid recursive lock (isMarkedDeleted takes shared lock) + if (__atomic_load_n(&idToMetaData[vectorId].flags, 0) & DELETE_MARK) + continue; - DiskElementMetaData &metadata = idToMetaData[vectorId]; - if (entrypointNode == INVALID_ID || metadata.topLevel > maxLevel) { + DiskElementMetaData &metadata = idToMetaData[vectorId]; entrypointNode = vectorId; maxLevel = metadata.topLevel; + break; // Only set first non-deleted vector as entry point } } } diff --git a/tests/benchmark/bm_vecsim_index.h b/tests/benchmark/bm_vecsim_index.h index 753fa8d80..8c62121b6 100644 --- a/tests/benchmark/bm_vecsim_index.h +++ b/tests/benchmark/bm_vecsim_index.h @@ -191,16 +191,9 @@ void BM_VecSimIndex::Initialize() { auto &mock_thread_pool = *BM_VecSimGeneral::mock_thread_pool; mock_thread_pool.ctx->index_strong_ref = indices[INDEX_HNSW_DISK].get_shared(); // Threads will be started on-demand by the benchmark via reconfigure_threads(). - } - - // Set up job queue for async operations on the disk index - auto *disk_index = dynamic_cast *>(indices[INDEX_HNSW_DISK].get()); - if (disk_index && BM_VecSimGeneral::mock_thread_pool) { - auto &mock_thread_pool = *BM_VecSimGeneral::mock_thread_pool; - disk_index->setJobQueue(&mock_thread_pool.jobQ, mock_thread_pool.ctx, - tieredIndexMock::submit_callback); - // Initialize threads for async processing - mock_thread_pool.init_threads(); + // NOTE: Job queue is NOT set here - individual benchmarks that need async + // processing should call setJobQueue() with appropriate thread configuration. + // The regular AddLabel benchmark uses single-threaded mode (no job queue). } } From 99f903a2e2e523308e4ac5be096b8653c1e0ff0d Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Thu, 18 Dec 2025 09:38:20 +0200 Subject: [PATCH 14/34] wait in the serialzer --- .../data/scripts/hnsw_disk_serializer.cpp | 70 +++++++++++++------ 1 file changed, 49 insertions(+), 21 deletions(-) diff --git a/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp b/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp index 954221f7e..3060f16ee 100644 --- a/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp +++ b/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp @@ -393,8 +393,8 @@ int main(int argc, char *argv[]) { mock_thread_pool->init_threads(); // Configure the disk index to use the job queue - // Use a larger batch threshold for bulk loading (default 10 is too small) - const size_t bulk_batch_threshold = 1000; + // With sibling injection, larger batches work well (vectors in batch can discover each other) + const size_t bulk_batch_threshold = 10; if (type == VecSimType_FLOAT32) { auto *disk_index = dynamic_cast *>(index); if (disk_index) { @@ -509,27 +509,55 @@ int main(int argc, char *argv[]) { std::cout << "\n"; // Flush any remaining pending vectors and wait for those jobs too - if (type == VecSimType_FLOAT32) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) disk_index->flushBatch(); - } else if (type == VecSimType_FLOAT64) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) disk_index->flushBatch(); - } else if (type == VecSimType_BFLOAT16) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) disk_index->flushBatch(); - } else if (type == VecSimType_FLOAT16) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) disk_index->flushBatch(); - } else if (type == VecSimType_INT8) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) disk_index->flushBatch(); - } else if (type == VecSimType_UINT8) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) disk_index->flushBatch(); + // Keep flushing until all pending vectors are processed + size_t pending = 1; // Start with non-zero to enter loop + while (pending > 0) { + // First wait for any in-progress batch to complete + mock_thread_pool->thread_pool_wait(); + + // Now try to flush remaining pending vectors + if (type == VecSimType_FLOAT32) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->flushBatch(); + pending = disk_index->getPendingInsertCount(); + } + } else if (type == VecSimType_FLOAT64) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->flushBatch(); + pending = disk_index->getPendingInsertCount(); + } + } else if (type == VecSimType_BFLOAT16) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->flushBatch(); + pending = disk_index->getPendingInsertCount(); + } + } else if (type == VecSimType_FLOAT16) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->flushBatch(); + pending = disk_index->getPendingInsertCount(); + } + } else if (type == VecSimType_INT8) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->flushBatch(); + pending = disk_index->getPendingInsertCount(); + } + } else if (type == VecSimType_UINT8) { + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->flushBatch(); + pending = disk_index->getPendingInsertCount(); + } + } else { + break; // Unknown type, exit loop + } } - // Wait again for the flush batch jobs to complete + // Final wait for all jobs to complete mock_thread_pool->thread_pool_wait(); std::cout << "All background jobs completed.\n"; From 37299d52a6238e2a0b073b61073f1230dda1208b Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Sun, 21 Dec 2025 14:14:12 +0200 Subject: [PATCH 15/34] batchless --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 1172 ++++++++++++++--- .../algorithms/hnsw/hnsw_disk_serializer.h | 3 +- src/VecSim/vec_sim_common.h | 1 + tests/unit/test_hnsw_disk.cpp | 109 -- 4 files changed, 966 insertions(+), 319 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 7e88a42b8..f1eead7f9 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -55,6 +55,7 @@ // #include #include #include +#include // Forward declaration for AsyncJob #include "VecSim/vec_sim_tiered_index.h" @@ -155,6 +156,30 @@ struct HNSWDiskFlushJob : public AsyncJob { : AsyncJob(allocator, HNSW_DISK_FLUSH_UPDATES_JOB, flushCb, index_) {} }; +/** + * Definition of a batchless job that inserts a single vector completely from start to end. + * Each job is self-contained and writes directly to disk upon completion. + * No batching or staging - optimized for workloads where disk write is cheap + * but reading (searching for neighbors) is the bottleneck. + * + * The job holds copies of the vector data to avoid external references and race conditions. + */ +struct HNSWDiskSingleInsertJob : public AsyncJob { + idType vectorId; + size_t elementMaxLevel; + // Store vector data directly in the job (no external references) + std::string rawVectorData; // Original float32 vector + std::string processedVectorData; // Preprocessed/quantized vector for distance calculations + + HNSWDiskSingleInsertJob(std::shared_ptr allocator, idType vectorId_, + size_t elementMaxLevel_, std::string &&rawVector, + std::string &&processedVector, JobCallback insertCb, + VecSimIndex *index_) + : AsyncJob(allocator, HNSW_DISK_SINGLE_INSERT_JOB, insertCb, index_), + vectorId(vectorId_), elementMaxLevel(elementMaxLevel_), + rawVectorData(std::move(rawVector)), processedVectorData(std::move(processedVector)) {} +}; + //////////////////////////////////// HNSW index implementation //////////////////////////////////// template @@ -333,6 +358,53 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Needed because unordered_map can rehash during insert, invalidating iterators mutable std::shared_mutex rawVectorsGuard; + /********************************** Batchless Mode Support **********************************/ + + // Striped neighbor cache for reduced lock contention in multi-threaded scenarios + // Uses lock striping: cache is partitioned into NUM_CACHE_STRIPES independent stripes + // Each stripe has its own lock, cache map, and dirty set + // This allows threads accessing different stripes to proceed in parallel + + static constexpr size_t NUM_CACHE_STRIPES = 64; // Power of 2 for efficient modulo + + // Cache stripe structure - each stripe is cache-line aligned to prevent false sharing + struct CacheStripe { + std::shared_mutex guard; + std::unordered_map> cache; + std::unordered_set dirty; + // Track nodes created in current batch (never written to disk yet) + // This helps avoid disk lookups for new nodes + std::unordered_set newNodes; + + CacheStripe() = default; + CacheStripe(const CacheStripe&) = delete; + CacheStripe& operator=(const CacheStripe&) = delete; + CacheStripe(CacheStripe&&) = delete; + CacheStripe& operator=(CacheStripe&&) = delete; + }; + + // Array of cache stripes - using unique_ptr for lazy initialization + mutable std::unique_ptr cacheStripes_; + + // Atomic counter for total dirty nodes (for fast threshold check without locking) + mutable std::atomic totalDirtyCount_{0}; + + // Helper function to compute stripe index from cache key + // Uses mixing function for better distribution + static size_t getStripeIndex(uint64_t key) { + // Mix the bits for better distribution (splitmix64-style mixing) + key ^= key >> 33; + key *= 0xff51afd7ed558ccdULL; + key ^= key >> 33; + return key % NUM_CACHE_STRIPES; + } + + // Threshold for flushing dirty nodes to disk (0 = flush after each insert, default = batch) + size_t diskWriteBatchThreshold = 1000; + + // Lock for protecting dirty nodes flush operations (global flush serialization) + mutable std::mutex diskWriteGuard; + protected: HNSWDiskIndex() = delete; // default constructor is disabled. // default (shallow) copy constructor is disabled. @@ -387,6 +459,40 @@ class HNSWDiskIndex : public VecSimIndexAbstract void mergeLocalStagedUpdates(vecsim_stl::vector &localGraphUpdates, vecsim_stl::vector &localNeighborUpdates); + /********************************** Batchless Mode Methods **********************************/ + + // Batchless job execution + static void executeSingleInsertJobWrapper(AsyncJob *job); + void executeSingleInsertJob(HNSWDiskSingleInsertJob *job); + + // Batchless graph insertion (updates cache instead of staging) + void insertElementToGraphBatchless(idType element_id, size_t element_max_level, + idType entry_point, size_t global_max_level, + const void *raw_vector_data, const void *vector_data, + vecsim_stl::vector &modifiedNodes); + idType mutuallyConnectNewElementBatchless(idType new_node_id, + vecsim_stl::updatable_max_heap &top_candidates, + size_t level, vecsim_stl::vector &modifiedNodes); + + // Cache management methods + void addNeighborToCache(idType nodeId, size_t level, idType newNeighborId); + void setNeighborsInCache(idType nodeId, size_t level, const vecsim_stl::vector &neighbors, bool isNewNode = false); + void getNeighborsFromCache(idType nodeId, size_t level, vecsim_stl::vector &result) const; + // Atomic check-and-add for cache - returns true if added, false if at capacity + bool tryAddNeighborToCacheIfCapacity(idType nodeId, size_t level, idType newNeighborId, size_t maxCapacity); + + // Write dirty cache entries to disk + void writeDirtyNodesToDisk(const vecsim_stl::vector &modifiedNodes, + const void *newVectorRawData, idType newVectorId); + + // Synchronous fallback for batchless mode without job queue + void executeInsertAndWriteSynchronously(idType vectorId, size_t elementMaxLevel, + const void *rawVectorData, const void *processedVectorData); + + // Helper to write a single vector's entry to disk + void writeVectorToDisk(idType vectorId, const void *rawVectorData, + const vecsim_stl::vector &neighbors); + // Helper methods void emplaceHeap(vecsim_stl::abstract_priority_queue &heap, DistType dist, idType id) const; @@ -411,8 +517,13 @@ class HNSWDiskIndex : public VecSimIndexAbstract inline bool filterDeletedNodes(vecsim_stl::vector& neighbors) const { size_t original_size = neighbors.size(); size_t elementCount = curElementCount.load(std::memory_order_acquire); + size_t metadataSize = idToMetaData.size(); auto new_end = std::remove_if(neighbors.begin(), neighbors.end(), - [this, elementCount](idType id) { return id >= elementCount || isMarkedDeleted(id); }); + [this, elementCount, metadataSize](idType id) { + // Use lock-free check: bounds check first, then atomic flag read + return id >= elementCount || id >= metadataSize || + isMarkedAsUnsafe(id); + }); neighbors.erase(new_end, neighbors.end()); return neighbors.size() < original_size; } @@ -447,6 +558,7 @@ class HNSWDiskIndex : public VecSimIndexAbstract candidatesLabelsMaxHeap *getNewMaxPriorityQueue() const; bool isMarkedDeleted(idType id) const; bool isMarkedDeleted(labelType id) const; + bool isMarkedDeletedUnsafe(idType id) const; // Lock-free version for hot paths labelType getExternalLabel(idType id) const; // Helper methods for emplacing to heaps (overloaded for idType and labelType) @@ -553,6 +665,13 @@ class HNSWDiskIndex : public VecSimIndexAbstract return __atomic_load_n(&idToMetaData[internalId].flags, 0) & FLAG; } + // Lock-free version for hot paths where we know the ID is valid and in bounds + // (caller must ensure internalId < idToMetaData.size()) + template + bool isMarkedAsUnsafe(idType internalId) const { + return __atomic_load_n(&idToMetaData[internalId].flags, 0) & FLAG; + } + // Mark delete API vecsim_stl::vector markDelete(labelType label); size_t getNumMarkedDeleted() const { return numMarkedDeleted; } @@ -564,6 +683,12 @@ class HNSWDiskIndex : public VecSimIndexAbstract size_t getPendingInsertCount() const { return pendingVectorCount; } size_t getProcessingBatchCount() const { return processingBatch ? processingBatch->count : 0; } + // Disk write batching control + void setDiskWriteBatchThreshold(size_t threshold) { diskWriteBatchThreshold = threshold; } + size_t getDiskWriteBatchThreshold() const { return diskWriteBatchThreshold; } + size_t getPendingDiskWriteCount() const { return totalDirtyCount_.load(std::memory_order_relaxed); } + void flushDirtyNodesToDisk(); // Flush all pending dirty nodes to disk + // Job queue configuration (for multi-threaded processing) void setJobQueue(void *jobQueue_, void *jobQueueCtx_, SubmitCB submitCb_) { jobQueue = jobQueue_; @@ -636,7 +761,8 @@ HNSWDiskIndex::HNSWDiskIndex( stagedInsertUpdates(this->allocator), stagedDeleteUpdates(this->allocator), stagedRepairUpdates(this->allocator), stagedInsertNeighborUpdates(this->allocator), - jobQueue(jobQueue_), jobQueueCtx(jobQueueCtx_), SubmitJobsToQueue(submitCb_) { + jobQueue(jobQueue_), jobQueueCtx(jobQueueCtx_), SubmitJobsToQueue(submitCb_), + cacheStripes_(new CacheStripe[NUM_CACHE_STRIPES]) { M = params->M ? params->M : HNSW_DEFAULT_M; M0 = M * 2; @@ -897,8 +1023,10 @@ int HNSWDiskIndex::addVector( // fetch_add returns the OLD value before incrementing, giving us a unique ID idType newElementId = static_cast(curElementCount.fetch_add(1, std::memory_order_acq_rel)); - // Store raw vector in RAM first (until flush batch) + // Store raw vector in RAM first (until written to disk) // We need to store the original vector before preprocessing + // NOTE: In batchless mode, we still use rawVectorsInRAM so other concurrent jobs can access + // the raw vectors of vectors that haven't been written to disk yet const char* raw_data = reinterpret_cast(vector); { std::lock_guard lock(rawVectorsGuard); @@ -937,22 +1065,59 @@ int HNSWDiskIndex::addVector( // Use load() to read atomic value visitedNodesHandlerPool.resize(curElementCount.load(std::memory_order_acquire)); - // Add only the vector ID to pending vectors for indexing - // Protected by batchSwapGuard to synchronize with query threads and processBatch - bool shouldProcessBatch = false; + // Each vector is processed immediately and written to disk + // Get entry point info + idType currentEntryPoint; { - std::lock_guard lock(batchSwapGuard); - pendingVectorIds.push_back(newElementId); - pendingVectorCount++; - shouldProcessBatch = (pendingVectorCount >= batchThreshold); + std::shared_lock lock(indexDataGuard); + currentEntryPoint = entrypointNode; } - // Process batch if threshold reached (outside lock to avoid holding lock during batch processing) - if (shouldProcessBatch) { - processBatch(); + // Handle first vector (becomes entry point) + if (currentEntryPoint == INVALID_ID) { + std::unique_lock lock(indexDataGuard); + if (entrypointNode == INVALID_ID) { + entrypointNode = newElementId; + maxLevel = elementMaxLevel; + } + // Write initial vector to disk with empty neighbors + vecsim_stl::vector emptyNeighbors(this->allocator); + for (size_t level = 0; level <= elementMaxLevel; level++) { + GraphKey graphKey(newElementId, level); + std::string value = serializeGraphValue(vector, emptyNeighbors); + auto writeOptions = rocksdb::WriteOptions(); + writeOptions.disableWAL = true; + db->Put(writeOptions, cf, graphKey.asSlice(), value); + } + // Remove raw vector from RAM now that it's on disk + { + std::lock_guard rawLock(rawVectorsGuard); + rawVectorsInRAM.erase(newElementId); + } + return 1; + } + + // Check if we have a job queue for async processing + if (SubmitJobsToQueue != nullptr) { + // Create job with vector data copies + std::string rawVectorCopy(raw_data, this->inputBlobSize); + std::string processedVectorCopy( + reinterpret_cast(processedBlobs.getStorageBlob()), + this->dataSize); + + auto *job = new (this->allocator) HNSWDiskSingleInsertJob( + this->allocator, newElementId, elementMaxLevel, std::move(rawVectorCopy), + std::move(processedVectorCopy), + HNSWDiskIndex::executeSingleInsertJobWrapper, this); + + submitSingleJob(job); + } else { + // Synchronous fallback + executeInsertAndWriteSynchronously(newElementId, elementMaxLevel, vector, + processedBlobs.getStorageBlob()); } - return 1; // Success + return 1; } template @@ -1014,23 +1179,17 @@ idType HNSWDiskIndex::mutuallyConnectNewElement( assert(top_candidates_list.size() <= M && "Should be not be more than M candidates returned by the heuristic"); - // Instead of writing to disk immediately, stage the updates in memory - // Stage the new node's neighbors + // Build the neighbor list for the new node vecsim_stl::vector neighbor_ids(this->allocator); neighbor_ids.reserve(top_candidates_list.size()); for (size_t i = 0; i < top_candidates_list.size(); ++i) { neighbor_ids.push_back(top_candidates_list[i].second); } - // Add to staged graph updates (for insertions) - protected by stagedUpdatesGuard - { - std::unique_lock lock(stagedUpdatesGuard); - uint64_t insert_key = makeRepairKey(new_node_id, level); - stagedInsertMap[insert_key] = stagedInsertUpdates.size(); - stagedInsertUpdates.emplace_back(new_node_id, level, neighbor_ids, this->allocator); - } + // Set the new node's neighbors in cache (this is a new node, so mark it as such) + setNeighborsInCache(new_node_id, level, neighbor_ids, true /* isNewNode */); - // Stage updates to existing nodes to include the new node in their neighbor lists + // Update existing nodes to include the new node in their neighbor lists for (const auto &neighbor_data : top_candidates_list) { idType selected_neighbor = neighbor_data.second; DistType distance = neighbor_data.first; @@ -1167,9 +1326,9 @@ void HNSWDiskIndex::stageRevisitNeighborConnections(idType n idType selected_neighbor, size_t level, DistType distance) { - // Read the neighbor's current neighbor list (checks staged updates first, then disk) + // Read the neighbor's current neighbor list from cache (source of truth) vecsim_stl::vector existing_neighbors(this->allocator); - getNeighbors(selected_neighbor, level, existing_neighbors); + getNeighborsFromCache(selected_neighbor, level, existing_neighbors); // Collect all candidates: existing neighbors + new node candidatesList candidates(this->allocator); @@ -1207,30 +1366,23 @@ void HNSWDiskIndex::stageRevisitNeighborConnections(idType n if (new_node_selected) { // The new node was selected, so we need to update the neighbor's neighbor list - // Extract the selected neighbor IDs + // Extract the selected neighbor IDs and update cache directly vecsim_stl::vector selected_neighbor_ids(this->allocator); selected_neighbor_ids.reserve(candidates.size()); for (const auto &candidate : candidates) { selected_neighbor_ids.push_back(candidate.second); } - // Stage this update - the neighbor's neighbor list will be completely replaced - // We'll need to handle this specially in flushStagedGraphUpdates - // Protected by stagedUpdatesGuard for thread safety - std::unique_lock lock(stagedUpdatesGuard); - uint64_t insert_key = makeRepairKey(selected_neighbor, level); - stagedInsertMap[insert_key] = stagedInsertUpdates.size(); - stagedInsertUpdates.emplace_back(selected_neighbor, level, selected_neighbor_ids, - this->allocator); + // Update the selected_neighbor's neighbor list in cache (not a new node) + setNeighborsInCache(selected_neighbor, level, selected_neighbor_ids, false /* isNewNode */); - // Also stage the bidirectional connection from new node to selected neighbor - stagedInsertNeighborUpdates.emplace_back(new_node_id, level, selected_neighbor); + // Also add bidirectional connection from new node to selected neighbor + addNeighborToCache(new_node_id, level, selected_neighbor); } else { - // The new node was not selected, so we only need to stage the unidirectional connection + // The new node was not selected, so we only need to add unidirectional connection // from new node to selected neighbor - std::unique_lock lock(stagedUpdatesGuard); - stagedInsertNeighborUpdates.emplace_back(new_node_id, level, selected_neighbor); + addNeighborToCache(new_node_id, level, selected_neighbor); } } @@ -1578,6 +1730,16 @@ bool HNSWDiskIndex::isMarkedDeleted(idType id) const { return id < idToMetaData.size() && isMarkedAs(id); } +// Lock-free version for hot paths - still needs bounds checking +template +bool HNSWDiskIndex::isMarkedDeletedUnsafe(idType id) const { + // Must check bounds even in unsafe version - accessing out of bounds is UB + if (id >= idToMetaData.size()) { + return true; // Treat out-of-bounds as deleted (won't be added to results) + } + return isMarkedAsUnsafe(id); +} + template bool HNSWDiskIndex::isMarkedDeleted(labelType id) const { auto it = labelToIdMap.find(id); @@ -1597,6 +1759,9 @@ template void HNSWDiskIndex::greedySearchLevel(const void *data_point, size_t level, idType &curr_element, DistType &cur_dist) const { + // Hold shared lock for entire search to prevent idToMetaData resize during isMarkedDeletedUnsafe calls + std::shared_lock indexLock(indexDataGuard); + bool changed; idType bestCand = curr_element; idType bestNonDeletedCand = bestCand; @@ -1635,7 +1800,8 @@ void HNSWDiskIndex::greedySearchLevel(const void *data_point cur_dist = d; bestCand = candidate; changed = true; - if (!running_query && !isMarkedDeleted(candidate)) { + // Use lock-free version - candidate was already validated by getNeighbors + if (!running_query && !isMarkedDeletedUnsafe(candidate)) { bestNonDeletedCand = bestCand; } } @@ -1710,7 +1876,7 @@ void HNSWDiskIndex::processCandidate( continue; } visited_set->insert(candidate_id); - // TODO: possibly use cached raw vectors + // TODO: possibly use cached raw vectors DistType cur_dist = this->calcDistance(data_point, getDataByInternalId(candidate_id)); if (lowerBound > cur_dist || top_candidates.size() < ef) { @@ -1718,8 +1884,9 @@ void HNSWDiskIndex::processCandidate( candidate_set.emplace(-cur_dist, candidate_id); // Insert the candidate to the top candidates heap only if it is not marked as - // deleted. - if (!isMarkedDeleted(candidate_id)) + // deleted. Use lock-free version since candidate_id was already bounds-checked + // by getNeighbors (which filters invalid IDs). + if (!isMarkedDeletedUnsafe(candidate_id)) emplaceHeap(top_candidates, cur_dist, candidate_id); if (top_candidates.size() > ef) @@ -1817,43 +1984,43 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level // Clear the result vector first result.clear(); - // First check staged graph updates using hash maps for O(1) lookup uint64_t lookup_key = makeRepairKey(nodeId, level); - bool foundInStaged = false; - - // Collect incremental neighbor updates for this node/level - vecsim_stl::vector incrementalNeighbors(this->allocator); + bool foundInCache = false; + bool foundOnDisk = false; - // Check staged updates under lock - use shared_lock for read access + // Step 1: Check cache first (source of truth for pending updates) { - std::shared_lock lock(stagedUpdatesGuard); + size_t stripe = getStripeIndex(lookup_key); + auto& cacheStripe = cacheStripes_[stripe]; + std::shared_lock cacheLock(cacheStripe.guard); + auto it = cacheStripe.cache.find(lookup_key); + if (it != cacheStripe.cache.end()) { + result.reserve(it->second.size()); + for (idType id : it->second) { + result.push_back(id); + } + foundInCache = true; + } else if (cacheStripe.newNodes.find(lookup_key) != cacheStripe.newNodes.end()) { + // New node not yet connected - return empty + foundInCache = true; // Treat as found (empty is valid) + } + } - // Check insert staging area (complete neighbor list replacements) - auto insert_it = stagedInsertMap.find(lookup_key); - if (insert_it != stagedInsertMap.end()) { - const auto &update = stagedInsertUpdates[insert_it->second]; + // Step 2: Check delete staging area (for deletion operations) + if (!foundInCache) { + std::shared_lock lock(stagedUpdatesGuard); + auto delete_it = stagedDeleteMap.find(lookup_key); + if (delete_it != stagedDeleteMap.end()) { + const auto &update = stagedDeleteUpdates[delete_it->second]; result.reserve(update.neighbors.size()); for (size_t i = 0; i < update.neighbors.size(); i++) { result.push_back(update.neighbors[i]); } - foundInStaged = true; - } - - // Check delete staging area - if (!foundInStaged) { - auto delete_it = stagedDeleteMap.find(lookup_key); - if (delete_it != stagedDeleteMap.end()) { - const auto &update = stagedDeleteUpdates[delete_it->second]; - result.reserve(update.neighbors.size()); - for (size_t i = 0; i < update.neighbors.size(); i++) { - result.push_back(update.neighbors[i]); - } - foundInStaged = true; - } + foundInCache = true; } // Also check staged repair updates (already cleaned neighbors waiting to be flushed) - if (!foundInStaged) { + if (!foundInCache) { auto repair_it = stagedRepairMap.find(lookup_key); if (repair_it != stagedRepairMap.end()) { const auto &update = stagedRepairUpdates[repair_it->second]; @@ -1861,48 +2028,27 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level for (size_t i = 0; i < update.neighbors.size(); i++) { result.push_back(update.neighbors[i]); } - foundInStaged = true; - } - } - - // Collect incremental neighbor updates (these are additions, not replacements) - // These need to be applied on top of the base neighbor list - for (const auto &update : stagedInsertNeighborUpdates) { - if (update.node_id == nodeId && update.level == level) { - incrementalNeighbors.push_back(update.new_neighbor_id); + foundInCache = true; } } } - // If not found in staged updates, check disk - if (!foundInStaged) { + // Step 3: If not found in cache or staging, read from disk + if (!foundInCache) { GraphKey graphKey(nodeId, level); - std::string graph_value; rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); if (status.ok()) { deserializeGraphValue(graph_value, result); - } - } - - // Apply incremental neighbor updates (add new neighbors that aren't already in the list) - if (!incrementalNeighbors.empty()) { - std::unordered_set existingNeighbors(result.begin(), result.end()); - for (idType neighbor : incrementalNeighbors) { - if (existingNeighbors.find(neighbor) == existingNeighbors.end()) { - result.push_back(neighbor); - existingNeighbors.insert(neighbor); - } + foundOnDisk = true; } } // Filter out deleted nodes and check if any were filtered - if (filterDeletedNodes(result) && !foundInStaged) { + if (filterDeletedNodes(result) && foundOnDisk) { // Lazy repair: if we filtered any deleted nodes from disk data, stage for cleanup - // Use hash map for O(1) duplicate detection - // Note: stagedRepairMap and stagedRepairUpdates are protected by stagedUpdatesGuard std::lock_guard repairLock(stagedUpdatesGuard); uint64_t repair_key = makeRepairKey(nodeId, level); if (stagedRepairMap.find(repair_key) == stagedRepairMap.end()) { @@ -1917,30 +2063,27 @@ void HNSWDiskIndex::getNeighborsAndVector(idType nodeId, siz // Clear the result vector first result.clear(); - // First check staged graph updates (protected by stagedUpdatesGuard) - // Use shared_lock for read-only access and hash map for O(1) lookup - bool foundNeighborsInStaged = false; - vecsim_stl::vector incrementalNeighbors(this->allocator); - { - std::shared_lock lock(stagedUpdatesGuard); - uint64_t lookup_key = makeRepairKey(nodeId, level); - auto it = stagedInsertMap.find(lookup_key); - if (it != stagedInsertMap.end()) { - const auto& update = stagedInsertUpdates[it->second]; - result.reserve(update.neighbors.size()); - for (size_t i = 0; i < update.neighbors.size(); i++) { - result.push_back(update.neighbors[i]); - } - foundNeighborsInStaged = true; - } + uint64_t lookup_key = makeRepairKey(nodeId, level); + bool foundNeighborsInCache = false; - // Collect incremental neighbor updates - for (const auto &update : stagedInsertNeighborUpdates) { - if (update.node_id == nodeId && update.level == level) { - incrementalNeighbors.push_back(update.new_neighbor_id); + // Step 1: Check cache first (source of truth for pending updates) + { + size_t stripe = getStripeIndex(lookup_key); + auto& cacheStripe = cacheStripes_[stripe]; + std::shared_lock cacheLock(cacheStripe.guard); + auto it = cacheStripe.cache.find(lookup_key); + if (it != cacheStripe.cache.end()) { + result.reserve(it->second.size()); + for (idType id : it->second) { + result.push_back(id); } + foundNeighborsInCache = true; + } else if (cacheStripe.newNodes.find(lookup_key) != cacheStripe.newNodes.end()) { + // New node not yet connected - return empty neighbors + foundNeighborsInCache = true; } } + // Check rawVectorsInRAM with shared lock bool foundVectorInRAM = false; { @@ -1959,19 +2102,19 @@ void HNSWDiskIndex::getNeighborsAndVector(idType nodeId, siz foundVectorInRAM = true; } } - if (foundNeighborsInStaged && foundVectorInRAM && incrementalNeighbors.empty()) { + if (foundNeighborsInCache && foundVectorInRAM) { return; } - // If not found in staged updates, check disk with shared lock for thread safety - GraphKey graphKey(nodeId, level); + // If not found in cache, check disk + GraphKey graphKey(nodeId, level); std::string graph_value; rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); if (status.ok()) { - // Only update neighbors if we didn't find them in staged updates - if (!foundNeighborsInStaged) { + // Only update neighbors if we didn't find them in cache + if (!foundNeighborsInCache) { deserializeGraphValue(graph_value, result); } // Only update vector if we didn't find it in RAM @@ -1979,17 +2122,6 @@ void HNSWDiskIndex::getNeighborsAndVector(idType nodeId, siz std::memcpy(vector_data, graph_value.data(), this->inputBlobSize); } } - - // Apply incremental neighbor updates - if (!incrementalNeighbors.empty()) { - std::unordered_set existingNeighbors(result.begin(), result.end()); - for (idType neighbor : incrementalNeighbors) { - if (existingNeighbors.find(neighbor) == existingNeighbors.end()) { - result.push_back(neighbor); - existingNeighbors.insert(neighbor); - } - } - } } template @@ -2007,37 +2139,23 @@ void HNSWDiskIndex::getNeighborsAndVector(labelType nodeId, template size_t HNSWDiskIndex::getNeighborsCount(idType nodeId, size_t level) const { uint64_t lookup_key = makeRepairKey(nodeId, level); - size_t base_count = 0; - bool foundInStaged = false; - size_t incrementalCount = 0; - // Check staged updates under lock for thread safety + // Step 1: Check cache first (source of truth for pending updates) { - std::shared_lock lock(stagedUpdatesGuard); - - // Check insert staging area (complete neighbor list replacements) - auto insert_it = stagedInsertMap.find(lookup_key); - if (insert_it != stagedInsertMap.end()) { - const auto &update = stagedInsertUpdates[insert_it->second]; - base_count = update.neighbors.size(); - foundInStaged = true; + size_t stripe = getStripeIndex(lookup_key); + auto& cacheStripe = cacheStripes_[stripe]; + std::shared_lock cacheLock(cacheStripe.guard); + auto it = cacheStripe.cache.find(lookup_key); + if (it != cacheStripe.cache.end()) { + return it->second.size(); } - - // Count incremental neighbor updates for this node/level - // These are additions on top of the base neighbor list - for (const auto &update : stagedInsertNeighborUpdates) { - if (update.node_id == nodeId && update.level == level) { - incrementalCount++; - } + // Check if this is a new node (never written to disk) + if (cacheStripe.newNodes.find(lookup_key) != cacheStripe.newNodes.end()) { + return 0; // New node not yet connected } } - if (foundInStaged) { - // Return staged count + incremental updates - return base_count + incrementalCount; - } - - // If not found in staged updates, check disk + // Step 2: If not found in cache, check disk GraphKey graphKey(nodeId, level); std::string graph_value; rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); @@ -2046,79 +2164,81 @@ size_t HNSWDiskIndex::getNeighborsCount(idType nodeId, size_ const char* ptr = graph_value.data(); ptr += this->inputBlobSize; size_t neighbor_count = *reinterpret_cast(ptr); - - // Return disk count + incremental updates from staging - return neighbor_count + incrementalCount; + return neighbor_count; } - return incrementalCount; // Only incremental updates if not found elsewhere + return 0; } template bool HNSWDiskIndex::tryAddNeighborIfCapacity( idType nodeId, size_t level, idType newNeighborId, size_t maxCapacity) { - // Atomic check-and-add: under a single lock, check capacity and add if there's room - // This prevents race conditions where multiple threads think there's capacity + // Atomic check-and-add using cache stripes: O(1) lookup, 1/64th lock contention + // The cache is the source of truth - it contains either disk data or pending updates - uint64_t lookup_key = makeRepairKey(nodeId, level); + uint64_t key = makeRepairKey(nodeId, level); + size_t stripe = getStripeIndex(key); + auto& cacheStripe = cacheStripes_[stripe]; - std::unique_lock lock(stagedUpdatesGuard); - - size_t base_count = 0; - bool foundInStaged = false; - size_t incrementalCount = 0; - - // Check insert staging area (complete neighbor list replacements) - auto insert_it = stagedInsertMap.find(lookup_key); - if (insert_it != stagedInsertMap.end()) { - const auto &update = stagedInsertUpdates[insert_it->second]; - base_count = update.neighbors.size(); - foundInStaged = true; - } - - // Count incremental neighbor updates for this node/level - for (const auto &update : stagedInsertNeighborUpdates) { - if (update.node_id == nodeId && update.level == level) { - incrementalCount++; + // First, try with just a shared lock for the common case (already in cache) + { + std::shared_lock readLock(cacheStripe.guard); + auto it = cacheStripe.cache.find(key); + if (it != cacheStripe.cache.end()) { + // Fast path: already in cache, check if we can add + if (it->second.size() >= maxCapacity) { + return false; // Full, no need to upgrade lock + } } } - size_t current_count; - if (foundInStaged) { - current_count = base_count + incrementalCount; - } else { - // Need to check disk (release lock temporarily for I/O, then re-acquire and re-check) - lock.unlock(); + // Need to modify - get exclusive lock + std::unique_lock lock(cacheStripe.guard); - GraphKey graphKey(nodeId, level); - std::string graph_value; - rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + auto it = cacheStripe.cache.find(key); + if (it == cacheStripe.cache.end()) { + // Check if this is a new node (never written to disk) - skip disk lookup + bool isNewNode = (cacheStripe.newNodes.find(key) != cacheStripe.newNodes.end()); - size_t disk_count = 0; - if (status.ok()) { - const char* ptr = graph_value.data(); - ptr += this->inputBlobSize; - disk_count = *reinterpret_cast(ptr); - } + if (!isNewNode) { + // Not in cache and not a new node - need to load from disk + // Keep lock held to avoid multiple threads loading the same data + vecsim_stl::vector diskNeighbors(this->allocator); + GraphKey graphKey(nodeId, level); + std::string graph_value; + rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + if (status.ok()) { + deserializeGraphValue(graph_value, diskNeighbors); + } - // Re-acquire lock and re-count incremental updates (they may have changed) - lock.lock(); - incrementalCount = 0; - for (const auto &update : stagedInsertNeighborUpdates) { - if (update.node_id == nodeId && update.level == level) { - incrementalCount++; + std::vector cacheEntry; + cacheEntry.reserve(diskNeighbors.size()); + for (idType id : diskNeighbors) { + cacheEntry.push_back(id); } + cacheStripe.cache[key] = std::move(cacheEntry); + } else { + // New node - initialize with empty neighbor list + cacheStripe.cache[key] = std::vector(); } - current_count = disk_count + incrementalCount; } - // Now atomically decide and add - if (current_count < maxCapacity) { - stagedInsertNeighborUpdates.emplace_back(nodeId, level, newNeighborId); - return true; // Successfully added + // Now we have the current neighbor list in cache - check capacity + auto& neighbors = cacheStripe.cache[key]; + if (neighbors.size() < maxCapacity) { + // Check for duplicates + if (std::find(neighbors.begin(), neighbors.end(), newNeighborId) == neighbors.end()) { + neighbors.push_back(newNeighborId); + // Mark as dirty and increment atomic counter + auto insertResult = cacheStripe.dirty.insert(key); + if (insertResult.second) { + totalDirtyCount_.fetch_add(1, std::memory_order_relaxed); + } + } + return true; // Successfully added (or already present) } - return false; // Neighbor is full, needs re-evaluation + return false; // Neighbor list is full, needs re-evaluation } template @@ -3209,6 +3329,640 @@ void HNSWDiskIndex::replaceEntryPoint() { maxLevel = 0; } +/********************************** Batchless Mode Implementation **********************************/ + +template +void HNSWDiskIndex::getNeighborsFromCache( + idType nodeId, size_t level, vecsim_stl::vector &result) const { + + result.clear(); + uint64_t key = makeRepairKey(nodeId, level); + size_t stripe = getStripeIndex(key); + auto& cacheStripe = cacheStripes_[stripe]; + + // Step 1: Check in-memory cache (includes pending writes from all jobs) + { + std::shared_lock lock(cacheStripe.guard); + auto it = cacheStripe.cache.find(key); + if (it != cacheStripe.cache.end()) { + // Copy from std::vector to vecsim_stl::vector + result.reserve(it->second.size()); + for (idType id : it->second) { + result.push_back(id); + } + filterDeletedNodes(result); + return; + } + + // Check if this is a new node (never written to disk) - return empty + if (cacheStripe.newNodes.find(key) != cacheStripe.newNodes.end()) { + // New node not yet in cache - return empty (will be populated when connected) + return; + } + } + + // Step 2: Not in cache and not a new node - read from disk + GraphKey graphKey(nodeId, level); + std::string graph_value; + rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + + if (status.ok()) { + deserializeGraphValue(graph_value, result); + } + + // Add to cache for future reads (convert to unique_lock for write) + { + std::unique_lock lock(cacheStripe.guard); + // Double-check: another thread may have populated it + auto it = cacheStripe.cache.find(key); + if (it == cacheStripe.cache.end()) { + // Copy from vecsim_stl::vector to std::vector + std::vector cacheEntry; + cacheEntry.reserve(result.size()); + for (idType id : result) { + cacheEntry.push_back(id); + } + cacheStripe.cache[key] = std::move(cacheEntry); + } + } + + filterDeletedNodes(result); +} + +template +void HNSWDiskIndex::addNeighborToCache( + idType nodeId, size_t level, idType newNeighborId) { + + uint64_t key = makeRepairKey(nodeId, level); + size_t stripe = getStripeIndex(key); + auto& cacheStripe = cacheStripes_[stripe]; + + std::unique_lock lock(cacheStripe.guard); + + auto it = cacheStripe.cache.find(key); + if (it == cacheStripe.cache.end()) { + // Check if this is a new node (never written to disk) - skip disk lookup + bool isNewNode = (cacheStripe.newNodes.find(key) != cacheStripe.newNodes.end()); + + if (!isNewNode) { + // First modification of existing node - need to load current state from disk + lock.unlock(); + vecsim_stl::vector diskNeighbors(this->allocator); + GraphKey graphKey(nodeId, level); + std::string graph_value; + rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + if (status.ok()) { + deserializeGraphValue(graph_value, diskNeighbors); + } + lock.lock(); + + // Re-check: another thread might have populated it + if (cacheStripe.cache.find(key) == cacheStripe.cache.end()) { + // Convert vecsim_stl::vector to std::vector + std::vector cacheEntry; + cacheEntry.reserve(diskNeighbors.size()); + for (idType id : diskNeighbors) { + cacheEntry.push_back(id); + } + cacheStripe.cache[key] = std::move(cacheEntry); + } + } else { + // New node - initialize with empty neighbor list + cacheStripe.cache[key] = std::vector(); + } + } + + // Add new neighbor (avoid duplicates) + auto &neighbors = cacheStripe.cache[key]; + if (std::find(neighbors.begin(), neighbors.end(), newNeighborId) == neighbors.end()) { + neighbors.push_back(newNeighborId); + } + + // Mark as dirty (needs disk write) and increment atomic counter + auto insertResult = cacheStripe.dirty.insert(key); + if (insertResult.second) { // Only increment if newly inserted + totalDirtyCount_.fetch_add(1, std::memory_order_relaxed); + } +} + +template +bool HNSWDiskIndex::tryAddNeighborToCacheIfCapacity( + idType nodeId, size_t level, idType newNeighborId, size_t maxCapacity) { + + uint64_t key = makeRepairKey(nodeId, level); + size_t stripe = getStripeIndex(key); + auto& cacheStripe = cacheStripes_[stripe]; + + std::unique_lock lock(cacheStripe.guard); + + auto it = cacheStripe.cache.find(key); + if (it == cacheStripe.cache.end()) { + // First modification - need to load current state from disk + lock.unlock(); + vecsim_stl::vector diskNeighbors(this->allocator); + GraphKey graphKey(nodeId, level); + std::string graph_value; + rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + if (status.ok()) { + deserializeGraphValue(graph_value, diskNeighbors); + } + lock.lock(); + + // Re-check: another thread might have populated it + if (cacheStripe.cache.find(key) == cacheStripe.cache.end()) { + // Convert vecsim_stl::vector to std::vector + std::vector cacheEntry; + cacheEntry.reserve(diskNeighbors.size()); + for (idType id : diskNeighbors) { + cacheEntry.push_back(id); + } + cacheStripe.cache[key] = std::move(cacheEntry); + } + } + + // Atomic check-and-add under the lock + auto &neighbors = cacheStripe.cache[key]; + + // Check if already present (avoid duplicates) + if (std::find(neighbors.begin(), neighbors.end(), newNeighborId) != neighbors.end()) { + return true; // Already added, consider it success + } + + // Check capacity + if (neighbors.size() >= maxCapacity) { + return false; // No capacity + } + + // Has capacity - add the neighbor + neighbors.push_back(newNeighborId); + auto insertResult = cacheStripe.dirty.insert(key); + if (insertResult.second) { // Only increment if newly inserted + totalDirtyCount_.fetch_add(1, std::memory_order_relaxed); + } + return true; +} + +template +void HNSWDiskIndex::setNeighborsInCache( + idType nodeId, size_t level, const vecsim_stl::vector &neighbors, bool isNewNode) { + + uint64_t key = makeRepairKey(nodeId, level); + size_t stripe = getStripeIndex(key); + auto& cacheStripe = cacheStripes_[stripe]; + + // Convert vecsim_stl::vector to std::vector + std::vector cacheEntry; + cacheEntry.reserve(neighbors.size()); + for (idType id : neighbors) { + cacheEntry.push_back(id); + } + + std::unique_lock lock(cacheStripe.guard); + cacheStripe.cache[key] = std::move(cacheEntry); + + // If this is a new node, track it to avoid disk lookups + if (isNewNode) { + cacheStripe.newNodes.insert(key); + } + + auto insertResult = cacheStripe.dirty.insert(key); + if (insertResult.second) { // Only increment if newly inserted + totalDirtyCount_.fetch_add(1, std::memory_order_relaxed); + } +} + +template +void HNSWDiskIndex::writeVectorToDisk( + idType vectorId, const void *rawVectorData, + const vecsim_stl::vector &neighbors) { + + auto writeOptions = rocksdb::WriteOptions(); + writeOptions.disableWAL = true; + + // Write all levels for this vector + size_t topLevel = idToMetaData[vectorId].topLevel; + rocksdb::WriteBatch batch; + + for (size_t level = 0; level <= topLevel; level++) { + GraphKey graphKey(vectorId, level); + std::string value = serializeGraphValue(rawVectorData, neighbors); + batch.Put(cf, graphKey.asSlice(), value); + } + + db->Write(writeOptions, &batch); +} + +template +void HNSWDiskIndex::writeDirtyNodesToDisk( + const vecsim_stl::vector &modifiedNodes, + const void *newVectorRawData, idType newVectorId) { + + if (modifiedNodes.empty()) { + return; + } + + auto writeOptions = rocksdb::WriteOptions(); + writeOptions.disableWAL = true; + + rocksdb::WriteBatch batch; + std::vector rawVectorBuffer(this->inputBlobSize); + + // Process each modified node - lock only the relevant stripe for each + for (uint64_t key : modifiedNodes) { + idType nodeId = static_cast(key >> 32); + size_t level = static_cast(key & 0xFFFFFFFF); + size_t stripe = getStripeIndex(key); + auto& cacheStripe = cacheStripes_[stripe]; + + // Get neighbors from cache under stripe lock + vecsim_stl::vector neighbors(this->allocator); + { + std::shared_lock stripeLock(cacheStripe.guard); + auto it = cacheStripe.cache.find(key); + if (it == cacheStripe.cache.end()) { + continue; + } + const std::vector &cacheNeighbors = it->second; + neighbors.reserve(cacheNeighbors.size()); + for (idType id : cacheNeighbors) { + neighbors.push_back(id); + } + } + + // Get raw vector data (no cache lock held during disk I/O) + const void *rawData; + if (nodeId == newVectorId) { + rawData = newVectorRawData; + } else { + // Read from RAM or disk + if (!getRawVectorInternal(nodeId, rawVectorBuffer.data())) { + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "WARNING: Could not get raw vector for node %u", nodeId); + continue; + } + rawData = rawVectorBuffer.data(); + } + + GraphKey graphKey(nodeId, level); + std::string value = serializeGraphValue(rawData, neighbors); + batch.Put(cf, graphKey.asSlice(), value); + } + + // Atomic write + rocksdb::Status status = db->Write(writeOptions, &batch); + if (!status.ok()) { + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "ERROR: Failed to write dirty nodes to disk: %s", status.ToString().c_str()); + } + + // Clear dirty flags for written nodes - lock each stripe individually + for (uint64_t key : modifiedNodes) { + size_t stripe = getStripeIndex(key); + auto& cacheStripe = cacheStripes_[stripe]; + std::unique_lock stripeLock(cacheStripe.guard); + if (cacheStripe.dirty.erase(key) > 0) { + totalDirtyCount_.fetch_sub(1, std::memory_order_relaxed); + } + } +} + +template +void HNSWDiskIndex::flushDirtyNodesToDisk() { + std::lock_guard flushLock(diskWriteGuard); + + // Collect all dirty nodes from all stripes + vecsim_stl::vector nodesToFlush(this->allocator); + // Only track level 0 nodes for raw vector removal (level 0 contains the actual vector data) + std::unordered_set level0VectorIds; + + // Check if there are any dirty nodes using atomic counter (fast path) + if (totalDirtyCount_.load(std::memory_order_relaxed) == 0) { + return; + } + + // Collect dirty nodes from all stripes + for (size_t s = 0; s < NUM_CACHE_STRIPES; ++s) { + auto& cacheStripe = cacheStripes_[s]; + std::shared_lock stripeLock(cacheStripe.guard); + for (uint64_t key : cacheStripe.dirty) { + nodesToFlush.push_back(key); + size_t level = static_cast(key & 0xFFFFFFFF); + if (level == 0) { + idType nodeId = static_cast(key >> 32); + level0VectorIds.insert(nodeId); + } + } + } + + if (nodesToFlush.empty()) { + return; + } + + // Build write batch + auto writeOptions = rocksdb::WriteOptions(); + writeOptions.disableWAL = true; + + rocksdb::WriteBatch batch; + std::vector rawVectorBuffer(this->inputBlobSize); + vecsim_stl::vector successfullyFlushed(this->allocator); + std::unordered_set successfulLevel0Ids; + + // Process each node - lock only the relevant stripe + for (uint64_t key : nodesToFlush) { + idType nodeId = static_cast(key >> 32); + size_t level = static_cast(key & 0xFFFFFFFF); + size_t stripe = getStripeIndex(key); + auto& cacheStripe = cacheStripes_[stripe]; + + // Get neighbors from cache under stripe lock + vecsim_stl::vector neighbors(this->allocator); + { + std::shared_lock stripeLock(cacheStripe.guard); + auto it = cacheStripe.cache.find(key); + if (it == cacheStripe.cache.end()) { + continue; + } + const std::vector &cacheNeighbors = it->second; + neighbors.reserve(cacheNeighbors.size()); + for (idType id : cacheNeighbors) { + neighbors.push_back(id); + } + } + + // Get raw vector data from RAM or disk (no cache lock held) + if (!getRawVectorInternal(nodeId, rawVectorBuffer.data())) { + // Skip this node - raw vector not available yet + // This can happen for neighbor updates to nodes that haven't been flushed yet + continue; + } + + GraphKey graphKey(nodeId, level); + std::string value = serializeGraphValue(rawVectorBuffer.data(), neighbors); + batch.Put(cf, graphKey.asSlice(), value); + successfullyFlushed.push_back(key); + if (level == 0) { + successfulLevel0Ids.insert(nodeId); + } + } + + if (successfullyFlushed.empty()) { + return; + } + + // Atomic write + rocksdb::Status status = db->Write(writeOptions, &batch); + if (!status.ok()) { + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "ERROR: Failed to flush dirty nodes to disk: %s", status.ToString().c_str()); + return; // Don't clear dirty flags on failure + } + + // Clear dirty flags and newNodes for successfully written nodes - lock each stripe individually + for (uint64_t key : successfullyFlushed) { + size_t stripe = getStripeIndex(key); + auto& cacheStripe = cacheStripes_[stripe]; + std::unique_lock stripeLock(cacheStripe.guard); + if (cacheStripe.dirty.erase(key) > 0) { + totalDirtyCount_.fetch_sub(1, std::memory_order_relaxed); + } + // Clear newNodes flag - node is now on disk + cacheStripe.newNodes.erase(key); + } + + // Only remove raw vectors for level 0 nodes that were successfully written + // (These are the nodes that now have their vector data on disk) + { + std::lock_guard lock(rawVectorsGuard); + for (idType vectorId : successfulLevel0Ids) { + rawVectorsInRAM.erase(vectorId); + } + } +} + +template +void HNSWDiskIndex::insertElementToGraphBatchless( + idType element_id, size_t element_max_level, idType entry_point, size_t global_max_level, + const void *raw_vector_data, const void *vector_data, + vecsim_stl::vector &modifiedNodes) { + + idType curr_element = entry_point; + DistType cur_dist = std::numeric_limits::max(); + size_t max_common_level; + + if (element_max_level < global_max_level) { + max_common_level = element_max_level; + cur_dist = this->calcDistance(vector_data, getDataByInternalId(curr_element)); + for (auto level = static_cast(global_max_level); + level > static_cast(element_max_level); level--) { + greedySearchLevel(vector_data, level, curr_element, cur_dist); + } + } else { + max_common_level = global_max_level; + } + + for (auto level = static_cast(max_common_level); level >= 0; level--) { + vecsim_stl::updatable_max_heap top_candidates = + searchLayer(curr_element, vector_data, level, efConstruction); + + if (!top_candidates.empty()) { + curr_element = mutuallyConnectNewElementBatchless(element_id, top_candidates, level, + modifiedNodes); + } else { + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "WARNING: No candidates found at level %d!", level); + } + } +} + +template +idType HNSWDiskIndex::mutuallyConnectNewElementBatchless( + idType new_node_id, vecsim_stl::updatable_max_heap &top_candidates, + size_t level, vecsim_stl::vector &modifiedNodes) { + + size_t max_M_cur = level ? M : M0; + + // Copy candidates to list for heuristic processing + candidatesList top_candidates_list(this->allocator); + top_candidates_list.insert(top_candidates_list.end(), top_candidates.begin(), + top_candidates.end()); + + // Use heuristic to filter candidates + idType next_closest_entry_point = getNeighborsByHeuristic2(top_candidates_list, M); + + // Extract selected neighbor IDs + vecsim_stl::vector neighbor_ids(this->allocator); + neighbor_ids.reserve(top_candidates_list.size()); + for (size_t i = 0; i < top_candidates_list.size(); ++i) { + neighbor_ids.push_back(top_candidates_list[i].second); + } + + // Update cache for the new node's neighbors + setNeighborsInCache(new_node_id, level, neighbor_ids); + modifiedNodes.push_back(makeRepairKey(new_node_id, level)); + + // Update existing nodes to include the new node in their neighbor lists + for (const auto &neighbor_data : top_candidates_list) { + idType selected_neighbor = neighbor_data.second; + DistType distance = neighbor_data.first; + + // Use atomic check-and-add to prevent race conditions + if (tryAddNeighborToCacheIfCapacity(selected_neighbor, level, new_node_id, max_M_cur)) { + // Successfully added - mark as modified + modifiedNodes.push_back(makeRepairKey(selected_neighbor, level)); + } else { + // Full - need to re-evaluate using heuristic + // Get current neighbors (might have changed since check) + vecsim_stl::vector existing_neighbors(this->allocator); + getNeighborsFromCache(selected_neighbor, level, existing_neighbors); + + candidatesList candidates(this->allocator); + candidates.reserve(existing_neighbors.size() + 1); + + // Add new node + candidates.emplace_back(distance, new_node_id); + + // Add existing neighbors with their distances + const void *selected_neighbor_data = getDataByInternalId(selected_neighbor); + for (idType existing_neighbor_id : existing_neighbors) { + const void *existing_data = getDataByInternalId(existing_neighbor_id); + DistType existing_dist = this->calcDistance(existing_data, selected_neighbor_data); + candidates.emplace_back(existing_dist, existing_neighbor_id); + } + + // Apply heuristic + vecsim_stl::vector removed_candidates(this->allocator); + getNeighborsByHeuristic2(candidates, max_M_cur, removed_candidates); + + // Extract selected neighbor IDs + vecsim_stl::vector new_neighbors(this->allocator); + new_neighbors.reserve(candidates.size()); + for (const auto &candidate : candidates) { + new_neighbors.push_back(candidate.second); + } + + // Update cache with the new neighbor list + setNeighborsInCache(selected_neighbor, level, new_neighbors); + modifiedNodes.push_back(makeRepairKey(selected_neighbor, level)); + } + } + + return next_closest_entry_point; +} + +template +void HNSWDiskIndex::executeSingleInsertJobWrapper(AsyncJob *job) { + auto *insertJob = static_cast(job); + auto *index = static_cast *>(job->index); + index->executeSingleInsertJob(insertJob); +} + +template +void HNSWDiskIndex::executeSingleInsertJob(HNSWDiskSingleInsertJob *job) { + if (!job->isValid) { + delete job; + return; + } + + const void *raw_vector = job->rawVectorData.data(); + const void *processed_vector = job->processedVectorData.data(); + + // Get current entry point + idType currentEntryPoint; + size_t currentMaxLevel; + { + std::shared_lock lock(indexDataGuard); + currentEntryPoint = entrypointNode; + currentMaxLevel = maxLevel; + } + + if (currentEntryPoint == INVALID_ID || job->vectorId == currentEntryPoint) { + // This is the entry point or no entry point set - nothing to connect + // Still need to remove from rawVectorsInRAM + { + std::lock_guard lock(rawVectorsGuard); + rawVectorsInRAM.erase(job->vectorId); + } + delete job; + return; + } + + // Track modified nodes for disk write + vecsim_stl::vector modifiedNodes(this->allocator); + + // Insert into graph using batchless method (updates cache immediately) + insertElementToGraphBatchless(job->vectorId, job->elementMaxLevel, currentEntryPoint, + currentMaxLevel, raw_vector, processed_vector, modifiedNodes); + + // Check if we should flush dirty nodes to disk (batched disk writes for performance) + // Use atomic counter for fast threshold check without locking + bool shouldFlush = (diskWriteBatchThreshold > 0 && + totalDirtyCount_.load(std::memory_order_relaxed) >= diskWriteBatchThreshold); + + if (shouldFlush) { + flushDirtyNodesToDisk(); + } + + // Update entry point if this vector has higher level + if (job->elementMaxLevel > currentMaxLevel) { + std::unique_lock lock(indexDataGuard); + if (job->elementMaxLevel > maxLevel) { + entrypointNode = job->vectorId; + maxLevel = job->elementMaxLevel; + } + } + + delete job; +} + +template +void HNSWDiskIndex::executeInsertAndWriteSynchronously( + idType vectorId, size_t elementMaxLevel, const void *rawVectorData, + const void *processedVectorData) { + + // Get current entry point + idType currentEntryPoint; + size_t currentMaxLevel; + { + std::shared_lock lock(indexDataGuard); + currentEntryPoint = entrypointNode; + currentMaxLevel = maxLevel; + } + + if (currentEntryPoint == INVALID_ID || vectorId == currentEntryPoint) { + // Still need to remove from rawVectorsInRAM + { + std::lock_guard lock(rawVectorsGuard); + rawVectorsInRAM.erase(vectorId); + } + return; + } + + // Track modified nodes + vecsim_stl::vector modifiedNodes(this->allocator); + + // Insert into graph + insertElementToGraphBatchless(vectorId, elementMaxLevel, currentEntryPoint, currentMaxLevel, + rawVectorData, processedVectorData, modifiedNodes); + + // Write to disk + writeDirtyNodesToDisk(modifiedNodes, rawVectorData, vectorId); + + // Remove raw vector from RAM now that it's on disk + { + std::lock_guard lock(rawVectorsGuard); + rawVectorsInRAM.erase(vectorId); + } + + // Update entry point if needed + if (elementMaxLevel > currentMaxLevel) { + std::unique_lock lock(indexDataGuard); + if (elementMaxLevel > maxLevel) { + entrypointNode = vectorId; + maxLevel = elementMaxLevel; + } + } +} + #ifdef BUILD_TESTS #include "hnsw_disk_serializer.h" #endif diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h index 12830e895..f27aa6091 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h @@ -62,7 +62,8 @@ HNSWDiskIndex::HNSWDiskIndex( stagedInsertUpdates(this->allocator), stagedDeleteUpdates(this->allocator), stagedRepairUpdates(this->allocator), stagedInsertNeighborUpdates(this->allocator), - jobQueue(nullptr), jobQueueCtx(nullptr), SubmitJobsToQueue(nullptr) { + jobQueue(nullptr), jobQueueCtx(nullptr), SubmitJobsToQueue(nullptr), + cacheStripes_(new CacheStripe[NUM_CACHE_STRIPES]) { // Restore index fields from file (including batchThreshold) this->restoreIndexFields(input); diff --git a/src/VecSim/vec_sim_common.h b/src/VecSim/vec_sim_common.h index 30fed7652..82359f62f 100644 --- a/src/VecSim/vec_sim_common.h +++ b/src/VecSim/vec_sim_common.h @@ -258,6 +258,7 @@ typedef enum { SVS_BATCH_UPDATE_JOB, HNSW_DISK_INSERT_VECTOR_JOB, HNSW_DISK_FLUSH_UPDATES_JOB, + HNSW_DISK_SINGLE_INSERT_JOB, // Batchless: single vector insert from start to disk write INVALID_JOB // to indicate that finding a JobType >= INVALID_JOB is an error } JobType; diff --git a/tests/unit/test_hnsw_disk.cpp b/tests/unit/test_hnsw_disk.cpp index 86fb79b4e..f41d3491f 100644 --- a/tests/unit/test_hnsw_disk.cpp +++ b/tests/unit/test_hnsw_disk.cpp @@ -1762,112 +1762,3 @@ TEST_F(HNSWDiskIndexTest, GraphRepairWithHeuristic) { ASSERT_GT(successful_queries, (n - deleted_labels.size()) / 2) << "Too many queries failed - graph may be disconnected"; } - -TEST_F(HNSWDiskIndexTest, MultiThreadedInsertTest) { - // Test multi-threaded insertion using mock thread pool (similar to tiered index tests) - const size_t dim = 64; - const size_t n = 100; - - // Create HNSW parameters - HNSWParams params; - params.dim = dim; - params.type = VecSimType_FLOAT32; - params.metric = VecSimMetric_L2; - params.multi = false; - params.M = 16; - params.efConstruction = 100; - params.efRuntime = 50; - params.epsilon = 0.01; - - // Create abstract init parameters - AbstractIndexInitParams abstractInitParams; - abstractInitParams.dim = dim; - abstractInitParams.vecType = params.type; - abstractInitParams.dataSize = dim * sizeof(int8_t); - abstractInitParams.blockSize = 1; - abstractInitParams.multi = false; - abstractInitParams.allocator = VecSimAllocator::newVecsimAllocator(); - - // Create index components - IndexComponents components = CreateQuantizedIndexComponents( - abstractInitParams.allocator, VecSimMetric_L2, dim, false); - - // Create mock thread pool (similar to tiered index tests) - auto mock_thread_pool = tieredIndexMock(); - mock_thread_pool.thread_pool_size = 4; - - // Create HNSWDiskIndex with job queue parameters - rocksdb::ColumnFamilyHandle *default_cf = db->DefaultColumnFamily(); - auto *index = new (abstractInitParams.allocator) HNSWDiskIndex( - ¶ms, abstractInitParams, components, db.get(), default_cf, "", 100, - &mock_thread_pool.jobQ, mock_thread_pool.ctx, tieredIndexMock::submit_callback); - - // Set the index in the mock thread pool context (required for job execution) - // This is similar to how TieredFactory::NewIndex sets up the index - mock_thread_pool.ctx->index_strong_ref.reset(index); - - // Set batch threshold high enough so we control when batch is processed - index->setBatchThreshold(n + 10); - - // Create test vectors and add them - std::mt19937 rng(42); - std::vector> vectors; - for (size_t i = 0; i < n; i++) { - vectors.push_back(createRandomVector(dim, rng)); - normalizeVector(vectors.back()); - } - - // Add all vectors (no batch processing triggered due to high threshold) - for (size_t i = 0; i < n; i++) { - int result = index->addVector(vectors[i].data(), i); - EXPECT_EQ(result, 1); - } - - // Verify vectors are pending - EXPECT_EQ(index->indexSize(), n); - - // Launch the BG threads loop - mock_thread_pool.init_threads(); - - // Now trigger batch processing - index->flushBatch(); - - // Wait for all jobs to complete - mock_thread_pool.thread_pool_join(); - - // Verify all vectors are in the index - EXPECT_EQ(index->indexSize(), n); - EXPECT_EQ(index->indexLabelCount(), n); - EXPECT_EQ(mock_thread_pool.jobQ.size(), 0); - - // Verify search works correctly - VecSimQueryParams queryParams; - queryParams.hnswRuntimeParams.efRuntime = 50; - - size_t successful_queries = 0; - for (size_t i = 0; i < n; i++) { - auto results = index->topKQuery(vectors[i].data(), 5, &queryParams); - ASSERT_TRUE(results != nullptr); - ASSERT_EQ(results->code, VecSim_OK); - - if (results->results.size() > 0) { - successful_queries++; - // The query vector should be in the results (exact match) - bool found_self = false; - for (const auto &result : results->results) { - if (result.id == i) { - found_self = true; - break; - } - } - EXPECT_TRUE(found_self) << "Vector " << i << " not found in its own query results"; - } - - delete results; - } - - // Most queries should succeed - EXPECT_GT(successful_queries, n * 0.9) << "Too many queries failed"; - - // Note: The mock_thread_pool destructor will clean up the index via index_strong_ref -} From ed5b84f9922aab3d1e5fd919dd40cd3f68ee9dca Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Sun, 21 Dec 2025 15:55:58 +0200 Subject: [PATCH 16/34] fix unit_test --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 165 +++++++++++-------------- tests/unit/test_hnsw_disk.cpp | 48 ++++--- 2 files changed, 106 insertions(+), 107 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index f1eead7f9..9b17762df 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -368,13 +368,14 @@ class HNSWDiskIndex : public VecSimIndexAbstract static constexpr size_t NUM_CACHE_STRIPES = 64; // Power of 2 for efficient modulo // Cache stripe structure - each stripe is cache-line aligned to prevent false sharing - struct CacheStripe { + struct alignas(64) CacheStripe { std::shared_mutex guard; std::unordered_map> cache; std::unordered_set dirty; // Track nodes created in current batch (never written to disk yet) // This helps avoid disk lookups for new nodes std::unordered_set newNodes; + char padding[64]; // Ensure no overlap CacheStripe() = default; CacheStripe(const CacheStripe&) = delete; @@ -485,9 +486,12 @@ class HNSWDiskIndex : public VecSimIndexAbstract void writeDirtyNodesToDisk(const vecsim_stl::vector &modifiedNodes, const void *newVectorRawData, idType newVectorId); - // Synchronous fallback for batchless mode without job queue - void executeInsertAndWriteSynchronously(idType vectorId, size_t elementMaxLevel, - const void *rawVectorData, const void *processedVectorData); + // Unified core function for graph insertion - used by both single-threaded and multi-threaded paths + // immediateFlush: true = write to disk immediately (single-threaded), false = batched writes (multi-threaded) + void executeGraphInsertionCore(idType vectorId, size_t elementMaxLevel, + idType entryPoint, size_t globalMaxLevel, + const void *rawVectorData, const void *processedVectorData, + bool immediateFlush); // Helper to write a single vector's entry to disk void writeVectorToDisk(idType vectorId, const void *rawVectorData, @@ -517,6 +521,8 @@ class HNSWDiskIndex : public VecSimIndexAbstract inline bool filterDeletedNodes(vecsim_stl::vector& neighbors) const { size_t original_size = neighbors.size(); size_t elementCount = curElementCount.load(std::memory_order_acquire); + // Hold shared lock to prevent idToMetaData resize during access + std::shared_lock lock(indexDataGuard); size_t metadataSize = idToMetaData.size(); auto new_end = std::remove_if(neighbors.begin(), neighbors.end(), [this, elementCount, metadataSize](idType id) { @@ -568,7 +574,7 @@ class HNSWDiskIndex : public VecSimIndexAbstract DistType dist, idType id) const; template - void processCandidate(Identifier candidate_id, const void *data_point, size_t level, size_t ef, + void processCandidate(idType curNodeId, const void *data_point, size_t level, size_t ef, std::unordered_set *visited_set, vecsim_stl::updatable_max_heap &top_candidates, candidatesMaxHeap &candidate_set, DistType &lowerBound) const; @@ -1068,9 +1074,11 @@ int HNSWDiskIndex::addVector( // Each vector is processed immediately and written to disk // Get entry point info idType currentEntryPoint; + size_t currentMaxLevel; { std::shared_lock lock(indexDataGuard); currentEntryPoint = entrypointNode; + currentMaxLevel = maxLevel; } // Handle first vector (becomes entry point) @@ -1099,7 +1107,7 @@ int HNSWDiskIndex::addVector( // Check if we have a job queue for async processing if (SubmitJobsToQueue != nullptr) { - // Create job with vector data copies + // Multi-threaded: submit job for async processing std::string rawVectorCopy(raw_data, this->inputBlobSize); std::string processedVectorCopy( reinterpret_cast(processedBlobs.getStorageBlob()), @@ -1112,9 +1120,10 @@ int HNSWDiskIndex::addVector( submitSingleJob(job); } else { - // Synchronous fallback - executeInsertAndWriteSynchronously(newElementId, elementMaxLevel, vector, - processedBlobs.getStorageBlob()); + // Single-threaded: execute inline with immediate disk write + executeGraphInsertionCore(newElementId, elementMaxLevel, currentEntryPoint, + currentMaxLevel, vector, processedBlobs.getStorageBlob(), + true /* immediateFlush */); } return 1; @@ -1709,7 +1718,9 @@ HNSWDiskIndex::searchLayerLabels(idType ep_id, const void *d } candidate_set.pop(); - processCandidate(getExternalLabel(curr_el_pair.second), data_point, layer, ef, + // Pass internal ID to processCandidate - getNeighbors expects idType, not labelType + // The emplaceHeap overload handles conversion to labelType for top_candidates + processCandidate(curr_el_pair.second, data_point, layer, ef, &visited_set, top_candidates, candidate_set, lowerBound); } @@ -1859,7 +1870,7 @@ size_t HNSWDiskIndex::getRandomLevel(double reverse_size) { template template void HNSWDiskIndex::processCandidate( - Identifier curNodeId, const void *data_point, size_t level, size_t ef, std::unordered_set *visited_set, + idType curNodeId, const void *data_point, size_t level, size_t ef, std::unordered_set *visited_set, vecsim_stl::updatable_max_heap &top_candidates, candidatesMaxHeap &candidate_set, DistType &lowerBound) const { assert(visited_set != nullptr); @@ -1954,7 +1965,9 @@ size_t HNSWDiskIndex::indexCapacity() const { template size_t HNSWDiskIndex::indexSize() const { - return this->curElementCount - this->numMarkedDeleted; + // Return active element count (curElementCount never decreases in disk mode, + // so we subtract numMarkedDeleted to get the actual number of active elements) + return curElementCount.load(std::memory_order_acquire) - this->numMarkedDeleted; } template @@ -1986,7 +1999,6 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level uint64_t lookup_key = makeRepairKey(nodeId, level); bool foundInCache = false; - bool foundOnDisk = false; // Step 1: Check cache first (source of truth for pending updates) { @@ -2042,20 +2054,15 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level if (status.ok()) { deserializeGraphValue(graph_value, result); - foundOnDisk = true; } } - // Filter out deleted nodes and check if any were filtered - if (filterDeletedNodes(result) && foundOnDisk) { - // Lazy repair: if we filtered any deleted nodes from disk data, stage for cleanup - std::lock_guard repairLock(stagedUpdatesGuard); - uint64_t repair_key = makeRepairKey(nodeId, level); - if (stagedRepairMap.find(repair_key) == stagedRepairMap.end()) { - stagedRepairMap[repair_key] = stagedRepairUpdates.size(); - stagedRepairUpdates.emplace_back(nodeId, level, result, this->allocator); - } - } + // Note: We do NOT filter deleted nodes here. During search, we need to explore + // through deleted nodes to find non-deleted neighbors. The filtering happens in + // processCandidate when adding to top_candidates (via isMarkedDeletedUnsafe check). + // + // Lazy repair for deleted nodes is handled separately when we detect stale edges + // during graph maintenance operations, not during search. } template @@ -3284,9 +3291,10 @@ void HNSWDiskIndex::replaceEntryPoint() { idType old_entry_point_id = entrypointNode; size_t currentMaxLevel = maxLevel; - // Try to find a new entrypoint at the current max level - while (currentMaxLevel > 0) { - // First, try to find a neighbor of the old entrypoint at the top level + // Try to find a new entrypoint at the current max level (including level 0) + // Use a do-while or check >= 0 to include level 0 + while (true) { + // First, try to find a neighbor of the old entrypoint at this level GraphKey graphKey(old_entry_point_id, currentMaxLevel); std::string graph_value; rocksdb::Status status = @@ -3299,7 +3307,8 @@ void HNSWDiskIndex::replaceEntryPoint() { // Try to find a non-deleted neighbor for (size_t i = 0; i < neighbors.size(); i++) { - if (!(idToMetaData[neighbors[i]].flags & DELETE_MARK)) { + if (neighbors[i] < idToMetaData.size() && + !(idToMetaData[neighbors[i]].flags & DELETE_MARK)) { entrypointNode = neighbors[i]; maxLevel = currentMaxLevel; return; @@ -3312,7 +3321,7 @@ void HNSWDiskIndex::replaceEntryPoint() { for (idType id = 0; id < elementCount; id++) { if (id != old_entry_point_id && id < idToMetaData.size() && idToMetaData[id].label != INVALID_LABEL && - idToMetaData[id].topLevel == currentMaxLevel && + idToMetaData[id].topLevel >= currentMaxLevel && !(idToMetaData[id].flags & DELETE_MARK)) { entrypointNode = id; maxLevel = currentMaxLevel; @@ -3321,6 +3330,7 @@ void HNSWDiskIndex::replaceEntryPoint() { } // No non-deleted nodes at this level, decrease maxLevel and try again + if (currentMaxLevel == 0) break; currentMaxLevel--; } @@ -3863,9 +3873,6 @@ void HNSWDiskIndex::executeSingleInsertJob(HNSWDiskSingleIns return; } - const void *raw_vector = job->rawVectorData.data(); - const void *processed_vector = job->processedVectorData.data(); - // Get current entry point idType currentEntryPoint; size_t currentMaxLevel; @@ -3875,86 +3882,60 @@ void HNSWDiskIndex::executeSingleInsertJob(HNSWDiskSingleIns currentMaxLevel = maxLevel; } - if (currentEntryPoint == INVALID_ID || job->vectorId == currentEntryPoint) { - // This is the entry point or no entry point set - nothing to connect - // Still need to remove from rawVectorsInRAM - { - std::lock_guard lock(rawVectorsGuard); - rawVectorsInRAM.erase(job->vectorId); - } - delete job; - return; - } - - // Track modified nodes for disk write - vecsim_stl::vector modifiedNodes(this->allocator); - - // Insert into graph using batchless method (updates cache immediately) - insertElementToGraphBatchless(job->vectorId, job->elementMaxLevel, currentEntryPoint, - currentMaxLevel, raw_vector, processed_vector, modifiedNodes); - - // Check if we should flush dirty nodes to disk (batched disk writes for performance) - // Use atomic counter for fast threshold check without locking - bool shouldFlush = (diskWriteBatchThreshold > 0 && - totalDirtyCount_.load(std::memory_order_relaxed) >= diskWriteBatchThreshold); - - if (shouldFlush) { - flushDirtyNodesToDisk(); - } - - // Update entry point if this vector has higher level - if (job->elementMaxLevel > currentMaxLevel) { - std::unique_lock lock(indexDataGuard); - if (job->elementMaxLevel > maxLevel) { - entrypointNode = job->vectorId; - maxLevel = job->elementMaxLevel; - } - } + // Use unified core function with batched flush (multi-threaded mode) + executeGraphInsertionCore(job->vectorId, job->elementMaxLevel, currentEntryPoint, + currentMaxLevel, job->rawVectorData.data(), + job->processedVectorData.data(), + false /* immediateFlush = batched */); delete job; } template -void HNSWDiskIndex::executeInsertAndWriteSynchronously( - idType vectorId, size_t elementMaxLevel, const void *rawVectorData, - const void *processedVectorData) { - - // Get current entry point - idType currentEntryPoint; - size_t currentMaxLevel; - { - std::shared_lock lock(indexDataGuard); - currentEntryPoint = entrypointNode; - currentMaxLevel = maxLevel; - } +void HNSWDiskIndex::executeGraphInsertionCore( + idType vectorId, size_t elementMaxLevel, + idType entryPoint, size_t globalMaxLevel, + const void *rawVectorData, const void *processedVectorData, + bool immediateFlush) { - if (currentEntryPoint == INVALID_ID || vectorId == currentEntryPoint) { - // Still need to remove from rawVectorsInRAM - { + if (entryPoint == INVALID_ID || vectorId == entryPoint) { + // Entry point or first vector - nothing to connect + if (immediateFlush) { std::lock_guard lock(rawVectorsGuard); rawVectorsInRAM.erase(vectorId); } return; } - // Track modified nodes + // Track modified nodes for disk write vecsim_stl::vector modifiedNodes(this->allocator); - // Insert into graph - insertElementToGraphBatchless(vectorId, elementMaxLevel, currentEntryPoint, currentMaxLevel, - rawVectorData, processedVectorData, modifiedNodes); + // Hold shared lock during graph insertion to prevent idToMetaData resize + // while we're reading from it in searchLayer/processCandidate + { + std::shared_lock lock(indexDataGuard); + // Insert into graph using cache-based method + insertElementToGraphBatchless(vectorId, elementMaxLevel, entryPoint, globalMaxLevel, + rawVectorData, processedVectorData, modifiedNodes); + } - // Write to disk - writeDirtyNodesToDisk(modifiedNodes, rawVectorData, vectorId); + // Handle disk writes based on mode + if (immediateFlush) { + // Single-threaded: write immediately + writeDirtyNodesToDisk(modifiedNodes, rawVectorData, vectorId); - // Remove raw vector from RAM now that it's on disk - { std::lock_guard lock(rawVectorsGuard); rawVectorsInRAM.erase(vectorId); + } else { + // Multi-threaded: batched writes based on threshold + if (diskWriteBatchThreshold > 0 && + totalDirtyCount_.load(std::memory_order_relaxed) >= diskWriteBatchThreshold) { + flushDirtyNodesToDisk(); + } } - // Update entry point if needed - if (elementMaxLevel > currentMaxLevel) { + // Update entry point if this vector has higher level + if (elementMaxLevel > globalMaxLevel) { std::unique_lock lock(indexDataGuard); if (elementMaxLevel > maxLevel) { entrypointNode = vectorId; diff --git a/tests/unit/test_hnsw_disk.cpp b/tests/unit/test_hnsw_disk.cpp index f41d3491f..790c41ea3 100644 --- a/tests/unit/test_hnsw_disk.cpp +++ b/tests/unit/test_hnsw_disk.cpp @@ -943,18 +943,28 @@ TEST_F(HNSWDiskIndexTest, markDelete) { } ASSERT_EQ(index.getNumMarkedDeleted(), n / 2); - ASSERT_EQ(index.indexSize(), n); + // indexSize() returns active elements (curElementCount - numMarkedDeleted) in disk mode + ASSERT_EQ(index.indexSize(), n / 2); // Search for k results around the middle. Expect to find only non-deleted results. auto verify_res_half = [&](size_t id, double score, size_t result_index) { + // Verify the result is not from the deleted set ASSERT_NE(id % 2, ep_reminder); size_t diff_id = (id > 50) ? (id - 50) : (50 - id); // The results alternate between below and above 50, with pairs at the same distance - // Pattern: 49,51 (diff=1), 47,53 (diff=3), 45,55 (diff=5), etc. - // So expected_diff = result_index | 1 (make it odd) - size_t expected_diff = result_index | 1; + // If ep_reminder == 0 (deleted even), remaining are odd: 49,51 (diff=1), 47,53 (diff=3), etc. + // Pattern: expected_diff = (result_index + 1) | 1 = make it odd, starting from 1 + // If ep_reminder == 1 (deleted odd), remaining are even: 50 (diff=0), 48,52 (diff=2), etc. + // Pattern: expected_diff = ((result_index + 1) / 2) * 2 = pairs of even numbers + size_t expected_diff; + if (ep_reminder == 0) { + // Deleted even, remaining odd + expected_diff = (result_index + 1) | 1; + } else { + // Deleted odd, remaining even + expected_diff = ((result_index + 1) / 2) * 2; + } ASSERT_EQ(diff_id, expected_diff); - // ASSERT_EQ(score, (dim * expected_diff * expected_diff)); }; // Run search test after marking deleted @@ -986,7 +996,11 @@ TEST_F(HNSWDiskIndexTest, markDelete) { } } - ASSERT_EQ(index.indexSize(), n + n / 2 + 1); + // indexSize() returns active elements (curElementCount - numMarkedDeleted) + // curElementCount = n + 1 + n/2 = 151 (original + 1 new + re-added) + // numMarkedDeleted = n/2 = 50 + // indexSize() = 151 - 50 = 101 = n + 1 + ASSERT_EQ(index.indexSize(), n + 1); ASSERT_EQ(index.getNumMarkedDeleted(), n / 2); // Search for k results around the middle again. Expect to find the same results we @@ -1063,8 +1077,9 @@ TEST_F(HNSWDiskIndexTest, BatchedDeletionTest) { index.flushDeleteBatch(); // Verify the index size and label count - // Note: indexSize includes marked deleted vectors - ASSERT_EQ(index.indexSize(), n); + // indexSize() returns active elements (curElementCount - numMarkedDeleted) + // deleted_count = n/2 + ASSERT_EQ(index.indexSize(), n - deleted_count); ASSERT_EQ(index.indexLabelCount(), n - deleted_count); // Verify that deleted vectors cannot be deleted again (they don't exist) @@ -1189,10 +1204,11 @@ TEST_F(HNSWDiskIndexTest, InterleavedInsertDeleteTest) { index.flushDeleteBatch(); // Verify index state - // indexSize() returns curElementCount (highest ID + 1) - // Without ID recycling, curElementCount grows with each insertion. - // After initial_count insertions + insert_count new insertions = initial_count + insert_count - ASSERT_EQ(index.indexSize(), initial_count + insert_count); + // indexSize() returns active elements (curElementCount - numMarkedDeleted) + // curElementCount = initial_count + insert_count (no ID recycling) + // numMarkedDeleted = delete_count + // indexSize() = initial_count + insert_count - delete_count + ASSERT_EQ(index.indexSize(), initial_count + insert_count - delete_count); // indexLabelCount() returns labelToIdMap.size() which reflects active (non-deleted) labels // So it should be: initial_count - delete_count + insert_count = 100 - 20 + 20 = 100 ASSERT_EQ(index.indexLabelCount(), initial_count - delete_count + insert_count); @@ -1260,9 +1276,11 @@ TEST_F(HNSWDiskIndexTest, InterleavedInsertDeleteTest) { index.flushDeleteBatch(); // Final verification - // Without ID recycling, indexSize() grows with each insertion. - // Total insertions: initial_count + 20 (Phase 2) + 10 (Phase 6) = initial_count + 30 - ASSERT_EQ(index.indexSize(), initial_count + 30); + // indexSize() returns active elements (curElementCount - numMarkedDeleted) + // curElementCount = initial_count + 30 (total insertions) + // numMarkedDeleted = 30 (total deletions) + // indexSize() = initial_count + 30 - 30 = initial_count + ASSERT_EQ(index.indexSize(), initial_count); // indexLabelCount() = initial_count - total_deletes + total_inserts = 100 - 30 + 30 = 100 size_t expected_label_count = initial_count - 30 + 30; // deleted 30 total, added 30 total ASSERT_EQ(index.indexLabelCount(), expected_label_count); From 42bfed10244632884e9653280a3605c300bb45b0 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Sun, 21 Dec 2025 17:50:27 +0200 Subject: [PATCH 17/34] diskWriteBatchThreshold in serialize --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 3 +- .../data/scripts/hnsw_disk_serializer.cpp | 77 +++++++------------ 2 files changed, 28 insertions(+), 52 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 9b17762df..336c21a22 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -33,6 +33,7 @@ #include "VecSim/spaces/computer/preprocessor_container.h" #include "VecSim/spaces/computer/preprocessors.h" #include "VecSim/algorithms/hnsw/visited_nodes_handler.h" +#include "VecSim/algorithms/hnsw/sparse_visited_set.h" #include "VecSim/algorithms/hnsw/hnsw.h" // For HNSWAddVectorState definition #include "VecSim/utils/updatable_heap.h" @@ -1912,8 +1913,6 @@ void HNSWDiskIndex::processCandidate( } } - - template VecSimQueryReply * HNSWDiskIndex::rangeQuery(const void *query_data, double radius, diff --git a/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp b/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp index 3060f16ee..d914673ab 100644 --- a/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp +++ b/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp @@ -5,7 +5,7 @@ * It supports both .raw files (no header) and .fbin files (with header). * * Usage: - * ./hnsw_disk_serializer [M] [efConstruction] [threads] + * ./hnsw_disk_serializer [M] [efConstruction] [threads] [diskWriteBatchThreshold] * * Arguments: * input_file - Binary file containing vectors (.raw or .fbin) @@ -18,6 +18,7 @@ * M - HNSW M parameter (default: 64) * efConstruction - HNSW efConstruction parameter (default: 512) * threads - Number of threads for parallel indexing (default: 4, use 0 for single-threaded) + * diskWriteBatchThreshold - Threshold for disk write batching (default: 1, larger = fewer disk writes) * * Examples: * # Using .raw file (dimension required) @@ -267,7 +268,7 @@ void saveIndexByType(VecSimIndex *index, const std::string &output_file) { int main(int argc, char *argv[]) { if (argc < 6) { - std::cerr << "Usage: " << argv[0] << " [M] [efConstruction] [threads]\n"; + std::cerr << "Usage: " << argv[0] << " [M] [efConstruction] [threads] [diskWriteBatchThreshold]\n"; std::cerr << "\nArguments:\n"; std::cerr << " input_file - Binary file (.raw or .fbin)\n"; std::cerr << " output_name - Base name for output files\n"; @@ -277,6 +278,7 @@ int main(int argc, char *argv[]) { std::cerr << " M - HNSW M parameter (default: 64)\n"; std::cerr << " efConstruction - HNSW efConstruction parameter (default: 512)\n"; std::cerr << " threads - Number of threads for parallel indexing (default: 4, use 0 for single-threaded)\n"; + std::cerr << " diskWriteBatchThreshold - Threshold for disk write batching (default: 1)\n"; return 1; } @@ -288,6 +290,7 @@ int main(int argc, char *argv[]) { size_t M = (argc > 6) ? std::stoull(argv[6]) : 64; size_t efConstruction = (argc > 7) ? std::stoull(argv[7]) : 512; size_t num_threads = (argc > 8) ? std::stoull(argv[8]) : 4; + size_t disk_write_batch_threshold = (argc > 9) ? std::stoull(argv[9]) : 1; // Check if input file exists if (!std::filesystem::exists(input_file)) { @@ -346,6 +349,7 @@ int main(int argc, char *argv[]) { std::cout << "M: " << M << "\n"; std::cout << "efConstruction: " << efConstruction << "\n"; std::cout << "Threads: " << (num_threads > 0 ? std::to_string(num_threads) : "single-threaded") << "\n"; + std::cout << "DiskWriteBatchThreshold: " << disk_write_batch_threshold << "\n"; std::cout << "Number of vectors: " << num_vectors << "\n"; std::cout << "==================================\n\n"; @@ -393,53 +397,16 @@ int main(int argc, char *argv[]) { mock_thread_pool->init_threads(); // Configure the disk index to use the job queue - // With sibling injection, larger batches work well (vectors in batch can discover each other) - const size_t bulk_batch_threshold = 10; - if (type == VecSimType_FLOAT32) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, - tieredIndexMock::submit_callback); - disk_index->setBatchThreshold(bulk_batch_threshold); - } - } else if (type == VecSimType_FLOAT64) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, - tieredIndexMock::submit_callback); - disk_index->setBatchThreshold(bulk_batch_threshold); - } - } else if (type == VecSimType_BFLOAT16) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, - tieredIndexMock::submit_callback); - disk_index->setBatchThreshold(bulk_batch_threshold); - } - } else if (type == VecSimType_FLOAT16) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, - tieredIndexMock::submit_callback); - disk_index->setBatchThreshold(bulk_batch_threshold); - } - } else if (type == VecSimType_INT8) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, - tieredIndexMock::submit_callback); - disk_index->setBatchThreshold(bulk_batch_threshold); - } - } else if (type == VecSimType_UINT8) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, - tieredIndexMock::submit_callback); - disk_index->setBatchThreshold(bulk_batch_threshold); - } + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->setJobQueue(&mock_thread_pool->jobQ, mock_thread_pool->ctx, + tieredIndexMock::submit_callback); + // Set disk write batch threshold for better performance + // Larger batches = fewer disk writes = faster indexing + disk_index->setDiskWriteBatchThreshold(disk_write_batch_threshold); } + std::cout << "Multi-threaded indexing enabled with " << num_threads << " threads\n"; - std::cout << "Batch threshold set to " << bulk_batch_threshold << "\n"; } std::cout << "Index created successfully\n"; @@ -477,8 +444,8 @@ int main(int argc, char *argv[]) { if (mock_thread_pool->isIdle()) { break; } - - // Print progress every second + + // Print progress every 1 seconds size_t current_indexed = VecSimIndex_IndexSize(index); size_t pending = 0; size_t processing = 0; @@ -495,7 +462,7 @@ int main(int argc, char *argv[]) { auto elapsed = std::chrono::duration_cast( std::chrono::steady_clock::now() - start_time).count(); - if (current_indexed != last_indexed || elapsed % 5 == 0) { + if (current_indexed != last_indexed || elapsed % 1 == 0) { std::cout << "\rIndexed: " << current_indexed << "/" << num_vectors << " | Pending: " << pending << " | Processing: " << processing @@ -561,6 +528,16 @@ int main(int argc, char *argv[]) { mock_thread_pool->thread_pool_wait(); std::cout << "All background jobs completed.\n"; + // Flush any remaining dirty nodes to disk + std::cout << "Flushing remaining dirty nodes to disk...\n"; + + auto *disk_index = dynamic_cast *>(index); + if (disk_index) { + disk_index->flushDirtyNodesToDisk(); + } + + std::cout << "Dirty nodes flushed.\n"; + // Stop the thread pool before saving mock_thread_pool->thread_pool_join(); } From 4b274a8b65fb108aac54171b2eb3058c4b87f37a Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Sun, 21 Dec 2025 17:52:05 +0200 Subject: [PATCH 18/34] remove the sparse --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 336c21a22..52bbd8459 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -33,7 +33,6 @@ #include "VecSim/spaces/computer/preprocessor_container.h" #include "VecSim/spaces/computer/preprocessors.h" #include "VecSim/algorithms/hnsw/visited_nodes_handler.h" -#include "VecSim/algorithms/hnsw/sparse_visited_set.h" #include "VecSim/algorithms/hnsw/hnsw.h" // For HNSWAddVectorState definition #include "VecSim/utils/updatable_heap.h" From 256c7c7a9a5d87b25b4a6dda8284914df8e43764 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Mon, 22 Dec 2025 09:44:07 +0200 Subject: [PATCH 19/34] Remove unused functions --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 827 ++++-------------- .../algorithms/hnsw/hnsw_disk_serializer.h | 34 +- .../data/scripts/hnsw_disk_serializer.cpp | 63 +- tests/unit/test_hnsw_disk.cpp | 37 +- tests/unit/test_quantized_hnsw_disk.cpp | 6 +- 5 files changed, 186 insertions(+), 781 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 52bbd8459..645929abd 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -228,12 +228,6 @@ class HNSWDiskIndex : public VecSimIndexAbstract mutable std::shared_mutex indexDataGuard; mutable VisitedNodesHandlerPool visitedNodesHandlerPool; - // Batch processing state - size_t batchThreshold; // Number of vectors to accumulate before batch update - vecsim_stl::vector pendingVectorIds; // Vector IDs waiting to be indexed - vecsim_stl::vector pendingMetadata; // Metadata for pending vectors - size_t pendingVectorCount; // Count of vectors in memory - /** * Threshold for batching delete operations. * When the number of pending deletions reaches this value, the deletions are processed in a batch. @@ -360,15 +354,15 @@ class HNSWDiskIndex : public VecSimIndexAbstract /********************************** Batchless Mode Support **********************************/ - // Striped neighbor cache for reduced lock contention in multi-threaded scenarios - // Uses lock striping: cache is partitioned into NUM_CACHE_STRIPES independent stripes - // Each stripe has its own lock, cache map, and dirty set - // This allows threads accessing different stripes to proceed in parallel + // Segmented neighbor cache for reduced lock contention in multi-threaded scenarios + // Cache is partitioned into NUM_CACHE_SEGMENTS independent segments + // Each segment has its own lock, cache map, and dirty set + // This allows threads accessing different segments to proceed in parallel - static constexpr size_t NUM_CACHE_STRIPES = 64; // Power of 2 for efficient modulo + static constexpr size_t NUM_CACHE_SEGMENTS = 64; // Power of 2 for efficient modulo - // Cache stripe structure - each stripe is cache-line aligned to prevent false sharing - struct alignas(64) CacheStripe { + // Cache segment structure - each segment is cache-line aligned to prevent false sharing + struct alignas(64) CacheSegment { std::shared_mutex guard; std::unordered_map> cache; std::unordered_set dirty; @@ -377,27 +371,27 @@ class HNSWDiskIndex : public VecSimIndexAbstract std::unordered_set newNodes; char padding[64]; // Ensure no overlap - CacheStripe() = default; - CacheStripe(const CacheStripe&) = delete; - CacheStripe& operator=(const CacheStripe&) = delete; - CacheStripe(CacheStripe&&) = delete; - CacheStripe& operator=(CacheStripe&&) = delete; + CacheSegment() = default; + CacheSegment(const CacheSegment&) = delete; + CacheSegment& operator=(const CacheSegment&) = delete; + CacheSegment(CacheSegment&&) = delete; + CacheSegment& operator=(CacheSegment&&) = delete; }; - // Array of cache stripes - using unique_ptr for lazy initialization - mutable std::unique_ptr cacheStripes_; + // Array of cache segments - using unique_ptr for lazy initialization + mutable std::unique_ptr cacheSegments_; // Atomic counter for total dirty nodes (for fast threshold check without locking) mutable std::atomic totalDirtyCount_{0}; - // Helper function to compute stripe index from cache key + // Helper function to compute segment index from cache key // Uses mixing function for better distribution - static size_t getStripeIndex(uint64_t key) { + static size_t getSegmentIndex(uint64_t key) { // Mix the bits for better distribution (splitmix64-style mixing) key ^= key >> 33; key *= 0xff51afd7ed558ccdULL; key ^= key >> 33; - return key % NUM_CACHE_STRIPES; + return key % NUM_CACHE_SEGMENTS; } // Threshold for flushing dirty nodes to disk (0 = flush after each insert, default = batch) @@ -406,6 +400,9 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Lock for protecting dirty nodes flush operations (global flush serialization) mutable std::mutex diskWriteGuard; + // Atomic counter for pending single insert jobs (batchless mode) + std::atomic pendingSingleInsertJobs_{0}; + protected: HNSWDiskIndex() = delete; // default constructor is disabled. // default (shallow) copy constructor is disabled. @@ -432,49 +429,26 @@ class HNSWDiskIndex : public VecSimIndexAbstract public: // Core vector addition methods - void insertElementToGraph(idType element_id, size_t element_max_level, idType entry_point, - size_t global_max_level, const void *raw_vector_data, const void *vector_data); + void insertElementToGraph(idType element_id, size_t element_max_level, + idType entry_point, size_t global_max_level, + const void *raw_vector_data, const void *vector_data, + vecsim_stl::vector &modifiedNodes); idType mutuallyConnectNewElement(idType new_node_id, - vecsim_stl::updatable_max_heap &top_candidates, size_t level); - - // Batch processing methods - void singleThreadProcessBatch(); - void processBatch(); - void flushBatch(); // Force flush current batch + vecsim_stl::updatable_max_heap &top_candidates, + size_t level, vecsim_stl::vector &modifiedNodes); + // Delete batch processing methods void processDeleteBatch(); void flushDeleteBatch(); // Force flush current delete batch - void setBatchThreshold(size_t threshold); // Set batch threshold - - // Multi-threaded job execution methods - static void executeInsertJobWrapper(AsyncJob *job); - static void executeFlushJobWrapper(AsyncJob *job); - void executeInsertJob(HNSWDiskInsertJob *job); - void executeFlushJob(HNSWDiskFlushJob *job); // Job submission helpers void submitSingleJob(AsyncJob *job); void submitJobs(vecsim_stl::vector &jobs); - // Thread-safe staging merge - void mergeLocalStagedUpdates(vecsim_stl::vector &localGraphUpdates, - vecsim_stl::vector &localNeighborUpdates); - - /********************************** Batchless Mode Methods **********************************/ - - // Batchless job execution + // Job execution static void executeSingleInsertJobWrapper(AsyncJob *job); void executeSingleInsertJob(HNSWDiskSingleInsertJob *job); - // Batchless graph insertion (updates cache instead of staging) - void insertElementToGraphBatchless(idType element_id, size_t element_max_level, - idType entry_point, size_t global_max_level, - const void *raw_vector_data, const void *vector_data, - vecsim_stl::vector &modifiedNodes); - idType mutuallyConnectNewElementBatchless(idType new_node_id, - vecsim_stl::updatable_max_heap &top_candidates, - size_t level, vecsim_stl::vector &modifiedNodes); - // Cache management methods void addNeighborToCache(idType nodeId, size_t level, idType newNeighborId); void setNeighborsInCache(idType nodeId, size_t level, const vecsim_stl::vector &neighbors, bool isNewNode = false); @@ -487,11 +461,10 @@ class HNSWDiskIndex : public VecSimIndexAbstract const void *newVectorRawData, idType newVectorId); // Unified core function for graph insertion - used by both single-threaded and multi-threaded paths - // immediateFlush: true = write to disk immediately (single-threaded), false = batched writes (multi-threaded) + // Batching is controlled by diskWriteBatchThreshold (0 = no batching, >0 = batch size) void executeGraphInsertionCore(idType vectorId, size_t elementMaxLevel, idType entryPoint, size_t globalMaxLevel, - const void *rawVectorData, const void *processedVectorData, - bool immediateFlush); + const void *rawVectorData, const void *processedVectorData); // Helper to write a single vector's entry to disk void writeVectorToDisk(idType vectorId, const void *rawVectorData, @@ -509,10 +482,9 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Thread-safe atomic check-and-add for neighbor updates // Returns true if neighbor was added (had capacity), false if full (needs re-evaluation) bool tryAddNeighborIfCapacity(idType nodeId, size_t level, idType newNeighborId, size_t maxCapacity); - void searchPendingVectors(const void* query_data, candidatesLabelsMaxHeap& top_candidates, size_t k) const; - // Manual control of staged updates - void flushStagedUpdates(); // Manually flush any pending staged updates + // Manual control of staged delete updates + void flushStagedDeleteUpdates(); // Manually flush any pending staged delete updates protected: // Helper method to filter deleted or invalid nodes from a neighbor list (DRY principle) @@ -686,8 +658,6 @@ class HNSWDiskIndex : public VecSimIndexAbstract void setDeleteBatchThreshold(size_t threshold) { deleteBatchThreshold = threshold; } size_t getDeleteBatchThreshold() const { return deleteBatchThreshold; } size_t getPendingDeleteCount() const { return pendingDeleteIds.size(); } - size_t getPendingInsertCount() const { return pendingVectorCount; } - size_t getProcessingBatchCount() const { return processingBatch ? processingBatch->count : 0; } // Disk write batching control void setDiskWriteBatchThreshold(size_t threshold) { diskWriteBatchThreshold = threshold; } @@ -761,14 +731,12 @@ HNSWDiskIndex::HNSWDiskIndex( idToMetaData(INITIAL_CAPACITY, this->allocator), labelToIdMap(this->allocator), db(db), cf(cf), dbPath(dbPath), indexDataGuard(), visitedNodesHandlerPool(INITIAL_CAPACITY, this->allocator), - batchThreshold(10), - pendingVectorIds(this->allocator), pendingMetadata(this->allocator), pendingVectorCount(0), pendingDeleteIds(this->allocator), stagedInsertUpdates(this->allocator), stagedDeleteUpdates(this->allocator), stagedRepairUpdates(this->allocator), stagedInsertNeighborUpdates(this->allocator), jobQueue(jobQueue_), jobQueueCtx(jobQueueCtx_), SubmitJobsToQueue(submitCb_), - cacheStripes_(new CacheStripe[NUM_CACHE_STRIPES]) { + cacheSegments_(new CacheSegment[NUM_CACHE_SEGMENTS]) { M = params->M ? params->M : HNSW_DEFAULT_M; M0 = M * 2; @@ -812,8 +780,6 @@ HNSWDiskIndex::~HNSWDiskIndex() { stagedInsertNeighborUpdates.clear(); // Clear pending vectors - pendingVectorIds.clear(); - pendingMetadata.clear(); pendingDeleteIds.clear(); // Clear raw vectors in RAM @@ -839,13 +805,8 @@ HNSWDiskIndex::topKQuery(const void *query_data, size_t k, auto rep = new VecSimQueryReply(this->allocator); this->lastMode = STANDARD_KNN; - // Check if index is empty - need to lock to read pendingVectorCount safely - size_t pendingCount; - { - std::lock_guard lock(batchSwapGuard); - pendingCount = pendingVectorCount; - } - if ((curElementCount.load(std::memory_order_acquire) == 0 && pendingCount == 0) || k == 0) { + // Check if index is empty + if (curElementCount.load(std::memory_order_acquire) == 0 || k == 0) { return rep; } @@ -881,11 +842,6 @@ HNSWDiskIndex::topKQuery(const void *query_data, size_t k, // Step 2: Search bottom layer using quantized distances auto results = searchLayerLabels(bottom_layer_ep, processed_query, 0, query_ef); - if (pendingVectorCount > 0) { - // Search pending vectors using the helper method - searchPendingVectors(query_data, results, k); - } - // Step 3: Re-rank candidates using raw float32 distances for improved recall if (useRawData && !results.empty()) { rerankWithRawDistances(results, query_data, k); @@ -1120,101 +1076,14 @@ int HNSWDiskIndex::addVector( submitSingleJob(job); } else { - // Single-threaded: execute inline with immediate disk write + // Single-threaded: execute inline (batching controlled by diskWriteBatchThreshold) executeGraphInsertionCore(newElementId, elementMaxLevel, currentEntryPoint, - currentMaxLevel, vector, processedBlobs.getStorageBlob(), - true /* immediateFlush */); + currentMaxLevel, vector, processedBlobs.getStorageBlob()); } return 1; } -template -void HNSWDiskIndex::insertElementToGraph(idType element_id, - size_t element_max_level, - idType entry_point, - size_t global_max_level, - const void *raw_vector_data, - const void *vector_data) { - - idType curr_element = entry_point; - DistType cur_dist = std::numeric_limits::max(); - size_t max_common_level; - if (element_max_level < global_max_level) { - max_common_level = element_max_level; - cur_dist = this->calcDistance(vector_data, getDataByInternalId(curr_element)); - for (auto level = static_cast(global_max_level); - level > static_cast(element_max_level); level--) { - // this is done for the levels which are above the max level - // to which we are going to insert the new element. We do - // a greedy search in the graph starting from the entry point - // at each level, and move on with the closest element we can find. - // When there is no improvement to do, we take a step down. - greedySearchLevel(vector_data, level, curr_element, cur_dist); - } - } else { - max_common_level = global_max_level; - } - - for (auto level = static_cast(max_common_level); level >= 0; level--) { - vecsim_stl::updatable_max_heap top_candidates = - searchLayer(curr_element, vector_data, level, efConstruction); - - // If the entry point was marked deleted between iterations, we may receive an empty - // candidates set. - if (!top_candidates.empty()) { - curr_element = mutuallyConnectNewElement(element_id, top_candidates, level); - } else { - this->log(VecSimCommonStrings::LOG_WARNING_STRING, - "WARNING: No candidates found at level %d!", level); - } - } -} - -template -idType HNSWDiskIndex::mutuallyConnectNewElement( - idType new_node_id, vecsim_stl::updatable_max_heap &top_candidates, size_t level) { - - // The maximum number of neighbors allowed for an existing neighbor (not new). - size_t max_M_cur = level ? M : M0; - - // Filter the top candidates to the selected neighbors by the algorithm heuristics. - // First, we need to copy the top candidates to a vector. - candidatesList top_candidates_list(this->allocator); - top_candidates_list.insert(top_candidates_list.end(), top_candidates.begin(), - top_candidates.end()); - // Use the heuristic to filter the top candidates, and get the next closest entry point. - idType next_closest_entry_point = getNeighborsByHeuristic2(top_candidates_list, M); - assert(top_candidates_list.size() <= M && - "Should be not be more than M candidates returned by the heuristic"); - - // Build the neighbor list for the new node - vecsim_stl::vector neighbor_ids(this->allocator); - neighbor_ids.reserve(top_candidates_list.size()); - for (size_t i = 0; i < top_candidates_list.size(); ++i) { - neighbor_ids.push_back(top_candidates_list[i].second); - } - - // Set the new node's neighbors in cache (this is a new node, so mark it as such) - setNeighborsInCache(new_node_id, level, neighbor_ids, true /* isNewNode */); - - // Update existing nodes to include the new node in their neighbor lists - for (const auto &neighbor_data : top_candidates_list) { - idType selected_neighbor = neighbor_data.second; - DistType distance = neighbor_data.first; - - // Use atomic check-and-add to prevent race conditions where multiple threads - // think there's capacity and all add neighbors, exceeding max_M_cur - if (!tryAddNeighborIfCapacity(selected_neighbor, level, new_node_id, max_M_cur)) { - // Neighbor is full, need to re-evaluate connections using revisitNeighborConnections - // logic - stageRevisitNeighborConnections(new_node_id, selected_neighbor, level, distance); - } - } - - return next_closest_entry_point; -} - template void HNSWDiskIndex::flushStagedGraphUpdates( vecsim_stl::vector& graphUpdates, @@ -2000,17 +1869,17 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level // Step 1: Check cache first (source of truth for pending updates) { - size_t stripe = getStripeIndex(lookup_key); - auto& cacheStripe = cacheStripes_[stripe]; - std::shared_lock cacheLock(cacheStripe.guard); - auto it = cacheStripe.cache.find(lookup_key); - if (it != cacheStripe.cache.end()) { + size_t segment = getSegmentIndex(lookup_key); + auto& cacheSegment = cacheSegments_[segment]; + std::shared_lock cacheLock(cacheSegment.guard); + auto it = cacheSegment.cache.find(lookup_key); + if (it != cacheSegment.cache.end()) { result.reserve(it->second.size()); for (idType id : it->second) { result.push_back(id); } foundInCache = true; - } else if (cacheStripe.newNodes.find(lookup_key) != cacheStripe.newNodes.end()) { + } else if (cacheSegment.newNodes.find(lookup_key) != cacheSegment.newNodes.end()) { // New node not yet connected - return empty foundInCache = true; // Treat as found (empty is valid) } @@ -2073,17 +1942,17 @@ void HNSWDiskIndex::getNeighborsAndVector(idType nodeId, siz // Step 1: Check cache first (source of truth for pending updates) { - size_t stripe = getStripeIndex(lookup_key); - auto& cacheStripe = cacheStripes_[stripe]; - std::shared_lock cacheLock(cacheStripe.guard); - auto it = cacheStripe.cache.find(lookup_key); - if (it != cacheStripe.cache.end()) { + size_t segment = getSegmentIndex(lookup_key); + auto& cacheSegment = cacheSegments_[segment]; + std::shared_lock cacheLock(cacheSegment.guard); + auto it = cacheSegment.cache.find(lookup_key); + if (it != cacheSegment.cache.end()) { result.reserve(it->second.size()); for (idType id : it->second) { result.push_back(id); } foundNeighborsInCache = true; - } else if (cacheStripe.newNodes.find(lookup_key) != cacheStripe.newNodes.end()) { + } else if (cacheSegment.newNodes.find(lookup_key) != cacheSegment.newNodes.end()) { // New node not yet connected - return empty neighbors foundNeighborsInCache = true; } @@ -2147,15 +2016,15 @@ size_t HNSWDiskIndex::getNeighborsCount(idType nodeId, size_ // Step 1: Check cache first (source of truth for pending updates) { - size_t stripe = getStripeIndex(lookup_key); - auto& cacheStripe = cacheStripes_[stripe]; - std::shared_lock cacheLock(cacheStripe.guard); - auto it = cacheStripe.cache.find(lookup_key); - if (it != cacheStripe.cache.end()) { + size_t segment = getSegmentIndex(lookup_key); + auto& cacheSegment = cacheSegments_[segment]; + std::shared_lock cacheLock(cacheSegment.guard); + auto it = cacheSegment.cache.find(lookup_key); + if (it != cacheSegment.cache.end()) { return it->second.size(); } // Check if this is a new node (never written to disk) - if (cacheStripe.newNodes.find(lookup_key) != cacheStripe.newNodes.end()) { + if (cacheSegment.newNodes.find(lookup_key) != cacheSegment.newNodes.end()) { return 0; // New node not yet connected } } @@ -2178,18 +2047,18 @@ size_t HNSWDiskIndex::getNeighborsCount(idType nodeId, size_ template bool HNSWDiskIndex::tryAddNeighborIfCapacity( idType nodeId, size_t level, idType newNeighborId, size_t maxCapacity) { - // Atomic check-and-add using cache stripes: O(1) lookup, 1/64th lock contention + // Atomic check-and-add using cache segments: O(1) lookup, 1/64th lock contention // The cache is the source of truth - it contains either disk data or pending updates uint64_t key = makeRepairKey(nodeId, level); - size_t stripe = getStripeIndex(key); - auto& cacheStripe = cacheStripes_[stripe]; + size_t segment = getSegmentIndex(key); + auto& cacheSegment = cacheSegments_[segment]; // First, try with just a shared lock for the common case (already in cache) { - std::shared_lock readLock(cacheStripe.guard); - auto it = cacheStripe.cache.find(key); - if (it != cacheStripe.cache.end()) { + std::shared_lock readLock(cacheSegment.guard); + auto it = cacheSegment.cache.find(key); + if (it != cacheSegment.cache.end()) { // Fast path: already in cache, check if we can add if (it->second.size() >= maxCapacity) { return false; // Full, no need to upgrade lock @@ -2198,12 +2067,12 @@ bool HNSWDiskIndex::tryAddNeighborIfCapacity( } // Need to modify - get exclusive lock - std::unique_lock lock(cacheStripe.guard); + std::unique_lock lock(cacheSegment.guard); - auto it = cacheStripe.cache.find(key); - if (it == cacheStripe.cache.end()) { + auto it = cacheSegment.cache.find(key); + if (it == cacheSegment.cache.end()) { // Check if this is a new node (never written to disk) - skip disk lookup - bool isNewNode = (cacheStripe.newNodes.find(key) != cacheStripe.newNodes.end()); + bool isNewNode = (cacheSegment.newNodes.find(key) != cacheSegment.newNodes.end()); if (!isNewNode) { // Not in cache and not a new node - need to load from disk @@ -2221,21 +2090,21 @@ bool HNSWDiskIndex::tryAddNeighborIfCapacity( for (idType id : diskNeighbors) { cacheEntry.push_back(id); } - cacheStripe.cache[key] = std::move(cacheEntry); + cacheSegment.cache[key] = std::move(cacheEntry); } else { // New node - initialize with empty neighbor list - cacheStripe.cache[key] = std::vector(); + cacheSegment.cache[key] = std::vector(); } } // Now we have the current neighbor list in cache - check capacity - auto& neighbors = cacheStripe.cache[key]; + auto& neighbors = cacheSegment.cache[key]; if (neighbors.size() < maxCapacity) { // Check for duplicates if (std::find(neighbors.begin(), neighbors.end(), newNeighborId) == neighbors.end()) { neighbors.push_back(newNeighborId); // Mark as dirty and increment atomic counter - auto insertResult = cacheStripe.dirty.insert(key); + auto insertResult = cacheSegment.dirty.insert(key); if (insertResult.second) { totalDirtyCount_.fetch_add(1, std::memory_order_relaxed); } @@ -2246,229 +2115,6 @@ bool HNSWDiskIndex::tryAddNeighborIfCapacity( return false; // Neighbor list is full, needs re-evaluation } -template -void HNSWDiskIndex::searchPendingVectors( - const void *query_data, candidatesLabelsMaxHeap &top_candidates, size_t k) const { - // Copy pending vector data under lock to avoid holding lock during expensive distance calculations - vecsim_stl::vector> pendingData(this->allocator); - std::vector vectorDataCopies; - - { - std::lock_guard batchLock(batchSwapGuard); - std::shared_lock lock(rawVectorsGuard); - - if (pendingVectorCount == 0) { - return; - } - - pendingData.reserve(pendingVectorCount); - vectorDataCopies.reserve(pendingVectorCount); - - for (size_t i = 0; i < pendingVectorCount; i++) { - idType vectorId = pendingVectorIds[i]; - // Use direct flag check to avoid taking indexDataGuard while holding other locks - if (__atomic_load_n(&idToMetaData[vectorId].flags, 0) & DELETE_MARK) { - continue; - } - - auto it = rawVectorsInRAM.find(vectorId); - if (it == rawVectorsInRAM.end()) { - continue; - } - - const DiskElementMetaData &metadata = idToMetaData[vectorId]; - pendingData.emplace_back(vectorId, metadata.label); - vectorDataCopies.push_back(it->second); // Copy the vector data - } - } - // Locks released - now do expensive distance calculations without holding locks - - for (size_t i = 0; i < pendingData.size(); i++) { - labelType label = pendingData[i].second; - const void* vector_data = vectorDataCopies[i].data(); - - DistType dist = this->calcDistanceRaw(query_data, vector_data); - - if (top_candidates.size() < k) { - top_candidates.emplace(dist, label); - } else if (dist < top_candidates.top().first) { - top_candidates.pop(); - top_candidates.emplace(dist, label); - } - } -} - -/********************************** Batch Processing Methods **********************************/ - -template -void HNSWDiskIndex:: singleThreadProcessBatch(){ - // Clear any previous staged updates (for insertions) - stagedInsertUpdates.clear(); - stagedInsertMap.clear(); - stagedInsertNeighborUpdates.clear(); - - // Process each pending vector ID (vectors are already stored in memory) - for (size_t i = 0; i < pendingVectorCount; i++) { - idType vectorId = pendingVectorIds[i]; - if (isMarkedDeleted(vectorId)) { - continue; - } - - const void *vector_data = this->vectors->getElement(vectorId); - const void *raw_vector_data = rawVectorsInRAM.find(vectorId)->second.data(); - DiskElementMetaData &metadata = idToMetaData[vectorId]; - size_t elementMaxLevel = metadata.topLevel; - - if (entrypointNode != INVALID_ID) { - insertElementToGraph(vectorId, elementMaxLevel, entrypointNode, - maxLevel, raw_vector_data, vector_data); - } else { - entrypointNode = vectorId; - maxLevel = elementMaxLevel; - } - } - - flushStagedGraphUpdates(stagedInsertUpdates, stagedInsertNeighborUpdates); - stagedInsertMap.clear(); - pendingVectorIds.clear(); - rawVectorsInRAM.clear(); - pendingMetadata.clear(); - pendingVectorCount = 0; -} - -template -void HNSWDiskIndex::processBatch() { - // Lock to swap batches atomically - std::lock_guard batchLock(batchSwapGuard); - - if (pendingVectorCount == 0) { - return; - } - - // Check if job queue is available for multi-threaded processing - if (SubmitJobsToQueue == nullptr) { - // Fall back to single-threaded processing - singleThreadProcessBatch(); - return; - } - - // Check if previous batch is still processing - if (batchInProgress.load()) { - // Previous batch still running, return - caller can retry later - return; - } - - batchInProgress.store(true); - - // Bound the batch size to prevent large batches when vectors accumulate - // while a previous batch is processing - size_t vectorsToMove = std::min(pendingVectorCount, batchThreshold); - - // Move only up to batchThreshold vectors to processing batch (bounded double-buffering) - processingBatch->vectorIds.assign( - pendingVectorIds.begin(), - pendingVectorIds.begin() + vectorsToMove - ); - processingBatch->count = vectorsToMove; - - // Move corresponding raw vectors to processing batch - { - std::lock_guard lock(rawVectorsGuard); - for (idType id : processingBatch->vectorIds) { - auto it = rawVectorsInRAM.find(id); - if (it != rawVectorsInRAM.end()) { - processingBatch->rawVectors[id] = std::move(it->second); - rawVectorsInRAM.erase(it); - } - } - } - - // Copy processed vectors to processing batch (prevents race with main thread resizing vectors) - { - std::shared_lock lock(vectorsGuard); - for (idType vectorId : processingBatch->vectorIds) { - const void *processed = this->vectors->getElement(vectorId); - if (processed) { - processingBatch->processedVectors[vectorId] = - std::string(static_cast(processed), this->dataSize); - } - } - } - - // Remove moved vectors from pending structures, keep remaining for next batch - pendingVectorIds.erase( - pendingVectorIds.begin(), - pendingVectorIds.begin() + vectorsToMove - ); - pendingVectorCount -= vectorsToMove; - - // Clear any previous staged updates - stagedInsertUpdates.clear(); - stagedInsertMap.clear(); - stagedInsertNeighborUpdates.clear(); - - // Set entry point only if it doesn't exist (matches original disk-poc algorithm) - // In the original algorithm, only the first vector becomes entry point when graph is empty. - // Subsequent vectors always insert using the existing entry point. - { - std::unique_lock lock(indexDataGuard); - if (entrypointNode == INVALID_ID) { - // Find first non-deleted vector to be the entry point - for (size_t i = 0; i < processingBatch->count; i++) { - idType vectorId = processingBatch->vectorIds[i]; - // Use direct flag check to avoid recursive lock (isMarkedDeleted takes shared lock) - if (__atomic_load_n(&idToMetaData[vectorId].flags, 0) & DELETE_MARK) - continue; - - DiskElementMetaData &metadata = idToMetaData[vectorId]; - entrypointNode = vectorId; - maxLevel = metadata.topLevel; - break; // Only set first non-deleted vector as entry point - } - } - } - // std::cout << "processBatch memory: " << this->getAllocationSize()/1024/1024 << " MB" << std::endl; - - // Create jobs for all vectors (entry point check is done in executeInsertJob) - vecsim_stl::vector jobs(this->allocator); - jobs.reserve(processingBatch->count); - - for (size_t i = 0; i < processingBatch->count; i++) { - idType vectorId = processingBatch->vectorIds[i]; - if (isMarkedDeleted(vectorId)) - continue; - - DiskElementMetaData &metadata = idToMetaData[vectorId]; - - HNSWDiskInsertJob *job = - new (this->allocator) HNSWDiskInsertJob(this->allocator, vectorId, metadata.topLevel, - executeInsertJobWrapper, this); - jobs.push_back(job); - } - - if (jobs.empty()) { - // No jobs to submit (e.g., only entry point was set) - batchInProgress.store(false); - processingBatch->clear(); - return; - } - - // Set up counter for synchronization - pendingInsertJobsCounter.store(jobs.size()); - - // Create flush job (will be submitted by last insert job) - pendingFlushJob = - new (this->allocator) HNSWDiskFlushJob(this->allocator, executeFlushJobWrapper, this); - - // Submit all insert jobs - submitJobs(jobs); -} - -template -void HNSWDiskIndex::flushBatch() { - processBatch(); -} - /********************************** Multi-threaded Job Execution **********************************/ template @@ -2486,143 +2132,6 @@ void HNSWDiskIndex::submitJobs(vecsim_stl::vector -void HNSWDiskIndex::executeInsertJobWrapper(AsyncJob *job) { - auto *insertJob = static_cast(job); - auto *index = static_cast *>(job->index); - index->executeInsertJob(insertJob); -} - -template -void HNSWDiskIndex::executeFlushJobWrapper(AsyncJob *job) { - auto *flushJob = static_cast(job); - auto *index = static_cast *>(job->index); - index->executeFlushJob(flushJob); -} - -template -void HNSWDiskIndex::executeInsertJob(HNSWDiskInsertJob *job) { - if (!job->isValid) { - // Job was invalidated, decrement counter and check for flush - if (pendingInsertJobsCounter.fetch_sub(1) == 1) { - submitSingleJob(pendingFlushJob); - } - delete job; - return; - } - - // Get vector data from processing batch (read-only, thread-safe) - auto rawIt = processingBatch->rawVectors.find(job->vectorId); - if (rawIt == processingBatch->rawVectors.end()) { - this->log(VecSimCommonStrings::LOG_WARNING_STRING, - "Raw vector not found in processing batch for vectorId %u", job->vectorId); - if (pendingInsertJobsCounter.fetch_sub(1) == 1) { - submitSingleJob(pendingFlushJob); - } - delete job; - return; - } - const void *raw_vector_data = rawIt->second.data(); - - auto procIt = processingBatch->processedVectors.find(job->vectorId); - if (procIt == processingBatch->processedVectors.end()) { - this->log(VecSimCommonStrings::LOG_WARNING_STRING, - "Processed vector not found in processing batch for vectorId %u", job->vectorId); - if (pendingInsertJobsCounter.fetch_sub(1) == 1) { - submitSingleJob(pendingFlushJob); - } - delete job; - return; - } - const void *vector_data = procIt->second.data(); - - // Read current entry point state dynamically (like the original disk-poc algorithm) - // This allows jobs to benefit from entry point updates made by previously completed jobs - idType currentEntryPoint; - size_t currentMaxLevel; - { - std::shared_lock lock(indexDataGuard); - currentEntryPoint = entrypointNode; - currentMaxLevel = maxLevel; - } - - // Skip if this is the entry point (no connections to make for the first element) - if (job->vectorId != currentEntryPoint && currentEntryPoint != INVALID_ID) { - // Insert into graph using the original algorithm - insertElementToGraph(job->vectorId, job->elementMaxLevel, currentEntryPoint, - currentMaxLevel, raw_vector_data, vector_data); - } - - // Decrement counter and submit flush job if this is the last insert job - if (pendingInsertJobsCounter.fetch_sub(1) == 1) { - submitSingleJob(pendingFlushJob); - } - - delete job; -} - -template -void HNSWDiskIndex::executeFlushJob(HNSWDiskFlushJob *job) { - // Flush all staged updates to RocksDB (RocksDB is thread-safe for writes) - flushStagedGraphUpdates(stagedInsertUpdates, stagedInsertNeighborUpdates); - - // Clear staging - must hold stagedUpdatesGuard to prevent race with getNeighbors - { - std::lock_guard stagingLock(stagedUpdatesGuard); - stagedInsertMap.clear(); - stagedInsertUpdates.clear(); - stagedInsertNeighborUpdates.clear(); - } - - // Clear processing batch (raw vectors now persisted to disk) - processingBatch->clear(); - - // Mark batch as complete - batchInProgress.store(false); - - // Clean up flush job - pendingFlushJob = nullptr; - delete job; - - // Check if there are more pending vectors to process - // This ensures we process accumulated vectors in bounded batches - bool shouldTriggerNextBatch = false; - { - std::lock_guard lock(batchSwapGuard); - shouldTriggerNextBatch = (pendingVectorCount >= batchThreshold); - } - - if (shouldTriggerNextBatch) { - processBatch(); - } -} - -template -void HNSWDiskIndex::mergeLocalStagedUpdates( - vecsim_stl::vector &localGraphUpdates, - vecsim_stl::vector &localNeighborUpdates) { - std::lock_guard lock(stagedUpdatesGuard); - - // Merge graph updates - for (auto &update : localGraphUpdates) { - uint64_t key = makeRepairKey(update.node_id, update.level); - auto it = stagedInsertMap.find(key); - if (it != stagedInsertMap.end()) { - // Update existing entry - stagedInsertUpdates[it->second] = std::move(update); - } else { - // Add new entry - stagedInsertMap[key] = stagedInsertUpdates.size(); - stagedInsertUpdates.push_back(std::move(update)); - } - } - - // Merge neighbor updates - for (auto &update : localNeighborUpdates) { - stagedInsertNeighborUpdates.push_back(std::move(update)); - } -} - template void HNSWDiskIndex::repairNeighborConnections( idType neighbor_id, size_t level, idType deleted_id, @@ -2819,11 +2328,6 @@ void HNSWDiskIndex::flushDeleteBatch() { processDeleteBatch(); } -template -void HNSWDiskIndex::setBatchThreshold(size_t threshold) { - batchThreshold = threshold; -} - /********************************** Debug Methods **********************************/ template @@ -3039,20 +2543,12 @@ void HNSWDiskIndex::debugValidateGraphConnectivity() const { template -void HNSWDiskIndex::flushStagedUpdates() { - // Flush both insert and delete staged updates +void HNSWDiskIndex::flushStagedDeleteUpdates() { + // Flush staged delete updates to disk // Note: This is a non-const method that modifies the staging areas - flushStagedGraphUpdates(stagedInsertUpdates, stagedInsertNeighborUpdates); - stagedInsertMap.clear(); vecsim_stl::vector emptyNeighborUpdates(this->allocator); flushStagedGraphUpdates(stagedDeleteUpdates, emptyNeighborUpdates); stagedDeleteMap.clear(); - - // Also flush staged repair updates (opportunistic cleanup from getNeighbors) - if (!stagedRepairUpdates.empty()) { - flushStagedGraphUpdates(stagedRepairUpdates, emptyNeighborUpdates); - stagedRepairMap.clear(); - } } template @@ -3345,14 +2841,14 @@ void HNSWDiskIndex::getNeighborsFromCache( result.clear(); uint64_t key = makeRepairKey(nodeId, level); - size_t stripe = getStripeIndex(key); - auto& cacheStripe = cacheStripes_[stripe]; + size_t segment = getSegmentIndex(key); + auto& cacheSegment = cacheSegments_[segment]; // Step 1: Check in-memory cache (includes pending writes from all jobs) { - std::shared_lock lock(cacheStripe.guard); - auto it = cacheStripe.cache.find(key); - if (it != cacheStripe.cache.end()) { + std::shared_lock lock(cacheSegment.guard); + auto it = cacheSegment.cache.find(key); + if (it != cacheSegment.cache.end()) { // Copy from std::vector to vecsim_stl::vector result.reserve(it->second.size()); for (idType id : it->second) { @@ -3363,7 +2859,7 @@ void HNSWDiskIndex::getNeighborsFromCache( } // Check if this is a new node (never written to disk) - return empty - if (cacheStripe.newNodes.find(key) != cacheStripe.newNodes.end()) { + if (cacheSegment.newNodes.find(key) != cacheSegment.newNodes.end()) { // New node not yet in cache - return empty (will be populated when connected) return; } @@ -3380,17 +2876,17 @@ void HNSWDiskIndex::getNeighborsFromCache( // Add to cache for future reads (convert to unique_lock for write) { - std::unique_lock lock(cacheStripe.guard); + std::unique_lock lock(cacheSegment.guard); // Double-check: another thread may have populated it - auto it = cacheStripe.cache.find(key); - if (it == cacheStripe.cache.end()) { + auto it = cacheSegment.cache.find(key); + if (it == cacheSegment.cache.end()) { // Copy from vecsim_stl::vector to std::vector std::vector cacheEntry; cacheEntry.reserve(result.size()); for (idType id : result) { cacheEntry.push_back(id); } - cacheStripe.cache[key] = std::move(cacheEntry); + cacheSegment.cache[key] = std::move(cacheEntry); } } @@ -3402,15 +2898,15 @@ void HNSWDiskIndex::addNeighborToCache( idType nodeId, size_t level, idType newNeighborId) { uint64_t key = makeRepairKey(nodeId, level); - size_t stripe = getStripeIndex(key); - auto& cacheStripe = cacheStripes_[stripe]; + size_t segment = getSegmentIndex(key); + auto& cacheSegment = cacheSegments_[segment]; - std::unique_lock lock(cacheStripe.guard); + std::unique_lock lock(cacheSegment.guard); - auto it = cacheStripe.cache.find(key); - if (it == cacheStripe.cache.end()) { + auto it = cacheSegment.cache.find(key); + if (it == cacheSegment.cache.end()) { // Check if this is a new node (never written to disk) - skip disk lookup - bool isNewNode = (cacheStripe.newNodes.find(key) != cacheStripe.newNodes.end()); + bool isNewNode = (cacheSegment.newNodes.find(key) != cacheSegment.newNodes.end()); if (!isNewNode) { // First modification of existing node - need to load current state from disk @@ -3425,29 +2921,29 @@ void HNSWDiskIndex::addNeighborToCache( lock.lock(); // Re-check: another thread might have populated it - if (cacheStripe.cache.find(key) == cacheStripe.cache.end()) { + if (cacheSegment.cache.find(key) == cacheSegment.cache.end()) { // Convert vecsim_stl::vector to std::vector std::vector cacheEntry; cacheEntry.reserve(diskNeighbors.size()); for (idType id : diskNeighbors) { cacheEntry.push_back(id); } - cacheStripe.cache[key] = std::move(cacheEntry); + cacheSegment.cache[key] = std::move(cacheEntry); } } else { // New node - initialize with empty neighbor list - cacheStripe.cache[key] = std::vector(); + cacheSegment.cache[key] = std::vector(); } } // Add new neighbor (avoid duplicates) - auto &neighbors = cacheStripe.cache[key]; + auto &neighbors = cacheSegment.cache[key]; if (std::find(neighbors.begin(), neighbors.end(), newNeighborId) == neighbors.end()) { neighbors.push_back(newNeighborId); } // Mark as dirty (needs disk write) and increment atomic counter - auto insertResult = cacheStripe.dirty.insert(key); + auto insertResult = cacheSegment.dirty.insert(key); if (insertResult.second) { // Only increment if newly inserted totalDirtyCount_.fetch_add(1, std::memory_order_relaxed); } @@ -3458,13 +2954,13 @@ bool HNSWDiskIndex::tryAddNeighborToCacheIfCapacity( idType nodeId, size_t level, idType newNeighborId, size_t maxCapacity) { uint64_t key = makeRepairKey(nodeId, level); - size_t stripe = getStripeIndex(key); - auto& cacheStripe = cacheStripes_[stripe]; + size_t segment = getSegmentIndex(key); + auto& cacheSegment = cacheSegments_[segment]; - std::unique_lock lock(cacheStripe.guard); + std::unique_lock lock(cacheSegment.guard); - auto it = cacheStripe.cache.find(key); - if (it == cacheStripe.cache.end()) { + auto it = cacheSegment.cache.find(key); + if (it == cacheSegment.cache.end()) { // First modification - need to load current state from disk lock.unlock(); vecsim_stl::vector diskNeighbors(this->allocator); @@ -3477,19 +2973,19 @@ bool HNSWDiskIndex::tryAddNeighborToCacheIfCapacity( lock.lock(); // Re-check: another thread might have populated it - if (cacheStripe.cache.find(key) == cacheStripe.cache.end()) { + if (cacheSegment.cache.find(key) == cacheSegment.cache.end()) { // Convert vecsim_stl::vector to std::vector std::vector cacheEntry; cacheEntry.reserve(diskNeighbors.size()); for (idType id : diskNeighbors) { cacheEntry.push_back(id); } - cacheStripe.cache[key] = std::move(cacheEntry); + cacheSegment.cache[key] = std::move(cacheEntry); } } // Atomic check-and-add under the lock - auto &neighbors = cacheStripe.cache[key]; + auto &neighbors = cacheSegment.cache[key]; // Check if already present (avoid duplicates) if (std::find(neighbors.begin(), neighbors.end(), newNeighborId) != neighbors.end()) { @@ -3503,7 +2999,7 @@ bool HNSWDiskIndex::tryAddNeighborToCacheIfCapacity( // Has capacity - add the neighbor neighbors.push_back(newNeighborId); - auto insertResult = cacheStripe.dirty.insert(key); + auto insertResult = cacheSegment.dirty.insert(key); if (insertResult.second) { // Only increment if newly inserted totalDirtyCount_.fetch_add(1, std::memory_order_relaxed); } @@ -3515,8 +3011,8 @@ void HNSWDiskIndex::setNeighborsInCache( idType nodeId, size_t level, const vecsim_stl::vector &neighbors, bool isNewNode) { uint64_t key = makeRepairKey(nodeId, level); - size_t stripe = getStripeIndex(key); - auto& cacheStripe = cacheStripes_[stripe]; + size_t segment = getSegmentIndex(key); + auto& cacheSegment = cacheSegments_[segment]; // Convert vecsim_stl::vector to std::vector std::vector cacheEntry; @@ -3525,15 +3021,15 @@ void HNSWDiskIndex::setNeighborsInCache( cacheEntry.push_back(id); } - std::unique_lock lock(cacheStripe.guard); - cacheStripe.cache[key] = std::move(cacheEntry); + std::unique_lock lock(cacheSegment.guard); + cacheSegment.cache[key] = std::move(cacheEntry); // If this is a new node, track it to avoid disk lookups if (isNewNode) { - cacheStripe.newNodes.insert(key); + cacheSegment.newNodes.insert(key); } - auto insertResult = cacheStripe.dirty.insert(key); + auto insertResult = cacheSegment.dirty.insert(key); if (insertResult.second) { // Only increment if newly inserted totalDirtyCount_.fetch_add(1, std::memory_order_relaxed); } @@ -3575,19 +3071,19 @@ void HNSWDiskIndex::writeDirtyNodesToDisk( rocksdb::WriteBatch batch; std::vector rawVectorBuffer(this->inputBlobSize); - // Process each modified node - lock only the relevant stripe for each + // Process each modified node - lock only the relevant segment for each for (uint64_t key : modifiedNodes) { idType nodeId = static_cast(key >> 32); size_t level = static_cast(key & 0xFFFFFFFF); - size_t stripe = getStripeIndex(key); - auto& cacheStripe = cacheStripes_[stripe]; + size_t segment = getSegmentIndex(key); + auto& cacheSegment = cacheSegments_[segment]; - // Get neighbors from cache under stripe lock + // Get neighbors from cache under segment lock vecsim_stl::vector neighbors(this->allocator); { - std::shared_lock stripeLock(cacheStripe.guard); - auto it = cacheStripe.cache.find(key); - if (it == cacheStripe.cache.end()) { + std::shared_lock segmentLock(cacheSegment.guard); + auto it = cacheSegment.cache.find(key); + if (it == cacheSegment.cache.end()) { continue; } const std::vector &cacheNeighbors = it->second; @@ -3623,12 +3119,12 @@ void HNSWDiskIndex::writeDirtyNodesToDisk( "ERROR: Failed to write dirty nodes to disk: %s", status.ToString().c_str()); } - // Clear dirty flags for written nodes - lock each stripe individually + // Clear dirty flags for written nodes - lock each segment individually for (uint64_t key : modifiedNodes) { - size_t stripe = getStripeIndex(key); - auto& cacheStripe = cacheStripes_[stripe]; - std::unique_lock stripeLock(cacheStripe.guard); - if (cacheStripe.dirty.erase(key) > 0) { + size_t segment = getSegmentIndex(key); + auto& cacheSegment = cacheSegments_[segment]; + std::unique_lock segmentLock(cacheSegment.guard); + if (cacheSegment.dirty.erase(key) > 0) { totalDirtyCount_.fetch_sub(1, std::memory_order_relaxed); } } @@ -3638,7 +3134,7 @@ template void HNSWDiskIndex::flushDirtyNodesToDisk() { std::lock_guard flushLock(diskWriteGuard); - // Collect all dirty nodes from all stripes + // Collect all dirty nodes from all segments vecsim_stl::vector nodesToFlush(this->allocator); // Only track level 0 nodes for raw vector removal (level 0 contains the actual vector data) std::unordered_set level0VectorIds; @@ -3648,11 +3144,11 @@ void HNSWDiskIndex::flushDirtyNodesToDisk() { return; } - // Collect dirty nodes from all stripes - for (size_t s = 0; s < NUM_CACHE_STRIPES; ++s) { - auto& cacheStripe = cacheStripes_[s]; - std::shared_lock stripeLock(cacheStripe.guard); - for (uint64_t key : cacheStripe.dirty) { + // Collect dirty nodes from all segments + for (size_t s = 0; s < NUM_CACHE_SEGMENTS; ++s) { + auto& cacheSegment = cacheSegments_[s]; + std::shared_lock segmentLock(cacheSegment.guard); + for (uint64_t key : cacheSegment.dirty) { nodesToFlush.push_back(key); size_t level = static_cast(key & 0xFFFFFFFF); if (level == 0) { @@ -3675,19 +3171,19 @@ void HNSWDiskIndex::flushDirtyNodesToDisk() { vecsim_stl::vector successfullyFlushed(this->allocator); std::unordered_set successfulLevel0Ids; - // Process each node - lock only the relevant stripe + // Process each node - lock only the relevant segment for (uint64_t key : nodesToFlush) { idType nodeId = static_cast(key >> 32); size_t level = static_cast(key & 0xFFFFFFFF); - size_t stripe = getStripeIndex(key); - auto& cacheStripe = cacheStripes_[stripe]; + size_t segment = getSegmentIndex(key); + auto& cacheSegment = cacheSegments_[segment]; - // Get neighbors from cache under stripe lock + // Get neighbors from cache under segment lock vecsim_stl::vector neighbors(this->allocator); { - std::shared_lock stripeLock(cacheStripe.guard); - auto it = cacheStripe.cache.find(key); - if (it == cacheStripe.cache.end()) { + std::shared_lock segmentLock(cacheSegment.guard); + auto it = cacheSegment.cache.find(key); + if (it == cacheSegment.cache.end()) { continue; } const std::vector &cacheNeighbors = it->second; @@ -3725,16 +3221,16 @@ void HNSWDiskIndex::flushDirtyNodesToDisk() { return; // Don't clear dirty flags on failure } - // Clear dirty flags and newNodes for successfully written nodes - lock each stripe individually + // Clear dirty flags and newNodes for successfully written nodes - lock each segment individually for (uint64_t key : successfullyFlushed) { - size_t stripe = getStripeIndex(key); - auto& cacheStripe = cacheStripes_[stripe]; - std::unique_lock stripeLock(cacheStripe.guard); - if (cacheStripe.dirty.erase(key) > 0) { + size_t segment = getSegmentIndex(key); + auto& cacheSegment = cacheSegments_[segment]; + std::unique_lock segmentLock(cacheSegment.guard); + if (cacheSegment.dirty.erase(key) > 0) { totalDirtyCount_.fetch_sub(1, std::memory_order_relaxed); } // Clear newNodes flag - node is now on disk - cacheStripe.newNodes.erase(key); + cacheSegment.newNodes.erase(key); } // Only remove raw vectors for level 0 nodes that were successfully written @@ -3748,7 +3244,7 @@ void HNSWDiskIndex::flushDirtyNodesToDisk() { } template -void HNSWDiskIndex::insertElementToGraphBatchless( +void HNSWDiskIndex::insertElementToGraph( idType element_id, size_t element_max_level, idType entry_point, size_t global_max_level, const void *raw_vector_data, const void *vector_data, vecsim_stl::vector &modifiedNodes) { @@ -3773,8 +3269,8 @@ void HNSWDiskIndex::insertElementToGraphBatchless( searchLayer(curr_element, vector_data, level, efConstruction); if (!top_candidates.empty()) { - curr_element = mutuallyConnectNewElementBatchless(element_id, top_candidates, level, - modifiedNodes); + curr_element = mutuallyConnectNewElement(element_id, top_candidates, level, + modifiedNodes); } else { this->log(VecSimCommonStrings::LOG_WARNING_STRING, "WARNING: No candidates found at level %d!", level); @@ -3783,7 +3279,7 @@ void HNSWDiskIndex::insertElementToGraphBatchless( } template -idType HNSWDiskIndex::mutuallyConnectNewElementBatchless( +idType HNSWDiskIndex::mutuallyConnectNewElement( idType new_node_id, vecsim_stl::updatable_max_heap &top_candidates, size_t level, vecsim_stl::vector &modifiedNodes) { @@ -3880,11 +3376,10 @@ void HNSWDiskIndex::executeSingleInsertJob(HNSWDiskSingleIns currentMaxLevel = maxLevel; } - // Use unified core function with batched flush (multi-threaded mode) + // Use unified core function (batching controlled by diskWriteBatchThreshold) executeGraphInsertionCore(job->vectorId, job->elementMaxLevel, currentEntryPoint, currentMaxLevel, job->rawVectorData.data(), - job->processedVectorData.data(), - false /* immediateFlush = batched */); + job->processedVectorData.data()); delete job; } @@ -3893,15 +3388,11 @@ template void HNSWDiskIndex::executeGraphInsertionCore( idType vectorId, size_t elementMaxLevel, idType entryPoint, size_t globalMaxLevel, - const void *rawVectorData, const void *processedVectorData, - bool immediateFlush) { + const void *rawVectorData, const void *processedVectorData) { if (entryPoint == INVALID_ID || vectorId == entryPoint) { // Entry point or first vector - nothing to connect - if (immediateFlush) { - std::lock_guard lock(rawVectorsGuard); - rawVectorsInRAM.erase(vectorId); - } + // Raw vector cleanup happens in addVector for entry point case return; } @@ -3912,25 +3403,25 @@ void HNSWDiskIndex::executeGraphInsertionCore( // while we're reading from it in searchLayer/processCandidate { std::shared_lock lock(indexDataGuard); - // Insert into graph using cache-based method - insertElementToGraphBatchless(vectorId, elementMaxLevel, entryPoint, globalMaxLevel, - rawVectorData, processedVectorData, modifiedNodes); + // Insert into graph + insertElementToGraph(vectorId, elementMaxLevel, entryPoint, globalMaxLevel, + rawVectorData, processedVectorData, modifiedNodes); } - // Handle disk writes based on mode - if (immediateFlush) { - // Single-threaded: write immediately + // Handle disk writes - both single-threaded and multi-threaded support batching + // diskWriteBatchThreshold controls when to flush: + // 0 = flush after every insert (no batching) + // >0 = flush when dirty count reaches threshold + if (diskWriteBatchThreshold == 0) { + // No batching: write immediately writeDirtyNodesToDisk(modifiedNodes, rawVectorData, vectorId); - std::lock_guard lock(rawVectorsGuard); rawVectorsInRAM.erase(vectorId); - } else { - // Multi-threaded: batched writes based on threshold - if (diskWriteBatchThreshold > 0 && - totalDirtyCount_.load(std::memory_order_relaxed) >= diskWriteBatchThreshold) { - flushDirtyNodesToDisk(); - } + } else if (totalDirtyCount_.load(std::memory_order_relaxed) >= diskWriteBatchThreshold) { + // Threshold reached: flush all dirty nodes + flushDirtyNodesToDisk(); } + // else: batching enabled but threshold not reached, keep accumulating // Update entry point if this vector has higher level if (elementMaxLevel > globalMaxLevel) { diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h index f27aa6091..1e17ad78f 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h @@ -56,16 +56,14 @@ HNSWDiskIndex::HNSWDiskIndex( : VecSimIndexAbstract(abstractInitParams, components), idToMetaData(this->allocator), labelToIdMap(this->allocator), db(db), cf(cf), dbPath(""), indexDataGuard(), visitedNodesHandlerPool(INITIAL_CAPACITY, this->allocator), - batchThreshold(0), // Will be restored from file - pendingVectorIds(this->allocator), pendingMetadata(this->allocator), - pendingVectorCount(0), pendingDeleteIds(this->allocator), + pendingDeleteIds(this->allocator), stagedInsertUpdates(this->allocator), stagedDeleteUpdates(this->allocator), stagedRepairUpdates(this->allocator), stagedInsertNeighborUpdates(this->allocator), jobQueue(nullptr), jobQueueCtx(nullptr), SubmitJobsToQueue(nullptr), - cacheStripes_(new CacheStripe[NUM_CACHE_STRIPES]) { + cacheSegments_(new CacheSegment[NUM_CACHE_SEGMENTS]) { - // Restore index fields from file (including batchThreshold) + // Restore index fields from file this->restoreIndexFields(input); // Validate the restored fields @@ -260,14 +258,11 @@ void HNSWDiskIndex::restoreVectors(std::ifstream &input, Enc */ template void HNSWDiskIndex::saveIndexIMP(std::ofstream &output) { - // Flush any pending updates before saving to ensure consistent snapshot - this->flushStagedUpdates(); - this->flushBatch(); + // Flush any pending delete updates before saving to ensure consistent snapshot + this->flushStagedDeleteUpdates(); + this->flushDirtyNodesToDisk(); // Verify that all pending state has been flushed // These assertions ensure data integrity during serialization - if (!pendingVectorIds.empty()) { - throw std::runtime_error("Serialization error: pendingVectorIds not empty after flush"); - } if (!stagedInsertUpdates.empty()) { throw std::runtime_error("Serialization error: stagedInsertUpdates not empty after flush"); } @@ -280,9 +275,6 @@ void HNSWDiskIndex::saveIndexIMP(std::ofstream &output) { if (!rawVectorsInRAM.empty()) { throw std::runtime_error("Serialization error: rawVectorsInRAM not empty after flush"); } - if (pendingVectorCount != 0) { - throw std::runtime_error("Serialization error: pendingVectorCount not zero after flush"); - } if (!stagedRepairUpdates.empty()) { throw std::runtime_error("Serialization error: stagedRepairUpdates not empty after flush"); } @@ -612,8 +604,10 @@ void HNSWDiskIndex::restoreIndexFields(std::ifstream &input) Serializer::readBinaryPOD(input, this->maxLevel); Serializer::readBinaryPOD(input, this->entrypointNode); - // Restore batch processing configuration - Serializer::readBinaryPOD(input, this->batchThreshold); + // Restore batch processing configuration (legacy field, now unused - read and discard) + size_t legacyBatchThreshold; + Serializer::readBinaryPOD(input, legacyBatchThreshold); + (void)legacyBatchThreshold; // Suppress unused warning // Restore dbPath (string: length + data) size_t dbPathLength; @@ -700,9 +694,6 @@ void HNSWDiskIndex::restoreGraph(std::ifstream &input, "RocksDB checkpoint loaded. Vectors will be loaded on-demand from disk during queries."); // Clear any pending state (must be empty after deserialization) - this->pendingVectorIds.clear(); - this->pendingMetadata.clear(); - this->pendingVectorCount = 0; this->stagedInsertUpdates.clear(); this->stagedDeleteUpdates.clear(); this->stagedInsertNeighborUpdates.clear(); @@ -745,8 +736,9 @@ void HNSWDiskIndex::saveIndexFields(std::ofstream &output) c Serializer::writeBinaryPOD(output, this->maxLevel); Serializer::writeBinaryPOD(output, this->entrypointNode); - // Save batch processing configuration - Serializer::writeBinaryPOD(output, this->batchThreshold); + // Save batch processing configuration (legacy field for compatibility, write 0) + size_t legacyBatchThreshold = 0; + Serializer::writeBinaryPOD(output, legacyBatchThreshold); // Save dbPath (string: length + data) size_t dbPathLength = this->dbPath.length(); diff --git a/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp b/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp index d914673ab..81b0ff3eb 100644 --- a/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp +++ b/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp @@ -447,25 +447,13 @@ int main(int argc, char *argv[]) { // Print progress every 1 seconds size_t current_indexed = VecSimIndex_IndexSize(index); - size_t pending = 0; - size_t processing = 0; size_t queue_size = mock_thread_pool->jobQ.size(); - if (type == VecSimType_FLOAT32) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - pending = disk_index->getPendingInsertCount(); - processing = disk_index->getProcessingBatchCount(); - } - } - auto elapsed = std::chrono::duration_cast( std::chrono::steady_clock::now() - start_time).count(); if (current_indexed != last_indexed || elapsed % 1 == 0) { std::cout << "\rIndexed: " << current_indexed << "/" << num_vectors - << " | Pending: " << pending - << " | Processing: " << processing << " | Queue: " << queue_size << " | Time: " << elapsed << "s " << std::flush; last_indexed = current_indexed; @@ -475,56 +463,7 @@ int main(int argc, char *argv[]) { } std::cout << "\n"; - // Flush any remaining pending vectors and wait for those jobs too - // Keep flushing until all pending vectors are processed - size_t pending = 1; // Start with non-zero to enter loop - while (pending > 0) { - // First wait for any in-progress batch to complete - mock_thread_pool->thread_pool_wait(); - - // Now try to flush remaining pending vectors - if (type == VecSimType_FLOAT32) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - disk_index->flushBatch(); - pending = disk_index->getPendingInsertCount(); - } - } else if (type == VecSimType_FLOAT64) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - disk_index->flushBatch(); - pending = disk_index->getPendingInsertCount(); - } - } else if (type == VecSimType_BFLOAT16) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - disk_index->flushBatch(); - pending = disk_index->getPendingInsertCount(); - } - } else if (type == VecSimType_FLOAT16) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - disk_index->flushBatch(); - pending = disk_index->getPendingInsertCount(); - } - } else if (type == VecSimType_INT8) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - disk_index->flushBatch(); - pending = disk_index->getPendingInsertCount(); - } - } else if (type == VecSimType_UINT8) { - auto *disk_index = dynamic_cast *>(index); - if (disk_index) { - disk_index->flushBatch(); - pending = disk_index->getPendingInsertCount(); - } - } else { - break; // Unknown type, exit loop - } - } - - // Final wait for all jobs to complete + // Wait for all background jobs to complete (batchless mode - no pending vectors) mock_thread_pool->thread_pool_wait(); std::cout << "All background jobs completed.\n"; diff --git a/tests/unit/test_hnsw_disk.cpp b/tests/unit/test_hnsw_disk.cpp index 790c41ea3..a69800de2 100644 --- a/tests/unit/test_hnsw_disk.cpp +++ b/tests/unit/test_hnsw_disk.cpp @@ -375,12 +375,9 @@ TEST_F(HNSWDiskIndexTest, AddVectorTest) { auto test_vector2 = createRandomVector(dim, rng); normalizeVector(test_vector2); - // Add vector to batch (should be stored in memory) + // Add vector (written to disk immediately in batchless mode) index.addVector(test_vector2.data(), label2); - // Force flush the batch to disk - index.flushBatch(); - // Test 3: Query to verify both vectors were added correctly VecSimQueryParams queryParams; queryParams.hnswRuntimeParams.efRuntime = 50; @@ -494,10 +491,7 @@ TEST_F(HNSWDiskIndexTest, BatchingTest) { EXPECT_EQ(index.indexSize(), 15); EXPECT_EQ(index.indexLabelCount(), 15); - // Force flush to process the batch - index.flushBatch(); - - // Verify that vectors are now on disk + // Vectors are written to disk immediately in batchless mode EXPECT_EQ(index.indexSize(), 15); EXPECT_EQ(index.indexLabelCount(), 15); @@ -586,8 +580,7 @@ TEST_F(HNSWDiskIndexTest, HierarchicalSearchTest) { index.addVector(test_vectors[i].data(), labels[i]); } - // Force flush to process all vectors - index.flushBatch(); + // Vectors are written to disk immediately in batchless mode std::cout << "\n=== Index state after adding vectors ===" << std::endl; std::cout << "Index size: " << index.indexSize() << std::endl; @@ -1057,8 +1050,7 @@ TEST_F(HNSWDiskIndexTest, BatchedDeletionTest) { index.addVector(vec.data(), label); } - // Flush any pending batches - index.flushBatch(); + // Vectors are written to disk immediately in batchless mode // Verify all vectors were added ASSERT_EQ(index.indexSize(), n); @@ -1168,8 +1160,7 @@ TEST_F(HNSWDiskIndexTest, InterleavedInsertDeleteTest) { ASSERT_EQ(ret, 1) << "Failed to add vector " << label; } - // Flush any pending batches - index.flushBatch(); + // Vectors are written to disk immediately in batchless mode ASSERT_EQ(index.indexSize(), initial_count); ASSERT_EQ(index.indexLabelCount(), initial_count); @@ -1199,8 +1190,7 @@ TEST_F(HNSWDiskIndexTest, InterleavedInsertDeleteTest) { ASSERT_EQ(insert_ret, 1) << "Failed to add vector " << insert_label; } - // Flush any pending batches - index.flushBatch(); + // Vectors are written to disk immediately, flush pending deletes index.flushDeleteBatch(); // Verify index state @@ -1271,8 +1261,7 @@ TEST_F(HNSWDiskIndexTest, InterleavedInsertDeleteTest) { } } - // Flush any pending batches - index.flushBatch(); + // Vectors are written to disk immediately, flush pending deletes index.flushDeleteBatch(); // Final verification @@ -1337,8 +1326,7 @@ TEST_F(HNSWDiskIndexTest, StagedRepairTest) { ASSERT_EQ(ret, 1) << "Failed to add vector " << label; } - // Flush to disk so all graph data is persisted - index.flushBatch(); + // Vectors are written to disk immediately in batchless mode ASSERT_EQ(index.indexSize(), n); ASSERT_EQ(index.indexLabelCount(), n); @@ -1479,8 +1467,7 @@ TEST_F(HNSWDiskIndexTest, GraphRepairBidirectionalEdges) { } } - // Flush to disk - index.flushBatch(); + // Vectors are written to disk immediately in batchless mode ASSERT_EQ(index.indexSize(), n); // Delete a vector from the middle of a cluster (should trigger repair) @@ -1594,8 +1581,7 @@ TEST_F(HNSWDiskIndexTest, UnidirectionalEdgeCleanup) { ASSERT_EQ(ret, 1) << "Failed to add vector " << label; } - // Flush to disk - index.flushBatch(); + // Vectors are written to disk immediately in batchless mode ASSERT_EQ(index.indexSize(), n); // Delete some vectors - this may create unidirectional dangling edges @@ -1711,8 +1697,7 @@ TEST_F(HNSWDiskIndexTest, GraphRepairWithHeuristic) { ASSERT_EQ(ret, 1) << "Failed to add vector " << label; } - // Flush to disk - index.flushBatch(); + // Vectors are written to disk immediately in batchless mode ASSERT_EQ(index.indexSize(), n); // Delete a vector that likely has many neighbors diff --git a/tests/unit/test_quantized_hnsw_disk.cpp b/tests/unit/test_quantized_hnsw_disk.cpp index f85432029..675b65e18 100644 --- a/tests/unit/test_quantized_hnsw_disk.cpp +++ b/tests/unit/test_quantized_hnsw_disk.cpp @@ -104,8 +104,7 @@ TEST_F(QuantizedHNSWDiskTest, BasicQuantization) { ASSERT_EQ(index->indexSize(), n_vectors); - // Flush batch to process vectors and build graph - index->flushBatch(); + // Vectors are now written to disk immediately in batchless mode // Test query auto query = vectors[0]; @@ -182,8 +181,7 @@ TEST_F(QuantizedHNSWDiskTest, QuantizationAccuracy) { VecSimIndex_AddVector(index, v2.data(), 2); VecSimIndex_AddVector(index, v3.data(), 3); - // Flush batch to process vectors and build graph - index->flushBatch(); + // Vectors are now written to disk immediately in batchless mode // Query with v1 - should find itself first auto *results = VecSimIndex_TopKQuery(index, v1.data(), 3, nullptr, BY_SCORE); From a94f2227a469442f83151f5b99f7cc8c03494202 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Mon, 22 Dec 2025 10:05:47 +0200 Subject: [PATCH 20/34] remove more old functions --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 314 ++---------------- .../algorithms/hnsw/hnsw_disk_serializer.h | 3 - 2 files changed, 24 insertions(+), 293 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 645929abd..8da95dec4 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -146,18 +146,9 @@ struct HNSWDiskInsertJob : public AsyncJob { elementMaxLevel(elementMaxLevel_) {} }; -/** - * Definition of a job that flushes staged graph updates to RocksDB. - * This job must execute after all insert jobs complete. - */ -struct HNSWDiskFlushJob : public AsyncJob { - HNSWDiskFlushJob(std::shared_ptr allocator, JobCallback flushCb, - VecSimIndex *index_) - : AsyncJob(allocator, HNSW_DISK_FLUSH_UPDATES_JOB, flushCb, index_) {} -}; /** - * Definition of a batchless job that inserts a single vector completely from start to end. + * Definition of the job that inserts a single vector completely from start to end. * Each job is self-contained and writes directly to disk upon completion. * No batching or staging - optimized for workloads where disk write is cheap * but reading (searching for neighbors) is the bottleneck. @@ -310,37 +301,6 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Uses shared_mutex for better read concurrency - multiple readers can access simultaneously mutable std::shared_mutex stagedUpdatesGuard; - // Atomic counter for pending insert jobs (for synchronization with flush job) - std::atomic pendingInsertJobsCounter{0}; - - // Flag indicating if a batch is currently being processed - std::atomic batchInProgress{false}; - - // Reference to the flush job that will be submitted after all insert jobs complete - HNSWDiskFlushJob *pendingFlushJob = nullptr; - - // Double-buffering for pending vectors - // processingBatch contains vectors currently being inserted by worker threads - struct ProcessingBatch { - vecsim_stl::vector vectorIds; - std::unordered_map rawVectors; // Original float vectors - std::unordered_map processedVectors; // Processed/quantized vectors - size_t count = 0; - - ProcessingBatch(std::shared_ptr allocator) : vectorIds(allocator) {} - - void clear() { - vectorIds.clear(); - rawVectors.clear(); - processedVectors.clear(); - count = 0; - } - }; - std::unique_ptr processingBatch; - - // Lock for swapping between pending and processing batches - mutable std::mutex batchSwapGuard; - // Lock for protecting vectors container during concurrent access // Needed because addElement can resize the container, invalidating pointers mutable std::shared_mutex vectorsGuard; @@ -476,12 +436,7 @@ class HNSWDiskIndex : public VecSimIndexAbstract void emplaceHeap(vecsim_stl::abstract_priority_queue &heap, DistType dist, idType id) const; void getNeighbors(idType nodeId, size_t level, vecsim_stl::vector& result) const; - void getNeighborsAndVector(idType nodeId, size_t level, vecsim_stl::vector& result, void* vector_data) const; - void getNeighborsAndVector(labelType nodeId, size_t level, vecsim_stl::vector& result, void* vector_data) const; size_t getNeighborsCount(idType nodeId, size_t level) const; - // Thread-safe atomic check-and-add for neighbor updates - // Returns true if neighbor was added (had capacity), false if full (needs re-evaluation) - bool tryAddNeighborIfCapacity(idType nodeId, size_t level, idType newNeighborId, size_t maxCapacity); // Manual control of staged delete updates void flushStagedDeleteUpdates(); // Manually flush any pending staged delete updates @@ -515,10 +470,11 @@ class HNSWDiskIndex : public VecSimIndexAbstract void flushStagedGraphUpdates(vecsim_stl::vector& graphUpdates, vecsim_stl::vector& neighborUpdates); - // New method for handling neighbor connection updates when neighbor lists are full - void stageRevisitNeighborConnections(idType new_node_id, idType selected_neighbor, - size_t level, DistType distance); - + // Re-evaluate neighbor connections when a neighbor's list is full + // Applies heuristic to select best neighbors including the new node + void revisitNeighborConnections(idType new_node_id, idType selected_neighbor, + size_t level, DistType distance, + vecsim_stl::vector &modifiedNodes); public: // Methods needed by benchmark framework @@ -757,19 +713,10 @@ HNSWDiskIndex::HNSWDiskIndex( throw std::runtime_error("HNSW index parameter M cannot be 1"); mult = 1 / log(1.0 * M); levelGenerator.seed(random_seed); - - // Initialize processing batch for double-buffering - processingBatch = std::make_unique(this->allocator); } template HNSWDiskIndex::~HNSWDiskIndex() { - // Clean up any pending flush job - if (pendingFlushJob) { - delete pendingFlushJob; - pendingFlushJob = nullptr; - } - // Clear any staged updates before destruction stagedInsertUpdates.clear(); stagedInsertMap.clear(); @@ -1200,10 +1147,9 @@ void HNSWDiskIndex::flushStagedGraphUpdates( } template -void HNSWDiskIndex::stageRevisitNeighborConnections(idType new_node_id, - idType selected_neighbor, - size_t level, - DistType distance) { +void HNSWDiskIndex::revisitNeighborConnections( + idType new_node_id, idType selected_neighbor, size_t level, DistType distance, + vecsim_stl::vector &modifiedNodes) { // Read the neighbor's current neighbor list from cache (source of truth) vecsim_stl::vector existing_neighbors(this->allocator); getNeighborsFromCache(selected_neighbor, level, existing_neighbors); @@ -1216,52 +1162,28 @@ void HNSWDiskIndex::stageRevisitNeighborConnections(idType n candidates.emplace_back(distance, new_node_id); // Add existing neighbors with their distances to the selected neighbor - const void* selected_neighbor_data = getDataByInternalId(selected_neighbor); - for (size_t j = 0; j < existing_neighbors.size(); j++) { - idType existing_neighbor_id = existing_neighbors[j]; - const void *existing_neighbor_data = getDataByInternalId(existing_neighbor_id); - DistType existing_distance = - this->calcDistance(existing_neighbor_data, selected_neighbor_data); - candidates.emplace_back(existing_distance, existing_neighbor_id); + const void *selected_neighbor_data = getDataByInternalId(selected_neighbor); + for (idType existing_neighbor_id : existing_neighbors) { + const void *existing_data = getDataByInternalId(existing_neighbor_id); + DistType existing_dist = this->calcDistance(existing_data, selected_neighbor_data); + candidates.emplace_back(existing_dist, existing_neighbor_id); } - // Use the heuristic to select the best neighbors (similar to revisitNeighborConnections in - // hnsw.h) - size_t max_M_cur = level ? M : M0; - // Apply the neighbor selection heuristic + size_t max_M_cur = level ? M : M0; vecsim_stl::vector removed_candidates(this->allocator); getNeighborsByHeuristic2(candidates, max_M_cur, removed_candidates); - // Check if the new node was selected as a neighbor - bool new_node_selected = false; + // Extract selected neighbor IDs + vecsim_stl::vector new_neighbors(this->allocator); + new_neighbors.reserve(candidates.size()); for (const auto &candidate : candidates) { - if (candidate.second == new_node_id) { - new_node_selected = true; - break; - } + new_neighbors.push_back(candidate.second); } - if (new_node_selected) { - // The new node was selected, so we need to update the neighbor's neighbor list - // Extract the selected neighbor IDs and update cache directly - vecsim_stl::vector selected_neighbor_ids(this->allocator); - selected_neighbor_ids.reserve(candidates.size()); - for (const auto &candidate : candidates) { - selected_neighbor_ids.push_back(candidate.second); - } - - // Update the selected_neighbor's neighbor list in cache (not a new node) - setNeighborsInCache(selected_neighbor, level, selected_neighbor_ids, false /* isNewNode */); - - // Also add bidirectional connection from new node to selected neighbor - addNeighborToCache(new_node_id, level, selected_neighbor); - - } else { - // The new node was not selected, so we only need to add unidirectional connection - // from new node to selected neighbor - addNeighborToCache(new_node_id, level, selected_neighbor); - } + // Update cache with the new neighbor list + setNeighborsInCache(selected_neighbor, level, new_neighbors); + modifiedNodes.push_back(makeRepairKey(selected_neighbor, level)); } template @@ -1439,15 +1361,6 @@ bool HNSWDiskIndex::getRawVectorInternal(idType id, void* ou } } - // Also check processingBatch (for vectors being processed by worker threads) - if (processingBatch) { - auto pit = processingBatch->rawVectors.find(id); - if (pit != processingBatch->rawVectors.end()) { - std::memcpy(output_buffer, pit->second.data(), this->inputBlobSize); - return true; - } - } - // If not in RAM, retrieve from disk (NO LOCK - caller is expected to hold lock) GraphKey graphKey(id, 0); std::string level0_graph_value; @@ -1932,84 +1845,6 @@ void HNSWDiskIndex::getNeighbors(idType nodeId, size_t level // during graph maintenance operations, not during search. } -template -void HNSWDiskIndex::getNeighborsAndVector(idType nodeId, size_t level, vecsim_stl::vector& result, void* vector_data) const { - // Clear the result vector first - result.clear(); - - uint64_t lookup_key = makeRepairKey(nodeId, level); - bool foundNeighborsInCache = false; - - // Step 1: Check cache first (source of truth for pending updates) - { - size_t segment = getSegmentIndex(lookup_key); - auto& cacheSegment = cacheSegments_[segment]; - std::shared_lock cacheLock(cacheSegment.guard); - auto it = cacheSegment.cache.find(lookup_key); - if (it != cacheSegment.cache.end()) { - result.reserve(it->second.size()); - for (idType id : it->second) { - result.push_back(id); - } - foundNeighborsInCache = true; - } else if (cacheSegment.newNodes.find(lookup_key) != cacheSegment.newNodes.end()) { - // New node not yet connected - return empty neighbors - foundNeighborsInCache = true; - } - } - - // Check rawVectorsInRAM with shared lock - bool foundVectorInRAM = false; - { - std::shared_lock lock(rawVectorsGuard); - auto it = rawVectorsInRAM.find(nodeId); - if (it != rawVectorsInRAM.end()) { - std::memcpy(vector_data, it->second.data(), this->inputBlobSize); - foundVectorInRAM = true; - } - } - // Check processingBatch if not found in RAM - if (!foundVectorInRAM && processingBatch) { - auto it = processingBatch->rawVectors.find(nodeId); - if (it != processingBatch->rawVectors.end()) { - std::memcpy(vector_data, it->second.data(), this->inputBlobSize); - foundVectorInRAM = true; - } - } - if (foundNeighborsInCache && foundVectorInRAM) { - return; - } - - // If not found in cache, check disk - GraphKey graphKey(nodeId, level); - std::string graph_value; - rocksdb::Status status = - db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); - - if (status.ok()) { - // Only update neighbors if we didn't find them in cache - if (!foundNeighborsInCache) { - deserializeGraphValue(graph_value, result); - } - // Only update vector if we didn't find it in RAM - if (!foundVectorInRAM) { - std::memcpy(vector_data, graph_value.data(), this->inputBlobSize); - } - } -} - -template -void HNSWDiskIndex::getNeighborsAndVector(labelType nodeId, size_t level, vecsim_stl::vector& result, void* vector_data) const { - // Check if label exists in the map (it won't if it's been marked as deleted) - auto it = labelToIdMap.find(nodeId); - if (it == labelToIdMap.end()) { - // Label doesn't exist (has been marked as deleted), return empty neighbors - result.clear(); - return; - } - getNeighborsAndVector(it->second, level, result, vector_data); -} - template size_t HNSWDiskIndex::getNeighborsCount(idType nodeId, size_t level) const { uint64_t lookup_key = makeRepairKey(nodeId, level); @@ -2044,77 +1879,6 @@ size_t HNSWDiskIndex::getNeighborsCount(idType nodeId, size_ return 0; } -template -bool HNSWDiskIndex::tryAddNeighborIfCapacity( - idType nodeId, size_t level, idType newNeighborId, size_t maxCapacity) { - // Atomic check-and-add using cache segments: O(1) lookup, 1/64th lock contention - // The cache is the source of truth - it contains either disk data or pending updates - - uint64_t key = makeRepairKey(nodeId, level); - size_t segment = getSegmentIndex(key); - auto& cacheSegment = cacheSegments_[segment]; - - // First, try with just a shared lock for the common case (already in cache) - { - std::shared_lock readLock(cacheSegment.guard); - auto it = cacheSegment.cache.find(key); - if (it != cacheSegment.cache.end()) { - // Fast path: already in cache, check if we can add - if (it->second.size() >= maxCapacity) { - return false; // Full, no need to upgrade lock - } - } - } - - // Need to modify - get exclusive lock - std::unique_lock lock(cacheSegment.guard); - - auto it = cacheSegment.cache.find(key); - if (it == cacheSegment.cache.end()) { - // Check if this is a new node (never written to disk) - skip disk lookup - bool isNewNode = (cacheSegment.newNodes.find(key) != cacheSegment.newNodes.end()); - - if (!isNewNode) { - // Not in cache and not a new node - need to load from disk - // Keep lock held to avoid multiple threads loading the same data - vecsim_stl::vector diskNeighbors(this->allocator); - GraphKey graphKey(nodeId, level); - std::string graph_value; - rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); - if (status.ok()) { - deserializeGraphValue(graph_value, diskNeighbors); - } - - std::vector cacheEntry; - cacheEntry.reserve(diskNeighbors.size()); - for (idType id : diskNeighbors) { - cacheEntry.push_back(id); - } - cacheSegment.cache[key] = std::move(cacheEntry); - } else { - // New node - initialize with empty neighbor list - cacheSegment.cache[key] = std::vector(); - } - } - - // Now we have the current neighbor list in cache - check capacity - auto& neighbors = cacheSegment.cache[key]; - if (neighbors.size() < maxCapacity) { - // Check for duplicates - if (std::find(neighbors.begin(), neighbors.end(), newNeighborId) == neighbors.end()) { - neighbors.push_back(newNeighborId); - // Mark as dirty and increment atomic counter - auto insertResult = cacheSegment.dirty.insert(key); - if (insertResult.second) { - totalDirtyCount_.fetch_add(1, std::memory_order_relaxed); - } - } - return true; // Successfully added (or already present) - } - - return false; // Neighbor list is full, needs re-evaluation -} - /********************************** Multi-threaded Job Execution **********************************/ template @@ -3315,38 +3079,8 @@ idType HNSWDiskIndex::mutuallyConnectNewElement( modifiedNodes.push_back(makeRepairKey(selected_neighbor, level)); } else { // Full - need to re-evaluate using heuristic - // Get current neighbors (might have changed since check) - vecsim_stl::vector existing_neighbors(this->allocator); - getNeighborsFromCache(selected_neighbor, level, existing_neighbors); - - candidatesList candidates(this->allocator); - candidates.reserve(existing_neighbors.size() + 1); - - // Add new node - candidates.emplace_back(distance, new_node_id); - - // Add existing neighbors with their distances - const void *selected_neighbor_data = getDataByInternalId(selected_neighbor); - for (idType existing_neighbor_id : existing_neighbors) { - const void *existing_data = getDataByInternalId(existing_neighbor_id); - DistType existing_dist = this->calcDistance(existing_data, selected_neighbor_data); - candidates.emplace_back(existing_dist, existing_neighbor_id); - } - - // Apply heuristic - vecsim_stl::vector removed_candidates(this->allocator); - getNeighborsByHeuristic2(candidates, max_M_cur, removed_candidates); - - // Extract selected neighbor IDs - vecsim_stl::vector new_neighbors(this->allocator); - new_neighbors.reserve(candidates.size()); - for (const auto &candidate : candidates) { - new_neighbors.push_back(candidate.second); - } - - // Update cache with the new neighbor list - setNeighborsInCache(selected_neighbor, level, new_neighbors); - modifiedNodes.push_back(makeRepairKey(selected_neighbor, level)); + revisitNeighborConnections(new_node_id, selected_neighbor, level, distance, + modifiedNodes); } } diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h index 1e17ad78f..9546574c1 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk_serializer.h @@ -74,9 +74,6 @@ HNSWDiskIndex::HNSWDiskIndex( // Add a constant offset to avoid seed=0 for empty indexes this->levelGenerator.seed(200 + this->curElementCount.load(std::memory_order_acquire)); - // Initialize processing batch for double-buffering (needed for async operations) - this->processingBatch = std::make_unique(this->allocator); - // Restore graph and vectors from file this->restoreGraph(input, version); this->restoreVectors(input, version); From 119ef4cb2faddc7541dcbf32e4250c522937baa0 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Mon, 22 Dec 2025 14:05:04 +0200 Subject: [PATCH 21/34] Critical Race Condition --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 210 +++++++++++++++---------- 1 file changed, 124 insertions(+), 86 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 8da95dec4..e52af9a19 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -2829,20 +2829,34 @@ void HNSWDiskIndex::writeDirtyNodesToDisk( return; } + // ==================== SWAP-AND-FLUSH PATTERN ==================== + // To avoid the "Lost Update" race condition, we atomically clear the dirty flag + // BEFORE writing to disk (not after). If a new insert occurs while we're writing, + // it will simply re-add the node to the dirty set. No updates are lost. + // ================================================================ + auto writeOptions = rocksdb::WriteOptions(); writeOptions.disableWAL = true; rocksdb::WriteBatch batch; std::vector rawVectorBuffer(this->inputBlobSize); + vecsim_stl::vector nodesToWrite(this->allocator); + + // ==================== SWAP-AND-FLUSH PATTERN ==================== + // Two-phase approach to minimize lock contention while preventing lost updates: + // Phase 1: Read cache data under SHARED lock (allows concurrent writers) + // Phase 2: Clear dirty flags under EXCLUSIVE lock (brief, after all reads done) + // If writing fails, we re-add nodes to dirty set. + // ================================================================ - // Process each modified node - lock only the relevant segment for each + // Phase 1: Read cache data under shared locks (allows other writers to proceed) for (uint64_t key : modifiedNodes) { idType nodeId = static_cast(key >> 32); size_t level = static_cast(key & 0xFFFFFFFF); size_t segment = getSegmentIndex(key); auto& cacheSegment = cacheSegments_[segment]; - // Get neighbors from cache under segment lock + // Get neighbors from cache under SHARED lock (concurrent reads allowed) vecsim_stl::vector neighbors(this->allocator); { std::shared_lock segmentLock(cacheSegment.guard); @@ -2856,6 +2870,7 @@ void HNSWDiskIndex::writeDirtyNodesToDisk( neighbors.push_back(id); } } + // Shared lock released - other threads can add to cache // Get raw vector data (no cache lock held during disk I/O) const void *rawData; @@ -2874,17 +2889,12 @@ void HNSWDiskIndex::writeDirtyNodesToDisk( GraphKey graphKey(nodeId, level); std::string value = serializeGraphValue(rawData, neighbors); batch.Put(cf, graphKey.asSlice(), value); + nodesToWrite.push_back(key); } - // Atomic write - rocksdb::Status status = db->Write(writeOptions, &batch); - if (!status.ok()) { - this->log(VecSimCommonStrings::LOG_WARNING_STRING, - "ERROR: Failed to write dirty nodes to disk: %s", status.ToString().c_str()); - } - - // Clear dirty flags for written nodes - lock each segment individually - for (uint64_t key : modifiedNodes) { + // Phase 2: Clear dirty flags BEFORE writing (swap-and-flush pattern) + // Brief exclusive locks - one segment at a time + for (uint64_t key : nodesToWrite) { size_t segment = getSegmentIndex(key); auto& cacheSegment = cacheSegments_[segment]; std::unique_lock segmentLock(cacheSegment.guard); @@ -2892,114 +2902,142 @@ void HNSWDiskIndex::writeDirtyNodesToDisk( totalDirtyCount_.fetch_sub(1, std::memory_order_relaxed); } } + + if (nodesToWrite.empty()) { + return; + } + + // Phase 3: Write to disk (no locks held) + // Dirty flags already cleared in Phase 2 - any concurrent modifications + // will re-add to dirty set and be picked up in next flush + rocksdb::Status status = db->Write(writeOptions, &batch); + if (!status.ok()) { + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "ERROR: Failed to write dirty nodes to disk: %s", status.ToString().c_str()); + + // On failure, re-add nodes to dirty set so they'll be retried + for (uint64_t key : nodesToWrite) { + size_t segment = getSegmentIndex(key); + auto& cacheSegment = cacheSegments_[segment]; + std::unique_lock segmentLock(cacheSegment.guard); + if (cacheSegment.dirty.insert(key).second) { + totalDirtyCount_.fetch_add(1, std::memory_order_relaxed); + } + } + } + // Success case: dirty flags already cleared, nothing more to do } template void HNSWDiskIndex::flushDirtyNodesToDisk() { std::lock_guard flushLock(diskWriteGuard); - // Collect all dirty nodes from all segments - vecsim_stl::vector nodesToFlush(this->allocator); - // Only track level 0 nodes for raw vector removal (level 0 contains the actual vector data) - std::unordered_set level0VectorIds; - // Check if there are any dirty nodes using atomic counter (fast path) if (totalDirtyCount_.load(std::memory_order_relaxed) == 0) { return; } - // Collect dirty nodes from all segments - for (size_t s = 0; s < NUM_CACHE_SEGMENTS; ++s) { - auto& cacheSegment = cacheSegments_[s]; - std::shared_lock segmentLock(cacheSegment.guard); - for (uint64_t key : cacheSegment.dirty) { - nodesToFlush.push_back(key); - size_t level = static_cast(key & 0xFFFFFFFF); - if (level == 0) { - idType nodeId = static_cast(key >> 32); - level0VectorIds.insert(nodeId); - } - } - } - - if (nodesToFlush.empty()) { - return; - } + // ==================== SWAP-AND-FLUSH PATTERN ==================== + // Process one segment at a time to minimize lock contention: + // 1. Acquire exclusive lock on segment (blocks inserts briefly - nanoseconds for swap) + // 2. Swap dirty set with empty set (atomic ownership transfer) + // 3. Release lock immediately (segment is free for new inserts) + // 4. Process swapped nodes and write to disk (no locks held) + // + // If a new insert happens after the swap, it sees an empty dirty set + // and correctly adds itself. No updates are lost. + // ================================================================ - // Build write batch auto writeOptions = rocksdb::WriteOptions(); writeOptions.disableWAL = true; rocksdb::WriteBatch batch; std::vector rawVectorBuffer(this->inputBlobSize); - vecsim_stl::vector successfullyFlushed(this->allocator); std::unordered_set successfulLevel0Ids; - // Process each node - lock only the relevant segment - for (uint64_t key : nodesToFlush) { - idType nodeId = static_cast(key >> 32); - size_t level = static_cast(key & 0xFFFFFFFF); - size_t segment = getSegmentIndex(key); - auto& cacheSegment = cacheSegments_[segment]; + // Process each segment independently + for (size_t s = 0; s < NUM_CACHE_SEGMENTS; ++s) { + auto& cacheSegment = cacheSegments_[s]; + + // Local container to take ownership of dirty nodes + std::unordered_set nodesToFlush; - // Get neighbors from cache under segment lock - vecsim_stl::vector neighbors(this->allocator); { - std::shared_lock segmentLock(cacheSegment.guard); - auto it = cacheSegment.cache.find(key); - if (it == cacheSegment.cache.end()) { + // 1. Acquire exclusive lock (brief - just for the swap) + std::unique_lock segmentLock(cacheSegment.guard); + + if (cacheSegment.dirty.empty()) { continue; } - const std::vector &cacheNeighbors = it->second; - neighbors.reserve(cacheNeighbors.size()); - for (idType id : cacheNeighbors) { - neighbors.push_back(id); - } - } - // Get raw vector data from RAM or disk (no cache lock held) - if (!getRawVectorInternal(nodeId, rawVectorBuffer.data())) { - // Skip this node - raw vector not available yet - // This can happen for neighbor updates to nodes that haven't been flushed yet - continue; - } + // 2. Atomic swap - move dirty set contents to local, leave dirty empty + // New inserts after this will add to the now-empty dirty set + nodesToFlush.swap(cacheSegment.dirty); - GraphKey graphKey(nodeId, level); - std::string value = serializeGraphValue(rawVectorBuffer.data(), neighbors); - batch.Put(cf, graphKey.asSlice(), value); - successfullyFlushed.push_back(key); - if (level == 0) { - successfulLevel0Ids.insert(nodeId); + // 3. Update global counter - we've taken ownership of these nodes + totalDirtyCount_.fetch_sub(nodesToFlush.size(), std::memory_order_relaxed); } - } + // 4. Lock released here - segment is free for inserts - if (successfullyFlushed.empty()) { - return; - } + // 5. Process nodes and add to batch (no locks held) + for (uint64_t key : nodesToFlush) { + idType nodeId = static_cast(key >> 32); + size_t level = static_cast(key & 0xFFFFFFFF); - // Atomic write - rocksdb::Status status = db->Write(writeOptions, &batch); - if (!status.ok()) { - this->log(VecSimCommonStrings::LOG_WARNING_STRING, - "ERROR: Failed to flush dirty nodes to disk: %s", status.ToString().c_str()); - return; // Don't clear dirty flags on failure + // Get neighbors from cache (need shared lock briefly) + vecsim_stl::vector neighbors(this->allocator); + { + std::shared_lock segmentLock(cacheSegment.guard); + auto it = cacheSegment.cache.find(key); + if (it == cacheSegment.cache.end()) { + // Node not in cache - shouldn't happen, but skip if it does + continue; + } + const std::vector &cacheNeighbors = it->second; + neighbors.reserve(cacheNeighbors.size()); + for (idType id : cacheNeighbors) { + neighbors.push_back(id); + } + } + + // Get raw vector data (no segment lock needed) + if (!getRawVectorInternal(nodeId, rawVectorBuffer.data())) { + // Raw vector not available - this shouldn't happen for dirty nodes + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "WARNING: Raw vector not found for dirty node %u at level %zu", nodeId, level); + continue; + } + + GraphKey graphKey(nodeId, level); + std::string value = serializeGraphValue(rawVectorBuffer.data(), neighbors); + batch.Put(cf, graphKey.asSlice(), value); + + if (level == 0) { + successfulLevel0Ids.insert(nodeId); + } + + // Clear newNodes flag for this node (brief exclusive lock) + { + std::unique_lock segmentLock(cacheSegment.guard); + cacheSegment.newNodes.erase(key); + } + } } - // Clear dirty flags and newNodes for successfully written nodes - lock each segment individually - for (uint64_t key : successfullyFlushed) { - size_t segment = getSegmentIndex(key); - auto& cacheSegment = cacheSegments_[segment]; - std::unique_lock segmentLock(cacheSegment.guard); - if (cacheSegment.dirty.erase(key) > 0) { - totalDirtyCount_.fetch_sub(1, std::memory_order_relaxed); + // Write entire batch to RocksDB (no locks held during I/O) + if (batch.Count() > 0) { + rocksdb::Status status = db->Write(writeOptions, &batch); + if (!status.ok()) { + this->log(VecSimCommonStrings::LOG_WARNING_STRING, + "ERROR: Failed to flush dirty nodes to disk: %s", status.ToString().c_str()); + // Note: On failure, data is lost. In production DBs, this is typically fatal. + // The nodes were already removed from dirty set, so they won't be retried. + // Consider: re-adding to dirty set on failure if recovery is needed. } - // Clear newNodes flag - node is now on disk - cacheSegment.newNodes.erase(key); } - // Only remove raw vectors for level 0 nodes that were successfully written - // (These are the nodes that now have their vector data on disk) - { + // Remove raw vectors for level 0 nodes that were written to disk + if (!successfulLevel0Ids.empty()) { std::lock_guard lock(rawVectorsGuard); for (idType vectorId : successfulLevel0Ids) { rawVectorsInRAM.erase(vectorId); From aef11f38b97c24e3997ff8ad7d3e824a53016464 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Mon, 22 Dec 2025 14:26:27 +0200 Subject: [PATCH 22/34] old bm --- .../bm_hnsw_disk_initialize_fp32.h | 77 ++++++++----------- 1 file changed, 33 insertions(+), 44 deletions(-) diff --git a/tests/benchmark/bm_initialization/bm_hnsw_disk_initialize_fp32.h b/tests/benchmark/bm_initialization/bm_hnsw_disk_initialize_fp32.h index 32461b41e..f4d622a8d 100644 --- a/tests/benchmark/bm_initialization/bm_hnsw_disk_initialize_fp32.h +++ b/tests/benchmark/bm_initialization/bm_hnsw_disk_initialize_fp32.h @@ -29,50 +29,39 @@ BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(Disk, HNSWDisk), fp32_ (benchmark::State &st) { Disk(st, INDEX_HNSW_DISK); } BENCHMARK_REGISTER_F(BM_VecSimCommon, BM_FUNC_NAME(Disk, HNSWDisk))->Iterations(1); -// // AddLabel benchmarks -// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_ADD_LABEL, fp32_index_t) -// (benchmark::State &st) { AddLabel(st); } -// REGISTER_AddLabel(BM_ADD_LABEL, INDEX_HNSW_DISK); - -// // TopK benchmark (single-threaded) -// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK, HNSWDisk), fp32_index_t) -// (benchmark::State &st) { TopK_HNSW_DISK(st); } -// REGISTER_TopK_HNSW_DISK(BM_VecSimCommon, BM_FUNC_NAME(TopK, HNSWDisk)); - -// // TopK benchmark (parallel) -// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopKParallel, HNSWDisk), fp32_index_t) -// (benchmark::State &st) { TopK_HNSW_DISK_Parallel(st); } -// REGISTER_TopK_HNSW_DISK_PARALLEL(BM_VecSimCommon, BM_FUNC_NAME(TopKParallel, HNSWDisk)); - -// // TopK benchmark with marked deleted vectors -// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK_MarkDeleted, HNSWDisk), fp32_index_t) -// (benchmark::State &st) { TopK_HNSW_DISK_MarkDeleted(st); } -// REGISTER_TopK_HNSW_DISK_MarkDeleted(BM_VecSimCommon, BM_FUNC_NAME(TopK_MarkDeleted, HNSWDisk)); - -// // TopK benchmark after deleting vectors (with graph repair) -// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel, HNSWDisk), fp32_index_t) -// (benchmark::State &st) { TopK_HNSW_DISK_DeleteLabel(st); } -// REGISTER_TopK_HNSW_DISK_DeleteLabel(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel, HNSWDisk)); -// // AddLabel benchmarks -// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_FLUSH_BATCH_DISK, fp32_index_t) -// (benchmark::State &st) { FlushBatchDisk(st); } -// REGISTER_FlushBatchDisk(BM_FLUSH_BATCH_DISK); -// // -// // TopK benchmark after deleting vectors (with graph repair), protecting GT vectors for stable recall -// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel_ProtectGT, HNSWDisk), fp32_index_t) -// (benchmark::State &st) { TopK_HNSW_DISK_DeleteLabel_ProtectGT(st); } -// REGISTER_TopK_HNSW_DISK_DeleteLabel_ProtectGT(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel_ProtectGT, HNSWDisk)); - -// Range benchmarks -// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_FUNC_NAME(Range, BF), fp32_index_t) -// (benchmark::State &st) { Range_BF(st); } -// REGISTER_Range_BF(BM_FUNC_NAME(Range, BF), fp32_index_t); - -// Range HNSW -// BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_FUNC_NAME(Range, HNSW), fp32_index_t) -// (benchmark::State &st) { Range_HNSW(st); } -// REGISTER_Range_HNSW(BM_FUNC_NAME(Range, HNSW), fp32_index_t); - +// AddLabel benchmarks +BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_ADD_LABEL, fp32_index_t) +(benchmark::State &st) { AddLabel(st); } +REGISTER_AddLabel(BM_ADD_LABEL, INDEX_HNSW_DISK); + +// TopK benchmark (single-threaded) +BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK, HNSWDisk), fp32_index_t) +(benchmark::State &st) { TopK_HNSW_DISK(st); } +REGISTER_TopK_HNSW_DISK(BM_VecSimCommon, BM_FUNC_NAME(TopK, HNSWDisk)); + +// TopK benchmark (parallel) +BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopKParallel, HNSWDisk), fp32_index_t) +(benchmark::State &st) { TopK_HNSW_DISK_Parallel(st); } +REGISTER_TopK_HNSW_DISK_PARALLEL(BM_VecSimCommon, BM_FUNC_NAME(TopKParallel, HNSWDisk)); + +// TopK benchmark with marked deleted vectors +BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK_MarkDeleted, HNSWDisk), fp32_index_t) +(benchmark::State &st) { TopK_HNSW_DISK_MarkDeleted(st); } +REGISTER_TopK_HNSW_DISK_MarkDeleted(BM_VecSimCommon, BM_FUNC_NAME(TopK_MarkDeleted, HNSWDisk)); + +// TopK benchmark after deleting vectors (with graph repair) +BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel, HNSWDisk), fp32_index_t) +(benchmark::State &st) { TopK_HNSW_DISK_DeleteLabel(st); } +REGISTER_TopK_HNSW_DISK_DeleteLabel(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel, HNSWDisk)); +// AddLabel benchmarks +BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimBasics, BM_FLUSH_BATCH_DISK, fp32_index_t) +(benchmark::State &st) { FlushBatchDisk(st); } +REGISTER_FlushBatchDisk(BM_FLUSH_BATCH_DISK); +// +// TopK benchmark after deleting vectors (with graph repair), protecting GT vectors for stable recall +BENCHMARK_TEMPLATE_DEFINE_F(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel_ProtectGT, HNSWDisk), fp32_index_t) +(benchmark::State &st) { TopK_HNSW_DISK_DeleteLabel_ProtectGT(st); } +REGISTER_TopK_HNSW_DISK_DeleteLabel_ProtectGT(BM_VecSimCommon, BM_FUNC_NAME(TopK_DeleteLabel_ProtectGT, HNSWDisk)); // Special disk-based HNSW benchmarks for batch processing with multi-threaded async ingest // Args: {INDEX_HNSW_DISK, thread_count} // This benchmark reloads the disk index for each run since async operations modify the index From c8e7f12712bd117b8b150cb6cf7c6c5ee0b0828e Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Mon, 22 Dec 2025 14:41:43 +0200 Subject: [PATCH 23/34] Remove old --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 4d870eb50..877f81c9b 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -693,7 +693,7 @@ HNSWDiskIndex::HNSWDiskIndex( : VecSimIndexAbstract(abstractInitParams, components), idToMetaData(INITIAL_CAPACITY, this->allocator), labelToIdMap(this->allocator), db(db), cf(cf), dbPath(dbPath), indexDataGuard(), - visitedNodesHandlerPool(INITIAL_CAPACITY, this->allocator), pendingMetadata(this->allocator), pendingVectorCount(0), + visitedNodesHandlerPool(INITIAL_CAPACITY, this->allocator), pendingDeleteIds(this->allocator), num_visited_nodes(0), num_visited_nodes_higher_levels(0), stagedInsertUpdates(this->allocator), stagedDeleteUpdates(this->allocator), stagedRepairUpdates(this->allocator), From a1a12287f9c2b5a370e208123e8aaf48f176c7e9 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Mon, 22 Dec 2025 15:20:41 +0200 Subject: [PATCH 24/34] Segment-local flushing --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 144 ++++++++++++++----------- 1 file changed, 79 insertions(+), 65 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 877f81c9b..01c19f338 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -422,6 +422,13 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Atomic check-and-add for cache - returns true if added, false if at capacity bool tryAddNeighborToCacheIfCapacity(idType nodeId, size_t level, idType newNeighborId, size_t maxCapacity); +private: + // Helper to load neighbors from disk into cache if not already present + // Returns true if data is available in cache after call, false otherwise + // Caller must hold lock on cacheSegment.guard (will be temporarily released during disk I/O) + void loadNeighborsFromDiskIfNeeded(uint64_t key, CacheSegment& cacheSegment, + std::unique_lock& lock); + // Write dirty cache entries to disk void writeDirtyNodesToDisk(const vecsim_stl::vector &modifiedNodes, const void *newVectorRawData, idType newVectorId); @@ -2666,6 +2673,51 @@ void HNSWDiskIndex::getNeighborsFromCache( filterDeletedNodes(result); } +// Helper to load neighbors from disk into cache if not already present +// Caller must hold unique_lock on cacheSegment.guard (will be temporarily released during disk I/O) +template +void HNSWDiskIndex::loadNeighborsFromDiskIfNeeded( + uint64_t key, CacheSegment& cacheSegment, std::unique_lock& lock) { + + auto it = cacheSegment.cache.find(key); + if (it != cacheSegment.cache.end()) { + return; // Already in cache + } + + // Check if this is a new node (never written to disk) - skip disk lookup + bool isNewNode = (cacheSegment.newNodes.find(key) != cacheSegment.newNodes.end()); + + if (!isNewNode) { + // First modification of existing node - need to load current state from disk + idType nodeId = static_cast(key >> 32); + size_t level = static_cast(key & 0xFFFFFFFF); + + lock.unlock(); + vecsim_stl::vector diskNeighbors(this->allocator); + GraphKey graphKey(nodeId, level); + std::string graph_value; + rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); + if (status.ok()) { + deserializeGraphValue(graph_value, diskNeighbors); + } + lock.lock(); + + // Re-check: another thread might have populated it + if (cacheSegment.cache.find(key) == cacheSegment.cache.end()) { + // Convert vecsim_stl::vector to std::vector + std::vector cacheEntry; + cacheEntry.reserve(diskNeighbors.size()); + for (idType id : diskNeighbors) { + cacheEntry.push_back(id); + } + cacheSegment.cache[key] = std::move(cacheEntry); + } + } else { + // New node - initialize with empty neighbor list + cacheSegment.cache[key] = std::vector(); + } +} + template void HNSWDiskIndex::addNeighborToCache( idType nodeId, size_t level, idType newNeighborId) { @@ -2676,38 +2728,8 @@ void HNSWDiskIndex::addNeighborToCache( std::unique_lock lock(cacheSegment.guard); - auto it = cacheSegment.cache.find(key); - if (it == cacheSegment.cache.end()) { - // Check if this is a new node (never written to disk) - skip disk lookup - bool isNewNode = (cacheSegment.newNodes.find(key) != cacheSegment.newNodes.end()); - - if (!isNewNode) { - // First modification of existing node - need to load current state from disk - lock.unlock(); - vecsim_stl::vector diskNeighbors(this->allocator); - GraphKey graphKey(nodeId, level); - std::string graph_value; - rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); - if (status.ok()) { - deserializeGraphValue(graph_value, diskNeighbors); - } - lock.lock(); - - // Re-check: another thread might have populated it - if (cacheSegment.cache.find(key) == cacheSegment.cache.end()) { - // Convert vecsim_stl::vector to std::vector - std::vector cacheEntry; - cacheEntry.reserve(diskNeighbors.size()); - for (idType id : diskNeighbors) { - cacheEntry.push_back(id); - } - cacheSegment.cache[key] = std::move(cacheEntry); - } - } else { - // New node - initialize with empty neighbor list - cacheSegment.cache[key] = std::vector(); - } - } + // Load from disk if not in cache (handles newNodes check internally) + loadNeighborsFromDiskIfNeeded(key, cacheSegment, lock); // Add new neighbor (avoid duplicates) auto &neighbors = cacheSegment.cache[key]; @@ -2732,30 +2754,8 @@ bool HNSWDiskIndex::tryAddNeighborToCacheIfCapacity( std::unique_lock lock(cacheSegment.guard); - auto it = cacheSegment.cache.find(key); - if (it == cacheSegment.cache.end()) { - // First modification - need to load current state from disk - lock.unlock(); - vecsim_stl::vector diskNeighbors(this->allocator); - GraphKey graphKey(nodeId, level); - std::string graph_value; - rocksdb::Status status = db->Get(rocksdb::ReadOptions(), cf, graphKey.asSlice(), &graph_value); - if (status.ok()) { - deserializeGraphValue(graph_value, diskNeighbors); - } - lock.lock(); - - // Re-check: another thread might have populated it - if (cacheSegment.cache.find(key) == cacheSegment.cache.end()) { - // Convert vecsim_stl::vector to std::vector - std::vector cacheEntry; - cacheEntry.reserve(diskNeighbors.size()); - for (idType id : diskNeighbors) { - cacheEntry.push_back(id); - } - cacheSegment.cache[key] = std::move(cacheEntry); - } - } + // Load from disk if not in cache (handles newNodes check internally) + loadNeighborsFromDiskIfNeeded(key, cacheSegment, lock); // Atomic check-and-add under the lock auto &neighbors = cacheSegment.cache[key]; @@ -2964,6 +2964,9 @@ void HNSWDiskIndex::flushDirtyNodesToDisk() { std::vector rawVectorBuffer(this->inputBlobSize); std::unordered_set successfulLevel0Ids; + // Track all nodes we're trying to write (for error recovery) + vecsim_stl::vector allNodesToFlush(this->allocator); + // Process each segment independently for (size_t s = 0; s < NUM_CACHE_SEGMENTS; ++s) { auto& cacheSegment = cacheSegments_[s]; @@ -3020,16 +3023,11 @@ void HNSWDiskIndex::flushDirtyNodesToDisk() { GraphKey graphKey(nodeId, level); std::string value = serializeGraphValue(rawVectorBuffer.data(), neighbors); batch.Put(cf, graphKey.asSlice(), value); + allNodesToFlush.push_back(key); if (level == 0) { successfulLevel0Ids.insert(nodeId); } - - // Clear newNodes flag for this node (brief exclusive lock) - { - std::unique_lock segmentLock(cacheSegment.guard); - cacheSegment.newNodes.erase(key); - } } } @@ -3039,12 +3037,28 @@ void HNSWDiskIndex::flushDirtyNodesToDisk() { if (!status.ok()) { this->log(VecSimCommonStrings::LOG_WARNING_STRING, "ERROR: Failed to flush dirty nodes to disk: %s", status.ToString().c_str()); - // Note: On failure, data is lost. In production DBs, this is typically fatal. - // The nodes were already removed from dirty set, so they won't be retried. - // Consider: re-adding to dirty set on failure if recovery is needed. + + // On failure, re-add nodes to dirty set so they'll be retried + for (uint64_t key : allNodesToFlush) { + size_t segment = getSegmentIndex(key); + auto& cacheSegment = cacheSegments_[segment]; + std::unique_lock segmentLock(cacheSegment.guard); + if (cacheSegment.dirty.insert(key).second) { + totalDirtyCount_.fetch_add(1, std::memory_order_relaxed); + } + } + return; // Don't clear newNodes or remove raw vectors on failure } } + // Success: Clear newNodes flags for all flushed nodes + for (uint64_t key : allNodesToFlush) { + size_t segment = getSegmentIndex(key); + auto& cacheSegment = cacheSegments_[segment]; + std::unique_lock segmentLock(cacheSegment.guard); + cacheSegment.newNodes.erase(key); + } + // Remove raw vectors for level 0 nodes that were written to disk if (!successfulLevel0Ids.empty()) { std::lock_guard lock(rawVectorsGuard); From 4ccb195c32352c5785be8b91dafc13fff5191cf5 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Mon, 22 Dec 2025 15:44:05 +0200 Subject: [PATCH 25/34] guard enrty point --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 68 +++++++++++++++++++------- 1 file changed, 49 insertions(+), 19 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 01c19f338..f2ef4e6fb 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -335,7 +335,6 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Track nodes created in current batch (never written to disk yet) // This helps avoid disk lookups for new nodes std::unordered_set newNodes; - char padding[64]; // Ensure no overlap CacheSegment() = default; CacheSegment(const CacheSegment&) = delete; @@ -360,7 +359,18 @@ class HNSWDiskIndex : public VecSimIndexAbstract return key % NUM_CACHE_SEGMENTS; } - // Threshold for flushing dirty nodes to disk (0 = flush after each insert, default = batch) + // Threshold for flushing dirty nodes to disk, expressed as a count of dirty nodes. + // - Unit: number of distinct dirty nodes tracked in `totalDirtyCount_` across all cache segments. + // - Behavior: + // * 0 : flush after each insert/update (no batching). + // * >0 : accumulate dirty nodes and flush once the global dirty count reaches this threshold. + // - Tradeoffs: + // * Lower values (including 0) reduce the amount of data lost on crash and smooth memory usage, + // but increase the number of RocksDB writes and flush operations (higher write amplification + // and potentially higher latency per insert). + // * Higher values reduce the frequency of disk writes and can improve bulk-insert throughput, + // but increase peak memory usage for cached/dirty nodes and may cause longer flush pauses + // when the batch is written, as well as more data at risk between flushes. size_t diskWriteBatchThreshold = 1000; // Lock for protecting dirty nodes flush operations (global flush serialization) @@ -1000,26 +1010,37 @@ int HNSWDiskIndex::addVector( // Handle first vector (becomes entry point) if (currentEntryPoint == INVALID_ID) { + bool becameEntryPoint = false; std::unique_lock lock(indexDataGuard); if (entrypointNode == INVALID_ID) { entrypointNode = newElementId; maxLevel = elementMaxLevel; + becameEntryPoint = true; + } else { + // Another thread already set the entry point - update our local state + currentEntryPoint = entrypointNode; + currentMaxLevel = maxLevel; } - // Write initial vector to disk with empty neighbors - vecsim_stl::vector emptyNeighbors(this->allocator); - for (size_t level = 0; level <= elementMaxLevel; level++) { - GraphKey graphKey(newElementId, level); - std::string value = serializeGraphValue(vector, emptyNeighbors); - auto writeOptions = rocksdb::WriteOptions(); - writeOptions.disableWAL = true; - db->Put(writeOptions, cf, graphKey.asSlice(), value); - } - // Remove raw vector from RAM now that it's on disk - { - std::lock_guard rawLock(rawVectorsGuard); - rawVectorsInRAM.erase(newElementId); + lock.unlock(); + // If we became the entry point, write initial vector to disk + if (becameEntryPoint) { + // Write initial vector to disk with empty neighbors + vecsim_stl::vector emptyNeighbors(this->allocator); + for (size_t level = 0; level <= elementMaxLevel; level++) { + GraphKey graphKey(newElementId, level); + std::string value = serializeGraphValue(vector, emptyNeighbors); + auto writeOptions = rocksdb::WriteOptions(); + writeOptions.disableWAL = true; + db->Put(writeOptions, cf, graphKey.asSlice(), value); + } + // Remove raw vector from RAM now that it's on disk + { + std::lock_guard rawLock(rawVectorsGuard); + rawVectorsInRAM.erase(newElementId); + } + return 1; } - return 1; + // Fall through to normal graph insertion with updated entry point } // Check if we have a job queue for async processing @@ -1561,6 +1582,7 @@ bool HNSWDiskIndex::isMarkedDeleted(labelType id) const { template std::pair HNSWDiskIndex::safeGetEntryPointState() const { + std::shared_lock lock(indexDataGuard); return std::make_pair(entrypointNode, maxLevel); } @@ -3212,9 +3234,17 @@ void HNSWDiskIndex::executeGraphInsertionCore( writeDirtyNodesToDisk(modifiedNodes, rawVectorData, vectorId); std::lock_guard lock(rawVectorsGuard); rawVectorsInRAM.erase(vectorId); - } else if (totalDirtyCount_.load(std::memory_order_relaxed) >= diskWriteBatchThreshold) { - // Threshold reached: flush all dirty nodes - flushDirtyNodesToDisk(); + } else { + // Check if flush is needed based on dirty count OR raw vector memory accumulation + // This prevents unbounded memory growth when vectors are added but threshold isn't reached + bool shouldFlush = totalDirtyCount_.load(std::memory_order_relaxed) >= diskWriteBatchThreshold; + if (!shouldFlush) { + std::shared_lock lock(rawVectorsGuard); + shouldFlush = rawVectorsInRAM.size() >= diskWriteBatchThreshold; + } + if (shouldFlush) { + flushDirtyNodesToDisk(); + } } // else: batching enabled but threshold not reached, keep accumulating From f0de127961caab7002be2bdf0d9580486ee31452 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Tue, 23 Dec 2025 11:36:35 +0200 Subject: [PATCH 26/34] pr --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 87 +++++++------------------- src/VecSim/vec_sim_common.h | 2 - 2 files changed, 21 insertions(+), 68 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index f2ef4e6fb..c418f035b 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -132,21 +132,6 @@ struct GraphKey { template class HNSWDiskIndex; -/** - * Definition of a job that inserts a single vector into the HNSW disk graph. - * Each job handles one vector independently and can run in parallel with other insert jobs. - */ -struct HNSWDiskInsertJob : public AsyncJob { - idType vectorId; - size_t elementMaxLevel; - - HNSWDiskInsertJob(std::shared_ptr allocator, idType vectorId_, - size_t elementMaxLevel_, JobCallback insertCb, VecSimIndex *index_) - : AsyncJob(allocator, HNSW_DISK_INSERT_VECTOR_JOB, insertCb, index_), vectorId(vectorId_), - elementMaxLevel(elementMaxLevel_) {} -}; - - /** * Definition of the job that inserts a single vector completely from start to end. * Each job is self-contained and writes directly to disk upon completion. @@ -318,12 +303,12 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Needed because unordered_map can rehash during insert, invalidating iterators mutable std::shared_mutex rawVectorsGuard; - /********************************** Batchless Mode Support **********************************/ // Segmented neighbor cache for reduced lock contention in multi-threaded scenarios // Cache is partitioned into NUM_CACHE_SEGMENTS independent segments // Each segment has its own lock, cache map, and dirty set // This allows threads accessing different segments to proceed in parallel + // Tradeoff: Increased memory usage for NUM_CACHE_SEGMENTS * (cache map + dirty set) static constexpr size_t NUM_CACHE_SEGMENTS = 64; // Power of 2 for efficient modulo @@ -335,6 +320,7 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Track nodes created in current batch (never written to disk yet) // This helps avoid disk lookups for new nodes std::unordered_set newNodes; + char padding[64]; // Ensure no overlap CacheSegment() = default; CacheSegment(const CacheSegment&) = delete; @@ -359,18 +345,7 @@ class HNSWDiskIndex : public VecSimIndexAbstract return key % NUM_CACHE_SEGMENTS; } - // Threshold for flushing dirty nodes to disk, expressed as a count of dirty nodes. - // - Unit: number of distinct dirty nodes tracked in `totalDirtyCount_` across all cache segments. - // - Behavior: - // * 0 : flush after each insert/update (no batching). - // * >0 : accumulate dirty nodes and flush once the global dirty count reaches this threshold. - // - Tradeoffs: - // * Lower values (including 0) reduce the amount of data lost on crash and smooth memory usage, - // but increase the number of RocksDB writes and flush operations (higher write amplification - // and potentially higher latency per insert). - // * Higher values reduce the frequency of disk writes and can improve bulk-insert throughput, - // but increase peak memory usage for cached/dirty nodes and may cause longer flush pauses - // when the batch is written, as well as more data at risk between flushes. + // Threshold for flushing dirty nodes to disk (0 = flush after each insert, default = batch) size_t diskWriteBatchThreshold = 1000; // Lock for protecting dirty nodes flush operations (global flush serialization) @@ -952,7 +927,7 @@ int HNSWDiskIndex::addVector( const void *vector, labelType label ) { - // Atomically get a unique element ID - this is the critical fix for Race #1 + // Atomically get a unique element ID // fetch_add returns the OLD value before incrementing, giving us a unique ID idType newElementId = static_cast(curElementCount.fetch_add(1, std::memory_order_acq_rel)); @@ -1010,37 +985,26 @@ int HNSWDiskIndex::addVector( // Handle first vector (becomes entry point) if (currentEntryPoint == INVALID_ID) { - bool becameEntryPoint = false; std::unique_lock lock(indexDataGuard); if (entrypointNode == INVALID_ID) { entrypointNode = newElementId; maxLevel = elementMaxLevel; - becameEntryPoint = true; - } else { - // Another thread already set the entry point - update our local state - currentEntryPoint = entrypointNode; - currentMaxLevel = maxLevel; } - lock.unlock(); - // If we became the entry point, write initial vector to disk - if (becameEntryPoint) { - // Write initial vector to disk with empty neighbors - vecsim_stl::vector emptyNeighbors(this->allocator); - for (size_t level = 0; level <= elementMaxLevel; level++) { - GraphKey graphKey(newElementId, level); - std::string value = serializeGraphValue(vector, emptyNeighbors); - auto writeOptions = rocksdb::WriteOptions(); - writeOptions.disableWAL = true; - db->Put(writeOptions, cf, graphKey.asSlice(), value); - } - // Remove raw vector from RAM now that it's on disk - { - std::lock_guard rawLock(rawVectorsGuard); - rawVectorsInRAM.erase(newElementId); - } - return 1; + // Write initial vector to disk with empty neighbors + vecsim_stl::vector emptyNeighbors(this->allocator); + for (size_t level = 0; level <= elementMaxLevel; level++) { + GraphKey graphKey(newElementId, level); + std::string value = serializeGraphValue(vector, emptyNeighbors); + auto writeOptions = rocksdb::WriteOptions(); + writeOptions.disableWAL = true; + db->Put(writeOptions, cf, graphKey.asSlice(), value); } - // Fall through to normal graph insertion with updated entry point + // Remove raw vector from RAM now that it's on disk + { + std::lock_guard rawLock(rawVectorsGuard); + rawVectorsInRAM.erase(newElementId); + } + return 1; } // Check if we have a job queue for async processing @@ -1582,7 +1546,6 @@ bool HNSWDiskIndex::isMarkedDeleted(labelType id) const { template std::pair HNSWDiskIndex::safeGetEntryPointState() const { - std::shared_lock lock(indexDataGuard); return std::make_pair(entrypointNode, maxLevel); } @@ -3234,17 +3197,9 @@ void HNSWDiskIndex::executeGraphInsertionCore( writeDirtyNodesToDisk(modifiedNodes, rawVectorData, vectorId); std::lock_guard lock(rawVectorsGuard); rawVectorsInRAM.erase(vectorId); - } else { - // Check if flush is needed based on dirty count OR raw vector memory accumulation - // This prevents unbounded memory growth when vectors are added but threshold isn't reached - bool shouldFlush = totalDirtyCount_.load(std::memory_order_relaxed) >= diskWriteBatchThreshold; - if (!shouldFlush) { - std::shared_lock lock(rawVectorsGuard); - shouldFlush = rawVectorsInRAM.size() >= diskWriteBatchThreshold; - } - if (shouldFlush) { - flushDirtyNodesToDisk(); - } + } else if (totalDirtyCount_.load(std::memory_order_relaxed) >= diskWriteBatchThreshold) { + // Threshold reached: flush all dirty nodes + flushDirtyNodesToDisk(); } // else: batching enabled but threshold not reached, keep accumulating diff --git a/src/VecSim/vec_sim_common.h b/src/VecSim/vec_sim_common.h index 82359f62f..4ab155be5 100644 --- a/src/VecSim/vec_sim_common.h +++ b/src/VecSim/vec_sim_common.h @@ -256,8 +256,6 @@ typedef enum { HNSW_SEARCH_JOB, HNSW_SWAP_JOB, SVS_BATCH_UPDATE_JOB, - HNSW_DISK_INSERT_VECTOR_JOB, - HNSW_DISK_FLUSH_UPDATES_JOB, HNSW_DISK_SINGLE_INSERT_JOB, // Batchless: single vector insert from start to disk write INVALID_JOB // to indicate that finding a JobType >= INVALID_JOB is an error } JobType; From b842d1ef2b5c0bd207e97ce4824ce1e0dc027987 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Tue, 23 Dec 2025 11:37:24 +0200 Subject: [PATCH 27/34] char padding[64]; remove --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index c418f035b..7ac8e0958 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -320,7 +320,6 @@ class HNSWDiskIndex : public VecSimIndexAbstract // Track nodes created in current batch (never written to disk yet) // This helps avoid disk lookups for new nodes std::unordered_set newNodes; - char padding[64]; // Ensure no overlap CacheSegment() = default; CacheSegment(const CacheSegment&) = delete; From 17c2f99f4c49cd0d1540027f09ba5eb4cb9af4b8 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Tue, 23 Dec 2025 15:10:40 +0200 Subject: [PATCH 28/34] Mt explainations --- docs/DISK_HNSW_MT_ARCHITECTURE.md | 240 ++++++++++++++++++++++++++++++ 1 file changed, 240 insertions(+) create mode 100644 docs/DISK_HNSW_MT_ARCHITECTURE.md diff --git a/docs/DISK_HNSW_MT_ARCHITECTURE.md b/docs/DISK_HNSW_MT_ARCHITECTURE.md new file mode 100644 index 000000000..62aca3f7c --- /dev/null +++ b/docs/DISK_HNSW_MT_ARCHITECTURE.md @@ -0,0 +1,240 @@ +# HNSWDisk Multi-Threading Architecture + +## Overview + +This document describes the architectural changes introduced in the `dorer-disk-poc-add-delete-mt` branch compared to the original `disk-poc` branch. The focus is on multi-threading, synchronization, concurrency in writing to disk, and performance enhancements. + +## Key Architectural Changes + +### 1. Insertion Mode + +**Previous single threaded approach:** Vectors were accumulated in batches before being written to disk, requiring complex coordination between threads. + + +**Current approach:** Each insert job is self-contained and can write directly to disk upon completion, optimized for workloads where disk writes are cheap but neighbor searching (reads from disk) is the bottleneck. + +``` +┌──────────────────────────────────────────────────────────────┐ +│ HNSWDiskSingleInsertJob │ +├──────────────────────────────────────────────────────────────┤ +│ - vectorId │ +│ - elementMaxLevel │ +│ - rawVectorData (copied into job - no external references) │ +│ - processedVectorData (quantized vector for distance calc) │ +└──────────────────────────────────────────────────────────────┘ +``` + +### 2. Segmented Neighbor Cache + +To reduce lock contention in multi-threaded scenarios, the neighbor changes cache is partitioned into **64 independent segments**: + +```cpp +static constexpr size_t NUM_CACHE_SEGMENTS = 64; // Power of 2 for efficient modulo + +struct alignas(64) CacheSegment { + std::shared_mutex guard; // Per-segment lock + std::unordered_map> cache; // Neighbor lists + std::unordered_set dirty; // Nodes needing disk write + std::unordered_set newNodes; // Nodes never written to disk +}; +``` +Note: +NUM_CACHE_SEGMENTS can be changes which will be cause better separation of the cache, +but will require more RAM usage. + +**Key benefits:** +- Threads accessing different segments proceed in parallel. +- Cache-line alignment (`alignas(64)`) prevents false sharing. +- Hash-based segment distribution. + +### 3. Lock Hierarchy + +| Lock | Type | Protects | Notes | +|------|------|----------|------| +| `indexDataGuard` | `shared_mutex` | `entrypointNode`, `maxLevel`, `idToMetaData`, `labelToIdMap` | Metadata access during graph construction | +| `vectorsGuard` | `shared_mutex` | Vectors container (prevents resize during access) | Vectors access during graph construction | +| `rawVectorsGuard` | `shared_mutex` | `rawVectorsInRAM` map | Raw vectors access during graph construction | +| `stagedUpdatesGuard` | `shared_mutex` | Staged graph updates for deletions | +| `diskWriteGuard` | `mutex` | Serializes global flush operations | +| `cacheSegments_[i].guard` | `shared_mutex` | Per-segment cache, dirty set, newNodes | + +### 4. Atomic Variables + +```cpp +std::atomic curElementCount; // Thread-safe element counting +std::atomic totalDirtyCount_{0}; // Fast threshold check without locking +std::atomic pendingSingleInsertJobs_{0}; // Track pending async jobs +``` +Note: +We can think of more atomics that can be added to further improve performance. + +## Concurrency Patterns + +### Swap-and-Flush Pattern (Critical for Preventing Lost Updates) + +The `writeDirtyNodesToDisk` and `flushDirtyNodesToDisk` methods implement a two-phase approach: + +``` +Phase 1: Read cache data under SHARED lock (concurrent writers allowed) + - Read neighbor lists + - Build RocksDB WriteBatch + +Phase 2: Clear dirty flags under EXCLUSIVE lock (brief, per-segment) + - Atomically swap dirty set contents + - Release lock immediately + +Phase 3: Write to disk (NO locks held) + - Any new inserts after Phase 2 will re-add to dirty set + - On failure: re-add nodes to dirty set for retry +``` + +This prevents the "Lost Update" race condition where a concurrent insert's changes could be lost. + +### Atomic Check-and-Add for Neighbor Lists + +```cpp +bool tryAddNeighborToCacheIfCapacity(nodeId, level, newNeighborId, maxCapacity) { + // Under exclusive lock: + // 1. Check if already present (avoid duplicates) + // 2. Check capacity + // 3. If space available: add neighbor, mark dirty + // 4. If full: return false (caller must use heuristic) +} +``` + +This atomic operation prevents race conditions when multiple threads add neighbors to the same node. + +### Read-Copy-Update Pattern for Cache Access + +```cpp +// Read path (getNeighborsFromCache): +1. Try shared lock → read from cache +2. If miss: read from disk (no lock held during I/O) - rocksdb is thread-safe +3. Acquire exclusive lock → insert to cache (double-check) + +// Write path (addNeighborToCache): +1. Acquire exclusive lock +2. Load from disk if needed (release/re-acquire around I/O) - rocksdb is thread-safe +3. Add neighbor, mark dirty +``` + +## Disk Write Batching + +Controlled by `diskWriteBatchThreshold`: + +| Value | Behavior | +|-------|----------| +| `0` | Flush after every insert (no batching) | +| `>0` | Accumulate until `totalDirtyCount_ >= threshold`, then flush | + +```cpp +if (diskWriteBatchThreshold == 0) { + writeDirtyNodesToDisk(modifiedNodes, rawVectorData, vectorId); +} else if (totalDirtyCount_ >= diskWriteBatchThreshold) { + flushDirtyNodesToDisk(); // Flush ALL dirty nodes +} +``` + +## Job Queue Integration + +### Async Processing Flow + +``` +addVector() + │ + ├── Single-threaded path (no job queue): + │ └── executeGraphInsertionCore() [inline] + │ + └── Multi-threaded path (with job queue): + ├── Create HNSWDiskSingleInsertJob (copies vector data) + ├── Submit via SubmitJobsToQueue callback + └── Worker thread executes: + └── executeSingleInsertJob() + └── executeGraphInsertionCore() +``` + +### Job Structure + +```cpp +struct HNSWDiskSingleInsertJob : public AsyncJob { + idType vectorId; + size_t elementMaxLevel; + std::string rawVectorData; // Copied - no external references + std::string processedVectorData; // Quantized data for distance calc +}; +``` + +Copying vector data into the job eliminates race conditions with the caller's buffer. + +## Data Flow During Insert + +``` +1. addVector() + ├── Atomically allocate vectorId (curElementCount.fetch_add) + ├── Store raw vector in rawVectorsInRAM (for other jobs to access) + ├── Preprocess vector (quantization) + ├── Store processed vector in vectors container + └── Store metadata (label, topLevel) + +2. executeGraphInsertionCore() + ├── insertElementToGraph() + │ ├── greedySearchLevel() [levels > element_max_level] + │ └── For each level: + │ ├── searchLayer() → find neighbors + │ └── mutuallyConnectNewElement() + │ ├── setNeighborsInCache() [new node's neighbors] + │ └── For each neighbor: + │ ├── tryAddNeighborToCacheIfCapacity() + │ └── or revisitNeighborConnections() [if full] + └── Handle disk write (based on threshold) + +3. Disk Write (when triggered) + ├── writeDirtyNodesToDisk() [per-insert path] + └── or flushDirtyNodesToDisk() [batch flush path] +``` + +## Performance Optimizations + +### 1. Lock-Free Hot Paths + +```cpp +// Fast deleted check without acquiring indexDataGuard +template +bool isMarkedAsUnsafe(idType internalId) const { + return __atomic_load_n(&idToMetaData[internalId].flags, 0) & FLAG; +} +``` + +Used in `processCandidate()` to avoid lock contention during search. + +### 2. Atomic Counters for Fast Threshold Checks + +```cpp +// No locking needed to check if flush is required +if (totalDirtyCount_.load(std::memory_order_relaxed) >= diskWriteBatchThreshold) { + flushDirtyNodesToDisk(); +} +``` + +### 3. newNodes Tracking + +Nodes created in the current batch are tracked in `cacheSegment.newNodes`: +- Avoids disk lookups for vectors that haven't been written yet +- Cleared after successful flush to disk + +### 4. Raw Vectors in RAM + +Raw vectors are kept in `rawVectorsInRAM` until flushed to disk: +- Allows concurrent jobs to access vectors before disk write +- Eliminates redundant disk reads during graph construction +- Protected by `rawVectorsGuard` (shared_mutex) + +## Thread Safety Summary + +| Operation | Thread Safety | Notes | +|-----------|---------------|-------| +| `addVector()` | ✅ Safe | Atomic ID allocation, locked metadata access | +| `topKQuery()` | ✅ Safe | Read-only with lock-free deleted checks | +| Cache read | ✅ Safe | Shared lock per segment | +| Cache write | ✅ Safe | Exclusive lock per segment | +| Disk flush | ✅ Safe | Swap-and-flush pattern | From 3996df50dec600ce58664d33155d2a2c6ce62f06 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Tue, 23 Dec 2025 15:11:05 +0200 Subject: [PATCH 29/34] fix the test to use setDiskWriteBatchThreshold --- tests/benchmark/bm_vecsim_basics.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/benchmark/bm_vecsim_basics.h b/tests/benchmark/bm_vecsim_basics.h index 162d1ff1d..455b97a0f 100644 --- a/tests/benchmark/bm_vecsim_basics.h +++ b/tests/benchmark/bm_vecsim_basics.h @@ -332,7 +332,7 @@ void BM_VecSimBasics::FlushBatchDisk(benchmark::State &st) { auto *hnsw_disk_index = dynamic_cast *>(hnsw_index); size_t flush_threshold = st.range(0); - hnsw_disk_index->setBatchThreshold(flush_threshold); + hnsw_disk_index->setDiskWriteBatchThreshold(flush_threshold); for (size_t i = 0; i < flush_threshold-1; i++) { // add vectors to fill the batch VecSimIndex_AddVector(hnsw_disk_index, QUERIES[i%N_QUERIES].data(), i); From a55d5435d9d2287f6d0e37ca2cc582080184333c Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Wed, 24 Dec 2025 10:03:59 +0200 Subject: [PATCH 30/34] print every 5 seconds --- tests/benchmark/data/scripts/hnsw_disk_serializer.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp b/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp index 81b0ff3eb..c653f0dba 100644 --- a/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp +++ b/tests/benchmark/data/scripts/hnsw_disk_serializer.cpp @@ -445,21 +445,21 @@ int main(int argc, char *argv[]) { break; } - // Print progress every 1 seconds + // Print progress every 5 seconds size_t current_indexed = VecSimIndex_IndexSize(index); size_t queue_size = mock_thread_pool->jobQ.size(); auto elapsed = std::chrono::duration_cast( std::chrono::steady_clock::now() - start_time).count(); - if (current_indexed != last_indexed || elapsed % 1 == 0) { + if (current_indexed != last_indexed || elapsed % 5 == 0) { std::cout << "\rIndexed: " << current_indexed << "/" << num_vectors << " | Queue: " << queue_size << " | Time: " << elapsed << "s " << std::flush; last_indexed = current_indexed; } - std::this_thread::sleep_for(std::chrono::milliseconds(100)); + std::this_thread::sleep_for(std::chrono::milliseconds(1000)); } std::cout << "\n"; From fae054305e9b0351d50736c06dfb440b6a583c65 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Wed, 24 Dec 2025 13:51:54 +0200 Subject: [PATCH 31/34] name change --- ...MT_ARCHITECTURE.md => disk_hnsw_multithreaded_architecture.md} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/{DISK_HNSW_MT_ARCHITECTURE.md => disk_hnsw_multithreaded_architecture.md} (100%) diff --git a/docs/DISK_HNSW_MT_ARCHITECTURE.md b/docs/disk_hnsw_multithreaded_architecture.md similarity index 100% rename from docs/DISK_HNSW_MT_ARCHITECTURE.md rename to docs/disk_hnsw_multithreaded_architecture.md From 24cef8ac05deca8bdf250cfa35ecb7dce29ddb4a Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Wed, 24 Dec 2025 15:34:59 +0200 Subject: [PATCH 32/34] Don't copy data to job --- docs/disk_hnsw_multithreaded_architecture.md | 57 +++++++++++++------ src/VecSim/algorithms/hnsw/hnsw_disk.h | 60 +++++++++++++------- 2 files changed, 79 insertions(+), 38 deletions(-) diff --git a/docs/disk_hnsw_multithreaded_architecture.md b/docs/disk_hnsw_multithreaded_architecture.md index 62aca3f7c..c5cc6e320 100644 --- a/docs/disk_hnsw_multithreaded_architecture.md +++ b/docs/disk_hnsw_multithreaded_architecture.md @@ -2,16 +2,13 @@ ## Overview -This document describes the architectural changes introduced in the `dorer-disk-poc-add-delete-mt` branch compared to the original `disk-poc` branch. The focus is on multi-threading, synchronization, concurrency in writing to disk, and performance enhancements. +This document describes the multi-threaded architecture of the HNSWDisk index, focusing on synchronization, concurrency in writing to disk, and performance enhancements. -## Key Architectural Changes +## Key Architectural Components -### 1. Insertion Mode +### 1. Lightweight Insert Jobs -**Previous single threaded approach:** Vectors were accumulated in batches before being written to disk, requiring complex coordination between threads. - - -**Current approach:** Each insert job is self-contained and can write directly to disk upon completion, optimized for workloads where disk writes are cheap but neighbor searching (reads from disk) is the bottleneck. +Each insert job is lightweight and only stores metadata (vectorId, elementMaxLevel). Vector data is looked up from shared storage when the job executes, minimizing memory usage when many jobs are queued. ``` ┌──────────────────────────────────────────────────────────────┐ @@ -19,11 +16,13 @@ This document describes the architectural changes introduced in the `dorer-disk- ├──────────────────────────────────────────────────────────────┤ │ - vectorId │ │ - elementMaxLevel │ -│ - rawVectorData (copied into job - no external references) │ -│ - processedVectorData (quantized vector for distance calc) │ └──────────────────────────────────────────────────────────────┘ ``` +At execution time, jobs access vector data via: +- **Raw vectors**: `shared_ptr` from `rawVectorsInRAM` (refcount increment, no copy) +- **Processed vectors**: Direct access from `this->vectors` container + ### 2. Segmented Neighbor Cache To reduce lock contention in multi-threaded scenarios, the neighbor changes cache is partitioned into **64 independent segments**: @@ -39,7 +38,7 @@ struct alignas(64) CacheSegment { }; ``` Note: -NUM_CACHE_SEGMENTS can be changes which will be cause better separation of the cache, +NUM_CACHE_SEGMENTS can be changed which will cause better separation of the cache, but will require more RAM usage. **Key benefits:** @@ -146,10 +145,12 @@ addVector() │ └── executeGraphInsertionCore() [inline] │ └── Multi-threaded path (with job queue): - ├── Create HNSWDiskSingleInsertJob (copies vector data) + ├── Create HNSWDiskSingleInsertJob (just vectorId + level, no vector copy) ├── Submit via SubmitJobsToQueue callback └── Worker thread executes: └── executeSingleInsertJob() + ├── Get shared_ptr to raw vector from rawVectorsInRAM + ├── Get processed vector from this->vectors └── executeGraphInsertionCore() ``` @@ -159,12 +160,16 @@ addVector() struct HNSWDiskSingleInsertJob : public AsyncJob { idType vectorId; size_t elementMaxLevel; - std::string rawVectorData; // Copied - no external references - std::string processedVectorData; // Quantized data for distance calc + // No vector data stored - looked up from index when job executes + // This saves memory: 100M pending jobs don't need 100M vector copies }; ``` -Copying vector data into the job eliminates race conditions with the caller's buffer. +Jobs look up vector data at execution time: +- **Raw vectors**: Accessed via `shared_ptr` from `rawVectorsInRAM` (just increments refcount, no copy) +- **Processed vectors**: Accessed from `this->vectors` container + +This eliminates memory duplication while maintaining thread safety through reference counting. ## Data Flow During Insert @@ -222,13 +227,33 @@ Nodes created in the current batch are tracked in `cacheSegment.newNodes`: - Avoids disk lookups for vectors that haven't been written yet - Cleared after successful flush to disk -### 4. Raw Vectors in RAM +### 4. Raw Vectors in RAM with shared_ptr + +Raw vectors are stored in `rawVectorsInRAM` using `std::shared_ptr`: + +```cpp +std::unordered_map> rawVectorsInRAM; +``` -Raw vectors are kept in `rawVectorsInRAM` until flushed to disk: +**Benefits:** - Allows concurrent jobs to access vectors before disk write - Eliminates redundant disk reads during graph construction +- **Zero-copy job execution**: Jobs increment refcount instead of copying entire vector +- **Safe concurrent deletion**: If vector is erased from map while job is executing, the `shared_ptr` keeps data alive until job completes - Protected by `rawVectorsGuard` (shared_mutex) +**Execution flow:** +```cpp +// Job execution - no data copy, just refcount increment +std::shared_ptr localRawRef; +{ + std::shared_lock lock(rawVectorsGuard); + localRawRef = rawVectorsInRAM[job->vectorId]; // refcount++ +} +// Lock released, but data stays alive via localRawRef +// Use localRawRef->data() for graph insertion and disk write +``` + ## Thread Safety Summary | Operation | Thread Safety | Notes | diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 7ac8e0958..101f3589e 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -13,6 +13,7 @@ #include "VecSim/memory/vecsim_malloc.h" #include "VecSim/utils/vecsim_stl.h" #include "VecSim/utils/vec_utils.h" +#include #include // #include "VecSim/containers/data_block.h" // #include "VecSim/containers/raw_data_container_interface.h" @@ -143,17 +144,14 @@ class HNSWDiskIndex; struct HNSWDiskSingleInsertJob : public AsyncJob { idType vectorId; size_t elementMaxLevel; - // Store vector data directly in the job (no external references) - std::string rawVectorData; // Original float32 vector - std::string processedVectorData; // Preprocessed/quantized vector for distance calculations + // No vector data stored - looked up from index when job executes + // This saves memory: 100M pending jobs don't need 100M vector copies HNSWDiskSingleInsertJob(std::shared_ptr allocator, idType vectorId_, - size_t elementMaxLevel_, std::string &&rawVector, - std::string &&processedVector, JobCallback insertCb, + size_t elementMaxLevel_, JobCallback insertCb, VecSimIndex *index_) : AsyncJob(allocator, HNSW_DISK_SINGLE_INSERT_JOB, insertCb, index_), - vectorId(vectorId_), elementMaxLevel(elementMaxLevel_), - rawVectorData(std::move(rawVector)), processedVectorData(std::move(processedVector)) {} + vectorId(vectorId_), elementMaxLevel(elementMaxLevel_) {} }; //////////////////////////////////// HNSW index implementation //////////////////////////////////// @@ -277,8 +275,9 @@ class HNSWDiskIndex : public VecSimIndexAbstract vecsim_stl::vector stagedInsertNeighborUpdates; // Temporary storage for raw vectors in RAM (until flush batch) - // Maps idType -> raw vector data (stored as string for simplicity) - std::unordered_map rawVectorsInRAM; + // Maps idType -> raw vector data (using shared_ptr to avoid copying in job execution) + // When a job executes, it just increments refcount instead of copying the entire vector + std::unordered_map> rawVectorsInRAM; /********************************** Multi-threading Support **********************************/ @@ -934,10 +933,12 @@ int HNSWDiskIndex::addVector( // We need to store the original vector before preprocessing // NOTE: In batchless mode, we still use rawVectorsInRAM so other concurrent jobs can access // the raw vectors of vectors that haven't been written to disk yet + // Using shared_ptr so job execution can just increment refcount instead of copying const char* raw_data = reinterpret_cast(vector); + auto rawVectorPtr = std::make_shared(raw_data, this->inputBlobSize); { std::lock_guard lock(rawVectorsGuard); - rawVectorsInRAM[newElementId] = std::string(raw_data, this->inputBlobSize); + rawVectorsInRAM[newElementId] = rawVectorPtr; } // Preprocess the vector ProcessedBlobs processedBlobs = this->preprocess(vector); @@ -1009,14 +1010,9 @@ int HNSWDiskIndex::addVector( // Check if we have a job queue for async processing if (SubmitJobsToQueue != nullptr) { // Multi-threaded: submit job for async processing - std::string rawVectorCopy(raw_data, this->inputBlobSize); - std::string processedVectorCopy( - reinterpret_cast(processedBlobs.getStorageBlob()), - this->dataSize); - + // No vector copies in job - job will look up from rawVectorsInRAM and this->vectors auto *job = new (this->allocator) HNSWDiskSingleInsertJob( - this->allocator, newElementId, elementMaxLevel, std::move(rawVectorCopy), - std::move(processedVectorCopy), + this->allocator, newElementId, elementMaxLevel, HNSWDiskIndex::executeSingleInsertJobWrapper, this); submitSingleJob(job); @@ -1304,7 +1300,7 @@ bool HNSWDiskIndex::getRawVector(idType id, void* output_buf std::shared_lock lock(rawVectorsGuard); auto it = rawVectorsInRAM.find(id); if (it != rawVectorsInRAM.end()) { - const char* data_ptr = it->second.data(); + const char* data_ptr = it->second->data(); std::memcpy(output_buffer, data_ptr, this->inputBlobSize); return true; } @@ -1353,7 +1349,7 @@ bool HNSWDiskIndex::getRawVectorInternal(idType id, void* ou std::shared_lock lock(rawVectorsGuard); auto it = rawVectorsInRAM.find(id); if (it != rawVectorsInRAM.end()) { - const char* data_ptr = it->second.data(); + const char* data_ptr = it->second->data(); std::memcpy(output_buffer, data_ptr, this->inputBlobSize); return true; } @@ -3146,7 +3142,28 @@ void HNSWDiskIndex::executeSingleInsertJob(HNSWDiskSingleIns return; } - // Get current entry point + // Get shared_ptr to raw vector from rawVectorsInRAM (just increments refcount, no copy) + // This keeps the data alive even if erased from map before job finishes + std::shared_ptr localRawRef; + { + std::shared_lock lock(rawVectorsGuard); + auto it = rawVectorsInRAM.find(job->vectorId); + if (it == rawVectorsInRAM.end()) { + // Vector was already erased (e.g., deleted before job executed) + delete job; + return; + } + localRawRef = it->second; // Just increments refcount, no data copy + } + + // Get processed vector from vectors container + const void *processedVector; + { + std::shared_lock lock(vectorsGuard); + processedVector = this->vectors->getElement(job->vectorId); + } + + // Get current entry point and max level idType currentEntryPoint; size_t currentMaxLevel; { @@ -3157,8 +3174,7 @@ void HNSWDiskIndex::executeSingleInsertJob(HNSWDiskSingleIns // Use unified core function (batching controlled by diskWriteBatchThreshold) executeGraphInsertionCore(job->vectorId, job->elementMaxLevel, currentEntryPoint, - currentMaxLevel, job->rawVectorData.data(), - job->processedVectorData.data()); + currentMaxLevel, localRawRef->data(), processedVector); delete job; } From b6ed93a4d775b6817cf4b451c98e2875fe573fc3 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Wed, 24 Dec 2025 16:52:52 +0200 Subject: [PATCH 33/34] add lock --- src/VecSim/algorithms/hnsw/hnsw_disk.h | 29 ++++++++++++-------------- 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/src/VecSim/algorithms/hnsw/hnsw_disk.h b/src/VecSim/algorithms/hnsw/hnsw_disk.h index 101f3589e..df95d3020 100644 --- a/src/VecSim/algorithms/hnsw/hnsw_disk.h +++ b/src/VecSim/algorithms/hnsw/hnsw_disk.h @@ -1541,6 +1541,7 @@ bool HNSWDiskIndex::isMarkedDeleted(labelType id) const { template std::pair HNSWDiskIndex::safeGetEntryPointState() const { + std::shared_lock lock(indexDataGuard); return std::make_pair(entrypointNode, maxLevel); } @@ -3144,33 +3145,29 @@ void HNSWDiskIndex::executeSingleInsertJob(HNSWDiskSingleIns // Get shared_ptr to raw vector from rawVectorsInRAM (just increments refcount, no copy) // This keeps the data alive even if erased from map before job finishes - std::shared_ptr localRawRef; - { + std::shared_ptr localRawRef = [&]() -> std::shared_ptr { std::shared_lock lock(rawVectorsGuard); auto it = rawVectorsInRAM.find(job->vectorId); if (it == rawVectorsInRAM.end()) { // Vector was already erased (e.g., deleted before job executed) - delete job; - return; + return nullptr; } - localRawRef = it->second; // Just increments refcount, no data copy + return it->second; + }(); + + if (!localRawRef) { + delete job; + return; } // Get processed vector from vectors container - const void *processedVector; - { + const void *processedVector = [&]() -> const void * { std::shared_lock lock(vectorsGuard); - processedVector = this->vectors->getElement(job->vectorId); - } + return this->vectors->getElement(job->vectorId); + }(); // Get current entry point and max level - idType currentEntryPoint; - size_t currentMaxLevel; - { - std::shared_lock lock(indexDataGuard); - currentEntryPoint = entrypointNode; - currentMaxLevel = maxLevel; - } + auto [currentEntryPoint, currentMaxLevel] = safeGetEntryPointState(); // Use unified core function (batching controlled by diskWriteBatchThreshold) executeGraphInsertionCore(job->vectorId, job->elementMaxLevel, currentEntryPoint, From 14ef9f6b513f1521f4d3c798301ea038b7411b08 Mon Sep 17 00:00:00 2001 From: Dor Forer Date: Wed, 24 Dec 2025 17:59:07 +0200 Subject: [PATCH 34/34] Fixed the readme --- docs/disk_hnsw_multithreaded_architecture.md | 90 +++++++++++++++++--- 1 file changed, 76 insertions(+), 14 deletions(-) diff --git a/docs/disk_hnsw_multithreaded_architecture.md b/docs/disk_hnsw_multithreaded_architecture.md index c5cc6e320..4716b90c7 100644 --- a/docs/disk_hnsw_multithreaded_architecture.md +++ b/docs/disk_hnsw_multithreaded_architecture.md @@ -8,7 +8,8 @@ This document describes the multi-threaded architecture of the HNSWDisk index, f ### 1. Lightweight Insert Jobs -Each insert job is lightweight and only stores metadata (vectorId, elementMaxLevel). Vector data is looked up from shared storage when the job executes, minimizing memory usage when many jobs are queued. +Each insert job (to the threadpool) is lightweight and only stores metadata (vectorId, elementMaxLevel). Vector data is looked up from shared storage when the job executes, minimizing memory usage when many jobs are queued. + ``` ┌──────────────────────────────────────────────────────────────┐ @@ -39,13 +40,84 @@ struct alignas(64) CacheSegment { ``` Note: NUM_CACHE_SEGMENTS can be changed which will cause better separation of the cache, -but will require more RAM usage. +but will require more RAM usage - can be configured by the user or by the expected +number of vectors in the index. **Key benefits:** - Threads accessing different segments proceed in parallel. - Cache-line alignment (`alignas(64)`) prevents false sharing. - Hash-based segment distribution. +#### Cache Memory Management + +**Current Behavior: No Eviction** + +The segment cache (`cacheSegment.cache`) currently **grows unboundedly** and is **never evicted**. Once a node's neighbor list is loaded into cache (either from disk or created during insert), it remains in memory indefinitely. + +**Why Cache is Source of Truth** + +The cache cannot simply be cleared because it serves as the **source of truth** for pending updates that haven't been flushed to disk yet. The Swap-and-Flush pattern relies on: +1. Cache always having the latest neighbor lists +2. `dirty` set tracking which nodes need to be written +3. Flusher reading current cache state (not stale data) + +**Need to decide which strategy to implement (if any).** +**Another option is to not use the neighbors cache at all and always read from disk** + +**1. LRU Eviction for Clean Entries** + +Evict least-recently-used entries that are **not dirty** (already persisted to disk): +```cpp +// Pseudocode +if (cacheSize > maxCacheSize) { + for (auto& entry : lruOrder) { + if (!dirty.contains(entry.key)) { + cache.erase(entry.key); + if (--evicted >= targetEviction) break; + } + } +} +``` +*Pros:* Simple, safe (dirty entries always kept) +*Cons:* Requires LRU tracking overhead (linked list + map) + +**2. Time-Based Eviction** + +Evict clean entries older than a threshold: +```cpp +// Pseudocode +for (auto& entry : cache) { + if (!dirty.contains(entry.key) && + now - entry.lastAccessTime > evictionTimeout) { + cache.erase(entry.key); + } +} +``` +*Pros:* Predictable memory behavior +*Cons:* Requires timestamp tracking per entry + +**3. Write-Through with Immediate Eviction** + +After flushing to disk, immediately evict the written entries: +```cpp +// In flushDirtyNodesToDisk(), after successful write: +for (uint64_t key : flushedNodes) { + cacheSegment.cache.erase(key); // Evict after persist +} +``` +*Pros:* Minimal memory usage, no tracking overhead +*Cons:* Increases disk reads on subsequent access + +**4. Size-Limited Cache with Eviction Policy** + +Configure maximum cache size and evict when exceeded: +```cpp +size_t maxCacheEntries = 100000; // Configurable +// On insert, check size and evict clean entries if needed +``` +*Pros:* Bounded memory usage +*Cons:* Need to choose appropriate eviction policy + ### 3. Lock Hierarchy | Lock | Type | Protects | Notes | @@ -65,7 +137,7 @@ std::atomic totalDirtyCount_{0}; // Fast threshold check without lockin std::atomic pendingSingleInsertJobs_{0}; // Track pending async jobs ``` Note: -We can think of more atomics that can be added to further improve performance. +We can probably think of more atomics variables that can be added to further improve performance, I just used for the important ones. ## Concurrency Patterns @@ -79,7 +151,7 @@ Phase 1: Read cache data under SHARED lock (concurrent writers allowed) - Build RocksDB WriteBatch Phase 2: Clear dirty flags under EXCLUSIVE lock (brief, per-segment) - - Atomically swap dirty set contents + - "Atomically" (under a lock) swap dirty set contents - Release lock immediately Phase 3: Write to disk (NO locks held) @@ -253,13 +325,3 @@ std::shared_ptr localRawRef; // Lock released, but data stays alive via localRawRef // Use localRawRef->data() for graph insertion and disk write ``` - -## Thread Safety Summary - -| Operation | Thread Safety | Notes | -|-----------|---------------|-------| -| `addVector()` | ✅ Safe | Atomic ID allocation, locked metadata access | -| `topKQuery()` | ✅ Safe | Read-only with lock-free deleted checks | -| Cache read | ✅ Safe | Shared lock per segment | -| Cache write | ✅ Safe | Exclusive lock per segment | -| Disk flush | ✅ Safe | Swap-and-flush pattern |