Skip to main content

hermes_core/index/
writer.rs

1//! IndexWriter — async document indexing with parallel segment building.
2//!
3//! This module is only compiled with the "native" feature.
4//!
5//! # Architecture
6//!
7//! ```text
8//! add_document() ──try_send──► [shared bounded MPMC] ◄──recv── worker 0
9//!                                                     ◄──recv── worker 1
10//!                                                     ◄──recv── worker N
11//! ```
12//!
13//! - **Shared MPMC queue** (`async_channel`): all workers compete for documents.
14//!   Busy workers (building segments) naturally stop pulling; free workers pick up slack.
15//! - **Zero-copy pipeline**: `Document` is moved (never cloned) through every stage:
16//!   `add_document()` → channel → `recv_blocking()` → `SegmentBuilder::add_document()`.
17//! - `add_document` returns `QueueFull` when the queue is at capacity.
18//! - **Workers are OS threads**: CPU-intensive work (tokenization, posting list building)
19//!   runs on dedicated threads, never blocking the tokio async runtime.
20//!   Async I/O (segment file writes) is bridged via `Handle::block_on()`.
21//! - **Fixed per-worker memory budget**: `max_indexing_memory_bytes / num_workers`.
22//! - **Two-phase commit**:
23//!   1. `prepare_commit()` — closes queue, workers flush builders to disk.
24//!      Returns a `PreparedCommit` guard. No new documents accepted until resolved.
25//!   2. `PreparedCommit::commit()` — registers segments in metadata, resumes workers.
26//!   3. `PreparedCommit::abort()` — discards prepared segments, resumes workers.
27//!   4. `commit()` — convenience: `prepare_commit().await?.commit().await`.
28//!
29//! Since `prepare_commit`/`commit` take `&mut self`, Rust’s borrow checker
30//! guarantees no concurrent `add_document` calls during the commit window.
31
32use std::sync::Arc;
33use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering};
34
35use rustc_hash::FxHashMap;
36
37use crate::directories::DirectoryWriter;
38use crate::dsl::{Document, Field, Schema};
39use crate::error::{Error, Result};
40use crate::segment::{SegmentBuilder, SegmentBuilderConfig, SegmentId};
41use crate::tokenizer::BoxedTokenizer;
42
43use super::IndexConfig;
44
45/// Total pipeline capacity (in documents).
46const PIPELINE_MAX_SIZE_IN_DOCS: usize = 10_000;
47
48/// Async IndexWriter for adding documents and committing segments.
49///
50/// **Backpressure:** `add_document()` is sync, O(1). Returns `Error::QueueFull`
51/// when the shared queue is at capacity — caller must back off.
52///
53/// **Two-phase commit:**
54/// - `prepare_commit()` → `PreparedCommit::commit()` or `PreparedCommit::abort()`
55/// - `commit()` is a convenience that does both phases.
56/// - Between prepare and commit, the caller can do external work (WAL, sync, etc.)
57///   knowing that abort is possible if something fails.
58/// - Dropping `PreparedCommit` without calling commit/abort auto-aborts.
59pub struct IndexWriter<D: DirectoryWriter + 'static> {
60    pub(super) directory: Arc<D>,
61    pub(super) schema: Arc<Schema>,
62    pub(super) config: IndexConfig,
63    /// MPMC sender — `try_send(&self)` is thread-safe, no lock needed.
64    /// Replaced on each commit cycle (workers get new receiver via resume).
65    doc_sender: async_channel::Sender<Document>,
66    /// Worker OS thread handles — long-lived, survive across commits.
67    workers: Vec<std::thread::JoinHandle<()>>,
68    /// Shared worker state (immutable config + mutable segment output + sync)
69    worker_state: Arc<WorkerState<D>>,
70    /// Segment manager — owns metadata.json, handles segments and background merging
71    pub(super) segment_manager: Arc<crate::merge::SegmentManager<D>>,
72    /// Segments flushed to disk but not yet registered in metadata
73    flushed_segments: Vec<(String, u32)>,
74    /// Primary key dedup index (None if schema has no primary field)
75    primary_key_index: Option<super::primary_key::PrimaryKeyIndex>,
76}
77
78/// Shared state for worker threads.
79struct WorkerState<D: DirectoryWriter + 'static> {
80    directory: Arc<D>,
81    schema: Arc<Schema>,
82    builder_config: SegmentBuilderConfig,
83    tokenizers: parking_lot::RwLock<FxHashMap<Field, BoxedTokenizer>>,
84    /// Fixed per-worker memory budget (bytes). When a builder exceeds this, segment is built.
85    memory_budget_per_worker: usize,
86    /// Segment manager — workers read trained structures from its ArcSwap (lock-free).
87    segment_manager: Arc<crate::merge::SegmentManager<D>>,
88    /// Segments built by workers, collected by `prepare_commit()`. Sync mutex for sub-μs push.
89    built_segments: parking_lot::Mutex<Vec<(String, u32)>>,
90
91    // === Worker lifecycle synchronization ===
92    // Workers survive across commits. On prepare_commit the channel is closed;
93    // workers flush their builders, increment flush_count, then wait on
94    // resume_cvar for a new receiver. commit/abort creates a fresh channel
95    // and wakes them.
96    /// Number of workers that have completed their flush.
97    flush_count: AtomicUsize,
98    /// Mutex + condvar for prepare_commit to wait on all workers flushed.
99    flush_mutex: parking_lot::Mutex<()>,
100    flush_cvar: parking_lot::Condvar,
101    /// Holds the new channel receiver after commit/abort. Workers clone from this.
102    resume_receiver: parking_lot::Mutex<Option<async_channel::Receiver<Document>>>,
103    /// Monotonically increasing epoch, bumped by each resume_workers call.
104    /// Workers compare against their local epoch to avoid re-cloning a stale receiver.
105    resume_epoch: AtomicUsize,
106    /// Condvar for workers to wait for resume (new channel) or shutdown.
107    resume_cvar: parking_lot::Condvar,
108    /// When true, workers should exit permanently (IndexWriter dropped).
109    shutdown: AtomicBool,
110    /// Total number of worker threads.
111    num_workers: usize,
112}
113
114impl<D: DirectoryWriter + 'static> IndexWriter<D> {
115    /// Create a new index in the directory
116    pub async fn create(directory: D, schema: Schema, config: IndexConfig) -> Result<Self> {
117        Self::create_with_config(directory, schema, config, SegmentBuilderConfig::default()).await
118    }
119
120    /// Create a new index with custom builder config
121    pub async fn create_with_config(
122        directory: D,
123        schema: Schema,
124        config: IndexConfig,
125        builder_config: SegmentBuilderConfig,
126    ) -> Result<Self> {
127        let directory = Arc::new(directory);
128        let schema = Arc::new(schema);
129        let metadata = super::IndexMetadata::new((*schema).clone());
130
131        let segment_manager = Arc::new(crate::merge::SegmentManager::new(
132            Arc::clone(&directory),
133            Arc::clone(&schema),
134            metadata,
135            config.merge_policy.clone_box(),
136            config.term_cache_blocks,
137            config.max_concurrent_merges,
138        ));
139        segment_manager.update_metadata(|_| {}).await?;
140
141        Ok(Self::new_with_parts(
142            directory,
143            schema,
144            config,
145            builder_config,
146            segment_manager,
147        ))
148    }
149
150    /// Open an existing index for writing
151    pub async fn open(directory: D, config: IndexConfig) -> Result<Self> {
152        Self::open_with_config(directory, config, SegmentBuilderConfig::default()).await
153    }
154
155    /// Open an existing index with custom builder config
156    pub async fn open_with_config(
157        directory: D,
158        config: IndexConfig,
159        builder_config: SegmentBuilderConfig,
160    ) -> Result<Self> {
161        let directory = Arc::new(directory);
162        let metadata = super::IndexMetadata::load(directory.as_ref()).await?;
163        let schema = Arc::new(metadata.schema.clone());
164
165        let segment_manager = Arc::new(crate::merge::SegmentManager::new(
166            Arc::clone(&directory),
167            Arc::clone(&schema),
168            metadata,
169            config.merge_policy.clone_box(),
170            config.term_cache_blocks,
171            config.max_concurrent_merges,
172        ));
173        segment_manager.load_and_publish_trained().await;
174
175        Ok(Self::new_with_parts(
176            directory,
177            schema,
178            config,
179            builder_config,
180            segment_manager,
181        ))
182    }
183
184    /// Create an IndexWriter from an existing Index.
185    /// Shares the SegmentManager for consistent segment lifecycle management.
186    pub fn from_index(index: &super::Index<D>) -> Self {
187        Self::new_with_parts(
188            Arc::clone(&index.directory),
189            Arc::clone(&index.schema),
190            index.config.clone(),
191            SegmentBuilderConfig::default(),
192            Arc::clone(&index.segment_manager),
193        )
194    }
195
196    // ========================================================================
197    // Construction + pipeline management
198    // ========================================================================
199
200    /// Common construction: creates worker state, spawns workers, assembles `Self`.
201    fn new_with_parts(
202        directory: Arc<D>,
203        schema: Arc<Schema>,
204        config: IndexConfig,
205        builder_config: SegmentBuilderConfig,
206        segment_manager: Arc<crate::merge::SegmentManager<D>>,
207    ) -> Self {
208        // Auto-configure tokenizers from schema for all text fields
209        let registry = crate::tokenizer::TokenizerRegistry::new();
210        let mut tokenizers = FxHashMap::default();
211        for (field, entry) in schema.fields() {
212            if matches!(entry.field_type, crate::dsl::FieldType::Text)
213                && let Some(ref tok_name) = entry.tokenizer
214                && let Some(tok) = registry.get(tok_name)
215            {
216                tokenizers.insert(field, tok);
217            }
218        }
219
220        let num_workers = config.num_indexing_threads.max(1);
221        let worker_state = Arc::new(WorkerState {
222            directory: Arc::clone(&directory),
223            schema: Arc::clone(&schema),
224            builder_config,
225            tokenizers: parking_lot::RwLock::new(tokenizers),
226            memory_budget_per_worker: config.max_indexing_memory_bytes / num_workers,
227            segment_manager: Arc::clone(&segment_manager),
228            built_segments: parking_lot::Mutex::new(Vec::new()),
229            flush_count: AtomicUsize::new(0),
230            flush_mutex: parking_lot::Mutex::new(()),
231            flush_cvar: parking_lot::Condvar::new(),
232            resume_receiver: parking_lot::Mutex::new(None),
233            resume_epoch: AtomicUsize::new(0),
234            resume_cvar: parking_lot::Condvar::new(),
235            shutdown: AtomicBool::new(false),
236            num_workers,
237        });
238        let (doc_sender, workers) = Self::spawn_workers(&worker_state, num_workers);
239
240        Self {
241            directory,
242            schema,
243            config,
244            doc_sender,
245            workers,
246            worker_state,
247            segment_manager,
248            flushed_segments: Vec::new(),
249            primary_key_index: None,
250        }
251    }
252
253    fn spawn_workers(
254        worker_state: &Arc<WorkerState<D>>,
255        num_workers: usize,
256    ) -> (
257        async_channel::Sender<Document>,
258        Vec<std::thread::JoinHandle<()>>,
259    ) {
260        let (sender, receiver) = async_channel::bounded(PIPELINE_MAX_SIZE_IN_DOCS);
261        let handle = tokio::runtime::Handle::current();
262        let mut workers = Vec::with_capacity(num_workers);
263        for i in 0..num_workers {
264            let state = Arc::clone(worker_state);
265            let rx = receiver.clone();
266            let rt = handle.clone();
267            workers.push(
268                std::thread::Builder::new()
269                    .name(format!("index-worker-{}", i))
270                    .spawn(move || Self::worker_loop(state, rx, rt))
271                    .expect("failed to spawn index worker thread"),
272            );
273        }
274        (sender, workers)
275    }
276
277    /// Get the schema
278    pub fn schema(&self) -> &Schema {
279        &self.schema
280    }
281
282    /// Set tokenizer for a field.
283    /// Propagated to worker threads — takes effect for the next SegmentBuilder they create.
284    pub fn set_tokenizer<T: crate::tokenizer::Tokenizer>(&mut self, field: Field, tokenizer: T) {
285        self.worker_state
286            .tokenizers
287            .write()
288            .insert(field, Box::new(tokenizer));
289    }
290
291    /// Initialize primary key deduplication from committed segments.
292    ///
293    /// Reads existing primary key values from fast-field text dictionaries and
294    /// builds a bloom filter for O(1) negative lookups. Must be called before
295    /// `add_document` for dedup to take effect. No-op if schema has no primary field.
296    pub async fn init_primary_key_dedup(&mut self) -> Result<()> {
297        let field = match self.schema.primary_field() {
298            Some(f) => f,
299            None => return Ok(()),
300        };
301
302        let snapshot = self.segment_manager.acquire_snapshot().await;
303        // Open all segment readers concurrently for faster PK init
304        let open_futures: Vec<_> = snapshot
305            .segment_ids()
306            .iter()
307            .map(|seg_id_str| {
308                let seg_id_str = seg_id_str.clone();
309                let dir = self.directory.as_ref();
310                let schema = Arc::clone(&self.schema);
311                let cache_blocks = self.config.term_cache_blocks;
312                async move {
313                    let seg_id =
314                        crate::segment::SegmentId::from_hex(&seg_id_str).ok_or_else(|| {
315                            Error::Internal(format!("Invalid segment id: {}", seg_id_str))
316                        })?;
317                    let reader =
318                        crate::segment::SegmentReader::open(dir, seg_id, schema, cache_blocks)
319                            .await?;
320                    Ok::<_, Error>(Arc::new(reader))
321                }
322            })
323            .collect();
324        let readers = futures::future::try_join_all(open_futures).await?;
325
326        self.primary_key_index = Some(super::primary_key::PrimaryKeyIndex::new(
327            field, readers, snapshot,
328        ));
329        Ok(())
330    }
331
332    /// Add a document to the indexing queue (sync, O(1), lock-free).
333    ///
334    /// `Document` is moved into the channel (zero-copy). Workers compete to pull it.
335    /// Returns `Error::QueueFull` when the queue is at capacity — caller must back off.
336    pub fn add_document(&self, doc: Document) -> Result<()> {
337        if let Some(ref pk_index) = self.primary_key_index {
338            pk_index.check_and_insert(&doc)?;
339        }
340        match self.doc_sender.try_send(doc) {
341            Ok(()) => Ok(()),
342            Err(async_channel::TrySendError::Full(doc)) => {
343                // Roll back PK registration so the caller can retry later
344                if let Some(ref pk_index) = self.primary_key_index {
345                    pk_index.rollback_uncommitted_key(&doc);
346                }
347                Err(Error::QueueFull)
348            }
349            Err(async_channel::TrySendError::Closed(doc)) => {
350                // Roll back PK registration for defense-in-depth
351                if let Some(ref pk_index) = self.primary_key_index {
352                    pk_index.rollback_uncommitted_key(&doc);
353                }
354                Err(Error::Internal("Document channel closed".into()))
355            }
356        }
357    }
358
359    /// Add multiple documents to the indexing queue.
360    ///
361    /// Returns the number of documents successfully queued. Stops at the first
362    /// `QueueFull` and returns the count queued so far.
363    pub fn add_documents(&self, documents: Vec<Document>) -> Result<usize> {
364        let total = documents.len();
365        for (i, doc) in documents.into_iter().enumerate() {
366            match self.add_document(doc) {
367                Ok(()) => {}
368                Err(Error::QueueFull) => return Ok(i),
369                Err(e) => return Err(e),
370            }
371        }
372        Ok(total)
373    }
374
375    // ========================================================================
376    // Worker loop
377    // ========================================================================
378
379    /// Worker loop — runs on a dedicated OS thread, survives across commits.
380    ///
381    /// Outer loop: each iteration processes one commit cycle.
382    ///   Inner loop: pull documents from MPMC queue, index them, build segments
383    ///   when memory budget is exceeded.
384    ///   On channel close (prepare_commit): flush current builder, signal
385    ///   flush_count, wait for resume with new receiver.
386    ///   On shutdown (Drop): exit permanently.
387    fn worker_loop(
388        state: Arc<WorkerState<D>>,
389        initial_receiver: async_channel::Receiver<Document>,
390        handle: tokio::runtime::Handle,
391    ) {
392        let mut receiver = initial_receiver;
393        let mut my_epoch = 0usize;
394
395        loop {
396            // Wrap the recv+build phase in catch_unwind so a panic doesn't
397            // prevent flush_count from being signaled (which would hang
398            // prepare_commit forever).
399            let build_result = std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| {
400                let mut builder: Option<SegmentBuilder> = None;
401
402                while let Ok(doc) = receiver.recv_blocking() {
403                    // Initialize builder if needed
404                    if builder.is_none() {
405                        match SegmentBuilder::new(
406                            Arc::clone(&state.schema),
407                            state.builder_config.clone(),
408                        ) {
409                            Ok(mut b) => {
410                                for (field, tokenizer) in state.tokenizers.read().iter() {
411                                    b.set_tokenizer(*field, tokenizer.clone_box());
412                                }
413                                builder = Some(b);
414                            }
415                            Err(e) => {
416                                log::error!("Failed to create segment builder: {:?}", e);
417                                continue;
418                            }
419                        }
420                    }
421
422                    let b = builder.as_mut().unwrap();
423                    if let Err(e) = b.add_document(doc) {
424                        log::error!("Failed to index document: {:?}", e);
425                        continue;
426                    }
427
428                    let builder_memory = b.estimated_memory_bytes();
429
430                    if b.num_docs() & 0x3FFF == 0 {
431                        log::debug!(
432                            "[indexing] docs={}, memory={:.2} MB, budget={:.2} MB",
433                            b.num_docs(),
434                            builder_memory as f64 / (1024.0 * 1024.0),
435                            state.memory_budget_per_worker as f64 / (1024.0 * 1024.0)
436                        );
437                    }
438
439                    // Require minimum 100 docs before flushing to avoid tiny segments
440                    const MIN_DOCS_BEFORE_FLUSH: u32 = 100;
441
442                    if builder_memory >= state.memory_budget_per_worker
443                        && b.num_docs() >= MIN_DOCS_BEFORE_FLUSH
444                    {
445                        log::info!(
446                            "[indexing] memory budget reached, building segment: \
447                             docs={}, memory={:.2} MB, budget={:.2} MB",
448                            b.num_docs(),
449                            builder_memory as f64 / (1024.0 * 1024.0),
450                            state.memory_budget_per_worker as f64 / (1024.0 * 1024.0),
451                        );
452                        let full_builder = builder.take().unwrap();
453                        Self::build_segment_inline(&state, full_builder, &handle);
454                    }
455                }
456
457                // Channel closed — flush current builder
458                if let Some(b) = builder.take()
459                    && b.num_docs() > 0
460                {
461                    Self::build_segment_inline(&state, b, &handle);
462                }
463            }));
464
465            if build_result.is_err() {
466                log::error!(
467                    "[worker] panic during indexing cycle — documents in this cycle may be lost"
468                );
469            }
470
471            // Signal flush completion (always, even after panic — prevents
472            // prepare_commit from hanging)
473            let prev = state.flush_count.fetch_add(1, Ordering::Release);
474            if prev + 1 == state.num_workers {
475                // Last worker — wake prepare_commit
476                let _lock = state.flush_mutex.lock();
477                state.flush_cvar.notify_one();
478            }
479
480            // Wait for resume (new channel) or shutdown.
481            // Check resume_epoch to avoid re-cloning a stale receiver from
482            // a previous cycle.
483            {
484                let mut lock = state.resume_receiver.lock();
485                loop {
486                    if state.shutdown.load(Ordering::Acquire) {
487                        return;
488                    }
489                    let current_epoch = state.resume_epoch.load(Ordering::Acquire);
490                    if current_epoch > my_epoch
491                        && let Some(rx) = lock.as_ref()
492                    {
493                        receiver = rx.clone();
494                        my_epoch = current_epoch;
495                        break;
496                    }
497                    state.resume_cvar.wait(&mut lock);
498                }
499            }
500        }
501    }
502
503    /// Build a segment on the worker thread. Uses `Handle::block_on()` to bridge
504    /// into async context for I/O (streaming writers). CPU work (rayon) stays on
505    /// the worker thread / rayon pool.
506    fn build_segment_inline(
507        state: &WorkerState<D>,
508        builder: SegmentBuilder,
509        handle: &tokio::runtime::Handle,
510    ) {
511        let segment_id = SegmentId::new();
512        let segment_hex = segment_id.to_hex();
513        let trained = state.segment_manager.trained();
514        let doc_count = builder.num_docs();
515        let build_start = std::time::Instant::now();
516
517        log::info!(
518            "[segment_build] segment_id={} doc_count={} ann={}",
519            segment_hex,
520            doc_count,
521            trained.is_some()
522        );
523
524        match handle.block_on(builder.build(
525            state.directory.as_ref(),
526            segment_id,
527            trained.as_deref(),
528        )) {
529            Ok(meta) if meta.num_docs > 0 => {
530                let duration_ms = build_start.elapsed().as_millis() as u64;
531                log::info!(
532                    "[segment_build_done] segment_id={} doc_count={} duration_ms={}",
533                    segment_hex,
534                    meta.num_docs,
535                    duration_ms,
536                );
537                state
538                    .built_segments
539                    .lock()
540                    .push((segment_hex, meta.num_docs));
541            }
542            Ok(_) => {}
543            Err(e) => {
544                log::error!(
545                    "[segment_build_failed] segment_id={} error={:?}",
546                    segment_hex,
547                    e
548                );
549            }
550        }
551    }
552
553    // ========================================================================
554    // Public API — commit, merge, etc.
555    // ========================================================================
556
557    /// Check merge policy and spawn a background merge if needed.
558    pub async fn maybe_merge(&self) {
559        self.segment_manager.maybe_merge().await;
560    }
561
562    /// Wait for the in-flight background merge to complete (if any).
563    pub async fn wait_for_merging_thread(&self) {
564        self.segment_manager.wait_for_merging_thread().await;
565    }
566
567    /// Wait for all eligible merges to complete, including cascading merges.
568    pub async fn wait_for_all_merges(&self) {
569        self.segment_manager.wait_for_all_merges().await;
570    }
571
572    /// Get the segment tracker for sharing with readers.
573    pub fn tracker(&self) -> std::sync::Arc<crate::segment::SegmentTracker> {
574        self.segment_manager.tracker()
575    }
576
577    /// Acquire a snapshot of current segments for reading.
578    pub async fn acquire_snapshot(&self) -> crate::segment::SegmentSnapshot {
579        self.segment_manager.acquire_snapshot().await
580    }
581
582    /// Clean up orphan segment files not registered in metadata.
583    pub async fn cleanup_orphan_segments(&self) -> Result<usize> {
584        self.segment_manager.cleanup_orphan_segments().await
585    }
586
587    /// Prepare commit — signal workers to flush, wait for completion, collect segments.
588    ///
589    /// All documents sent via `add_document` before this call are guaranteed
590    /// to be written to segment files on disk. Segments are NOT yet registered
591    /// in metadata — call `PreparedCommit::commit()` for that.
592    ///
593    /// Workers are NOT destroyed — they flush their builders and wait for
594    /// `resume_workers()` to give them a new channel.
595    ///
596    /// `add_document` will return `Closed` error until commit/abort resumes workers.
597    pub async fn prepare_commit(&mut self) -> Result<PreparedCommit<'_, D>> {
598        // 1. Close channel → workers drain remaining docs and flush builders
599        self.doc_sender.close();
600
601        // Wake any workers still waiting on resume_cvar from previous cycle.
602        // They'll clone the stale receiver, enter recv_blocking, get Err
603        // immediately (sender already closed), flush, and signal completion.
604        self.worker_state.resume_cvar.notify_all();
605
606        // 2. Wait for all workers to complete their flush (via spawn_blocking
607        //    to avoid blocking the tokio runtime)
608        let state = Arc::clone(&self.worker_state);
609        let all_flushed = tokio::task::spawn_blocking(move || {
610            let mut lock = state.flush_mutex.lock();
611            let deadline = std::time::Instant::now() + std::time::Duration::from_secs(300);
612            while state.flush_count.load(Ordering::Acquire) < state.num_workers {
613                let remaining = deadline.saturating_duration_since(std::time::Instant::now());
614                if remaining.is_zero() {
615                    log::error!(
616                        "[prepare_commit] timed out waiting for workers: {}/{} flushed",
617                        state.flush_count.load(Ordering::Acquire),
618                        state.num_workers
619                    );
620                    return false;
621                }
622                state.flush_cvar.wait_for(&mut lock, remaining);
623            }
624            true
625        })
626        .await
627        .map_err(|e| Error::Internal(format!("Failed to wait for workers: {}", e)))?;
628
629        if !all_flushed {
630            // Resume workers so the system isn't stuck, then return error
631            self.resume_workers();
632            return Err(Error::Internal(format!(
633                "prepare_commit timed out: {}/{} workers flushed",
634                self.worker_state.flush_count.load(Ordering::Acquire),
635                self.worker_state.num_workers
636            )));
637        }
638
639        // 3. Collect built segments
640        let built = std::mem::take(&mut *self.worker_state.built_segments.lock());
641        self.flushed_segments.extend(built);
642
643        Ok(PreparedCommit {
644            writer: self,
645            is_resolved: false,
646        })
647    }
648
649    /// Commit (convenience): prepare_commit + commit in one call.
650    ///
651    /// Guarantees all prior `add_document` calls are committed.
652    /// Vector training is decoupled — call `build_vector_index()` manually.
653    pub async fn commit(&mut self) -> Result<()> {
654        self.prepare_commit().await?.commit().await
655    }
656
657    /// Force merge all segments into one.
658    pub async fn force_merge(&mut self) -> Result<()> {
659        self.prepare_commit().await?.commit().await?;
660        self.segment_manager.force_merge().await
661    }
662
663    /// Resume workers with a fresh channel. Called after commit or abort.
664    ///
665    /// Workers are already alive — just give them a new channel and wake them.
666    /// If the tokio runtime has shut down (e.g., program exit), this is a no-op.
667    fn resume_workers(&mut self) {
668        if tokio::runtime::Handle::try_current().is_err() {
669            // Runtime is gone — signal permanent shutdown so workers don't
670            // hang forever on resume_cvar.
671            self.worker_state.shutdown.store(true, Ordering::Release);
672            self.worker_state.resume_cvar.notify_all();
673            return;
674        }
675
676        // Reset flush count for next cycle
677        self.worker_state.flush_count.store(0, Ordering::Release);
678
679        // Create new channel
680        let (sender, receiver) = async_channel::bounded(PIPELINE_MAX_SIZE_IN_DOCS);
681        self.doc_sender = sender;
682
683        // Set new receiver, bump epoch, and wake all workers
684        {
685            let mut lock = self.worker_state.resume_receiver.lock();
686            *lock = Some(receiver);
687        }
688        self.worker_state
689            .resume_epoch
690            .fetch_add(1, Ordering::Release);
691        self.worker_state.resume_cvar.notify_all();
692    }
693
694    // Vector index methods (build_vector_index, etc.) are in vector_builder.rs
695}
696
697impl<D: DirectoryWriter + 'static> Drop for IndexWriter<D> {
698    fn drop(&mut self) {
699        // 1. Signal permanent shutdown
700        self.worker_state.shutdown.store(true, Ordering::Release);
701        // 2. Close channel to wake workers blocked on recv_blocking
702        self.doc_sender.close();
703        // 3. Wake workers that might be waiting on resume_cvar
704        self.worker_state.resume_cvar.notify_all();
705        // 4. Join worker threads
706        for w in std::mem::take(&mut self.workers) {
707            let _ = w.join();
708        }
709    }
710}
711
712/// A prepared commit that can be finalized or aborted.
713///
714/// Two-phase commit guard. Between `prepare_commit()` and
715/// `commit()`/`abort()`, segments are on disk but NOT in metadata.
716/// Dropping without calling either will auto-abort (discard segments,
717/// respawn workers).
718pub struct PreparedCommit<'a, D: DirectoryWriter + 'static> {
719    writer: &'a mut IndexWriter<D>,
720    is_resolved: bool,
721}
722
723impl<'a, D: DirectoryWriter + 'static> PreparedCommit<'a, D> {
724    /// Finalize: register segments in metadata, evaluate merge policy, resume workers.
725    pub async fn commit(mut self) -> Result<()> {
726        self.is_resolved = true;
727        let segments = std::mem::take(&mut self.writer.flushed_segments);
728        self.writer.segment_manager.commit(segments).await?;
729
730        // Refresh primary key index with new committed readers (parallel open)
731        if let Some(ref mut pk_index) = self.writer.primary_key_index {
732            let snapshot = self.writer.segment_manager.acquire_snapshot().await;
733            let open_futures: Vec<_> = snapshot
734                .segment_ids()
735                .iter()
736                .filter_map(|seg_id_str| {
737                    let seg_id = crate::segment::SegmentId::from_hex(seg_id_str)?;
738                    let dir = self.writer.directory.as_ref();
739                    let schema = Arc::clone(&self.writer.schema);
740                    let cache_blocks = self.writer.config.term_cache_blocks;
741                    Some(async move {
742                        crate::segment::SegmentReader::open(dir, seg_id, schema, cache_blocks)
743                            .await
744                            .map(Arc::new)
745                    })
746                })
747                .collect();
748            let readers = futures::future::try_join_all(open_futures).await?;
749            pk_index.refresh(readers, snapshot);
750        }
751
752        self.writer.segment_manager.maybe_merge().await;
753        self.writer.resume_workers();
754        Ok(())
755    }
756
757    /// Abort: discard prepared segments, resume workers.
758    /// Segment files become orphans (cleaned up by `cleanup_orphan_segments`).
759    pub fn abort(mut self) {
760        self.is_resolved = true;
761        self.writer.flushed_segments.clear();
762        if let Some(ref mut pk_index) = self.writer.primary_key_index {
763            pk_index.clear_uncommitted();
764        }
765        self.writer.resume_workers();
766    }
767}
768
769impl<D: DirectoryWriter + 'static> Drop for PreparedCommit<'_, D> {
770    fn drop(&mut self) {
771        if !self.is_resolved {
772            log::warn!("PreparedCommit dropped without commit/abort — auto-aborting");
773            self.writer.flushed_segments.clear();
774            if let Some(ref mut pk_index) = self.writer.primary_key_index {
775                pk_index.clear_uncommitted();
776            }
777            self.writer.resume_workers();
778        }
779    }
780}