From 9c6e6141c092d3b4263ba94f291d7e2c94489d7e Mon Sep 17 00:00:00 2001 From: wangjingjing Date: Mon, 30 Dec 2019 19:15:18 +0800 Subject: [PATCH 01/49] only retain one commit for numeric doc values update --- src/core/index/reader/directory_reader.rs | 14 ++++++++++++++ src/core/index/writer/delete_policy.rs | 3 --- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/core/index/reader/directory_reader.rs b/src/core/index/reader/directory_reader.rs index 7780f20..7187db9 100644 --- a/src/core/index/reader/directory_reader.rs +++ b/src/core/index/reader/directory_reader.rs @@ -245,6 +245,20 @@ where self.writer = writer; } + pub fn set_associate_writer( + &mut self, + writer: Option>, + ) -> Result<()> { + if self.writer.is_none() && self.readers.len() > 0 && writer.is_some() { + writer + .as_ref() + .unwrap() + .inc_ref_deleter(&self.segment_infos)?; + self.writer = writer; + } + Ok(()) + } + pub fn get_writer(&self) -> Option> { self.writer.clone() } diff --git a/src/core/index/writer/delete_policy.rs b/src/core/index/writer/delete_policy.rs index 2cc10a8..957f890 100644 --- a/src/core/index/writer/delete_policy.rs +++ b/src/core/index/writer/delete_policy.rs @@ -98,9 +98,6 @@ impl KeepOnlyLastCommitDeletionPolicy { pub fn on_commit(&self, mut commits: Vec<&mut CommitPoint>) -> Result<()> { commits.pop(); - if commits.len() > 0 && commits[commits.len() - 1].has_dv_updates() { - commits.pop(); - } for commit in commits { commit.delete()?; } From 0871501069951f8e20cb4824c23f28b9720cfd01 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Thu, 2 Jan 2020 15:24:40 +0800 Subject: [PATCH 02/49] explicit merge --- src/core/index/merge/merge_policy.rs | 2 +- src/core/index/writer/index_writer.rs | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/core/index/merge/merge_policy.rs b/src/core/index/merge/merge_policy.rs index 241a69a..10c4ec5 100644 --- a/src/core/index/merge/merge_policy.rs +++ b/src/core/index/merge/merge_policy.rs @@ -431,7 +431,7 @@ impl Default for TieredMergePolicy { max_merged_segment_bytes: 5 * 1024 * 1024 * 1024, max_merge_at_once_explicit: 30, floor_segment_bytes: 2 * 1024 * 1024, - segs_per_tier: 5.0, + segs_per_tier: 10.0, force_merge_deletes_pct_allowed: 10.0, reclaim_deletes_weight: 2.0, } diff --git a/src/core/index/writer/index_writer.rs b/src/core/index/writer/index_writer.rs index 75ca888..3d96fcf 100644 --- a/src/core/index/writer/index_writer.rs +++ b/src/core/index/writer/index_writer.rs @@ -644,6 +644,10 @@ where pub fn tragedy(&self) -> Option<&Error> { self.writer.tragedy.as_ref() } + + pub fn explicit_merge(&self) -> Result<()> { + IndexWriterInner::maybe_merge(self, MergerTrigger::Explicit, None) + } } impl IndexWriter From fd535d6c6a3ffd6b82c21c3e5b83fecfefcedfc2 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Mon, 6 Jan 2020 12:26:49 +0800 Subject: [PATCH 03/49] less segment --- src/core/index/merge/merge_policy.rs | 125 ++++++++++++++++-- src/core/index/merge/merge_scheduler.rs | 4 +- .../index/writer/doc_writer_per_thread.rs | 37 +++--- src/core/index/writer/index_writer.rs | 6 +- 4 files changed, 135 insertions(+), 37 deletions(-) diff --git a/src/core/index/merge/merge_policy.rs b/src/core/index/merge/merge_policy.rs index 10c4ec5..fd22d9d 100644 --- a/src/core/index/merge/merge_policy.rs +++ b/src/core/index/merge/merge_policy.rs @@ -27,6 +27,7 @@ use error::{ Result, }; +use core::index::merge::merge_scheduler::MAX_MERGING_COUNT; use std::cell::Cell; use std::cmp::Ordering; use std::collections::{HashMap, HashSet}; @@ -431,7 +432,7 @@ impl Default for TieredMergePolicy { max_merged_segment_bytes: 5 * 1024 * 1024 * 1024, max_merge_at_once_explicit: 30, floor_segment_bytes: 2 * 1024 * 1024, - segs_per_tier: 10.0, + segs_per_tier: 5.0, force_merge_deletes_pct_allowed: 10.0, reclaim_deletes_weight: 2.0, } @@ -539,12 +540,108 @@ impl TieredMergePolicy { MergeScore::new(merge_score, skew, non_del_ratio) } + + fn find_merges_explicit( + &self, + segment_infos: &SegmentInfos, + writer: &IndexWriter, + ) -> Result>> + where + D: Directory + Send + Sync + 'static, + C: Codec, + MS: MergeScheduler, + MP: MergePolicy, + { + let mut infos_sorted = segment_infos.segments.clone(); + if infos_sorted.len() <= self.segs_per_tier as usize { + return Ok(None); + } + + let comparator = SegmentByteSizeDescending::new(writer, self); + infos_sorted.sort_by(|o1, o2| comparator.compare(o1.as_ref(), o2.as_ref())); + + let mut next_idx = 0; + let mut info_seg_bytes = vec![]; + for info in infos_sorted.iter() { + let seg_bytes = self.size(info.as_ref(), writer); + info_seg_bytes.push(seg_bytes); + + if seg_bytes > (self.max_merged_segment_bytes as f64 * 0.8) as i64 { + next_idx += 1; + } + } + + let merging = writer.merging_segments(); + let mut to_be_merged = HashSet::new(); + let mut candidates = vec![]; + let mut spec = MergeSpecification::default(); + + for i in next_idx..infos_sorted.len() { + if merging.contains(&infos_sorted[i].info.name) + || to_be_merged.contains(&infos_sorted[i].info.name) + { + continue; + } + + let mut next_merges = vec![i]; + let mut curr_merge_bytes = info_seg_bytes[i]; + for j in i + 1..infos_sorted.len() { + if curr_merge_bytes > self.max_merged_segment_bytes as i64 + || next_merges.len() >= self.max_merge_at_once as usize + || (info_seg_bytes[i] > (self.max_merged_segment_bytes as f64 * 0.1) as i64 + && info_seg_bytes[i] > info_seg_bytes[j] * (self.max_merge_at_once as i64)) + { + break; + } else if curr_merge_bytes + info_seg_bytes[j] + > self.max_merged_segment_bytes as i64 + || merging.contains(&infos_sorted[j].info.name) + || to_be_merged.contains(&infos_sorted[j].info.name) + { + continue; + } + + next_merges.push(j); + curr_merge_bytes += info_seg_bytes[j]; + } + + if next_merges.len() == 1 { + continue; + } + + let mut segments = Vec::with_capacity(next_merges.len()); + for idx in next_merges { + segments.push(infos_sorted[idx].clone()); + to_be_merged.insert(infos_sorted[idx].info.name.clone()); + } + + let merge = OneMerge::new(segments, writer.next_merge_id())?; + candidates.push(merge); + } + + loop { + if spec.merges.len() >= MAX_MERGING_COUNT { + break; + } + + if let Some(one_merge) = candidates.pop() { + spec.add(one_merge); + } else { + break; + } + } + + if spec.merges.is_empty() { + return Ok(None); + } else { + return Ok(Some(spec)); + } + } } impl MergePolicy for TieredMergePolicy { fn find_merges( &self, - _merge_trigger: MergerTrigger, + merge_trigger: MergerTrigger, segment_infos: &SegmentInfos, writer: &IndexWriter, ) -> Result>> @@ -554,18 +651,20 @@ impl MergePolicy for TieredMergePolicy { MS: MergeScheduler, MP: MergePolicy, { + match merge_trigger { + MergerTrigger::Explicit => { + return self.find_merges_explicit(segment_infos, writer); + } + _ => {} + } + if segment_infos.len() == 0 { return Ok(None); } - let merging = &writer.merging_segments(); - let mut to_be_merged = HashSet::new(); - let mut infos_sorted = segment_infos.segments.clone(); - { - let comparator = SegmentByteSizeDescending::new(writer, self); - infos_sorted.sort_by(|o1, o2| comparator.compare(o1.as_ref(), o2.as_ref())); - } + let comparator = SegmentByteSizeDescending::new(writer, self); + infos_sorted.sort_by(|o1, o2| comparator.compare(o1.as_ref(), o2.as_ref())); // Compute total index bytes & print details about the index let mut total_index_bytes = 0; @@ -620,10 +719,8 @@ impl MergePolicy for TieredMergePolicy { } } - min_segment_bytes = self.floor_size(min_segment_bytes); - // Compute max allowed segs in the index - let mut level_size = min_segment_bytes; + let mut level_size = self.floor_size(min_segment_bytes); let mut bytes_left = total_index_bytes; let mut allowed_seg_count = 0.0; loop { @@ -637,6 +734,8 @@ impl MergePolicy for TieredMergePolicy { level_size *= self.max_merge_at_once as i64; } + let merging = writer.merging_segments(); + let mut to_be_merged = HashSet::new(); let allowed_seg_count_int = allowed_seg_count as u32; let mut spec = MergeSpecification::default(); @@ -734,7 +833,7 @@ impl MergePolicy for TieredMergePolicy { } } - if !best.is_empty() { + if best.len() > 1 { let mut segments = Vec::with_capacity(best.len()); for s in best { segments.push(Arc::clone(s)); diff --git a/src/core/index/merge/merge_scheduler.rs b/src/core/index/merge/merge_scheduler.rs index 6068250..ccf8b54 100644 --- a/src/core/index/merge/merge_scheduler.rs +++ b/src/core/index/merge/merge_scheduler.rs @@ -182,13 +182,15 @@ const START_MB_PER_SEC: f64 = 20.0; // their own thread (up until maxMergeCount). const MIN_BIG_MERGE_MB: f64 = 50.0; +pub const MAX_MERGING_COUNT: usize = 5; + impl ConcurrentMergeSchedulerInner { fn new(max_thread_count: usize) -> Self { ConcurrentMergeSchedulerInner { lock: Mutex::new(()), cond: Condvar::new(), merge_tasks: vec![], - max_merge_count: max_thread_count.max(5), + max_merge_count: max_thread_count.max(MAX_MERGING_COUNT), max_thread_count, merge_thread_count: 0, target_mb_per_sec: START_MB_PER_SEC, diff --git a/src/core/index/writer/doc_writer_per_thread.rs b/src/core/index/writer/doc_writer_per_thread.rs index 44f8f0a..1239c37 100644 --- a/src/core/index/writer/doc_writer_per_thread.rs +++ b/src/core/index/writer/doc_writer_per_thread.rs @@ -31,12 +31,9 @@ use core::{ use std::collections::{HashMap, HashSet}; use std::sync::atomic::{AtomicBool, AtomicI64, AtomicU64, Ordering}; -use std::sync::{Arc, Mutex, MutexGuard, Weak}; +use std::sync::{Arc, Mutex, MutexGuard, RwLock, RwLockWriteGuard, Weak}; use std::time::SystemTime; -use crossbeam::queue::ArrayQueue; -use crossbeam::sync::{ShardedLock, ShardedLockWriteGuard}; - use core::codec::{PackedLongDocMap, SorterDocMap}; use core::util::external::Volatile; use core::util::Bits; @@ -664,7 +661,7 @@ pub struct DocumentsWriterPerThreadPool< MS: MergeScheduler, MP: MergePolicy, > { - inner: ShardedLock>, + inner: RwLock>, aborted: Volatile, } @@ -676,7 +673,7 @@ struct DWPTPoolInner< > { thread_states: Vec>>, // valid thread_state index in `self.thread_states` - free_list: ArrayQueue, + free_list: Vec, } impl DocumentsWriterPerThreadPool @@ -689,10 +686,10 @@ where pub fn new() -> Self { let inner = DWPTPoolInner { thread_states: vec![], - free_list: ArrayQueue::new(32), + free_list: Vec::with_capacity(64), }; DocumentsWriterPerThreadPool { - inner: ShardedLock::new(inner), + inner: RwLock::new(inner), aborted: Volatile::new(false), } } @@ -716,19 +713,19 @@ where /// NOTE: the returned `ThreadState` is already locked iff non-None fn new_thread_state( &self, - mut guard: ShardedLockWriteGuard<'_, DWPTPoolInner>, + mut guard: RwLockWriteGuard<'_, DWPTPoolInner>, ) -> Result>> { let thread_state = Arc::new(ThreadState::new(None, guard.thread_states.len())); guard.thread_states.push(thread_state); let idx = guard.thread_states.len() - 1; // recap the free list queue - if guard.thread_states.len() > guard.free_list.capacity() { - let new_free_list = ArrayQueue::new(guard.free_list.capacity() * 2); - while let Ok(idx) = guard.free_list.pop() { - new_free_list.push(idx).unwrap(); - } - guard.free_list = new_free_list; - } + // if guard.thread_states.len() > guard.free_list.capacity() { + // let mut new_free_list = Vec::with_capacity(guard.free_list.capacity() * 2); + // while let Some(idx) = guard.free_list.pop() { + // new_free_list.push(idx); + // } + // guard.free_list = new_free_list; + // } Ok(Arc::clone(&guard.thread_states[idx])) } @@ -745,8 +742,8 @@ where pub fn get_and_lock(&self) -> Result>> { { - let guard = self.inner.read().unwrap(); - if let Ok(idx) = guard.free_list.pop() { + let mut guard = self.inner.write().unwrap(); + if let Some(idx) = guard.free_list.pop() { return Ok(Arc::clone(&guard.thread_states[idx])); } } @@ -755,9 +752,9 @@ where } pub fn release(&self, state: Arc>) { - let guard = self.inner.read().unwrap(); + let mut guard = self.inner.write().unwrap(); // this shouldn't fail - guard.free_list.push(state.index).unwrap(); + guard.free_list.push(state.index); // In case any thread is waiting, wake one of them up since we just // released a thread state; notify() should be sufficient but we do // notifyAll defensively: diff --git a/src/core/index/writer/index_writer.rs b/src/core/index/writer/index_writer.rs index 3d96fcf..418fce8 100644 --- a/src/core/index/writer/index_writer.rs +++ b/src/core/index/writer/index_writer.rs @@ -305,7 +305,7 @@ where } pub fn num_docs(&self) -> u32 { - let _l = self.writer.lock.lock().unwrap(); + // let _l = self.writer.lock.lock().unwrap(); let mut count = self.writer.doc_writer.num_docs(); for info in &self.writer.segment_infos.segments { count += info.info.max_doc() as u32 - self.writer.num_deleted_docs(&info); @@ -1106,7 +1106,7 @@ where )?; if any_changes { - let _ = Self::maybe_merge(index_writer, MergerTrigger::FullFlush, None); + let _ = Self::maybe_merge(index_writer, MergerTrigger::Explicit, None); } debug!( @@ -2102,7 +2102,7 @@ where // We can be called during close, when closing==true, so we must pass false to ensureOpen: index_writer.writer.ensure_open(false)?; if Self::do_flush(index_writer, apply_all_deletes)? && trigger_merge { - Self::maybe_merge(index_writer, MergerTrigger::FullFlush, None)?; + Self::maybe_merge(index_writer, MergerTrigger::Explicit, None)?; } Ok(()) } From 4c41990a0b66ac60e136010b36b68a441d321ccd Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Tue, 11 Feb 2020 16:34:54 +0800 Subject: [PATCH 04/49] variant_seq --- src/core/util/variant_value.rs | 23 ++++++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/src/core/util/variant_value.rs b/src/core/util/variant_value.rs index 20618b3..45e435a 100644 --- a/src/core/util/variant_value.rs +++ b/src/core/util/variant_value.rs @@ -12,7 +12,7 @@ // limitations under the License. use serde; -use serde::ser::SerializeMap; +use serde::ser::{SerializeMap, SerializeSeq}; use std::cmp::Ordering; use std::collections::HashMap; use std::fmt; @@ -29,8 +29,9 @@ pub enum VariantValue { Long(i64), Float(f32), Double(f64), - VString(String), // SHOULD BORROW ? - Binary(Vec), // SHOULD BORROW ? + VString(String), + Binary(Vec), + Vec(Vec), Map(HashMap), } @@ -113,6 +114,13 @@ impl VariantValue { } } + pub fn get_vec(&self) -> Option<&Vec> { + match self { + VariantValue::Vec(v) => Some(v), + _ => None, + } + } + pub fn get_map(&self) -> Option<&HashMap> { match self { VariantValue::Map(m) => Some(m), @@ -135,6 +143,7 @@ impl fmt::Display for VariantValue { VariantValue::Double(d) => write!(f, "{:.6}", d), VariantValue::VString(ref s) => write!(f, "{}", s), VariantValue::Binary(ref _b) => write!(f, "Binary(unprintable)"), + VariantValue::Vec(ref v) => write!(f, "{:?}", v), VariantValue::Map(ref m) => write!(f, "{:?}", m), } } @@ -155,6 +164,14 @@ impl serde::Serialize for VariantValue { VariantValue::Double(d) => serializer.serialize_f64(d), VariantValue::VString(ref s) => serializer.serialize_str(s.as_str()), VariantValue::Binary(ref b) => serializer.serialize_bytes(b), + VariantValue::Vec(ref vec) => { + let mut seq = serializer.serialize_seq(Some(vec.len())).unwrap(); + for v in vec { + seq.serialize_element(v)?; + } + + seq.end() + } VariantValue::Map(ref m) => { let mut map = serializer.serialize_map(Some(m.len())).unwrap(); for (k, v) in m { From 00988ca7b46b9fe203fe59ed4beaea65a3b6dbbd Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Wed, 19 Feb 2020 11:17:02 +0800 Subject: [PATCH 05/49] some tips --- src/core/index/merge/merge_policy.rs | 2 ++ src/core/index/writer/doc_writer.rs | 11 +++++++---- src/core/index/writer/index_writer.rs | 4 ++++ 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/core/index/merge/merge_policy.rs b/src/core/index/merge/merge_policy.rs index fd22d9d..6261cc7 100644 --- a/src/core/index/merge/merge_policy.rs +++ b/src/core/index/merge/merge_policy.rs @@ -571,6 +571,8 @@ impl TieredMergePolicy { } } + next_idx = next_idx.max((self.segs_per_tier * 0.5) as usize); + let merging = writer.merging_segments(); let mut to_be_merged = HashSet::new(); let mut candidates = vec![]; diff --git a/src/core/index/writer/doc_writer.rs b/src/core/index/writer/doc_writer.rs index 635545e..9dfe250 100644 --- a/src/core/index/writer/doc_writer.rs +++ b/src/core/index/writer/doc_writer.rs @@ -579,10 +579,13 @@ where thread::sleep(Duration::from_millis(100)); - let doc_writer = &index_writer_inner.upgrade().unwrap().doc_writer; - if let Some(next_pending_flush) = doc_writer.flush_control.next_pending_flush() { - if let Err(e) = doc_writer.do_flush(next_pending_flush) { - error!("flush err:{:?}", e); + if let Some(index_writer_inner_upgrade) = index_writer_inner.upgrade() { + let doc_writer = &index_writer_inner_upgrade.doc_writer; + if let Some(next_pending_flush) = doc_writer.flush_control.next_pending_flush() + { + if let Err(e) = doc_writer.do_flush(next_pending_flush) { + error!("flush err:{:?}", e); + } } } }); diff --git a/src/core/index/writer/index_writer.rs b/src/core/index/writer/index_writer.rs index 418fce8..29ee6b6 100644 --- a/src/core/index/writer/index_writer.rs +++ b/src/core/index/writer/index_writer.rs @@ -4506,6 +4506,10 @@ where ) -> Result>> { let mut new_dv_files = HashMap::new(); for (field, updates) in &self.pending_dv_updates { + if self.reader.is_none() || infos.field_info_by_name(field).is_none() { + continue; + } + let info = self.reader.as_ref().unwrap().si.clone(); let tracker = Arc::new(TrackingDirectoryWrapper::new(info.info.directory.as_ref())); let dv_gen = info.next_write_doc_values_gen(); From c87b14860c0bd6426a71ac8c809fd1a05aaa199d Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Fri, 21 Feb 2020 20:16:30 +0800 Subject: [PATCH 06/49] fix total hits --- src/core/index/merge/merge_policy.rs | 4 ---- src/core/search/collector/top_docs.rs | 4 +--- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/src/core/index/merge/merge_policy.rs b/src/core/index/merge/merge_policy.rs index 6261cc7..508ff28 100644 --- a/src/core/index/merge/merge_policy.rs +++ b/src/core/index/merge/merge_policy.rs @@ -553,10 +553,6 @@ impl TieredMergePolicy { MP: MergePolicy, { let mut infos_sorted = segment_infos.segments.clone(); - if infos_sorted.len() <= self.segs_per_tier as usize { - return Ok(None); - } - let comparator = SegmentByteSizeDescending::new(writer, self); infos_sorted.sort_by(|o1, o2| comparator.compare(o1.as_ref(), o2.as_ref())); diff --git a/src/core/search/collector/top_docs.rs b/src/core/search/collector/top_docs.rs index f6cd52f..15c56a5 100644 --- a/src/core/search/collector/top_docs.rs +++ b/src/core/search/collector/top_docs.rs @@ -66,9 +66,6 @@ impl TopDocsBaseCollector { fn add_doc(&mut self, doc_id: DocId, score: f32) { debug_assert!(self.pq.len() <= self.estimated_hits); - - self.total_hits += 1; - let at_capacity = self.pq.len() == self.estimated_hits; if !at_capacity { @@ -94,6 +91,7 @@ impl Collector for TopDocsBaseCollector { let id = doc + self.cur_doc_base; self.add_doc(id, score); + self.total_hits += 1; Ok(()) } From a700992069a31f2ba3144bac9c224cbbe844373e Mon Sep 17 00:00:00 2001 From: wangjingjing Date: Wed, 4 Mar 2020 10:45:07 +0800 Subject: [PATCH 07/49] ef encoder --- src/core/codec/postings/for_util.rs | 134 +++++- src/core/codec/postings/posting_reader.rs | 515 ++++++++++++++++++--- src/core/codec/postings/posting_writer.rs | 42 +- src/core/util/bit_set.rs | 83 +++- src/core/util/bit_util.rs | 213 +++++++++ src/core/util/context.rs | 2 +- src/core/util/doc_id_set.rs | 182 ++++++++ src/core/util/packed/elias_fano_decoder.rs | 420 +++++++++++++++++ src/core/util/packed/elias_fano_encoder.rs | 436 +++++++++++++++++ src/core/util/packed/mod.rs | 8 + src/lib.rs | 4 + 11 files changed, 1975 insertions(+), 64 deletions(-) create mode 100644 src/core/util/packed/elias_fano_decoder.rs create mode 100644 src/core/util/packed/elias_fano_encoder.rs diff --git a/src/core/codec/postings/for_util.rs b/src/core/codec/postings/for_util.rs index fc7c366..d6263cb 100644 --- a/src/core/codec/postings/for_util.rs +++ b/src/core/codec/postings/for_util.rs @@ -17,8 +17,9 @@ use std::sync::{Arc, Once}; use core::codec::postings::posting_format::BLOCK_SIZE; use core::store::io::{DataOutput, IndexInput, IndexOutput}; use core::util::packed::*; -use core::util::BitsRequired; +use core::util::{BitSet, BitsRequired, DocId, FixedBitSet}; +use core::codec::postings::{EfWriterMeta, EncodeType}; use error::Result; use std::mem::MaybeUninit; use std::ptr; @@ -178,8 +179,20 @@ impl ForUtilInstance { input: &mut dyn IndexInput, encoded: &mut [u8], decoded: &mut [i32], + encode_type: Option<&mut EncodeType>, ) -> Result<()> { - let num_bits = input.read_byte()? as usize; + let code = input.read_byte()?; + if encode_type.is_some() { + let etype = ForUtil::encode_type_from_code(code); + match etype { + EncodeType::PF => {} + _ => { + *(encode_type.unwrap()) = etype; + return Ok(()); + } + } + } + let num_bits = (code & 0x3F) as usize; debug_assert!(num_bits <= 32); if num_bits as i32 == ALL_VALUES_EQUAL { @@ -241,8 +254,47 @@ impl ForUtil { input: &mut dyn IndexInput, encoded: &mut [u8], decoded: &mut [i32], + encode_type: Option<&mut EncodeType>, ) -> Result<()> { - self.instance.read_block(input, encoded, decoded) + self.instance + .read_block(input, encoded, decoded, encode_type) + } + + pub fn read_other_encode_block( + doc_in: &mut dyn IndexInput, + ef_decoder: &mut Option, + encode_type: &EncodeType, + doc_bits: &mut FixedBitSet, + bits_min_doc: &mut DocId, + ) -> Result<()> { + match encode_type { + EncodeType::EF => { + let upper_bound = doc_in.read_vlong()?; + if ef_decoder.is_some() { + let encoder = unsafe { + &mut *(ef_decoder.as_mut().unwrap().get_encoder().as_ref() + as *const EliasFanoEncoder + as *mut EliasFanoEncoder) + }; + encoder.rebuild_not_with_check(BLOCK_SIZE as i64, upper_bound)?; + encoder.deserialize2(doc_in)?; + ef_decoder.as_mut().unwrap().refresh(); + } else { + let mut encoder = + EliasFanoEncoder::get_encoder(BLOCK_SIZE as i64, upper_bound)?; + encoder.deserialize2(doc_in)?; + *ef_decoder = Some(EliasFanoDecoder::new(Arc::new(encoder))); + } + } + EncodeType::BITSET => { + *bits_min_doc = doc_in.read_vint()?; + let num_longs = doc_in.read_byte()? as usize; + doc_bits.resize((num_longs << 6) as usize); + EliasFanoEncoder::read_data2(&mut doc_bits.bits, doc_in)?; + } + _ => {} + } + Ok(()) } fn is_all_equal(data: &[i32]) -> bool { @@ -272,6 +324,7 @@ impl ForUtil { data: &[i32], encoded: &mut [u8], out: &mut impl IndexOutput, + ef_writer_meta: Option<&mut EfWriterMeta>, ) -> Result<()> { if Self::is_all_equal(data) { out.write_byte(0)?; @@ -286,7 +339,58 @@ impl ForUtil { assert!(iters * encoder.byte_value_count() as i32 >= BLOCK_SIZE); let encoded_size = self.instance.encoded_sizes[num_bits - 1]; debug_assert!(iters * encoder.byte_block_count() as i32 >= encoded_size); - + if ef_writer_meta.is_some() { + let meta = ef_writer_meta.unwrap(); + if meta.use_ef { + let mut ef_encoder = EliasFanoEncoder::get_encoder( + BLOCK_SIZE as i64, + (meta.ef_upper_doc - meta.ef_base_doc - 1) as i64, + )?; + let ef_encode_size = ef_encoder.encode_size(); + if ef_encode_size <= MAX_ENCODED_SIZE as i32 { + let mut doc_id = meta.ef_base_doc; + if doc_id < 0 { + doc_id = 0; + } + let mut min_doc = i32::max_value(); + let mut max_doc = 0; + for i in 0..BLOCK_SIZE as usize { + doc_id += data[i]; + if doc_id > max_doc { + max_doc = doc_id; + } + if doc_id < min_doc { + min_doc = doc_id; + } + + ef_encoder.encode_next((doc_id - meta.ef_base_doc - 1) as i64)?; + } + + // write bitset + meta.bits.resize((max_doc - min_doc + 1) as usize); + if meta.bits.encode_size() as i32 <= encoded_size { + // meta.bits.clear_batch(0, meta.bits.len()); + meta.bits.clear_all(); + let mut doc_id = meta.ef_base_doc; + if doc_id < 0 { + doc_id = 0; + } + for i in 0..BLOCK_SIZE as usize { + doc_id += data[i]; + meta.bits.set((doc_id - min_doc) as usize); + } + out.write_byte(ForUtil::encode_type_to_code(EncodeType::BITSET))?; + out.write_vint(min_doc)?; + out.write_byte(meta.bits.num_words as u8)?; + return EliasFanoEncoder::write_data(&mut meta.bits.bits, out); + } + + if !meta.with_pf || ef_encoder.encode_size() <= encoded_size { + return ef_encoder.serialize(out); + } + } + } + } out.write_byte(num_bits as u8)?; encoder.encode_int_to_byte(data, encoded, iters as usize); out.write_bytes(encoded, 0, encoded_size as usize) @@ -295,4 +399,26 @@ impl ForUtil { pub fn skip_block(&self, input: &mut dyn IndexInput) -> Result<()> { self.instance.skip_block(input) } + + #[inline] + pub fn encode_type_from_code(code: u8) -> EncodeType { + match code >> 6 { + 0 => EncodeType::PF, + 1 => EncodeType::EF, + 2 => EncodeType::BITSET, + 3 => EncodeType::FULL, + _ => EncodeType::PF, + } + } + + #[inline] + pub fn encode_type_to_code(etype: EncodeType) -> u8 { + let code: u8 = match etype { + EncodeType::PF => 0, + EncodeType::EF => 1, + EncodeType::BITSET => 2, + EncodeType::FULL => 3, + }; + code << 6 + } } diff --git a/src/core/codec/postings/posting_reader.rs b/src/core/codec/postings/posting_reader.rs index 4b2a291..6ddf3d2 100644 --- a/src/core/codec/postings/posting_reader.rs +++ b/src/core/codec/postings/posting_reader.rs @@ -22,11 +22,13 @@ use core::codec::{PostingIterator, PostingIteratorFlags}; use core::search::{DocIterator, Payload, NO_MORE_DOCS}; use core::store::directory::Directory; use core::store::io::{DataInput, IndexInput}; -use core::util::DocId; use core::util::UnsignedShift; +use core::util::{Bits, DocId, FixedBitSet, ImmutableBitSet}; use error::{ErrorKind::IllegalState, Result}; +use core::util::packed::{EliasFanoDecoder, NO_MORE_VALUES}; +use std::intrinsics::unlikely; use std::sync::Arc; /// Filename extension for document number, frequencies, and skip data. @@ -317,6 +319,14 @@ fn read_vint_block( Ok(()) } +#[derive(Debug, Copy, Clone)] +pub enum EncodeType { + PF, + EF, + BITSET, + FULL, +} + struct BlockDocIterator { encoded: [u8; MAX_ENCODED_SIZE], @@ -367,6 +377,14 @@ struct BlockDocIterator { singleton_doc_id: DocId, for_util: ForUtil, + /// PF/EF/BITSET/FULL + encode_type: EncodeType, + ef_decoder: Option, + ef_base_doc: DocId, + ef_base_total: i32, + doc_bits: FixedBitSet, + bits_min_doc: DocId, + bits_index: i32, } impl BlockDocIterator { @@ -403,6 +421,13 @@ impl BlockDocIterator { index_has_offsets: options.has_offsets(), index_has_payloads: field_info.has_store_payloads, for_util, + encode_type: EncodeType::PF, + ef_decoder: None, + ef_base_doc: -1, + ef_base_total: 0, + doc_bits: FixedBitSet::default(), + bits_min_doc: 0, + bits_index: 0, }; iterator.reset(term_state, flags)?; Ok(iterator) @@ -438,10 +463,27 @@ impl BlockDocIterator { self.next_skip_doc = BLOCK_SIZE - 1; // we won't skip if target is found in first block self.doc_buffer_upto = BLOCK_SIZE; self.skipped = false; + + self.encode_type = EncodeType::PF; + self.ef_decoder = None; + self.ef_base_doc = -1; + self.ef_base_total = 0; + // self.doc_bits.clear_batch(0, self.doc_bits.len()); + self.doc_bits.clear_all(); + self.bits_index = 0; + Ok(()) } fn refill_docs(&mut self) -> Result<()> { + // EF & PF compatible + if self.accum > 0 { + self.ef_base_doc = self.accum; + } + self.ef_base_total = self.doc_upto; + self.encode_type = EncodeType::PF; + self.bits_index = 0; + let left = self.doc_freq - self.doc_upto; debug_assert!(left > 0); if left >= BLOCK_SIZE { @@ -450,6 +492,15 @@ impl BlockDocIterator { doc_in.as_mut(), &mut self.encoded, &mut self.doc_delta_buffer, + Some(&mut self.encode_type), + )?; + + ForUtil::read_other_encode_block( + doc_in.as_mut(), + &mut self.ef_decoder, + &self.encode_type, + &mut self.doc_bits, + &mut self.bits_min_doc, )?; if self.index_has_freq { @@ -458,6 +509,7 @@ impl BlockDocIterator { doc_in.as_mut(), &mut self.encoded, &mut self.freq_buffer, + None, )?; } else { self.for_util.skip_block(doc_in.as_mut())?; // skip over freqs @@ -519,18 +571,41 @@ impl DocIterator for BlockDocIterator { self.refill_docs()?; } - self.accum += self.doc_delta_buffer[self.doc_buffer_upto as usize]; + // set doc id + self.doc = match self.encode_type { + EncodeType::PF => self.accum + self.doc_delta_buffer[self.doc_buffer_upto as usize], + + EncodeType::EF => { + self.ef_decoder.as_mut().unwrap().next_value() as i32 + 1 + self.ef_base_doc + } + + EncodeType::BITSET => { + self.bits_index = self.doc_bits.next_set_bit(self.bits_index as usize); + let doc = self.bits_min_doc + self.bits_index; + self.bits_index += 1; + doc + } + + _ => { + unimplemented!(); + } + }; + + self.accum = self.doc; self.doc_upto += 1; - self.doc = self.accum; + // set doc freq self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; self.doc_buffer_upto += 1; Ok(self.doc) } fn advance(&mut self, target: DocId) -> Result { + if unsafe {unlikely(target == NO_MORE_DOCS)} { + self.doc = NO_MORE_DOCS; + return Ok(self.doc); + } // TODO: make frq block load lazy/skippable - // current skip docID < docIDs generated from current buffer <= next skip docID // we don't need to skip if target is buffered already if self.doc_freq > BLOCK_SIZE && target > self.next_skip_doc { @@ -589,25 +664,81 @@ impl DocIterator for BlockDocIterator { self.refill_docs()?; } - // Now scan... this is an inlined/pared down version - // of nextDoc(): - loop { - self.accum += self.doc_delta_buffer[self.doc_buffer_upto as usize]; - self.doc_upto += 1; + self.doc = match self.encode_type { + EncodeType::PF => { + // Now scan... this is an inlined/pared down version + // of nextDoc(): + loop { + self.accum += self.doc_delta_buffer[self.doc_buffer_upto as usize]; + self.doc_upto += 1; - if self.accum >= target { - break; + if self.accum >= target { + break; + } + self.doc_buffer_upto += 1; + if self.doc_upto == self.doc_freq { + self.doc = NO_MORE_DOCS; + return Ok(self.doc); + } + } + self.accum } - self.doc_buffer_upto += 1; - if self.doc_upto == self.doc_freq { - self.doc = NO_MORE_DOCS; - return Ok(self.doc); + + EncodeType::EF => { + let decoder = self.ef_decoder.as_mut().unwrap(); + let doc = decoder.advance_to_value((target - 1 - self.ef_base_doc) as i64); + if doc == NO_MORE_VALUES { + self.doc = NO_MORE_DOCS; + return Ok(self.doc); + } + self.doc_buffer_upto = decoder.current_index()? as i32; + self.doc_upto = self.ef_base_total + self.doc_buffer_upto + 1; + self.accum = doc as i32 + 1 + self.ef_base_doc; + self.accum } - } + + EncodeType::BITSET => { + if target < self.bits_min_doc { + self.accum = self.bits_min_doc; + self.bits_index = 1; + self.doc_buffer_upto = 0; + } else { + let mut index = target - self.bits_min_doc; + if index >= self.doc_bits.num_bits as i32 { + self.doc = NO_MORE_DOCS; + return Ok(NO_MORE_DOCS); + } + let find = self.doc_bits.get(index as usize)?; + self.accum = if find { + target + } else { + index = self.doc_bits.next_set_bit(index as usize); + if index == NO_MORE_DOCS { + self.doc = NO_MORE_DOCS; + return Ok(NO_MORE_DOCS); + } + self.bits_min_doc + index + }; + self.doc_buffer_upto = self.doc_bits.count_ones_before_index2( + self.doc_buffer_upto, + self.bits_index as usize, + index as usize, + ) as i32; + // self.doc_buffer_upto = + // self.doc_bits.count_ones_before_index(index as usize) as i32; + self.bits_index = index + 1; + } + self.doc_upto = self.ef_base_total + self.doc_buffer_upto + 1; + self.accum + } + + _ => { + unimplemented!(); + } + }; self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; self.doc_buffer_upto += 1; - self.doc = self.accum; Ok(self.doc) } @@ -687,6 +818,14 @@ struct BlockPostingIterator { singleton_doc_id: i32, for_util: ForUtil, + /// PF/EF/BITSET/FULL + encode_type: EncodeType, + ef_decoder: Option, + ef_base_doc: DocId, + ef_base_total: i32, + doc_bits: FixedBitSet, + bits_min_doc: DocId, + bits_index: i32, } impl BlockPostingIterator { @@ -731,6 +870,13 @@ impl BlockPostingIterator { index_has_offsets: options.has_offsets(), index_has_payloads: field_info.has_store_payloads, for_util, + encode_type: EncodeType::PF, + ef_decoder: None, + ef_base_doc: -1, + ef_base_total: 0, + doc_bits: FixedBitSet::default(), + bits_min_doc: 0, + bits_index: 0, }; iterator.reset(term_state)?; Ok(iterator) @@ -776,10 +922,27 @@ impl BlockPostingIterator { }; self.doc_buffer_upto = BLOCK_SIZE; self.skipped = false; + + self.encode_type = EncodeType::PF; + self.ef_decoder = None; + self.ef_base_doc = -1; + self.ef_base_total = 0; + // self.doc_bits.clear_batch(0, self.doc_bits.len()); + self.doc_bits.clear_all(); + self.bits_index = 0; + Ok(()) } fn refill_docs(&mut self) -> Result<()> { + // EF & PF compatible + if self.accum > 0 { + self.ef_base_doc = self.accum; + } + self.ef_base_total = self.doc_upto; + self.encode_type = EncodeType::PF; + self.bits_index = 0; + let left = self.doc_freq - self.doc_upto; if left >= BLOCK_SIZE { let doc_in = self.doc_in.as_mut().unwrap(); @@ -787,9 +950,23 @@ impl BlockPostingIterator { doc_in.as_mut(), &mut self.encoded, &mut self.doc_delta_buffer, + Some(&mut self.encode_type), + )?; + + ForUtil::read_other_encode_block( + doc_in.as_mut(), + &mut self.ef_decoder, + &self.encode_type, + &mut self.doc_bits, + &mut self.bits_min_doc, + )?; + + self.for_util.read_block( + doc_in.as_mut(), + &mut self.encoded, + &mut self.freq_buffer, + None, )?; - self.for_util - .read_block(doc_in.as_mut(), &mut self.encoded, &mut self.freq_buffer)?; } else if self.doc_freq == 1 { self.doc_delta_buffer[0] = self.singleton_doc_id; self.freq_buffer[0] = self.total_term_freq as i32; @@ -837,6 +1014,7 @@ impl BlockPostingIterator { pos_in.as_mut(), &mut self.encoded, &mut self.pos_delta_buffer, + None, )?; } @@ -929,19 +1107,42 @@ impl DocIterator for BlockPostingIterator { if self.doc_buffer_upto == BLOCK_SIZE { self.refill_docs()?; } + // set doc id + self.doc = match self.encode_type { + EncodeType::PF => self.accum + self.doc_delta_buffer[self.doc_buffer_upto as usize], + + EncodeType::EF => { + self.ef_decoder.as_mut().unwrap().next_value() as i32 + 1 + self.ef_base_doc + } + + EncodeType::BITSET => { + self.bits_index = self.doc_bits.next_set_bit(self.bits_index as usize); + let doc = self.bits_min_doc + self.bits_index; + self.bits_index += 1; + doc + } + + _ => { + unimplemented!(); + } + }; + + self.accum = self.doc; - self.accum += self.doc_delta_buffer[self.doc_buffer_upto as usize]; self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; self.pos_pending_count += self.freq; self.doc_buffer_upto += 1; self.doc_upto += 1; - self.doc = self.accum; self.position = 0; Ok(self.doc) } fn advance(&mut self, target: DocId) -> Result { + if unsafe {unlikely(target == NO_MORE_DOCS)} { + self.doc = NO_MORE_DOCS; + return Ok(self.doc); + } // TODO: make frq block load lazy/skippable if target > self.next_skip_doc { @@ -999,26 +1200,91 @@ impl DocIterator for BlockPostingIterator { self.refill_docs()?; } - // Now scan... this is an inlined/pared down version - // of nextDoc(): - loop { - self.accum += self.doc_delta_buffer[self.doc_buffer_upto as usize]; - self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; - self.pos_pending_count += self.freq; - self.doc_buffer_upto += 1; - self.doc_upto += 1; - - if self.accum >= target { - break; + self.doc = match self.encode_type { + EncodeType::PF => { + // Now scan... this is an inlined/pared down version + // of nextDoc(): + loop { + self.accum += self.doc_delta_buffer[self.doc_buffer_upto as usize]; + self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; + self.pos_pending_count += self.freq; + self.doc_buffer_upto += 1; + self.doc_upto += 1; + + if self.accum >= target { + break; + } + if self.doc_upto == self.doc_freq { + self.doc = NO_MORE_DOCS; + return Ok(self.doc); + } + } + self.accum } - if self.doc_upto == self.doc_freq { - self.doc = NO_MORE_DOCS; - return Ok(self.doc); + + EncodeType::EF => { + let decoder = self.ef_decoder.as_mut().unwrap(); + let doc = decoder.advance_to_value((target - 1 - self.ef_base_doc) as i64); + if doc == NO_MORE_VALUES { + self.doc = NO_MORE_DOCS; + return Ok(self.doc); + } + + let current_index = decoder.current_index()? as i32; + for i in self.doc_buffer_upto..=current_index { + self.pos_pending_count += self.freq_buffer[i as usize]; + } + self.freq = self.freq_buffer[current_index as usize]; + self.doc_buffer_upto = current_index + 1; + self.doc_upto = self.ef_base_total + self.doc_buffer_upto; + self.accum = doc as i32 + 1 + self.ef_base_doc; + self.accum + } + + EncodeType::BITSET => { + if target < self.bits_min_doc { + self.accum = self.bits_min_doc; + self.bits_index = 1; + self.doc_buffer_upto = 0; + } else { + let mut index = target - self.bits_min_doc; + if index >= self.doc_bits.num_bits as i32 { + self.doc = NO_MORE_DOCS; + return Ok(NO_MORE_DOCS); + } + let find = self.doc_bits.get(index as usize)?; + // self.bits_index = index + 1; + self.accum = if find { + target + } else { + index = self.doc_bits.next_set_bit(index as usize); + if index == NO_MORE_DOCS { + self.doc = NO_MORE_DOCS; + return Ok(NO_MORE_DOCS); + } + self.bits_min_doc + index + }; + self.doc_buffer_upto = self.doc_bits.count_ones_before_index2( + self.doc_buffer_upto, + self.bits_index as usize, + index as usize, + ) as i32; + // self.doc_buffer_upto = + // self.doc_bits.count_ones_before_index(index as usize) as i32; + self.bits_index = index + 1; + } + self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; + self.doc_buffer_upto += 1; + self.doc_upto = self.ef_base_total + self.doc_buffer_upto; + self.accum } - } + + _ => { + unimplemented!(); + } + }; self.position = 0; - self.doc = self.accum; Ok(self.doc) } @@ -1118,6 +1384,14 @@ struct EverythingIterator { singleton_doc_id: i32, // docid when there is a single pulsed posting, otherwise -1 for_util: ForUtil, + /// PF/EF/BITSET/FULL + encode_type: EncodeType, + ef_decoder: Option, + ef_base_doc: DocId, + ef_base_total: i32, + doc_bits: FixedBitSet, + bits_min_doc: DocId, + bits_index: i32, } impl<'a> EverythingIterator { @@ -1193,6 +1467,13 @@ impl<'a> EverythingIterator { skipped: false, total_term_freq: 0, for_util, + encode_type: EncodeType::PF, + ef_decoder: None, + ef_base_doc: -1, + ef_base_total: 0, + doc_bits: FixedBitSet::default(), + bits_min_doc: 0, + bits_index: 0, }; iterator.reset(term_state, flags)?; @@ -1244,10 +1525,27 @@ impl<'a> EverythingIterator { }; self.doc_buffer_upto = BLOCK_SIZE; self.skipped = false; + + self.encode_type = EncodeType::PF; + self.ef_decoder = None; + self.ef_base_doc = -1; + self.ef_base_total = 0; + // self.doc_bits.clear_batch(0, self.doc_bits.len()); + self.doc_bits.clear_all(); + self.bits_index = 0; + Ok(()) } pub fn refill_docs(&mut self) -> Result<()> { + // EF & PF compatible + if self.accum > 0 { + self.ef_base_doc = self.accum; + } + self.ef_base_total = self.doc_upto; + self.encode_type = EncodeType::PF; + self.bits_index = 0; + let left = self.doc_freq - self.doc_upto; if left >= BLOCK_SIZE { let doc_in = self.doc_in.as_mut().unwrap(); @@ -1255,9 +1553,23 @@ impl<'a> EverythingIterator { doc_in.as_mut(), &mut self.encoded, &mut self.doc_delta_buffer, + Some(&mut self.encode_type), + )?; + + ForUtil::read_other_encode_block( + doc_in.as_mut(), + &mut self.ef_decoder, + &self.encode_type, + &mut self.doc_bits, + &mut self.bits_min_doc, + )?; + + self.for_util.read_block( + doc_in.as_mut(), + &mut self.encoded, + &mut self.freq_buffer, + None, )?; - self.for_util - .read_block(doc_in.as_mut(), &mut self.encoded, &mut self.freq_buffer)?; } else if self.doc_freq == 1 { self.doc_delta_buffer[0] = self.singleton_doc_id; self.freq_buffer[0] = self.total_term_freq as i32; @@ -1321,6 +1633,7 @@ impl<'a> EverythingIterator { pos_in.as_mut(), self.encoded.as_mut(), self.pos_delta_buffer.as_mut(), + None, )?; let pay_in = &mut self.pay_in; @@ -1330,6 +1643,7 @@ impl<'a> EverythingIterator { pay_in.as_mut(), &mut self.encoded, self.payload_length_buffer.as_mut(), + None, )?; let num_bytes = pay_in.read_vint()? as usize; if num_bytes > self.payload_bytes.len() { @@ -1353,11 +1667,13 @@ impl<'a> EverythingIterator { pay_in.as_mut(), &mut self.encoded, self.offset_start_delta_buffer.as_mut(), + None, )?; self.for_util.read_block( pay_in.as_mut(), &mut self.encoded, self.offset_length_buffer.as_mut(), + None, )?; } else { // this works, because when writing a vint block we always force the first @@ -1527,20 +1843,43 @@ impl DocIterator for EverythingIterator { if self.doc_buffer_upto == BLOCK_SIZE { self.refill_docs()?; } + // set doc id + self.doc = match self.encode_type { + EncodeType::PF => self.accum + self.doc_delta_buffer[self.doc_buffer_upto as usize], + + EncodeType::EF => { + self.ef_decoder.as_mut().unwrap().next_value() as i32 + 1 + self.ef_base_doc + } + + EncodeType::BITSET => { + self.bits_index = self.doc_bits.next_set_bit(self.bits_index as usize); + let doc = self.bits_min_doc + self.bits_index; + self.bits_index += 1; + doc + } + + _ => { + unimplemented!(); + } + }; + + self.accum = self.doc; - self.accum += self.doc_delta_buffer[self.doc_buffer_upto as usize]; self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; self.pos_pending_count += self.freq; self.doc_buffer_upto += 1; self.doc_upto += 1; - self.doc = self.accum; self.position = 0; self.last_start_offset = 0; Ok(self.doc) } fn advance(&mut self, target: DocId) -> Result { + if unsafe {unlikely(target == NO_MORE_DOCS)} { + self.doc = NO_MORE_DOCS; + return Ok(self.doc); + } // TODO: make frq block load lazy/skippable if target > self.next_skip_doc { @@ -1596,26 +1935,92 @@ impl DocIterator for EverythingIterator { self.refill_docs()?; } - // Now scan: - loop { - self.accum += self.doc_delta_buffer[self.doc_buffer_upto as usize]; - self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; - self.pos_pending_count += self.freq; - self.doc_buffer_upto += 1; - self.doc_upto += 1; + self.doc = match self.encode_type { + EncodeType::PF => { + // Now scan... this is an inlined/pared down version + // of nextDoc(): + loop { + self.accum += self.doc_delta_buffer[self.doc_buffer_upto as usize]; + self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; + self.pos_pending_count += self.freq; + self.doc_buffer_upto += 1; + self.doc_upto += 1; + + if self.accum >= target { + break; + } + if self.doc_upto == self.doc_freq { + self.doc = NO_MORE_DOCS; + return Ok(self.doc); + } + } + self.accum + } + + EncodeType::EF => { + let decoder = self.ef_decoder.as_mut().unwrap(); + let doc = decoder.advance_to_value((target - 1 - self.ef_base_doc) as i64); + if doc == NO_MORE_VALUES { + self.doc = NO_MORE_DOCS; + return Ok(self.doc); + } + + let current_index = decoder.current_index()? as i32; + for i in self.doc_buffer_upto..=current_index { + self.pos_pending_count += self.freq_buffer[i as usize]; + } + self.freq = self.freq_buffer[current_index as usize]; + self.doc_buffer_upto = current_index + 1; + self.doc_upto = self.ef_base_total + self.doc_buffer_upto; + self.accum = doc as i32 + 1 + self.ef_base_doc; + self.accum + } - if self.accum >= target { - break; + EncodeType::BITSET => { + if target < self.bits_min_doc { + self.accum = self.bits_min_doc; + self.bits_index = 1; + self.doc_buffer_upto = 0; + } else { + let mut index = target - self.bits_min_doc; + if index >= self.doc_bits.num_bits as i32 { + self.doc = NO_MORE_DOCS; + return Ok(NO_MORE_DOCS); + } + let find = self.doc_bits.get(index as usize)?; + // self.bits_index = index + 1; + self.accum = if find { + target + } else { + index = self.doc_bits.next_set_bit(index as usize); + if index == NO_MORE_DOCS { + self.doc = NO_MORE_DOCS; + return Ok(NO_MORE_DOCS); + } + self.bits_min_doc + index + }; + self.doc_buffer_upto = self.doc_bits.count_ones_before_index2( + self.doc_buffer_upto, + self.bits_index as usize, + index as usize, + ) as i32; + // self.doc_buffer_upto = + // self.doc_bits.count_ones_before_index(index as usize) as i32; + self.bits_index = index + 1; + } + self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; + self.doc_buffer_upto += 1; + self.doc_upto = self.ef_base_total + self.doc_buffer_upto; + self.accum } - if self.doc_upto == self.doc_freq { - self.doc = NO_MORE_DOCS; - return Ok(self.doc); + + _ => { + unimplemented!(); } - } + }; self.position = 0; self.last_start_offset = 0; - self.doc = self.accum; Ok(self.doc) } diff --git a/src/core/codec/postings/posting_writer.rs b/src/core/codec/postings/posting_writer.rs index 4c254ed..6d01257 100644 --- a/src/core/codec/postings/posting_writer.rs +++ b/src/core/codec/postings/posting_writer.rs @@ -27,9 +27,36 @@ use core::search::{DocIterator, NO_MORE_DOCS}; use core::store::directory::Directory; use core::store::io::{DataOutput, IndexOutput}; use core::util::packed::COMPACT; -use core::util::{BitSet, DocId, FixedBitSet}; +use core::util::{BitSet, Bits, DocId, FixedBitSet}; use error::{ErrorKind, Result}; +pub struct EfWriterMeta { + pub ef_base_doc: DocId, + pub ef_upper_doc: DocId, + pub use_ef: bool, + pub with_pf: bool, + pub bits: FixedBitSet, +} + +impl EfWriterMeta { + pub fn new() -> Self { + Self { + ef_base_doc: -1, + ef_upper_doc: 0, + use_ef: true, + with_pf: true, + bits: FixedBitSet::default(), + } + } + + pub fn reset(&mut self) { + self.ef_base_doc = -1; + self.ef_upper_doc = 0; + self.bits.clear_batch(0, self.bits.len()); + // self.bits.clear_all(); + } +} + /// Concrete class that writes docId(maybe frq,pos,offset,payloads) list /// with postings format. /// @@ -82,6 +109,7 @@ pub struct Lucene50PostingsWriter { write_positions: bool, write_payloads: bool, write_offsets: bool, + ef_writer_meta: EfWriterMeta, } impl Lucene50PostingsWriter { @@ -211,6 +239,7 @@ impl Lucene50PostingsWriter { write_positions: false, write_payloads: false, write_offsets: false, + ef_writer_meta: EfWriterMeta::new(), }) } @@ -252,6 +281,7 @@ impl Lucene50PostingsWriter { pub fn start_term(&mut self) { self.doc_start_fp = self.doc_out.file_pointer(); + self.ef_writer_meta.reset(); if self.write_positions { self.pos_start_fp = self.pos_out.as_ref().unwrap().file_pointer(); if self.write_payloads || self.write_offsets { @@ -295,16 +325,19 @@ impl Lucene50PostingsWriter { self.doc_count += 1; if self.doc_buffer_upto == BLOCK_SIZE as usize { + self.ef_writer_meta.ef_upper_doc = doc_id; self.for_util.write_block( &self.doc_delta_buffer, &mut self.encoded, &mut self.doc_out, + Some(&mut self.ef_writer_meta), )?; if self.write_freqs { self.for_util.write_block( &self.freq_buffer, &mut self.encoded, &mut self.doc_out, + None, )?; } // NOTE: don't set docBufferUpto back to 0 here; @@ -367,6 +400,7 @@ impl Lucene50PostingsWriter { &self.pos_delta_buffer, &mut self.encoded, self.pos_out.as_mut().unwrap(), + None, )?; if self.write_payloads { @@ -374,6 +408,7 @@ impl Lucene50PostingsWriter { &self.payload_length_buffer, &mut self.encoded, self.pay_out.as_mut().unwrap(), + None, )?; self.pay_out .as_mut() @@ -392,11 +427,13 @@ impl Lucene50PostingsWriter { &self.offset_start_delta_buffer, &mut self.encoded, self.pay_out.as_mut().unwrap(), + None, )?; self.for_util.write_block( &self.offset_length_buffer, &mut self.encoded, self.pay_out.as_mut().unwrap(), + None, )?; } self.pos_buffer_upto = 0; @@ -419,6 +456,7 @@ impl Lucene50PostingsWriter { self.last_block_payload_byte_upto = self.payload_byte_upto; } self.doc_buffer_upto = 0; + self.ef_writer_meta.ef_base_doc = self.last_block_doc_id; } } @@ -575,7 +613,6 @@ impl PostingsWriterBase for Lucene50PostingsWriter { ) -> Result> { self.start_term(); let mut postings_enum = terms.postings_with_flags(self.enum_flags)?; - let mut doc_freq = 0; let mut total_term_freq = 0i32; loop { @@ -592,7 +629,6 @@ impl PostingsWriterBase for Lucene50PostingsWriter { } else { -1 }; - self.start_doc(doc_id, freq)?; if self.write_positions { diff --git a/src/core/util/bit_set.rs b/src/core/util/bit_set.rs index e097657..23b0b50 100644 --- a/src/core/util/bit_set.rs +++ b/src/core/util/bit_set.rs @@ -189,6 +189,87 @@ impl FixedBitSet { } } + pub fn resize(&mut self, num_bits: usize) { + let num_words = bits2words(num_bits); + if num_words != self.bits.len() { + self.bits.resize(num_words, 0); + self.num_words = num_words; + self.num_bits = self.num_words << 6usize; + } + } + + pub fn encode_size(&self) -> usize { + self.num_words << 3 + } + + pub fn count_ones_before_index2( + &self, + doc_upto: i32, + bit_index: usize, + end_index: usize, + ) -> u32 { + let mut count = doc_upto as u32; + if end_index > bit_index { + let mut start_high = bit_index >> 6; + let end_high = end_index >> 6; + if start_high < end_high { + let remain = self.bits[start_high] as usize >> (bit_index & 0x3F); + if remain != 0 { + count += (remain as i64).count_ones(); + } + start_high += 1; + for i in start_high..end_high { + if self.bits[i] != 0 { + count += self.bits[i].count_ones(); + } + } + let low_value = end_index & 0x3F; + if low_value > 0 { + let value = self.bits[end_high] & ((1usize << low_value) - 1) as i64; + if value > 0 { + count += value.count_ones(); + } + } + } else { + let end_remain = end_index & 0x3F; + if end_remain > 0 { + let value = self.bits[end_high] & ((1usize << end_remain) - 1) as i64; + if value > 0 { + let bit_remain = bit_index & 0x3F; + let value = value >> bit_remain as i64; + if value > 0 { + count += value.count_ones(); + } + } + } + } + } + count + } + + pub fn count_ones_before_index(&self, end_index: usize) -> u32 { + let mut count = 0; + if end_index > 0 { + let index = end_index - 1; + let max = index >> 6; + for i in 0..max { + count += self.bits[i].count_ones(); + } + let num_bits = (index & 0x3Fusize) + 1; + count += if num_bits == 64 { + self.bits[max].count_ones() + } else { + (self.bits[max] & ((1usize << num_bits) - 1) as i64).count_ones() + }; + } + count + } + + #[inline] + pub fn clear_all(&mut self) { + self.bits.iter_mut().for_each(|x| *x = 0i64); + } + /// Checks if the bits past numBits are clear. Some methods rely on this implicit /// assumption: search for "Depends on the ghost bits being clear!" /// @return true if the bits past numBits are clear. @@ -335,7 +416,7 @@ impl BitSet for FixedBitSet { } fn clear_batch(&mut self, start_index: usize, end_index: usize) { - debug_assert!(start_index < self.num_bits); + debug_assert!(start_index <= self.num_bits); debug_assert!(end_index <= self.num_bits); if end_index <= start_index { return; diff --git a/src/core/util/bit_util.rs b/src/core/util/bit_util.rs index 8e8fecc..a774f1c 100644 --- a/src/core/util/bit_util.rs +++ b/src/core/util/bit_util.rs @@ -96,3 +96,216 @@ pub fn pop_array(arr: &[i64], word_offset: usize, num_words: usize) -> usize { } pop_count } + +pub const LONG_SIZE: i64 = 64; +pub const LONG_SIZE_32: i32 = LONG_SIZE as i32; +pub const LOG2_LONG_SIZE: i32 = 6; + +/// the number of 0s of the value bits from the left +#[inline] +pub fn number_of_leading_zeros(value: i64) -> i32 { + if value <= 0 { + return if value == 0 { 64 } else { 0 }; + } + let mut value = value; + let mut n = 63; + if value >= (1_i64 << 32) { + n -= 32; + value = value.unsigned_shift(32); + } + if value >= (1_i64 << 16) { + n -= 16; + value = value.unsigned_shift(16); + } + if value >= (1_i64 << 8) { + n -= 8; + value = value.unsigned_shift(8); + } + if value >= (1_i64 << 4) { + n -= 4; + value = value.unsigned_shift(4); + } + if value >= (1_i64 << 2) { + n -= 2; + value = value.unsigned_shift(2); + } + n - value.unsigned_shift(1) as i32 +} + +/// the number of 0s of the value bits from the right +#[inline] +pub fn number_of_trailing_zeros(value: i64) -> i32 { + if value == 0 { + return 64; + } + let mut value = value; + let mut n = 63; + let mut y = value << 32_i64; + if y != 0 { + n -= 32; + value = y; + } + y = value << 16_i64; + if y != 0 { + n -= 16; + value = y; + } + y = value << 8_i64; + if y != 0 { + n -= 8; + value = y; + } + y = value << 4_i64; + if y != 0 { + n -= 4; + value = y; + } + y = value << 2_i64; + if y != 0 { + n -= 2; + value = y; + } + n - (value << 1).unsigned_shift(63) as i32 +} + +/// Input: i -> the value +/// Output: the number of ones in the value bits +#[inline] +pub fn bit_count(i: i64) -> i32 { + // HD, Figure 5-2 + let mut i = i - ((i.unsigned_shift(1)) & 0x5555555555555555_i64); + i = (i & 0x3333333333333333_i64) + ((i.unsigned_shift(2)) & 0x3333333333333333_i64); + i = (i + (i.unsigned_shift(4))) & 0x0f0f0f0f0f0f0f0f_i64; + i = i + (i.unsigned_shift(8)); + i = i + (i.unsigned_shift(16)); + i = i + (i.unsigned_shift(32)); + return (i & 0x7f) as i32; +} + +const L8_L: i64 = 0x0101010101010101; + +lazy_static! { + static ref PS_OVERFLOW: [i64; 64] = { + let mut ps_overflow: [i64; 64] = [0; 64]; + for s in 1..=ps_overflow.len() { + ps_overflow[s - 1] = (128 - s as i64) * L8_L; + } + ps_overflow + }; + static ref SELECT256: [u8; 2048] = { + let mut select256: [u8; 2048] = [0; 2048]; + for b in 0..=0xFF { + for s in 1..=8 { + let byte_index = b | ((s - 1) << 8); + let mut bit_index = select_naive(b, s as i32); + if bit_index < 0 { + bit_index = 127; + } + assert!(bit_index >= 0); + select256[byte_index as usize] = bit_index as u8; + } + } + select256 + }; +} + +/// Select a 1-bit from a long. See also LUCENE-6040. +/// @return The index of the r-th 1 bit in x. This bit must exist. +#[inline] +pub fn select(x: i64, r: i32) -> i32 { + let mut s = (x as i128 - (x & 0xAAAAAAAAAAAAAAAA_u64 as i64).unsigned_shift(1) as i128) as i64; + s = (s & 0x3333333333333333_i64) + (s.unsigned_shift(2) & 0x3333333333333333_i64); + s = (((s + s.unsigned_shift(4)) & 0x0F0F0F0F0F0F0F0F_i64) as u128 * L8_L as u128) as i64; + let b = (((s as u128 + PS_OVERFLOW[(r - 1) as usize] as u128) as i64 & (L8_L << 7)) + .trailing_zeros() + >> 3) + << 3 as i32; + let l = r as i64 - ((s << 8).unsigned_shift((b & 0x3F_u32) as usize) & 0xFF_i64); + + let select_index = + ((x.unsigned_shift((b & 0x3F_u32) as usize) & 0xFF_i64) | ((l - 1) << 8)) as usize; + b as i32 + SELECT256[select_index] as i32 +} + +/// Naive implementation of {@link #select(i64, i32)}, using(@link num::trailing_zeros()) +/// repetitively. Works relatively fast for low ranks. +/// @return The index of the r-th 1 bit in x, or -1 if no such bit exists. +#[inline] +pub fn select_naive(mut x: i64, mut r: i32) -> i32 { + assert!(r >= 1); + let mut s = -1; + while (x != 0) && (r > 0) { + let ntz = x.trailing_zeros() as i32; + x = x.unsigned_shift((ntz + 1) as usize); + s += ntz + 1; + r -= 1; + } + if r > 0 { + -1 + } else { + s + } +} + +#[cfg(test)] +mod tests { + use core::util::bit_util; + + #[test] + fn number_of_leading_zeros() { + assert_eq!(bit_util::number_of_leading_zeros(-1), 0); + assert_eq!(bit_util::number_of_leading_zeros(-2), 0); + assert_eq!(bit_util::number_of_leading_zeros(-100002), 0); + assert_eq!(bit_util::number_of_leading_zeros(128), 56); + assert_eq!(bit_util::number_of_leading_zeros(2), 62); + assert_eq!(bit_util::number_of_leading_zeros(0), 64); + assert_eq!(bit_util::number_of_leading_zeros(i64::max_value()), 1); + } + + #[test] + fn number_of_trailing_zeros() { + assert_eq!(bit_util::number_of_trailing_zeros(-1), 0); + assert_eq!(bit_util::number_of_trailing_zeros(-2), 1); + assert_eq!(bit_util::number_of_trailing_zeros(-3), 0); + assert_eq!(bit_util::number_of_trailing_zeros(0), 64); + assert_eq!(bit_util::number_of_trailing_zeros(i64::max_value()), 0); + } + #[test] + fn bit_count() { + assert_eq!(bit_util::bit_count(9), 2); + assert_eq!(bit_util::bit_count(1), 1); + assert_eq!(bit_util::bit_count(2), 1); + assert_eq!(bit_util::bit_count(-1), 64); + assert_eq!(bit_util::bit_count(0x180000000_i64), 2); + assert_eq!(bit_util::bit_count(0xEFFFFFFFFFFFFFFF_u64 as i64), 63); + assert_eq!(bit_util::bit_count(0xEFEFEFFEFFFFFFFF_u64 as i64), 60); + } + + #[test] + fn select_naive() { + assert_eq!(bit_util::select_naive(1, 1), 0); + assert_eq!(bit_util::select_naive(2, 1), 1); + assert_eq!(bit_util::select_naive(4, 1), 2); + assert_eq!(bit_util::select_naive(5, 2), 2); + assert_eq!(bit_util::select_naive(10, 2), 3); + assert_eq!( + bit_util::select_naive(0xEFEFEFFEFFFFFFFF_u64 as i64, 44), + 45 + ); + assert_eq!(bit_util::select_naive(-1, 1), 0); + assert_eq!(bit_util::select_naive(0x8000000000000001_u64 as i64, 2), 63); + } + + #[test] + fn select() { + assert_eq!(bit_util::select(1, 1), 0); + assert_eq!(bit_util::select(2, 1), 1); + assert_eq!(bit_util::select(4, 1), 2); + assert_eq!(bit_util::select(5, 2), 2); + assert_eq!(bit_util::select(10, 2), 3); + assert_eq!(bit_util::select(0xEFEFEFFEFFFFFFFF_u64 as i64, 44), 45); + assert_eq!(bit_util::select(-1, 1), 0); + // assert_eq!(bit_util::select(0x8000000000000001_u64 as i64, 2), 63); + // assert_eq!(bit_util::select(0xF000000000000001_u64 as i64, 61), 60); + } +} diff --git a/src/core/util/context.rs b/src/core/util/context.rs index bdd577e..86004eb 100644 --- a/src/core/util/context.rs +++ b/src/core/util/context.rs @@ -73,7 +73,7 @@ macro_rules! define_typed_getter { pub fn set_binary(&mut self, key: $type, value: Vec) -> Option { self.set(key, VariantValue::Binary(value)) } - } + }; } #[derive(Clone, Debug, PartialEq)] diff --git a/src/core/util/doc_id_set.rs b/src/core/util/doc_id_set.rs index 512d88f..61acce8 100644 --- a/src/core/util/doc_id_set.rs +++ b/src/core/util/doc_id_set.rs @@ -15,7 +15,10 @@ use error::Result; use core::search::{DocIdSet, DocIterator, NO_MORE_DOCS}; use core::util::bit_set::{FixedBitSet, ImmutableBitSet}; +use core::util::packed::{EliasFanoDecoder, EliasFanoEncoder, NO_MORE_VALUES}; use core::util::DocId; +use error::ErrorKind::*; +use std::borrow::Cow; use std::sync::Arc; pub struct BitDocIdSet { @@ -356,6 +359,105 @@ impl DocIterator for NotDocIterator { } } +#[derive(Debug)] +pub struct EliasFanoDocIdSet { + ef_encoder: Arc, +} + +impl EliasFanoDocIdSet { + pub fn new(num_values: i64, upper_bound: i64) -> Result { + Ok(Self { + ef_encoder: Arc::new(EliasFanoEncoder::get_encoder(num_values, upper_bound)?), + }) + } + + pub fn bytes_used(&self) -> i32 { + (self.ef_encoder.upper_longs.len() + self.ef_encoder.upper_longs.len()) as i32 * 8 + } + + pub fn sufficiently_smaller_than_bit_set(num_values: i64, upper_bound: i64) -> bool { + EliasFanoEncoder::sufficiently_smaller_than_bit_set(num_values, upper_bound) + } + + pub fn encode_from_disi(&mut self, mut disi: impl DocIterator) -> Result<()> { + let encoder = unsafe { + &mut *(self.ef_encoder.as_ref() as *const EliasFanoEncoder as *mut EliasFanoEncoder) + }; + while self.ef_encoder.num_encoded < self.ef_encoder.num_values { + let x = disi.next()?; + if x == NO_MORE_DOCS { + bail!(IllegalArgument(format!( + "disi has {} docs, but at least {} are required.", + self.ef_encoder.num_encoded, self.ef_encoder.num_values + ))); + } + encoder.encode_next(x as i64)?; + } + Ok(()) + } +} + +impl DocIdSet for EliasFanoDocIdSet { + type Iter = EliasFanoDocIdSetIterator; + + fn iterator(&self) -> Result> { + if self.ef_encoder.last_encoded >= NO_MORE_DOCS as i64 { + bail!(UnsupportedOperation(Cow::Owned(format!( + "Highest encoded value too high for NO_MORE_DOCS: {}", + self.ef_encoder.last_encoded + )))); + } + Ok(Some(EliasFanoDocIdSetIterator::new( + self.ef_encoder.clone(), + ))) + } +} + +pub struct EliasFanoDocIdSetIterator { + ef_decoder: EliasFanoDecoder, + cur_docid: i32, +} + +impl EliasFanoDocIdSetIterator { + pub fn new(ef_encoder: Arc) -> Self { + Self { + ef_decoder: EliasFanoDecoder::new(ef_encoder), + cur_docid: -1, + } + } + + fn set_cur_doc_id(&mut self, value: i64) -> i32 { + self.cur_docid = if value == NO_MORE_VALUES { + NO_MORE_DOCS + } else { + value as i32 + }; + self.cur_docid + } +} + +impl DocIterator for EliasFanoDocIdSetIterator { + fn doc_id(&self) -> i32 { + self.cur_docid + } + + fn next(&mut self) -> Result { + let v = self.ef_decoder.next_value(); + Ok(self.set_cur_doc_id(v)) + // Ok(self.set_cur_doc_id(self.ef_decoder.next_value())) + } + + fn advance(&mut self, target: i32) -> Result { + let v = self.ef_decoder.advance_to_value(target as i64); + Ok(self.set_cur_doc_id(v)) + // Ok(self.set_cur_doc_id(self.ef_decoder.advance_to_value(target as i64))) + } + + fn cost(&self) -> usize { + self.ef_decoder.num_encoded() as usize + } +} + pub enum DocIdSetDocIterEnum { ShortArray(ShortArrayDocIterator), IntArray(IntArrayDocIterator), @@ -452,3 +554,83 @@ impl DocIterator for DocIdSetDocIterEnum { } } } + +#[cfg(test)] +mod tests { + use super::*; + use error::Result; + #[test] + fn ef_doc_id_set() -> Result<()> { + let doc_ids = IntArrayDocIdSet::new(vec![2, 3, 5, 7, 11, 13, 24, NO_MORE_DOCS], 7) + .iterator()? + .unwrap(); + let mut efs = EliasFanoDocIdSet::new(7, 24)?; + efs.encode_from_disi(doc_ids)?; + let mut docs = efs.iterator()?.unwrap(); + // assert_eq!(docs.next().unwrap(), 2); + // assert_eq!(docs.next().unwrap(), 3); + // assert_eq!(docs.next().unwrap(), 5); + // assert_eq!(docs.next().unwrap(), 7); + // assert_eq!(docs.next().unwrap(), 11); + // assert_eq!(docs.next().unwrap(), 13); + // assert_eq!(docs.next().unwrap(), 24); + // assert_eq!(docs.next().unwrap(), NO_MORE_DOCS); + assert_eq!(docs.advance(6).unwrap(), 7); + assert_eq!(docs.advance(14).unwrap(), 24); + // assert_eq!(docs.advance(25).unwrap(), NO_MORE_DOCS); + Ok(()) + } + + #[test] + fn ef_large() -> Result<()> { + const NUM: usize = 256; + let mut ds = vec![NO_MORE_DOCS; NUM]; + for i in 0..ds.len() - 1 { + ds[i] = i as i32; + } + + let length = ds.len(); + let upper_bound = ds[length - 2]; + let doc_ids = IntArrayDocIdSet::new(ds, length - 1).iterator()?.unwrap(); + let mut efs = EliasFanoDocIdSet::new((length - 1) as i64, upper_bound as i64)?; + efs.encode_from_disi(doc_ids)?; + for i in &efs.ef_encoder.upper_longs { + println!("WJJ->{}", i); + } + let mut docs = efs.iterator()?.unwrap(); + // assert_eq!(docs.advance(130).unwrap(), 130); + // println!("efs: {:?}", efs); + for i in 0..255 { + assert_eq!(docs.next().unwrap(), i); + } + // assert_eq!(docs.advance(upper_bound).unwrap(), upper_bound); + Ok(()) + } + + #[test] + fn ef_test() -> Result<()> { + const NUM: usize = 5e7 as usize; + let mut ds = vec![NO_MORE_DOCS; NUM]; + for i in 0..ds.len() - 1 { + ds[i] = i as i32 * 10; + } + + let length = ds.len(); + let upper_bound = ds[length - 2]; + let doc_ids = IntArrayDocIdSet::new(ds, length - 1).iterator()?.unwrap(); + let mut efs = EliasFanoDocIdSet::new((length - 1) as i64, upper_bound as i64)?; + efs.encode_from_disi(doc_ids)?; + // for i in &efs.ef_encoder.upper_longs { + // println!("WJJ->{}", i); + // } + let mut docs = efs.iterator()?.unwrap(); + // assert_eq!(docs.advance(130).unwrap(), 130); + // println!("efs: {:?}", efs); + // for i in 0..NUM - 1 { + // assert_eq!(docs.next().unwrap(), i as i32 * 10); + // } + // assert_eq!(docs.advance(499999980).unwrap(), 499999980); + assert_eq!(docs.advance(25010008).unwrap(), 25010010); + Ok(()) + } +} diff --git a/src/core/util/packed/elias_fano_decoder.rs b/src/core/util/packed/elias_fano_decoder.rs new file mode 100644 index 0000000..a056b17 --- /dev/null +++ b/src/core/util/packed/elias_fano_decoder.rs @@ -0,0 +1,420 @@ +use core::util::bit_util::{self, UnsignedShift, LOG2_LONG_SIZE, LONG_SIZE, LONG_SIZE_32}; +use core::util::packed::EliasFanoEncoder; +use error::ErrorKind::*; +use error::Result; +use std::sync::Arc; + +pub const NO_MORE_VALUES: i64 = -1; + +#[derive(Debug)] +pub struct EliasFanoDecoder { + ef_encoder: Arc, + // EF编码的整数个数 + num_encoded: i64, + // 当前指向第几个数的索引下标,有效值从0开始,初始值-1 + ef_index: i64, + // high bits 当前扫过的bits计数,减去ef_index个1就是高位base + // high bits index, 从0开始,初始值-1 + set_bit_for_index: i64, + // EF first level index & mask value + num_index_entries: i64, + index_mask: i64, + + // upper longs array, current long + cur_high_long: i64, +} + +impl EliasFanoDecoder { + pub fn new(ef_encoder: Arc) -> Self { + Self { + ef_encoder: ef_encoder.clone(), + num_encoded: ef_encoder.num_encoded, + ef_index: -1, + set_bit_for_index: -1, + num_index_entries: ef_encoder.num_index_entries, + index_mask: (1i64 << ef_encoder.n_index_entry_bits as i64) - 1, + cur_high_long: 0, + } + } + + pub fn refresh(&mut self) { + *self = Self::new(self.ef_encoder.clone()); + } + + pub fn get_encoder(&self) -> Arc { + self.ef_encoder.clone() + } + + pub fn num_encoded(&self) -> i64 { + self.num_encoded + } + + /// 当前访问第几个数,从0开始 + pub fn current_index(&self) -> Result { + if self.ef_index < 0 { + bail!(IllegalState(format!("index before sequence"))); + } + if self.ef_index >= self.num_encoded { + bail!(IllegalState(format!("index after sequence"))); + } + Ok(self.ef_index) + } + + pub fn current_value(&self) -> i64 { + self.combine_high_low_values(self.current_high_value(), self.current_low_value()) + } + + /// 通过数1/0的个数计算第几个base, 000/001/010/011/100等这种称作base + pub fn current_high_value(&self) -> i64 { + self.set_bit_for_index - self.ef_index + } + + pub fn current_low_value(&self) -> i64 { + assert!((self.ef_index >= 0) && (self.ef_index < self.num_encoded)); + Self::unpack_value( + &self.ef_encoder.lower_longs, + self.ef_encoder.num_low_bits, + self.ef_index, + self.ef_encoder.lower_bits_mask, + ) + } + + fn combine_high_low_values(&self, high_value: i64, low_value: i64) -> i64 { + (high_value << self.ef_encoder.num_low_bits as i64) | low_value + } + /// 等bits数组中取对应索引的值 + fn unpack_value(long_array: &Vec, num_bits: i32, pack_index: i64, bits_mask: i64) -> i64 { + if num_bits == 0 { + return 0; + } + let bit_pos = pack_index * num_bits as i64; + let index = bit_pos.unsigned_shift(LOG2_LONG_SIZE as usize) as usize; + let bit_pos_at_index = (bit_pos & LONG_SIZE - 1) as i64; + let mut value = long_array[index].unsigned_shift(bit_pos_at_index as usize); + if bit_pos_at_index + num_bits as i64 > LONG_SIZE { + value |= long_array[index + 1] << (LONG_SIZE - bit_pos_at_index); + } + value & bits_mask + } + + /// reset index before the first integer + pub fn to_before_sequence(&mut self) { + self.ef_index = -1; + self.set_bit_for_index = -1; + } + + /// 当前current high long 需要右移的位数 + fn get_current_right_shift(&self) -> i32 { + (self.set_bit_for_index & LONG_SIZE - 1) as i32 + } + + /// ef_index & set_bit_for_index increment, low bits to next, but high bits maybe not arrived + fn to_after_current_high_bit(&mut self) -> bool { + self.ef_index += 1; + if self.ef_index >= self.num_encoded { + return false; + } + self.set_bit_for_index += 1; + let high_index = self + .set_bit_for_index + .unsigned_shift(LOG2_LONG_SIZE as usize) as usize; + self.cur_high_long = self.ef_encoder.upper_longs[high_index] + .unsigned_shift(self.get_current_right_shift() as usize); + true + } + + /// move to next upper long,advance set_bit_for_index + fn to_next_high_long(&mut self) { + self.set_bit_for_index += LONG_SIZE - (self.set_bit_for_index & LONG_SIZE - 1); + let high_index = self + .set_bit_for_index + .unsigned_shift(LOG2_LONG_SIZE as usize) as usize; + self.cur_high_long = self.ef_encoder.upper_longs[high_index]; + } + /// advance to next 1 bit in upper longs + /// must be used after to_after_current_high_bit + fn to_next_high_value(&mut self) { + while self.cur_high_long == 0 { + self.to_next_high_long(); + } + self.set_bit_for_index += self.cur_high_long.trailing_zeros() as i64; + } + + fn next_high_value(&mut self) -> i64 { + self.to_next_high_value(); + self.current_high_value() + } + /// ef_index + 1 get the next low bits + /// set_bit_for_index advance to next 1 bit, then get high bits + /// by set_bit_for_index - ef_index + pub fn next_value(&mut self) -> i64 { + if !self.to_after_current_high_bit() { + return NO_MORE_VALUES; + } + let high_value = self.next_high_value(); + return self.combine_high_low_values(high_value, self.current_low_value()); + } + /// advance the index to the [index]th integer + pub fn advance_to_index(&mut self, index: i64) -> bool { + assert!(index > self.ef_index); + if index >= self.num_encoded { + self.ef_index = self.num_encoded; + return false; + } + if !self.to_after_current_high_bit() { + assert!(false); + } + let mut cur_set_bits = self.cur_high_long.count_ones() as i32; + while (self.ef_index + cur_set_bits as i64) < index { + self.ef_index += cur_set_bits as i64; + self.to_next_high_long(); + cur_set_bits = self.cur_high_long.count_ones() as i32; + } + while self.ef_index < index { + if !self.to_after_current_high_bit() { + assert!(false); + } + self.to_next_high_value(); + } + true + } + + pub fn advance_to_value(&mut self, target: i64) -> i64 { + // equals to to_after_current_high_bit + self.ef_index += 1; + if self.ef_index >= self.num_encoded { + return NO_MORE_VALUES; + } + self.set_bit_for_index += 1; + + let mut high_index = self + .set_bit_for_index + .unsigned_shift(LOG2_LONG_SIZE as usize) as usize; + let mut upper_long = self.ef_encoder.upper_longs[high_index]; + self.cur_high_long = + upper_long.unsigned_shift((self.set_bit_for_index & LONG_SIZE - 1) as usize); + + let high_target = target.unsigned_shift(self.ef_encoder.num_low_bits as usize); + let mut index_entry_index = (high_target / self.ef_encoder.index_interval) - 1; + // 跳过部分数据 + if index_entry_index >= 0 { + if index_entry_index >= self.num_index_entries { + index_entry_index = self.num_index_entries - 1; + } + let index_high_value = (index_entry_index + 1) * self.ef_encoder.index_interval; + assert!(index_high_value <= high_target); + if index_high_value > (self.set_bit_for_index - self.ef_index) { + self.set_bit_for_index = Self::unpack_value( + &self.ef_encoder.upper_zero_bit_position_index, + self.ef_encoder.n_index_entry_bits, + index_entry_index, + self.index_mask, + ); + self.ef_index = self.set_bit_for_index - index_high_value; + high_index = self + .set_bit_for_index + .unsigned_shift(LOG2_LONG_SIZE as usize) as usize; + upper_long = self.ef_encoder.upper_longs[high_index]; + self.cur_high_long = + upper_long.unsigned_shift((self.set_bit_for_index & LONG_SIZE - 1) as usize); + } + assert!(self.ef_index < self.num_encoded); + } + + let mut cur_set_bits = self.cur_high_long.count_ones() as i32; + let mut cur_clear_bits = + LONG_SIZE_32 - cur_set_bits - (self.set_bit_for_index & LONG_SIZE - 1) as i32; + + // 定位到最终upper long + while (self.set_bit_for_index - self.ef_index + cur_clear_bits as i64) < high_target { + self.ef_index += cur_set_bits as i64; + if self.ef_index >= self.num_encoded { + return NO_MORE_VALUES; + } + self.set_bit_for_index += LONG_SIZE - (self.set_bit_for_index & LONG_SIZE - 1); + assert_eq!( + high_index + 1, + (self + .set_bit_for_index + .unsigned_shift(LOG2_LONG_SIZE as usize) as usize) + ); + high_index += 1; + // upper_long += self.ef_encoder.upper_longs[high_index]; + upper_long = self.ef_encoder.upper_longs[high_index]; + self.cur_high_long = upper_long; + cur_set_bits = self.cur_high_long.count_ones() as i32; + cur_clear_bits = LONG_SIZE_32 - cur_set_bits; + } + + while self.cur_high_long == 0 { + self.set_bit_for_index += LONG_SIZE - (self.set_bit_for_index & LONG_SIZE - 1); + assert_eq!( + high_index + 1, + (self + .set_bit_for_index + .unsigned_shift(LOG2_LONG_SIZE as usize) as usize) + ); + high_index += 1; + // if high_index >= self.ef_encoder.upper_longs.len() {} + upper_long = self.ef_encoder.upper_longs[high_index]; + self.cur_high_long = upper_long; + } + + let rank = (high_target - (self.set_bit_for_index - self.ef_index)) as i32; + assert!(rank <= LONG_SIZE_32); + if rank >= 1 { + let inv_cur_high_long = !self.cur_high_long; + let clear_bit_for_value = if rank <= 8 { + bit_util::select_naive(inv_cur_high_long, rank) + } else { + bit_util::select(inv_cur_high_long, rank) + }; + assert!(clear_bit_for_value >= 0); + assert!(clear_bit_for_value <= LONG_SIZE_32 - 1); + self.set_bit_for_index += (clear_bit_for_value + 1) as i64; + let one_bits_before_clear_bit = clear_bit_for_value - rank + 1; + self.ef_index += one_bits_before_clear_bit as i64; + if self.ef_index >= self.num_encoded { + return NO_MORE_VALUES; + } + + if (self.set_bit_for_index & LONG_SIZE - 1) == 0 { + assert_eq!( + high_index + 1, + self.set_bit_for_index + .unsigned_shift(LOG2_LONG_SIZE as usize) as usize + ); + high_index += 1; + upper_long = self.ef_encoder.upper_longs[high_index]; + self.cur_high_long = upper_long; + } else { + assert_eq!( + high_index, + self.set_bit_for_index + .unsigned_shift(LOG2_LONG_SIZE as usize) as usize + ); + self.cur_high_long = + upper_long.unsigned_shift((self.set_bit_for_index & LONG_SIZE - 1) as usize); + } + + while self.cur_high_long == 0 { + self.set_bit_for_index += LONG_SIZE - (self.set_bit_for_index & LONG_SIZE - 1); + assert_eq!( + high_index + 1, + self.set_bit_for_index + .unsigned_shift(LOG2_LONG_SIZE as usize) as usize + ); + high_index += 1; + upper_long = self.ef_encoder.upper_longs[high_index]; + self.cur_high_long = upper_long; + } + } + // 当前定位到的current_value + self.set_bit_for_index += self.cur_high_long.trailing_zeros() as i64; + assert!(self.set_bit_for_index - self.ef_index >= high_target); + let mut current_value = self.combine_high_low_values( + self.set_bit_for_index - self.ef_index, + self.current_low_value(), + ); + // 线性扫描下一个值 + while current_value < target { + current_value = self.next_value(); + if current_value == NO_MORE_VALUES { + return NO_MORE_VALUES; + } + } + current_value + } + + pub fn to_after_sequence(&mut self) { + self.ef_index = self.num_encoded; + self.set_bit_for_index = self + .ef_encoder + .last_encoded + .unsigned_shift(self.ef_encoder.num_low_bits as usize) + + self.num_encoded; + } + /// reversed operator for next_value + pub fn previous_value(&mut self) -> i64 { + if !self.to_before_current_high_bit() { + return NO_MORE_VALUES; + } + let high_value = self.previous_high_value(); + self.combine_high_low_values(high_value, self.current_low_value()) + } + /// reversed operator for advance_to_value + pub fn back_to_value(&mut self, target: i64) -> i64 { + if !self.to_before_current_high_bit() { + return NO_MORE_VALUES; + } + let high_target = target.unsigned_shift(self.ef_encoder.num_low_bits as usize); + let high_value = self.back_to_high_value(high_target); + if high_value == NO_MORE_VALUES { + return NO_MORE_VALUES; + } + let mut current_value = self.combine_high_low_values(high_value, self.current_low_value()); + while current_value > target { + current_value = self.previous_value(); + if current_value == NO_MORE_VALUES { + return NO_MORE_VALUES; + } + } + current_value + } + + fn get_current_left_shift(&self) -> i32 { + LONG_SIZE_32 - 1 - (self.set_bit_for_index & LONG_SIZE - 1) as i32 + } + + fn to_before_current_high_bit(&mut self) -> bool { + self.ef_index -= 1; + if self.ef_index < 0 { + return false; + } + self.set_bit_for_index -= 1; + let high_index = self + .set_bit_for_index + .unsigned_shift(LOG2_LONG_SIZE as usize) as usize; + self.cur_high_long = + self.ef_encoder.upper_longs[high_index] << self.get_current_left_shift() as i64; + true + } + + fn to_previous_high_long(&mut self) { + self.set_bit_for_index -= (self.set_bit_for_index & LONG_SIZE - 1) + 1; + let high_index = self + .set_bit_for_index + .unsigned_shift(LOG2_LONG_SIZE as usize) as usize; + self.cur_high_long = self.ef_encoder.upper_longs[high_index]; + } + + fn previous_high_value(&mut self) -> i64 { + while self.cur_high_long == 0 { + self.to_previous_high_long(); + } + self.set_bit_for_index -= self.cur_high_long.leading_zeros() as i64; + self.current_high_value() + } + + fn back_to_high_value(&mut self, high_target: i64) -> i64 { + let mut cur_set_bits = self.cur_high_long.count_ones() as i32; + let mut cur_clear_bits = LONG_SIZE_32 - cur_set_bits - self.get_current_left_shift(); + while self.current_high_value() - cur_clear_bits as i64 > high_target { + self.ef_index -= cur_set_bits as i64; + if self.ef_index < 0 { + return NO_MORE_VALUES; + } + self.to_previous_high_long(); + cur_set_bits = self.cur_high_long.count_ones() as i32; + cur_clear_bits = LONG_SIZE_32 - cur_set_bits; + } + let mut high_value = self.previous_high_value(); + while high_value > high_target { + if !self.to_before_current_high_bit() { + return NO_MORE_VALUES; + } + high_value = self.previous_high_value(); + } + high_value + } +} diff --git a/src/core/util/packed/elias_fano_encoder.rs b/src/core/util/packed/elias_fano_encoder.rs new file mode 100644 index 0000000..f595edb --- /dev/null +++ b/src/core/util/packed/elias_fano_encoder.rs @@ -0,0 +1,436 @@ +use core::codec::postings::{EncodeType, ForUtil}; +use core::store::io::{IndexInput, IndexOutput}; +use core::util::bit_util::*; +use error::ErrorKind::*; +use error::Result; +use std::ptr::{slice_from_raw_parts, slice_from_raw_parts_mut}; + +/// The default index interval for zero upper bits. +pub const DEFAULT_INDEX_INTERVAL: i64 = 256; + +#[derive(Debug)] +pub struct EliasFanoEncoder { + pub num_values: i64, + upper_bound: i64, + pub num_low_bits: i32, + pub lower_bits_mask: i64, + pub upper_longs: Vec, + pub lower_longs: Vec, + + pub num_encoded: i64, + pub last_encoded: i64, + + pub num_index_entries: i64, + pub index_interval: i64, + // index entry bits num + pub n_index_entry_bits: i32, + /// upper_zero_bit_position_index[i] (filled using packValue) will contain the bit position + /// just after the zero bit ((i+1) * index_interval) in the upper bits. + pub upper_zero_bit_position_index: Vec, + current_entry_index: i64, // also indicates how many entries in the index are valid. +} + +impl EliasFanoEncoder { + pub fn new(num_values: i64, upper_bound: i64, index_interval: i64) -> Result { + if num_values < 0 { + bail!(IllegalArgument(format!( + "num_values should not be negative: {}", + num_values + ))); + } + if num_values > 0 && upper_bound < 0 { + bail!(IllegalArgument(format!( + "upper_bound should not be negative: {} when num_values > 0", + upper_bound + ))); + } + let upper_bound = if num_values > 0 { upper_bound } else { -1 }; + + // the number of lower bits + let mut num_low_bits = 0; + if num_values > 0 { + let low_bits_fac = upper_bound / num_values; + if low_bits_fac > 0 { + // different from lucene version + // num_low_bits = LONG_SIZE_32 - number_of_leading_zeros(low_bits_fac); + // floor(2_log(upper_bound / num_values)), default + // ceil(2_log(upper_bound / num_values - 1)) + num_low_bits = LONG_SIZE_32 - 1 - low_bits_fac.leading_zeros() as i32; + } + } + let lower_bits_mask = + i64::max_value().unsigned_shift((LONG_SIZE_32 - 1 - num_low_bits) as usize); + let num_longs_for_low_bits = Self::num_longs_for_bits(num_values * num_low_bits as i64); + if num_longs_for_low_bits > i32::max_value().into() { + bail!(IllegalArgument(format!( + "num_longs_for_low_bits too large to index a long array: {}", + num_longs_for_low_bits + ))); + } + let lower_longs = vec![0; num_longs_for_low_bits as usize]; + + // high bits + let mut num_high_bits_clear = if upper_bound > 0 { upper_bound } else { 0 }; + num_high_bits_clear = num_high_bits_clear.unsigned_shift(num_low_bits as usize); + assert!(num_high_bits_clear <= 2 * num_values); + let num_high_bits_set = num_values; + + // Todo: 感觉这里少计算了 + let num_longs_for_high_bits = + Self::num_longs_for_bits(num_high_bits_clear + num_high_bits_set); + if num_longs_for_high_bits > i32::max_value() as i64 { + bail!(IllegalArgument(format!( + "num_longs_for_high_bits too large to index a long array: {}", + num_longs_for_high_bits + ))); + } + let upper_longs = vec![0; num_longs_for_high_bits as usize]; + if index_interval < 2 { + bail!(IllegalArgument(format!( + "index_interval should at least 2: {}", + index_interval + ))); + } + + // high bits的分区索引 + let max_high_value = upper_bound.unsigned_shift(num_low_bits as usize); + let n_index_entries = max_high_value / index_interval; + let num_index_entries = if n_index_entries >= 0 { + n_index_entries + } else { + 0 + }; + // Todo max value & first index + let max_index_entry = max_high_value + num_values - 1; + let n_index_entry_bits = if max_index_entry <= 0 { + 0 + } else { + LONG_SIZE_32 - max_index_entry.leading_zeros() as i32 + }; + let num_longs_for_index_bits = + Self::num_longs_for_bits(num_index_entries * n_index_entry_bits as i64); + if num_longs_for_index_bits > i32::max_value() as i64 { + bail!(IllegalArgument(format!( + "num_longs_for_index_bits too large to index a long array: {}", + num_longs_for_index_bits + ))); + } + Ok(Self { + num_values, + upper_bound, + num_low_bits, + lower_bits_mask, + upper_longs, + lower_longs, + num_encoded: 0, + last_encoded: 0, + num_index_entries, + index_interval, + n_index_entry_bits, + upper_zero_bit_position_index: vec![0; num_longs_for_index_bits as usize], + current_entry_index: 0, + }) + } + + pub fn rebuild_not_with_check(&mut self, num_values: i64, upper_bound: i64) -> Result<()> { + self.num_values = num_values; + self.upper_bound = upper_bound; + self.num_encoded = num_values; + self.last_encoded = upper_bound; + // low bits num & mask + self.num_low_bits = if num_values > 0 { + let low_bits_fac = upper_bound / num_values; + if low_bits_fac > 0 { + LONG_SIZE_32 - 1 - low_bits_fac.leading_zeros() as i32 + } else { + 0 + } + } else { + 0 + }; + self.lower_bits_mask = + i64::max_value().unsigned_shift((LONG_SIZE_32 - 1 - self.num_low_bits) as usize); + + // low bits + self.lower_longs.resize( + Self::num_longs_for_bits(num_values * self.num_low_bits as i64) as usize, + 0, + ); + // high bits + self.upper_longs.resize( + Self::num_longs_for_bits( + upper_bound.unsigned_shift(self.num_low_bits as usize) + num_values, + ) as usize, + 0, + ); + + // index + let max_high_value = upper_bound.unsigned_shift(self.num_low_bits as usize); + let n_index_entries = max_high_value / self.index_interval; + let num_index_entries = if n_index_entries >= 0 { + n_index_entries + } else { + 0 + }; + let max_index_entry = max_high_value + num_values - 1; + let n_index_entry_bits = if max_index_entry <= 0 { + 0 + } else { + LONG_SIZE_32 - max_index_entry.leading_zeros() as i32 + }; + self.upper_zero_bit_position_index.resize( + Self::num_longs_for_bits(num_index_entries * n_index_entry_bits as i64) as usize, + 0, + ); + self.n_index_entry_bits = n_index_entry_bits; + self.num_index_entries = num_index_entries; + self.current_entry_index = 0; + Ok(()) + } + + pub fn get_encoder(num_values: i64, upper_bound: i64) -> Result { + Self::new(num_values, upper_bound, DEFAULT_INDEX_INTERVAL) + } + + #[inline] + pub fn encode_size(&self) -> i32 { + ((self.upper_longs.len() + + self.lower_longs.len() + + self.upper_zero_bit_position_index.len()) + << 3) as i32 + } + + pub fn encode_next(&mut self, x: i64) -> Result<()> { + if self.num_encoded >= self.num_values { + bail!(IllegalState(format!( + "encode_next called more than {} times.", + self.num_values + ))); + } + if self.last_encoded > x { + bail!(IllegalArgument(format!( + "{} smaller than previous {}", + x, self.last_encoded + ))); + } + if x > self.upper_bound { + bail!(IllegalArgument(format!( + "{} larger than upperBound {}", + x, self.upper_bound + ))); + } + let high_value = x.unsigned_shift(self.num_low_bits as usize); + self.encode_upper_bits(high_value); + self.encode_lower_bits(x & self.lower_bits_mask); + self.last_encoded = x; + let mut index_value = (self.current_entry_index + 1) * self.index_interval; + while index_value <= high_value { + let after_zero_bit_position = index_value + self.num_encoded; + Self::pack_value( + after_zero_bit_position, + &mut self.upper_zero_bit_position_index, + self.n_index_entry_bits, + self.current_entry_index, + ); + self.current_entry_index += 1; + index_value += self.index_interval; + } + self.num_encoded += 1; + Ok(()) + } + + pub fn serialize(&mut self, out: &mut impl IndexOutput) -> Result<()> { + out.write_byte(ForUtil::encode_type_to_code(EncodeType::EF))?; + out.write_vlong(self.upper_bound)?; + Self::write_data(&self.upper_longs, out)?; + Self::write_data(&self.lower_longs, out)?; + Self::write_data(&self.upper_zero_bit_position_index, out)?; + Ok(()) + } + + pub fn deserialize(&mut self, encoded_data: &[u8]) -> Result<()> { + self.num_encoded = self.num_values; + self.last_encoded = self.upper_bound; + let mut index = 0; + Self::read_data(&mut self.upper_longs, encoded_data, &mut index); + Self::read_data(&mut self.lower_longs, encoded_data, &mut index); + Self::read_data( + &mut self.upper_zero_bit_position_index, + encoded_data, + &mut index, + ); + + Ok(()) + } + + pub fn deserialize2(&mut self, input: &mut dyn IndexInput) -> Result<()> { + self.num_encoded = self.num_values; + self.last_encoded = self.upper_bound; + Self::read_data2(&mut self.upper_longs, input)?; + Self::read_data2(&mut self.lower_longs, input)?; + Self::read_data2(&mut self.upper_zero_bit_position_index, input)?; + Ok(()) + } + + #[inline] + pub fn sufficiently_smaller_than_bit_set(num_values: i64, upper_bound: i64) -> bool { + return (upper_bound > (4 * LONG_SIZE)) && (upper_bound / 7) > num_values; + } + + // pub get_decoder(&self) -> Result { + // EliasFanoDecoder::new(self) + // } + + pub fn get_lower_bits(&self) -> &Vec { + &self.lower_longs + } + + pub fn get_upper_bits(&self) -> &Vec { + &self.upper_longs + } + + pub fn get_index_bits(&self) -> &Vec { + &self.upper_zero_bit_position_index + } + #[inline] + fn num_longs_for_bits(n: i64) -> i64 { + assert!(n >= 0); + (n + LONG_SIZE - 1).unsigned_shift(LOG2_LONG_SIZE as usize) + } + #[inline] + fn encode_upper_bits(&mut self, high_value: i64) { + let next_high_bit_num = self.num_encoded + high_value; + self.upper_longs[next_high_bit_num.unsigned_shift(LOG2_LONG_SIZE as usize) as usize] |= + 1_i64 << (next_high_bit_num & LONG_SIZE - 1) + } + #[inline] + fn encode_lower_bits(&mut self, low_value: i64) { + Self::pack_value( + low_value, + &mut self.lower_longs, + self.num_low_bits, + self.num_encoded, + ); + } + + /// 用Vec存储固定长度的bits array + /// value: 待存储值,取从右到左的num_bits个位 + /// long_array: 用于存储的Vec + /// num_bits: 固定bits的个数 + /// pack_index: 已经存储的值的个数 + #[inline] + fn pack_value(value: i64, long_array: &mut Vec, num_bits: i32, pack_index: i64) { + if num_bits != 0 { + let bit_pos = num_bits as i64 * pack_index; + let index = bit_pos.unsigned_shift(LOG2_LONG_SIZE as usize) as usize; + let bit_pos_at_index = (bit_pos & LONG_SIZE - 1) as i32; + long_array[index] |= value << bit_pos_at_index as i64; + if (bit_pos_at_index + num_bits) > LONG_SIZE_32 { + long_array[index + 1] = + value.unsigned_shift((LONG_SIZE_32 - bit_pos_at_index) as usize); + } + } + } + + pub fn write_data(data: &Vec, out: &mut impl IndexOutput) -> Result<()> { + if !data.is_empty() { + let ptr = data.as_ptr() as *const u8; + let length = data.len() << 3; + let data = unsafe { &*slice_from_raw_parts(ptr, length) }; + out.write_bytes(data, 0, length)?; + } + Ok(()) + } + + pub fn read_data(data: &mut Vec, encoded_data: &[u8], index: &mut usize) { + let length = data.len(); + if length > 0 { + data.clear(); + let ptr = encoded_data[*index..].as_ptr() as *mut i64; + + let v = unsafe { Vec::from_raw_parts(ptr, length, length) }; + let _ = v.iter().map(|&x| data.push(x)).collect::<()>(); + v.into_raw_parts(); + *index += length << 3; + } + } + + pub fn read_data2(buf: &mut Vec, input: &mut dyn IndexInput) -> Result<()> { + if buf.len() > 0 { + let ptr = buf.as_mut_ptr() as *mut u8; + let new_buf = unsafe { &mut *slice_from_raw_parts_mut(ptr, buf.len() << 3) }; + input.read_exact(new_buf)?; + } + Ok(()) + } +} +// impl Drop for EliasFanoEncoder { +// fn drop(&mut self) { +// if self.upper_longs.len() > 0 { +// mem::take(&mut self.upper_longs).into_raw_parts(); +// } +// if self.lower_longs.len() > 0 { +// mem::take(&mut self.lower_longs).into_raw_parts(); +// } +// if self.upper_zero_bit_position_index.len() > 0 { +// mem::take(&mut self.upper_zero_bit_position_index).into_raw_parts(); +// } +// } +// } + +#[cfg(test)] +mod tests { + use core::util::packed::EliasFanoEncoder; + + #[test] + fn num_longs_for_bits() { + assert_eq!(EliasFanoEncoder::num_longs_for_bits(5), 1); + assert_eq!(EliasFanoEncoder::num_longs_for_bits(31), 1); + assert_eq!(EliasFanoEncoder::num_longs_for_bits(32), 1); + assert_eq!(EliasFanoEncoder::num_longs_for_bits(33), 1); + assert_eq!(EliasFanoEncoder::num_longs_for_bits(65), 2); + assert_eq!(EliasFanoEncoder::num_longs_for_bits(128), 2); + assert_eq!(EliasFanoEncoder::num_longs_for_bits(129), 3); + } + #[test] + fn get_encoder() { + let efe = EliasFanoEncoder::get_encoder(128, 510901); + println!("efe: {:#?}", efe); + } + + #[test] + fn pack_value() { + let mut lv = vec![0_i64; 2]; + EliasFanoEncoder::pack_value(2, &mut lv, 2, 31); + println!("wjj: {}", lv[0]); + assert_eq!(lv[0], 0x8000000000000000_u64 as i64); + lv[0] = 0; + EliasFanoEncoder::pack_value(0b11111_i64, &mut lv, 5, 12); + println! {"wjj: {:?}", lv}; + assert_eq!(lv[0], 0xF000000000000000_u64 as i64); + assert_eq!(lv[1], 1_i64); + } + + #[test] + fn encode_upper() { + let mut ef = EliasFanoEncoder::new(7, 24, 256).unwrap(); + println!("encoder: {:?}", ef); + ef.encode_upper_bits(0); + ef.num_encoded += 1; + assert_eq!(ef.upper_longs[0], 1_i64); + println!("encoder: {:?}, num: {}", ef.upper_longs[0], ef.num_encoded); + ef.encode_upper_bits(0); + ef.num_encoded += 1; + assert_eq!(ef.upper_longs[0], 3_i64); + println!("encoder: {:?}, num: {}", ef.upper_longs[0], ef.num_encoded); + ef.encode_upper_bits(1); + ef.num_encoded += 1; + assert_eq!(ef.upper_longs[0], 11_i64); + ef.encode_upper_bits(1); + ef.num_encoded += 1; + assert_eq!(ef.upper_longs[0], 27_i64); + ef.encode_upper_bits(2); + ef.num_encoded += 1; + assert_eq!(ef.upper_longs[0], 91_i64); + } +} diff --git a/src/core/util/packed/mod.rs b/src/core/util/packed/mod.rs index c2b0992..ef5a6c8 100644 --- a/src/core/util/packed/mod.rs +++ b/src/core/util/packed/mod.rs @@ -54,3 +54,11 @@ pub use self::packed_long_values::*; mod block_packed_writer; pub use self::block_packed_writer::*; + +mod elias_fano_encoder; + +pub use self::elias_fano_encoder::*; + +mod elias_fano_decoder; + +pub use self::elias_fano_decoder::*; diff --git a/src/lib.rs b/src/lib.rs index 7b59fff..69f8519 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -25,6 +25,10 @@ #![feature(fn_traits)] #![feature(maybe_uninit_ref)] #![feature(maybe_uninit_extra)] +#![feature(in_band_lifetimes)] +#![feature(vec_into_raw_parts)] +#![feature(slice_from_raw_parts)] +#![feature(core_intrinsics)] #[macro_use] extern crate error_chain; From 82bcc2a1a69489898af382f7962ecf0081e94862 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Fri, 6 Mar 2020 11:49:16 +0800 Subject: [PATCH 08/49] use clear_all of memset --- src/core/codec/postings/posting_reader.rs | 7 +++---- src/core/codec/postings/posting_writer.rs | 5 ++--- src/core/index/reader/segment_reader.rs | 14 ++++++-------- src/core/util/bit_set.rs | 5 ++++- 4 files changed, 15 insertions(+), 16 deletions(-) diff --git a/src/core/codec/postings/posting_reader.rs b/src/core/codec/postings/posting_reader.rs index 6ddf3d2..681a11d 100644 --- a/src/core/codec/postings/posting_reader.rs +++ b/src/core/codec/postings/posting_reader.rs @@ -468,7 +468,6 @@ impl BlockDocIterator { self.ef_decoder = None; self.ef_base_doc = -1; self.ef_base_total = 0; - // self.doc_bits.clear_batch(0, self.doc_bits.len()); self.doc_bits.clear_all(); self.bits_index = 0; @@ -601,7 +600,7 @@ impl DocIterator for BlockDocIterator { } fn advance(&mut self, target: DocId) -> Result { - if unsafe {unlikely(target == NO_MORE_DOCS)} { + if unsafe { unlikely(target == NO_MORE_DOCS) } { self.doc = NO_MORE_DOCS; return Ok(self.doc); } @@ -1139,7 +1138,7 @@ impl DocIterator for BlockPostingIterator { } fn advance(&mut self, target: DocId) -> Result { - if unsafe {unlikely(target == NO_MORE_DOCS)} { + if unsafe { unlikely(target == NO_MORE_DOCS) } { self.doc = NO_MORE_DOCS; return Ok(self.doc); } @@ -1876,7 +1875,7 @@ impl DocIterator for EverythingIterator { } fn advance(&mut self, target: DocId) -> Result { - if unsafe {unlikely(target == NO_MORE_DOCS)} { + if unsafe { unlikely(target == NO_MORE_DOCS) } { self.doc = NO_MORE_DOCS; return Ok(self.doc); } diff --git a/src/core/codec/postings/posting_writer.rs b/src/core/codec/postings/posting_writer.rs index 6d01257..036dc8e 100644 --- a/src/core/codec/postings/posting_writer.rs +++ b/src/core/codec/postings/posting_writer.rs @@ -27,7 +27,7 @@ use core::search::{DocIterator, NO_MORE_DOCS}; use core::store::directory::Directory; use core::store::io::{DataOutput, IndexOutput}; use core::util::packed::COMPACT; -use core::util::{BitSet, Bits, DocId, FixedBitSet}; +use core::util::{BitSet, DocId, FixedBitSet}; use error::{ErrorKind, Result}; pub struct EfWriterMeta { @@ -52,8 +52,7 @@ impl EfWriterMeta { pub fn reset(&mut self) { self.ef_base_doc = -1; self.ef_upper_doc = 0; - self.bits.clear_batch(0, self.bits.len()); - // self.bits.clear_all(); + self.bits.clear_all(); } } diff --git a/src/core/index/reader/segment_reader.rs b/src/core/index/reader/segment_reader.rs index 2a063a5..61bae47 100644 --- a/src/core/index/reader/segment_reader.rs +++ b/src/core/index/reader/segment_reader.rs @@ -958,7 +958,6 @@ where v.insert(DocValuesProviderEnum::Numeric(Arc::clone(&cell))); cell.get() } - _ => bail!(IllegalArgument(format!( "non-dv-segment or non-exist or non-numeric field: {}", field @@ -987,7 +986,6 @@ where v.insert(DocValuesProviderEnum::Binary(Arc::clone(&dv))); dv.get() } - _ => bail!(IllegalArgument(format!( "non-dv-segment or non-exist or non-binary field: {}", field @@ -1005,7 +1003,7 @@ where Entry::Occupied(o) => match *o.get() { DocValuesProviderEnum::Sorted(ref dv) => dv.get(), _ => bail!(IllegalArgument(format!( - "non-binary dv found for field {}", + "non-sorted dv found for field {}", field ))), }, @@ -1017,7 +1015,7 @@ where dv.get() } _ => bail!(IllegalArgument(format!( - "non-dv-segment or non-exist or non-binary field: {}", + "non-dv-segment or non-exist or non-sorted field: {}", field ))), }, @@ -1036,7 +1034,7 @@ where Entry::Occupied(o) => match *o.get() { DocValuesProviderEnum::SortedNumeric(ref dv) => dv.get(), _ => bail!(IllegalArgument(format!( - "non-binary dv found for field {}", + "non-sorted_numeric dv found for field {}", field ))), }, @@ -1049,7 +1047,7 @@ where cell.get() } _ => bail!(IllegalArgument(format!( - "non-dv-segment or non-exist or non-binary field: {}", + "non-dv-segment or non-exist or non-sorted_numeric field: {}", field ))), }, @@ -1065,7 +1063,7 @@ where Entry::Occupied(o) => match *o.get() { DocValuesProviderEnum::SortedSet(ref dv) => dv.get(), _ => bail!(IllegalArgument(format!( - "non-binary dv found for field {}", + "non-sorted_set dv found for field {}", field ))), }, @@ -1079,7 +1077,7 @@ where } _ => bail!(IllegalArgument(format!( - "non-dv-segment or non-exist or non-binary field: {}", + "non-dv-segment or non-exist or non-sorted_set field: {}", field ))), }, diff --git a/src/core/util/bit_set.rs b/src/core/util/bit_set.rs index 23b0b50..e540dd0 100644 --- a/src/core/util/bit_set.rs +++ b/src/core/util/bit_set.rs @@ -18,6 +18,7 @@ use core::util::bit_util::{self, UnsignedShift}; use core::util::{Bits, BitsRef}; use error::{ErrorKind, Result}; +use std::intrinsics::volatile_set_memory; pub trait ImmutableBitSet: Bits { /// Return the number of bits that are set. @@ -267,7 +268,9 @@ impl FixedBitSet { #[inline] pub fn clear_all(&mut self) { - self.bits.iter_mut().for_each(|x| *x = 0i64); + unsafe { + volatile_set_memory(self.bits.as_mut_ptr(), 0, self.bits.len()); + } } /// Checks if the bits past numBits are clear. Some methods rely on this implicit From 6c73534419daef79f68d9d89a847ef893359a5ae Mon Sep 17 00:00:00 2001 From: wangjingjing Date: Mon, 9 Mar 2020 12:39:53 +0800 Subject: [PATCH 09/49] close ef writer flag --- src/core/codec/postings/posting_writer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/core/codec/postings/posting_writer.rs b/src/core/codec/postings/posting_writer.rs index 036dc8e..f3b7f46 100644 --- a/src/core/codec/postings/posting_writer.rs +++ b/src/core/codec/postings/posting_writer.rs @@ -43,7 +43,7 @@ impl EfWriterMeta { Self { ef_base_doc: -1, ef_upper_doc: 0, - use_ef: true, + use_ef: false, with_pf: true, bits: FixedBitSet::default(), } From 17c6ef6bced2e6021e645a4c3c0a440357a36569 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Mon, 9 Mar 2020 16:22:40 +0800 Subject: [PATCH 10/49] balance the size of segment --- src/core/index/merge/merge_policy.rs | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/src/core/index/merge/merge_policy.rs b/src/core/index/merge/merge_policy.rs index 508ff28..1cf0c38 100644 --- a/src/core/index/merge/merge_policy.rs +++ b/src/core/index/merge/merge_policy.rs @@ -552,6 +552,10 @@ impl TieredMergePolicy { MS: MergeScheduler, MP: MergePolicy, { + if segment_infos.len() < 1 { + return Ok(None); + } + let mut infos_sorted = segment_infos.segments.clone(); let comparator = SegmentByteSizeDescending::new(writer, self); infos_sorted.sort_by(|o1, o2| comparator.compare(o1.as_ref(), o2.as_ref())); @@ -567,7 +571,11 @@ impl TieredMergePolicy { } } - next_idx = next_idx.max((self.segs_per_tier * 0.5) as usize); + next_idx = next_idx.max(self.segs_per_tier as usize - 1); + let mut reserved_min = self.max_merged_segment_bytes as i64; + for i in 0..next_idx { + reserved_min = reserved_min.min(info_seg_bytes[i]); + } let merging = writer.merging_segments(); let mut to_be_merged = HashSet::new(); @@ -586,12 +594,15 @@ impl TieredMergePolicy { for j in i + 1..infos_sorted.len() { if curr_merge_bytes > self.max_merged_segment_bytes as i64 || next_merges.len() >= self.max_merge_at_once as usize - || (info_seg_bytes[i] > (self.max_merged_segment_bytes as f64 * 0.1) as i64 + || (info_seg_bytes[i] + > (self.max_merged_segment_bytes / self.max_merge_at_once as u64) as i64 && info_seg_bytes[i] > info_seg_bytes[j] * (self.max_merge_at_once as i64)) { break; } else if curr_merge_bytes + info_seg_bytes[j] > self.max_merged_segment_bytes as i64 + || curr_merge_bytes + info_seg_bytes[j] + > reserved_min * self.segs_per_tier as i64 || merging.contains(&infos_sorted[j].info.name) || to_be_merged.contains(&infos_sorted[j].info.name) { From 644a51ec9a6e2d26d512ed5a7bc26a6f54344fcb Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Fri, 13 Mar 2020 12:02:33 +0800 Subject: [PATCH 11/49] reduce total merging and buffering docs in memory --- src/core/index/merge/merge_policy.rs | 43 +++++++++++++++++----------- 1 file changed, 27 insertions(+), 16 deletions(-) diff --git a/src/core/index/merge/merge_policy.rs b/src/core/index/merge/merge_policy.rs index 1cf0c38..095bb81 100644 --- a/src/core/index/merge/merge_policy.rs +++ b/src/core/index/merge/merge_policy.rs @@ -562,25 +562,28 @@ impl TieredMergePolicy { let mut next_idx = 0; let mut info_seg_bytes = vec![]; + let mut merge_bytes = 0; for info in infos_sorted.iter() { let seg_bytes = self.size(info.as_ref(), writer); info_seg_bytes.push(seg_bytes); if seg_bytes > (self.max_merged_segment_bytes as f64 * 0.8) as i64 { next_idx += 1; + } else { + merge_bytes += seg_bytes; } } - next_idx = next_idx.max(self.segs_per_tier as usize - 1); - let mut reserved_min = self.max_merged_segment_bytes as i64; - for i in 0..next_idx { - reserved_min = reserved_min.min(info_seg_bytes[i]); - } + let reserved_min = (merge_bytes / (self.segs_per_tier * 0.5 + 1.0) as i64) + .min(self.max_merged_segment_bytes as i64); + info!( + "merge_bytes={} reserved_min={} next_idx={} info_seg_bytes={:?}", + merge_bytes, reserved_min, next_idx, info_seg_bytes + ); let merging = writer.merging_segments(); let mut to_be_merged = HashSet::new(); let mut candidates = vec![]; - let mut spec = MergeSpecification::default(); for i in next_idx..infos_sorted.len() { if merging.contains(&infos_sorted[i].info.name) @@ -594,15 +597,12 @@ impl TieredMergePolicy { for j in i + 1..infos_sorted.len() { if curr_merge_bytes > self.max_merged_segment_bytes as i64 || next_merges.len() >= self.max_merge_at_once as usize - || (info_seg_bytes[i] - > (self.max_merged_segment_bytes / self.max_merge_at_once as u64) as i64 - && info_seg_bytes[i] > info_seg_bytes[j] * (self.max_merge_at_once as i64)) + || info_seg_bytes[i] + > info_seg_bytes[j] + * (self.max_merge_at_once * self.max_merge_at_once) as i64 { break; - } else if curr_merge_bytes + info_seg_bytes[j] - > self.max_merged_segment_bytes as i64 - || curr_merge_bytes + info_seg_bytes[j] - > reserved_min * self.segs_per_tier as i64 + } else if curr_merge_bytes + info_seg_bytes[j] > reserved_min || merging.contains(&infos_sorted[j].info.name) || to_be_merged.contains(&infos_sorted[j].info.name) { @@ -617,6 +617,15 @@ impl TieredMergePolicy { continue; } + let next_merges_bytes: Vec = + next_merges.iter().map(|i| info_seg_bytes[*i]).collect(); + + info!( + "segment_count={} curr_merge_bytes={} one_merge={:?}", + next_merges_bytes.len(), + curr_merge_bytes, + next_merges_bytes + ); let mut segments = Vec::with_capacity(next_merges.len()); for idx in next_merges { segments.push(infos_sorted[idx].clone()); @@ -625,13 +634,15 @@ impl TieredMergePolicy { let merge = OneMerge::new(segments, writer.next_merge_id())?; candidates.push(merge); - } - loop { - if spec.merges.len() >= MAX_MERGING_COUNT { + if candidates.len() > MAX_MERGING_COUNT { break; } + } + let mut spec = MergeSpecification::default(); + + loop { if let Some(one_merge) = candidates.pop() { spec.add(one_merge); } else { From 86dfd0a2f8922b1602cb5a5b757275cd867eef6d Mon Sep 17 00:00:00 2001 From: wangjingjing Date: Mon, 16 Mar 2020 15:22:13 +0800 Subject: [PATCH 12/49] boolean support must_nots --- src/core/search/query/boolean_query.rs | 75 ++++++++-- src/core/search/query/match_all_query.rs | 2 +- src/core/search/query/query_string.rs | 4 +- src/core/search/scorer/mod.rs | 4 + src/core/search/scorer/req_not_scorer.rs | 168 +++++++++++++++++++++++ 5 files changed, 239 insertions(+), 14 deletions(-) create mode 100644 src/core/search/scorer/req_not_scorer.rs diff --git a/src/core/search/query/boolean_query.rs b/src/core/search/query/boolean_query.rs index e98d961..d759e8d 100644 --- a/src/core/search/query/boolean_query.rs +++ b/src/core/search/query/boolean_query.rs @@ -17,17 +17,21 @@ use std::fmt; use core::codec::Codec; use core::index::reader::LeafReaderContext; use core::search::explanation::Explanation; -use core::search::query::{ConstantScoreQuery, Query, TermQuery, Weight}; -use core::search::scorer::{ConjunctionScorer, DisjunctionSumScorer, ReqOptScorer, Scorer}; +use core::search::query::{ConstantScoreQuery, MatchAllDocsQuery, Query, TermQuery, Weight}; +use core::search::scorer::{ + ConjunctionScorer, DisjunctionSumScorer, ReqNotScorer, ReqOptScorer, Scorer, +}; use core::search::searcher::SearchPlanBuilder; use core::util::DocId; use error::{ErrorKind::IllegalArgument, Result}; +use std::intrinsics::unlikely; /// A Query that matches documents matching boolean combinations of other queries. pub struct BooleanQuery { must_queries: Vec>>, should_queries: Vec>>, filter_queries: Vec>>, + must_not_queries: Vec>>, minimum_should_match: i32, } @@ -38,17 +42,19 @@ impl BooleanQuery { musts: Vec>>, shoulds: Vec>>, filters: Vec>>, + must_nots: Vec>>, ) -> Result>> { let minimum_should_match = if musts.is_empty() { 1 } else { 0 }; let mut musts = musts; let mut shoulds = shoulds; let mut filters = filters; - if musts.len() + shoulds.len() + filters.len() == 0 { + let must_nots = must_nots; + if musts.len() + shoulds.len() + filters.len() + must_nots.len() == 0 { bail!(IllegalArgument( "boolean query should at least contain one inner query!".into() )); } - if musts.len() + shoulds.len() + filters.len() == 1 { + if must_nots.len() == 0 && musts.len() + shoulds.len() + filters.len() == 1 { let query = if musts.len() == 1 { musts.remove(0) } else if shoulds.len() == 1 { @@ -58,10 +64,15 @@ impl BooleanQuery { }; return Ok(query); } + if musts.len() + shoulds.len() + filters.len() == 0 { + // only must_not exists + musts.push(Box::new(MatchAllDocsQuery {})); + } Ok(Box::new(BooleanQuery { must_queries: musts, should_queries: shoulds, filter_queries: filters, + must_not_queries: must_nots, minimum_should_match, })) } @@ -90,10 +101,15 @@ impl Query for BooleanQuery { for q in &self.should_queries { should_weights.push(searcher.create_weight(q.as_ref(), needs_scores)?); } + let mut must_not_weights = Vec::with_capacity(self.must_not_queries.len()); + for q in &self.must_not_queries { + must_not_weights.push(searcher.create_weight(q.as_ref(), false)?); + } Ok(Box::new(BooleanWeight::new( must_weights, should_weights, + must_not_weights, needs_scores, ))) } @@ -126,10 +142,11 @@ impl fmt::Display for BooleanQuery { let must_str = self.queries_to_str(&self.must_queries); let should_str = self.queries_to_str(&self.should_queries); let filters_str = self.queries_to_str(&self.filter_queries); + let must_not_str = self.queries_to_str(&self.must_not_queries); write!( f, - "BooleanQuery(must: [{}], should: [{}], filters: [{}], match: {})", - must_str, should_str, filters_str, self.minimum_should_match + "BooleanQuery(must: [{}], should: [{}], filters: [{}], must_not: [{}], match: {})", + must_str, should_str, filters_str, must_not_str, self.minimum_should_match ) } } @@ -137,6 +154,7 @@ impl fmt::Display for BooleanQuery { struct BooleanWeight { must_weights: Vec>>, should_weights: Vec>>, + must_not_weights: Vec>>, #[allow(dead_code)] minimum_should_match: i32, needs_scores: bool, @@ -146,12 +164,14 @@ impl BooleanWeight { pub fn new( musts: Vec>>, shoulds: Vec>>, + must_nots: Vec>>, needs_scores: bool, ) -> BooleanWeight { let minimum_should_match = if musts.is_empty() { 1 } else { 0 }; BooleanWeight { must_weights: musts, should_weights: shoulds, + must_not_weights: must_nots, minimum_should_match, needs_scores, } @@ -201,15 +221,43 @@ impl Weight for BooleanWeight { ))), } }; + let must_not_scorer: Option> = { + let mut scorers = vec![]; + for weight in &self.must_not_weights { + if let Some(scorer) = weight.create_scorer(leaf_reader)? { + scorers.push(scorer); + } + } + match scorers.len() { + 0 => None, + 1 => Some(scorers.remove(0)), + _ => Some(Box::new(DisjunctionSumScorer::new(scorers, false))), + } + }; if let Some(must) = must_scorer { if let Some(should) = should_scorer { - Ok(Some(Box::new(ReqOptScorer::new(must, should)))) + if let Some(must_not) = must_not_scorer { + Ok(Some(Box::new(ReqNotScorer::new( + Box::new(ReqOptScorer::new(must, should)), + must_not, + )))) + } else { + Ok(Some(Box::new(ReqOptScorer::new(must, should)))) + } } else { - Ok(Some(must)) + if let Some(must_not) = must_not_scorer { + Ok(Some(Box::new(ReqNotScorer::new(must, must_not)))) + } else { + Ok(Some(must)) + } } } else if let Some(should) = should_scorer { - Ok(Some(should)) + if let Some(must_not) = must_not_scorer { + Ok(Some(Box::new(ReqNotScorer::new(should, must_not)))) + } else { + Ok(Some(should)) + } } else { Ok(None) } @@ -226,6 +274,9 @@ impl Weight for BooleanWeight { for should in &mut self.should_weights { should.normalize(norm, boost); } + for must_not in &mut self.must_not_weights { + must_not.normalize(norm, boost); + } } fn value_for_normalization(&self) -> f32 { @@ -339,10 +390,12 @@ impl fmt::Display for BooleanWeight { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { let must_str = self.weights_to_str(&self.must_weights); let should_str = self.weights_to_str(&self.should_weights); + let must_not_str = self.weights_to_str(&self.must_not_weights); write!( f, - "BooleanWeight(must: [{}], should: [{}], min match: {}, needs score: {})", - must_str, should_str, self.minimum_should_match, self.needs_scores + "BooleanWeight(must: [{}], should: [{}], must_not: [{}], min match: {}, needs score: \ + {})", + must_str, should_str, must_not_str, self.minimum_should_match, self.needs_scores ) } } diff --git a/src/core/search/query/match_all_query.rs b/src/core/search/query/match_all_query.rs index 7684ac2..caa7d96 100644 --- a/src/core/search/query/match_all_query.rs +++ b/src/core/search/query/match_all_query.rs @@ -37,7 +37,7 @@ impl Query for MatchAllDocsQuery { } fn extract_terms(&self) -> Vec { - unimplemented!() + vec![] } fn as_any(&self) -> &dyn (::std::any::Any) { diff --git a/src/core/search/query/query_string.rs b/src/core/search/query/query_string.rs index 3f49ab6..d43e33d 100644 --- a/src/core/search/query/query_string.rs +++ b/src/core/search/query/query_string.rs @@ -171,7 +171,7 @@ impl QueryStringQueryBuilder { shoulds.remove(0) } } else { - BooleanQuery::build(musts, shoulds, vec![])? + BooleanQuery::build(musts, shoulds, vec![], vec![])? }; Ok(Some(query)) } @@ -190,7 +190,7 @@ impl QueryStringQueryBuilder { let res = if queries.len() == 1 { queries.remove(0) } else { - BooleanQuery::build(Vec::new(), queries, vec![])? + BooleanQuery::build(Vec::new(), queries, vec![], vec![])? }; Ok(res) } diff --git a/src/core/search/scorer/mod.rs b/src/core/search/scorer/mod.rs index 573a6d9..f833730 100644 --- a/src/core/search/scorer/mod.rs +++ b/src/core/search/scorer/mod.rs @@ -31,6 +31,10 @@ mod req_opt_scorer; pub use self::req_opt_scorer::*; +mod req_not_scorer; + +pub use self::req_not_scorer::*; + mod rescorer; pub use self::rescorer::*; diff --git a/src/core/search/scorer/req_not_scorer.rs b/src/core/search/scorer/req_not_scorer.rs new file mode 100644 index 0000000..94718cb --- /dev/null +++ b/src/core/search/scorer/req_not_scorer.rs @@ -0,0 +1,168 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use core::search::scorer::Scorer; +use core::search::{DocIterator, NO_MORE_DOCS}; +use core::util::DocId; +use error::Result; +use std::intrinsics::{likely, unlikely}; + +/// A Scorer for queries with not queries. +pub struct ReqNotScorer { + req_scorer: Box, + not_scorer: Box, +} + +impl ReqNotScorer { + pub fn new(req_scorer: Box, not_scorer: Box) -> Self { + Self { + req_scorer, + not_scorer, + } + } +} + +impl Scorer for ReqNotScorer { + fn score(&mut self) -> Result { + self.req_scorer.score() + } +} + +impl DocIterator for ReqNotScorer { + fn doc_id(&self) -> DocId { + self.req_scorer.doc_id() + } + + fn next(&mut self) -> Result { + while let Ok(doc) = self.req_scorer.next() { + if unsafe { unlikely(doc == NO_MORE_DOCS) } { + break; + } + if doc == self.not_scorer.doc_id() { + continue; + } else if doc < self.not_scorer.doc_id() { + return Ok(doc); + } + let not_doc = self.not_scorer.advance(doc)?; + if doc < not_doc { + return Ok(doc); + } + } + Ok(NO_MORE_DOCS) + } + + fn advance(&mut self, target: DocId) -> Result { + let doc = self.req_scorer.advance(target)?; + if unsafe { likely(doc < NO_MORE_DOCS) } { + loop { + if doc == self.not_scorer.doc_id() { + return self.next(); + } else if doc < self.not_scorer.doc_id() { + return Ok(doc); + } + self.not_scorer.advance(doc)?; + } + } + Ok(NO_MORE_DOCS) + } + + fn cost(&self) -> usize { + self.req_scorer.cost() + } + + fn matches(&mut self) -> Result { + self.req_scorer.matches() + } + + fn approximate_next(&mut self) -> Result { + while let Ok(doc) = self.req_scorer.approximate_next() { + if unsafe { unlikely(doc == NO_MORE_DOCS) } { + break; + } + if doc == self.not_scorer.doc_id() { + continue; + } else if doc < self.not_scorer.doc_id() { + return Ok(doc); + } + let not_doc = self.not_scorer.approximate_advance(doc)?; + if doc < not_doc { + return Ok(doc); + } + } + Ok(NO_MORE_DOCS) + } + + fn approximate_advance(&mut self, target: DocId) -> Result { + let doc = self.req_scorer.approximate_advance(target)?; + if unsafe { likely(doc < NO_MORE_DOCS) } { + loop { + if doc == self.not_scorer.doc_id() { + return self.approximate_next(); + } else if doc < self.not_scorer.doc_id() { + return Ok(doc); + } + self.not_scorer.approximate_advance(doc)?; + } + } + Ok(NO_MORE_DOCS) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use core::search::scorer::*; + use core::search::tests::*; + use core::search::*; + + #[test] + fn test_next() { + let s1 = create_mock_scorer(vec![1, 2, 3, 4, 5, 6, 7, 8, 9]); + let s2 = create_mock_scorer(vec![2, 3, 5, 7, 9, 10]); + let s3 = create_mock_scorer(vec![2, 5]); + let s4 = create_mock_scorer(vec![1, 4, 5]); + + let conjunction_scorer: Box = Box::new(ConjunctionScorer::new(vec![s1, s2])); + let disjunction_scorer: Box = + Box::new(DisjunctionSumScorer::new(vec![s3, s4], true)); + let mut scorer = ReqNotScorer::new(conjunction_scorer, disjunction_scorer); + + assert_eq!(scorer.doc_id(), -1); + + assert_eq!(scorer.next().unwrap(), 3); + assert_eq!(scorer.next().unwrap(), 7); + assert_eq!(scorer.next().unwrap(), 9); + + assert_eq!(scorer.next().unwrap(), NO_MORE_DOCS); + } + + #[test] + fn test_advance() { + let s1 = create_mock_scorer(vec![1, 2, 3, 4, 5, 6, 7, 8, 9]); + let s2 = create_mock_scorer(vec![2, 3, 5, 7, 9, 10]); + let s3 = create_mock_scorer(vec![2, 5]); + let s4 = create_mock_scorer(vec![1, 4, 5]); + + let conjunction_scorer: Box = Box::new(ConjunctionScorer::new(vec![s1, s2])); + let disjunction_scorer: Box = + Box::new(DisjunctionSumScorer::new(vec![s3, s4], true)); + let mut scorer = ReqNotScorer::new(conjunction_scorer, disjunction_scorer); + + // 2, 3, 5, 7, 9 + // 1, 2, 4, 5 + assert_eq!(scorer.advance(1).unwrap(), 3); + assert_eq!(scorer.advance(4).unwrap(), 7); + assert_eq!(scorer.advance(8).unwrap(), 9); + assert_eq!(scorer.advance(10).unwrap(), NO_MORE_DOCS); + } +} From 909535100f0bbe32015134674dbd947e24ff6e0f Mon Sep 17 00:00:00 2001 From: wangjingjing Date: Mon, 16 Mar 2020 15:48:16 +0800 Subject: [PATCH 13/49] fix some unit test --- src/core/search/query/boolean_query.rs | 1 - src/core/search/query/query_string.rs | 24 ++++++++++++------------ 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/src/core/search/query/boolean_query.rs b/src/core/search/query/boolean_query.rs index d759e8d..1c83506 100644 --- a/src/core/search/query/boolean_query.rs +++ b/src/core/search/query/boolean_query.rs @@ -24,7 +24,6 @@ use core::search::scorer::{ use core::search::searcher::SearchPlanBuilder; use core::util::DocId; use error::{ErrorKind::IllegalArgument, Result}; -use std::intrinsics::unlikely; /// A Query that matches documents matching boolean combinations of other queries. pub struct BooleanQuery { diff --git a/src/core/search/query/query_string.rs b/src/core/search/query/query_string.rs index d43e33d..b732533 100644 --- a/src/core/search/query/query_string.rs +++ b/src/core/search/query/query_string.rs @@ -279,7 +279,7 @@ mod tests { term_str, String::from( "BooleanQuery(must: [], should: [TermQuery(field: title, term: test, boost: 0.2), \ - TermQuery(field: title, term: 测试, boost: 2)], filters: [], match: 1)", + TermQuery(field: title, term: 测试, boost: 2)], filters: [], must_not: [], match: 1)", ) ); @@ -294,7 +294,7 @@ mod tests { term_str, String::from( "BooleanQuery(must: [], should: [TermQuery(field: title, term: test, boost: 0.2), \ - TermQuery(field: title, term: 测试, boost: 2)], filters: [], match: 1)", + TermQuery(field: title, term: 测试, boost: 2)], filters: [], must_not: [], match: 1)", ) ); @@ -320,7 +320,7 @@ mod tests { term_str, String::from( "BooleanQuery(must: [], should: [TermQuery(field: title, term: test, boost: 1), \ - TermQuery(field: title, term: search, boost: 1)], filters: [], match: 1)", + TermQuery(field: title, term: search, boost: 1)], filters: [], must_not: [], match: 1)", ) ); @@ -335,7 +335,7 @@ mod tests { term_str, String::from( "BooleanQuery(must: [TermQuery(field: title, term: search, boost: 1)], should: \ - [TermQuery(field: title, term: test, boost: 1)], filters: [], match: 0)", + [TermQuery(field: title, term: test, boost: 1)], filters: [], must_not: [], match: 0)", ) ); @@ -351,8 +351,8 @@ mod tests { String::from( "BooleanQuery(must: [BooleanQuery(must: [], should: [TermQuery(field: title, \ term: search, boost: 1), TermQuery(field: title, term: 搜索, boost: 1)], \ - filters: [], match: 1)], should: [TermQuery(field: title, term: test, boost: \ - 1)], filters: [], match: 0)", + filters: [], must_not: [], match: 1)], should: [TermQuery(field: title, term: test, boost: \ + 1)], filters: [], must_not: [], match: 0)", ) ); @@ -371,9 +371,9 @@ mod tests { String::from( "BooleanQuery(must: [BooleanQuery(must: [], should: [TermQuery(field: title, \ term: search, boost: 1), TermQuery(field: content, term: search, boost: 1)], \ - filters: [], match: 1)], should: [BooleanQuery(must: [], should: \ + filters: [], must_not: [], match: 1)], should: [BooleanQuery(must: [], should: \ [TermQuery(field: title, term: test, boost: 1), TermQuery(field: content, term: \ - test, boost: 1)], filters: [], match: 1)], filters: [], match: 0)", + test, boost: 1)], filters: [], must_not: [], match: 1)], filters: [], must_not: [], match: 0)", ) ); @@ -392,15 +392,15 @@ mod tests { "BooleanQuery(must: [BooleanQuery(must: [], should: [TermQuery(field: title, \ term: 市场定位, boost: 1), BooleanQuery(must: [TermQuery(field: title, term: \ 市场, boost: 1), TermQuery(field: title, term: 定位, boost: 1)], should: [], \ - filters: [], match: 0)], filters: [], match: 1), TermQuery(field: title, term: \ + filters: [], must_not: [], match: 0)], filters: [], must_not: [], match: 1), TermQuery(field: title, term: \ b2b, boost: 1), BooleanQuery(must: [], should: [BooleanQuery(must: [], should: \ [TermQuery(field: title, term: 电子商务, boost: 1), TermQuery(field: title, \ - term: 电商, boost: 0.8)], filters: [], match: 1), BooleanQuery(must: \ + term: 电商, boost: 0.8)], filters: [], must_not: [], match: 1), BooleanQuery(must: \ [TermQuery(field: title, term: 电子, boost: 1), TermQuery(field: title, term: \ - 商务, boost: 1)], should: [], filters: [], match: 0)], filters: [], match: 1), \ + 商务, boost: 1)], should: [], filters: [], must_not: [], match: 0)], filters: [], must_not: [], match: 1), \ TermQuery(field: title, term: 网站, boost: 1)], should: [TermQuery(field: title, \ term: 从, boost: 1), TermQuery(field: title, term: 分析, boost: 1)], filters: \ - [], match: 0)", + [], must_not: [], match: 0)", ) ); } From 9bb1aa1b113f79aa3ccedd1e32208318b06e456b Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Fri, 27 Mar 2020 16:10:05 +0800 Subject: [PATCH 14/49] fmt & tune merge params --- src/core/index/merge/merge_policy.rs | 14 ++++++---- src/core/search/query/query_string.rs | 38 ++++++++++++++++----------- 2 files changed, 31 insertions(+), 21 deletions(-) diff --git a/src/core/index/merge/merge_policy.rs b/src/core/index/merge/merge_policy.rs index 095bb81..12b4d99 100644 --- a/src/core/index/merge/merge_policy.rs +++ b/src/core/index/merge/merge_policy.rs @@ -574,8 +574,12 @@ impl TieredMergePolicy { } } - let reserved_min = (merge_bytes / (self.segs_per_tier * 0.5 + 1.0) as i64) - .min(self.max_merged_segment_bytes as i64); + let min_segment = (self.segs_per_tier * 0.5 + 1.0) as i64; + let reserved_min = if next_idx as i64 >= min_segment { + self.max_merged_segment_bytes as i64 + } else { + (merge_bytes / min_segment).min(self.max_merged_segment_bytes as i64) + }; info!( "merge_bytes={} reserved_min={} next_idx={} info_seg_bytes={:?}", merge_bytes, reserved_min, next_idx, info_seg_bytes @@ -597,9 +601,9 @@ impl TieredMergePolicy { for j in i + 1..infos_sorted.len() { if curr_merge_bytes > self.max_merged_segment_bytes as i64 || next_merges.len() >= self.max_merge_at_once as usize - || info_seg_bytes[i] - > info_seg_bytes[j] - * (self.max_merge_at_once * self.max_merge_at_once) as i64 + || (info_seg_bytes[i] + > (self.max_merged_segment_bytes as f64 / self.segs_per_tier) as i64 + && info_seg_bytes[i] > info_seg_bytes[j] * self.max_merge_at_once as i64) { break; } else if curr_merge_bytes + info_seg_bytes[j] > reserved_min diff --git a/src/core/search/query/query_string.rs b/src/core/search/query/query_string.rs index b732533..c4c696d 100644 --- a/src/core/search/query/query_string.rs +++ b/src/core/search/query/query_string.rs @@ -279,7 +279,8 @@ mod tests { term_str, String::from( "BooleanQuery(must: [], should: [TermQuery(field: title, term: test, boost: 0.2), \ - TermQuery(field: title, term: 测试, boost: 2)], filters: [], must_not: [], match: 1)", + TermQuery(field: title, term: 测试, boost: 2)], filters: [], must_not: [], \ + match: 1)", ) ); @@ -294,7 +295,8 @@ mod tests { term_str, String::from( "BooleanQuery(must: [], should: [TermQuery(field: title, term: test, boost: 0.2), \ - TermQuery(field: title, term: 测试, boost: 2)], filters: [], must_not: [], match: 1)", + TermQuery(field: title, term: 测试, boost: 2)], filters: [], must_not: [], \ + match: 1)", ) ); @@ -320,7 +322,8 @@ mod tests { term_str, String::from( "BooleanQuery(must: [], should: [TermQuery(field: title, term: test, boost: 1), \ - TermQuery(field: title, term: search, boost: 1)], filters: [], must_not: [], match: 1)", + TermQuery(field: title, term: search, boost: 1)], filters: [], must_not: [], \ + match: 1)", ) ); @@ -335,7 +338,8 @@ mod tests { term_str, String::from( "BooleanQuery(must: [TermQuery(field: title, term: search, boost: 1)], should: \ - [TermQuery(field: title, term: test, boost: 1)], filters: [], must_not: [], match: 0)", + [TermQuery(field: title, term: test, boost: 1)], filters: [], must_not: [], \ + match: 0)", ) ); @@ -351,8 +355,8 @@ mod tests { String::from( "BooleanQuery(must: [BooleanQuery(must: [], should: [TermQuery(field: title, \ term: search, boost: 1), TermQuery(field: title, term: 搜索, boost: 1)], \ - filters: [], must_not: [], match: 1)], should: [TermQuery(field: title, term: test, boost: \ - 1)], filters: [], must_not: [], match: 0)", + filters: [], must_not: [], match: 1)], should: [TermQuery(field: title, term: \ + test, boost: 1)], filters: [], must_not: [], match: 0)", ) ); @@ -373,7 +377,8 @@ mod tests { term: search, boost: 1), TermQuery(field: content, term: search, boost: 1)], \ filters: [], must_not: [], match: 1)], should: [BooleanQuery(must: [], should: \ [TermQuery(field: title, term: test, boost: 1), TermQuery(field: content, term: \ - test, boost: 1)], filters: [], must_not: [], match: 1)], filters: [], must_not: [], match: 0)", + test, boost: 1)], filters: [], must_not: [], match: 1)], filters: [], must_not: \ + [], match: 0)", ) ); @@ -392,15 +397,16 @@ mod tests { "BooleanQuery(must: [BooleanQuery(must: [], should: [TermQuery(field: title, \ term: 市场定位, boost: 1), BooleanQuery(must: [TermQuery(field: title, term: \ 市场, boost: 1), TermQuery(field: title, term: 定位, boost: 1)], should: [], \ - filters: [], must_not: [], match: 0)], filters: [], must_not: [], match: 1), TermQuery(field: title, term: \ - b2b, boost: 1), BooleanQuery(must: [], should: [BooleanQuery(must: [], should: \ - [TermQuery(field: title, term: 电子商务, boost: 1), TermQuery(field: title, \ - term: 电商, boost: 0.8)], filters: [], must_not: [], match: 1), BooleanQuery(must: \ - [TermQuery(field: title, term: 电子, boost: 1), TermQuery(field: title, term: \ - 商务, boost: 1)], should: [], filters: [], must_not: [], match: 0)], filters: [], must_not: [], match: 1), \ - TermQuery(field: title, term: 网站, boost: 1)], should: [TermQuery(field: title, \ - term: 从, boost: 1), TermQuery(field: title, term: 分析, boost: 1)], filters: \ - [], must_not: [], match: 0)", + filters: [], must_not: [], match: 0)], filters: [], must_not: [], match: 1), \ + TermQuery(field: title, term: b2b, boost: 1), BooleanQuery(must: [], should: \ + [BooleanQuery(must: [], should: [TermQuery(field: title, term: 电子商务, boost: \ + 1), TermQuery(field: title, term: 电商, boost: 0.8)], filters: [], must_not: [], \ + match: 1), BooleanQuery(must: [TermQuery(field: title, term: 电子, boost: 1), \ + TermQuery(field: title, term: 商务, boost: 1)], should: [], filters: [], \ + must_not: [], match: 0)], filters: [], must_not: [], match: 1), TermQuery(field: \ + title, term: 网站, boost: 1)], should: [TermQuery(field: title, term: 从, boost: \ + 1), TermQuery(field: title, term: 分析, boost: 1)], filters: [], must_not: [], \ + match: 0)", ) ); } From 7e9420d239dd2812316f315f87bfb68a8e90b077 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Fri, 27 Mar 2020 22:45:03 +0800 Subject: [PATCH 15/49] rustup to 2020-03-12 --- joker.yml | 0 rust-toolchain | 2 +- src/core/codec/postings/posting_reader.rs | 6 +++--- .../codec/term_vectors/term_vectors_reader.rs | 1 - src/core/search/cache/query_cache.rs | 2 +- src/core/search/query/spans/span.rs | 2 +- src/core/search/scorer/req_not_scorer.rs | 8 ++++---- src/core/util/bit_util.rs | 2 +- src/core/util/bkd/bkd_writer.rs | 16 ++++++++-------- src/core/util/bkd/offline_point.rs | 2 +- src/core/util/fst/fst_builder.rs | 2 +- src/lib.rs | 1 - 12 files changed, 21 insertions(+), 23 deletions(-) create mode 100644 joker.yml diff --git a/joker.yml b/joker.yml new file mode 100644 index 0000000..e69de29 diff --git a/rust-toolchain b/rust-toolchain index cbdbf5d..7b70b33 100644 --- a/rust-toolchain +++ b/rust-toolchain @@ -1 +1 @@ -nightly-2019-10-28 +nightly-2020-03-12 diff --git a/src/core/codec/postings/posting_reader.rs b/src/core/codec/postings/posting_reader.rs index 681a11d..2ce975e 100644 --- a/src/core/codec/postings/posting_reader.rs +++ b/src/core/codec/postings/posting_reader.rs @@ -600,7 +600,7 @@ impl DocIterator for BlockDocIterator { } fn advance(&mut self, target: DocId) -> Result { - if unsafe { unlikely(target == NO_MORE_DOCS) } { + if unlikely(target == NO_MORE_DOCS) { self.doc = NO_MORE_DOCS; return Ok(self.doc); } @@ -1138,7 +1138,7 @@ impl DocIterator for BlockPostingIterator { } fn advance(&mut self, target: DocId) -> Result { - if unsafe { unlikely(target == NO_MORE_DOCS) } { + if unlikely(target == NO_MORE_DOCS) { self.doc = NO_MORE_DOCS; return Ok(self.doc); } @@ -1875,7 +1875,7 @@ impl DocIterator for EverythingIterator { } fn advance(&mut self, target: DocId) -> Result { - if unsafe { unlikely(target == NO_MORE_DOCS) } { + if unlikely(target == NO_MORE_DOCS) { self.doc = NO_MORE_DOCS; return Ok(self.doc); } diff --git a/src/core/codec/term_vectors/term_vectors_reader.rs b/src/core/codec/term_vectors/term_vectors_reader.rs index af0a896..e9fc327 100644 --- a/src/core/codec/term_vectors/term_vectors_reader.rs +++ b/src/core/codec/term_vectors/term_vectors_reader.rs @@ -416,7 +416,6 @@ impl CompressingTermVectorsReader { }) } - #[allow(cyclomatic_complexity)] fn get_mut(&mut self, doc: i32) -> Result> { let mut vectors_stream = self.vectors_stream.as_ref().clone()?; let vectors_stream = vectors_stream.as_mut(); diff --git a/src/core/search/cache/query_cache.rs b/src/core/search/cache/query_cache.rs index 866be11..2c53642 100644 --- a/src/core/search/cache/query_cache.rs +++ b/src/core/search/cache/query_cache.rs @@ -773,7 +773,7 @@ impl RoaringDocIterator { } } - fn first_doc_from_next_block(&mut self) -> Result<(DocId)> { + fn first_doc_from_next_block(&mut self) -> Result { loop { self.block += 1; if self.block as usize >= self.doc_id_sets.len() { diff --git a/src/core/search/query/spans/span.rs b/src/core/search/query/spans/span.rs index 7b57b58..8918e3b 100644 --- a/src/core/search/query/spans/span.rs +++ b/src/core/search/query/spans/span.rs @@ -522,7 +522,7 @@ impl SpanScorer { Ok(()) } - fn score_current_doc(&mut self) -> Result<(f32)> { + fn score_current_doc(&mut self) -> Result { debug_assert!(self.doc_scorer.is_some()); let doc = self.doc_id(); let freq = self.freq; diff --git a/src/core/search/scorer/req_not_scorer.rs b/src/core/search/scorer/req_not_scorer.rs index 94718cb..34f0a6e 100644 --- a/src/core/search/scorer/req_not_scorer.rs +++ b/src/core/search/scorer/req_not_scorer.rs @@ -45,7 +45,7 @@ impl DocIterator for ReqNotScorer { fn next(&mut self) -> Result { while let Ok(doc) = self.req_scorer.next() { - if unsafe { unlikely(doc == NO_MORE_DOCS) } { + if unlikely(doc == NO_MORE_DOCS) { break; } if doc == self.not_scorer.doc_id() { @@ -63,7 +63,7 @@ impl DocIterator for ReqNotScorer { fn advance(&mut self, target: DocId) -> Result { let doc = self.req_scorer.advance(target)?; - if unsafe { likely(doc < NO_MORE_DOCS) } { + if likely(doc < NO_MORE_DOCS) { loop { if doc == self.not_scorer.doc_id() { return self.next(); @@ -86,7 +86,7 @@ impl DocIterator for ReqNotScorer { fn approximate_next(&mut self) -> Result { while let Ok(doc) = self.req_scorer.approximate_next() { - if unsafe { unlikely(doc == NO_MORE_DOCS) } { + if unlikely(doc == NO_MORE_DOCS) { break; } if doc == self.not_scorer.doc_id() { @@ -104,7 +104,7 @@ impl DocIterator for ReqNotScorer { fn approximate_advance(&mut self, target: DocId) -> Result { let doc = self.req_scorer.approximate_advance(target)?; - if unsafe { likely(doc < NO_MORE_DOCS) } { + if likely(doc < NO_MORE_DOCS) { loop { if doc == self.not_scorer.doc_id() { return self.approximate_next(); diff --git a/src/core/util/bit_util.rs b/src/core/util/bit_util.rs index a774f1c..9d46171 100644 --- a/src/core/util/bit_util.rs +++ b/src/core/util/bit_util.rs @@ -32,7 +32,7 @@ impl ZigZagEncoding for i64 { (*self >> 63) ^ (self << 1) } fn decode(&self) -> i64 { - ((*self as u64 >> 1) as i64 ^ -(self & 1)) + (*self as u64 >> 1) as i64 ^ -(self & 1) } } diff --git a/src/core/util/bkd/bkd_writer.rs b/src/core/util/bkd/bkd_writer.rs index 130fa2a..cb39fcc 100644 --- a/src/core/util/bkd/bkd_writer.rs +++ b/src/core/util/bkd/bkd_writer.rs @@ -262,7 +262,7 @@ impl<'a, D: Directory, O: IndexOutput> OneDimensionBKDWriter<'a, D, O> { bkd_writer.common_prefix_lengths[0] = prefix; debug_assert!(bkd_writer.scratch_out.position() == 0); - let scratch_out = (&mut bkd_writer.scratch_out) as (*mut GrowableByteArrayDataOutput); + let scratch_out = (&mut bkd_writer.scratch_out) as *mut GrowableByteArrayDataOutput; let common_prefix_lengths: *mut [usize] = bkd_writer.common_prefix_lengths.as_mut(); bkd_writer.write_leaf_block_docs( @@ -615,8 +615,8 @@ impl BKDWriter { self.heap_point_writer = None; } - let min_packed_value = (&mut self.min_packed_value) as (*mut Vec); - let max_packed_value = (&mut self.max_packed_value) as (*mut Vec); + let min_packed_value = (&mut self.min_packed_value) as *mut Vec; + let max_packed_value = (&mut self.max_packed_value) as *mut Vec; let mut parent_splits: Vec = vec![0i32; self.num_dims as usize]; self.build( 1, @@ -772,7 +772,7 @@ impl BKDWriter { ); let mut one_dim_writer = OneDimensionBKDWriter::new(out, self)?; - let one_dim_writer_ptr = (&mut one_dim_writer) as (*mut OneDimensionBKDWriter); + let one_dim_writer_ptr = (&mut one_dim_writer) as *mut OneDimensionBKDWriter; let mut visitor = OneDimIntersectVisitor::new(unsafe { &mut (*one_dim_writer_ptr) }); reader.intersect(field_name, &mut visitor)?; @@ -1453,7 +1453,7 @@ impl BKDWriter { // like how terms dict does so from the FST: // Write the common prefixes: - let scratch1 = (&mut self.scratch1) as (*mut Vec); + let scratch1 = (&mut self.scratch1) as *mut Vec; self.write_common_prefixes(out, &self.common_prefix_lengths, unsafe { &mut (*scratch1) })?; @@ -1491,8 +1491,8 @@ impl BKDWriter { } else { // Inner node: partition/recurse let split_dim = if self.num_dims > 1 { - let min_packed_value = (&self.min_packed_value) as (*const Vec); - let max_packed_value = (&self.max_packed_value) as (*const Vec); + let min_packed_value = (&self.min_packed_value) as *const Vec; + let max_packed_value = (&self.max_packed_value) as *const Vec; self.split( unsafe { &(*min_packed_value) }, unsafe { &(*max_packed_value) }, @@ -1502,7 +1502,7 @@ impl BKDWriter { 0 }; - let source = (&slices[split_dim]) as (*const PathSlice>); + let source = (&slices[split_dim]) as *const PathSlice>; let source = unsafe { &(*source) }; debug_assert!((node_id as usize) < split_packed_values.len()); diff --git a/src/core/util/bkd/offline_point.rs b/src/core/util/bkd/offline_point.rs index 06dd572..3ce9070 100644 --- a/src/core/util/bkd/offline_point.rs +++ b/src/core/util/bkd/offline_point.rs @@ -144,7 +144,7 @@ impl OfflinePointReader { pos += 1; let i1p4 = bytes[pos] as u32 as i32; - (i1p1 << 24 | i1p2 << 16 | i1p3 << 8 | i1p4) + i1p1 << 24 | i1p2 << 16 | i1p3 << 8 | i1p4 } } diff --git a/src/core/util/fst/fst_builder.rs b/src/core/util/fst/fst_builder.rs index cb555de..b835b3d 100644 --- a/src/core/util/fst/fst_builder.rs +++ b/src/core/util/fst/fst_builder.rs @@ -626,7 +626,7 @@ impl NodeHash { Ok(h) } - pub fn add(&mut self, builder: &mut FstBuilder, node_index: usize) -> Result<(u64)> { + pub fn add(&mut self, builder: &mut FstBuilder, node_index: usize) -> Result { let h = self.node_hash_uncompiled(&builder.frontier[node_index]); let mut pos = h & self.mask as u64; let mut c = 0; diff --git a/src/lib.rs b/src/lib.rs index 69f8519..ddfbb72 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -27,7 +27,6 @@ #![feature(maybe_uninit_extra)] #![feature(in_band_lifetimes)] #![feature(vec_into_raw_parts)] -#![feature(slice_from_raw_parts)] #![feature(core_intrinsics)] #[macro_use] From ade8aa5d6d24b60eb53661ec0c4e493dc973f0f8 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Mon, 13 Apr 2020 12:06:05 +0800 Subject: [PATCH 16/49] for hundreds of sub queries in [disisum/disimax] --- src/core/search/scorer/disjunction_scorer.rs | 163 +++++++++++-------- src/core/search/scorer/req_not_scorer.rs | 1 - 2 files changed, 96 insertions(+), 68 deletions(-) diff --git a/src/core/search/scorer/disjunction_scorer.rs b/src/core/search/scorer/disjunction_scorer.rs index 4b1cc29..65ed747 100644 --- a/src/core/search/scorer/disjunction_scorer.rs +++ b/src/core/search/scorer/disjunction_scorer.rs @@ -12,46 +12,63 @@ // limitations under the License. use core::search::scorer::Scorer; -use core::search::{DocIterator, NO_MORE_DOCS}; -use core::util::DocId; +use core::search::DocIterator; +use core::util::{DisiPriorityQueue, DocId}; use error::Result; use std::f32; /// A Scorer for OR like queries, counterpart of `ConjunctionScorer`. pub struct DisjunctionSumScorer { - sub_scorers: Vec, - curr_doc: DocId, + sub_scorers: DisiPriorityQueue, + needs_scores: bool, cost: usize, } impl DisjunctionSumScorer { - pub fn new(children: Vec, _needs_scores: bool) -> DisjunctionSumScorer { + pub fn new(children: Vec, needs_scores: bool) -> DisjunctionSumScorer { assert!(children.len() > 1); let cost = children.iter().map(|w| w.cost()).sum(); - let mut curr_doc = NO_MORE_DOCS; - for s in children.iter() { - curr_doc = curr_doc.min(s.doc_id()); - } DisjunctionSumScorer { - sub_scorers: children, - curr_doc, + sub_scorers: DisiPriorityQueue::new(children), + needs_scores, cost, } } + + pub fn sub_scorers(&self) -> &DisiPriorityQueue { + &self.sub_scorers + } + + pub fn sub_scorers_mut(&mut self) -> &mut DisiPriorityQueue { + &mut self.sub_scorers + } + + pub fn get_cost(&self) -> usize { + self.cost + } } impl Scorer for DisjunctionSumScorer { fn score(&mut self) -> Result { let mut score: f32 = 0.0f32; - let doc_id = self.doc_id(); - for s in self.sub_scorers.iter_mut() { - if s.doc_id() == doc_id { - let sub_score = s.score()?; - score += sub_score; + if !self.needs_scores { + return Ok(score); + } + + let mut disi = self.sub_scorers_mut().top_list(); + + loop { + let sub_score = disi.inner_mut().score()?; + score += sub_score; + + if disi.next.is_null() { + break; + } else { + unsafe { disi = &mut *disi.next }; } } @@ -61,7 +78,7 @@ impl Scorer for DisjunctionSumScorer { impl DocIterator for DisjunctionSumScorer { fn doc_id(&self) -> DocId { - self.curr_doc + self.sub_scorers().peek().doc() } fn next(&mut self) -> Result { @@ -85,32 +102,29 @@ impl DocIterator for DisjunctionSumScorer { } fn approximate_next(&mut self) -> Result { - let curr_doc = self.doc_id(); - let mut min_doc = NO_MORE_DOCS; - for s in self.sub_scorers.iter_mut() { - if s.doc_id() == curr_doc { - s.approximate_next()?; - } + let sub_scorers = self.sub_scorers_mut(); + let doc = sub_scorers.peek().doc(); - min_doc = min_doc.min(s.doc_id()); + loop { + sub_scorers.peek_mut().approximate_next()?; + if sub_scorers.peek().doc() != doc { + break; + } } - self.curr_doc = min_doc; - Ok(self.doc_id()) + Ok(sub_scorers.peek().doc()) } fn approximate_advance(&mut self, target: DocId) -> Result { - let mut min_doc = NO_MORE_DOCS; - for s in self.sub_scorers.iter_mut() { - if s.doc_id() < target { - s.approximate_advance(target)?; + let sub_scorers = self.sub_scorers_mut(); + loop { + sub_scorers.peek_mut().approximate_advance(target)?; + if sub_scorers.peek().doc() >= target { + break; } - - min_doc = min_doc.min(s.doc_id()); } - self.curr_doc = min_doc; - Ok(self.doc_id()) + Ok(sub_scorers.peek().doc()) } } @@ -120,8 +134,8 @@ impl DocIterator for DisjunctionSumScorer { /// tieBreakerMultiplier times the sum of the scores for the other subqueries that generate the /// document. pub struct DisjunctionMaxScorer { - sub_scorers: Vec, - curr_doc: DocId, + sub_scorers: DisiPriorityQueue, + needs_scores: bool, cost: usize, tie_breaker_multiplier: f32, } @@ -130,37 +144,55 @@ impl DisjunctionMaxScorer { pub fn new( children: Vec, tie_breaker_multiplier: f32, - _needs_score: bool, + needs_scores: bool, ) -> DisjunctionMaxScorer { assert!(children.len() > 1); let cost = children.iter().map(|w| w.cost()).sum(); - let mut curr_doc = NO_MORE_DOCS; - for s in children.iter() { - curr_doc = curr_doc.min(s.doc_id()); - } DisjunctionMaxScorer { - sub_scorers: children, - curr_doc, + sub_scorers: DisiPriorityQueue::new(children), + needs_scores, cost, tie_breaker_multiplier, } } + + pub fn sub_scorers(&self) -> &DisiPriorityQueue { + &self.sub_scorers + } + + pub fn sub_scorers_mut(&mut self) -> &mut DisiPriorityQueue { + &mut self.sub_scorers + } + + pub fn get_cost(&self) -> usize { + self.cost + } } impl Scorer for DisjunctionMaxScorer { fn score(&mut self) -> Result { let mut score_sum = 0.0f32; + + if !self.needs_scores { + return Ok(score_sum); + } + let mut score_max = f32::NEG_INFINITY; + let mut disi = self.sub_scorers_mut().top_list(); - let doc_id = self.doc_id(); - for s in self.sub_scorers.iter_mut() { - if s.doc_id() == doc_id { - let sub_score = s.score()?; + loop { + let sub_score = disi.inner_mut().score()?; + score_sum += sub_score; + if sub_score > score_max { + score_max = sub_score; + } - score_sum += sub_score; - score_max = score_max.max(sub_score); + if disi.next.is_null() { + break; + } else { + unsafe { disi = &mut *disi.next }; } } @@ -170,7 +202,7 @@ impl Scorer for DisjunctionMaxScorer { impl DocIterator for DisjunctionMaxScorer { fn doc_id(&self) -> DocId { - self.curr_doc + self.sub_scorers().peek().doc() } fn next(&mut self) -> Result { @@ -194,31 +226,28 @@ impl DocIterator for DisjunctionMaxScorer { } fn approximate_next(&mut self) -> Result { - let curr_doc = self.doc_id(); - let mut min_doc = NO_MORE_DOCS; - for s in self.sub_scorers.iter_mut() { - if s.doc_id() == curr_doc { - s.approximate_next()?; - } + let sub_scorers = self.sub_scorers_mut(); + let doc = sub_scorers.peek().doc(); - min_doc = min_doc.min(s.doc_id()); + loop { + sub_scorers.peek_mut().approximate_next()?; + if sub_scorers.peek().doc() != doc { + break; + } } - self.curr_doc = min_doc; - Ok(self.doc_id()) + Ok(sub_scorers.peek().doc()) } fn approximate_advance(&mut self, target: DocId) -> Result { - let mut min_doc = NO_MORE_DOCS; - for s in self.sub_scorers.iter_mut() { - if s.doc_id() < target { - s.approximate_advance(target)?; + let sub_scorers = self.sub_scorers_mut(); + loop { + sub_scorers.peek_mut().approximate_advance(target)?; + if sub_scorers.peek().doc() >= target { + break; } - - min_doc = min_doc.min(s.doc_id()); } - self.curr_doc = min_doc; - Ok(self.doc_id()) + Ok(sub_scorers.peek().doc()) } } diff --git a/src/core/search/scorer/req_not_scorer.rs b/src/core/search/scorer/req_not_scorer.rs index 34f0a6e..6f0c9b1 100644 --- a/src/core/search/scorer/req_not_scorer.rs +++ b/src/core/search/scorer/req_not_scorer.rs @@ -123,7 +123,6 @@ mod tests { use super::*; use core::search::scorer::*; use core::search::tests::*; - use core::search::*; #[test] fn test_next() { From fd988abbd0ece6cd17b64e6605b6c7c8f08046a6 Mon Sep 17 00:00:00 2001 From: xiezhi Date: Mon, 13 Apr 2020 19:01:32 +0800 Subject: [PATCH 17/49] boosting query code complete --- src/core/search/query/boosting_query.rs | 140 ++++++++++++++++++++++ src/core/search/query/mod.rs | 4 + src/core/search/scorer/boosting_scorer.rs | 64 ++++++++++ src/core/search/scorer/mod.rs | 4 + 4 files changed, 212 insertions(+) create mode 100644 src/core/search/query/boosting_query.rs create mode 100644 src/core/search/scorer/boosting_scorer.rs diff --git a/src/core/search/query/boosting_query.rs b/src/core/search/query/boosting_query.rs new file mode 100644 index 0000000..c365f7e --- /dev/null +++ b/src/core/search/query/boosting_query.rs @@ -0,0 +1,140 @@ +use std::any::Any; +use std::fmt; + +use core::codec::Codec; +use core::index::reader::LeafReaderContext; +use core::search::explanation::Explanation; +use core::search::query::{Query, TermQuery, Weight}; +use core::search::scorer::Scorer; +use core::search::searcher::SearchPlanBuilder; +use core::util::DocId; +use core::search::scorer::BoostingScorer; +use error::Result; + +const BOOSTING_QUERY: &str = "boosting"; + +pub struct BoostingQuery { + positive: Box>, + negative: Box>, + negative_boost: f32, +} + +impl BoostingQuery { + pub fn build( + positive: Box>, + negative: Box>, + negative_boost: f32, + ) -> Box> { + Box::new(BoostingQuery { + positive, + negative, + negative_boost, + }) + } +} + +impl Query for BoostingQuery { + fn create_weight( + &self, + searcher: &dyn SearchPlanBuilder, + needs_scores: bool, + ) -> Result>> { + Ok(Box::new( + BoostingWeight::new( + self.positive.create_weight(searcher, needs_scores)?, + self.negative.create_weight(searcher, false)?, + self.negative_boost, + ))) + } + + fn extract_terms(&self) -> Vec { + self.positive.extract_terms() + } + + fn as_any(&self) -> &dyn Any { + self + } +} + +impl fmt::Display for BoostingQuery { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "BoostingQuery(positive: {}, negative: {}, negative_boost: {})", + &self.positive, &self.negative, self.negative_boost + ) + } +} + +struct BoostingWeight { + positive_weight: Box>, + negative_weight: Box>, + negative_boost: f32, +} + +impl BoostingWeight { + pub fn new( + positive_weight: Box>, + negative_weight: Box>, + negative_boost: f32, + ) -> BoostingWeight { + BoostingWeight { + positive_weight, + negative_weight, + negative_boost, + } + } +} + +impl Weight for BoostingWeight { + fn create_scorer( + &self, + leaf_reader: &LeafReaderContext<'_, C>, + ) -> Result>> { + if let (Some(positive_scorer), Some(negative_scorer)) = + (self.positive_weight.create_scorer(leaf_reader)?, self.negative_weight.create_scorer(leaf_reader)?) { + Ok(Some(Box::new(BoostingScorer::new( + positive_scorer, + negative_scorer, + self.negative_boost, + )))) + } else { + Ok(None) + } + } + + fn query_type(&self) -> &'static str { + BOOSTING_QUERY + } + + fn actual_query_type(&self) -> &'static str { + BOOSTING_QUERY + } + + fn normalize(&mut self, norm: f32, boost: f32) { + self.positive_weight.normalize(norm, boost) + } + + fn value_for_normalization(&self) -> f32 { + self.positive_weight.value_for_normalization() + } + + fn needs_scores(&self) -> bool { + self.positive_weight.needs_scores() + } + + fn explain(&self, reader: &LeafReaderContext<'_, C>, doc: DocId) -> Result { + self.positive_weight.explain(reader, doc) + } +} + + +impl fmt::Display for BoostingWeight { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "BoostingWeight(positive: {}, negative: {}, negative_boost: {})", + &self.positive_weight, &self.negative_weight, self.negative_boost + ) + } +} diff --git a/src/core/search/query/mod.rs b/src/core/search/query/mod.rs index 6d87f13..869a1c2 100644 --- a/src/core/search/query/mod.rs +++ b/src/core/search/query/mod.rs @@ -50,6 +50,10 @@ mod disjunction_max_query; pub use self::disjunction_max_query::*; +mod boosting_query; + +pub use self::boosting_query::*; + use core::codec::Codec; use core::index::reader::LeafReaderContext; use core::search::explanation::Explanation; diff --git a/src/core/search/scorer/boosting_scorer.rs b/src/core/search/scorer/boosting_scorer.rs new file mode 100644 index 0000000..d3d097a --- /dev/null +++ b/src/core/search/scorer/boosting_scorer.rs @@ -0,0 +1,64 @@ +use core::search::scorer::Scorer; +use core::search::DocIterator; +use core::util::DocId; +use error::Result; + +pub struct BoostingScorer { + positive: Box, + negative: Box, + negative_boost: f32, +} + +impl BoostingScorer { + pub fn new(positive: Box, negative: Box, negative_boost: f32) -> BoostingScorer { + assert!(negative_boost > 0.0 && negative_boost < 1.0); + BoostingScorer { + positive, + negative, + negative_boost, + } + } +} + +impl Scorer for BoostingScorer { + fn score(&mut self) -> Result { + let current_doc = self.positive.doc_id(); + let mut score = self.positive.score()?; + + if current_doc == self.negative.advance(current_doc)? { + score *= self.negative_boost; + } + + Ok(score) + } +} + +impl DocIterator for BoostingScorer { + fn doc_id(&self) -> DocId { + self.positive.doc_id() + } + + fn next(&mut self) -> Result { + self.positive.next() + } + + fn advance(&mut self, target: DocId) -> Result { + self.positive.advance(target) + } + + fn cost(&self) -> usize { + self.positive.cost() + } + + fn matches(&mut self) -> Result { + self.positive.matches() + } + + fn approximate_next(&mut self) -> Result { + self.positive.approximate_next() + } + + fn approximate_advance(&mut self, target: DocId) -> Result { + self.positive.approximate_advance(target) + } +} \ No newline at end of file diff --git a/src/core/search/scorer/mod.rs b/src/core/search/scorer/mod.rs index f833730..65ce6b3 100644 --- a/src/core/search/scorer/mod.rs +++ b/src/core/search/scorer/mod.rs @@ -43,6 +43,10 @@ mod term_scorer; pub use self::term_scorer::*; +mod boosting_scorer; + +pub use self::boosting_scorer::*; + use std::collections::HashMap; use std::i32; From bf4e13e82bae8beb586244fa923612a5829b41b3 Mon Sep 17 00:00:00 2001 From: xiezhi Date: Tue, 14 Apr 2020 12:25:57 +0800 Subject: [PATCH 18/49] debug_assert negative_boost --- src/core/search/scorer/boosting_scorer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/core/search/scorer/boosting_scorer.rs b/src/core/search/scorer/boosting_scorer.rs index d3d097a..4ba8da9 100644 --- a/src/core/search/scorer/boosting_scorer.rs +++ b/src/core/search/scorer/boosting_scorer.rs @@ -11,7 +11,7 @@ pub struct BoostingScorer { impl BoostingScorer { pub fn new(positive: Box, negative: Box, negative_boost: f32) -> BoostingScorer { - assert!(negative_boost > 0.0 && negative_boost < 1.0); + debug_assert!(negative_boost > 0.0 && negative_boost < 1.0); BoostingScorer { positive, negative, From 53aa9f63e671b3e1055b78b6e1789097091a7f64 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Tue, 14 Apr 2020 16:41:27 +0800 Subject: [PATCH 19/49] set parallel for small dataset --- src/core/search/collector/top_docs.rs | 5 +-- src/core/search/query/boosting_query.rs | 20 ++++----- src/core/search/scorer/boosting_scorer.rs | 8 +++- src/core/search/searcher.rs | 49 +++++++++++++---------- 4 files changed, 45 insertions(+), 37 deletions(-) diff --git a/src/core/search/collector/top_docs.rs b/src/core/search/collector/top_docs.rs index 15c56a5..077d5ef 100644 --- a/src/core/search/collector/top_docs.rs +++ b/src/core/search/collector/top_docs.rs @@ -65,10 +65,7 @@ impl TopDocsBaseCollector { } fn add_doc(&mut self, doc_id: DocId, score: f32) { - debug_assert!(self.pq.len() <= self.estimated_hits); - let at_capacity = self.pq.len() == self.estimated_hits; - - if !at_capacity { + if self.pq.len() < self.estimated_hits { let score_doc = ScoreDoc::new(doc_id, score); self.pq.push(score_doc); } else if let Some(mut doc) = self.pq.peek_mut() { diff --git a/src/core/search/query/boosting_query.rs b/src/core/search/query/boosting_query.rs index c365f7e..38717dc 100644 --- a/src/core/search/query/boosting_query.rs +++ b/src/core/search/query/boosting_query.rs @@ -5,10 +5,10 @@ use core::codec::Codec; use core::index::reader::LeafReaderContext; use core::search::explanation::Explanation; use core::search::query::{Query, TermQuery, Weight}; +use core::search::scorer::BoostingScorer; use core::search::scorer::Scorer; use core::search::searcher::SearchPlanBuilder; use core::util::DocId; -use core::search::scorer::BoostingScorer; use error::Result; const BOOSTING_QUERY: &str = "boosting"; @@ -39,12 +39,11 @@ impl Query for BoostingQuery { searcher: &dyn SearchPlanBuilder, needs_scores: bool, ) -> Result>> { - Ok(Box::new( - BoostingWeight::new( - self.positive.create_weight(searcher, needs_scores)?, - self.negative.create_weight(searcher, false)?, - self.negative_boost, - ))) + Ok(Box::new(BoostingWeight::new( + self.positive.create_weight(searcher, needs_scores)?, + self.negative.create_weight(searcher, false)?, + self.negative_boost, + ))) } fn extract_terms(&self) -> Vec { @@ -91,8 +90,10 @@ impl Weight for BoostingWeight { &self, leaf_reader: &LeafReaderContext<'_, C>, ) -> Result>> { - if let (Some(positive_scorer), Some(negative_scorer)) = - (self.positive_weight.create_scorer(leaf_reader)?, self.negative_weight.create_scorer(leaf_reader)?) { + if let (Some(positive_scorer), Some(negative_scorer)) = ( + self.positive_weight.create_scorer(leaf_reader)?, + self.negative_weight.create_scorer(leaf_reader)?, + ) { Ok(Some(Box::new(BoostingScorer::new( positive_scorer, negative_scorer, @@ -128,7 +129,6 @@ impl Weight for BoostingWeight { } } - impl fmt::Display for BoostingWeight { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { write!( diff --git a/src/core/search/scorer/boosting_scorer.rs b/src/core/search/scorer/boosting_scorer.rs index 4ba8da9..cbd0e3b 100644 --- a/src/core/search/scorer/boosting_scorer.rs +++ b/src/core/search/scorer/boosting_scorer.rs @@ -10,7 +10,11 @@ pub struct BoostingScorer { } impl BoostingScorer { - pub fn new(positive: Box, negative: Box, negative_boost: f32) -> BoostingScorer { + pub fn new( + positive: Box, + negative: Box, + negative_boost: f32, + ) -> BoostingScorer { debug_assert!(negative_boost > 0.0 && negative_boost < 1.0); BoostingScorer { positive, @@ -61,4 +65,4 @@ impl DocIterator for BoostingScorer { fn approximate_advance(&mut self, target: DocId) -> Result { self.positive.approximate_advance(target) } -} \ No newline at end of file +} diff --git a/src/core/search/searcher.rs b/src/core/search/searcher.rs index 6647d7f..5e90b4b 100644 --- a/src/core/search/searcher.rs +++ b/src/core/search/searcher.rs @@ -279,23 +279,13 @@ pub struct DefaultIndexSearcher< thread_pool: Option>>, // used for concurrent search - each slice holds a set of LeafReader's ord that // executed within one thread. - leaf_ord_slices: Vec, + leaf_ord_slices: Vec>, next_limit: usize, } const MAX_DOCS_PER_SLICE: i32 = 250_000; -const MAX_SEGMENTS_PER_SLICE: usize = 20; - -struct LeafOrdSlice(Vec); - -impl<'a> IntoIterator for &'a LeafOrdSlice { - type Item = &'a usize; - type IntoIter = ::std::slice::Iter<'a, usize>; - - fn into_iter(self) -> Self::IntoIter { - (&self.0).into_iter() - } -} +const MAX_SEGMENTS_PER_SLICE: i32 = 20; +const MIN_PARALLEL_SLICES: i32 = 3; impl + ?Sized, IR: Deref> DefaultIndexSearcher @@ -369,6 +359,7 @@ where self.reader.leaves(), MAX_DOCS_PER_SLICE, MAX_SEGMENTS_PER_SLICE, + MIN_PARALLEL_SLICES, ); } @@ -409,8 +400,9 @@ where fn slice( mut leaves: Vec>, max_docs_per_slice: i32, - max_segments_per_slice: usize, - ) -> Vec { + max_segments_per_slice: i32, + min_parallel_slices: i32, + ) -> Vec> { if leaves.is_empty() { return vec![]; } @@ -421,14 +413,29 @@ where let mut slices = vec![]; let mut doc_sum = 0; let mut ords = vec![]; + + if leaves.len() <= min_parallel_slices as usize { + for ctx in &leaves { + slices.push(vec![ctx.ord]); + } + return slices; + } + + let mut total_docs = 0; + for ctx in &leaves { + total_docs += ctx.reader.max_doc(); + } + + let reserved = max_docs_per_slice.min(total_docs / min_parallel_slices); + for ctx in &leaves { let max_doc = ctx.reader.max_doc(); - if max_doc >= max_docs_per_slice { - slices.push(LeafOrdSlice(vec![ctx.ord])); + if max_doc >= reserved { + slices.push(vec![ctx.ord]); } else { - if doc_sum + max_doc > max_docs_per_slice || ords.len() >= max_segments_per_slice { + if doc_sum + max_doc > reserved || ords.len() >= max_segments_per_slice as usize { ords.sort(); - slices.push(LeafOrdSlice(ords)); + slices.push(ords); ords = vec![]; doc_sum = 0; } @@ -438,7 +445,7 @@ where } if !ords.is_empty() { ords.sort(); - slices.push(LeafOrdSlice(ords)); + slices.push(ords); } slices } @@ -513,7 +520,7 @@ where for leaf_slice in &self.leaf_ord_slices { let mut scorer_and_collectors = vec![]; - for ord in leaf_slice.into_iter() { + for ord in leaf_slice.iter() { let leaf_ctx = &leaf_readers[*ord]; match collector.leaf_collector(leaf_ctx) { Ok(leaf_collector) => { From 0c676568c3b38c4350b00b739ffc6024b618747a Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Fri, 17 Apr 2020 12:35:53 +0800 Subject: [PATCH 20/49] parallelize fetching statistics & posting build directly --- examples/example.rs | 2 +- src/core/index/mod.rs | 2 +- src/core/index/writer/bufferd_updates.rs | 2 +- src/core/search/query/phrase_query.rs | 73 +++---- src/core/search/query/spans/span.rs | 56 +++-- src/core/search/query/spans/span_boost.rs | 32 +-- src/core/search/query/spans/span_near.rs | 33 ++- src/core/search/query/spans/span_or.rs | 26 +-- src/core/search/query/spans/span_term.rs | 67 +++--- src/core/search/query/term_query.rs | 66 +++--- src/core/search/searcher.rs | 199 +++++++++--------- src/core/search/similarity/bm25_similarity.rs | 12 +- src/core/search/statistics.rs | 7 +- 13 files changed, 252 insertions(+), 325 deletions(-) diff --git a/examples/example.rs b/examples/example.rs index 2133348..d783efd 100644 --- a/examples/example.rs +++ b/examples/example.rs @@ -105,7 +105,7 @@ fn main() -> Result<()> { // new index search let reader = writer.get_reader(true, false)?; - let index_searcher = DefaultIndexSearcher::new(Arc::new(reader), None, None); + let index_searcher = DefaultIndexSearcher::new(Arc::new(reader), None); // search let query: TermQuery = TermQuery::new( diff --git a/src/core/index/mod.rs b/src/core/index/mod.rs index 357aaae..e0c86c1 100644 --- a/src/core/index/mod.rs +++ b/src/core/index/mod.rs @@ -157,7 +157,7 @@ pub mod tests { } fn fields(&self) -> Result { - unimplemented!() + bail!("unimplemented") } fn name(&self) -> &str { diff --git a/src/core/index/writer/bufferd_updates.rs b/src/core/index/writer/bufferd_updates.rs index 43266d4..331f53f 100644 --- a/src/core/index/writer/bufferd_updates.rs +++ b/src/core/index/writer/bufferd_updates.rs @@ -564,7 +564,7 @@ impl BufferedUpdatesStream { { let mut del_count: u64 = 0; let mut rld = seg_state.rld.inner.lock()?; - let mut searcher = DefaultIndexSearcher::new(Arc::clone(rld.reader()), None, None); + let mut searcher = DefaultIndexSearcher::new(Arc::clone(rld.reader()), None); let query_cache: Arc> = Arc::new(NoCacheQueryCache::new()); searcher.set_query_cache(query_cache); let reader = searcher.reader().leaves().remove(0); diff --git a/src/core/search/query/phrase_query.rs b/src/core/search/query/phrase_query.rs index 1019526..7dc2853 100644 --- a/src/core/search/query/phrase_query.rs +++ b/src/core/search/query/phrase_query.rs @@ -19,7 +19,7 @@ use std::collections::{HashMap, HashSet}; use std::f32; use std::fmt; -use core::codec::{Codec, CodecTermState}; +use core::codec::Codec; use core::codec::{PostingIterator, PostingIteratorFlags}; use core::codec::{TermIterator, Terms}; use core::doc::Term; @@ -155,20 +155,28 @@ impl Query for PhraseQuery { ); let max_doc = i64::from(searcher.max_doc()); - let mut term_states = Vec::with_capacity(self.terms.len()); let mut term_stats: Vec = Vec::with_capacity(self.terms.len()); for i in 0..self.terms.len() { - let term_context = searcher.term_state(&self.terms[i])?; - - term_stats.push(searcher.term_statistics(&self.terms[i], term_context.as_ref())); - term_states.push(term_context.term_states()); + if needs_scores { + term_stats.push(searcher.term_statistics(&self.terms[i])?); + } else { + term_stats.push(TermStatistics::new( + self.terms[i].bytes.clone(), + max_doc, + -1, + )); + }; } let collection_stats = if needs_scores { - searcher.collections_statistics(&self.field)? + if let Some(stat) = searcher.collections_statistics(&self.field) { + stat.clone() + } else { + CollectionStatistics::new(self.field.clone(), 0, max_doc, -1, -1, -1) + } } else { - CollectionStatistics::new(self.field.clone(), max_doc, -1, -1, -1) + CollectionStatistics::new(self.field.clone(), 0, max_doc, -1, -1, -1) }; let similarity = searcher.similarity(&self.field, needs_scores); @@ -184,7 +192,6 @@ impl Query for PhraseQuery { similarity, sim_weight, needs_scores, - term_states, ))) } @@ -229,7 +236,6 @@ struct PhraseWeight { similarity: Box>, sim_weight: Box>, needs_scores: bool, - term_states: Vec>>, } impl PhraseWeight { @@ -242,7 +248,6 @@ impl PhraseWeight { similarity: Box>, sim_weight: Box>, needs_scores: bool, - term_states: Vec>>, ) -> PhraseWeight { PhraseWeight { field, @@ -252,7 +257,6 @@ impl PhraseWeight { similarity, sim_weight, needs_scores, - term_states, } } @@ -272,14 +276,11 @@ impl PhraseWeight { } impl Weight for PhraseWeight { - fn create_scorer( - &self, - reader_context: &LeafReaderContext<'_, C>, - ) -> Result>> { + fn create_scorer(&self, reader: &LeafReaderContext<'_, C>) -> Result>> { debug_assert!(!self.terms.len() >= 2); let mut postings_freqs = Vec::with_capacity(self.terms.len()); - let mut term_iter = if let Some(field_terms) = reader_context.reader.terms(&self.field)? { + let mut term_iter = if let Some(field_terms) = reader.reader.terms(&self.field)? { debug_assert!( field_terms.has_positions()?, format!( @@ -295,23 +296,17 @@ impl Weight for PhraseWeight { let mut total_match_cost = 0f32; for i in 0..self.terms.len() { - let postings = if let Some(state) = self.term_states[i].get(&reader_context.doc_base) { - term_iter.seek_exact_state(self.terms[i].bytes.as_ref(), state)?; - total_match_cost += self.term_positions_cost(&mut term_iter)?; - - term_iter.postings_with_flags(PostingIteratorFlags::POSITIONS)? - } else { - return Ok(None); - }; + term_iter.seek_exact(self.terms[i].bytes.as_ref())?; + total_match_cost += self.term_positions_cost(&mut term_iter)?; postings_freqs.push(PostingsAndFreq::new( - postings, + term_iter.postings_with_flags(PostingIteratorFlags::POSITIONS)?, self.positions[i], &self.terms[i], )); } - let sim_scorer = self.sim_weight.sim_scorer(reader_context.reader)?; + let sim_scorer = self.sim_weight.sim_scorer(reader.reader)?; let scorer: Box = if self.slop == 0 { // sort by increasing docFreq order // optimize exact case @@ -373,21 +368,15 @@ impl Weight for PhraseWeight { let mut total_match_cost = 0f32; for i in 0..self.terms.len() { - if let Some(state) = self.term_states[i].get(&reader.doc_base()) { - if let Some(ref mut term_iter) = term_iter { - term_iter.seek_exact_state(self.terms[i].bytes.as_ref(), state)?; - total_match_cost += self.term_positions_cost(term_iter)?; - - let postings = - term_iter.postings_with_flags(PostingIteratorFlags::POSITIONS)?; - postings_freqs.push(PostingsAndFreq::new( - postings, - self.positions[i], - &self.terms[i], - )); - } - } else { - matched = false; + if let Some(ref mut term_iter) = term_iter { + term_iter.seek_exact(self.terms[i].bytes.as_ref())?; + total_match_cost += self.term_positions_cost(term_iter)?; + + postings_freqs.push(PostingsAndFreq::new( + term_iter.postings_with_flags(PostingIteratorFlags::POSITIONS)?, + self.positions[i], + &self.terms[i], + )); } } diff --git a/src/core/search/query/spans/span.rs b/src/core/search/query/spans/span.rs index 8918e3b..fc19473 100644 --- a/src/core/search/query/spans/span.rs +++ b/src/core/search/query/spans/span.rs @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use core::codec::{Codec, CodecPostingIterator, CodecTermState}; +use core::codec::{Codec, CodecPostingIterator}; use core::codec::{PostingIterator, PostingIteratorFlags}; use core::doc::Term; use core::index::reader::{LeafReaderContext, SearchLeafReader}; @@ -27,22 +27,18 @@ use core::search::query::{Query, TermQuery, Weight}; use core::search::scorer::{ConjunctionScorer, Scorer}; use core::search::searcher::SearchPlanBuilder; use core::search::similarity::{SimScorer, SimWeight}; -use core::search::TermContext; use core::search::{DocIterator, NO_MORE_DOCS}; use core::util::{DocId, KeyedContext}; use error::{ErrorKind, Result}; -use std::collections::HashMap; +use core::search::statistics::CollectionStatistics; use std::fmt; -use std::sync::Arc; -pub fn term_contexts>( - weights: &[T], -) -> HashMap>>> { - let mut terms = HashMap::new(); +pub fn term_keys>(weights: &[T]) -> Vec { + let mut terms = Vec::new(); for w in weights { - w.extract_term_contexts(&mut terms); + w.extract_term_keys(&mut terms); } terms } @@ -578,19 +574,25 @@ impl DocIterator for SpanScorer { pub fn build_sim_weight + ?Sized>( field: &str, searcher: &IS, - term_contexts: HashMap>>>, + terms: Vec, ctx: Option, ) -> Result>>> { - if field.is_empty() || term_contexts.is_empty() { + if field.is_empty() || terms.is_empty() { return Ok(None); } - let similarity = searcher.similarity(field, !term_contexts.is_empty()); - let mut term_stats = Vec::with_capacity(term_contexts.len()); - for (term, ctx) in term_contexts { - term_stats.push(searcher.term_statistics(&term, ctx.as_ref())); + let similarity = searcher.similarity(field, !terms.is_empty()); + let mut term_stats = Vec::with_capacity(terms.len()); + for term in terms.iter() { + term_stats.push(searcher.term_statistics(term)?); } - let collection_stats = searcher.collections_statistics(field)?; + + let collection_stats = if let Some(stat) = searcher.collections_statistics(field) { + stat.clone() + } else { + CollectionStatistics::new(field.to_string(), 0, searcher.max_doc() as i64, -1, -1, -1) + }; + Ok(Some(similarity.compute_weight( &collection_stats, &term_stats, @@ -612,11 +614,8 @@ pub trait SpanWeight: Weight { required_postings: &PostingsFlag, ) -> Result>>>; - /// Collect all TermContexts used by this Weight - fn extract_term_contexts( - &self, - contexts: &mut HashMap>>>, - ); + /// Collect all Terms used by this Weight + fn extract_term_keys(&self, terms: &mut Vec); fn do_create_scorer(&self, ctx: &LeafReaderContext<'_, C>) -> Result>> { if let Some(spans) = self.get_spans(ctx, &PostingsFlag::Positions)? { @@ -725,16 +724,13 @@ impl SpanWeight for SpanWeightEnum { } } - fn extract_term_contexts( - &self, - contexts: &mut HashMap>>>, - ) { + fn extract_term_keys(&self, terms: &mut Vec) { match self { - SpanWeightEnum::Term(w) => w.extract_term_contexts(contexts), - SpanWeightEnum::Gap(w) => w.extract_term_contexts(contexts), - SpanWeightEnum::Or(w) => w.extract_term_contexts(contexts), - SpanWeightEnum::Near(w) => w.extract_term_contexts(contexts), - SpanWeightEnum::Boost(w) => w.extract_term_contexts(contexts), + SpanWeightEnum::Term(w) => w.extract_term_keys(terms), + SpanWeightEnum::Gap(w) => w.extract_term_keys(terms), + SpanWeightEnum::Or(w) => w.extract_term_keys(terms), + SpanWeightEnum::Near(w) => w.extract_term_keys(terms), + SpanWeightEnum::Boost(w) => w.extract_term_keys(terms), } } diff --git a/src/core/search/query/spans/span_boost.rs b/src/core/search/query/spans/span_boost.rs index 493a654..353f7be 100644 --- a/src/core/search/query/spans/span_boost.rs +++ b/src/core/search/query/spans/span_boost.rs @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use core::codec::{Codec, CodecPostingIterator, CodecTermState}; +use core::codec::{Codec, CodecPostingIterator}; use core::doc::Term; use core::index::reader::{LeafReaderContext, SearchLeafReader}; use core::search::explanation::Explanation; @@ -21,7 +21,6 @@ use core::search::query::spans::{ SpanWeightEnum, SpansEnum, }; use core::search::searcher::SearchPlanBuilder; -use core::search::TermContext; use core::search::{ query::Query, query::TermQuery, query::Weight, scorer::Scorer, similarity::SimScorer, similarity::SimWeight, @@ -31,10 +30,8 @@ use core::util::{DocId, KeyedContext}; use error::Result; use std::any::Any; -use std::collections::HashMap; use std::f32; use std::fmt; -use std::sync::Arc; const SPAN_BOOST_QUERY: &str = "span_boost"; @@ -240,10 +237,9 @@ impl SpanBoostWeight { needs_scores: bool, ) -> Result { let mut weight = query.query.span_weight(searcher, needs_scores)?; - let mut term_contexts = HashMap::new(); - weight.extract_term_contexts(&mut term_contexts); - let sim_weight = - build_sim_weight(SpanQuery::::field(query), searcher, term_contexts, None)?; + let mut terms = Vec::new(); + weight.extract_term_keys(&mut terms); + let sim_weight = build_sim_weight(SpanQuery::::field(query), searcher, terms, None)?; weight.do_normalize(1.0, query.boost); Ok(SpanBoostWeight { sim_weight, @@ -274,11 +270,8 @@ impl SpanWeight for SpanBoostWeight { self.weight.get_spans(reader, required_postings) } - fn extract_term_contexts( - &self, - contexts: &mut HashMap>>>, - ) { - self.weight.extract_term_contexts(contexts) + fn extract_term_keys(&self, terms: &mut Vec) { + self.weight.extract_term_keys(terms) } } @@ -361,15 +354,12 @@ impl SpanWeight for SpanBoostWeightEnum { } } - fn extract_term_contexts( - &self, - contexts: &mut HashMap>>>, - ) { + fn extract_term_keys(&self, terms: &mut Vec) { match self { - SpanBoostWeightEnum::Term(w) => w.extract_term_contexts(contexts), - SpanBoostWeightEnum::Gap(w) => w.extract_term_contexts(contexts), - SpanBoostWeightEnum::Or(w) => w.extract_term_contexts(contexts), - SpanBoostWeightEnum::Near(w) => w.extract_term_contexts(contexts), + SpanBoostWeightEnum::Term(w) => w.extract_term_keys(terms), + SpanBoostWeightEnum::Gap(w) => w.extract_term_keys(terms), + SpanBoostWeightEnum::Or(w) => w.extract_term_keys(terms), + SpanBoostWeightEnum::Near(w) => w.extract_term_keys(terms), } } diff --git a/src/core/search/query/spans/span_near.rs b/src/core/search/query/spans/span_near.rs index fcb04e8..2e7af70 100644 --- a/src/core/search/query/spans/span_near.rs +++ b/src/core/search/query/spans/span_near.rs @@ -11,17 +11,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use core::codec::{Codec, CodecEnum, CodecPostingIterator, CodecTermState}; +use core::codec::{Codec, CodecEnum, CodecPostingIterator}; use core::doc::Term; use core::index::reader::LeafReaderContext; use core::search::explanation::Explanation; use core::search::query::spans::{ build_sim_weight, PostingsFlag, SpanQueryEnum, SpanWeightEnum, SpansEnum, NO_MORE_POSITIONS, }; -use core::search::query::spans::{term_contexts, ConjunctionSpanBase, ConjunctionSpans}; +use core::search::query::spans::{ConjunctionSpanBase, ConjunctionSpans}; use core::search::query::spans::{SpanCollector, SpanQuery, SpanWeight, Spans}; use core::search::searcher::SearchPlanBuilder; -use core::search::TermContext; use core::search::{ query::Query, query::TermQuery, query::Weight, scorer::Scorer, similarity::SimWeight, DocIterator, NO_MORE_DOCS, @@ -31,13 +30,12 @@ use core::util::{DocId, KeyedContext, BM25_SIMILARITY_IDF}; use error::{ErrorKind, Result}; use core::codec::PostingIterator; +use core::search::query::spans::span::term_keys; use std::cmp::Ordering; use std::collections::BinaryHeap; -use std::collections::HashMap; use std::fmt; use std::mem::MaybeUninit; use std::ptr; -use std::sync::Arc; pub struct SpanNearQueryBuilder { ordered: bool, @@ -153,12 +151,12 @@ impl SpanNearQuery { sub_weights.push(clause.span_weight(searcher, needs_scores)?); ctx = Self::merge_idf_ctx(ctx, SpanQuery::::ctx(clause)); } - let term_contexts = if needs_scores { - term_contexts(&sub_weights) + let terms = if needs_scores { + term_keys(&sub_weights) } else { - HashMap::new() + Vec::new() }; - SpanNearWeight::new(self, sub_weights, searcher, term_contexts, ctx) + SpanNearWeight::new(self, sub_weights, searcher, terms, ctx) } } @@ -227,7 +225,7 @@ impl SpanNearWeight { query: &SpanNearQuery, sub_weights: Vec>, searcher: &IS, - terms: HashMap>>>, + terms: Vec, ctx: Option, ) -> Result { let field = SpanQuery::::field(query).to_string(); @@ -284,12 +282,9 @@ impl SpanWeight for SpanNearWeight { Ok(None) } - fn extract_term_contexts( - &self, - contexts: &mut HashMap>>>, - ) { + fn extract_term_keys(&self, terms: &mut Vec) { for weight in &self.sub_weights { - weight.extract_term_contexts(contexts) + weight.extract_term_keys(terms) } } } @@ -928,7 +923,7 @@ impl SpanGapWeight { width: i32, ) -> Result { let sim_weight = - build_sim_weight(SpanQuery::::field(query), searcher, HashMap::new(), None)?; + build_sim_weight(SpanQuery::::field(query), searcher, Vec::new(), None)?; Ok(SpanGapWeight { width, sim_weight }) } } @@ -956,11 +951,7 @@ impl SpanWeight for SpanGapWeight { Ok(Some(SpansEnum::Gap(GapSpans::new(self.width)))) } - fn extract_term_contexts( - &self, - _contexts: &mut HashMap>>>, - ) { - } + fn extract_term_keys(&self, _terms: &mut Vec) {} } impl Weight for SpanGapWeight { diff --git a/src/core/search/query/spans/span_or.rs b/src/core/search/query/spans/span_or.rs index e69081e..9f2f742 100644 --- a/src/core/search/query/spans/span_or.rs +++ b/src/core/search/query/spans/span_or.rs @@ -11,17 +11,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use core::codec::{Codec, CodecEnum, CodecPostingIterator, CodecTermState}; +use core::codec::{Codec, CodecEnum, CodecPostingIterator}; use core::doc::Term; use core::index::reader::LeafReaderContext; use core::search::explanation::Explanation; use core::search::query::spans::{ - build_sim_weight, term_contexts, PostingsFlag, SpanQueryEnum, SpanWeightEnum, SpansEnum, - NO_MORE_POSITIONS, + build_sim_weight, PostingsFlag, SpanQueryEnum, SpanWeightEnum, SpansEnum, NO_MORE_POSITIONS, }; use core::search::query::spans::{SpanCollector, SpanQuery, SpanWeight, Spans}; use core::search::searcher::SearchPlanBuilder; -use core::search::TermContext; use core::search::{ query::Query, query::TermQuery, query::Weight, scorer::Scorer, similarity::SimWeight, DocIterator, @@ -32,11 +30,10 @@ use core::util::DocId; use error::{ErrorKind, Result}; use core::codec::PostingIterator; +use core::search::query::spans::span::term_keys; use std::cmp::{max, Ordering}; use std::collections::BinaryHeap; -use std::collections::HashMap; use std::fmt; -use std::sync::Arc; const SPAN_OR_QUERY: &str = "span_or"; @@ -74,12 +71,12 @@ impl SpanOrQuery { for clause in &self.clauses { sub_weights.push(clause.span_weight(searcher, needs_scores)?); } - let term_contexts = if needs_scores { - term_contexts(&sub_weights) + let terms = if needs_scores { + term_keys(&sub_weights) } else { - HashMap::new() + Vec::new() }; - SpanOrWeight::new(self, sub_weights, searcher, term_contexts) + SpanOrWeight::new(self, sub_weights, searcher, terms) } } @@ -144,7 +141,7 @@ impl SpanOrWeight { query: &SpanOrQuery, sub_weights: Vec>, searcher: &IS, - terms: HashMap>>>, + terms: Vec, ) -> Result { assert!(sub_weights.len() >= 2); let sim_weight = build_sim_weight(SpanQuery::::field(query), searcher, terms, None)?; @@ -192,12 +189,9 @@ impl SpanWeight for SpanOrWeight { )))) } - fn extract_term_contexts( - &self, - contexts: &mut HashMap>>>, - ) { + fn extract_term_keys(&self, terms: &mut Vec) { for spans in &self.sub_weights { - spans.extract_term_contexts(contexts) + spans.extract_term_keys(terms) } } } diff --git a/src/core/search/query/spans/span_term.rs b/src/core/search/query/spans/span_term.rs index 4eb67c5..f1abb9a 100644 --- a/src/core/search/query/spans/span_term.rs +++ b/src/core/search/query/spans/span_term.rs @@ -12,7 +12,7 @@ // limitations under the License. use core::codec::PostingIterator; -use core::codec::{Codec, CodecPostingIterator, CodecTermState}; +use core::codec::{Codec, CodecPostingIterator}; use core::codec::{TermIterator, Terms}; use core::doc::Term; use core::index::reader::LeafReaderContext; @@ -20,7 +20,6 @@ use core::search::explanation::Explanation; use core::search::query::spans::{build_sim_weight, PostingsFlag, SpansEnum, NO_MORE_POSITIONS}; use core::search::query::spans::{SpanCollector, SpanQuery, SpanWeight, Spans}; use core::search::searcher::SearchPlanBuilder; -use core::search::TermContext; use core::search::{ query::Query, query::TermQuery, query::Weight, scorer::Scorer, similarity::SimWeight, DocIterator, NO_MORE_DOCS, @@ -29,9 +28,7 @@ use core::util::{DocId, KeyedContext}; use error::{ErrorKind, Result}; -use std::collections::HashMap; use std::fmt; -use std::sync::Arc; const SPAN_TERM_QUERY: &str = "span_term"; @@ -55,10 +52,8 @@ impl Query for SpanTermQuery { searcher: &dyn SearchPlanBuilder, needs_scores: bool, ) -> Result>> { - let term_context = searcher.term_state(&self.term)?; Ok(Box::new(SpanTermWeight::new( self, - term_context, searcher, self.ctx.clone(), needs_scores, @@ -86,10 +81,8 @@ impl SpanQuery for SpanTermQuery { searcher: &dyn SearchPlanBuilder, needs_scores: bool, ) -> Result { - let term_context = searcher.term_state(&self.term)?; Ok(SpanTermWeight::new( self, - term_context, searcher, self.ctx.clone(), needs_scores, @@ -224,26 +217,20 @@ const TERM_OPS_PER_POS: i32 = 7; pub struct SpanTermWeight { term: Term, sim_weight: Option>>, - term_context: Arc>>, } impl SpanTermWeight { pub fn new + ?Sized>( query: &SpanTermQuery, - term_context: Arc>>, searcher: &IS, ctx: Option, - needs_scores: bool, + _needs_scores: bool, ) -> Result { - let mut term_contexts = HashMap::new(); - if needs_scores { - term_contexts.insert(query.term.clone(), Arc::clone(&term_context)); - } - let sim_weight = build_sim_weight(query.term.field(), searcher, term_contexts, ctx)?; + let sim_weight = + build_sim_weight(query.term.field(), searcher, vec![query.term.clone()], ctx)?; Ok(SpanTermWeight { term: query.term.clone(), sim_weight, - term_context, }) } @@ -279,37 +266,31 @@ impl SpanWeight for SpanTermWeight { reader: &LeafReaderContext<'_, C>, required_postings: &PostingsFlag, ) -> Result>>> { - if let Some(state) = self.term_context.get_term_state(reader) { - if let Some(terms) = reader.reader.terms(self.term.field())? { - if !terms.has_positions()? { - bail!(ErrorKind::IllegalState(format!( - "field '{}' was indexed without position data; cannot run SpanTermQuery \ - (term={:?})", - &self.term.field, - &self.term.text() - ))); - } - let mut terms_iter = terms.iterator()?; - terms_iter.seek_exact_state(&self.term.bytes, state)?; - let postings = - terms_iter.postings_with_flags(required_postings.required_postings())?; - let positions_cost = Self::term_positions_cost(&mut terms_iter)? - + PHRASE_TO_SPAN_TERM_POSITIONS_COST; - return Ok(Some(SpansEnum::Term(TermSpans::new( - postings, - self.term.clone(), - positions_cost, - )))); + if let Some(terms) = reader.reader.terms(self.term.field())? { + if !terms.has_positions()? { + bail!(ErrorKind::IllegalState(format!( + "field '{}' was indexed without position data; cannot run SpanTermQuery \ + (term={:?})", + &self.term.field, + &self.term.text() + ))); } + let mut terms_iter = terms.iterator()?; + terms_iter.seek_exact(&self.term.bytes)?; + let postings = terms_iter.postings_with_flags(required_postings.required_postings())?; + let positions_cost = + Self::term_positions_cost(&mut terms_iter)? + PHRASE_TO_SPAN_TERM_POSITIONS_COST; + return Ok(Some(SpansEnum::Term(TermSpans::new( + postings, + self.term.clone(), + positions_cost, + )))); } Ok(None) } - fn extract_term_contexts( - &self, - contexts: &mut HashMap>>>, - ) { - contexts.insert(self.term.clone(), Arc::clone(&self.term_context)); + fn extract_term_keys(&self, terms: &mut Vec) { + terms.push(self.term.clone()); } } diff --git a/src/core/search/query/term_query.rs b/src/core/search/query/term_query.rs index ba2de09..01831c4 100644 --- a/src/core/search/query/term_query.rs +++ b/src/core/search/query/term_query.rs @@ -13,10 +13,9 @@ use error::Result; -use std::collections::HashMap; use std::fmt; -use core::codec::{Codec, CodecPostingIterator, CodecTermState}; +use core::codec::Codec; use core::codec::{PostingIterator, PostingIteratorFlags}; use core::doc::Term; use core::index::reader::LeafReaderContext; @@ -61,19 +60,24 @@ impl Query for TermQuery { searcher: &dyn SearchPlanBuilder, needs_scores: bool, ) -> Result>> { - let term_context = searcher.term_state(&self.term)?; - let max_doc = i64::from(searcher.max_doc()); - let (term_stats, collection_stats) = if needs_scores { - ( - vec![searcher.term_statistics(&self.term, term_context.as_ref())], - searcher.collections_statistics(&self.term.field)?, - ) + let max_doc = searcher.max_doc() as i64; + + let term_stats = if needs_scores { + vec![searcher.term_statistics(&self.term)?] } else { - ( - vec![TermStatistics::new(self.term.bytes.clone(), max_doc, -1)], - CollectionStatistics::new(self.term.field.clone(), max_doc, -1, -1, -1), - ) + vec![TermStatistics::new(self.term.bytes.clone(), max_doc, -1)] }; + + let collection_stats = if needs_scores { + if let Some(stat) = searcher.collections_statistics(&self.term.field) { + stat.clone() + } else { + CollectionStatistics::new(self.term.field.clone(), 0, max_doc, -1, -1, -1) + } + } else { + CollectionStatistics::new(self.term.field.clone(), 0, max_doc, -1, -1, -1) + }; + let similarity = searcher.similarity(&self.term.field, needs_scores); let sim_weight = similarity.compute_weight( &collection_stats, @@ -83,7 +87,6 @@ impl Query for TermQuery { ); Ok(Box::new(TermWeight::new( self.term.clone(), - term_context.term_states(), self.boost, similarity, sim_weight, @@ -118,13 +121,11 @@ struct TermWeight { similarity: Box>, sim_weight: Box>, needs_scores: bool, - term_states: HashMap>, } impl TermWeight { pub fn new( term: Term, - term_states: HashMap>, boost: f32, similarity: Box>, sim_weight: Box>, @@ -136,30 +137,14 @@ impl TermWeight { similarity, sim_weight, needs_scores, - term_states, - } - } - - fn create_postings_iterator( - &self, - reader: &LeafReaderContext<'_, C>, - flags: i32, - ) -> Result>> { - if let Some(state) = self.term_states.get(&reader.doc_base) { - reader.reader.postings_from_state(&self.term, &state, flags) - } else { - Ok(None) } } } impl Weight for TermWeight { - fn create_scorer( - &self, - reader_context: &LeafReaderContext<'_, C>, - ) -> Result>> { - let _norms = reader_context.reader.norm_values(&self.term.field); - let sim_scorer = self.sim_weight.sim_scorer(reader_context.reader)?; + fn create_scorer(&self, reader: &LeafReaderContext<'_, C>) -> Result>> { + let _norms = reader.reader.norm_values(&self.term.field); + let sim_scorer = self.sim_weight.sim_scorer(reader.reader)?; let flags = if self.needs_scores { PostingIteratorFlags::FREQS @@ -167,8 +152,11 @@ impl Weight for TermWeight { PostingIteratorFlags::NONE }; - if let Some(postings) = self.create_postings_iterator(reader_context, i32::from(flags))? { - Ok(Some(Box::new(TermScorer::new(sim_scorer, postings)))) + if let Some(postings_iterator) = reader.reader.postings(&self.term, flags as i32)? { + Ok(Some(Box::new(TermScorer::new( + sim_scorer, + postings_iterator, + )))) } else { Ok(None) } @@ -197,9 +185,7 @@ impl Weight for TermWeight { PostingIteratorFlags::NONE }; - if let Some(mut postings_iterator) = - self.create_postings_iterator(reader, i32::from(flags))? - { + if let Some(mut postings_iterator) = reader.reader.postings(&self.term, flags as i32)? { let new_doc = postings_iterator.advance(doc)?; if new_doc == doc { let freq = postings_iterator.freq()? as f32; diff --git a/src/core/search/searcher.rs b/src/core/search/searcher.rs index 5e90b4b..8f233c5 100644 --- a/src/core/search/searcher.rs +++ b/src/core/search/searcher.rs @@ -14,14 +14,14 @@ use std::collections::HashMap; use std::fmt; use std::ops::Deref; -use std::sync::{Arc, RwLock}; +use std::sync::Arc; use crossbeam::channel::{unbounded, Receiver, Sender}; use core::codec::postings::FieldsProducer; -use core::codec::{get_terms, TermIterator, TermState}; -use core::codec::{Codec, CodecTermState, Terms}; -use core::doc::Term; +use core::codec::{Codec, Terms}; +use core::codec::{TermIterator, TermState}; +use core::doc::{IndexOptions, Term}; use core::index::reader::{IndexReader, LeafReaderContext, LeafReaderContextPtr, SearchLeafReader}; use core::search::cache::{ LRUQueryCache, QueryCache, QueryCachingPolicy, UsageTrackingQueryCachingPolicy, @@ -40,6 +40,12 @@ use core::util::{Bits, DocId, KeyedContext}; use error::{Error, ErrorKind, Result}; +const MAX_DOCS_PER_SLICE: i32 = 250_000; +const MAX_SEGMENTS_PER_SLICE: i32 = 20; +const MIN_PARALLEL_SLICES: i32 = 3; + +const DEFAULT_DISMATCH_NEXT_LIMIT: usize = 500_000; + pub struct TermContext { pub doc_freq: i32, pub total_term_freq: i64, @@ -220,15 +226,9 @@ pub trait SearchPlanBuilder { fn similarity(&self, field: &str, needs_scores: bool) -> Box>; - fn term_state(&self, term: &Term) -> Result>>>; - - fn term_statistics( - &self, - term: &Term, - context: &TermContext>, - ) -> TermStatistics; + fn term_statistics(&self, term: &Term) -> Result; - fn collections_statistics(&self, field: &str) -> Result; + fn collections_statistics(&self, field: &str) -> Option<&CollectionStatistics>; } pub trait IndexSearcher: SearchPlanBuilder { @@ -270,37 +270,29 @@ pub struct DefaultIndexSearcher< SP: SimilarityProducer, > { reader: IR, - sim_producer: SP, - query_cache: Arc>, - cache_policy: Arc>, - collection_statistics: RwLock>, - term_contexts: RwLock>>>>, - term_contexts_limit: usize, thread_pool: Option>>, // used for concurrent search - each slice holds a set of LeafReader's ord that // executed within one thread. leaf_ord_slices: Vec>, + + query_cache: Arc>, + cache_policy: Arc>, + + sim_producer: SP, + collection_statistics: HashMap, + + // dismatch next limit to break. next_limit: usize, } -const MAX_DOCS_PER_SLICE: i32 = 250_000; -const MAX_SEGMENTS_PER_SLICE: i32 = 20; -const MIN_PARALLEL_SLICES: i32 = 3; - impl + ?Sized, IR: Deref> DefaultIndexSearcher { pub fn new( reader: IR, - term_contexts_limit: Option, next_limit: Option, ) -> DefaultIndexSearcher { - Self::with_similarity( - reader, - DefaultSimilarityProducer {}, - term_contexts_limit, - next_limit, - ) + Self::with_similarity(reader, DefaultSimilarityProducer {}, next_limit) } } @@ -314,32 +306,59 @@ where pub fn with_similarity( reader: IR, sim_producer: SP, - term_contexts_limit: Option, next_limit: Option, ) -> DefaultIndexSearcher { - let term_contexts_limit = if term_contexts_limit.is_some() { - term_contexts_limit.unwrap() - } else { - 1_000_000 - }; + let mut leaves = reader.leaves(); + leaves.sort_by(|l1, l2| l2.reader.max_doc().cmp(&l1.reader.max_doc())); - let next_limit = if next_limit.is_some() { - next_limit.unwrap() - } else { - 500_000 - }; + let mut collection_statistics: HashMap = HashMap::new(); + + for leaf_reader in leaves.iter() { + for (field_name, field_info) in &leaf_reader.reader.field_infos().by_name { + // use top-max-doc segment instead. + if field_info.index_options == IndexOptions::Null + || collection_statistics.contains_key(field_name) + { + continue; + } + + let mut doc_count = 0i32; + let mut sum_doc_freq = 0i64; + let mut sum_total_term_freq = 0i64; + + if let Ok(Some(terms)) = leaf_reader.reader.terms(field_name) { + if let Ok(dc) = terms.doc_count() { + doc_count = dc; + } + if let Ok(s) = terms.sum_doc_freq() { + sum_doc_freq = s; + } + if let Ok(s) = terms.sum_total_term_freq() { + sum_total_term_freq = s; + } + } + let field_stat = CollectionStatistics::new( + field_name.clone(), + leaf_reader.doc_base(), + reader.max_doc() as i64, + doc_count as i64, + sum_total_term_freq, + sum_doc_freq, + ); + + collection_statistics.insert(field_name.clone(), field_stat); + } + } DefaultIndexSearcher { reader, sim_producer, query_cache: Arc::new(LRUQueryCache::new(1000)), cache_policy: Arc::new(UsageTrackingQueryCachingPolicy::default()), - collection_statistics: RwLock::new(HashMap::new()), - term_contexts: RwLock::new(HashMap::with_capacity(term_contexts_limit * 2)), - term_contexts_limit, + collection_statistics, thread_pool: None, leaf_ord_slices: vec![], - next_limit, + next_limit: next_limit.unwrap_or(DEFAULT_DISMATCH_NEXT_LIMIT), } } @@ -674,7 +693,8 @@ where needs_scores: bool, ) -> Result>> { let mut weight = query.create_weight(self, needs_scores)?; - if !needs_scores { + // currently not to use query_cache. + if false && !needs_scores { weight = self .query_cache .do_cache(weight, Arc::clone(&self.cache_policy)); @@ -709,70 +729,45 @@ where } } - fn term_state(&self, term: &Term) -> Result>>> { - let term_context: Arc>>; - let mut builded = false; - let term_key = format!("{}_{}", term.field, term.text()?); - if self.term_contexts.read().unwrap().contains_key(&term_key) { - builded = true; - } - - if builded { - term_context = Arc::clone(self.term_contexts.read().unwrap().get(&term_key).unwrap()); + fn term_statistics(&self, term: &Term) -> Result { + let doc_base = if let Some(field_stat) = self.collection_statistics.get(&term.field) { + field_stat.doc_base } else { - let mut context = TermContext::new(&*self.reader); - context.build(&*self.reader, &term)?; - term_context = Arc::new(context); - if self.term_contexts.read().unwrap().len() < self.term_contexts_limit { - self.term_contexts - .write() - .unwrap() - .insert(term_key.clone(), Arc::clone(&term_context)); - } + return Ok(TermStatistics::new( + term.bytes.clone(), + self.reader.max_doc() as i64, + -1, + )); }; - Ok(term_context) - } + let mut doc_freq = 0; + let mut total_term_freq = 0; - fn term_statistics( - &self, - term: &Term, - context: &TermContext>, - ) -> TermStatistics { - TermStatistics::new( - term.bytes.clone(), - context.doc_freq as i64, - context.total_term_freq, - ) - } + for leaf_reader in self.reader.leaves() { + if leaf_reader.doc_base() < doc_base { + continue; + } else if leaf_reader.doc_base() > doc_base { + break; + } - fn collections_statistics(&self, field: &str) -> Result { - { - let statistics = self.collection_statistics.read().unwrap(); - if let Some(stat) = statistics.get(field) { - return Ok(stat.clone()); + if let Some(terms) = leaf_reader.reader.terms(&term.field)? { + let mut terms_enum = terms.iterator()?; + if terms_enum.seek_exact(&term.bytes)? { + doc_freq = terms_enum.doc_freq()?; + total_term_freq = terms_enum.total_term_freq()?; + } } } - // slow path - let mut doc_count = 0i32; - let mut sum_total_term_freq = 0i64; - let mut sum_doc_freq = 0i64; - if let Some(terms) = get_terms(&*self.reader, field)? { - doc_count = terms.doc_count()?; - sum_total_term_freq = terms.sum_total_term_freq()?; - sum_doc_freq = terms.sum_doc_freq()?; - } - let stat = CollectionStatistics::new( - field.into(), - self.reader.max_doc() as i64, - doc_count as i64, - sum_total_term_freq, - sum_doc_freq, - ); - let mut statistics = self.collection_statistics.write().unwrap(); - statistics.insert(field.into(), stat); - Ok(statistics[field].clone()) + Ok(TermStatistics::new( + term.bytes.clone(), + doc_freq as i64, + total_term_freq, + )) + } + + fn collections_statistics(&self, field: &str) -> Option<&CollectionStatistics> { + self.collection_statistics.get(field) } } @@ -938,7 +933,7 @@ mod tests { ChainedCollector::new(&mut early_terminating_collector, &mut top_collector); let query = MockQuery::new(vec![1, 5, 3, 4, 2]); { - let searcher = DefaultIndexSearcher::new(index_reader, None, None); + let searcher = DefaultIndexSearcher::new(index_reader, None); searcher.search(&query, &mut chained_collector).unwrap(); } } diff --git a/src/core/search/similarity/bm25_similarity.rs b/src/core/search/similarity/bm25_similarity.rs index 11119d3..dd882d8 100644 --- a/src/core/search/similarity/bm25_similarity.rs +++ b/src/core/search/similarity/bm25_similarity.rs @@ -412,14 +412,14 @@ mod tests { #[test] fn test_idf() { - let collection_stats = CollectionStatistics::new(String::from("world"), 11, -1, 0, 0); + let collection_stats = CollectionStatistics::new(String::from("world"), 0, 11, -1, 0, 0); let term_stats = vec![TermStatistics::new(Vec::new(), 1, -1)]; assert!( (BM25Similarity::idf(&term_stats, &collection_stats) - (8f32).ln()) < ::std::f32::EPSILON ); - let collection_stats = CollectionStatistics::new(String::from("world"), 35, 32, -1, -1); + let collection_stats = CollectionStatistics::new(String::from("world"), 0, 35, 32, -1, -1); let term_stats = vec![TermStatistics::new(Vec::new(), 1, -1)]; assert!( (BM25Similarity::idf(&term_stats, &collection_stats) - (22f32).ln()) @@ -429,19 +429,19 @@ mod tests { #[test] fn test_avg_field_length() { - let collection_stats = CollectionStatistics::new(String::from("world"), 11, 5, 0, -1); + let collection_stats = CollectionStatistics::new(String::from("world"), 0, 11, 5, 0, -1); assert!((BM25Similarity::avg_field_length(&collection_stats) - 1f32) < ::std::f32::EPSILON); - let collection_stats = CollectionStatistics::new(String::from("world"), 3, 2, 8, -1); + let collection_stats = CollectionStatistics::new(String::from("world"), 0, 3, 2, 8, -1); assert!((BM25Similarity::avg_field_length(&collection_stats) - 4f32) < ::std::f32::EPSILON); - let collection_stats = CollectionStatistics::new(String::from("world"), 3, -1, 9, -1); + let collection_stats = CollectionStatistics::new(String::from("world"), 0, 3, -1, 9, -1); assert!((BM25Similarity::avg_field_length(&collection_stats) - 3f32) < ::std::f32::EPSILON); } #[test] fn test_bm25_similarity() { - let collection_stats = CollectionStatistics::new(String::from("world"), 32, 32, 120, -1); + let collection_stats = CollectionStatistics::new(String::from("world"), 0, 32, 32, 120, -1); let term_stats = vec![TermStatistics::new(Vec::new(), 1, -1)]; let bm25_sim = BM25Similarity::new(1.2, 0.75); let sim_weight = bm25_sim.compute_weight(&collection_stats, &term_stats, None, 1.0f32); diff --git a/src/core/search/statistics.rs b/src/core/search/statistics.rs index b0c2e6e..a2420cc 100644 --- a/src/core/search/statistics.rs +++ b/src/core/search/statistics.rs @@ -11,10 +11,13 @@ // See the License for the specific language governing permissions and // limitations under the License. +use core::util::DocId; + /// Contains statistics for a collection (field) #[derive(Clone)] pub struct CollectionStatistics { pub field: String, + pub doc_base: DocId, pub max_doc: i64, pub doc_count: i64, pub sum_total_term_freq: i64, @@ -24,6 +27,7 @@ pub struct CollectionStatistics { impl CollectionStatistics { pub fn new( field: String, + doc_base: DocId, max_doc: i64, doc_count: i64, sum_total_term_freq: i64, @@ -35,6 +39,7 @@ impl CollectionStatistics { debug_assert!(sum_total_term_freq == -1 || sum_total_term_freq >= sum_doc_freq); // #positions must be >= #postings CollectionStatistics { field, + doc_base, max_doc, doc_count, sum_total_term_freq, @@ -71,7 +76,7 @@ mod tests { #[test] fn test_collection_statistics() { let collection_statistics = - CollectionStatistics::new(String::from("hello"), 25, 10, 14, 13); + CollectionStatistics::new(String::from("hello"), 0, 25, 10, 14, 13); assert_eq!(collection_statistics.field, "hello"); assert_eq!(collection_statistics.max_doc, 25); assert_eq!(collection_statistics.doc_count, 10); From f976d241d3724d7df2050e6621e20345dfd65f8a Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Mon, 20 Apr 2020 13:51:57 +0800 Subject: [PATCH 21/49] recover simple queue for disi --- src/core/search/scorer/disjunction_scorer.rs | 292 ++++++++++++------- 1 file changed, 189 insertions(+), 103 deletions(-) diff --git a/src/core/search/scorer/disjunction_scorer.rs b/src/core/search/scorer/disjunction_scorer.rs index 65ed747..749adbc 100644 --- a/src/core/search/scorer/disjunction_scorer.rs +++ b/src/core/search/scorer/disjunction_scorer.rs @@ -12,7 +12,7 @@ // limitations under the License. use core::search::scorer::Scorer; -use core::search::DocIterator; +use core::search::{DocIterator, NO_MORE_DOCS}; use core::util::{DisiPriorityQueue, DocId}; use error::Result; @@ -20,7 +20,7 @@ use std::f32; /// A Scorer for OR like queries, counterpart of `ConjunctionScorer`. pub struct DisjunctionSumScorer { - sub_scorers: DisiPriorityQueue, + sub_scorers: SubScorers, needs_scores: bool, cost: usize, } @@ -31,54 +31,33 @@ impl DisjunctionSumScorer { let cost = children.iter().map(|w| w.cost()).sum(); + let sub_scorers = if children.len() < 10 { + SubScorers::SQ(SimpleQueue::new(children)) + } else { + SubScorers::DPQ(DisiPriorityQueue::new(children)) + }; + DisjunctionSumScorer { - sub_scorers: DisiPriorityQueue::new(children), + sub_scorers, needs_scores, cost, } } - - pub fn sub_scorers(&self) -> &DisiPriorityQueue { - &self.sub_scorers - } - - pub fn sub_scorers_mut(&mut self) -> &mut DisiPriorityQueue { - &mut self.sub_scorers - } - - pub fn get_cost(&self) -> usize { - self.cost - } } impl Scorer for DisjunctionSumScorer { fn score(&mut self) -> Result { - let mut score: f32 = 0.0f32; - if !self.needs_scores { - return Ok(score); + return Ok(0.0f32); } - let mut disi = self.sub_scorers_mut().top_list(); - - loop { - let sub_score = disi.inner_mut().score()?; - score += sub_score; - - if disi.next.is_null() { - break; - } else { - unsafe { disi = &mut *disi.next }; - } - } - - Ok(score) + self.sub_scorers.score_sum() } } impl DocIterator for DisjunctionSumScorer { fn doc_id(&self) -> DocId { - self.sub_scorers().peek().doc() + self.sub_scorers.doc_id() } fn next(&mut self) -> Result { @@ -102,29 +81,11 @@ impl DocIterator for DisjunctionSumScorer { } fn approximate_next(&mut self) -> Result { - let sub_scorers = self.sub_scorers_mut(); - let doc = sub_scorers.peek().doc(); - - loop { - sub_scorers.peek_mut().approximate_next()?; - if sub_scorers.peek().doc() != doc { - break; - } - } - - Ok(sub_scorers.peek().doc()) + self.sub_scorers.approximate_next() } fn approximate_advance(&mut self, target: DocId) -> Result { - let sub_scorers = self.sub_scorers_mut(); - loop { - sub_scorers.peek_mut().approximate_advance(target)?; - if sub_scorers.peek().doc() >= target { - break; - } - } - - Ok(sub_scorers.peek().doc()) + self.sub_scorers.approximate_advance(target) } } @@ -134,7 +95,7 @@ impl DocIterator for DisjunctionSumScorer { /// tieBreakerMultiplier times the sum of the scores for the other subqueries that generate the /// document. pub struct DisjunctionMaxScorer { - sub_scorers: DisiPriorityQueue, + sub_scorers: SubScorers, needs_scores: bool, cost: usize, tie_breaker_multiplier: f32, @@ -150,59 +111,34 @@ impl DisjunctionMaxScorer { let cost = children.iter().map(|w| w.cost()).sum(); + let sub_scorers = if children.len() < 10 { + SubScorers::SQ(SimpleQueue::new(children)) + } else { + SubScorers::DPQ(DisiPriorityQueue::new(children)) + }; + DisjunctionMaxScorer { - sub_scorers: DisiPriorityQueue::new(children), + sub_scorers, needs_scores, cost, tie_breaker_multiplier, } } - - pub fn sub_scorers(&self) -> &DisiPriorityQueue { - &self.sub_scorers - } - - pub fn sub_scorers_mut(&mut self) -> &mut DisiPriorityQueue { - &mut self.sub_scorers - } - - pub fn get_cost(&self) -> usize { - self.cost - } } impl Scorer for DisjunctionMaxScorer { fn score(&mut self) -> Result { - let mut score_sum = 0.0f32; - if !self.needs_scores { - return Ok(score_sum); - } - - let mut score_max = f32::NEG_INFINITY; - let mut disi = self.sub_scorers_mut().top_list(); - - loop { - let sub_score = disi.inner_mut().score()?; - score_sum += sub_score; - if sub_score > score_max { - score_max = sub_score; - } - - if disi.next.is_null() { - break; - } else { - unsafe { disi = &mut *disi.next }; - } + return Ok(0.0f32); } - Ok(score_max + (score_sum - score_max) * self.tie_breaker_multiplier) + self.sub_scorers.score_max(self.tie_breaker_multiplier) } } impl DocIterator for DisjunctionMaxScorer { fn doc_id(&self) -> DocId { - self.sub_scorers().peek().doc() + self.sub_scorers.doc_id() } fn next(&mut self) -> Result { @@ -226,28 +162,178 @@ impl DocIterator for DisjunctionMaxScorer { } fn approximate_next(&mut self) -> Result { - let sub_scorers = self.sub_scorers_mut(); - let doc = sub_scorers.peek().doc(); + self.sub_scorers.approximate_next() + } + + fn approximate_advance(&mut self, target: DocId) -> Result { + self.sub_scorers.approximate_advance(target) + } +} + +pub struct SimpleQueue { + scorers: Vec, + curr_doc: DocId, +} + +impl SimpleQueue { + pub fn new(children: Vec) -> SimpleQueue { + let mut curr_doc = NO_MORE_DOCS; + for s in children.iter() { + curr_doc = curr_doc.min(s.doc_id()); + } + SimpleQueue { + scorers: children, + curr_doc, + } + } +} + +pub enum SubScorers { + SQ(SimpleQueue), + DPQ(DisiPriorityQueue), +} + +impl SubScorers { + fn score_sum(&mut self) -> Result { + match self { + SubScorers::SQ(sq) => { + let mut score: f32 = 0.0f32; + + let doc_id = sq.curr_doc; + for s in sq.scorers.iter_mut() { + if s.doc_id() == doc_id { + let sub_score = s.score()?; + score += sub_score; + } + } + + Ok(score) + } + SubScorers::DPQ(dpq) => { + let mut score: f32 = 0.0f32; + let mut disi = dpq.top_list(); + + loop { + let sub_score = disi.inner_mut().score()?; + score += sub_score; + + if disi.next.is_null() { + break; + } else { + unsafe { disi = &mut *disi.next }; + } + } + + Ok(score) + } + } + } + + fn score_max(&mut self, tie_breaker_multiplier: f32) -> Result { + match self { + SubScorers::SQ(sq) => { + let mut score_sum = 0.0f32; + let mut score_max = f32::NEG_INFINITY; + + let doc_id = sq.curr_doc; + for s in sq.scorers.iter_mut() { + if s.doc_id() == doc_id { + let sub_score = s.score()?; - loop { - sub_scorers.peek_mut().approximate_next()?; - if sub_scorers.peek().doc() != doc { - break; + score_sum += sub_score; + score_max = score_max.max(sub_score); + } + } + + Ok(score_max + (score_sum - score_max) * tie_breaker_multiplier) + } + SubScorers::DPQ(dbq) => { + let mut score_sum = 0.0f32; + let mut score_max = f32::NEG_INFINITY; + let mut disi = dbq.top_list(); + + loop { + let sub_score = disi.inner_mut().score()?; + score_sum += sub_score; + if sub_score > score_max { + score_max = sub_score; + } + + if disi.next.is_null() { + break; + } else { + unsafe { disi = &mut *disi.next }; + } + } + + Ok(score_max + (score_sum - score_max) * tie_breaker_multiplier) } } + } - Ok(sub_scorers.peek().doc()) + fn doc_id(&self) -> DocId { + match self { + SubScorers::SQ(sq) => sq.curr_doc, + SubScorers::DPQ(dbq) => dbq.peek().doc(), + } } - fn approximate_advance(&mut self, target: DocId) -> Result { - let sub_scorers = self.sub_scorers_mut(); - loop { - sub_scorers.peek_mut().approximate_advance(target)?; - if sub_scorers.peek().doc() >= target { - break; + fn approximate_next(&mut self) -> Result { + match self { + SubScorers::SQ(sq) => { + let curr_doc = sq.curr_doc; + let mut min_doc = NO_MORE_DOCS; + for s in sq.scorers.iter_mut() { + if s.doc_id() == curr_doc { + s.approximate_next()?; + } + + min_doc = min_doc.min(s.doc_id()); + } + + sq.curr_doc = min_doc; + Ok(sq.curr_doc) + } + SubScorers::DPQ(dbq) => { + let doc = dbq.peek().doc(); + + loop { + dbq.peek_mut().approximate_next()?; + if dbq.peek().doc() != doc { + break; + } + } + + Ok(dbq.peek().doc()) } } + } - Ok(sub_scorers.peek().doc()) + fn approximate_advance(&mut self, target: DocId) -> Result { + match self { + SubScorers::SQ(sq) => { + let mut min_doc = NO_MORE_DOCS; + for s in sq.scorers.iter_mut() { + if s.doc_id() < target { + s.approximate_advance(target)?; + } + + min_doc = min_doc.min(s.doc_id()); + } + + sq.curr_doc = min_doc; + Ok(sq.curr_doc) + } + SubScorers::DPQ(dbq) => { + loop { + dbq.peek_mut().approximate_advance(target)?; + if dbq.peek().doc() >= target { + break; + } + } + + Ok(dbq.peek().doc()) + } + } } } From 3725c3b4c4059d39c6231e9b2b84f50b980d8988 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Fri, 9 Oct 2020 13:29:46 +0800 Subject: [PATCH 22/49] exist query --- .../codec/doc_values/doc_values_iterator.rs | 146 ++++++++++++++++++ src/core/codec/doc_values/mod.rs | 4 + src/core/search/query/exists_query.rs | 137 ++++++++++++++++ src/core/search/query/mod.rs | 4 + 4 files changed, 291 insertions(+) create mode 100644 src/core/codec/doc_values/doc_values_iterator.rs create mode 100644 src/core/search/query/exists_query.rs diff --git a/src/core/codec/doc_values/doc_values_iterator.rs b/src/core/codec/doc_values/doc_values_iterator.rs new file mode 100644 index 0000000..a53f4ff --- /dev/null +++ b/src/core/codec/doc_values/doc_values_iterator.rs @@ -0,0 +1,146 @@ +use core::codec::Codec; +use core::codec::doc_values::{ + BinaryDocValues, NumericDocValues, + SortedDocValues, SortedNumericDocValues, SortedSetDocValues, +}; +use core::doc::DocValuesType; +use core::index::reader::LeafReaderContext; +use core::search::DocIterator; +use core::search::NO_MORE_DOCS; +use core::util::DocId; +use error::Result; + +pub struct DocValuesIterator { + cost: i32, + doc: DocId, + doc_values: DocValuesEnum, +} + +impl DocValuesIterator { + pub fn new(field: &str, cost: i32, leaf_reader: &LeafReaderContext<'_, C>) -> Self { + if let Some(field_info) = leaf_reader.reader.field_info(field) { + let mut doc_values = DocValuesEnum::None; + match field_info.doc_values_type { + DocValuesType::Binary => { + if let Ok(values) = leaf_reader.reader.get_binary_doc_values(field) { + doc_values = DocValuesEnum::Binary(values); + } + } + DocValuesType::Numeric => { + if let Ok(values) = leaf_reader.reader.get_numeric_doc_values(field) { + doc_values = DocValuesEnum::Numeric(values); + } + } + DocValuesType::Sorted => { + if let Ok(values) = leaf_reader.reader.get_sorted_doc_values(field) { + doc_values = DocValuesEnum::Sorted(values); + } + } + DocValuesType::SortedSet => { + if let Ok(values) = leaf_reader.reader.get_sorted_set_doc_values(field) { + doc_values = DocValuesEnum::SortedSet(values); + } + } + DocValuesType::SortedNumeric => { + if let Ok(values) = leaf_reader.reader.get_sorted_numeric_doc_values(field) { + doc_values = DocValuesEnum::SortedNumeric(values); + } + } + _ => {} + } + + return Self { + cost, + doc: -1, + doc_values, + }; + } + + Self { + cost: 0, + doc: -1, + doc_values: DocValuesEnum::None, + } + } + + fn exists(&mut self, doc_id: DocId) -> Result { + + match self.doc_values { + DocValuesEnum::Binary(ref mut binary) => { + if let Ok(v) = binary.get(doc_id) { + return Ok(v.len() > 0); + } + } + DocValuesEnum::Numeric(ref mut numeric) => { + if let Ok(_) = numeric.get(doc_id) { + return Ok(true); + } + + } + DocValuesEnum::Sorted(ref mut sorted) => { + if let Ok(o) = sorted.get_ord(doc_id) { + return Ok(o > -1); + } + } + DocValuesEnum::SortedNumeric(ref mut sorted_numeric) => { + if let Ok(()) = sorted_numeric.set_document(doc_id) { + return Ok(sorted_numeric.count() > 0); + } + } + DocValuesEnum::SortedSet(ref mut sorted_set) => { + if let Ok(()) = sorted_set.set_document(doc_id) { + if let Ok(o) = sorted_set.next_ord() { + return Ok(o > -1) + } + } + } + _ => {} + } + + Ok(false) + } +} + +enum DocValuesEnum { + Binary(Box), + Numeric(Box), + Sorted(Box), + SortedSet(Box), + SortedNumeric(Box), + None, +} + +impl DocIterator for DocValuesIterator { + fn doc_id(&self) -> DocId { + self.doc + } + + fn next(&mut self) -> Result { + let next = self.doc + 1; + self.advance(next) + } + + fn advance(&mut self, target: DocId) -> Result { + let mut target = target; + + loop { + if target >= self.cost() as i32 { + self.doc = NO_MORE_DOCS; + break; + } + + if self.exists(target)? { + self.doc = target; + break; + } + + target += 1; + } + + Ok(self.doc) + } + + fn cost(&self) -> usize { + self.cost as usize + } +} \ No newline at end of file diff --git a/src/core/codec/doc_values/mod.rs b/src/core/codec/doc_values/mod.rs index 8928f7a..b9bc693 100644 --- a/src/core/codec/doc_values/mod.rs +++ b/src/core/codec/doc_values/mod.rs @@ -29,6 +29,10 @@ mod doc_values_writer; pub use self::doc_values_writer::*; +mod doc_values_iterator; + +pub use self::doc_values_iterator::*; + use core::codec::doc_values::lucene54::DocValuesTermIterator; use core::util::DocId; diff --git a/src/core/search/query/exists_query.rs b/src/core/search/query/exists_query.rs new file mode 100644 index 0000000..6c942d5 --- /dev/null +++ b/src/core/search/query/exists_query.rs @@ -0,0 +1,137 @@ +use std::any::Any; +use std::fmt; + +use core::codec::Codec; +use core::codec::doc_values::DocValuesIterator; +use core::index::reader::LeafReaderContext; +use core::search::explanation::Explanation; +use core::search::query::{Query, TermQuery, Weight}; +use core::search::scorer::ConstantScoreScorer; +use core::search::scorer::Scorer; +use core::search::searcher::SearchPlanBuilder; +use core::util::DocId; +use error::Result; + +const EXISTS_QUERY: &str = "exists"; + +pub struct ExistsQuery { + field: String, +} + +impl ExistsQuery { + pub fn build( + field: String, + ) -> ExistsQuery { + ExistsQuery { + field, + } + } +} + +impl Query for ExistsQuery { + fn create_weight( + &self, + _searcher: &dyn SearchPlanBuilder, + _needs_scores: bool, + ) -> Result>> { + Ok(Box::new( + ExistsWeight::new( + self.field.clone(), + ))) + } + + fn extract_terms(&self) -> Vec { + vec![] + } + + fn as_any(&self) -> &dyn Any { + self + } +} + +impl fmt::Display for ExistsQuery { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "ExistsQuery(field={})", + &self.field + ) + } +} + +struct ExistsWeight { + field: String, + weight: f32, + norm: f32, +} + +impl ExistsWeight { + pub fn new( + field: String, + ) -> ExistsWeight { + ExistsWeight { + field, + weight: 0f32, + norm: 0f32, + } + } +} + +impl Weight for ExistsWeight { + fn create_scorer( + &self, + leaf_reader: &LeafReaderContext<'_, C>, + ) -> Result>> { + if let Some(field_info) = leaf_reader.reader.field_info(self.field.as_str()) { + let cost: i32 = leaf_reader.reader.max_doc(); + let doc_iterator = DocValuesIterator::new(field_info.name.as_str(), cost, leaf_reader); + + return Ok(Some(Box::new(ConstantScoreScorer::new(self.weight, doc_iterator, cost as usize)))); + } + + Ok(None) + } + + fn query_type(&self) -> &'static str { + EXISTS_QUERY + } + + fn actual_query_type(&self) -> &'static str { + EXISTS_QUERY + } + + fn normalize(&mut self, norm: f32, boost: f32) { + self.norm = norm; + self.weight = norm * boost; + } + + fn value_for_normalization(&self) -> f32 { + self.weight * self.weight + } + + fn needs_scores(&self) -> bool { + false + } + + fn explain(&self, _reader: &LeafReaderContext<'_, C>, _doc: DocId) -> Result { + Ok(Explanation::new( + true, + self.weight, + format!("{}, product of:", self), + vec![ + Explanation::new(true, self.weight, "exists".to_string(), vec![]), + ], + )) + } +} + + +impl fmt::Display for ExistsWeight { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "ExistsWeight(field={}, weight={}, norm={})", + &self.field, self.weight, self.norm + ) + } +} diff --git a/src/core/search/query/mod.rs b/src/core/search/query/mod.rs index 869a1c2..5635bdd 100644 --- a/src/core/search/query/mod.rs +++ b/src/core/search/query/mod.rs @@ -54,6 +54,10 @@ mod boosting_query; pub use self::boosting_query::*; +mod exists_query; + +pub use self::exists_query::*; + use core::codec::Codec; use core::index::reader::LeafReaderContext; use core::search::explanation::Explanation; From 8010137cc8e1e28b69c81ff7d0a7719bbc75ffc9 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Mon, 27 Apr 2020 11:04:21 +0800 Subject: [PATCH 23/49] truncate hf-term's df/tf --- .../postings/blocktree/blocktree_writer.rs | 21 ++++++++++++++++--- src/core/codec/postings/mod.rs | 6 ++++++ src/core/codec/postings/posting_writer.rs | 9 +++++++- src/core/index/merge/merge_policy.rs | 4 ++-- src/core/index/writer/index_writer_config.rs | 2 +- 5 files changed, 35 insertions(+), 7 deletions(-) diff --git a/src/core/codec/postings/blocktree/blocktree_writer.rs b/src/core/codec/postings/blocktree/blocktree_writer.rs index f7156a5..b0644b5 100644 --- a/src/core/codec/postings/blocktree/blocktree_writer.rs +++ b/src/core/codec/postings/blocktree/blocktree_writer.rs @@ -14,7 +14,9 @@ use core::codec::codec_util::{write_footer, write_index_header}; use core::codec::field_infos::{FieldInfo, FieldInfos}; use core::codec::postings::blocktree::*; -use core::codec::postings::{FieldsConsumer, PostingsWriterBase}; +use core::codec::postings::{ + FieldsConsumer, PostingsWriterBase, DEFAULT_DOC_TERM_FREQ, DEFAULT_SEGMENT_DOC_FREQ, +}; use core::codec::segment_infos::{segment_file_name, SegmentWriteState}; use core::codec::Codec; use core::codec::{Fields, TermIterator, Terms}; @@ -281,7 +283,12 @@ impl FieldsConsumer for BlockTreeTermsWri let mut terms_writer = TermsWriter::new(field_info, self); while let Some(term) = terms_iter.next()? { - terms_writer.write(&term, &mut terms_iter)?; + terms_writer.write( + &term, + &mut terms_iter, + DEFAULT_SEGMENT_DOC_FREQ, + DEFAULT_DOC_TERM_FREQ, + )?; } terms_writer.finish()?; } @@ -697,11 +704,19 @@ impl<'a, T: PostingsWriterBase, O: IndexOutput> TermsWriter<'a, T, O> { } // Writes one term's worth of postings. - pub fn write(&mut self, text: &[u8], terms_iter: &mut impl TermIterator) -> Result<()> { + pub fn write( + &mut self, + text: &[u8], + terms_iter: &mut impl TermIterator, + doc_freq_limit: i32, + term_freq_limit: i32, + ) -> Result<()> { if let Some(state) = self.block_tree_writer.postings_writer.write_term( text, terms_iter, &mut self.docs_seen, + doc_freq_limit, + term_freq_limit, )? { assert_ne!(state.doc_freq, 0); assert!( diff --git a/src/core/codec/postings/mod.rs b/src/core/codec/postings/mod.rs index be5c7c6..d305d1e 100644 --- a/src/core/codec/postings/mod.rs +++ b/src/core/codec/postings/mod.rs @@ -67,6 +67,10 @@ use error::ErrorKind::IllegalArgument; use error::Result; use std::sync::Arc; +// sometimes will cause miss increasing with phrase/highlight. +pub const DEFAULT_SEGMENT_DOC_FREQ: i32 = 500_000; +pub const DEFAULT_DOC_TERM_FREQ: i32 = 10; + /// Encodes/decodes terms, postings, and proximity data. ///

/// Note, when extending this class, the name ({@link #getName}) may @@ -336,6 +340,8 @@ pub trait PostingsWriterBase { term: &[u8], terms: &mut impl TermIterator, docs_seen: &mut FixedBitSet, + doc_freq_limit: i32, + term_freq_limit: i32, ) -> Result>; /// Encode metadata as [i64] and [u8]. {@param absolute} controls whether diff --git a/src/core/codec/postings/posting_writer.rs b/src/core/codec/postings/posting_writer.rs index f3b7f46..22342d6 100644 --- a/src/core/codec/postings/posting_writer.rs +++ b/src/core/codec/postings/posting_writer.rs @@ -609,6 +609,8 @@ impl PostingsWriterBase for Lucene50PostingsWriter { _term: &[u8], terms: &mut impl TermIterator, docs_seen: &mut FixedBitSet, + doc_freq_limit: i32, + term_freq_limit: i32, ) -> Result> { self.start_term(); let mut postings_enum = terms.postings_with_flags(self.enum_flags)?; @@ -622,7 +624,7 @@ impl PostingsWriterBase for Lucene50PostingsWriter { doc_freq += 1; docs_seen.set(doc_id as usize); let freq = if self.write_freqs { - let f = postings_enum.freq()?; + let f = postings_enum.freq()?.min(term_freq_limit); total_term_freq += f; f } else { @@ -648,6 +650,11 @@ impl PostingsWriterBase for Lucene50PostingsWriter { } self.finish_doc(); + + if doc_freq > doc_freq_limit { + println!("{} {}", String::from_utf8(_term.to_vec())?, doc_freq); + break; + } } if doc_freq == 0 { diff --git a/src/core/index/merge/merge_policy.rs b/src/core/index/merge/merge_policy.rs index 12b4d99..677b612 100644 --- a/src/core/index/merge/merge_policy.rs +++ b/src/core/index/merge/merge_policy.rs @@ -37,7 +37,7 @@ use std::ptr; use std::sync::Arc; use std::time::SystemTime; -pub const DEFAULT_NO_CFS_RATIO: f64 = 0.1; +pub const DEFAULT_NO_CFS_RATIO: f64 = 0.0; pub const DEFAULT_MAX_CFS_SEGMENT_SIZE: u64 = i64::max_value() as u64; @@ -159,7 +159,7 @@ pub trait MergePolicy: 'static { MP: MergePolicy, { let no_cfs_ratio = self.no_cfs_ratio(); - if no_cfs_ratio == 0.0 { + if no_cfs_ratio <= 0.0 { return false; } diff --git a/src/core/index/writer/index_writer_config.rs b/src/core/index/writer/index_writer_config.rs index 715d594..0cb6b86 100644 --- a/src/core/index/writer/index_writer_config.rs +++ b/src/core/index/writer/index_writer_config.rs @@ -85,7 +85,7 @@ impl Default for IndexWriterConfig IndexWriterConfig { pub fn new(codec: Arc, merge_scheduler: MS, merge_policy: MP) -> Self { IndexWriterConfig { - use_compound_file: true, + use_compound_file: false, max_buffered_delete_terms: None, max_buffered_docs: None, merge_policy, From e9fe7d727d5b6c3e870d46dbe8008f4b43eb4379 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Wed, 6 May 2020 11:32:22 +0800 Subject: [PATCH 24/49] fix commit dead lock & fmt token --- src/core/analysis/char_buffer.rs | 73 ---- src/core/analysis/mod.rs | 188 +++++++++- src/core/analysis/token_attributes.rs | 341 ------------------ src/core/analysis/token_stream.rs | 330 ++++++----------- src/core/analysis/whitespace_tokenizer.rs | 152 ++++---- .../codec/doc_values/doc_values_iterator.rs | 11 +- .../codec/postings/terms_hash_per_field.rs | 24 +- .../term_vectors/term_vector_consumer.rs | 8 +- src/core/index/writer/doc_consumer.rs | 14 +- src/core/index/writer/index_writer.rs | 30 +- src/core/search/query/exists_query.rs | 41 +-- 11 files changed, 438 insertions(+), 774 deletions(-) delete mode 100644 src/core/analysis/char_buffer.rs delete mode 100644 src/core/analysis/token_attributes.rs diff --git a/src/core/analysis/char_buffer.rs b/src/core/analysis/char_buffer.rs deleted file mode 100644 index d85b19b..0000000 --- a/src/core/analysis/char_buffer.rs +++ /dev/null @@ -1,73 +0,0 @@ -// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -use error::Result; -use std::io::Read; - -use unicode_reader::CodePoints; - -/// a simple IO buffer to use -#[derive(Debug)] -pub struct CharacterBuffer { - pub buffer: Vec, - pub offset: usize, - pub length: usize, -} - -impl CharacterBuffer { - pub fn new(buffer_size: usize) -> Self { - if buffer_size < 2 { - panic!("buffer size must be >= 2"); - } - CharacterBuffer { - buffer: vec!['\0'; buffer_size], - offset: 0, - length: 0, - } - } - - pub fn is_empty(&self) -> bool { - self.length == 0 - } - - pub fn char_at(&self, index: usize) -> char { - debug_assert!(index < self.buffer.len()); - self.buffer[index] - } - - pub fn reset(&mut self) { - self.offset = 0; - self.length = 0; - } - - pub fn fill(&mut self, reader: &mut T) -> Result { - let mut unicode_reader = CodePoints::from(reader); - self.offset = 0; - let mut offset = 0; - - loop { - if offset >= self.buffer.len() { - break; - } - if let Some(res) = unicode_reader.next() { - let cur_char = res?; - self.buffer[offset] = cur_char; - offset += 1; - } else { - break; - } - } - self.length = offset; - Ok(self.buffer.len() == self.length) - } -} diff --git a/src/core/analysis/mod.rs b/src/core/analysis/mod.rs index 1707287..00bdeda 100644 --- a/src/core/analysis/mod.rs +++ b/src/core/analysis/mod.rs @@ -15,14 +15,190 @@ mod token_stream; pub use self::token_stream::*; -mod char_buffer; - -pub use self::char_buffer::*; - mod whitespace_tokenizer; pub use self::whitespace_tokenizer::*; -mod token_attributes; +use error::Result; + +use std::fmt::Debug; + +pub const MIN_BUFFER_SIZE: usize = 10; +pub const MAX_WORD_LEN: usize = 128; + +#[derive(Debug, Clone)] +pub struct Token { + pub term: Vec, + /// Determines the position of this token relative to the previous Token in a + /// TokenStream, used in phrase searching. + /// + /// The default value is one. + /// + /// Some common uses for this are: + /// + /// * Set it to zero to put multiple terms in the same position. This is + /// useful if, e.g., a word has multiple stems. Searches for phrases + /// including either stem will match. In this case, all but the first stem's + /// increment should be set to zero: the increment of the first instance + /// should be one. Repeating a token with an increment of zero can also be + /// used to boost the scores of matches on that token. + /// + /// * Set it to values greater than one to inhibit exact phrase matches. + /// If, for example, one does not want phrases to match across removed stop + /// words, then one could build a stop word filter that removes stop words and + /// also sets the increment to the number of stop words removed before each + /// non-stop word. Then exact phrase queries will only match when the terms + /// occur with no intervening stop words. + pub position: usize, + /// The start and end character offset of a Token. + pub start_offset: usize, + pub end_offset: usize, + /// The payload of a Token. + /// + /// The payload is stored in the index at each position, and can + /// be used to influence scoring when using Payload-based queries. + /// + /// NOTE: because the payload will be stored at each position, it's usually + /// best to use the minimum number of bytes necessary. Some codec implementations + /// may optimize payload storage when all payloads have the same length. + pub payload: Vec, +} + +impl Token { + pub fn new() -> Token { + Token { + term: Vec::with_capacity(MIN_BUFFER_SIZE), + position: 1, + start_offset: 0, + end_offset: 0, + payload: Vec::with_capacity(0), + } + } + + pub fn clear(&mut self) { + self.position = 1; + self.start_offset = 0; + self.end_offset = 0; + self.payload.clear(); + self.term.clear(); + } + + pub fn end(&mut self) { + self.clear(); + self.position = 0; + } + + /// Set the starting and ending offset. + pub fn set_offset(&mut self, start_offset: usize, end_offset: usize) -> Result<()> { + if end_offset < start_offset { + bail!( + "endOffset must be >= startOffset; got startOffset={}, endOffset={}", + start_offset, + end_offset + ) + } + + self.start_offset = start_offset; + self.end_offset = end_offset; + Ok(()) + } +} + +/// A `TokenStream` enumerates the sequence of tokens, either from +/// `Field`s of a `Document` or from query text. +/// +/// The workflow of the `TokenStream` API is as follows: +/// +/// - The consumer calls {@link TokenStream#reset()}. +/// - The consumer retrieves attributes from the stream and stores local +/// references to all attributes it wants to access. +/// - The consumer calls {@link #increment_token()} until it returns false +/// consuming the attributes after each call. +/// - The consumer calls {@link #end()} so that any end-of-stream operations +/// can be performed. +/// - The consumer calls {@link #close()} to release any resource when finished +/// using the `TokenStream`. +/// +/// To make sure that filters and consumers know which attributes are available, +/// the attributes must be added during instantiation. Filters and consumers are +/// not required to check for availability of attributes in +/// {@link #increment_token()}. +/// +/// You can find some example code for the new API in the analysis package level +/// Javadoc. +/// +/// The `TokenStream`-API in Lucene is based on the decorator pattern. +/// Therefore all non-abstract subclasses must be final or have at least a final +/// implementation of {@link #incrementToken}! This is checked when Java +/// assertions are enabled. +pub trait TokenStream: Debug { + /// Consumers (i.e., `IndexWriter`) use this method to advance the stream to + /// the next token. Implementing classes must implement this method and update + /// the appropriate {@link AttributeImpl}s with the attributes of the next + /// token. + /// + /// The producer must make no assumptions about the attributes after the method + /// has been returned: the caller may arbitrarily change it. If the producer + /// needs to preserve the state for subsequent calls, it can use + /// {@link #captureState} to create a copy of the current attribute state. + /// + /// This method is called for every token of a document, so an efficient + /// implementation is crucial for good performance. To avoid calls to + /// {@link #addAttribute(Class)} and {@link #getAttribute(Class)}, + /// references to all {@link AttributeImpl}s that this stream uses should be + /// retrieved during instantiation. + /// + /// To ensure that filters and consumers know which attributes are available, + /// the attributes must be added during instantiation. Filters and consumers + /// are not required to check for availability of attributes in + /// {@link #incrementToken()}. + /// + /// @return false for end of stream; true otherwise + fn next_token(&mut self) -> Result; + + /// This method is called by the consumer after the last token has been + /// consumed, after {@link #incrementToken()} returned `false` + /// (using the new `TokenStream` API). Streams implementing the old API + /// should upgrade to use this feature. + /// + /// This method can be used to perform any end-of-stream operations, such as + /// setting the final offset of a stream. The final offset of a stream might + /// differ from the offset of the last token eg in case one or more whitespaces + /// followed after the last token, but a WhitespaceTokenizer was used. + /// + /// Additionally any skipped positions (such as those removed by a stopfilter) + /// can be applied to the position increment, or any adjustment of other + /// attributes where the end-of-stream value may be important. + /// + /// If you override this method, always call {@code super.end()}. + fn end(&mut self) -> Result<()>; + + /// This method is called by a consumer before it begins consumption using + /// {@link #incrementToken()}. + /// + /// Resets this stream to a clean state. Stateful implementations must implement + /// this method so that they can be reused, just as if they had been created fresh. + /// + /// If you override this method, always call {@code super.reset()}, otherwise + /// some internal state will not be correctly reset (e.g., {@link Tokenizer} will + /// throw {@link IllegalStateException} on further usage). + fn reset(&mut self) -> Result<()>; + + /// mutable access of the `OffsetAttribute` + fn token(&self) -> &Token; + + /// mutable access of the `OffsetAttribute` + fn token_mut(&mut self) -> &mut Token; + + /// Resets all attributes in this `TokenStream` by calling `clear` method + /// on each Attribute implementation. + fn clear_token(&mut self) { + self.token_mut().clear(); + } -pub use self::token_attributes::*; + /// Resets all attributes in this `TokenStream` by calling `end` method + /// on each Attribute implementation. + fn end_token(&mut self) { + self.token_mut().end(); + } +} diff --git a/src/core/analysis/token_attributes.rs b/src/core/analysis/token_attributes.rs deleted file mode 100644 index 4d84457..0000000 --- a/src/core/analysis/token_attributes.rs +++ /dev/null @@ -1,341 +0,0 @@ -// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -use core::util::BytesRef; -use error::{ErrorKind, Result}; - -use std::fmt; - -/// Sets the custom term frequency of a term within one document. -/// -/// If this attribute is present in your analysis chain for a given field, -/// that field must be indexed with IndexOptions#DocsAndFreqs -pub struct TermFreqAttribute { - term_freq: u32, -} - -impl Default for TermFreqAttribute { - fn default() -> Self { - TermFreqAttribute { term_freq: 1 } - } -} - -impl TermFreqAttribute { - pub fn new() -> Self { - Default::default() - } - - pub fn set_term_freq(&mut self, freq: u32) { - if freq < 1 { - panic!("Term frequency must be 1 or greater"); - } - self.term_freq = freq; - } - - pub fn term_freq(&self) -> u32 { - self.term_freq - } - - pub fn clear(&mut self) { - self.term_freq = 1; - } - - pub fn end(&mut self) { - self.term_freq = 1; - } -} - -/// The start and end character offset of a Token. -#[derive(Debug, Default)] -pub struct OffsetAttribute { - start_offset: usize, - end_offset: usize, -} - -impl OffsetAttribute { - pub fn new() -> OffsetAttribute { - OffsetAttribute { - start_offset: 0, - end_offset: 0, - } - } - - /// Returns this Token's starting offset, the position of the first character - /// corresponding to this token in the source text. - /// - /// Note that the difference between `#end_offset()` and `#start_offset()` may - /// not be equal to term.len(), as the term text may have been altered by a - /// stemmer or some other filter. - pub fn start_offset(&self) -> usize { - self.start_offset - } - - /// Returns this Token's ending offset, one greater than the position of the - /// last character corresponding to this token in the source text. The length - /// of the token in the source text is (end_offset() - start_offset()). - pub fn end_offset(&self) -> usize { - self.end_offset - } - - /// Set the starting and ending offset. - pub fn set_offset(&mut self, start_offset: usize, end_offset: usize) -> Result<()> { - // TODO: we could assert that this is set-once, ie, - // current values are -1? Very few token filters should - // change offsets once set by the tokenizer... and - // tokenizer should call clearAtts before re-using - // OffsetAtt - - if end_offset < start_offset { - bail!( - "endOffset must be >= startOffset; got startOffset={}, endOffset={}", - start_offset, - end_offset - ) - } - - self.start_offset = start_offset; - self.end_offset = end_offset; - Ok(()) - } - - pub fn clear(&mut self) { - // TODO: we could use -1 as default here? Then we can - // assert in setOffset... - self.start_offset = 0; - self.end_offset = 0; - } - - /// end this attribute - pub fn end(&mut self) { - self.clear(); - } -} - -/// Determines the position of this token relative to the previous Token in a -/// TokenStream, used in phrase searching. -/// -/// The default value is one. -/// -/// Some common uses for this are: -/// -/// * Set it to zero to put multiple terms in the same position. This is -/// useful if, e.g., a word has multiple stems. Searches for phrases -/// including either stem will match. In this case, all but the first stem's -/// increment should be set to zero: the increment of the first instance -/// should be one. Repeating a token with an increment of zero can also be -/// used to boost the scores of matches on that token. -/// -/// * Set it to values greater than one to inhibit exact phrase matches. -/// If, for example, one does not want phrases to match across removed stop -/// words, then one could build a stop word filter that removes stop words and -/// also sets the increment to the number of stop words removed before each -/// non-stop word. Then exact phrase queries will only match when the terms -/// occur with no intervening stop words. -#[derive(Debug, Clone, Copy)] -pub struct PositionAttribute { - position: u32, -} - -impl Default for PositionAttribute { - fn default() -> Self { - PositionAttribute::new() - } -} - -impl PositionAttribute { - #[inline] - pub fn new() -> PositionAttribute { - PositionAttribute { position: 1 } - } - - pub fn set_position(&mut self, position: u32) { - self.position = position; - } - - pub fn get_position(self) -> u32 { - self.position - } - - pub fn clear(&mut self) { - self.position = 1 - } - - pub fn end(&mut self) { - self.position = 0 - } -} - -/// The payload of a Token. -/// -/// The payload is stored in the index at each position, and can -/// be used to influence scoring when using Payload-based queries. -/// -/// NOTE: because the payload will be stored at each position, it's usually -/// best to use the minimum number of bytes necessary. Some codec implementations -/// may optimize payload storage when all payloads have the same length. -#[derive(Debug)] -pub struct PayloadAttribute { - payload: Vec, -} - -impl PayloadAttribute { - pub fn new(payload: Vec) -> PayloadAttribute { - PayloadAttribute { payload } - } - - pub fn from(payload: Vec) -> PayloadAttribute { - PayloadAttribute { payload } - } - - pub fn get_payload(&self) -> &[u8] { - &self.payload - } - - pub fn set_payload(&mut self, payload: Vec) { - self.payload = payload - } - - pub fn clear(&mut self) { - self.payload.clear() - } - - pub fn end(&mut self) { - self.clear(); - } -} - -/// This attribute is requested by TermsHashPerField to index the contents. -/// This attribute can be used to customize the final bytes encoding of terms. -/// -/// Consumers of this attribute call `get_bytes_ref` for each term. -pub trait TermToBytesRefAttribute { - fn get_bytes_ref(&self) -> BytesRef; - - fn clear(&mut self); - - fn end(&mut self); -} - -const MIN_BUFFER_SIZE: usize = 10; - -/// The term text of a Token -#[derive(Debug)] -pub struct CharTermAttribute { - pub term_buffer: Vec, -} - -impl CharTermAttribute { - pub fn new() -> Self { - CharTermAttribute { - term_buffer: Vec::with_capacity(MIN_BUFFER_SIZE), - } - } - - /// Returns the length of this character sequence. the length is calculated - /// by byte instead by char - pub fn len(&self) -> usize { - self.term_buffer.len() - } - - pub fn push_char(&mut self, c: char) { - let char_len = c.len_utf8(); - let term_len = self.term_buffer.len(); - self.term_buffer.resize(term_len + char_len, 0u8); - c.encode_utf8(&mut self.term_buffer[term_len..]); - } - - /// Copies the contents of buffer, starting at offset for length chars, into the term buffer vec - pub fn copy_buffer(&mut self, buffer: &[u8]) { - self.term_buffer.resize(buffer.len(), 0u8); - self.term_buffer[0..buffer.len()].copy_from_slice(buffer); - } - - /// Appends the specified string to this char sequence. - pub fn append(&mut self, s: &str) { - self.term_buffer.extend(s.bytes()) - } - - /// Set number of valid characters (length of the term) in the term buffer vector. - /// Use this to truncate the term buffer or to synchronize with external manipulation - /// of the termBuffer. - /// Note: to grow the size of the array, use {@link #resizeBuffer(int)} first. - pub fn set_length(&mut self, length: usize) -> Result<()> { - if length > self.term_buffer.len() { - bail!(ErrorKind::IllegalArgument(format!( - "length {} exceeds the size of the term_buffer", - length - ))); - } - self.term_buffer.truncate(length); - Ok(()) - } - - /// Sets the length of the termBuffer to zero. Use this method before appending contents - pub fn set_empty(&mut self) { - self.term_buffer.clear(); - } -} - -impl TermToBytesRefAttribute for CharTermAttribute { - fn get_bytes_ref(&self) -> BytesRef { - BytesRef::new(&self.term_buffer) - } - - fn clear(&mut self) { - self.term_buffer.clear(); - } - - fn end(&mut self) { - self.clear(); - } -} - -/// This attribute can be used if you have the raw term bytes to be indexed. -/// -/// It can be used as replacement for `CharTermAttribute`, if binary terms should be indexed. -#[derive(Default)] -pub struct BytesTermAttribute { - bytes: BytesRef, -} - -impl BytesTermAttribute { - pub fn new() -> Self { - Default::default() - } - - pub fn set_bytes(&mut self, bytes_ref: &[u8]) { - self.bytes = BytesRef::new(bytes_ref); - } -} - -impl fmt::Debug for BytesTermAttribute { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_struct("BytesTermAttribute") - .field("bytes", &self.bytes.bytes()) - .finish() - } -} - -impl TermToBytesRefAttribute for BytesTermAttribute { - fn get_bytes_ref(&self) -> BytesRef { - BytesRef::new(self.bytes.bytes()) - } - - fn clear(&mut self) { - self.bytes = BytesRef::default() - } - - fn end(&mut self) { - self.clear(); - } -} diff --git a/src/core/analysis/token_stream.rs b/src/core/analysis/token_stream.rs index 49287f5..14f1740 100644 --- a/src/core/analysis/token_stream.rs +++ b/src/core/analysis/token_stream.rs @@ -11,154 +11,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt::Debug; - -use core::analysis::{ - BytesTermAttribute, CharTermAttribute, OffsetAttribute, PayloadAttribute, PositionAttribute, - TermToBytesRefAttribute, -}; +use core::analysis::{Token, TokenStream, MAX_WORD_LEN}; use core::util::BytesRef; use error::Result; use std::cmp::Ordering; use std::collections::HashSet; -/// A `TokenStream` enumerates the sequence of tokens, either from -/// `Field`s of a `Document` or from query text. -/// -/// The workflow of the `TokenStream` API is as follows: -/// -/// - The consumer calls {@link TokenStream#reset()}. -/// - The consumer retrieves attributes from the stream and stores local -/// references to all attributes it wants to access. -/// - The consumer calls {@link #increment_token()} until it returns false -/// consuming the attributes after each call. -/// - The consumer calls {@link #end()} so that any end-of-stream operations -/// can be performed. -/// - The consumer calls {@link #close()} to release any resource when finished -/// using the `TokenStream`. -/// -/// To make sure that filters and consumers know which attributes are available, -/// the attributes must be added during instantiation. Filters and consumers are -/// not required to check for availability of attributes in -/// {@link #increment_token()}. -/// -/// You can find some example code for the new API in the analysis package level -/// Javadoc. -/// -/// The `TokenStream`-API in Lucene is based on the decorator pattern. -/// Therefore all non-abstract subclasses must be final or have at least a final -/// implementation of {@link #incrementToken}! This is checked when Java -/// assertions are enabled. -pub trait TokenStream: Debug { - /// Consumers (i.e., `IndexWriter`) use this method to advance the stream to - /// the next token. Implementing classes must implement this method and update - /// the appropriate {@link AttributeImpl}s with the attributes of the next - /// token. - /// - /// The producer must make no assumptions about the attributes after the method - /// has been returned: the caller may arbitrarily change it. If the producer - /// needs to preserve the state for subsequent calls, it can use - /// {@link #captureState} to create a copy of the current attribute state. - /// - /// This method is called for every token of a document, so an efficient - /// implementation is crucial for good performance. To avoid calls to - /// {@link #addAttribute(Class)} and {@link #getAttribute(Class)}, - /// references to all {@link AttributeImpl}s that this stream uses should be - /// retrieved during instantiation. - /// - /// To ensure that filters and consumers know which attributes are available, - /// the attributes must be added during instantiation. Filters and consumers - /// are not required to check for availability of attributes in - /// {@link #incrementToken()}. - /// - /// @return false for end of stream; true otherwise - fn increment_token(&mut self) -> Result; - - /// This method is called by the consumer after the last token has been - /// consumed, after {@link #incrementToken()} returned `false` - /// (using the new `TokenStream` API). Streams implementing the old API - /// should upgrade to use this feature. - /// - /// This method can be used to perform any end-of-stream operations, such as - /// setting the final offset of a stream. The final offset of a stream might - /// differ from the offset of the last token eg in case one or more whitespaces - /// followed after the last token, but a WhitespaceTokenizer was used. - /// - /// Additionally any skipped positions (such as those removed by a stopfilter) - /// can be applied to the position increment, or any adjustment of other - /// attributes where the end-of-stream value may be important. - /// - /// If you override this method, always call {@code super.end()}. - fn end(&mut self) -> Result<()>; - - /// This method is called by a consumer before it begins consumption using - /// {@link #incrementToken()}. - /// - /// Resets this stream to a clean state. Stateful implementations must implement - /// this method so that they can be reused, just as if they had been created fresh. - /// - /// If you override this method, always call {@code super.reset()}, otherwise - /// some internal state will not be correctly reset (e.g., {@link Tokenizer} will - /// throw {@link IllegalStateException} on further usage). - fn reset(&mut self) -> Result<()>; - - // attributes used for build invert index - - /// Resets all attributes in this `TokenStream` by calling `clear` method - /// on each Attribute implementation. - fn clear_attributes(&mut self) { - self.offset_attribute_mut().clear(); - self.position_attribute_mut().clear(); - if let Some(attr) = self.payload_attribute_mut() { - attr.clear(); - } - self.term_bytes_attribute_mut().clear(); - } - - /// Resets all attributes in this `TokenStream` by calling `end` method - /// on each Attribute implementation. - fn end_attributes(&mut self) { - self.offset_attribute_mut().end(); - self.position_attribute_mut().end(); - if let Some(attr) = self.payload_attribute_mut() { - attr.end(); - } - self.term_bytes_attribute_mut().end(); - } - - /// mutable access of the `OffsetAttribute` - fn offset_attribute_mut(&mut self) -> &mut OffsetAttribute; - - /// access of the `OffsetAttribute` - fn offset_attribute(&self) -> &OffsetAttribute; - - /// mutable access of the `PositionIncrementAttribute` - fn position_attribute_mut(&mut self) -> &mut PositionAttribute; - - /// mutable access of the `PayloadAttribute`, wound return None if not enabled - fn payload_attribute_mut(&mut self) -> Option<&mut PayloadAttribute> { - None - } - - /// access of the `PayloadAttribute`, wound return None if not enabled - fn payload_attribute(&self) -> Option<&PayloadAttribute> { - None - } - - /// mutable access of the `TermToBytesRefAttribute` - fn term_bytes_attribute_mut(&mut self) -> &mut dyn TermToBytesRefAttribute; - - /// access of the [`TermToBytesRefAttribute` - fn term_bytes_attribute(&self) -> &dyn TermToBytesRefAttribute; -} - #[derive(Debug)] pub struct StringTokenStream { - term_attribute: CharTermAttribute, - offset_attribute: OffsetAttribute, - position_attribute: PositionAttribute, - payload_attribute: PayloadAttribute, + token: Token, used: bool, value: String, } @@ -166,10 +28,7 @@ pub struct StringTokenStream { impl StringTokenStream { pub fn new(value: String) -> Self { StringTokenStream { - term_attribute: CharTermAttribute::new(), - offset_attribute: OffsetAttribute::new(), - position_attribute: PositionAttribute::new(), - payload_attribute: PayloadAttribute::new(Vec::with_capacity(0)), + token: Token::new(), used: true, value, } @@ -177,56 +36,43 @@ impl StringTokenStream { } impl TokenStream for StringTokenStream { - fn increment_token(&mut self) -> Result { + fn next_token(&mut self) -> Result { if self.used { return Ok(false); } - self.clear_attributes(); - self.term_attribute.append(&self.value); - self.offset_attribute.set_offset(0, self.value.len())?; + self.clear_token(); + self.token.term = self.value.as_bytes().to_vec(); + self.token.set_offset(0, self.value.len())?; + self.used = true; + Ok(true) } fn end(&mut self) -> Result<()> { - self.end_attributes(); - let final_offset = self.value.len(); - self.offset_attribute.set_offset(final_offset, final_offset) + self.end_token(); + self.token.set_offset(self.value.len(), self.value.len()) } fn reset(&mut self) -> Result<()> { self.used = false; - Ok(()) - } - fn offset_attribute_mut(&mut self) -> &mut OffsetAttribute { - &mut self.offset_attribute - } - - fn offset_attribute(&self) -> &OffsetAttribute { - &self.offset_attribute - } - - fn position_attribute_mut(&mut self) -> &mut PositionAttribute { - &mut self.position_attribute + Ok(()) } - fn term_bytes_attribute_mut(&mut self) -> &mut dyn TermToBytesRefAttribute { - &mut self.term_attribute + fn token(&self) -> &Token { + &self.token } - fn term_bytes_attribute(&self) -> &dyn TermToBytesRefAttribute { - &self.term_attribute + fn token_mut(&mut self) -> &mut Token { + &mut self.token } } #[derive(Debug)] pub struct BinaryTokenStream { - term_attribute: BytesTermAttribute, - offset_attribute: OffsetAttribute, - position_attribute: PositionAttribute, - payload_attribute: PayloadAttribute, + token: Token, used: bool, value: BytesRef, } @@ -234,10 +80,7 @@ pub struct BinaryTokenStream { impl BinaryTokenStream { pub fn new(value: BytesRef) -> Self { BinaryTokenStream { - term_attribute: BytesTermAttribute::new(), - offset_attribute: OffsetAttribute::new(), - position_attribute: PositionAttribute::new(), - payload_attribute: PayloadAttribute::new(Vec::with_capacity(0)), + token: Token::new(), used: true, value, } @@ -245,19 +88,22 @@ impl BinaryTokenStream { } impl TokenStream for BinaryTokenStream { - fn increment_token(&mut self) -> Result { + fn next_token(&mut self) -> Result { if self.used { return Ok(false); } - self.clear_attributes(); - self.term_attribute.set_bytes(self.value.bytes()); + self.clear_token(); + self.token.term = self.value.bytes().to_vec(); + self.used = true; + Ok(true) } fn end(&mut self) -> Result<()> { - self.end_attributes(); + self.end_token(); + Ok(()) } @@ -266,29 +112,15 @@ impl TokenStream for BinaryTokenStream { Ok(()) } - fn offset_attribute_mut(&mut self) -> &mut OffsetAttribute { - &mut self.offset_attribute - } - - fn offset_attribute(&self) -> &OffsetAttribute { - &self.offset_attribute - } - - fn position_attribute_mut(&mut self) -> &mut PositionAttribute { - &mut self.position_attribute + fn token(&self) -> &Token { + &self.token } - fn term_bytes_attribute_mut(&mut self) -> &mut dyn TermToBytesRefAttribute { - &mut self.term_attribute - } - - fn term_bytes_attribute(&self) -> &dyn TermToBytesRefAttribute { - &self.term_attribute + fn token_mut(&mut self) -> &mut Token { + &mut self.token } } -pub const MAX_WORD_LEN: usize = 128; - #[derive(Debug, Eq, PartialEq, Hash)] pub struct Word { value: String, @@ -316,10 +148,7 @@ impl Word { #[derive(Debug)] pub struct WordTokenStream { - term_attribute: CharTermAttribute, - offset_attribute: OffsetAttribute, - position_attribute: PositionAttribute, - payload_attribute: PayloadAttribute, + token: Token, values: Vec, current: usize, } @@ -340,10 +169,7 @@ impl WordTokenStream { }); WordTokenStream { - term_attribute: CharTermAttribute::new(), - offset_attribute: OffsetAttribute::new(), - position_attribute: PositionAttribute::new(), - payload_attribute: PayloadAttribute::new(Vec::with_capacity(0)), + token: Token::new(), values: elements, current: 0, } @@ -351,28 +177,30 @@ impl WordTokenStream { } impl TokenStream for WordTokenStream { - fn increment_token(&mut self) -> Result { + fn next_token(&mut self) -> Result { if self.current == self.values.len() { return Ok(false); } - self.clear_attributes(); + self.clear_token(); let word: &Word = &self.values[self.current]; - self.term_attribute.append(&word.value); - self.offset_attribute + self.token.term = word.value.as_bytes().to_vec(); + self.token .set_offset(word.begin, word.begin + word.length)?; self.current += 1; + Ok(true) } fn end(&mut self) -> Result<()> { - self.end_attributes(); + self.end_token(); + if let Some(word) = self.values.last() { let final_offset = word.begin + word.length; - self.offset_attribute.set_offset(final_offset, final_offset) + self.token.set_offset(final_offset, final_offset) } else { - self.offset_attribute.set_offset(0, 0) + self.token.set_offset(0, 0) } } @@ -381,23 +209,73 @@ impl TokenStream for WordTokenStream { Ok(()) } - fn offset_attribute_mut(&mut self) -> &mut OffsetAttribute { - &mut self.offset_attribute + fn token(&self) -> &Token { + &self.token } - fn offset_attribute(&self) -> &OffsetAttribute { - &self.offset_attribute - } - - fn position_attribute_mut(&mut self) -> &mut PositionAttribute { - &mut self.position_attribute - } - - fn term_bytes_attribute_mut(&mut self) -> &mut dyn TermToBytesRefAttribute { - &mut self.term_attribute + fn token_mut(&mut self) -> &mut Token { + &mut self.token } +} - fn term_bytes_attribute(&self) -> &dyn TermToBytesRefAttribute { - &self.term_attribute +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_string_token_stream() { + let value = "case".to_string(); + let mut token = StringTokenStream::new(value.clone()); + token.reset().unwrap(); + + assert_eq!(token.next_token().unwrap(), true); + assert_eq!(token.token().term.as_slice(), value.as_bytes()); + } + + #[test] + fn test_binary_token_stream() { + let value = "case".as_bytes().to_vec(); + let mut token = BinaryTokenStream::new(BytesRef::new(&value)); + token.reset().unwrap(); + + assert_eq!(token.next_token().unwrap(), true); + assert_eq!(token.token().term.as_slice(), value.as_slice()); + } + + #[test] + fn test_word_token_stream() { + let words = vec![ + Word::new("The", 0, 3), + Word::new("quick", 4, 9), + Word::new("brown", 10, 15), + Word::new("fox", 16, 19), + Word::new("jumps", 20, 25), + Word::new("over", 26, 30), + Word::new("the", 31, 34), + Word::new("lazy", 35, 39), + Word::new("dog", 40, 43), + ]; + let mut token = WordTokenStream::new(words); + token.reset().unwrap(); + + assert_eq!(token.next_token().unwrap(), true); + assert_eq!(token.token().term.as_slice(), "The".as_bytes()); + assert_eq!(token.next_token().unwrap(), true); + assert_eq!(token.token().term.as_slice(), "quick".as_bytes()); + assert_eq!(token.next_token().unwrap(), true); + assert_eq!(token.token().term.as_slice(), "brown".as_bytes()); + assert_eq!(token.next_token().unwrap(), true); + assert_eq!(token.token().term.as_slice(), "fox".as_bytes()); + assert_eq!(token.next_token().unwrap(), true); + assert_eq!(token.token().term.as_slice(), "jumps".as_bytes()); + assert_eq!(token.next_token().unwrap(), true); + assert_eq!(token.token().term.as_slice(), "over".as_bytes()); + assert_eq!(token.next_token().unwrap(), true); + assert_eq!(token.token().term.as_slice(), "the".as_bytes()); + assert_eq!(token.next_token().unwrap(), true); + assert_eq!(token.token().term.as_slice(), "lazy".as_bytes()); + assert_eq!(token.next_token().unwrap(), true); + assert_eq!(token.token().term.as_slice(), "dog".as_bytes()); + assert_eq!(token.next_token().unwrap(), false); } } diff --git a/src/core/analysis/whitespace_tokenizer.rs b/src/core/analysis/whitespace_tokenizer.rs index 8d97a35..c089bf9 100644 --- a/src/core/analysis/whitespace_tokenizer.rs +++ b/src/core/analysis/whitespace_tokenizer.rs @@ -11,19 +11,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -use core::analysis::{ - CharTermAttribute, OffsetAttribute, PositionAttribute, TermToBytesRefAttribute, -}; -use core::analysis::{CharacterBuffer, TokenStream}; +use core::analysis::{Token, TokenStream}; use error::Result; use std::fmt; use std::io::Read; -// NOTE: this length is length by byte, so it's different from Lucene's word length -const MAX_WORD_LEN: usize = 511; +use unicode_reader::CodePoints; +// NOTE: this length is length by byte, so it's different from Lucene's word length +const MAX_BYTES_LEN: usize = 511; const IO_BUFFER_SIZE: usize = 4096; /// A tokenizer that divides text at whitespace characters. @@ -35,9 +33,8 @@ pub struct WhitespaceTokenizer { buffer_index: usize, data_len: usize, final_offset: usize, - term_attr: CharTermAttribute, - offset_attr: OffsetAttribute, - position_attr: PositionAttribute, + + token: Token, io_buffer: CharacterBuffer, reader: Box, } @@ -49,8 +46,13 @@ impl fmt::Debug for WhitespaceTokenizer { .field("buffer_index", &self.buffer_index) .field("data_len", &self.data_len) .field("final_offset", &self.final_offset) - .field("term_attr", &self.term_attr) - .field("offset_attr", &self.offset_attr) + .field( + "term", + &String::from_utf8(self.token.term.clone()).unwrap_or("".to_string()), + ) + .field("position", &self.token.position) + .field("start_offset", &self.token.start_offset) + .field("end_offset", &self.token.end_offset) .field("io_buffer", &self.io_buffer) .finish() } @@ -63,39 +65,24 @@ impl WhitespaceTokenizer { buffer_index: 0, data_len: 0, final_offset: 0, - term_attr: CharTermAttribute::new(), - offset_attr: OffsetAttribute::new(), - position_attr: PositionAttribute::new(), + token: Token::new(), io_buffer: CharacterBuffer::new(IO_BUFFER_SIZE), reader, } } - pub fn is_token_char(&self, c: char) -> bool { - !c.is_whitespace() - } - - /// Called on each token character to normalize it before it is added to the - /// token. The default implementation does nothing. Subclasses may use this to, - /// e.g., lowercase tokens. - #[allow(dead_code)] - fn normalize(&self, c: i32) -> i32 { - c - } - - fn clear_attributes(&mut self) { - self.term_attr.clear(); - self.offset_attr.clear(); - } - - fn correct_offset(&self, offset: usize) -> usize { - offset + pub fn push_char(&mut self, c: char) { + let char_len = c.len_utf8(); + let term_len = self.token.term.len(); + self.token.term.resize(term_len + char_len, 0u8); + c.encode_utf8(&mut self.token.term[term_len..]); } } impl TokenStream for WhitespaceTokenizer { - fn increment_token(&mut self) -> Result { - self.clear_attributes(); + fn next_token(&mut self) -> Result { + self.clear_token(); + let mut length = 0; let mut start = -1; // this variable is always initialized let mut end = -1; @@ -108,7 +95,7 @@ impl TokenStream for WhitespaceTokenizer { if length > 0 { break; } else { - self.final_offset = self.correct_offset(self.offset); + self.final_offset = self.offset; return Ok(false); } } @@ -118,7 +105,7 @@ impl TokenStream for WhitespaceTokenizer { let cur_char = self.io_buffer.char_at(self.buffer_index); self.buffer_index += 1; - if self.is_token_char(cur_char) { + if !cur_char.is_whitespace() { if length == 0 { debug_assert_eq!(start, -1); start = (self.offset + self.buffer_index - 1) as isize; @@ -126,8 +113,8 @@ impl TokenStream for WhitespaceTokenizer { } end += 1; length += cur_char.len_utf8(); - self.term_attr.push_char(cur_char); - if self.term_attr.len() >= MAX_WORD_LEN { + self.push_char(cur_char); + if self.token.term.len() >= MAX_BYTES_LEN { break; } } else if length > 0 { @@ -136,16 +123,16 @@ impl TokenStream for WhitespaceTokenizer { } assert_ne!(start, -1); - let final_start = self.correct_offset(start as usize); - let final_end = self.correct_offset(end as usize); - self.final_offset = final_end; - self.offset_attr.set_offset(final_start, final_end)?; + + self.final_offset = end as usize; + self.token.set_offset(start as usize, end as usize)?; + Ok(true) } fn end(&mut self) -> Result<()> { - self.offset_attr.end(); - self.term_attr.end(); + self.end_token(); + Ok(()) } @@ -158,24 +145,68 @@ impl TokenStream for WhitespaceTokenizer { Ok(()) } - fn offset_attribute_mut(&mut self) -> &mut OffsetAttribute { - &mut self.offset_attr + fn token(&self) -> &Token { + &self.token + } + + fn token_mut(&mut self) -> &mut Token { + &mut self.token + } +} + +/// a simple IO buffer to use +#[derive(Debug)] +pub struct CharacterBuffer { + pub buffer: Vec, + pub offset: usize, + pub length: usize, +} + +impl CharacterBuffer { + pub fn new(buffer_size: usize) -> Self { + if buffer_size < 2 { + panic!("buffer size must be >= 2"); + } + CharacterBuffer { + buffer: vec!['\0'; buffer_size], + offset: 0, + length: 0, + } } - fn offset_attribute(&self) -> &OffsetAttribute { - &self.offset_attr + pub fn is_empty(&self) -> bool { + self.length == 0 } - fn position_attribute_mut(&mut self) -> &mut PositionAttribute { - &mut self.position_attr + pub fn char_at(&self, index: usize) -> char { + debug_assert!(index < self.buffer.len()); + self.buffer[index] } - fn term_bytes_attribute_mut(&mut self) -> &mut dyn TermToBytesRefAttribute { - &mut self.term_attr + pub fn reset(&mut self) { + self.offset = 0; + self.length = 0; } - fn term_bytes_attribute(&self) -> &dyn TermToBytesRefAttribute { - &self.term_attr + pub fn fill(&mut self, reader: &mut T) -> Result { + let mut unicode_reader = CodePoints::from(reader); + self.offset = 0; + let mut offset = 0; + + loop { + if offset >= self.buffer.len() { + break; + } + if let Some(res) = unicode_reader.next() { + let cur_char = res?; + self.buffer[offset] = cur_char; + offset += 1; + } else { + break; + } + } + self.length = offset; + Ok(self.buffer.len() == self.length) } } @@ -204,15 +235,12 @@ mod tests { let mut tokenizer = WhitespaceTokenizer::new(reader); for i in 0..9 { - let res = tokenizer.increment_token(); // Ok(true) + let res = tokenizer.next_token(); // Ok(true) assert!(res.is_ok()); assert!(res.unwrap()); - assert_eq!(tokenizer.offset_attribute().start_offset(), offsets[i].0); - assert_eq!(tokenizer.offset_attribute().end_offset(), offsets[i].1); - assert_eq!( - tokenizer.term_bytes_attribute().get_bytes_ref().bytes(), - words[i].as_bytes() - ); + assert_eq!(tokenizer.token().start_offset, offsets[i].0); + assert_eq!(tokenizer.token().end_offset, offsets[i].1); + assert_eq!(tokenizer.token().term.as_slice(), words[i].as_bytes()); } } } diff --git a/src/core/codec/doc_values/doc_values_iterator.rs b/src/core/codec/doc_values/doc_values_iterator.rs index a53f4ff..1d6061d 100644 --- a/src/core/codec/doc_values/doc_values_iterator.rs +++ b/src/core/codec/doc_values/doc_values_iterator.rs @@ -1,8 +1,7 @@ -use core::codec::Codec; use core::codec::doc_values::{ - BinaryDocValues, NumericDocValues, - SortedDocValues, SortedNumericDocValues, SortedSetDocValues, + BinaryDocValues, NumericDocValues, SortedDocValues, SortedNumericDocValues, SortedSetDocValues, }; +use core::codec::Codec; use core::doc::DocValuesType; use core::index::reader::LeafReaderContext; use core::search::DocIterator; @@ -64,7 +63,6 @@ impl DocValuesIterator { } fn exists(&mut self, doc_id: DocId) -> Result { - match self.doc_values { DocValuesEnum::Binary(ref mut binary) => { if let Ok(v) = binary.get(doc_id) { @@ -75,7 +73,6 @@ impl DocValuesIterator { if let Ok(_) = numeric.get(doc_id) { return Ok(true); } - } DocValuesEnum::Sorted(ref mut sorted) => { if let Ok(o) = sorted.get_ord(doc_id) { @@ -90,7 +87,7 @@ impl DocValuesIterator { DocValuesEnum::SortedSet(ref mut sorted_set) => { if let Ok(()) = sorted_set.set_document(doc_id) { if let Ok(o) = sorted_set.next_ord() { - return Ok(o > -1) + return Ok(o > -1); } } } @@ -143,4 +140,4 @@ impl DocIterator for DocValuesIterator { fn cost(&self) -> usize { self.cost as usize } -} \ No newline at end of file +} diff --git a/src/core/codec/postings/terms_hash_per_field.rs b/src/core/codec/postings/terms_hash_per_field.rs index 31e0731..81a477e 100644 --- a/src/core/codec/postings/terms_hash_per_field.rs +++ b/src/core/codec/postings/terms_hash_per_field.rs @@ -20,9 +20,9 @@ use core::doc::Fieldable; use core::doc::IndexOptions; use core::index::merge::{MergePolicy, MergeScheduler}; use core::store::directory::Directory; -use core::util::DocId; use core::util::UnsignedShift; use core::util::{ByteBlockPool, ByteSliceReader, BytesRefHash, BytesStartArray, IntBlockPool}; +use core::util::{BytesRef, DocId}; use core::util::{INT_BLOCK_MASK, INT_BLOCK_SHIFT, INT_BLOCK_SIZE}; use std::cmp::{max, Ordering}; @@ -291,7 +291,7 @@ pub trait TermsHashPerField: Ord + PartialOrd + Eq + PartialEq { ) -> Result<()> { // We are first in the chain so we must "insert" the // term text into text_start address - let bytes_ref = token_stream.term_bytes_attribute().get_bytes_ref(); + let bytes_ref = BytesRef::new(&token_stream.token().term); let term_id = unsafe { self.base_mut().bytes_hash.get_mut().add(&bytes_ref) }; if term_id >= 0 { @@ -481,16 +481,12 @@ where field_state: &FieldInvertState, token_stream: &dyn TokenStream, ) -> Result<()> { - if let Some(payload_attr) = token_stream.payload_attribute() { - let payload = payload_attr.get_payload(); - if !payload.is_empty() { - self.base.write_vint(1, (prox_code << 1 | 1) as i32); - self.base.write_vint(1, payload.len() as i32); - self.base.write_bytes(1, payload); - self.saw_payloads = true; - } else { - self.base.write_vint(1, (prox_code << 1) as i32); - } + let payload = &token_stream.token().payload; + if !payload.is_empty() { + self.base.write_vint(1, (prox_code << 1 | 1) as i32); + self.base.write_vint(1, payload.len() as i32); + self.base.write_bytes(1, payload); + self.saw_payloads = true; } else { self.base.write_vint(1, (prox_code << 1) as i32); } @@ -505,8 +501,8 @@ where offset_accum: usize, token_stream: &dyn TokenStream, ) { - let start_offset = (offset_accum + token_stream.offset_attribute().start_offset()) as u32; - let end_offset = (offset_accum + token_stream.offset_attribute().end_offset()) as u32; + let start_offset = (offset_accum + token_stream.token().start_offset) as u32; + let end_offset = (offset_accum + token_stream.token().end_offset) as u32; debug_assert!(start_offset >= self.base.postings_array.last_offsets[term_id]); let value = (start_offset - self.base.postings_array.last_offsets[term_id]) as i32; self.base.write_vint(1, value); diff --git a/src/core/codec/term_vectors/term_vector_consumer.rs b/src/core/codec/term_vectors/term_vector_consumer.rs index dd6a05b..6316eb1 100644 --- a/src/core/codec/term_vectors/term_vector_consumer.rs +++ b/src/core/codec/term_vectors/term_vector_consumer.rs @@ -713,8 +713,8 @@ where token_stream: &dyn TokenStream, ) { if self.do_vector_offsets { - let start_offset = field_state.offset + token_stream.offset_attribute().start_offset(); - let end_offset = field_state.offset + token_stream.offset_attribute().end_offset(); + let start_offset = field_state.offset + token_stream.token().start_offset; + let end_offset = field_state.offset + token_stream.token().end_offset; let delta = start_offset as i32 - self.base.postings_array.last_offsets[term_id] as i32; self.base.write_vint(1, delta); @@ -724,8 +724,8 @@ where if self.do_vector_positions { let mut payload: &[u8] = &[0u8; 0]; - if let Some(attr) = token_stream.payload_attribute() { - payload = attr.get_payload(); + if token_stream.token().payload.len() > 0 { + payload = &token_stream.token().payload; } let pos = field_state.position - self.base.postings_array.last_positions[term_id] as i32; diff --git a/src/core/index/writer/doc_consumer.rs b/src/core/index/writer/doc_consumer.rs index e53e56b..93b9a31 100644 --- a/src/core/index/writer/doc_consumer.rs +++ b/src/core/index/writer/doc_consumer.rs @@ -892,7 +892,7 @@ impl PerField { .start(&self.invert_state, field, first)?; loop { - let end = token_stream.increment_token()?; + let end = token_stream.next_token()?; if !end { break; } @@ -903,7 +903,7 @@ impl PerField { // non-aborting and (above) this one document // will be marked as deleted, but still // consume a docID - let pos_incr = token_stream.position_attribute_mut().get_position(); + let pos_incr = token_stream.token().position; self.invert_state.position += pos_incr as i32; if self.invert_state.position < self.invert_state.last_position { if pos_incr == 0 { @@ -926,10 +926,8 @@ impl PerField { } if check_offset { - let start_offset = - self.invert_state.offset + token_stream.offset_attribute_mut().start_offset(); - let end_offset = - self.invert_state.offset + token_stream.offset_attribute_mut().end_offset(); + let start_offset = self.invert_state.offset + token_stream.token().start_offset; + let end_offset = self.invert_state.offset + token_stream.token().end_offset; if (start_offset as i32) < self.invert_state.last_start_offset || end_offset < start_offset { @@ -964,8 +962,8 @@ impl PerField { // TODO: maybe add some safety? then again, it's already checked // when we come back around to the field... - self.invert_state.position += token_stream.position_attribute_mut().get_position() as i32; - self.invert_state.offset += token_stream.offset_attribute_mut().end_offset(); + self.invert_state.position += token_stream.token().position as i32; + self.invert_state.offset += token_stream.token().end_offset; // if analyzed { // self.invert_state.position += doc_state.analyzer.get_position_increment_gap(); diff --git a/src/core/index/writer/index_writer.rs b/src/core/index/writer/index_writer.rs index 29ee6b6..91fab96 100644 --- a/src/core/index/writer/index_writer.rs +++ b/src/core/index/writer/index_writer.rs @@ -1722,12 +1722,11 @@ where Ok(seq_no) } - // _l is self.commit_lock fn prepare_commit_internal( &mut self, do_maybe_merge: &mut bool, index_writer: &IndexWriter, - _l: &MutexGuard<()>, + commit_lock: &MutexGuard<()>, ) -> Result { // self.start_commit_time = SystemTime::now(); self.ensure_open(false)?; @@ -1784,7 +1783,7 @@ where if any_segments_flushed { *do_maybe_merge = true; } - let res = self.start_commit(to_commit); + let res = self.start_commit(to_commit, commit_lock); if res.is_err() { let lock = Arc::clone(&self.lock); let _l = lock.lock().unwrap(); @@ -1864,7 +1863,11 @@ where /// if it wasn't already. If that succeeds, then we /// prepare a new segments_N file but do not fully commit /// it. - fn start_commit(&mut self, to_sync: SegmentInfos) -> Result<()> { + fn start_commit( + &mut self, + to_sync: SegmentInfos, + commit_lock: &MutexGuard<()>, + ) -> Result<()> { debug_assert!(self.pending_commit.is_none()); if let Some(ref tragedy) = self.tragedy { @@ -1917,7 +1920,7 @@ where } } if let Err(e) = err { - self.tragic_event(e, "start_commit")?; + self.tragic_event(e, "start_commit", Some(commit_lock))?; } Ok(()) } @@ -2718,7 +2721,7 @@ where /// Merges the indicated segments, replacing them in the stack with a single segment. fn merge(index_writer: &IndexWriter, merge: &mut OneMerge) -> Result<()> { if let Err(e) = Self::do_merge(index_writer, merge) { - index_writer.writer.tragic_event(e, "merge")?; + index_writer.writer.tragic_event(e, "merge", None)?; } Ok(()) @@ -3634,7 +3637,12 @@ where Ok(()) } - fn tragic_event(&self, tragedy: Error, location: &str) -> Result<()> { + fn tragic_event( + &self, + tragedy: Error, + location: &str, + commit_lock: Option<&MutexGuard<()>>, + ) -> Result<()> { trace!("IW - hit tragic '{:?}' inside {}", &tragedy, location); { @@ -3651,8 +3659,12 @@ where // if we are already closed (e.g. called by rollback), this will be a no-op. if self.should_close(false) { - let commit_lock = self.commit_lock.lock()?; - self.rollback_internal(&commit_lock)?; + if let Some(lock) = commit_lock { + self.rollback_internal(lock)?; + } else { + let lock = self.commit_lock.lock()?; + self.rollback_internal(&lock)?; + } } bail!(IllegalState(format!( diff --git a/src/core/search/query/exists_query.rs b/src/core/search/query/exists_query.rs index 6c942d5..fa0d50b 100644 --- a/src/core/search/query/exists_query.rs +++ b/src/core/search/query/exists_query.rs @@ -1,8 +1,8 @@ use std::any::Any; use std::fmt; -use core::codec::Codec; use core::codec::doc_values::DocValuesIterator; +use core::codec::Codec; use core::index::reader::LeafReaderContext; use core::search::explanation::Explanation; use core::search::query::{Query, TermQuery, Weight}; @@ -19,12 +19,8 @@ pub struct ExistsQuery { } impl ExistsQuery { - pub fn build( - field: String, - ) -> ExistsQuery { - ExistsQuery { - field, - } + pub fn build(field: String) -> ExistsQuery { + ExistsQuery { field } } } @@ -34,10 +30,7 @@ impl Query for ExistsQuery { _searcher: &dyn SearchPlanBuilder, _needs_scores: bool, ) -> Result>> { - Ok(Box::new( - ExistsWeight::new( - self.field.clone(), - ))) + Ok(Box::new(ExistsWeight::new(self.field.clone()))) } fn extract_terms(&self) -> Vec { @@ -51,11 +44,7 @@ impl Query for ExistsQuery { impl fmt::Display for ExistsQuery { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!( - f, - "ExistsQuery(field={})", - &self.field - ) + write!(f, "ExistsQuery(field={})", &self.field) } } @@ -66,9 +55,7 @@ struct ExistsWeight { } impl ExistsWeight { - pub fn new( - field: String, - ) -> ExistsWeight { + pub fn new(field: String) -> ExistsWeight { ExistsWeight { field, weight: 0f32, @@ -86,7 +73,11 @@ impl Weight for ExistsWeight { let cost: i32 = leaf_reader.reader.max_doc(); let doc_iterator = DocValuesIterator::new(field_info.name.as_str(), cost, leaf_reader); - return Ok(Some(Box::new(ConstantScoreScorer::new(self.weight, doc_iterator, cost as usize)))); + return Ok(Some(Box::new(ConstantScoreScorer::new( + self.weight, + doc_iterator, + cost as usize, + )))); } Ok(None) @@ -118,14 +109,16 @@ impl Weight for ExistsWeight { true, self.weight, format!("{}, product of:", self), - vec![ - Explanation::new(true, self.weight, "exists".to_string(), vec![]), - ], + vec![Explanation::new( + true, + self.weight, + "exists".to_string(), + vec![], + )], )) } } - impl fmt::Display for ExistsWeight { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { write!( From 404429bc6d9422c34e511bf6a68392ff57739d39 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Thu, 7 May 2020 16:52:28 +0800 Subject: [PATCH 25/49] fix payload --- src/core/codec/postings/posting_reader.rs | 6 +++--- src/core/doc/field.rs | 21 ++++++++++----------- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/core/codec/postings/posting_reader.rs b/src/core/codec/postings/posting_reader.rs index 2ce975e..2b74b7e 100644 --- a/src/core/codec/postings/posting_reader.rs +++ b/src/core/codec/postings/posting_reader.rs @@ -1819,11 +1819,11 @@ impl PostingIterator for EverythingIterator { } fn payload(&self) -> Result { - if self.payload_length == 0 { + if self.payload_length == 0 || self.payload_byte_upto < self.payload_length { Ok(vec![]) } else { - let start = self.payload_byte_upto as usize; - let end = start + self.payload_length as usize; + let end = self.payload_byte_upto as usize; + let start = end - self.payload_length as usize; Ok(self.payload_bytes[start..end].to_vec()) } } diff --git a/src/core/doc/field.rs b/src/core/doc/field.rs index 396ca8c..61ea8ba 100644 --- a/src/core/doc/field.rs +++ b/src/core/doc/field.rs @@ -82,6 +82,10 @@ impl Fieldable for Field { // TODO currently this function should only be called once per doc field fn token_stream(&mut self) -> Result> { + if self.token_stream.is_some() { + return Ok(self.token_stream.take().unwrap()); + } + debug_assert_ne!(self.field_type.index_options, IndexOptions::Null); if !self.field_type.tokenized { @@ -93,21 +97,16 @@ impl Fieldable for Field { VariantValue::Binary(b) => { return Ok(Box::new(BinaryTokenStream::new(BytesRef::new(b.as_ref())))); } - _ => { - bail!(ErrorKind::IllegalArgument( - "Non-Tokenized Fields must have a String value".into() - )); - } + _ => bail!(ErrorKind::IllegalArgument( + "Non-Tokenized Fields must have a String value".into() + )), } } } - if self.token_stream.is_some() { - Ok(self.token_stream.take().unwrap()) - } else { - // TODO currently not support analyzer - unimplemented!() - } + bail!(ErrorKind::IllegalArgument( + "Tokenized field's token_stream should not be None ".into() + )) } fn binary_value(&self) -> Option<&[u8]> { From 4faf7912fcc0b5db42d56097f3a10b3cc3e57154 Mon Sep 17 00:00:00 2001 From: wangjingjing Date: Mon, 18 May 2020 14:58:24 +0800 Subject: [PATCH 26/49] partial decode block --- src/core/codec/postings/for_util.rs | 44 +++- src/core/codec/postings/mod.rs | 4 + .../codec/postings/partial_block_decoder.rs | 193 ++++++++++++++++++ src/core/codec/postings/posting_reader.rs | 43 +++- 4 files changed, 267 insertions(+), 17 deletions(-) create mode 100644 src/core/codec/postings/partial_block_decoder.rs diff --git a/src/core/codec/postings/for_util.rs b/src/core/codec/postings/for_util.rs index d6263cb..3d49060 100644 --- a/src/core/codec/postings/for_util.rs +++ b/src/core/codec/postings/for_util.rs @@ -19,7 +19,7 @@ use core::store::io::{DataOutput, IndexInput, IndexOutput}; use core::util::packed::*; use core::util::{BitSet, BitsRequired, DocId, FixedBitSet}; -use core::codec::postings::{EfWriterMeta, EncodeType}; +use core::codec::postings::{EfWriterMeta, EncodeType, PartialBlockDecoder}; use error::Result; use std::mem::MaybeUninit; use std::ptr; @@ -180,6 +180,7 @@ impl ForUtilInstance { encoded: &mut [u8], decoded: &mut [i32], encode_type: Option<&mut EncodeType>, + partial_decoder: Option<&mut PartialBlockDecoder>, ) -> Result<()> { let code = input.read_byte()?; if encode_type.is_some() { @@ -197,19 +198,30 @@ impl ForUtilInstance { if num_bits as i32 == ALL_VALUES_EQUAL { let value = input.read_vint()?; - decoded[0..BLOCK_SIZE as usize] - .iter_mut() - .map(|x| *x = value) - .count(); + if let Some(p) = partial_decoder { + p.set_single(value); + } else { + decoded[0..BLOCK_SIZE as usize] + .iter_mut() + .map(|x| *x = value) + .count(); + } return Ok(()); } let encoded_size = self.encoded_sizes[num_bits - 1]; - input.read_exact(&mut encoded[0..encoded_size as usize])?; - let decoder = unsafe { &self.decoders.get_ref()[num_bits - 1] }; - let iters = self.iterations[num_bits - 1] as usize; - decoder.decode_byte_to_int(encoded, decoded, iters); + if let Some(p) = partial_decoder { + let format = match decoder { + &BulkOperationEnum::Packed(_) => Format::Packed, + &BulkOperationEnum::PackedSB(_) => Format::PackedSingleBlock, + }; + p.parse_from(input, encoded_size as usize, num_bits, format)?; + } else { + input.read_exact(&mut encoded[0..encoded_size as usize])?; + let iters = self.iterations[num_bits - 1] as usize; + decoder.decode_byte_to_int(encoded, decoded, iters); + } Ok(()) } @@ -257,7 +269,19 @@ impl ForUtil { encode_type: Option<&mut EncodeType>, ) -> Result<()> { self.instance - .read_block(input, encoded, decoded, encode_type) + .read_block(input, encoded, decoded, encode_type, None) + } + + pub fn read_block_only( + &self, + input: &mut dyn IndexInput, + encoded: &mut [u8], + decoded: &mut [i32], + encode_type: Option<&mut EncodeType>, + partial_decoder: &mut PartialBlockDecoder, + ) -> Result<()> { + self.instance + .read_block(input, encoded, decoded, encode_type, Some(partial_decoder)) } pub fn read_other_encode_block( diff --git a/src/core/codec/postings/mod.rs b/src/core/codec/postings/mod.rs index d305d1e..1ce9530 100644 --- a/src/core/codec/postings/mod.rs +++ b/src/core/codec/postings/mod.rs @@ -49,6 +49,10 @@ mod terms_hash_per_field; pub use self::terms_hash_per_field::*; +mod partial_block_decoder; + +pub use self::partial_block_decoder::*; + use core::codec::field_infos::FieldInfo; use core::codec::multi_fields::{MappedMultiFields, MultiFields}; use core::codec::postings::blocktree::{ diff --git a/src/core/codec/postings/partial_block_decoder.rs b/src/core/codec/postings/partial_block_decoder.rs new file mode 100644 index 0000000..6e71527 --- /dev/null +++ b/src/core/codec/postings/partial_block_decoder.rs @@ -0,0 +1,193 @@ +use core::codec::postings::MAX_ENCODED_SIZE; +use core::store::io::IndexInput; +use core::util::packed::Format; +use error::Result; +use std::convert::TryInto; +use std::intrinsics::unlikely; + +pub struct PartialBlockDecoder { + encoded: [u8; MAX_ENCODED_SIZE + 8], + index: usize, + data: u64, + bits_num: usize, + format: Format, + is_single: bool, + block_capacity: usize, + mask: u64, +} + +impl PartialBlockDecoder { + pub fn new() -> Self { + Self { + encoded: [0u8; MAX_ENCODED_SIZE + 8], + index: 0, + data: 0, + bits_num: 0, + format: Format::Packed, + is_single: false, + block_capacity: 0, + mask: 0, + } + } + + pub fn set_single(&mut self, value: i32) { + self.is_single = true; + self.data = value as u64; + } + + pub fn parse_from( + &mut self, + input: &mut dyn IndexInput, + num: usize, + bits_num: usize, + format: Format, + ) -> Result<()> { + self.is_single = false; + input.read_exact(&mut self.encoded[0..num])?; + self.index = 0; + self.bits_num = bits_num; + self.format = format; + self.block_capacity = 64 / bits_num; + self.mask = (1 << self.bits_num) as u64 - 1; + Ok(()) + } + + #[inline(always)] + pub fn get(&mut self, index: usize) -> i32 { + if unsafe { unlikely(self.is_single) } { + self.data as i32 + } else { + match self.format { + Format::Packed => { + let bit_index = index * self.bits_num; + let pos = bit_index >> 3; + let target = u64::from_be_bytes(self.encoded[pos..pos + 8].try_into().unwrap()); + let left_bits = bit_index & 7usize; + ((target >> (64 - left_bits - self.bits_num)) & self.mask) as i32 + } + Format::PackedSingleBlock => { + let block_index = index / self.block_capacity; + let pos = block_index << 3; + let target = u64::from_be_bytes(self.encoded[pos..pos + 8].try_into().unwrap()); + ((target >> index % self.block_capacity * self.bits_num) & self.mask) as i32 + } + } + } + } + + #[inline(always)] + pub fn next(&mut self) -> i32 { + if unsafe { unlikely(self.is_single) } { + self.data as i32 + } else { + match self.format { + Format::Packed => { + let pos = self.index >> 3; + let mut target = + u64::from_be_bytes(self.encoded[pos..pos + 8].try_into().unwrap()); + target = (target >> (64 - (self.index & 7) - self.bits_num)) & self.mask; + self.index += self.bits_num; + target as i32 + } + Format::PackedSingleBlock => { + let inner_index = self.index % self.block_capacity; + if inner_index == 0 { + let pos = self.index / self.block_capacity << 3; + self.data = + u64::from_be_bytes(self.encoded[pos..pos + 8].try_into().unwrap()); + } + self.index += 1; + ((self.data >> (inner_index * self.bits_num) as u64) & self.mask) as i32 + } + } + } + } +} + +#[cfg(test)] +mod tests { + use core::codec::postings::PartialBlockDecoder; + use core::util::fst::{BytesStore, StoreBytesReader}; + use core::util::packed::Format; + use error::Result; + use std::io::{Read, Write}; + + #[test] + fn test_get() -> Result<()> { + // Test Packed block + let mut decoder = create_decoder(&[0xFFu8, 0xFF, 0, 0xFF], 4, Format::Packed); + assert_eq!(decoder.get(0), 0xFi32); + assert_eq!(decoder.get(1), 0xFi32); + assert_eq!(decoder.get(2), 0xFi32); + assert_eq!(decoder.get(3), 0xFi32); + assert_eq!(decoder.get(4), 0x0i32); + assert_eq!(decoder.get(5), 0x0i32); + assert_eq!(decoder.get(6), 0xFi32); + assert_eq!(decoder.get(7), 0xFi32); + + // Test PackedSingleBlock + let data = [ + 0xFFu8, 0xF, 0, 0, 0, 0, 0xFF, 0, 0x8F, 0xFF, 0x8F, 0x8F, 0x8F, 0x8F, 0x8F, 0x8F, + ]; + let mut decoder = create_decoder(&data, 6, Format::PackedSingleBlock); + assert_eq!(decoder.get(0), 0); + assert_eq!(decoder.get(1), 0x3C); + assert_eq!(decoder.get(9), 0x3C); + assert_eq!(decoder.get(10), 0xF); + Ok(()) + } + + #[test] + fn test_next() -> Result<()> { + // Test Packed block + let mut decoder = create_decoder(&[0xFFu8, 0xFF, 0, 0xFF], 4, Format::Packed); + assert_eq!(decoder.next(), 0xFi32); + assert_eq!(decoder.next(), 0xFi32); + assert_eq!(decoder.next(), 0xFi32); + assert_eq!(decoder.next(), 0xFi32); + assert_eq!(decoder.next(), 0x0i32); + assert_eq!(decoder.next(), 0x0i32); + assert_eq!(decoder.next(), 0xFi32); + assert_eq!(decoder.next(), 0xFi32); + + // Test PackedSingleBlock + let data = [ + 0xFFu8, 0xF, 0, 0, 0, 0, 0xFF, 0, 0x8F, 0xFF, 0x8F, 0x8F, 0x8F, 0x8F, 0x8F, 0x8F, + ]; + let mut decoder = create_decoder(&data, 6, Format::PackedSingleBlock); + assert_eq!(decoder.next(), 0); + assert_eq!(decoder.next(), 0x3C); + assert_eq!(decoder.next(), 0xF); + assert_eq!(decoder.next(), 0); + for _i in 1..=6 { + decoder.next(); + } + assert_eq!(decoder.next(), 0xF); + assert_eq!(decoder.next(), 0x3E); + Ok(()) + } + + #[test] + fn test_single() -> Result<()> { + let mut decoder = PartialBlockDecoder::new(); + decoder.set_single(5); + for i in 0..128 { + assert_eq!(decoder.get(i), 5); + } + for _i in 0..128 { + assert_eq!(decoder.next(), 5); + } + Ok(()) + } + + fn create_decoder(data: &[u8], bits_per_value: usize, format: Format) -> PartialBlockDecoder { + let mut bs = BytesStore::with_block_bits(8); + bs.write(data); + let mut sbr = StoreBytesReader::from_bytes_store(bs, false); + let mut decoder = PartialBlockDecoder::new(); + decoder + .parse_from(&mut sbr, data.len(), bits_per_value, format) + .unwrap(); + decoder + } +} diff --git a/src/core/codec/postings/posting_reader.rs b/src/core/codec/postings/posting_reader.rs index 2b74b7e..361feb4 100644 --- a/src/core/codec/postings/posting_reader.rs +++ b/src/core/codec/postings/posting_reader.rs @@ -27,8 +27,9 @@ use core::util::{Bits, DocId, FixedBitSet, ImmutableBitSet}; use error::{ErrorKind::IllegalState, Result}; +use core::codec::postings::PartialBlockDecoder; use core::util::packed::{EliasFanoDecoder, NO_MORE_VALUES}; -use std::intrinsics::unlikely; +use std::intrinsics::{likely, unlikely}; use std::sync::Arc; /// Filename extension for document number, frequencies, and skip data. @@ -385,6 +386,9 @@ struct BlockDocIterator { doc_bits: FixedBitSet, bits_min_doc: DocId, bits_index: i32, + partial_decode: bool, + partial_doc_deltas: PartialBlockDecoder, + partial_freqs: PartialBlockDecoder, } impl BlockDocIterator { @@ -428,6 +432,9 @@ impl BlockDocIterator { doc_bits: FixedBitSet::default(), bits_min_doc: 0, bits_index: 0, + partial_decode: false, + partial_doc_deltas: PartialBlockDecoder::new(), + partial_freqs: PartialBlockDecoder::new(), }; iterator.reset(term_state, flags)?; Ok(iterator) @@ -482,16 +489,19 @@ impl BlockDocIterator { self.ef_base_total = self.doc_upto; self.encode_type = EncodeType::PF; self.bits_index = 0; + self.partial_decode = false; let left = self.doc_freq - self.doc_upto; debug_assert!(left > 0); if left >= BLOCK_SIZE { + self.partial_decode = true; let doc_in = self.doc_in.as_mut().unwrap(); - self.for_util.read_block( + self.for_util.read_block_only( doc_in.as_mut(), &mut self.encoded, &mut self.doc_delta_buffer, Some(&mut self.encode_type), + &mut self.partial_doc_deltas, )?; ForUtil::read_other_encode_block( @@ -504,11 +514,12 @@ impl BlockDocIterator { if self.index_has_freq { if self.needs_freq { - self.for_util.read_block( + self.for_util.read_block_only( doc_in.as_mut(), &mut self.encoded, &mut self.freq_buffer, None, + &mut self.partial_freqs, )?; } else { self.for_util.skip_block(doc_in.as_mut())?; // skip over freqs @@ -531,6 +542,24 @@ impl BlockDocIterator { self.doc_buffer_upto = 0; Ok(()) } + + #[inline(always)] + fn decode_current_doc_delta(&mut self) -> i32 { + if unsafe { likely(self.partial_decode) } { + self.partial_doc_deltas.next() + } else { + self.doc_delta_buffer[self.doc_buffer_upto as usize] + } + } + + #[inline(always)] + fn decode_current_freq(&mut self) -> i32 { + if unsafe { likely(self.partial_decode && self.needs_freq) } { + self.partial_freqs.get(self.doc_buffer_upto as usize) + } else { + self.freq_buffer[self.doc_buffer_upto as usize] + } + } } impl PostingIterator for BlockDocIterator { @@ -572,7 +601,7 @@ impl DocIterator for BlockDocIterator { // set doc id self.doc = match self.encode_type { - EncodeType::PF => self.accum + self.doc_delta_buffer[self.doc_buffer_upto as usize], + EncodeType::PF => self.accum + self.decode_current_doc_delta(), EncodeType::EF => { self.ef_decoder.as_mut().unwrap().next_value() as i32 + 1 + self.ef_base_doc @@ -594,7 +623,7 @@ impl DocIterator for BlockDocIterator { self.doc_upto += 1; // set doc freq - self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; + self.freq = self.decode_current_freq(); self.doc_buffer_upto += 1; Ok(self.doc) } @@ -668,7 +697,7 @@ impl DocIterator for BlockDocIterator { // Now scan... this is an inlined/pared down version // of nextDoc(): loop { - self.accum += self.doc_delta_buffer[self.doc_buffer_upto as usize]; + self.accum += self.decode_current_doc_delta(); self.doc_upto += 1; if self.accum >= target { @@ -736,7 +765,7 @@ impl DocIterator for BlockDocIterator { } }; - self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; + self.freq = self.decode_current_freq(); self.doc_buffer_upto += 1; Ok(self.doc) } From c9229a966fe9feeb3056f7b6db86f09d3286b2eb Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Mon, 25 May 2020 19:39:08 +0800 Subject: [PATCH 27/49] fix payload panic --- src/core/codec/postings/partial_block_decoder.rs | 4 ++-- src/core/codec/postings/posting_reader.rs | 7 +++---- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/core/codec/postings/partial_block_decoder.rs b/src/core/codec/postings/partial_block_decoder.rs index 6e71527..4db1ba3 100644 --- a/src/core/codec/postings/partial_block_decoder.rs +++ b/src/core/codec/postings/partial_block_decoder.rs @@ -54,7 +54,7 @@ impl PartialBlockDecoder { #[inline(always)] pub fn get(&mut self, index: usize) -> i32 { - if unsafe { unlikely(self.is_single) } { + if unlikely(self.is_single) { self.data as i32 } else { match self.format { @@ -77,7 +77,7 @@ impl PartialBlockDecoder { #[inline(always)] pub fn next(&mut self) -> i32 { - if unsafe { unlikely(self.is_single) } { + if unlikely(self.is_single) { self.data as i32 } else { match self.format { diff --git a/src/core/codec/postings/posting_reader.rs b/src/core/codec/postings/posting_reader.rs index 361feb4..9d96c53 100644 --- a/src/core/codec/postings/posting_reader.rs +++ b/src/core/codec/postings/posting_reader.rs @@ -545,7 +545,7 @@ impl BlockDocIterator { #[inline(always)] fn decode_current_doc_delta(&mut self) -> i32 { - if unsafe { likely(self.partial_decode) } { + if likely(self.partial_decode) { self.partial_doc_deltas.next() } else { self.doc_delta_buffer[self.doc_buffer_upto as usize] @@ -554,7 +554,7 @@ impl BlockDocIterator { #[inline(always)] fn decode_current_freq(&mut self) -> i32 { - if unsafe { likely(self.partial_decode && self.needs_freq) } { + if likely(self.partial_decode && self.needs_freq) { self.partial_freqs.get(self.doc_buffer_upto as usize) } else { self.freq_buffer[self.doc_buffer_upto as usize] @@ -1621,7 +1621,6 @@ impl<'a> EverythingIterator { let count = (self.total_term_freq % i64::from(BLOCK_SIZE)) as usize; let mut payload_length = 0i32; let mut offset_length = 0i32; - let payload_byte_upto = 0i32; for i in 0..count { let code = pos_in.read_vint()?; if self.index_has_payloads { @@ -1634,7 +1633,7 @@ impl<'a> EverythingIterator { if self.payload_byte_upto + payload_length > self.payload_bytes.len() as i32 { self.payload_bytes - .resize((payload_byte_upto + payload_length) as usize, 0); + .resize((self.payload_byte_upto + payload_length) as usize, 0); } pos_in.read_exact( &mut self.payload_bytes[self.payload_byte_upto as usize From 6629d2f7971bdc7ff113dbfa627b291bbfb257e9 Mon Sep 17 00:00:00 2001 From: wangjingjing Date: Tue, 2 Jun 2020 11:04:18 +0800 Subject: [PATCH 28/49] optimize encode/decode by simd --- Cargo.toml | 1 + src/core/codec/postings/for_util.rs | 89 ++- src/core/codec/postings/mod.rs | 4 + .../codec/postings/partial_block_decoder.rs | 4 +- src/core/codec/postings/posting_reader.rs | 321 +++++++++- src/core/codec/postings/posting_writer.rs | 19 +- src/core/codec/postings/simd_block_decoder.rs | 184 ++++++ src/core/store/io/index_input.rs | 4 + src/core/store/io/mmap_index_input.rs | 8 + src/core/util/packed/mod.rs | 4 + src/core/util/packed/packed_simd.rs | 551 ++++++++++++++++++ src/lib.rs | 3 + 12 files changed, 1168 insertions(+), 24 deletions(-) create mode 100644 src/core/codec/postings/simd_block_decoder.rs create mode 100644 src/core/util/packed/packed_simd.rs diff --git a/Cargo.toml b/Cargo.toml index 794f9ad..7a50f83 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -34,6 +34,7 @@ thread_local = "0.3" unicode_reader = "0.1.1" num-traits = "0.2" byteorder = "1" +crunchy = "0.2.2" [dev-dependencies] tempfile = "3.0.8" diff --git a/src/core/codec/postings/for_util.rs b/src/core/codec/postings/for_util.rs index 3d49060..0c985f5 100644 --- a/src/core/codec/postings/for_util.rs +++ b/src/core/codec/postings/for_util.rs @@ -19,10 +19,11 @@ use core::store::io::{DataOutput, IndexInput, IndexOutput}; use core::util::packed::*; use core::util::{BitSet, BitsRequired, DocId, FixedBitSet}; -use core::codec::postings::{EfWriterMeta, EncodeType, PartialBlockDecoder}; +use core::codec::postings::{EfWriterMeta, EncodeType, PartialBlockDecoder, SIMDBlockDecoder}; use error::Result; use std::mem::MaybeUninit; use std::ptr; +use std::slice::{from_raw_parts, from_raw_parts_mut}; /// Special number of bits per value used whenever all values to encode are equal. const ALL_VALUES_EQUAL: i32 = 0; @@ -40,6 +41,15 @@ pub const MAX_ENCODED_SIZE: usize = BLOCK_SIZE as usize * 4; /// something like lazy_static can allow us use [; MAX_DATA_SIZE] instead of Vec. pub const MAX_DATA_SIZE: usize = 147; +lazy_static! { + static ref SIMD_ENCODE_SIZE: [usize; 64] = { + let mut buffer = [0usize; 64]; + for i in 1..=32 { + buffer[i] = i * BLOCK_SIZE as usize / 8; + } + buffer + }; +} #[test] fn test_max_data_size() { assert_eq!(MAX_DATA_SIZE, max_data_size()); @@ -181,6 +191,7 @@ impl ForUtilInstance { decoded: &mut [i32], encode_type: Option<&mut EncodeType>, partial_decoder: Option<&mut PartialBlockDecoder>, + by_simd: bool, ) -> Result<()> { let code = input.read_byte()?; if encode_type.is_some() { @@ -217,6 +228,12 @@ impl ForUtilInstance { &BulkOperationEnum::PackedSB(_) => Format::PackedSingleBlock, }; p.parse_from(input, encoded_size as usize, num_bits, format)?; + } else if by_simd { + let encoded = unsafe { input.get_and_advance(SIMD_ENCODE_SIZE[num_bits]) }; + let decoded = unsafe { + from_raw_parts_mut(decoded.as_mut_ptr() as *mut u32, BLOCK_SIZE as usize) + }; + SIMD128Packer::unpack(encoded, decoded, num_bits as u8); } else { input.read_exact(&mut encoded[0..encoded_size as usize])?; let iters = self.iterations[num_bits - 1] as usize; @@ -225,6 +242,24 @@ impl ForUtilInstance { Ok(()) } + pub fn read_block_by_simd( + &self, + input: &mut dyn IndexInput, + decoder: &mut SIMDBlockDecoder, + ) -> Result<()> { + let code = input.read_byte()?; + let num_bits = (code & 0x3F) as usize; + debug_assert!(num_bits <= 32); + + if num_bits as i32 == ALL_VALUES_EQUAL { + let value = input.read_vint()?; + decoder.set_single(value); + return Ok(()); + } + + decoder.parse_from_no_copy(input, num_bits * BLOCK_SIZE as usize / 8, num_bits) + } + pub fn skip_block(&self, input: &mut dyn IndexInput) -> Result<()> { let num_bits = input.read_byte()? as usize; if num_bits as i32 == ALL_VALUES_EQUAL { @@ -267,9 +302,10 @@ impl ForUtil { encoded: &mut [u8], decoded: &mut [i32], encode_type: Option<&mut EncodeType>, + by_simd: bool, ) -> Result<()> { self.instance - .read_block(input, encoded, decoded, encode_type, None) + .read_block(input, encoded, decoded, encode_type, None, by_simd) } pub fn read_block_only( @@ -280,8 +316,22 @@ impl ForUtil { encode_type: Option<&mut EncodeType>, partial_decoder: &mut PartialBlockDecoder, ) -> Result<()> { - self.instance - .read_block(input, encoded, decoded, encode_type, Some(partial_decoder)) + self.instance.read_block( + input, + encoded, + decoded, + encode_type, + Some(partial_decoder), + false, + ) + } + + pub fn read_block_by_simd( + &self, + input: &mut dyn IndexInput, + decoder: &mut SIMDBlockDecoder, + ) -> Result<()> { + self.instance.read_block_by_simd(input, decoder) } pub fn read_other_encode_block( @@ -349,6 +399,7 @@ impl ForUtil { encoded: &mut [u8], out: &mut impl IndexOutput, ef_writer_meta: Option<&mut EfWriterMeta>, + by_simd: bool, ) -> Result<()> { if Self::is_all_equal(data) { out.write_byte(0)?; @@ -416,8 +467,34 @@ impl ForUtil { } } out.write_byte(num_bits as u8)?; - encoder.encode_int_to_byte(data, encoded, iters as usize); - out.write_bytes(encoded, 0, encoded_size as usize) + if by_simd { + let data = unsafe { from_raw_parts(data.as_ptr() as *const u32, BLOCK_SIZE as usize) }; + SIMD128Packer::pack(data, encoded, num_bits as u8); + out.write_bytes(encoded, 0, SIMD_ENCODE_SIZE[num_bits]) + } else { + encoder.encode_int_to_byte(data, encoded, iters as usize); + out.write_bytes(encoded, 0, encoded_size as usize) + } + } + + pub fn write_block_by_simd( + &self, + data: &[i32], + encoded: &mut [u8], + out: &mut impl IndexOutput, + ) -> Result<()> { + if Self::is_all_equal(data) { + out.write_byte(0)?; + return out.write_vint(data[0]); + } + + let num_bits = Self::bits_required(data) as usize; + assert!(num_bits > 0 && num_bits <= 32); + + out.write_byte(num_bits as u8)?; + let data = unsafe { from_raw_parts(data.as_ptr() as *const u32, BLOCK_SIZE as usize) }; + SIMD128Packer::pack(data, encoded, num_bits as u8); + out.write_bytes(encoded, 0, num_bits * BLOCK_SIZE as usize / 8) } pub fn skip_block(&self, input: &mut dyn IndexInput) -> Result<()> { diff --git a/src/core/codec/postings/mod.rs b/src/core/codec/postings/mod.rs index 1ce9530..1ec3c40 100644 --- a/src/core/codec/postings/mod.rs +++ b/src/core/codec/postings/mod.rs @@ -53,6 +53,10 @@ mod partial_block_decoder; pub use self::partial_block_decoder::*; +mod simd_block_decoder; + +pub use self::simd_block_decoder::*; + use core::codec::field_infos::FieldInfo; use core::codec::multi_fields::{MappedMultiFields, MultiFields}; use core::codec::postings::blocktree::{ diff --git a/src/core/codec/postings/partial_block_decoder.rs b/src/core/codec/postings/partial_block_decoder.rs index 4db1ba3..350b15f 100644 --- a/src/core/codec/postings/partial_block_decoder.rs +++ b/src/core/codec/postings/partial_block_decoder.rs @@ -110,7 +110,7 @@ mod tests { use core::util::fst::{BytesStore, StoreBytesReader}; use core::util::packed::Format; use error::Result; - use std::io::{Read, Write}; + use std::io::Write; #[test] fn test_get() -> Result<()> { @@ -182,7 +182,7 @@ mod tests { fn create_decoder(data: &[u8], bits_per_value: usize, format: Format) -> PartialBlockDecoder { let mut bs = BytesStore::with_block_bits(8); - bs.write(data); + let _ = bs.write(data); let mut sbr = StoreBytesReader::from_bytes_store(bs, false); let mut decoder = PartialBlockDecoder::new(); decoder diff --git a/src/core/codec/postings/posting_reader.rs b/src/core/codec/postings/posting_reader.rs index 9d96c53..5b3dfc3 100644 --- a/src/core/codec/postings/posting_reader.rs +++ b/src/core/codec/postings/posting_reader.rs @@ -27,8 +27,8 @@ use core::util::{Bits, DocId, FixedBitSet, ImmutableBitSet}; use error::{ErrorKind::IllegalState, Result}; -use core::codec::postings::PartialBlockDecoder; -use core::util::packed::{EliasFanoDecoder, NO_MORE_VALUES}; +use core::codec::postings::{PartialBlockDecoder, SIMDBlockDecoder}; +use core::util::packed::{EliasFanoDecoder, SIMD128Packer, SIMDPacker, NO_MORE_VALUES}; use std::intrinsics::{likely, unlikely}; use std::sync::Arc; @@ -54,8 +54,8 @@ pub const POS_CODEC: &str = "Lucene50PostingsWriterPos"; pub const PAY_CODEC: &str = "Lucene50PostingsWriterPay"; // Increment version to change it -const VERSION_START: i32 = 0; -pub const VERSION_CURRENT: i32 = VERSION_START; +pub const VERSION_START: i32 = 0; +pub const VERSION_CURRENT: i32 = VERSION_START + 1; fn clone_option_index_input(input: &Option>) -> Result> { debug_assert!(input.is_some()); @@ -72,6 +72,7 @@ pub struct Lucene50PostingsReader { pay_in: Option>, pub version: i32, pub for_util: ForUtil, + use_simd: bool, } impl Lucene50PostingsReader { @@ -98,6 +99,13 @@ impl Lucene50PostingsReader { &state.segment_info.id, &state.segment_suffix, )?; + + let use_simd = if version > VERSION_START && SIMD128Packer::is_support() { + true + } else { + false + }; + let for_util = ForUtil::with_input(doc_in.as_mut())?; codec_util::retrieve_checksum(doc_in.as_mut())?; let mut pos_in = None; @@ -145,6 +153,7 @@ impl Lucene50PostingsReader { pay_in, version, for_util, + use_simd, }) } @@ -198,6 +207,7 @@ impl Lucene50PostingsReader { state, flags, self.for_util.clone(), + self.use_simd, )?, ))) } else if (!index_has_offsets @@ -213,6 +223,7 @@ impl Lucene50PostingsReader { state, flags, self.for_util.clone(), + self.use_simd, )?, ))) } else { @@ -227,6 +238,7 @@ impl Lucene50PostingsReader { state, flags, self.for_util.clone(), + self.use_simd, )?), )) } @@ -387,8 +399,11 @@ struct BlockDocIterator { bits_min_doc: DocId, bits_index: i32, partial_decode: bool, + #[allow(dead_code)] partial_doc_deltas: PartialBlockDecoder, + #[allow(dead_code)] partial_freqs: PartialBlockDecoder, + use_simd: bool, } impl BlockDocIterator { @@ -398,6 +413,7 @@ impl BlockDocIterator { term_state: &BlockTermState, flags: u16, for_util: ForUtil, + use_simd: bool, ) -> Result { let options = &field_info.index_options; let mut iterator = BlockDocIterator { @@ -435,6 +451,7 @@ impl BlockDocIterator { partial_decode: false, partial_doc_deltas: PartialBlockDecoder::new(), partial_freqs: PartialBlockDecoder::new(), + use_simd, }; iterator.reset(term_state, flags)?; Ok(iterator) @@ -496,12 +513,12 @@ impl BlockDocIterator { if left >= BLOCK_SIZE { self.partial_decode = true; let doc_in = self.doc_in.as_mut().unwrap(); - self.for_util.read_block_only( + self.for_util.read_block( doc_in.as_mut(), &mut self.encoded, &mut self.doc_delta_buffer, Some(&mut self.encode_type), - &mut self.partial_doc_deltas, + self.use_simd, )?; ForUtil::read_other_encode_block( @@ -514,12 +531,12 @@ impl BlockDocIterator { if self.index_has_freq { if self.needs_freq { - self.for_util.read_block_only( + self.for_util.read_block( doc_in.as_mut(), &mut self.encoded, &mut self.freq_buffer, None, - &mut self.partial_freqs, + self.use_simd, )?; } else { self.for_util.skip_block(doc_in.as_mut())?; // skip over freqs @@ -543,6 +560,7 @@ impl BlockDocIterator { Ok(()) } + #[allow(dead_code)] #[inline(always)] fn decode_current_doc_delta(&mut self) -> i32 { if likely(self.partial_decode) { @@ -552,6 +570,7 @@ impl BlockDocIterator { } } + #[allow(dead_code)] #[inline(always)] fn decode_current_freq(&mut self) -> i32 { if likely(self.partial_decode && self.needs_freq) { @@ -601,8 +620,7 @@ impl DocIterator for BlockDocIterator { // set doc id self.doc = match self.encode_type { - EncodeType::PF => self.accum + self.decode_current_doc_delta(), - + EncodeType::PF => self.accum + self.doc_delta_buffer[self.doc_buffer_upto as usize], EncodeType::EF => { self.ef_decoder.as_mut().unwrap().next_value() as i32 + 1 + self.ef_base_doc } @@ -623,7 +641,7 @@ impl DocIterator for BlockDocIterator { self.doc_upto += 1; // set doc freq - self.freq = self.decode_current_freq(); + self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; self.doc_buffer_upto += 1; Ok(self.doc) } @@ -697,7 +715,7 @@ impl DocIterator for BlockDocIterator { // Now scan... this is an inlined/pared down version // of nextDoc(): loop { - self.accum += self.decode_current_doc_delta(); + self.accum += self.doc_delta_buffer[self.doc_buffer_upto as usize]; self.doc_upto += 1; if self.accum >= target { @@ -765,7 +783,7 @@ impl DocIterator for BlockDocIterator { } }; - self.freq = self.decode_current_freq(); + self.freq = self.freq_buffer[self.doc_buffer_upto as usize]; self.doc_buffer_upto += 1; Ok(self.doc) } @@ -775,6 +793,252 @@ impl DocIterator for BlockDocIterator { } } +struct SIMDBlockDocIterator { + doc_iter: BlockDocIterator, + simd_doc_deltas: SIMDBlockDecoder, + total_base: i32, +} + +impl SIMDBlockDocIterator { + #[allow(dead_code)] + pub fn new( + start_doc_in: Box, + field_info: &FieldInfo, + term_state: &BlockTermState, + flags: u16, + for_util: ForUtil, + ) -> Result { + Ok(Self { + doc_iter: BlockDocIterator::new( + start_doc_in, + field_info, + term_state, + flags, + for_util, + false, + )?, + simd_doc_deltas: SIMDBlockDecoder::new(), + total_base: 0, + }) + } + + fn refill_docs(&mut self) -> Result<()> { + self.doc_iter.partial_decode = false; + + let left = self.doc_iter.doc_freq - self.doc_iter.doc_upto; + debug_assert!(left > 0); + if left >= BLOCK_SIZE { + self.doc_iter.partial_decode = true; + let doc_in = self.doc_iter.doc_in.as_mut().unwrap(); + self.doc_iter + .for_util + .read_block_by_simd(doc_in.as_mut(), &mut self.simd_doc_deltas)?; + + if self.doc_iter.index_has_freq { + if self.doc_iter.needs_freq { + self.doc_iter.for_util.read_block_only( + doc_in.as_mut(), + &mut self.doc_iter.encoded, + &mut self.doc_iter.freq_buffer, + None, + &mut self.doc_iter.partial_freqs, + )?; + } else { + self.doc_iter.for_util.skip_block(doc_in.as_mut())?; // skip over freqs + } + } + } else if self.doc_iter.doc_freq == 1 { + self.doc_iter.doc_delta_buffer[0] = self.doc_iter.singleton_doc_id; + self.doc_iter.freq_buffer[0] = self.doc_iter.total_term_freq as i32; + } else { + let doc_in = self.doc_iter.doc_in.as_mut().unwrap(); + // Read vInts: + read_vint_block( + doc_in.as_mut(), + &mut self.doc_iter.doc_delta_buffer, + &mut self.doc_iter.freq_buffer, + left, + self.doc_iter.index_has_freq, + )?; + } + self.doc_iter.doc_buffer_upto = 0; + Ok(()) + } +} + +impl DocIterator for SIMDBlockDocIterator { + fn doc_id(&self) -> DocId { + self.doc_iter.doc_id() + } + + fn next(&mut self) -> Result { + if unlikely(self.doc_iter.doc_upto == self.doc_iter.doc_freq) { + self.doc_iter.doc = NO_MORE_DOCS; + return Ok(self.doc_iter.doc); + } + if unlikely(self.doc_iter.doc_buffer_upto == BLOCK_SIZE) { + if unlikely(self.doc_iter.doc < 0) { + self.doc_iter.accum = 0; + self.total_base = 0; + self.simd_doc_deltas.reset_delta_base(0); + } else { + self.doc_iter.accum = self.doc_iter.doc; + self.total_base = self.doc_iter.doc_upto; + self.simd_doc_deltas.reset_delta_base(self.doc_iter.accum); + } + self.refill_docs()?; + } + self.doc_iter.doc = if likely(self.doc_iter.partial_decode) { + self.doc_iter.accum = self.simd_doc_deltas.next(); + self.doc_iter.accum + } else { + self.doc_iter.accum = self.doc_iter.accum + + self.doc_iter.doc_delta_buffer[self.doc_iter.doc_buffer_upto as usize]; + self.doc_iter.accum + }; + + self.doc_iter.freq = self.doc_iter.decode_current_freq(); + self.doc_iter.doc_buffer_upto += 1; + self.doc_iter.doc_upto += 1; + Ok(self.doc_iter.doc) + } + + fn advance(&mut self, target: i32) -> Result { + if unlikely(target == NO_MORE_DOCS) { + self.doc_iter.doc = NO_MORE_DOCS; + return Ok(self.doc_iter.doc); + } + // TODO: make frq block load lazy/skippable + // current skip docID < docIDs generated from current buffer <= next skip docID + // we don't need to skip if target is buffered already + if self.doc_iter.doc_freq > BLOCK_SIZE && target > self.doc_iter.next_skip_doc { + if self.doc_iter.skipper.is_none() { + // Lazy init: first time this enum has ever been used for skipping + self.doc_iter.skipper = Some(Lucene50SkipReader::new( + clone_option_index_input(&self.doc_iter.doc_in)?, + MAX_SKIP_LEVELS, + self.doc_iter.index_has_pos, + self.doc_iter.index_has_offsets, + self.doc_iter.index_has_payloads, + )); + } + + let skipper = self.doc_iter.skipper.as_mut().unwrap(); + + if !self.doc_iter.skipped { + debug_assert_ne!(self.doc_iter.skip_offset, -1); + // This is the first time this enum has skipped + // since reset() was called; load the skip data: + skipper.init( + self.doc_iter.doc_term_start_fp + self.doc_iter.skip_offset, + self.doc_iter.doc_term_start_fp, + 0, + 0, + self.doc_iter.doc_freq, + )?; + self.doc_iter.skipped = true; + } + + // always plus one to fix the result, since skip position in Lucene50SkipReader + // is a little different from MultiLevelSkipListReader + let new_doc_upto = skipper.skip_to(target)? + 1; + + if new_doc_upto >= self.doc_iter.doc_upto { + // Skipper moved + debug_assert_eq!(new_doc_upto % BLOCK_SIZE, 0); + self.doc_iter.doc_upto = new_doc_upto; + + // Force to read next block + self.doc_iter.doc_buffer_upto = BLOCK_SIZE; + self.doc_iter.accum = skipper.doc(); // actually, this is just lastSkipEntry + self.doc_iter + .doc_in + .as_mut() + .unwrap() + .seek(skipper.doc_pointer())?; // now point to the + // block we want to + // search + } + // next time we call advance, this is used to + // foresee whether skipper is necessary. + self.doc_iter.next_skip_doc = skipper.next_skip_doc(); + } + if self.doc_iter.doc_upto == self.doc_iter.doc_freq { + self.doc_iter.doc = NO_MORE_DOCS; + return Ok(self.doc_iter.doc); + } + + if unlikely(self.doc_iter.doc_buffer_upto == BLOCK_SIZE) { + if unlikely(self.doc_iter.accum <= 0) { + // self.doc_iter.accum = 0; + self.total_base = 0; + self.simd_doc_deltas.reset_delta_base(0); + } else { + // self.doc_iter.accum = self.doc_iter.skipper.as_ref().unwrap().doc(); + self.total_base = self.doc_iter.doc_upto; + self.simd_doc_deltas.reset_delta_base(self.doc_iter.accum); + } + self.refill_docs()?; + } + + if likely(self.doc_iter.partial_decode) { + // let (doc, pos) = self.simd_doc_deltas.advance_by_binary_search(target); + let (doc, pos) = self.simd_doc_deltas.advance(target); + self.doc_iter.doc = doc; + if unlikely(doc == NO_MORE_DOCS) { + return Ok(NO_MORE_DOCS); + } + self.doc_iter.doc_buffer_upto = pos as i32; + self.doc_iter.doc_upto = self.total_base + pos as i32 + 1; + } else { + loop { + self.doc_iter.accum += + self.doc_iter.doc_delta_buffer[self.doc_iter.doc_buffer_upto as usize]; + self.doc_iter.doc_upto += 1; + + if self.doc_iter.accum >= target { + break; + } + self.doc_iter.doc_buffer_upto += 1; + if self.doc_iter.doc_upto == self.doc_iter.doc_freq { + self.doc_iter.doc = NO_MORE_DOCS; + return Ok(self.doc_iter.doc); + } + } + self.doc_iter.doc = self.doc_iter.accum; + } + self.doc_iter.freq = self.doc_iter.decode_current_freq(); + self.doc_iter.doc_buffer_upto += 1; + Ok(self.doc_iter.doc) + } + + fn cost(&self) -> usize { + self.doc_iter.cost() + } +} + +impl PostingIterator for SIMDBlockDocIterator { + fn freq(&self) -> Result { + self.doc_iter.freq() + } + + fn next_position(&mut self) -> Result { + self.doc_iter.next_position() + } + + fn start_offset(&self) -> Result { + self.doc_iter.start_offset() + } + + fn end_offset(&self) -> Result { + self.doc_iter.end_offset() + } + + fn payload(&self) -> Result { + self.doc_iter.payload() + } +} + struct BlockPostingIterator { encoded: [u8; MAX_ENCODED_SIZE], @@ -854,6 +1118,7 @@ struct BlockPostingIterator { doc_bits: FixedBitSet, bits_min_doc: DocId, bits_index: i32, + use_simd: bool, } impl BlockPostingIterator { @@ -864,6 +1129,7 @@ impl BlockPostingIterator { term_state: &BlockTermState, _flags: u16, for_util: ForUtil, + use_simd: bool, ) -> Result { let options = &field_info.index_options; let mut iterator = BlockPostingIterator { @@ -905,6 +1171,7 @@ impl BlockPostingIterator { doc_bits: FixedBitSet::default(), bits_min_doc: 0, bits_index: 0, + use_simd, }; iterator.reset(term_state)?; Ok(iterator) @@ -979,6 +1246,7 @@ impl BlockPostingIterator { &mut self.encoded, &mut self.doc_delta_buffer, Some(&mut self.encode_type), + self.use_simd, )?; ForUtil::read_other_encode_block( @@ -994,6 +1262,7 @@ impl BlockPostingIterator { &mut self.encoded, &mut self.freq_buffer, None, + self.use_simd, )?; } else if self.doc_freq == 1 { self.doc_delta_buffer[0] = self.singleton_doc_id; @@ -1043,6 +1312,7 @@ impl BlockPostingIterator { &mut self.encoded, &mut self.pos_delta_buffer, None, + self.use_simd, )?; } @@ -1420,6 +1690,7 @@ struct EverythingIterator { doc_bits: FixedBitSet, bits_min_doc: DocId, bits_index: i32, + use_simd: bool, } impl<'a> EverythingIterator { @@ -1432,6 +1703,7 @@ impl<'a> EverythingIterator { term_state: &BlockTermState, flags: u16, for_util: ForUtil, + use_simd: bool, ) -> Result { let encoded = [0u8; MAX_ENCODED_SIZE]; let index_has_offsets = field_info.index_options.has_offsets(); @@ -1502,6 +1774,7 @@ impl<'a> EverythingIterator { doc_bits: FixedBitSet::default(), bits_min_doc: 0, bits_index: 0, + use_simd, }; iterator.reset(term_state, flags)?; @@ -1582,6 +1855,7 @@ impl<'a> EverythingIterator { &mut self.encoded, &mut self.doc_delta_buffer, Some(&mut self.encode_type), + self.use_simd, )?; ForUtil::read_other_encode_block( @@ -1597,6 +1871,7 @@ impl<'a> EverythingIterator { &mut self.encoded, &mut self.freq_buffer, None, + self.use_simd, )?; } else if self.doc_freq == 1 { self.doc_delta_buffer[0] = self.singleton_doc_id; @@ -1661,6 +1936,7 @@ impl<'a> EverythingIterator { self.encoded.as_mut(), self.pos_delta_buffer.as_mut(), None, + self.use_simd, )?; let pay_in = &mut self.pay_in; @@ -1671,6 +1947,7 @@ impl<'a> EverythingIterator { &mut self.encoded, self.payload_length_buffer.as_mut(), None, + self.use_simd, )?; let num_bytes = pay_in.read_vint()? as usize; if num_bytes > self.payload_bytes.len() { @@ -1695,12 +1972,14 @@ impl<'a> EverythingIterator { &mut self.encoded, self.offset_start_delta_buffer.as_mut(), None, + self.use_simd, )?; self.for_util.read_block( pay_in.as_mut(), &mut self.encoded, self.offset_length_buffer.as_mut(), None, + self.use_simd, )?; } else { // this works, because when writing a vint block we always force the first @@ -2061,6 +2340,8 @@ pub struct Lucene50PostingIterator(Lucene50PostingIterEnum); enum Lucene50PostingIterEnum { Doc(BlockDocIterator), + #[allow(dead_code)] + SDoc(SIMDBlockDocIterator), Posting(BlockPostingIterator), Everything(EverythingIterator), } @@ -2069,6 +2350,7 @@ impl PostingIterator for Lucene50PostingIterator { fn freq(&self) -> Result { match &self.0 { Lucene50PostingIterEnum::Doc(i) => i.freq(), + Lucene50PostingIterEnum::SDoc(i) => i.freq(), Lucene50PostingIterEnum::Posting(i) => i.freq(), Lucene50PostingIterEnum::Everything(i) => i.freq(), } @@ -2077,6 +2359,7 @@ impl PostingIterator for Lucene50PostingIterator { fn next_position(&mut self) -> Result { match &mut self.0 { Lucene50PostingIterEnum::Doc(i) => i.next_position(), + Lucene50PostingIterEnum::SDoc(i) => i.next_position(), Lucene50PostingIterEnum::Posting(i) => i.next_position(), Lucene50PostingIterEnum::Everything(i) => i.next_position(), } @@ -2085,6 +2368,7 @@ impl PostingIterator for Lucene50PostingIterator { fn start_offset(&self) -> Result { match &self.0 { Lucene50PostingIterEnum::Doc(i) => i.start_offset(), + Lucene50PostingIterEnum::SDoc(i) => i.start_offset(), Lucene50PostingIterEnum::Posting(i) => i.start_offset(), Lucene50PostingIterEnum::Everything(i) => i.start_offset(), } @@ -2093,6 +2377,7 @@ impl PostingIterator for Lucene50PostingIterator { fn end_offset(&self) -> Result { match &self.0 { Lucene50PostingIterEnum::Doc(i) => i.end_offset(), + Lucene50PostingIterEnum::SDoc(i) => i.end_offset(), Lucene50PostingIterEnum::Posting(i) => i.end_offset(), Lucene50PostingIterEnum::Everything(i) => i.end_offset(), } @@ -2101,6 +2386,7 @@ impl PostingIterator for Lucene50PostingIterator { fn payload(&self) -> Result { match &self.0 { Lucene50PostingIterEnum::Doc(i) => i.payload(), + Lucene50PostingIterEnum::SDoc(i) => i.payload(), Lucene50PostingIterEnum::Posting(i) => i.payload(), Lucene50PostingIterEnum::Everything(i) => i.payload(), } @@ -2111,6 +2397,7 @@ impl DocIterator for Lucene50PostingIterator { fn doc_id(&self) -> DocId { match &self.0 { Lucene50PostingIterEnum::Doc(i) => i.doc_id(), + Lucene50PostingIterEnum::SDoc(i) => i.doc_id(), Lucene50PostingIterEnum::Posting(i) => i.doc_id(), Lucene50PostingIterEnum::Everything(i) => i.doc_id(), } @@ -2119,6 +2406,7 @@ impl DocIterator for Lucene50PostingIterator { fn next(&mut self) -> Result { match &mut self.0 { Lucene50PostingIterEnum::Doc(i) => i.next(), + Lucene50PostingIterEnum::SDoc(i) => i.next(), Lucene50PostingIterEnum::Posting(i) => i.next(), Lucene50PostingIterEnum::Everything(i) => i.next(), } @@ -2127,6 +2415,7 @@ impl DocIterator for Lucene50PostingIterator { fn advance(&mut self, target: i32) -> Result { match &mut self.0 { Lucene50PostingIterEnum::Doc(i) => i.advance(target), + Lucene50PostingIterEnum::SDoc(i) => i.advance(target), Lucene50PostingIterEnum::Posting(i) => i.advance(target), Lucene50PostingIterEnum::Everything(i) => i.advance(target), } @@ -2135,6 +2424,7 @@ impl DocIterator for Lucene50PostingIterator { fn slow_advance(&mut self, target: i32) -> Result { match &mut self.0 { Lucene50PostingIterEnum::Doc(i) => i.slow_advance(target), + Lucene50PostingIterEnum::SDoc(i) => i.slow_advance(target), Lucene50PostingIterEnum::Posting(i) => i.slow_advance(target), Lucene50PostingIterEnum::Everything(i) => i.slow_advance(target), } @@ -2143,6 +2433,7 @@ impl DocIterator for Lucene50PostingIterator { fn cost(&self) -> usize { match &self.0 { Lucene50PostingIterEnum::Doc(i) => i.cost(), + Lucene50PostingIterEnum::SDoc(i) => i.cost(), Lucene50PostingIterEnum::Posting(i) => i.cost(), Lucene50PostingIterEnum::Everything(i) => i.cost(), } @@ -2151,6 +2442,7 @@ impl DocIterator for Lucene50PostingIterator { fn matches(&mut self) -> Result { match &mut self.0 { Lucene50PostingIterEnum::Doc(i) => i.matches(), + Lucene50PostingIterEnum::SDoc(i) => i.matches(), Lucene50PostingIterEnum::Posting(i) => i.matches(), Lucene50PostingIterEnum::Everything(i) => i.matches(), } @@ -2159,6 +2451,7 @@ impl DocIterator for Lucene50PostingIterator { fn match_cost(&self) -> f32 { match &self.0 { Lucene50PostingIterEnum::Doc(i) => i.match_cost(), + Lucene50PostingIterEnum::SDoc(i) => i.match_cost(), Lucene50PostingIterEnum::Posting(i) => i.match_cost(), Lucene50PostingIterEnum::Everything(i) => i.match_cost(), } @@ -2167,6 +2460,7 @@ impl DocIterator for Lucene50PostingIterator { fn approximate_next(&mut self) -> Result { match &mut self.0 { Lucene50PostingIterEnum::Doc(i) => i.approximate_next(), + Lucene50PostingIterEnum::SDoc(i) => i.approximate_next(), Lucene50PostingIterEnum::Posting(i) => i.approximate_next(), Lucene50PostingIterEnum::Everything(i) => i.approximate_next(), } @@ -2175,6 +2469,7 @@ impl DocIterator for Lucene50PostingIterator { fn approximate_advance(&mut self, target: i32) -> Result { match &mut self.0 { Lucene50PostingIterEnum::Doc(i) => i.approximate_advance(target), + Lucene50PostingIterEnum::SDoc(i) => i.approximate_advance(target), Lucene50PostingIterEnum::Posting(i) => i.approximate_advance(target), Lucene50PostingIterEnum::Everything(i) => i.approximate_advance(target), } diff --git a/src/core/codec/postings/posting_writer.rs b/src/core/codec/postings/posting_writer.rs index 22342d6..3720d6e 100644 --- a/src/core/codec/postings/posting_writer.rs +++ b/src/core/codec/postings/posting_writer.rs @@ -17,7 +17,7 @@ use core::codec::postings::for_util::*; use core::codec::postings::posting_format::BLOCK_SIZE; use core::codec::postings::posting_reader::*; use core::codec::postings::skip_writer::Lucene50SkipWriter; -use core::codec::postings::PostingsWriterBase; +use core::codec::postings::{PostingsWriterBase, VERSION_START}; use core::codec::segment_infos::{segment_file_name, SegmentWriteState}; use core::codec::{write_footer, write_index_header, Codec, TermIterator}; use core::codec::{PostingIterator, PostingIteratorFlags}; @@ -26,7 +26,7 @@ use core::index::writer::INDEX_MAX_POSITION; use core::search::{DocIterator, NO_MORE_DOCS}; use core::store::directory::Directory; use core::store::io::{DataOutput, IndexOutput}; -use core::util::packed::COMPACT; +use core::util::packed::{SIMD128Packer, SIMDPacker, COMPACT}; use core::util::{BitSet, DocId, FixedBitSet}; use error::{ErrorKind, Result}; @@ -109,6 +109,7 @@ pub struct Lucene50PostingsWriter { write_payloads: bool, write_offsets: bool, ef_writer_meta: EfWriterMeta, + use_simd: bool, } impl Lucene50PostingsWriter { @@ -197,6 +198,12 @@ impl Lucene50PostingsWriter { pay_out.is_some(), ); + let use_simd = if VERSION_CURRENT > VERSION_START && SIMD128Packer::is_support() { + true + } else { + false + }; + Ok(Lucene50PostingsWriter { doc_out, pos_out, @@ -239,6 +246,7 @@ impl Lucene50PostingsWriter { write_payloads: false, write_offsets: false, ef_writer_meta: EfWriterMeta::new(), + use_simd, }) } @@ -330,6 +338,7 @@ impl Lucene50PostingsWriter { &mut self.encoded, &mut self.doc_out, Some(&mut self.ef_writer_meta), + self.use_simd, )?; if self.write_freqs { self.for_util.write_block( @@ -337,6 +346,7 @@ impl Lucene50PostingsWriter { &mut self.encoded, &mut self.doc_out, None, + self.use_simd, )?; } // NOTE: don't set docBufferUpto back to 0 here; @@ -400,6 +410,7 @@ impl Lucene50PostingsWriter { &mut self.encoded, self.pos_out.as_mut().unwrap(), None, + self.use_simd, )?; if self.write_payloads { @@ -408,6 +419,7 @@ impl Lucene50PostingsWriter { &mut self.encoded, self.pay_out.as_mut().unwrap(), None, + self.use_simd, )?; self.pay_out .as_mut() @@ -427,12 +439,14 @@ impl Lucene50PostingsWriter { &mut self.encoded, self.pay_out.as_mut().unwrap(), None, + self.use_simd, )?; self.for_util.write_block( &self.offset_length_buffer, &mut self.encoded, self.pay_out.as_mut().unwrap(), None, + self.use_simd, )?; } self.pos_buffer_upto = 0; @@ -652,7 +666,6 @@ impl PostingsWriterBase for Lucene50PostingsWriter { self.finish_doc(); if doc_freq > doc_freq_limit { - println!("{} {}", String::from_utf8(_term.to_vec())?, doc_freq); break; } } diff --git a/src/core/codec/postings/simd_block_decoder.rs b/src/core/codec/postings/simd_block_decoder.rs new file mode 100644 index 0000000..2a08b43 --- /dev/null +++ b/src/core/codec/postings/simd_block_decoder.rs @@ -0,0 +1,184 @@ +use core::codec::postings::BLOCK_SIZE; +use core::search::NO_MORE_DOCS; +use core::store::io::IndexInput; +use core::util::packed::{SIMD128Packer, SIMDPacker}; +use error::Result; +use std::arch::x86_64 as simd; + +#[repr(align(128))] +struct AlignedBuffer([u32; (BLOCK_SIZE + 4) as usize]); + +pub struct SIMDBlockDecoder { + data: AlignedBuffer, + encoded: [u8; BLOCK_SIZE as usize * 4], + packer: SIMD128Packer, + next_index: usize, + base_value: i32, +} + +impl SIMDBlockDecoder { + pub fn new() -> Self { + assert_eq!(BLOCK_SIZE, 128); + Self { + data: AlignedBuffer([NO_MORE_DOCS as u32; (BLOCK_SIZE + 4) as usize]), + encoded: [0u8; BLOCK_SIZE as usize * 4], + packer: SIMD128Packer::new(BLOCK_SIZE as usize), + next_index: 0, + base_value: 0, + } + } + + pub fn reset_delta_base(&mut self, base: i32) { + self.base_value = base; + } + + #[inline(always)] + pub fn set_single(&mut self, value: i32) { + self.next_index = 0; + let mut v = self.base_value; + self.data.0[..BLOCK_SIZE as usize].iter_mut().for_each(|e| { + v += value; + *e = v as u32; + }); + } + + pub fn parse_from( + &mut self, + input: &mut dyn IndexInput, + num: usize, + bits_num: usize, + ) -> Result<()> { + input.read_exact(&mut self.encoded[0..num])?; + self.next_index = 0; + self.packer.delta_unpack( + &self.encoded[0..num], + &mut self.data.0, + self.base_value as u32, + bits_num as u8, + ); + Ok(()) + } + + pub fn parse_from_no_copy( + &mut self, + input: &mut dyn IndexInput, + num: usize, + bits_num: usize, + ) -> Result<()> { + let encoded = unsafe { input.get_and_advance(num) }; + self.next_index = 0; + self.packer.delta_unpack( + encoded, + &mut self.data.0, + self.base_value as u32, + bits_num as u8, + ); + Ok(()) + } + + #[inline(always)] + pub fn next(&mut self) -> i32 { + let pos = self.next_index; + self.next_index += 1; + self.data.0[pos] as i32 + } + + #[inline(always)] + pub fn advance(&mut self, target: i32) -> (i32, usize) { + unsafe { + let input = self.data.0.as_ptr() as *const simd::__m128i; + let target = simd::_mm_set1_epi32(target); + let mut count = simd::_mm_set1_epi32(0); + unroll! { + for i in 0..8 { + let r1 = simd::_mm_cmplt_epi32( + simd::_mm_load_si128(input.add(i * 4)), target); + let r2 = simd::_mm_cmplt_epi32( + simd::_mm_load_si128(input.add(i * 4 + 1)), target); + let r3 = simd::_mm_cmplt_epi32( + simd::_mm_load_si128(input.add(i * 4 + 2)), target); + let r4 = simd::_mm_cmplt_epi32( + simd::_mm_load_si128(input.add(i * 4 + 3)), target); + let sum = simd::_mm_add_epi32( + simd::_mm_add_epi32(r1, r2), + simd::_mm_add_epi32(r3, r4) + ); + count = simd::_mm_sub_epi32(count, sum); + } + }; + let count = simd::_mm_add_epi32(count, simd::_mm_srli_si128(count, 8)); + let count = simd::_mm_add_epi32(count, simd::_mm_srli_si128(count, 4)); + let count = simd::_mm_cvtsi128_si32(count) as usize; + self.next_index = count + 1; + (self.data.0[count] as i32, count) + } + } + + #[inline(always)] + pub fn advance_by_partial(&mut self, target: i32) -> (i32, usize) { + let mut index = self.next_index & 0xFCusize; + let mut input = self.data.0[index..].as_ptr() as *const simd::__m128i; + unsafe { + let target = simd::_mm_set1_epi32(target); + while index < 128 { + let res = simd::_mm_cmplt_epi32(simd::_mm_load_si128(input), target); + let res = simd::_mm_movemask_epi8(res); + if res != 0xFFFF { + index += ((32 - res.leading_zeros()) >> 2) as usize; + break; + } else { + index += 4; + input = input.add(1); + } + } + self.next_index = index + 1; + (self.data.0[index] as i32, index) + } + } + + #[inline(always)] + pub fn advance_by_binary_search(&mut self, target: i32) -> (i32, usize) { + match self.data.0[self.next_index..BLOCK_SIZE as usize].binary_search(&(target as u32)) { + Ok(p) | Err(p) => { + let pos = self.next_index + p; + self.next_index = pos + 1; + (self.data.0[pos] as i32, pos) + } + } + } +} + +#[cfg(test)] +mod tests { + use core::codec::postings::SIMDBlockDecoder; + + #[test] + fn test_simd_advance() { + let mut decoder = SIMDBlockDecoder::new(); + let mut i = 0; + decoder.data.0.iter_mut().for_each(|e| { + i += 128; + *e = i; + }); + assert_eq!(decoder.advance(1).0, 128); + assert_eq!(decoder.advance(129).0, 256); + assert_eq!(decoder.advance(130).0, 256); + assert_eq!(decoder.advance(255).0, 256); + assert_eq!(decoder.advance(256).0, 256); + assert_eq!(decoder.advance(257).0, 384); + assert_eq!(decoder.next(), 512); + assert_eq!(decoder.advance(16283).0, 16384); + } + #[test] + fn test_binary_search() { + let mut decoder = SIMDBlockDecoder::new(); + let mut i = 0; + decoder.data.0.iter_mut().for_each(|e| { + i += 128; + *e = i; + }); + assert_eq!(decoder.advance_by_binary_search(1), (128, 0)); + assert_eq!(decoder.advance_by_binary_search(129), (256, 1)); + assert_eq!(decoder.advance_by_binary_search(512), (512, 3)); + } +} diff --git a/src/core/store/io/index_input.rs b/src/core/store/io/index_input.rs index 81b2115..a457762 100644 --- a/src/core/store/io/index_input.rs +++ b/src/core/store/io/index_input.rs @@ -33,6 +33,10 @@ pub trait IndexInput: DataInput + Send + Sync { unimplemented!(); } + unsafe fn get_and_advance(&mut self, _length: usize) -> *const u8 { + unimplemented!() + } + fn is_buffered(&self) -> bool { false } diff --git a/src/core/store/io/mmap_index_input.rs b/src/core/store/io/mmap_index_input.rs index 82c106d..c06104c 100644 --- a/src/core/store/io/mmap_index_input.rs +++ b/src/core/store/io/mmap_index_input.rs @@ -242,6 +242,14 @@ impl IndexInput for MmapIndexInput { Ok(Box::new(boxed)) } + #[inline(always)] + unsafe fn get_and_advance(&mut self, length: usize) -> *const u8 { + debug_assert!(self.position + length <= self.slice.len()); + let ptr = self.slice.as_ptr().add(self.position); + self.position += length; + ptr + } + fn slice(&self, description: &str, offset: i64, length: i64) -> Result> { let boxed = self.slice_impl(description, offset, length)?; Ok(Box::new(boxed)) diff --git a/src/core/util/packed/mod.rs b/src/core/util/packed/mod.rs index ef5a6c8..3a98aa0 100644 --- a/src/core/util/packed/mod.rs +++ b/src/core/util/packed/mod.rs @@ -62,3 +62,7 @@ pub use self::elias_fano_encoder::*; mod elias_fano_decoder; pub use self::elias_fano_decoder::*; + +mod packed_simd; + +pub use self::packed_simd::*; diff --git a/src/core/util/packed/packed_simd.rs b/src/core/util/packed/packed_simd.rs new file mode 100644 index 0000000..85681a8 --- /dev/null +++ b/src/core/util/packed/packed_simd.rs @@ -0,0 +1,551 @@ +use std::arch::x86_64 as simd; +use std::slice::{from_raw_parts, from_raw_parts_mut}; + +pub trait SIMDPacker { + const BLOCK_LEN: usize; + type DataType; + fn pack(data: &[u32], encoded_data: &mut [u8], bits_num: u8); + fn unpack(encoded_data: T, data: &mut [u32], bits_num: u8); + fn delta_pack(&mut self, data: &[u32], encoded_data: &mut [u8], base: u32, bits_num: u8); + fn delta_unpack( + &mut self, + encoded_data: T, + data: &mut [u32], + base: u32, + bits_num: u8, + ); + fn trans_to_delta(&mut self, data: Self::DataType) -> Self::DataType { + data + } + fn trans_from_delta(&mut self, delta: Self::DataType) -> Self::DataType { + delta + } + fn max_bits_num(data: &[u32]) -> u8 { + 32 - data + .iter() + .fold(0u32, |result, &element| result | element) + .leading_zeros() as u8 + } + fn is_support() -> bool; +} + +#[derive(Debug, Clone, Copy)] +pub struct SIMD128Packer { + delta_base: u32, +} + +impl SIMD128Packer { + pub fn new(block_len: usize) -> Self { + assert_eq!(block_len, 128usize); + Self { delta_base: 0 } + } + + #[inline(always)] + fn direct_copy_to(data: &[u32], encoded_data: &mut [u8]) { + encoded_data.copy_from_slice(unsafe { from_raw_parts(data.as_ptr() as *const u8, 512) }) + } + + #[inline(always)] + fn direct_copy_from(encoded_data: &[u8], data: &mut [u32]) { + unsafe { from_raw_parts_mut(data.as_mut_ptr() as *mut u8, 512) } + .copy_from_slice(encoded_data); + } + + #[allow(dead_code)] + fn print_m128i_as_i32(v: simd::__m128i) { + unsafe { + let data = vec![ + simd::_mm_cvtsi128_si32(v), + simd::_mm_cvtsi128_si32(simd::_mm_srli_si128(v, 4)), + simd::_mm_cvtsi128_si32(simd::_mm_srli_si128(v, 8)), + simd::_mm_cvtsi128_si32(simd::_mm_srli_si128(v, 16)), + ]; + println!("{:?}", data); + } + } +} + +macro_rules! pack_bits { + ($data: expr, $encoded_data: expr, $num: literal $(,$transfer: expr, $obj: expr)?) => { + unsafe { + let mut _input = $data.as_ptr() as *const simd::__m128i; + let mut _output = $encoded_data.as_mut_ptr() as *mut simd::__m128i; + let mut _buffer = simd::_mm_set1_epi32(0); + unroll! { + for i in 0..32 { + let mut _input_data = simd::_mm_lddqu_si128(_input); + $(_input_data = $transfer($obj, _input_data);)? + const inner_pos: i32 = i as i32 * $num % 32; + _buffer = simd::_mm_or_si128(_buffer, simd::_mm_slli_epi32(_input_data, inner_pos)); + const new_pos: i32 = inner_pos + $num; + if new_pos >= 32 { // buffer is full, store + simd::_mm_storeu_si128(_output, _buffer); + _output = _output.add(1); + _buffer = if new_pos > 32 { + simd::_mm_srli_epi32(_input_data, 32 - inner_pos) + } else { + simd::_mm_set1_epi32(0) + } + } + _input = _input.add(1); + } + } + } + }; +} + +pub trait AsRawPtr { + fn as_ptr(&self) -> *const u8; +} + +impl AsRawPtr for *const u8 { + fn as_ptr(&self) -> *const u8 { + *self + } +} + +impl AsRawPtr for &[u8] { + fn as_ptr(&self) -> *const u8 { + (*self).as_ptr() + } +} + +macro_rules! unpack_bits { + ($encoded_data: expr, $data: expr, $num: literal $(,$transfer: expr, $obj: expr)?) => { + unsafe { + let mut input = $encoded_data.as_ptr() as *const simd::__m128i; + let mut _output = $data.as_mut_ptr() as *mut simd::__m128i; + let mask = simd::_mm_set1_epi32(((1u32 << $num) - 1) as i32); + let mut _buffer = simd::_mm_lddqu_si128(input); + unroll! { + for i in 0..32 { + const inner_pos: i32 = i as i32 * $num % 32; + const new_pos: i32 = inner_pos + $num; + if new_pos >= 32 { + input = input.add(1); + _buffer = if new_pos == 32 { + $(_buffer = $transfer($obj, _buffer);)? + simd::_mm_storeu_si128(_output, _buffer); + simd::_mm_lddqu_si128(input) + } else { + const remain: i32 = 32 - inner_pos; + let temp = simd::_mm_lddqu_si128(input); + _buffer = simd::_mm_and_si128(simd::_mm_or_si128( + _buffer, simd::_mm_slli_epi32(temp, remain)), mask); + $(_buffer = $transfer($obj, _buffer);)? + simd::_mm_storeu_si128(_output, _buffer); + simd::_mm_srli_epi32(temp, $num - remain) + } + } else { + let mut _data = simd::_mm_and_si128(_buffer, mask); + $(_data = $transfer($obj, _data);)? + simd::_mm_storeu_si128(_output, _data); + _buffer = simd::_mm_srli_epi32(_buffer, $num); + } + _output = _output.add(1); + } + } + } + }; +} + +impl SIMDPacker for SIMD128Packer { + const BLOCK_LEN: usize = 128; + type DataType = simd::__m128i; + + fn pack(data: &[u32], encoded_data: &mut [u8], bits_num: u8) { + match bits_num { + // for unroll loop, reducing branch missing + 1 => pack_bits!(data, encoded_data, 1), + 2 => pack_bits!(data, encoded_data, 2), + 3 => pack_bits!(data, encoded_data, 3), + 4 => pack_bits!(data, encoded_data, 4), + 5 => pack_bits!(data, encoded_data, 5), + 6 => pack_bits!(data, encoded_data, 6), + 7 => pack_bits!(data, encoded_data, 7), + 8 => pack_bits!(data, encoded_data, 8), + 9 => pack_bits!(data, encoded_data, 9), + 10 => pack_bits!(data, encoded_data, 10), + 11 => pack_bits!(data, encoded_data, 11), + 12 => pack_bits!(data, encoded_data, 12), + 13 => pack_bits!(data, encoded_data, 13), + 14 => pack_bits!(data, encoded_data, 14), + 15 => pack_bits!(data, encoded_data, 15), + 16 => pack_bits!(data, encoded_data, 16), + 17 => pack_bits!(data, encoded_data, 17), + 18 => pack_bits!(data, encoded_data, 18), + 19 => pack_bits!(data, encoded_data, 19), + 20 => pack_bits!(data, encoded_data, 20), + 21 => pack_bits!(data, encoded_data, 21), + 22 => pack_bits!(data, encoded_data, 22), + 23 => pack_bits!(data, encoded_data, 23), + 24 => pack_bits!(data, encoded_data, 24), + 25 => pack_bits!(data, encoded_data, 25), + 26 => pack_bits!(data, encoded_data, 26), + 27 => pack_bits!(data, encoded_data, 27), + 28 => pack_bits!(data, encoded_data, 28), + 29 => pack_bits!(data, encoded_data, 29), + 30 => pack_bits!(data, encoded_data, 30), + 31 => pack_bits!(data, encoded_data, 31), + 32 => Self::direct_copy_to(data, encoded_data), + 0 => { /* do nothing! */ } + _ => unimplemented!(), + } + } + + fn unpack(encoded_data: T, data: &mut [u32], bits_num: u8) { + match bits_num { + // for unroll loop, reducing branch missing + 1 => unpack_bits!(encoded_data, data, 1), + 2 => unpack_bits!(encoded_data, data, 2), + 3 => unpack_bits!(encoded_data, data, 3), + 4 => unpack_bits!(encoded_data, data, 4), + 5 => unpack_bits!(encoded_data, data, 5), + 6 => unpack_bits!(encoded_data, data, 6), + 7 => unpack_bits!(encoded_data, data, 7), + 8 => unpack_bits!(encoded_data, data, 8), + 9 => unpack_bits!(encoded_data, data, 9), + 10 => unpack_bits!(encoded_data, data, 10), + 11 => unpack_bits!(encoded_data, data, 11), + 12 => unpack_bits!(encoded_data, data, 12), + 13 => unpack_bits!(encoded_data, data, 13), + 14 => unpack_bits!(encoded_data, data, 14), + 15 => unpack_bits!(encoded_data, data, 15), + 16 => unpack_bits!(encoded_data, data, 16), + 17 => unpack_bits!(encoded_data, data, 17), + 18 => unpack_bits!(encoded_data, data, 18), + 19 => unpack_bits!(encoded_data, data, 19), + 20 => unpack_bits!(encoded_data, data, 20), + 21 => unpack_bits!(encoded_data, data, 21), + 22 => unpack_bits!(encoded_data, data, 22), + 23 => unpack_bits!(encoded_data, data, 23), + 24 => unpack_bits!(encoded_data, data, 24), + 25 => unpack_bits!(encoded_data, data, 25), + 26 => unpack_bits!(encoded_data, data, 26), + 27 => unpack_bits!(encoded_data, data, 27), + 28 => unpack_bits!(encoded_data, data, 28), + 29 => unpack_bits!(encoded_data, data, 29), + 30 => unpack_bits!(encoded_data, data, 30), + 31 => unpack_bits!(encoded_data, data, 31), + 32 => { + Self::direct_copy_from( + unsafe { from_raw_parts(encoded_data.as_ptr(), 4 * 128) }, + data, + ); + } + 0 => { /* do nothing! */ } + _ => unimplemented!(), + } + } + + fn delta_pack(&mut self, data: &[u32], encoded_data: &mut [u8], base: u32, bits_num: u8) { + self.delta_base = base; + match bits_num { + // for unroll loop, reducing branch missing + 1 => pack_bits!(data, encoded_data, 1, Self::trans_to_delta, self), + 2 => pack_bits!(data, encoded_data, 2, Self::trans_to_delta, self), + 3 => pack_bits!(data, encoded_data, 3, Self::trans_to_delta, self), + 4 => pack_bits!(data, encoded_data, 4, Self::trans_to_delta, self), + 5 => pack_bits!(data, encoded_data, 5, Self::trans_to_delta, self), + 6 => pack_bits!(data, encoded_data, 6, Self::trans_to_delta, self), + 7 => pack_bits!(data, encoded_data, 7, Self::trans_to_delta, self), + 8 => pack_bits!(data, encoded_data, 8, Self::trans_to_delta, self), + 9 => pack_bits!(data, encoded_data, 9, Self::trans_to_delta, self), + 10 => pack_bits!(data, encoded_data, 10, Self::trans_to_delta, self), + 11 => pack_bits!(data, encoded_data, 11, Self::trans_to_delta, self), + 12 => pack_bits!(data, encoded_data, 12, Self::trans_to_delta, self), + 13 => pack_bits!(data, encoded_data, 13, Self::trans_to_delta, self), + 14 => pack_bits!(data, encoded_data, 14, Self::trans_to_delta, self), + 15 => pack_bits!(data, encoded_data, 15, Self::trans_to_delta, self), + 16 => pack_bits!(data, encoded_data, 16, Self::trans_to_delta, self), + 17 => pack_bits!(data, encoded_data, 17, Self::trans_to_delta, self), + 18 => pack_bits!(data, encoded_data, 18, Self::trans_to_delta, self), + 19 => pack_bits!(data, encoded_data, 19, Self::trans_to_delta, self), + 20 => pack_bits!(data, encoded_data, 20, Self::trans_to_delta, self), + 21 => pack_bits!(data, encoded_data, 21, Self::trans_to_delta, self), + 22 => pack_bits!(data, encoded_data, 22, Self::trans_to_delta, self), + 23 => pack_bits!(data, encoded_data, 23, Self::trans_to_delta, self), + 24 => pack_bits!(data, encoded_data, 24, Self::trans_to_delta, self), + 25 => pack_bits!(data, encoded_data, 25, Self::trans_to_delta, self), + 26 => pack_bits!(data, encoded_data, 26, Self::trans_to_delta, self), + 27 => pack_bits!(data, encoded_data, 27, Self::trans_to_delta, self), + 28 => pack_bits!(data, encoded_data, 28, Self::trans_to_delta, self), + 29 => pack_bits!(data, encoded_data, 29, Self::trans_to_delta, self), + 30 => pack_bits!(data, encoded_data, 30, Self::trans_to_delta, self), + 31 => pack_bits!(data, encoded_data, 31, Self::trans_to_delta, self), + 32 => Self::direct_copy_to(data, encoded_data), + 0 => { /* do nothing! */ } + _ => unimplemented!(), + } + } + + fn delta_unpack( + &mut self, + encoded_data: T, + data: &mut [u32], + base: u32, + bits_num: u8, + ) { + self.delta_base = base; + match bits_num { + // for unroll loop, reducing branch missing + 1 => unpack_bits!(encoded_data, data, 1, Self::trans_from_delta, self), + 2 => unpack_bits!(encoded_data, data, 2, Self::trans_from_delta, self), + 3 => unpack_bits!(encoded_data, data, 3, Self::trans_from_delta, self), + 4 => unpack_bits!(encoded_data, data, 4, Self::trans_from_delta, self), + 5 => unpack_bits!(encoded_data, data, 5, Self::trans_from_delta, self), + 6 => unpack_bits!(encoded_data, data, 6, Self::trans_from_delta, self), + 7 => unpack_bits!(encoded_data, data, 7, Self::trans_from_delta, self), + 8 => unpack_bits!(encoded_data, data, 8, Self::trans_from_delta, self), + 9 => unpack_bits!(encoded_data, data, 9, Self::trans_from_delta, self), + 10 => unpack_bits!(encoded_data, data, 10, Self::trans_from_delta, self), + 11 => unpack_bits!(encoded_data, data, 11, Self::trans_from_delta, self), + 12 => unpack_bits!(encoded_data, data, 12, Self::trans_from_delta, self), + 13 => unpack_bits!(encoded_data, data, 13, Self::trans_from_delta, self), + 14 => unpack_bits!(encoded_data, data, 14, Self::trans_from_delta, self), + 15 => unpack_bits!(encoded_data, data, 15, Self::trans_from_delta, self), + 16 => unpack_bits!(encoded_data, data, 16, Self::trans_from_delta, self), + 17 => unpack_bits!(encoded_data, data, 17, Self::trans_from_delta, self), + 18 => unpack_bits!(encoded_data, data, 18, Self::trans_from_delta, self), + 19 => unpack_bits!(encoded_data, data, 19, Self::trans_from_delta, self), + 20 => unpack_bits!(encoded_data, data, 20, Self::trans_from_delta, self), + 21 => unpack_bits!(encoded_data, data, 21, Self::trans_from_delta, self), + 22 => unpack_bits!(encoded_data, data, 22, Self::trans_from_delta, self), + 23 => unpack_bits!(encoded_data, data, 23, Self::trans_from_delta, self), + 24 => unpack_bits!(encoded_data, data, 24, Self::trans_from_delta, self), + 25 => unpack_bits!(encoded_data, data, 25, Self::trans_from_delta, self), + 26 => unpack_bits!(encoded_data, data, 26, Self::trans_from_delta, self), + 27 => unpack_bits!(encoded_data, data, 27, Self::trans_from_delta, self), + 28 => unpack_bits!(encoded_data, data, 28, Self::trans_from_delta, self), + 29 => unpack_bits!(encoded_data, data, 29, Self::trans_from_delta, self), + 30 => unpack_bits!(encoded_data, data, 30, Self::trans_from_delta, self), + 31 => unpack_bits!(encoded_data, data, 31, Self::trans_from_delta, self), + 32 => { + Self::direct_copy_from( + unsafe { from_raw_parts(encoded_data.as_ptr(), 4 * 128) }, + data, + ); + } + 0 => { /* do nothing! */ } + _ => unimplemented!(), + } + } + + #[inline(always)] + fn trans_to_delta(&mut self, data: simd::__m128i) -> simd::__m128i { + unsafe { + let deltas = simd::_mm_sub_epi32( + data, + simd::_mm_or_si128( + simd::_mm_slli_si128(data, 4), + simd::_mm_set_epi32(0, 0, 0, self.delta_base as i32), + ), + ); + + self.delta_base = simd::_mm_cvtsi128_si32(simd::_mm_srli_si128(data, 12)) as u32; + deltas + } + } + + #[inline(always)] + fn trans_from_delta(&mut self, delta: simd::__m128i) -> simd::__m128i { + unsafe { + let a_ab_bc_cd = simd::_mm_add_epi32(delta, simd::_mm_slli_si128(delta, 4)); + let a_ab_abc_abcd = + simd::_mm_add_epi32(a_ab_bc_cd, simd::_mm_slli_si128(a_ab_bc_cd, 8)); + let value = + simd::_mm_add_epi32(a_ab_abc_abcd, simd::_mm_set1_epi32(self.delta_base as i32)); + self.delta_base = simd::_mm_cvtsi128_si32(simd::_mm_srli_si128(value, 12)) as u32; + value + } + } + + fn max_bits_num(data: &[u32]) -> u8 { + unsafe { + let mut data_input = data.as_ptr() as *const simd::__m128i; + let mut result = simd::_mm_lddqu_si128(data_input); + for _ in 1..32 { + data_input = data_input.add(1); + let data = simd::_mm_lddqu_si128(data_input); + result = simd::_mm_or_si128(result, data); + } + // result like a_b_c_d + let a_b_c = simd::_mm_srli_si128(result, 4); + let a_ab_bc_cd = simd::_mm_or_si128(result, a_b_c); + let a_ab = simd::_mm_srli_si128(a_ab_bc_cd, 8); + let a_ab_abc_abcd = simd::_mm_or_si128(a_ab_bc_cd, a_ab); + 32 - simd::_mm_cvtsi128_si32(a_ab_abc_abcd).leading_zeros() as u8 + } + } + + #[inline(always)] + fn is_support() -> bool { + std::is_x86_feature_detected!("sse3") + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_max_bits_num() { + // test basic algorithm + assert_eq!(MockPacker::max_bits_num(&[5, 8, 7]), 4); + assert_eq!(MockPacker::max_bits_num(&[0b10101, 0b111, 0b11101]), 5); + assert_eq!( + MockPacker::max_bits_num(&[0b10101, 0b1000000111, 0b11101]), + 10 + ); + // construct 128 array, test simd algorithm + let mut data = [0u32; 128]; + for (i, e) in data.iter_mut().enumerate() { + *e = i as u32 * 5; + } + assert_eq!( + MockPacker::max_bits_num(&data), + SIMD128Packer::max_bits_num(&data) + ); + } + + #[test] + fn test_direct_copy() { + let mut data = [0u32; 128]; + for (i, e) in data.iter_mut().enumerate() { + *e = i as u32 % 9 * (i + 1) as u32; + } + + let mut temp = [0u8; 4 * 128]; + SIMD128Packer::direct_copy_to(&data, &mut temp); + let mut data2 = [0u32; 128]; + SIMD128Packer::direct_copy_from(&temp, &mut data2); + assert_eq!(data2[0], 0); + assert_eq!(data2[1], 2); + assert_eq!(data2[9], 0); + assert_eq!(data2[10], 11); + assert_eq!(&data[0..32], &data2[0..32]); + assert_eq!(&data[33..64], &data2[33..64]); + assert_eq!(&data[65..96], &data2[65..96]); + assert_eq!(&data[97..128], &data2[97..128]); + } + + macro_rules! assert_128_array_eq { + ($arr1: expr, $arr2: expr) => { + let mut is_equal = true; + let mut pos = 0; + let mut left = 0; + let mut right = 0; + assert_eq!($arr1.len(), 128usize); + assert_eq!($arr2.len(), 128usize); + for i in 0..128 { + if $arr1[i] != $arr2[i] { + is_equal = false; + pos = i; + left = $arr1[i]; + right = $arr2[i]; + break; + } + } + assert!( + is_equal, + "value@{} is not equal! left={}, right={}.", + pos, left, right + ); + }; + } + + #[test] + fn test_pack_unpack_bits() { + let mut data_1bits = [0u32; 128]; + let mut data_5bits = [0u32; 128]; + let mut data_31bits = [0u32; 128]; + for i in 0..128 { + if i % 5 == 0 { + data_1bits[i] = 1; + data_5bits[i] = (0b10000 | (i & 0b1111)) as u32; + data_31bits[i] = 0x40000000u32 | i as u32; + } + } + let mut encoded = [0u8; 512]; + let mut decoded = [0u32; 128]; + pack_bits!(&data_1bits, &mut encoded, 1); + unpack_bits!(&encoded, &mut decoded, 1); + assert_128_array_eq!(&data_1bits, &decoded); + assert_eq!(decoded[0], 1); + assert_eq!(decoded[1], 0); + assert_eq!(decoded[4], 0); + assert_eq!(decoded[5], 1); + assert_eq!(decoded[6], 0); + assert_eq!(decoded[34], 0); + assert_eq!(decoded[35], 1); + assert_eq!(decoded[36], 0); + + pack_bits!(&data_5bits, &mut encoded, 5); + unpack_bits!(&encoded, &mut decoded, 5); + assert_128_array_eq!(&data_5bits, &decoded); + assert_eq!(decoded[35], (0b10000 | (35 & 0b1111)) as u32); + + pack_bits!(&data_31bits, &mut encoded, 31); + unpack_bits!(&encoded, &mut decoded, 31); + assert_128_array_eq!(&data_31bits, &decoded); + assert_eq!(decoded[40], 0x40000000u32 | 40 as u32); + } + + #[test] + fn test_delta_pack_unpack() { + let mut data = [0u32; 128]; + let mut i = 1; + data.iter_mut().for_each(|x| { + *x = i * 128; + i += 1; + }); + assert_eq!(data[127], 128 * 128); + let mut encoded = [0u8; 512]; + let mut decoded = [0u32; 128]; + let mut packer = SIMD128Packer::new(128); + SIMD128Packer::pack(&data, &mut encoded, 15); + SIMD128Packer::unpack(&encoded[..], &mut decoded, 15); + assert_128_array_eq!(&data, &decoded); + packer.delta_pack(&data, &mut encoded, 128, 14); + packer.delta_unpack(&encoded[..], &mut decoded, 128, 14); + assert_128_array_eq!(&data, &decoded); + } + + struct MockPacker; + impl SIMDPacker for MockPacker { + const BLOCK_LEN: usize = 1; + type DataType = (); + + fn pack(_data: &[u32], _encoded_data: &mut [u8], _bits_num: u8) { + unimplemented!() + } + + fn unpack(_encoded_data: T, _data: &mut [u32], _bits_num: u8) { + unimplemented!() + } + + fn delta_pack( + &mut self, + _data: &[u32], + _encoded_data: &mut [u8], + _base: u32, + _bits_num: u8, + ) { + unimplemented!() + } + + fn delta_unpack( + &mut self, + _encoded_data: T, + _data: &mut [u32], + _base: u32, + _bits_num: u8, + ) { + unimplemented!() + } + + fn is_support() -> bool { + unimplemented!() + } + } +} diff --git a/src/lib.rs b/src/lib.rs index ddfbb72..8a63258 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -28,6 +28,7 @@ #![feature(in_band_lifetimes)] #![feature(vec_into_raw_parts)] #![feature(core_intrinsics)] +#![feature(stmt_expr_attributes)] #[macro_use] extern crate error_chain; @@ -55,6 +56,8 @@ extern crate num_traits; extern crate smallvec; extern crate thread_local; extern crate unicode_reader; +#[macro_use] +extern crate crunchy; pub mod core; pub mod error; From 5e2917b9e9357279c47a0d93b40375fafa064174 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Fri, 9 Oct 2020 13:36:21 +0800 Subject: [PATCH 29/49] phrase query --- src/core/search/query/phrase_query.rs | 1219 +++----------------- src/core/search/scorer/mod.rs | 4 + src/core/search/scorer/phrase_scorer.rs | 1070 +++++++++++++++++ src/core/util/packed/elias_fano_decoder.rs | 13 + src/core/util/packed/elias_fano_encoder.rs | 13 + 5 files changed, 1276 insertions(+), 1043 deletions(-) create mode 100644 src/core/search/scorer/phrase_scorer.rs diff --git a/src/core/search/query/phrase_query.rs b/src/core/search/query/phrase_query.rs index 7dc2853..cce80ba 100644 --- a/src/core/search/query/phrase_query.rs +++ b/src/core/search/query/phrase_query.rs @@ -11,28 +11,23 @@ // See the License for the specific language governing permissions and // limitations under the License. -use error::{ErrorKind, Result}; use std::boxed::Box; -use std::cmp::{min, Ord, Ordering}; -use std::collections::BinaryHeap; -use std::collections::{HashMap, HashSet}; -use std::f32; use std::fmt; -use core::codec::Codec; -use core::codec::{PostingIterator, PostingIteratorFlags}; +use core::codec::PostingIteratorFlags; use core::codec::{TermIterator, Terms}; +use core::codec::Codec; use core::doc::Term; use core::index::reader::LeafReaderContext; +use core::search::DocIterator; use core::search::explanation::Explanation; use core::search::query::{Query, TermQuery, Weight}; -use core::search::scorer::{two_phase_next, ConjunctionScorer, Scorer}; +use core::search::scorer::{ExactPhraseScorer, PostingsAndFreq, Scorer, SloppyPhraseScorer}; use core::search::searcher::SearchPlanBuilder; -use core::search::similarity::{SimScorer, SimWeight, Similarity}; +use core::search::similarity::{Similarity, SimWeight}; use core::search::statistics::{CollectionStatistics, TermStatistics}; -use core::search::{DocIterator, NO_MORE_DOCS}; -use core::util::{BitSet, FixedBitSet, ImmutableBitSet}; -use core::util::{Bits, DocId, KeyedContext}; +use core::util::{DocId, KeyedContext}; +use error::{ErrorKind, Result}; pub const PHRASE: &str = "phrase"; @@ -392,7 +387,7 @@ impl Weight for PhraseWeight { ); if scorer.advance(doc)? == doc { - let freq = scorer.freq as f32; + let freq = scorer.freq() as f32; let freq_expl = Explanation::new(true, freq, format!("phraseFreq={}", freq), vec![]); let score_expl = self.sim_weight.explain(reader.reader, doc, freq_expl)?; @@ -414,7 +409,7 @@ impl Weight for PhraseWeight { ); if scorer.advance(doc)? == doc { - let freq = scorer.sloppy_freq; + let freq = scorer.sloppy_freq(); let freq_expl = Explanation::new(true, freq, format!("phraseFreq={}", freq), vec![]); let score_expl = self.sim_weight.explain(reader.reader, doc, freq_expl)?; @@ -448,1039 +443,177 @@ impl fmt::Display for PhraseWeight { } } -struct PostingsAndFreq { - pub postings: T, - pub pos: i32, - pub terms: Vec, - pub nterms: i32, - // for faster comparisons -} - -impl PostingsAndFreq { - fn new(postings: T, pos: i32, term: &Term) -> Self { - PostingsAndFreq { - postings, - pos, - terms: vec![term.clone()], - nterms: 1, - } - } -} - -impl Ord for PostingsAndFreq { - fn cmp(&self, other: &Self) -> Ordering { - self.partial_cmp(&other).unwrap() - } -} - -impl PartialOrd for PostingsAndFreq { - fn partial_cmp(&self, other: &Self) -> Option { - if self.pos != other.pos { - return Some(self.pos.cmp(&other.pos)); - } - - if self.nterms != other.nterms { - return Some(self.nterms.cmp(&other.nterms)); - } - - if self.nterms == 0 { - return Some(Ordering::Equal); - } - - for i in 0..self.terms.len() { - let res = if self.terms[i].field.eq(&other.terms[i].field) { - self.terms[i].bytes.cmp(&other.terms[i].bytes) - } else { - self.terms[i].field.cmp(&other.terms[i].field) - }; - - if res != Ordering::Equal { - return Some(res); - } - } - - Some(Ordering::Equal) - } -} - -impl Eq for PostingsAndFreq {} - -impl PartialEq for PostingsAndFreq { - fn eq(&self, other: &Self) -> bool { - self == other - } -} - -pub struct PostingsAndPosition { - postings: *mut dyn PostingIterator, - pos: i32, - offset: i32, - freq: i32, - up_to: i32, -} - -unsafe impl Send for PostingsAndPosition {} - -impl PostingsAndPosition { - pub fn new(postings: *mut dyn PostingIterator, offset: i32) -> PostingsAndPosition { - PostingsAndPosition { - postings, - pos: -1, - offset, - freq: 0, - up_to: 1, - } - } -} - -pub struct ExactPhraseScorer { - freq: i32, - needs_scores: bool, - match_cost: f32, - postings: Vec, - doc_scorer: Box, - conjunction: ConjunctionScorer>, -} - -impl ExactPhraseScorer { - fn new( - postings: Vec>, - doc_scorer: Box, - needs_scores: bool, - match_cost: f32, - ) -> Self { - let mut iterators = Vec::with_capacity(postings.len()); - let mut postings_and_positions: Vec = - Vec::with_capacity(postings.len()); - - for (i, posing) in postings.into_iter().enumerate() { - let iterator = posing.postings; - iterators.push(PostingsIterAsScorer { iterator }); - postings_and_positions.push(PostingsAndPosition::new( - &mut iterators[i].iterator, - posing.pos, - )); - } - - let conjunction = ConjunctionScorer::new(iterators); - - ExactPhraseScorer { - freq: 0, - needs_scores, - match_cost, - postings: postings_and_positions, - doc_scorer, - conjunction, - } - } - - fn advance_position(posting: &mut PostingsAndPosition, target: i32) -> Result { - while posting.pos < target { - if posting.up_to == posting.freq { - return Ok(false); - } else { - unsafe { - posting.pos = (*posting.postings).next_position()?; - } - posting.up_to += 1; - } - } - - Ok(true) - } - - fn phrase_freq(&mut self) -> Result { - // reset state - for posting in &mut self.postings { - unsafe { - posting.freq = (*posting.postings).freq()?; - posting.pos = (*posting.postings).next_position()?; - } - - posting.up_to = 1; - } - - let mut freq = 0; - let mut lead = self.postings.remove(0); - - 'advanceHead: loop { - let phrase_pos = lead.pos - lead.offset; - for posting in &mut self.postings { - let expected_pos = phrase_pos + posting.offset; - - // advance up to the same position as the lead - if !Self::advance_position(posting, expected_pos)? { - break 'advanceHead; - } - - if posting.pos != expected_pos { - // we advanced too far - let target = posting.pos - posting.offset + lead.offset; - - if Self::advance_position(&mut lead, target)? { - continue 'advanceHead; - } else { - break 'advanceHead; - } - } - } - - freq += 1; - if !self.needs_scores { - break; - } - - if lead.up_to == lead.freq { - break; - } - - unsafe { - lead.pos = (*lead.postings).next_position()?; - } - lead.up_to += 1; - } - - self.postings.insert(0, lead); - - self.freq = freq; - Ok(self.freq) - } - - pub fn do_next(&mut self, doc_id: DocId) -> Result { - let mut doc = doc_id; - loop { - if doc == NO_MORE_DOCS { - return Ok(NO_MORE_DOCS); - } else if self.matches()? { - return Ok(doc); - } - - doc = self.conjunction.next()?; - } - } -} - -impl Scorer for ExactPhraseScorer { - fn score(&mut self) -> Result { - let doc_id = self.conjunction.doc_id(); - let freq = self.freq as f32; - self.doc_scorer.score(doc_id, freq) - } -} - -impl DocIterator for ExactPhraseScorer { - fn doc_id(&self) -> DocId { - self.conjunction.doc_id() - } - - fn next(&mut self) -> Result { - let doc_id = self.conjunction.next()?; - self.do_next(doc_id) - } - - fn advance(&mut self, target: DocId) -> Result { - let doc_id = self.conjunction.advance(target)?; - self.do_next(doc_id) - } - - fn cost(&self) -> usize { - self.conjunction.cost() - } - - fn matches(&mut self) -> Result { - Ok(self.phrase_freq()? > 0) - } - - fn match_cost(&self) -> f32 { - self.match_cost - } - - /// advance to the next approximate match doc - fn approximate_next(&mut self) -> Result { - self.conjunction.next() - } - - /// Advances to the first approximate doc beyond the current doc - fn approximate_advance(&mut self, target: DocId) -> Result { - self.conjunction.advance(target) - } -} - -/// Position of a term in a document that takes into account the term offset within the phrase. -struct PhrasePositions { - pub position: i32, - // position in doc - pub count: i32, - // remaining pos in this doc - pub offset: i32, - // position in phrase - pub ord: i32, - // unique across all PhrasePositions instances - pub postings: *mut dyn PostingIterator, - // stream of docs & positions - pub next_pp_idx: i32, - // used to make list - pub rpt_group: i32, - // >=0 indicates that this is a repeating PP - pub rpt_ind: i32, - // index in the rptGroup - pub terms: Vec, - // for repetitions initialization -} - -unsafe impl Send for PhrasePositions {} - -impl PhrasePositions { - fn new(postings: *mut dyn PostingIterator, offset: i32, ord: i32, terms: Vec) -> Self { - PhrasePositions { - position: 0, - count: 0, - offset, - ord, - postings, - next_pp_idx: -1, - rpt_group: -1, - rpt_ind: 0, - terms, - } - } - - fn first_position(&mut self) -> Result<()> { - unsafe { - self.count = (*self.postings).freq()?; - } - self.next_position()?; // read first pos - Ok(()) - } - - /// Go to next location of this term current document, and set - /// position as location - offset, so that a - /// matching exact phrase is easily identified when all PhrasePositions - /// have exactly the same position. - fn next_position(&mut self) -> Result { - if self.count > 0 { - // read subsequent pos's - self.count -= 1; - unsafe { - self.position = (*self.postings).next_position()? - self.offset; - } - Ok(true) - } else { - Ok(false) - } - } -} - -impl fmt::Debug for PhrasePositions { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!( - f, - "offset: {}, position: {}, count: {}", - self.offset, self.position, self.count - )?; - if self.rpt_group >= 0 { - write!(f, " , rpt: {}, rpt_index: {}", self.rpt_group, self.rpt_ind)?; - } - Ok(()) - } -} - -/// PhrasePositions element in priority queue -struct PPElement { - pub index: usize, - // index in SloppyPhraseScorer.phrasePositions - pub pp: *const PhrasePositions, - // pointer to target -} - -unsafe impl Send for PPElement {} - -impl fmt::Debug for PPElement { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - unsafe { write!(f, "index: {}, pp: ({:?})", self.index, *self.pp) } - } -} - -impl PartialOrd for PPElement { - fn partial_cmp(&self, other: &Self) -> Option { - // reversed ordering for priority queue - unsafe { - let position_cmp = (*other.pp).position.cmp(&(*self.pp).position); - if position_cmp != Ordering::Equal { - Some(position_cmp) - } else { - let offset_cmp = (*other.pp).offset.cmp(&(*self.pp).offset); - if offset_cmp != Ordering::Equal { - Some(offset_cmp) - } else { - Some((*other.pp).ord.cmp(&(*self.pp).ord)) - } - } - } - } -} - -impl Ord for PPElement { - fn cmp(&self, other: &Self) -> Ordering { - self.partial_cmp(other).unwrap() - } -} - -impl PartialEq for PPElement { - fn eq(&self, other: &Self) -> bool { - self.index.eq(&other.index) - } -} - -impl Eq for PPElement {} - -// TODO a fake scorer struct used for `ConjunctionScorer` -struct PostingsIterAsScorer { - pub iterator: T, -} - -impl Scorer for PostingsIterAsScorer { - fn score(&mut self) -> Result { - unreachable!() - } -} - -impl DocIterator for PostingsIterAsScorer { - fn doc_id(&self) -> i32 { - self.iterator.doc_id() - } - - fn next(&mut self) -> Result { - self.iterator.next() - } - - fn advance(&mut self, target: i32) -> Result { - self.iterator.advance(target) - } - - fn cost(&self) -> usize { - self.iterator.cost() - } -} - -pub struct SloppyPhraseScorer { - conjunction: ConjunctionScorer>, - // a conjunction doc id set iterator - phrase_positions: Vec, - sloppy_freq: f32, - // phrase frequency in current doc as computed by phraseFreq(). - doc_scorer: Box, - slop: i32, - num_postings: usize, - pq: BinaryHeap, - // for advancing min position - end: i32, - // current largest phrase position - has_rpts: bool, - // flag indicating that there are repetitions (as checked in - // first candidate doc) - checked_rpts: bool, - // flag to only check for repetitions in first candidate doc - has_multi_term_rpts: bool, - // in each group are PPs that repeats each other (i.e. same term), sorted by (query) offset - // value are index of related pp in self.phrase_positions - rpt_group: Vec>, - rpt_stack: Vec, - // temporary stack for switching colliding repeating pps - num_matches: i32, - needs_scores: bool, - match_cost: f32, -} - -impl SloppyPhraseScorer { - fn new( - postings: Vec>, - slop: i32, - doc_scorer: Box, - needs_scores: bool, - match_cost: f32, - ) -> Self { - let num_postings = postings.len(); - let mut doc_iterators = Vec::with_capacity(num_postings); - let mut phrase_positions = Vec::with_capacity(num_postings); - for (idx, posting) in postings.into_iter().enumerate() { - let iterator = posting.postings; - doc_iterators.push(PostingsIterAsScorer { iterator }); - phrase_positions.push(PhrasePositions::new( - &mut doc_iterators[idx].iterator, - posting.pos, - idx as i32, - posting.terms.clone(), - )); - } - let conjunction = ConjunctionScorer::new(doc_iterators); - let pq = BinaryHeap::with_capacity(num_postings); - SloppyPhraseScorer { - conjunction, - phrase_positions, - sloppy_freq: 0f32, - doc_scorer, - slop, - num_postings, - pq, - end: 0, - has_rpts: false, - checked_rpts: false, - has_multi_term_rpts: false, - rpt_group: Vec::new(), - rpt_stack: Vec::new(), - num_matches: 0, - needs_scores, - match_cost, - } - } - - /// Score a candidate doc for all slop-valid position-combinations (matches) - /// encountered while traversing/hopping the PhrasePositions. - ///
The score contribution of a match depends on the distance: - ///
- highest score for distance=0 (exact match). - ///
- score gets lower as distance gets higher. - ///
Example: for query "a b"~2, a document "x a b a y" can be scored twice: - /// once for "a b" (distance=0), and once for "b a" (distance=2). - ///
Possibly not all valid combinations are encountered, because for efficiency - /// we always propagate the least PhrasePosition. This allows to base on - /// PriorityQueue and move forward faster. - /// As result, for example, document "a b c b a" - /// would score differently for queries "a b c"~4 and "c b a"~4, although - /// they really are equivalent. - /// Similarly, for doc "a b c b a f g", query "c b"~2 - /// would get same score as "g f"~2, although "c b"~2 could be matched twice. - /// We may want to fix this in the future (currently not, for performance reasons). - fn phrase_freq(&mut self) -> Result { - if !self.init_phrase_positions()? { - return Ok(0.0f32); - } - let mut freq = 0.0f32; - self.num_matches = 0; - let mut pp_idx = self.pq.pop().unwrap().index; - let mut match_length = self.end - self.phrase_positions[pp_idx].position; - let top_idx = self.pq.peek().unwrap().index; - let mut next = self.phrase_positions[top_idx].position; - while self.advance_pp(pp_idx)? { - if self.has_rpts && !self.advance_rpts(pp_idx)? { - break; // pps exhausted - } - if self.phrase_positions[pp_idx].position > next { - // done minimizing current match-length - if match_length <= self.slop as i32 { - freq += self.doc_scorer.compute_slop_factor(match_length); - self.num_matches += 1; - if !self.needs_scores { - return Ok(freq); - } - } - let ele = self.pp_element(pp_idx); - self.pq.push(ele); - pp_idx = self.pq.pop().unwrap().index; - next = self.phrase_positions[self.pq.peek().unwrap().index].position; - match_length = self.end - self.phrase_positions[pp_idx].position; - } else { - let match_length2 = self.end - self.phrase_positions[pp_idx].position; - match_length = min(match_length, match_length2); - } - } - if match_length <= self.slop { - freq += self.doc_scorer.compute_slop_factor(match_length); // score match - self.num_matches += 1; - } - Ok(freq) - } - - /// Initialize PhrasePositions in place. - /// A one time initialization for this scorer (on first doc matching all terms): - /// - Check if there are repetitions - /// - If there are, find groups of repetitions. - /// Examples: - /// 1. no repetitions: "ho my"~2 - /// 2. repetitions: "ho my my"~2 - /// 3. repetitions: "my ho my"~2 - /// - /// @return false if PPs are exhausted (and so current doc will not be a match) - fn init_phrase_positions(&mut self) -> Result { - self.end = i32::min_value(); - if !self.checked_rpts { - return self.init_first_time(); - } - if !self.has_rpts { - self.init_simple()?; - return Ok(true); - } - self.init_complex() - } - - /// no repeats: simplest case, and most common. - /// It is important to keep this piece of the code simple and efficient - fn init_simple(&mut self) -> Result<()> { - self.pq.clear(); - // position pps and build queue from list - for idx in 0..self.num_postings { - self.phrase_positions[idx].first_position()?; - if self.phrase_positions[idx].position > self.end { - self.end = self.phrase_positions[idx].position; - } - let ele = self.pp_element(idx); - self.pq.push(ele); - } - Ok(()) - } - - /// with repeats: not so simple. - fn init_complex(&mut self) -> Result { - self.place_first_positions()?; - if !self.advance_repeat_groups()? { - return Ok(false); - } - self.fill_queue(); - Ok(true) - } - - /// move all PPs to their first position - fn place_first_positions(&mut self) -> Result<()> { - for pp in &mut self.phrase_positions { - pp.first_position()?; - } - Ok(()) - } - - /// advance a PhrasePosition and update 'end', return false if exhausted - fn advance_pp(&mut self, pp_idx: usize) -> Result { - if !self.phrase_positions[pp_idx].next_position()? { - return Ok(false); - } - let position = self.phrase_positions[pp_idx].position; - if position > self.end { - self.end = position; - } - Ok(true) - } - - /// pp was just advanced. If that caused a repeater collision, resolve by advancing the lesser - /// of the two colliding pps. Note that there can only be one collision, as by the - /// initialization there were no collisions before pp was advanced. - fn advance_rpts(&mut self, pp_idx: usize) -> Result { - if self.phrase_positions[pp_idx].rpt_group < 0 { - return Ok(true); // not a repeater - } - let mut bits = FixedBitSet::new( - self.rpt_group[self.phrase_positions[pp_idx].rpt_group as usize].len(), - ); - let k0 = self.phrase_positions[pp_idx].rpt_ind; - let mut cur_idx = pp_idx; - loop { - let k = self.collide(cur_idx); - if k < 0 { - break; - } - // always advance the lesser of the (only) two colliding pps - cur_idx = self.lesser( - cur_idx, - self.rpt_group[self.phrase_positions[pp_idx].rpt_group as usize][k as usize], - ); - if !self.advance_pp(cur_idx)? { - return Ok(false); - } - if k != k0 { - // careful: mark only those currently in the queue - bits.ensure_capacity(k as usize); - bits.set(k as usize); - } - } - // collisions resolved, now re-queue - // empty (partially) the queue until seeing all pps advanced for resolving collisions - let mut n = 0usize; - // TODO would be good if we can avoid calling cardinality() in each iteration! - let num_bits = bits.len(); - - while bits.cardinality() > 0 { - let pp2_idx = self.pq.pop().unwrap().index; - self.rpt_stack[n] = pp2_idx; - n += 1; - { - let pp2 = &self.phrase_positions[pp2_idx]; - if pp2.rpt_group >= 0 && pp2.rpt_ind < num_bits as i32 // this bit may not have been set - && bits.get(pp2.rpt_ind as usize)? - { - bits.clear_batch(pp2.rpt_ind as usize, (pp2.rpt_ind + 1) as usize); - } - } - } - // add back to queue - for i in 0..n { - let ele = self.pp_element(self.rpt_stack[n - 1 - i]); - self.pq.push(ele); - } - Ok(true) - } - - /// compare two pps, but only by position and offset - fn lesser(&self, idx: usize, idx2: usize) -> usize { - let pp = &self.phrase_positions[idx]; - let pp2 = &self.phrase_positions[idx2]; - if pp.position < pp2.position || (pp.position == pp2.position && pp.offset < pp2.offset) { - idx - } else { - idx2 - } - } +#[cfg(test)] +mod tests { + use super::*; - /// index of a pp2 colliding with pp, or -1 if none - fn collide(&self, pp_idx: usize) -> i32 { - let pp = &self.phrase_positions[pp_idx]; - let tp_pos = Self::tp_pos(pp); - let rg = &self.rpt_group[pp.rpt_group as usize]; - for i in rg { - let pp2 = &self.phrase_positions[*i]; - if pp_idx != *i && Self::tp_pos(pp2) == tp_pos { - return pp2.rpt_ind; - } - } - -1 - } + use core::analysis::WhitespaceTokenizer; + use core::doc::{Field, FieldType, Fieldable, IndexOptions, Term}; + use core::index::writer::{IndexWriter, IndexWriterConfig}; + use core::search::collector::TopDocsCollector; + use core::search::{DefaultIndexSearcher, IndexSearcher}; + use core::store::directory::FSDirectory; - fn pp_element(&self, index: usize) -> PPElement { - let pp = &self.phrase_positions[index] as *const PhrasePositions; - PPElement { index, pp } - } + use std::fs; + use std::io; + use std::path::Path; + use std::sync::Arc; - /// Fill the queue (all pps are already placed - fn fill_queue(&mut self) { - self.pq.clear(); - let mut end = self.end; - for (idx, pq) in self.phrase_positions.iter().enumerate() { - if pq.position > end { - end = pq.position; - } - let ele = self.pp_element(idx); - self.pq.push(ele); - } - self.end = end; + fn indexed_text_field_type() -> FieldType { + let mut field_type = FieldType::default(); + field_type.index_options = IndexOptions::DocsAndFreqsAndPositionsAndOffsets; + field_type.store_term_vectors = true; + field_type.store_term_vector_offsets = true; + field_type.store_term_vector_positions = true; + field_type } - /// At initialization (each doc), each repetition group is sorted by (query) offset. - /// This provides the start condition: no collisions. - /// Case 1: no multi-term repeats - /// It is sufficient to advance each pp in the group by one less than its group index. - /// So lesser pp is not advanced, 2nd one advance once, 3rd one advanced twice, etc. - /// Case 2: multi-term repeats - /// - /// @return false if PPs are exhausted. - fn advance_repeat_groups(&mut self) -> Result { - for rg_idx in 0..self.rpt_group.len() { - if self.has_multi_term_rpts { - // more involved, some may not collide - let mut incr; - let mut i = 0; - while i < self.rpt_group[rg_idx].len() { - incr = 1; - let pp_idx = self.rpt_group[rg_idx][i]; - let mut k = self.collide(pp_idx); - while k >= 0 { - let pp_idx2 = self.lesser(pp_idx, self.rpt_group[rg_idx][k as usize]); - // at initialization always advance pp with higher offset - if !self.advance_pp(pp_idx2)? { - return Ok(false); // exhausted - } - if self.phrase_positions[pp_idx2].rpt_ind < i as i32 { - // should no happen? - incr = 0; - break; - } - k = self.collide(pp_idx); - } - i += incr; - } - } else { - // simpler, we know exactly how much to advance - for j in 1..self.rpt_group[rg_idx].len() { - for _ in 0..j { - if !self.phrase_positions[self.rpt_group[rg_idx][j]].next_position()? { - return Ok(false); // PPs exhausted - } - } - } - } - } - Ok(true) - } - - /// initialize with checking for repeats. Heavy work, but done only for the first candidate - /// doc. If there are repetitions, check if multi-term postings (MTP) are involved. - /// Without MTP, once PPs are placed in the first candidate doc, repeats (and groups) are - /// visible. With MTP, a more complex check is needed, up-front, as there may be "hidden - /// collisions". For example P1 has {A,B}, P1 has {B,C}, and the first doc is: "A C B". At - /// start, P1 would point to "A", p2 to "C", and it will not be identified that P1 and P2 - /// are repetitions of each other. The more complex initialization has two parts: - /// (1) identification of repetition groups. - /// (2) advancing repeat groups at the start of the doc. - /// For (1), a possible solution is to just create a single repetition group, - /// made of all repeating pps. But this would slow down the check for collisions, - /// as all pps would need to be checked. Instead, we compute "connected regions" - /// on the bipartite graph of postings and terms. - fn init_first_time(&mut self) -> Result { - self.checked_rpts = true; - self.place_first_positions()?; - - let (rpt_terms, terms) = self.repeating_terms(); - self.has_rpts = !terms.is_empty(); - - if self.has_rpts { - self.rpt_stack = vec![0usize; self.num_postings]; - let rgs = self.gather_rpt_groups(&rpt_terms, &terms); - self.sort_rpt_groups(rgs); - if !self.advance_repeat_groups()? { - return Ok(false); // PPs exhausted - } - } - self.fill_queue(); - Ok(true) - } - - /// sort each repetition group by (query) offset. - /// Done only once (at first doc) and allows to initialize faster for each doc. - fn sort_rpt_groups(&mut self, rgs: Vec>) { - for mut rg in rgs { - rg.sort_by(|idx1, idx2| { - self.phrase_positions[*idx1] - .offset - .cmp(&self.phrase_positions[*idx2].offset) - }); - for (j, idx) in rg.iter().enumerate() { - // we use this index for efficient re-queuing - self.phrase_positions[*idx].rpt_ind = j as i32; - } - self.rpt_group.push(rg); - } - } - - /// Detect repetition groups. Done once - for first doc - fn gather_rpt_groups( - &mut self, - rpt_terms: &HashMap, - terms: &[Term], - ) -> Vec> { - let rpp = self.repeating_pps(rpt_terms); - let mut res = Vec::new(); - if !self.has_multi_term_rpts { - // simpler - no multi-terms - can base on positions in first doc - for i in 0..rpp.len() { - let idx1 = rpp[i]; - if self.phrase_positions[idx1].rpt_group >= 0 { - // already marked as a repetition - continue; - } - let tp_pos = Self::tp_pos(&self.phrase_positions[idx1]); - for idx2 in rpp.iter().skip(i + 1) { - let idx2 = *idx2; - if self.phrase_positions[idx2].rpt_group >= 0 || // already marked as a repetition - self.phrase_positions[idx2].offset == self.phrase_positions[idx1].offset || // not a repetition: two PPs are originally in same offset in the query! - Self::tp_pos(&self.phrase_positions[idx2]) != tp_pos - { - continue; - } - // a repetition - let mut g = self.phrase_positions[idx1].rpt_group; - if g < 0 { - g = res.len() as i32; - self.phrase_positions[idx1].rpt_group = g; - let mut rl = Vec::with_capacity(2); - rl.push(idx1); - res.push(rl); - } - self.phrase_positions[idx2].rpt_group = g; - res[g as usize].push(idx2); - } - } - } else { - // more involved - has multi-terms - let mut tmp = Vec::new(); - let mut bb = self.pp_terms_bit_sets(&rpp, rpt_terms); - Self::union_term_groups(&mut bb); - let tg = self.term_groups(terms, &bb); - let mut distinct_group_ids = HashSet::new(); - for v in tg.values() { - distinct_group_ids.insert(*v); - } - for _ in 0..distinct_group_ids.len() { - tmp.push(HashSet::new()); - } - for pp_idx in &rpp { - for term_idx in 0..self.phrase_positions[*pp_idx].terms.len() { - if rpt_terms.contains_key(&self.phrase_positions[*pp_idx].terms[term_idx]) { - let g = tg[&self.phrase_positions[*pp_idx].terms[term_idx]]; - tmp[g].insert(*pp_idx); - assert!( - self.phrase_positions[*pp_idx].rpt_group == -1 - || self.phrase_positions[*pp_idx].rpt_group == g as i32 - ); - self.phrase_positions[*pp_idx].rpt_group = g as i32; - } - } - } - for hs in &tmp { - let mut data = Vec::with_capacity(hs.len()); - for v in hs { - data.push(*v); - } - res.push(data); - } - } - res - } - - /// Actual position in doc of a PhrasePosition, relies on that position = tpPos - offset) - fn tp_pos(pp: &PhrasePositions) -> i32 { - pp.position + pp.offset - } - - /// find repeating terms and assign them ordinal values - fn repeating_terms(&self) -> (HashMap, Vec) { - let mut tord = HashMap::new(); - let mut terms = Vec::new(); - let mut tcnt = HashMap::new(); - for pp in &self.phrase_positions { - for t in &pp.terms { - let cnt = if let Some(v) = tcnt.get(t) { - *v + 1usize - } else { - 1usize - }; - tcnt.insert(t.clone(), cnt); - if cnt == 2 { - let ord = tord.len(); - tord.insert(t.clone(), ord); - terms.push(t.clone()); - } - } - } - (tord, terms) - } - - /// find repeating pps, and for each, if has multi-terms, update this.hasMultiTermRpts - fn repeating_pps(&mut self, rpt_terms: &HashMap) -> Vec { - let mut rp = Vec::new(); - let mut has_multi_term_rpts = false; - for (idx, pp) in self.phrase_positions.iter().enumerate() { - for t in &pp.terms { - if rpt_terms.contains_key(t) { - rp.push(idx); - has_multi_term_rpts |= pp.terms.len() > 1; - break; - } - } - } - self.has_multi_term_rpts |= has_multi_term_rpts; - rp - } - - /// bit-sets - for each repeating pp, for each of its repeating terms, the term ordinal values - /// is set - fn pp_terms_bit_sets( - &self, - rpp_idx: &[usize], - tord: &HashMap, - ) -> Vec { - let mut bb = Vec::with_capacity(rpp_idx.len()); - for idx in rpp_idx { - let mut b = FixedBitSet::new(tord.len()); - for t in &self.phrase_positions[*idx].terms { - if let Some(ord) = tord.get(t) { - b.set(*ord); - } - } - bb.push(b); - } - bb - } - - /// union (term group) bit-sets until they are disjoint (O(n^^2)), - /// and each group have different terms - fn union_term_groups(bb: &mut Vec) { - let mut incr; - let mut i = 0; - while i < bb.len() - 1 { - incr = 1; - let mut j = i + 1; - while j < bb.len() { - if bb[i].intersects(&bb[j]) { - let bbj = bb.remove(j); - bb[i].set_or(&bbj); - incr = 0; - } else { - j += 1; - } - } - i += incr; - } - } - - /// map each term to the single group that contains it - fn term_groups(&self, terms: &[Term], bb: &[FixedBitSet]) -> HashMap { - let mut tg = HashMap::new(); - for (i, bit) in bb.iter().enumerate() { - let mut ord = bit.next_set_bit(0); - while ord != NO_MORE_DOCS { - tg.insert(terms[ord as usize].clone(), i); - ord = if ord as usize >= bit.len() { - NO_MORE_DOCS - } else { - bit.next_set_bit((ord + 1) as usize) - }; - } - } - tg - } -} - -impl Scorer for SloppyPhraseScorer { - fn score(&mut self) -> Result { - let doc_id = self.doc_id(); - self.doc_scorer.score(doc_id, self.sloppy_freq) - } -} - -impl DocIterator for SloppyPhraseScorer { - fn doc_id(&self) -> i32 { - self.conjunction.doc_id() - } - - fn next(&mut self) -> Result { - self.approximate_next()?; - two_phase_next(self) - } - - fn advance(&mut self, target: i32) -> Result { - self.approximate_advance(target)?; - two_phase_next(self) - } - - fn cost(&self) -> usize { - self.conjunction.cost() - } - - fn matches(&mut self) -> Result { - let sloppy_freq = self.phrase_freq()?; // check for phrase - self.sloppy_freq = sloppy_freq; - Ok(sloppy_freq > f32::EPSILON) - } - - fn match_cost(&self) -> f32 { - self.match_cost - } - - fn support_two_phase(&self) -> bool { - true - } - - fn approximate_next(&mut self) -> Result { - self.conjunction.next() + fn new_index_text_field(field_name: String, text: String) -> Field { + let token_stream = WhitespaceTokenizer::new(Box::new(StringReader::new(text))); + Field::new( + field_name, + indexed_text_field_type(), + None, + Some(Box::new(token_stream)), + ) } - fn approximate_advance(&mut self, target: i32) -> Result { - self.conjunction.advance(target) - } -} + struct StringReader { + text: String, + index: usize, + } + + impl StringReader { + fn new(text: String) -> Self { + StringReader { text, index: 0 } + } + } + + impl io::Read for StringReader { + fn read(&mut self, buf: &mut [u8]) -> io::Result { + let remain = buf.len().min(self.text.len() - self.index); + if remain > 0 { + buf[..remain].copy_from_slice(&self.text.as_bytes()[self.index..self.index + remain]); + self.index += remain; + } + Ok(remain) + } + } + + #[test] + fn phrase_query() { + // create index directory + let path = "/tmp/test_rucene"; + let dir_path = Path::new(path); + if dir_path.exists() { + fs::remove_dir_all(&dir_path).unwrap(); + fs::create_dir(&dir_path).unwrap(); + } + + // create index writer + let config = Arc::new(IndexWriterConfig::default()); + let directory = Arc::new(FSDirectory::with_path(&dir_path).unwrap()); + let writer = IndexWriter::new(directory, config).unwrap(); + + { + let mut doc: Vec> = vec![]; + // add indexed text field + let text = "The quick brown fox jumps over a lazy dog"; + doc.push(Box::new(new_index_text_field("title".into(), text.into()))); + // add the document + writer.add_document(doc).unwrap(); + } + + { + let mut doc: Vec> = vec![]; + // add indexed text field + let text = "The quick fox jumps over a lazy dog"; + doc.push(Box::new(new_index_text_field("title".into(), text.into()))); + // add the document + writer.add_document(doc).unwrap(); + } + + { + let mut doc: Vec> = vec![]; + // add indexed text field + let text = "The fox jumps quick over a lazy dog"; + doc.push(Box::new(new_index_text_field("title".into(), text.into()))); + // add the document + writer.add_document(doc).unwrap(); + } + // flush to disk + writer.commit().unwrap(); + + // new index search + let reader = writer.get_reader(true, false).unwrap(); + let index_searcher = DefaultIndexSearcher::new(Arc::new(reader), None); + + { + // search slop 0 + let query: PhraseQuery = PhraseQuery::new( + vec![ + Term::new("title".into(), "quick".as_bytes().to_vec()), + Term::new("title".into(), "fox".as_bytes().to_vec()), + ], + vec![0, 1], + 0, + None, None, + ).unwrap(); + let mut collector = TopDocsCollector::new(10); + index_searcher.search(&query, &mut collector).unwrap(); + let top_docs = collector.top_docs(); + assert_eq!(top_docs.total_hits(), 1); + } + + { + // search slop 1 + let query: PhraseQuery = PhraseQuery::new( + vec![ + Term::new("title".into(), "quick".as_bytes().to_vec()), + Term::new("title".into(), "fox".as_bytes().to_vec()), + ], + vec![0, 1], + 1, + None, None, + ).unwrap(); + let mut collector = TopDocsCollector::new(10); + index_searcher.search(&query, &mut collector).unwrap(); + let top_docs = collector.top_docs(); + assert_eq!(top_docs.total_hits(), 2); + } + + { + // search slop 2 + let query: PhraseQuery = PhraseQuery::new( + vec![ + Term::new("title".into(), "quick".as_bytes().to_vec()), + Term::new("title".into(), "fox".as_bytes().to_vec()), + ], + vec![0, 1], + 2, + None, None, + ).unwrap(); + let mut collector = TopDocsCollector::new(10); + index_searcher.search(&query, &mut collector).unwrap(); + let top_docs = collector.top_docs(); + assert_eq!(top_docs.total_hits(), 2); + } + + { + // search slop 3 + let query: PhraseQuery = PhraseQuery::new( + vec![ + Term::new("title".into(), "quick".as_bytes().to_vec()), + Term::new("title".into(), "fox".as_bytes().to_vec()), + ], + vec![0, 1], + 3, + None, None, + ).unwrap(); + let mut collector = TopDocsCollector::new(10); + index_searcher.search(&query, &mut collector).unwrap(); + let top_docs = collector.top_docs(); + assert_eq!(top_docs.total_hits(), 3); + } + } +} \ No newline at end of file diff --git a/src/core/search/scorer/mod.rs b/src/core/search/scorer/mod.rs index 65ce6b3..525b5ca 100644 --- a/src/core/search/scorer/mod.rs +++ b/src/core/search/scorer/mod.rs @@ -47,6 +47,10 @@ mod boosting_scorer; pub use self::boosting_scorer::*; +mod phrase_scorer; + +pub use self::phrase_scorer::*; + use std::collections::HashMap; use std::i32; diff --git a/src/core/search/scorer/phrase_scorer.rs b/src/core/search/scorer/phrase_scorer.rs new file mode 100644 index 0000000..95e7a81 --- /dev/null +++ b/src/core/search/scorer/phrase_scorer.rs @@ -0,0 +1,1070 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::cell::RefCell; +use std::cmp::{min, Ord, Ordering}; +use std::collections::{BinaryHeap, HashMap, HashSet}; +use std::fmt; +use std::rc::Rc; + +use core::codec::PostingIterator; +use core::doc::Term; +use core::search::{DocIterator, NO_MORE_DOCS}; +use core::search::scorer::{ConjunctionScorer, Scorer, two_phase_next}; +use core::search::similarity::SimScorer; +use core::util::{Bits, BitSet, DocId, FixedBitSet, ImmutableBitSet}; +use error::Result; + +// a fake scorer struct used for `ConjunctionScorer` +pub struct PostingsIterAsScorer { + pub iterator: Rc>, +} + +unsafe impl Send for PostingsIterAsScorer {} + +impl Scorer for PostingsIterAsScorer { + fn score(&mut self) -> Result { + unreachable!() + } +} + +impl DocIterator for PostingsIterAsScorer { + fn doc_id(&self) -> i32 { + self.iterator.borrow().doc_id() + } + + fn next(&mut self) -> Result { + self.iterator.borrow_mut().next() + } + + fn advance(&mut self, target: i32) -> Result { + self.iterator.borrow_mut().advance(target) + } + + fn cost(&self) -> usize { + self.iterator.borrow().cost() + } +} + + +pub struct PostingsAndFreq { + pub postings: T, + pub pos: i32, + pub terms: Vec, + pub nterms: i32, + // for faster comparisons +} + +impl PostingsAndFreq { + pub fn new(postings: T, pos: i32, term: &Term) -> Self { + PostingsAndFreq { + postings, + pos, + terms: vec![term.clone()], + nterms: 1, + } + } +} + +impl Ord for PostingsAndFreq { + fn cmp(&self, other: &Self) -> Ordering { + self.partial_cmp(&other).unwrap() + } +} + +impl PartialOrd for PostingsAndFreq { + fn partial_cmp(&self, other: &Self) -> Option { + if self.pos != other.pos { + return Some(self.pos.cmp(&other.pos)); + } + + if self.nterms != other.nterms { + return Some(self.nterms.cmp(&other.nterms)); + } + + if self.nterms == 0 { + return Some(Ordering::Equal); + } + + for i in 0..self.terms.len() { + let res = if self.terms[i].field.eq(&other.terms[i].field) { + self.terms[i].bytes.cmp(&other.terms[i].bytes) + } else { + self.terms[i].field.cmp(&other.terms[i].field) + }; + + if res != Ordering::Equal { + return Some(res); + } + } + + Some(Ordering::Equal) + } +} + +impl Eq for PostingsAndFreq {} + +impl PartialEq for PostingsAndFreq { + fn eq(&self, other: &Self) -> bool { + self == other + } +} + +pub struct ExactPhraseScorer { + freq: i32, + needs_scores: bool, + match_cost: f32, + postings: Vec>, + doc_scorer: Box, + conjunction: ConjunctionScorer>, +} + +impl ExactPhraseScorer { + pub fn new( + postings: Vec>, + doc_scorer: Box, + needs_scores: bool, + match_cost: f32, + ) -> Self { + let mut iterators = Vec::with_capacity(postings.len()); + let mut postings_and_positions: Vec> = Vec::with_capacity(postings.len()); + + for (_, posting) in postings.into_iter().enumerate() { + let iterator = Rc::new(RefCell::new(posting.postings)); + iterators.push(PostingsIterAsScorer { iterator: iterator.clone() }); + postings_and_positions.push(PostingsAndPosition::new( + iterator.clone(), + posting.pos, + )); + } + + let conjunction = ConjunctionScorer::new(iterators); + + ExactPhraseScorer { + freq: 0, + needs_scores, + match_cost, + postings: postings_and_positions, + doc_scorer, + conjunction, + } + } + + pub fn freq(&self) -> i32 { + self.freq + } + + fn advance_position(posting: &mut PostingsAndPosition, target: i32) -> Result { + while posting.pos < target { + if posting.up_to == posting.freq { + return Ok(false); + } else { + posting.pos = posting.postings.borrow_mut().next_position()?; + posting.up_to += 1; + } + } + + Ok(true) + } + + fn phrase_freq(&mut self) -> Result { + // reset state + for pp in &mut self.postings { + pp.freq = pp.postings.borrow_mut().freq()?; + pp.pos = pp.postings.borrow_mut().next_position()?; + + pp.up_to = 1; + } + + let mut freq = 0; + let mut lead = self.postings.remove(0); + + 'advanceHead: loop { + let phrase_pos = lead.pos - lead.offset; + for posting in &mut self.postings { + let expected_pos = phrase_pos + posting.offset; + + // advance up to the same position as the lead + if !Self::advance_position(posting, expected_pos)? { + break 'advanceHead; + } + + if posting.pos != expected_pos { + // we advanced too far + let target = posting.pos - posting.offset + lead.offset; + if Self::advance_position(&mut lead, target)? { + continue 'advanceHead; + } else { + break 'advanceHead; + } + } + } + + freq += 1; + if !self.needs_scores { + break; + } + + if lead.up_to == lead.freq { + break; + } + + lead.pos = lead.postings.borrow_mut().next_position()?; + lead.up_to += 1; + } + + self.postings.insert(0, lead); + self.freq = freq; + + Ok(self.freq) + } + + pub fn do_next(&mut self, doc_id: DocId) -> Result { + let mut doc = doc_id; + loop { + if doc == NO_MORE_DOCS { + return Ok(NO_MORE_DOCS); + } else if self.matches()? { + return Ok(doc); + } + + doc = self.conjunction.next()?; + } + } +} + +impl Scorer for ExactPhraseScorer { + fn score(&mut self) -> Result { + let doc_id = self.conjunction.doc_id(); + let freq = self.freq as f32; + self.doc_scorer.score(doc_id, freq) + } +} + +unsafe impl Send for ExactPhraseScorer {} + +impl DocIterator for ExactPhraseScorer { + fn doc_id(&self) -> DocId { + self.conjunction.doc_id() + } + + fn next(&mut self) -> Result { + let doc_id = self.conjunction.next()?; + self.do_next(doc_id) + } + + fn advance(&mut self, target: DocId) -> Result { + let doc_id = self.conjunction.advance(target)?; + self.do_next(doc_id) + } + + fn cost(&self) -> usize { + self.conjunction.cost() + } + + fn matches(&mut self) -> Result { + Ok(self.phrase_freq()? > 0) + } + + fn match_cost(&self) -> f32 { + println!("SELF.MATCH_COST {}", self.match_cost); + self.match_cost + } + + /// advance to the next approximate match doc + fn approximate_next(&mut self) -> Result { + let doc_id = self.conjunction.next()?; + self.do_next(doc_id) + } + + /// Advances to the first approximate doc beyond the current doc + fn approximate_advance(&mut self, target: DocId) -> Result { + let doc_id = self.conjunction.advance(target)?; + self.do_next(doc_id) + } +} + +pub struct PostingsAndPosition { + postings: Rc>, + pos: i32, + offset: i32, + freq: i32, + up_to: i32, +} + +unsafe impl Send for PostingsAndPosition {} + +impl PostingsAndPosition { + pub fn new(postings: Rc>, offset: i32) -> PostingsAndPosition { + PostingsAndPosition { + postings, + pos: -1, + offset, + freq: 0, + up_to: 1, + } + } +} + +/// Position of a term in a document that takes into account the term offset within the phrase. +struct PhrasePositions { + pub position: i32, + // position in doc + pub count: i32, + // remaining pos in this doc + pub offset: i32, + // position in phrase + pub ord: i32, + // unique across all PhrasePositions instances + pub postings: Rc>, + // stream of docs & positions + pub next_pp_idx: i32, + // used to make list + pub rpt_group: i32, + // >=0 indicates that this is a repeating PP + pub rpt_ind: i32, + // index in the rptGroup + pub terms: Vec, + // for repetitions initialization +} + +unsafe impl Send for PhrasePositions {} + +impl PhrasePositions { + fn new(postings: Rc>, offset: i32, ord: i32, terms: Vec) -> Self { + PhrasePositions { + position: 0, + count: 0, + offset, + ord, + postings, + next_pp_idx: -1, + rpt_group: -1, + rpt_ind: 0, + terms, + } + } + + fn first_position(&mut self) -> Result<()> { + self.count = self.postings.borrow_mut().freq()?; + self.next_position()?; // read first pos + Ok(()) + } + + /// Go to next location of this term current document, and set + /// position as location - offset, so that a + /// matching exact phrase is easily identified when all PhrasePositions + /// have exactly the same position. + fn next_position(&mut self) -> Result { + if self.count > 0 { + // read subsequent pos's + self.count -= 1; + self.position = self.postings.borrow_mut().next_position()? - self.offset; + Ok(true) + } else { + Ok(false) + } + } +} + +impl fmt::Debug for PhrasePositions { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "offset: {}, position: {}, count: {}", + self.offset, self.position, self.count + )?; + if self.rpt_group >= 0 { + write!(f, " , rpt: {}, rpt_index: {}", self.rpt_group, self.rpt_ind)?; + } + Ok(()) + } +} + +/// PhrasePositions element in priority queue +struct PPElement { + pub index: usize, + // index in SloppyPhraseScorer.phrasePositions + pub pp: *const PhrasePositions, + // pointer to target +} + +unsafe impl Send for PPElement {} + +impl fmt::Debug for PPElement { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + unsafe { write!(f, "index: {}, pp: ({:?})", self.index, *self.pp) } + } +} + +impl PartialOrd for PPElement { + fn partial_cmp(&self, other: &Self) -> Option { + // reversed ordering for priority queue + unsafe { + let position_cmp = (*other.pp).position.cmp(&(*self.pp).position); + if position_cmp != Ordering::Equal { + Some(position_cmp) + } else { + let offset_cmp = (*other.pp).offset.cmp(&(*self.pp).offset); + if offset_cmp != Ordering::Equal { + Some(offset_cmp) + } else { + Some((*other.pp).ord.cmp(&(*self.pp).ord)) + } + } + } + } +} + +impl Ord for PPElement { + fn cmp(&self, other: &Self) -> Ordering { + self.partial_cmp(other).unwrap() + } +} + +impl PartialEq for PPElement { + fn eq(&self, other: &Self) -> bool { + self.index.eq(&other.index) + } +} + +impl Eq for PPElement {} + +pub struct SloppyPhraseScorer { + conjunction: ConjunctionScorer>, + // a conjunction doc id set iterator + phrase_positions: Vec>, + sloppy_freq: f32, + // phrase frequency in current doc as computed by phraseFreq(). + doc_scorer: Box, + slop: i32, + num_postings: usize, + pq: BinaryHeap>, + // for advancing min position + end: i32, + // current largest phrase position + has_rpts: bool, + // flag indicating that there are repetitions (as checked in + // first candidate doc) + checked_rpts: bool, + // flag to only check for repetitions in first candidate doc + has_multi_term_rpts: bool, + // in each group are PPs that repeats each other (i.e. same term), sorted by (query) offset + // value are index of related pp in self.phrase_positions + rpt_group: Vec>, + rpt_stack: Vec, + // temporary stack for switching colliding repeating pps + num_matches: i32, + needs_scores: bool, + match_cost: f32, +} + +impl SloppyPhraseScorer { + pub fn new( + postings: Vec>, + slop: i32, + doc_scorer: Box, + needs_scores: bool, + match_cost: f32, + ) -> Self { + let num_postings = postings.len(); + let mut doc_iterators = Vec::with_capacity(num_postings); + let mut phrase_positions = Vec::with_capacity(num_postings); + for (idx, posting) in postings.into_iter().enumerate() { + let iterator = Rc::new(RefCell::new(posting.postings)); + phrase_positions.push(PhrasePositions::new( + iterator.clone(), + posting.pos, + idx as i32, + posting.terms.clone(), + )); + doc_iterators.push(PostingsIterAsScorer { iterator: iterator.clone() }); + } + let conjunction = ConjunctionScorer::new(doc_iterators); + + let pq = BinaryHeap::with_capacity(num_postings); + SloppyPhraseScorer { + conjunction, + phrase_positions, + sloppy_freq: 0f32, + doc_scorer, + slop, + num_postings, + pq, + end: 0, + has_rpts: false, + checked_rpts: false, + has_multi_term_rpts: false, + rpt_group: Vec::new(), + rpt_stack: Vec::new(), + num_matches: 0, + needs_scores, + match_cost, + } + } + + pub fn sloppy_freq(&self) -> f32 { + self.sloppy_freq + } + + /// Score a candidate doc for all slop-valid position-combinations (matches) + /// encountered while traversing/hopping the PhrasePositions. + ///
The score contribution of a match depends on the distance: + ///
- highest score for distance=0 (exact match). + ///
- score gets lower as distance gets higher. + ///
Example: for query "a b"~2, a document "x a b a y" can be scored twice: + /// once for "a b" (distance=0), and once for "b a" (distance=2). + ///
Possibly not all valid combinations are encountered, because for efficiency + /// we always propagate the least PhrasePosition. This allows to base on + /// PriorityQueue and move forward faster. + /// As result, for example, document "a b c b a" + /// would score differently for queries "a b c"~4 and "c b a"~4, although + /// they really are equivalent. + /// Similarly, for doc "a b c b a f g", query "c b"~2 + /// would get same score as "g f"~2, although "c b"~2 could be matched twice. + /// We may want to fix this in the future (currently not, for performance reasons). + fn phrase_freq(&mut self) -> Result { + if !self.init_phrase_positions()? { + return Ok(0.0f32); + } + let mut freq = 0.0f32; + self.num_matches = 0; + let mut pp_idx = self.pq.pop().unwrap().index; + let mut match_length = self.end - self.phrase_positions[pp_idx].position; + let top_idx = self.pq.peek().unwrap().index; + let mut next = self.phrase_positions[top_idx].position; + while self.advance_pp(pp_idx)? { + if self.has_rpts && !self.advance_rpts(pp_idx)? { + break; // pps exhausted + } + if self.phrase_positions[pp_idx].position > next { + // done minimizing current match-length + if match_length <= self.slop as i32 { + freq += self.doc_scorer.compute_slop_factor(match_length); + self.num_matches += 1; + if !self.needs_scores { + return Ok(freq); + } + } + let ele = self.pp_element(pp_idx); + self.pq.push(ele); + pp_idx = self.pq.pop().unwrap().index; + next = self.phrase_positions[self.pq.peek().unwrap().index].position; + match_length = self.end - self.phrase_positions[pp_idx].position; + } else { + let match_length2 = self.end - self.phrase_positions[pp_idx].position; + match_length = min(match_length, match_length2); + } + } + if match_length <= self.slop { + freq += self.doc_scorer.compute_slop_factor(match_length); // score match + self.num_matches += 1; + } + Ok(freq) + } + + /// Initialize PhrasePositions in place. + /// A one time initialization for this scorer (on first doc matching all terms): + /// - Check if there are repetitions + /// - If there are, find groups of repetitions. + /// Examples: + /// 1. no repetitions: "ho my"~2 + /// 2. repetitions: "ho my my"~2 + /// 3. repetitions: "my ho my"~2 + /// + /// @return false if PPs are exhausted (and so current doc will not be a match) + fn init_phrase_positions(&mut self) -> Result { + self.end = i32::min_value(); + if !self.checked_rpts { + return self.init_first_time(); + } + if !self.has_rpts { + self.init_simple()?; + return Ok(true); + } + self.init_complex() + } + + /// no repeats: simplest case, and most common. + /// It is important to keep this piece of the code simple and efficient + fn init_simple(&mut self) -> Result<()> { + self.pq.clear(); + // position pps and build queue from list + for idx in 0..self.num_postings { + self.phrase_positions[idx].first_position()?; + if self.phrase_positions[idx].position > self.end { + self.end = self.phrase_positions[idx].position; + } + let ele = self.pp_element(idx); + self.pq.push(ele); + } + Ok(()) + } + + /// with repeats: not so simple. + fn init_complex(&mut self) -> Result { + self.place_first_positions()?; + if !self.advance_repeat_groups()? { + return Ok(false); + } + self.fill_queue(); + Ok(true) + } + + /// move all PPs to their first position + fn place_first_positions(&mut self) -> Result<()> { + for pp in &mut self.phrase_positions { + pp.first_position()?; + } + Ok(()) + } + + /// advance a PhrasePosition and update 'end', return false if exhausted + fn advance_pp(&mut self, pp_idx: usize) -> Result { + if !self.phrase_positions[pp_idx].next_position()? { + return Ok(false); + } + let position = self.phrase_positions[pp_idx].position; + if position > self.end { + self.end = position; + } + Ok(true) + } + + /// pp was just advanced. If that caused a repeater collision, resolve by advancing the lesser + /// of the two colliding pps. Note that there can only be one collision, as by the + /// initialization there were no collisions before pp was advanced. + fn advance_rpts(&mut self, pp_idx: usize) -> Result { + if self.phrase_positions[pp_idx].rpt_group < 0 { + return Ok(true); // not a repeater + } + let mut bits = FixedBitSet::new( + self.rpt_group[self.phrase_positions[pp_idx].rpt_group as usize].len(), + ); + let k0 = self.phrase_positions[pp_idx].rpt_ind; + let mut cur_idx = pp_idx; + loop { + let k = self.collide(cur_idx); + if k < 0 { + break; + } + // always advance the lesser of the (only) two colliding pps + cur_idx = self.lesser( + cur_idx, + self.rpt_group[self.phrase_positions[pp_idx].rpt_group as usize][k as usize], + ); + if !self.advance_pp(cur_idx)? { + return Ok(false); + } + if k != k0 { + // careful: mark only those currently in the queue + bits.ensure_capacity(k as usize); + bits.set(k as usize); + } + } + // collisions resolved, now re-queue + // empty (partially) the queue until seeing all pps advanced for resolving collisions + let mut n = 0usize; + // TODO would be good if we can avoid calling cardinality() in each iteration! + let num_bits = bits.len(); + + while bits.cardinality() > 0 { + let pp2_idx = self.pq.pop().unwrap().index; + self.rpt_stack[n] = pp2_idx; + n += 1; + { + let pp2 = &self.phrase_positions[pp2_idx]; + if pp2.rpt_group >= 0 && pp2.rpt_ind < num_bits as i32 // this bit may not have been set + && bits.get(pp2.rpt_ind as usize)? + { + bits.clear_batch(pp2.rpt_ind as usize, (pp2.rpt_ind + 1) as usize); + } + } + } + // add back to queue + for i in 0..n { + let ele = self.pp_element(self.rpt_stack[n - 1 - i]); + self.pq.push(ele); + } + Ok(true) + } + + /// compare two pps, but only by position and offset + fn lesser(&self, idx: usize, idx2: usize) -> usize { + let pp = &self.phrase_positions[idx]; + let pp2 = &self.phrase_positions[idx2]; + if pp.position < pp2.position || (pp.position == pp2.position && pp.offset < pp2.offset) { + idx + } else { + idx2 + } + } + + /// index of a pp2 colliding with pp, or -1 if none + fn collide(&self, pp_idx: usize) -> i32 { + let pp = &self.phrase_positions[pp_idx]; + let tp_pos = Self::tp_pos(pp); + let rg = &self.rpt_group[pp.rpt_group as usize]; + for i in rg { + let pp2 = &self.phrase_positions[*i]; + if pp_idx != *i && Self::tp_pos(pp2) == tp_pos { + return pp2.rpt_ind; + } + } + -1 + } + + fn pp_element(&self, index: usize) -> PPElement { + let pp = &self.phrase_positions[index] as *const PhrasePositions; + PPElement { index, pp } + } + + /// Fill the queue (all pps are already placed + fn fill_queue(&mut self) { + self.pq.clear(); + let mut end = self.end; + for (idx, pq) in self.phrase_positions.iter().enumerate() { + if pq.position > end { + end = pq.position; + } + let ele = self.pp_element(idx); + self.pq.push(ele); + } + self.end = end; + } + + /// At initialization (each doc), each repetition group is sorted by (query) offset. + /// This provides the start condition: no collisions. + /// Case 1: no multi-term repeats + /// It is sufficient to advance each pp in the group by one less than its group index. + /// So lesser pp is not advanced, 2nd one advance once, 3rd one advanced twice, etc. + /// Case 2: multi-term repeats + /// + /// @return false if PPs are exhausted. + fn advance_repeat_groups(&mut self) -> Result { + for rg_idx in 0..self.rpt_group.len() { + if self.has_multi_term_rpts { + // more involved, some may not collide + let mut incr; + let mut i = 0; + while i < self.rpt_group[rg_idx].len() { + incr = 1; + let pp_idx = self.rpt_group[rg_idx][i]; + let mut k = self.collide(pp_idx); + while k >= 0 { + let pp_idx2 = self.lesser(pp_idx, self.rpt_group[rg_idx][k as usize]); + // at initialization always advance pp with higher offset + if !self.advance_pp(pp_idx2)? { + return Ok(false); // exhausted + } + if self.phrase_positions[pp_idx2].rpt_ind < i as i32 { + // should no happen? + incr = 0; + break; + } + k = self.collide(pp_idx); + } + i += incr; + } + } else { + // simpler, we know exactly how much to advance + for j in 1..self.rpt_group[rg_idx].len() { + for _ in 0..j { + if !self.phrase_positions[self.rpt_group[rg_idx][j]].next_position()? { + return Ok(false); // PPs exhausted + } + } + } + } + } + Ok(true) + } + + /// initialize with checking for repeats. Heavy work, but done only for the first candidate + /// doc. If there are repetitions, check if multi-term postings (MTP) are involved. + /// Without MTP, once PPs are placed in the first candidate doc, repeats (and groups) are + /// visible. With MTP, a more complex check is needed, up-front, as there may be "hidden + /// collisions". For example P1 has {A,B}, P1 has {B,C}, and the first doc is: "A C B". At + /// start, P1 would point to "A", p2 to "C", and it will not be identified that P1 and P2 + /// are repetitions of each other. The more complex initialization has two parts: + /// (1) identification of repetition groups. + /// (2) advancing repeat groups at the start of the doc. + /// For (1), a possible solution is to just create a single repetition group, + /// made of all repeating pps. But this would slow down the check for collisions, + /// as all pps would need to be checked. Instead, we compute "connected regions" + /// on the bipartite graph of postings and terms. + fn init_first_time(&mut self) -> Result { + self.checked_rpts = true; + self.place_first_positions()?; + + let (rpt_terms, terms) = self.repeating_terms(); + self.has_rpts = !terms.is_empty(); + + if self.has_rpts { + self.rpt_stack = vec![0usize; self.num_postings]; + let rgs = self.gather_rpt_groups(&rpt_terms, &terms); + self.sort_rpt_groups(rgs); + if !self.advance_repeat_groups()? { + return Ok(false); // PPs exhausted + } + } + self.fill_queue(); + Ok(true) + } + + /// sort each repetition group by (query) offset. + /// Done only once (at first doc) and allows to initialize faster for each doc. + fn sort_rpt_groups(&mut self, rgs: Vec>) { + for mut rg in rgs { + rg.sort_by(|idx1, idx2| { + self.phrase_positions[*idx1] + .offset + .cmp(&self.phrase_positions[*idx2].offset) + }); + for (j, idx) in rg.iter().enumerate() { + // we use this index for efficient re-queuing + self.phrase_positions[*idx].rpt_ind = j as i32; + } + self.rpt_group.push(rg); + } + } + + /// Detect repetition groups. Done once - for first doc + fn gather_rpt_groups( + &mut self, + rpt_terms: &HashMap, + terms: &[Term], + ) -> Vec> { + let rpp = self.repeating_pps(rpt_terms); + let mut res = Vec::new(); + if !self.has_multi_term_rpts { + // simpler - no multi-terms - can base on positions in first doc + for i in 0..rpp.len() { + let idx1 = rpp[i]; + if self.phrase_positions[idx1].rpt_group >= 0 { + // already marked as a repetition + continue; + } + let tp_pos = Self::tp_pos(&self.phrase_positions[idx1]); + for idx2 in rpp.iter().skip(i + 1) { + let idx2 = *idx2; + if self.phrase_positions[idx2].rpt_group >= 0 || // already marked as a repetition + self.phrase_positions[idx2].offset == self.phrase_positions[idx1].offset || // not a repetition: two PPs are originally in same offset in the query! + Self::tp_pos(&self.phrase_positions[idx2]) != tp_pos + { + continue; + } + // a repetition + let mut g = self.phrase_positions[idx1].rpt_group; + if g < 0 { + g = res.len() as i32; + self.phrase_positions[idx1].rpt_group = g; + let mut rl = Vec::with_capacity(2); + rl.push(idx1); + res.push(rl); + } + self.phrase_positions[idx2].rpt_group = g; + res[g as usize].push(idx2); + } + } + } else { + // more involved - has multi-terms + let mut tmp = Vec::new(); + let mut bb = self.pp_terms_bit_sets(&rpp, rpt_terms); + Self::union_term_groups(&mut bb); + let tg = self.term_groups(terms, &bb); + let mut distinct_group_ids = HashSet::new(); + for v in tg.values() { + distinct_group_ids.insert(*v); + } + for _ in 0..distinct_group_ids.len() { + tmp.push(HashSet::new()); + } + for pp_idx in &rpp { + for term_idx in 0..self.phrase_positions[*pp_idx].terms.len() { + if rpt_terms.contains_key(&self.phrase_positions[*pp_idx].terms[term_idx]) { + let g = tg[&self.phrase_positions[*pp_idx].terms[term_idx]]; + tmp[g].insert(*pp_idx); + assert!( + self.phrase_positions[*pp_idx].rpt_group == -1 + || self.phrase_positions[*pp_idx].rpt_group == g as i32 + ); + self.phrase_positions[*pp_idx].rpt_group = g as i32; + } + } + } + for hs in &tmp { + let mut data = Vec::with_capacity(hs.len()); + for v in hs { + data.push(*v); + } + res.push(data); + } + } + res + } + + /// Actual position in doc of a PhrasePosition, relies on that position = tpPos - offset) + fn tp_pos(pp: &PhrasePositions) -> i32 { + pp.position + pp.offset + } + + /// find repeating terms and assign them ordinal values + fn repeating_terms(&self) -> (HashMap, Vec) { + let mut tord = HashMap::new(); + let mut terms = Vec::new(); + let mut tcnt = HashMap::new(); + for pp in &self.phrase_positions { + for t in &pp.terms { + let cnt = if let Some(v) = tcnt.get(t) { + *v + 1usize + } else { + 1usize + }; + tcnt.insert(t.clone(), cnt); + if cnt == 2 { + let ord = tord.len(); + tord.insert(t.clone(), ord); + terms.push(t.clone()); + } + } + } + (tord, terms) + } + + /// find repeating pps, and for each, if has multi-terms, update this.hasMultiTermRpts + fn repeating_pps(&mut self, rpt_terms: &HashMap) -> Vec { + let mut rp = Vec::new(); + let mut has_multi_term_rpts = false; + for (idx, pp) in self.phrase_positions.iter().enumerate() { + for t in &pp.terms { + if rpt_terms.contains_key(t) { + rp.push(idx); + has_multi_term_rpts |= pp.terms.len() > 1; + break; + } + } + } + self.has_multi_term_rpts |= has_multi_term_rpts; + rp + } + + /// bit-sets - for each repeating pp, for each of its repeating terms, the term ordinal values + /// is set + fn pp_terms_bit_sets( + &self, + rpp_idx: &[usize], + tord: &HashMap, + ) -> Vec { + let mut bb = Vec::with_capacity(rpp_idx.len()); + for idx in rpp_idx { + let mut b = FixedBitSet::new(tord.len()); + for t in &self.phrase_positions[*idx].terms { + if let Some(ord) = tord.get(t) { + b.set(*ord); + } + } + bb.push(b); + } + bb + } + + /// union (term group) bit-sets until they are disjoint (O(n^^2)), + /// and each group have different terms + fn union_term_groups(bb: &mut Vec) { + let mut incr; + let mut i = 0; + while i < bb.len() - 1 { + incr = 1; + let mut j = i + 1; + while j < bb.len() { + if bb[i].intersects(&bb[j]) { + let bbj = bb.remove(j); + bb[i].set_or(&bbj); + incr = 0; + } else { + j += 1; + } + } + i += incr; + } + } + + /// map each term to the single group that contains it + fn term_groups(&self, terms: &[Term], bb: &[FixedBitSet]) -> HashMap { + let mut tg = HashMap::new(); + for (i, bit) in bb.iter().enumerate() { + let mut ord = bit.next_set_bit(0); + while ord != NO_MORE_DOCS { + tg.insert(terms[ord as usize].clone(), i); + ord = if ord as usize >= bit.len() { + NO_MORE_DOCS + } else { + bit.next_set_bit((ord + 1) as usize) + }; + } + } + tg + } +} + +unsafe impl Send for SloppyPhraseScorer {} + +impl Scorer for SloppyPhraseScorer { + fn score(&mut self) -> Result { + let doc_id = self.doc_id(); + self.doc_scorer.score(doc_id, self.sloppy_freq) + } +} + +impl DocIterator for SloppyPhraseScorer { + fn doc_id(&self) -> i32 { + self.conjunction.doc_id() + } + + fn next(&mut self) -> Result { + self.approximate_next()?; + two_phase_next(self) + } + + fn advance(&mut self, target: i32) -> Result { + self.approximate_advance(target)?; + two_phase_next(self) + } + + fn cost(&self) -> usize { + self.conjunction.cost() + } + + fn matches(&mut self) -> Result { + let sloppy_freq = self.phrase_freq()?; // check for phrase + self.sloppy_freq = sloppy_freq; + Ok(sloppy_freq > f32::EPSILON) + } + + fn match_cost(&self) -> f32 { + self.match_cost + } + + fn support_two_phase(&self) -> bool { + true + } + + fn approximate_next(&mut self) -> Result { + self.conjunction.next() + } + + fn approximate_advance(&mut self, target: i32) -> Result { + self.conjunction.advance(target) + } +} \ No newline at end of file diff --git a/src/core/util/packed/elias_fano_decoder.rs b/src/core/util/packed/elias_fano_decoder.rs index a056b17..e6ee9c5 100644 --- a/src/core/util/packed/elias_fano_decoder.rs +++ b/src/core/util/packed/elias_fano_decoder.rs @@ -1,3 +1,16 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + use core::util::bit_util::{self, UnsignedShift, LOG2_LONG_SIZE, LONG_SIZE, LONG_SIZE_32}; use core::util::packed::EliasFanoEncoder; use error::ErrorKind::*; diff --git a/src/core/util/packed/elias_fano_encoder.rs b/src/core/util/packed/elias_fano_encoder.rs index f595edb..f7a0e58 100644 --- a/src/core/util/packed/elias_fano_encoder.rs +++ b/src/core/util/packed/elias_fano_encoder.rs @@ -1,3 +1,16 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + use core::codec::postings::{EncodeType, ForUtil}; use core::store::io::{IndexInput, IndexOutput}; use core::util::bit_util::*; From b01ae344aec88412777fbeb377775fb1242213b6 Mon Sep 17 00:00:00 2001 From: wangjingjing Date: Tue, 9 Jun 2020 14:04:44 +0800 Subject: [PATCH 30/49] bugfix for delete by query --- src/core/index/writer/bufferd_updates.rs | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/src/core/index/writer/bufferd_updates.rs b/src/core/index/writer/bufferd_updates.rs index 331f53f..67a2fc6 100644 --- a/src/core/index/writer/bufferd_updates.rs +++ b/src/core/index/writer/bufferd_updates.rs @@ -186,6 +186,7 @@ pub struct FrozenBufferedUpdates { query_and_limits: Vec<(Arc>, DocId)>, doc_values_updates: HashMap>>, pub num_term_deletes: usize, + pub num_query_deletes: usize, pub num_updates: usize, pub gen: u64, // assigned by BufferedUpdatesStream once pushed @@ -241,6 +242,8 @@ impl FrozenBufferedUpdates { } deletes.doc_values_updates.clear(); + let num_query_deletes = query_and_limits.len(); + // TODO if a Term affects multiple fields, we could keep the updates key'd by Term // so that it maps to all fields it affects, sorted by their docUpto, and traverse // that Term only once, applying the update to all fields that still need to be @@ -250,6 +253,7 @@ impl FrozenBufferedUpdates { query_and_limits, doc_values_updates: dv_updates, num_term_deletes: deletes.num_term_deletes.load(Ordering::Acquire), + num_query_deletes, num_updates, gen: u64::max_value(), // used as a sentinel of invalid @@ -294,6 +298,7 @@ pub struct BufferedUpdatesStream { // used only by assert: last_delete_term: Vec, num_terms: AtomicUsize, + num_queries: AtomicUsize, num_updates: AtomicUsize, } @@ -305,6 +310,7 @@ impl Default for BufferedUpdatesStream { next_gen: AtomicU64::new(1), last_delete_term: Vec::with_capacity(0), num_terms: AtomicUsize::new(0), + num_queries: AtomicUsize::new(0), num_updates: AtomicUsize::new(0), } } @@ -328,6 +334,8 @@ impl BufferedUpdatesStream { let del_gen = packet.gen; self.num_terms .fetch_add(packet.num_term_deletes, Ordering::AcqRel); + self.num_queries + .fetch_add(packet.num_query_deletes, Ordering::AcqRel); self.num_updates .fetch_add(packet.num_updates, Ordering::AcqRel); updates.push(packet); @@ -339,12 +347,15 @@ impl BufferedUpdatesStream { self.updates.lock()?.clear(); self.next_gen.store(1, Ordering::Release); self.num_terms.store(0, Ordering::Release); + self.num_queries.store(0, Ordering::Release); self.num_updates.store(0, Ordering::Release); Ok(()) } pub fn any(&self) -> bool { - self.num_terms.load(Ordering::Acquire) > 0 || self.num_updates.load(Ordering::Acquire) > 0 + self.num_terms.load(Ordering::Acquire) > 0 + || self.num_updates.load(Ordering::Acquire) > 0 + || self.num_queries.load(Ordering::Acquire) > 0 } pub fn num_terms(&self) -> usize { @@ -537,6 +548,7 @@ impl BufferedUpdatesStream { } self.num_updates.store(0, Ordering::Release); + self.num_queries.store(0, Ordering::Release); debug!( "BD - apply_deletes took {:?} for {} segments, {} newly deleted docs (query deletes), \ @@ -919,6 +931,7 @@ impl BufferedUpdatesStream { } } self.do_prune(&mut updates, limit); + self.num_queries.store(0, Ordering::Release); debug_assert!(!self.any()); debug_assert!(self.check_delete_stats(&updates)); } From 5c76188de8a59c57fb8506a915201d94df6b48c7 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Wed, 10 Jun 2020 11:47:24 +0800 Subject: [PATCH 31/49] fix: sort field cmp panic while merging. --- src/core/codec/sorter.rs | 13 ++++++--- src/core/index/merge/merge_scheduler.rs | 8 ++++++ src/core/index/writer/index_writer.rs | 7 +++++ src/core/search/query/phrase_query.rs | 35 ++++++++++++++++--------- src/core/search/scorer/phrase_scorer.rs | 25 +++++++++--------- 5 files changed, 59 insertions(+), 29 deletions(-) diff --git a/src/core/codec/sorter.rs b/src/core/codec/sorter.rs index fc1b59a..d3ff0d7 100644 --- a/src/core/codec/sorter.rs +++ b/src/core/codec/sorter.rs @@ -622,11 +622,16 @@ impl CrossReaderComparator for DoubleCrossReaderComparator { } else { self.missing_value }; - let res = value1.partial_cmp(&value2).unwrap(); - if self.reverse { - Ok(res.reverse()) + + // NaN + if let Some(res) = value1.partial_cmp(&value2) { + if self.reverse { + Ok(res.reverse()) + } else { + Ok(res) + } } else { - Ok(res) + Ok(Ordering::Equal) } } } diff --git a/src/core/index/merge/merge_scheduler.rs b/src/core/index/merge/merge_scheduler.rs index ccf8b54..86d1ac5 100644 --- a/src/core/index/merge/merge_scheduler.rs +++ b/src/core/index/merge/merge_scheduler.rs @@ -51,6 +51,10 @@ pub trait MergeScheduler: Send + Sync + Clone + 'static { MP: MergePolicy; fn close(&self) -> Result<()>; + + fn merging_thread_count(&self) -> Option { + None + } } #[derive(Copy, Clone)] @@ -454,6 +458,10 @@ impl MergeScheduler for ConcurrentMergeScheduler { // IndexWrite live long enough before all the threads finish running Ok(()) } + + fn merging_thread_count(&self) -> Option { + Some(self.inner.merge_thread_count()) + } } fn bytes_to_mb(bytes: u64) -> f64 { diff --git a/src/core/index/writer/index_writer.rs b/src/core/index/writer/index_writer.rs index 91fab96..320009d 100644 --- a/src/core/index/writer/index_writer.rs +++ b/src/core/index/writer/index_writer.rs @@ -3176,6 +3176,13 @@ where // take very long because they periodically check if // they are aborted. while !self.running_merges.is_empty() { + // in case merge panic. + if let Some(thread_count) = self.merge_scheduler.merging_thread_count() { + if thread_count == 0 { + break; + } + } + let (loc, _) = self.cond.wait_timeout(lock, Duration::from_millis(1000))?; warn!( "IW - abort merges, waiting for running_merges to be empty, current size: {}", diff --git a/src/core/search/query/phrase_query.rs b/src/core/search/query/phrase_query.rs index cce80ba..6650b49 100644 --- a/src/core/search/query/phrase_query.rs +++ b/src/core/search/query/phrase_query.rs @@ -14,18 +14,18 @@ use std::boxed::Box; use std::fmt; +use core::codec::Codec; use core::codec::PostingIteratorFlags; use core::codec::{TermIterator, Terms}; -use core::codec::Codec; use core::doc::Term; use core::index::reader::LeafReaderContext; -use core::search::DocIterator; use core::search::explanation::Explanation; use core::search::query::{Query, TermQuery, Weight}; use core::search::scorer::{ExactPhraseScorer, PostingsAndFreq, Scorer, SloppyPhraseScorer}; use core::search::searcher::SearchPlanBuilder; -use core::search::similarity::{Similarity, SimWeight}; +use core::search::similarity::{SimWeight, Similarity}; use core::search::statistics::{CollectionStatistics, TermStatistics}; +use core::search::DocIterator; use core::util::{DocId, KeyedContext}; use error::{ErrorKind, Result}; @@ -493,7 +493,8 @@ mod tests { fn read(&mut self, buf: &mut [u8]) -> io::Result { let remain = buf.len().min(self.text.len() - self.index); if remain > 0 { - buf[..remain].copy_from_slice(&self.text.as_bytes()[self.index..self.index + remain]); + buf[..remain] + .copy_from_slice(&self.text.as_bytes()[self.index..self.index + remain]); self.index += remain; } Ok(remain) @@ -557,8 +558,10 @@ mod tests { ], vec![0, 1], 0, - None, None, - ).unwrap(); + None, + None, + ) + .unwrap(); let mut collector = TopDocsCollector::new(10); index_searcher.search(&query, &mut collector).unwrap(); let top_docs = collector.top_docs(); @@ -574,8 +577,10 @@ mod tests { ], vec![0, 1], 1, - None, None, - ).unwrap(); + None, + None, + ) + .unwrap(); let mut collector = TopDocsCollector::new(10); index_searcher.search(&query, &mut collector).unwrap(); let top_docs = collector.top_docs(); @@ -591,8 +596,10 @@ mod tests { ], vec![0, 1], 2, - None, None, - ).unwrap(); + None, + None, + ) + .unwrap(); let mut collector = TopDocsCollector::new(10); index_searcher.search(&query, &mut collector).unwrap(); let top_docs = collector.top_docs(); @@ -608,12 +615,14 @@ mod tests { ], vec![0, 1], 3, - None, None, - ).unwrap(); + None, + None, + ) + .unwrap(); let mut collector = TopDocsCollector::new(10); index_searcher.search(&query, &mut collector).unwrap(); let top_docs = collector.top_docs(); assert_eq!(top_docs.total_hits(), 3); } } -} \ No newline at end of file +} diff --git a/src/core/search/scorer/phrase_scorer.rs b/src/core/search/scorer/phrase_scorer.rs index 95e7a81..d8f8bbe 100644 --- a/src/core/search/scorer/phrase_scorer.rs +++ b/src/core/search/scorer/phrase_scorer.rs @@ -19,10 +19,10 @@ use std::rc::Rc; use core::codec::PostingIterator; use core::doc::Term; -use core::search::{DocIterator, NO_MORE_DOCS}; -use core::search::scorer::{ConjunctionScorer, Scorer, two_phase_next}; +use core::search::scorer::{two_phase_next, ConjunctionScorer, Scorer}; use core::search::similarity::SimScorer; -use core::util::{Bits, BitSet, DocId, FixedBitSet, ImmutableBitSet}; +use core::search::{DocIterator, NO_MORE_DOCS}; +use core::util::{BitSet, Bits, DocId, FixedBitSet, ImmutableBitSet}; use error::Result; // a fake scorer struct used for `ConjunctionScorer` @@ -56,7 +56,6 @@ impl DocIterator for PostingsIterAsScorer { } } - pub struct PostingsAndFreq { pub postings: T, pub pos: i32, @@ -137,15 +136,15 @@ impl ExactPhraseScorer { match_cost: f32, ) -> Self { let mut iterators = Vec::with_capacity(postings.len()); - let mut postings_and_positions: Vec> = Vec::with_capacity(postings.len()); + let mut postings_and_positions: Vec> = + Vec::with_capacity(postings.len()); for (_, posting) in postings.into_iter().enumerate() { let iterator = Rc::new(RefCell::new(posting.postings)); - iterators.push(PostingsIterAsScorer { iterator: iterator.clone() }); - postings_and_positions.push(PostingsAndPosition::new( - iterator.clone(), - posting.pos, - )); + iterators.push(PostingsIterAsScorer { + iterator: iterator.clone(), + }); + postings_and_positions.push(PostingsAndPosition::new(iterator.clone(), posting.pos)); } let conjunction = ConjunctionScorer::new(iterators); @@ -488,7 +487,9 @@ impl SloppyPhraseScorer { idx as i32, posting.terms.clone(), )); - doc_iterators.push(PostingsIterAsScorer { iterator: iterator.clone() }); + doc_iterators.push(PostingsIterAsScorer { + iterator: iterator.clone(), + }); } let conjunction = ConjunctionScorer::new(doc_iterators); @@ -1067,4 +1068,4 @@ impl DocIterator for SloppyPhraseScorer { fn approximate_advance(&mut self, target: i32) -> Result { self.conjunction.advance(target) } -} \ No newline at end of file +} From 6a210da86475a3801e5a7775e8b980bf0601569c Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Fri, 12 Jun 2020 10:17:02 +0800 Subject: [PATCH 32/49] fix read payload --- src/core/codec/postings/posting_reader.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/core/codec/postings/posting_reader.rs b/src/core/codec/postings/posting_reader.rs index 5b3dfc3..fe1638d 100644 --- a/src/core/codec/postings/posting_reader.rs +++ b/src/core/codec/postings/posting_reader.rs @@ -1896,6 +1896,7 @@ impl<'a> EverythingIterator { let count = (self.total_term_freq % i64::from(BLOCK_SIZE)) as usize; let mut payload_length = 0i32; let mut offset_length = 0i32; + self.payload_byte_upto = 0; for i in 0..count { let code = pos_in.read_vint()?; if self.index_has_payloads { From 52ab3000d84f03b123f9cb9f8da133d0a4519060 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Fri, 19 Jun 2020 12:06:29 +0800 Subject: [PATCH 33/49] point range query --- src/core/index/merge/merge_policy.rs | 2 +- src/core/search/query/point_range_query.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/core/index/merge/merge_policy.rs b/src/core/index/merge/merge_policy.rs index 677b612..2ec3fcc 100644 --- a/src/core/index/merge/merge_policy.rs +++ b/src/core/index/merge/merge_policy.rs @@ -574,7 +574,7 @@ impl TieredMergePolicy { } } - let min_segment = (self.segs_per_tier * 0.5 + 1.0) as i64; + let min_segment = self.segs_per_tier as i64; let reserved_min = if next_idx as i64 >= min_segment { self.max_merged_segment_bytes as i64 } else { diff --git a/src/core/search/query/point_range_query.rs b/src/core/search/query/point_range_query.rs index 58417ce..acfabaf 100644 --- a/src/core/search/query/point_range_query.rs +++ b/src/core/search/query/point_range_query.rs @@ -426,7 +426,7 @@ impl Query for PointRangeQuery { } fn extract_terms(&self) -> Vec { - unimplemented!() + vec![] } fn as_any(&self) -> &dyn (::std::any::Any) { From 991fe410c600d6a4131f385f19eb15ea0459988b Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Sun, 28 Jun 2020 13:15:48 +0800 Subject: [PATCH 34/49] fix reader use after dropped --- src/core/index/writer/index_writer.rs | 35 ++++++++++++--------------- 1 file changed, 15 insertions(+), 20 deletions(-) diff --git a/src/core/index/writer/index_writer.rs b/src/core/index/writer/index_writer.rs index 320009d..5182f57 100644 --- a/src/core/index/writer/index_writer.rs +++ b/src/core/index/writer/index_writer.rs @@ -3610,34 +3610,29 @@ where let mut res = Ok(()); for reader in &merge.readers { - let rld = self.reader_pool.get(reader.si.as_ref()); - // We still hold a ref so it should not have been removed: - debug_assert!(rld.is_some()); - let rld = rld.unwrap(); - if drop { - rld.drop_changes(); - } else { - rld.drop_merging_updates(); - } - - let mut res_drop = self.reader_pool.release(&rld); - if res_drop.is_ok() { + if let Some(rld) = self.reader_pool.get(reader.si.as_ref()) { if drop { - res_drop = self.reader_pool.drop(rld.info.as_ref()); + rld.drop_changes(); + } else { + rld.drop_merging_updates(); } - } - if let Err(e) = res_drop { - if res.is_ok() { - res = Err(e); + let mut res_drop = self.reader_pool.release(&rld); + if res_drop.is_ok() { + if drop { + res_drop = self.reader_pool.drop(rld.info.as_ref()); + } + } + + if let Err(e) = res_drop { + if res.is_ok() { + res = Err(e); + } } } - // merge.readers[i] = None; } merge.readers.clear(); - // merge.merge_finished(); - if !suppress_errors { return res; } From efee24c5ab3373425fd545acd99ab85509957977 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Fri, 3 Jul 2020 16:24:39 +0800 Subject: [PATCH 35/49] set_segs_per_tier --- src/core/index/merge/merge_policy.rs | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/core/index/merge/merge_policy.rs b/src/core/index/merge/merge_policy.rs index 2ec3fcc..ac5effe 100644 --- a/src/core/index/merge/merge_policy.rs +++ b/src/core/index/merge/merge_policy.rs @@ -462,6 +462,17 @@ impl TieredMergePolicy { Ok(()) } + pub fn set_segs_per_tier(&mut self, v: f64) -> Result<()> { + if v < 0.0 { + bail!(IllegalArgument(format!( + "segs_per_tier must be > 0, got {}", + v + ))); + } + self.segs_per_tier = v; + Ok(()) + } + pub fn set_max_merged_segment_mb(&mut self, mut v: f64) -> Result<()> { if v < 0.0 { bail!(IllegalArgument(format!( From 5982696b80f15980cf93161717c932e1a3b25161 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Wed, 8 Jul 2020 18:09:22 +0800 Subject: [PATCH 36/49] let delete err go --- src/core/store/directory/fs_directory.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/core/store/directory/fs_directory.rs b/src/core/store/directory/fs_directory.rs index 64e370b..1eaeb56 100644 --- a/src/core/store/directory/fs_directory.rs +++ b/src/core/store/directory/fs_directory.rs @@ -67,7 +67,9 @@ impl FSDirectory { let mut deleted_set = BTreeSet::new(); for name in pending_deletes.iter() { let path = dir.join(name); - fs::remove_file(path)?; + if let Err(e) = fs::remove_file(path.clone()) { + info!("delete_pending_files {:?} failed. {}", path, e.to_string()); + } deleted_set.insert(name.clone()); } for name in deleted_set { From 21001ae44f4eed6aefc0ad0365239076d17dc966 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Mon, 13 Jul 2020 12:00:51 +0800 Subject: [PATCH 37/49] limit bytes size to BYTE_BLOCK_SIZE --- src/core/doc/field.rs | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/src/core/doc/field.rs b/src/core/doc/field.rs index 61ea8ba..a98b11d 100644 --- a/src/core/doc/field.rs +++ b/src/core/doc/field.rs @@ -13,7 +13,7 @@ use core::analysis::{BinaryTokenStream, StringTokenStream, TokenStream}; use core::doc::{DocValuesType, IndexOptions}; -use core::util::{BytesRef, Numeric, VariantValue}; +use core::util::{ByteBlockPool, BytesRef, Numeric, VariantValue}; use error::ErrorKind::IllegalArgument; use error::{ErrorKind, Result}; @@ -35,6 +35,22 @@ impl Field { field_data: Option, token_stream: Option>, ) -> Field { + let field_data = if let Some(data) = field_data { + match data { + VariantValue::Binary(b) => { + let bytes = b[..(ByteBlockPool::BYTE_BLOCK_SIZE - 2).min(b.len())].to_vec(); + Some(VariantValue::Binary(bytes)) + } + VariantValue::VString(vs) => { + let s = (&vs[..(ByteBlockPool::BYTE_BLOCK_SIZE - 2).min(vs.len())]).to_string(); + Some(VariantValue::VString(s)) + } + _ => Some(data), + } + } else { + None + }; + Field { field_type, field_name, @@ -45,6 +61,7 @@ impl Field { } pub fn new_bytes(field_name: String, bytes: Vec, field_type: FieldType) -> Self { + let bytes = bytes[..(ByteBlockPool::BYTE_BLOCK_SIZE - 2).min(bytes.len())].to_vec(); Field { field_name, field_data: Some(VariantValue::Binary(bytes)), From 8e5d29db7482b452d08a6651421be1832630820e Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Wed, 22 Jul 2020 17:18:39 +0800 Subject: [PATCH 38/49] rm postings limit for dmp --- src/core/codec/postings/mod.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/core/codec/postings/mod.rs b/src/core/codec/postings/mod.rs index 1ec3c40..88c3706 100644 --- a/src/core/codec/postings/mod.rs +++ b/src/core/codec/postings/mod.rs @@ -76,7 +76,9 @@ use error::Result; use std::sync::Arc; // sometimes will cause miss increasing with phrase/highlight. -pub const DEFAULT_SEGMENT_DOC_FREQ: i32 = 500_000; +// pub const DEFAULT_SEGMENT_DOC_FREQ: i32 = 500_000; +// for dmp +pub const DEFAULT_SEGMENT_DOC_FREQ: i32 = 1_000_000_000; pub const DEFAULT_DOC_TERM_FREQ: i32 = 10; /// Encodes/decodes terms, postings, and proximity data. From 7a09834f842ec14e8fdf2f192a655486a0258143 Mon Sep 17 00:00:00 2001 From: wangjingjing Date: Mon, 27 Jul 2020 19:57:16 +0800 Subject: [PATCH 39/49] bugfix for keyword field truncate --- src/core/doc/field.rs | 6 +++++- src/core/util/bytes_ref_hash.rs | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/core/doc/field.rs b/src/core/doc/field.rs index a98b11d..97b566f 100644 --- a/src/core/doc/field.rs +++ b/src/core/doc/field.rs @@ -42,7 +42,11 @@ impl Field { Some(VariantValue::Binary(bytes)) } VariantValue::VString(vs) => { - let s = (&vs[..(ByteBlockPool::BYTE_BLOCK_SIZE - 2).min(vs.len())]).to_string(); + let mut index = (ByteBlockPool::BYTE_BLOCK_SIZE - 2).min(vs.len()); + while !vs.is_char_boundary(index) { + index -= 1; + } + let s = (&vs[..index]).to_string(); Some(VariantValue::VString(s)) } _ => Some(data), diff --git a/src/core/util/bytes_ref_hash.rs b/src/core/util/bytes_ref_hash.rs index 801c44a..e409420 100644 --- a/src/core/util/bytes_ref_hash.rs +++ b/src/core/util/bytes_ref_hash.rs @@ -179,7 +179,7 @@ impl BytesRefHash { // new entry let len2 = 2 + bytes.len(); if len2 + self.byte_pool().byte_upto > ByteBlockPool::BYTE_BLOCK_SIZE { - assert!(len2 < ByteBlockPool::BYTE_BLOCK_SIZE); + assert!(len2 <= ByteBlockPool::BYTE_BLOCK_SIZE); // this length check already done in the caller func // if len2 > BYTE_BLOCK_SIZE { // bail!("bytes can be at most: {}, got: {}", BYTE_BLOCK_SIZE - From 8132ace63b90692d3f5647c9f329e4aac3e9cfa6 Mon Sep 17 00:00:00 2001 From: wangjingjing Date: Mon, 3 Aug 2020 11:02:58 +0800 Subject: [PATCH 40/49] fix for none local thread producer when merge dv field --- src/core/index/reader/segment_reader.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/core/index/reader/segment_reader.rs b/src/core/index/reader/segment_reader.rs index 61bae47..42689af 100644 --- a/src/core/index/reader/segment_reader.rs +++ b/src/core/index/reader/segment_reader.rs @@ -1148,6 +1148,7 @@ where } fn doc_values_reader(&self) -> Result>> { + self.init_local_doc_values_producer()?; Ok(self.doc_values_producer.get().map(Arc::clone)) } From c5d9e03b3a1aadd16dece863822c955a52f88182 Mon Sep 17 00:00:00 2001 From: wangjingjing Date: Mon, 3 Aug 2020 15:18:56 +0800 Subject: [PATCH 41/49] bugfix for sorted_numeric, fill numerics --- .../codec/doc_values/lucene54/lucene54_doc_values_producer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/core/codec/doc_values/lucene54/lucene54_doc_values_producer.rs b/src/core/codec/doc_values/lucene54/lucene54_doc_values_producer.rs index a0ce196..9809f20 100644 --- a/src/core/codec/doc_values/lucene54/lucene54_doc_values_producer.rs +++ b/src/core/codec/doc_values/lucene54/lucene54_doc_values_producer.rs @@ -279,7 +279,7 @@ impl Lucene54DocValuesProducer { )?; match entry { - Some(n) => ords.insert(info.name.clone(), n), + Some(n) => numerics.insert(info.name.clone(), n), _ => unreachable!(), }; From c2ee77cb9830b9db07ae988850fc675c6939b93a Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Mon, 10 Aug 2020 13:31:04 +0800 Subject: [PATCH 42/49] use score feature --- src/core/search/scorer/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/core/search/scorer/mod.rs b/src/core/search/scorer/mod.rs index 525b5ca..9a1751b 100644 --- a/src/core/search/scorer/mod.rs +++ b/src/core/search/scorer/mod.rs @@ -94,7 +94,7 @@ pub trait Scorer: DocIterator { } fn score_feature(&mut self) -> Result> { - unimplemented!() + Ok(vec![]) } } From cee9ad56d0438a1de06ca8bacb0f7893e9bc3baf Mon Sep 17 00:00:00 2001 From: wangjingjing Date: Wed, 19 Aug 2020 10:31:25 +0800 Subject: [PATCH 43/49] trans utf8 binary to string --- src/core/util/variant_value.rs | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/core/util/variant_value.rs b/src/core/util/variant_value.rs index 45e435a..0cb0c93 100644 --- a/src/core/util/variant_value.rs +++ b/src/core/util/variant_value.rs @@ -101,6 +101,22 @@ impl VariantValue { } } + pub fn get_utf8_string(&self) -> Option { + match self { + VariantValue::VString(s) => Some(s.clone()), + VariantValue::Binary(b) => { + if let Ok(s) = String::from_utf8(b.clone()) { + Some(s) + } else { + None + } + } + _ => { + None + } + } + } + // used for index sort check pub fn is_zero(&self) -> bool { match self { From 23b074e837fb427f158f6104eff01c416a444eb7 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Tue, 1 Sep 2020 16:45:06 +0800 Subject: [PATCH 44/49] fix memleak by multiple reopens of fields_producer --- .../postings/per_field_postings_format.rs | 30 +++++++++++++------ src/core/util/variant_value.rs | 4 +-- 2 files changed, 22 insertions(+), 12 deletions(-) diff --git a/src/core/codec/postings/per_field_postings_format.rs b/src/core/codec/postings/per_field_postings_format.rs index 6e0204d..ea4b732 100644 --- a/src/core/codec/postings/per_field_postings_format.rs +++ b/src/core/codec/postings/per_field_postings_format.rs @@ -87,7 +87,8 @@ impl PostingsFormat for PerFieldPostingsFormat { } pub struct PerFieldFieldsReader { - fields: BTreeMap, + _formats: HashMap>, + fields: BTreeMap>, segment: String, } @@ -96,7 +97,7 @@ impl PerFieldFieldsReader { state: &SegmentReadState<'_, D, DW, C>, ) -> Result { let mut fields = BTreeMap::new(); - let mut formats = HashMap::new(); + let mut _formats = HashMap::new(); for (name, info) in &state.field_infos.by_name { if let IndexOptions::Null = info.index_options { continue; @@ -113,13 +114,20 @@ impl PerFieldFieldsReader { .unwrap() .get(PER_FIELD_POSTING_SUFFIX_KEY) { - if !formats.contains_key(suffix) { - formats.insert(suffix.clone(), postings_format_for_name(format)?); + let segment_suffix = get_suffix(&format, suffix); + + if !_formats.contains_key(&segment_suffix) { + let postings_format = postings_format_for_name(format)?; + let state = SegmentReadState::with_suffix(state, &segment_suffix); + _formats.insert( + segment_suffix.clone(), + Arc::new(postings_format.fields_producer(&state)?), + ); + } + + if let Some(field_producer) = _formats.get(&segment_suffix) { + fields.insert(name.clone(), field_producer.clone()); } - let postings_format = formats.get(suffix).unwrap(); - let suffix = get_suffix(format, suffix); - let state = SegmentReadState::with_suffix(state, &suffix); - fields.insert(name.clone(), postings_format.fields_producer(&state)?); } else { bail!( "Illegal State: missing attribute: {} for field {}", @@ -130,7 +138,11 @@ impl PerFieldFieldsReader { } } let segment = state.segment_info.name.clone(); - Ok(PerFieldFieldsReader { fields, segment }) + Ok(PerFieldFieldsReader { + _formats, + fields, + segment, + }) } fn terms_impl(&self, field: &str) -> Result> { diff --git a/src/core/util/variant_value.rs b/src/core/util/variant_value.rs index 0cb0c93..d22222d 100644 --- a/src/core/util/variant_value.rs +++ b/src/core/util/variant_value.rs @@ -111,9 +111,7 @@ impl VariantValue { None } } - _ => { - None - } + _ => None, } } From 60f21c37c54eafc51fe84a6e00660e5e516cd7fe Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Mon, 7 Sep 2020 11:54:10 +0800 Subject: [PATCH 45/49] fix memleak by threadlocal clear --- .../codec/doc_values/doc_values_format.rs | 53 ++++++++----------- src/core/codec/field_infos/mod.rs | 10 ++-- .../postings/blocktree/blocktree_reader.rs | 10 ++-- .../postings/per_field_postings_format.rs | 31 ++++------- src/core/index/reader/segment_reader.rs | 18 +++++-- 5 files changed, 54 insertions(+), 68 deletions(-) diff --git a/src/core/codec/doc_values/doc_values_format.rs b/src/core/codec/doc_values/doc_values_format.rs index acdfea7..417c254 100644 --- a/src/core/codec/doc_values/doc_values_format.rs +++ b/src/core/codec/doc_values/doc_values_format.rs @@ -17,8 +17,8 @@ use core::codec::doc_values::{ SortedSetDocValues, }; use core::codec::doc_values::{ - BinaryDocValuesProvider, DocValuesProducer, DocValuesProducerRef, NumericDocValuesProvider, - SortedDocValuesProvider, SortedNumericDocValuesProvider, SortedSetDocValuesProvider, + BinaryDocValuesProvider, DocValuesProducer, NumericDocValuesProvider, SortedDocValuesProvider, + SortedNumericDocValuesProvider, SortedSetDocValuesProvider, }; use core::codec::field_infos::FieldInfo; use core::codec::segment_infos::{SegmentReadState, SegmentWriteState}; @@ -30,7 +30,6 @@ use core::util::{BitsMut, BytesRef, Numeric, ReusableIterator}; use error::ErrorKind::{IllegalArgument, IllegalState}; use error::Result; -use std::collections::hash_map::Entry as HashMapEntry; use std::collections::{BTreeMap, HashMap}; use std::mem; use std::sync::Arc; @@ -310,7 +309,7 @@ pub fn doc_values_format_for_name(format: &str) -> Result { } pub struct DocValuesFieldsReader { - fields: BTreeMap, + fields: BTreeMap>, } impl DocValuesFieldsReader { @@ -321,40 +320,34 @@ impl DocValuesFieldsReader { C: Codec, { let mut fields = BTreeMap::new(); - let mut formats = HashMap::new(); + let mut formats: HashMap> = HashMap::new(); for (name, info) in &state.field_infos.by_name { if info.doc_values_type == DocValuesType::Null { continue; } let attrs = info.attributes.read().unwrap(); if let Some(format) = attrs.get(PER_FIELD_VALUE_FORMAT_KEY) { - let suffix = attrs.get(PER_FIELD_VALUE_SUFFIX_KEY); - match suffix { - None => bail!(IllegalState(format!( - "Missing attribute {} for field: {}", - PER_FIELD_VALUE_SUFFIX_KEY, name - ))), - Some(suffix) => { + if let Some(suffix) = attrs.get(PER_FIELD_VALUE_SUFFIX_KEY) { + let segment_suffix = + get_full_segment_suffix(&state.segment_suffix, get_suffix(format, suffix)); + + if !formats.contains_key(&segment_suffix) { let dv_format = doc_values_format_for_name(format)?; - let segment_suffix = get_full_segment_suffix( - &state.segment_suffix, - get_suffix(format, suffix), - ); - - match formats.entry(segment_suffix.clone()) { - HashMapEntry::Occupied(occupied) => { - fields.insert(name.to_string(), Arc::clone(occupied.get())); - } - HashMapEntry::Vacant(vacant) => { - let segment_read_state = - SegmentReadState::with_suffix(state, &segment_suffix); - let dv_producer = dv_format.fields_producer(&segment_read_state)?; - let dv_producer = Arc::from(dv_producer); - vacant.insert(Arc::clone(&dv_producer)); - fields.insert(name.to_string(), dv_producer); - } - } + let segment_read_state = + SegmentReadState::with_suffix(state, &segment_suffix); + let dv_producer = dv_format.fields_producer(&segment_read_state)?; + let dv_producer = Arc::from(dv_producer); + formats.insert(segment_suffix.clone(), Arc::clone(&dv_producer)); + } + + if let Some(dv_producer) = formats.get(&segment_suffix) { + fields.insert(name.clone(), Arc::clone(dv_producer)); } + } else { + bail!(IllegalState(format!( + "Missing attribute {} for field: {}", + PER_FIELD_VALUE_SUFFIX_KEY, name + ))); } } } diff --git a/src/core/codec/field_infos/mod.rs b/src/core/codec/field_infos/mod.rs index 188e0f1..a5ac2d8 100644 --- a/src/core/codec/field_infos/mod.rs +++ b/src/core/codec/field_infos/mod.rs @@ -307,8 +307,6 @@ impl fmt::Display for FieldInfo { } } -pub type FieldInfoRef = Arc; - /// This class tracks the number and position / offset parameters of terms /// being added to the index. The information collected in this class is /// also used to calculate the normalization factor for a field. @@ -384,8 +382,8 @@ pub struct FieldInfos { pub has_doc_values: bool, pub has_point_values: bool, - pub by_number: BTreeMap, - pub by_name: HashMap, + pub by_number: BTreeMap>, + pub by_name: HashMap>, } impl Serialize for FieldInfos { @@ -434,8 +432,8 @@ impl FieldInfos { let mut has_doc_values = false; let mut has_point_values = false; - let mut by_number: BTreeMap = BTreeMap::new(); - let mut by_name: HashMap = HashMap::new(); + let mut by_number: BTreeMap> = BTreeMap::new(); + let mut by_name: HashMap> = HashMap::new(); let mut max_number = 0; let mut infos = infos; diff --git a/src/core/codec/postings/blocktree/blocktree_reader.rs b/src/core/codec/postings/blocktree/blocktree_reader.rs index 3beda40..3ed1e6e 100644 --- a/src/core/codec/postings/blocktree/blocktree_reader.rs +++ b/src/core/codec/postings/blocktree/blocktree_reader.rs @@ -21,7 +21,7 @@ use std::ops::DerefMut; use std::string::ToString; use std::sync::Arc; -use core::codec::field_infos::{FieldInfo, FieldInfoRef}; +use core::codec::field_infos::FieldInfo; use core::codec::postings::blocktree::{BlockTermState, SegmentTermsIterFrame, MAX_LONGS_SIZE}; use core::codec::postings::{ FieldsProducer, Lucene50PostingIterator, Lucene50PostingsReader, Lucene50PostingsReaderRef, @@ -389,7 +389,7 @@ type FSTRef = Arc>; /// BlockTree's implementation of `Terms` pub struct FieldReader { num_terms: i64, - field_info: FieldInfoRef, + field_info: Arc, sum_total_term_freq: i64, sum_doc_freq: i64, doc_count: i32, @@ -411,7 +411,7 @@ impl FieldReader { #[allow(clippy::too_many_arguments)] pub fn new( parent: BlockTreeTermsReader, - field_info: FieldInfoRef, + field_info: Arc, num_terms: i64, root_code: Vec, sum_total_term_freq: i64, @@ -913,7 +913,7 @@ impl SegmentTermIterator { field_reader: &FieldReader, terms_in: IndexInputRef, postings_reader: Lucene50PostingsReaderRef, - field_info: FieldInfoRef, + field_info: Arc, ) -> Self { let iter = Box::new(SegmentTermIteratorInner::new( field_reader, @@ -1028,7 +1028,7 @@ impl SegmentTermIteratorInner { field_reader: &FieldReader, terms_in: IndexInputRef, postings_reader: Lucene50PostingsReaderRef, - field_info: FieldInfoRef, + field_info: Arc, ) -> Self { let mut arcs = vec![]; if let Some(ref index) = field_reader.index { diff --git a/src/core/codec/postings/per_field_postings_format.rs b/src/core/codec/postings/per_field_postings_format.rs index ea4b732..8d56971 100644 --- a/src/core/codec/postings/per_field_postings_format.rs +++ b/src/core/codec/postings/per_field_postings_format.rs @@ -87,7 +87,6 @@ impl PostingsFormat for PerFieldPostingsFormat { } pub struct PerFieldFieldsReader { - _formats: HashMap>, fields: BTreeMap>, segment: String, } @@ -97,35 +96,27 @@ impl PerFieldFieldsReader { state: &SegmentReadState<'_, D, DW, C>, ) -> Result { let mut fields = BTreeMap::new(); - let mut _formats = HashMap::new(); + let mut formats = HashMap::new(); for (name, info) in &state.field_infos.by_name { if let IndexOptions::Null = info.index_options { continue; } - if let Some(format) = info - .attributes - .read() - .unwrap() - .get(PER_FIELD_POSTING_FORMAT_KEY) - { - if let Some(suffix) = info - .attributes - .read() - .unwrap() - .get(PER_FIELD_POSTING_SUFFIX_KEY) - { + + let attrs = info.attributes.read().unwrap(); + if let Some(format) = attrs.get(PER_FIELD_POSTING_FORMAT_KEY) { + if let Some(suffix) = attrs.get(PER_FIELD_POSTING_SUFFIX_KEY) { let segment_suffix = get_suffix(&format, suffix); - if !_formats.contains_key(&segment_suffix) { + if !formats.contains_key(&segment_suffix) { let postings_format = postings_format_for_name(format)?; let state = SegmentReadState::with_suffix(state, &segment_suffix); - _formats.insert( + formats.insert( segment_suffix.clone(), Arc::new(postings_format.fields_producer(&state)?), ); } - if let Some(field_producer) = _formats.get(&segment_suffix) { + if let Some(field_producer) = formats.get(&segment_suffix) { fields.insert(name.clone(), field_producer.clone()); } } else { @@ -138,11 +129,7 @@ impl PerFieldFieldsReader { } } let segment = state.segment_info.name.clone(); - Ok(PerFieldFieldsReader { - _formats, - fields, - segment, - }) + Ok(PerFieldFieldsReader { fields, segment }) } fn terms_impl(&self, field: &str) -> Result> { diff --git a/src/core/index/reader/segment_reader.rs b/src/core/index/reader/segment_reader.rs index 42689af..b009648 100644 --- a/src/core/index/reader/segment_reader.rs +++ b/src/core/index/reader/segment_reader.rs @@ -489,6 +489,15 @@ pub struct SegmentReader { unsafe impl Sync for SegmentReader {} +impl Drop for SegmentReader { + fn drop(&mut self) { + self.doc_values_producer_preload.write().unwrap().clear(); + self.doc_values_local_preload.write().unwrap().clear(); + self.doc_values_producer.clear(); + self.doc_values_local.clear(); + } +} + /// IndexReader implementation over a single segment. /// Instances pointing to the same segment (but with different deletes, etc) /// may share the same core data. @@ -502,11 +511,7 @@ impl SegmentReader { is_nrt: bool, field_infos: Arc, ) -> SegmentReader { - let doc_values_local = CachedThreadLocal::new(); - doc_values_local.get_or(|| Box::new(RefCell::new(HashMap::new()))); - - let max_preload_num = 5 * num_cpus::get_physical(); - + let max_preload_num = num_cpus::get_physical(); let mut doc_values_producer_preload: Vec> = Vec::with_capacity(max_preload_num); for _ in 0..max_preload_num { @@ -583,6 +588,9 @@ impl SegmentReader { doc_values_producer.get_or(|| Box::new(dv_producer)); } + let doc_values_local = CachedThreadLocal::new(); + doc_values_local.get_or(|| Box::new(RefCell::new(HashMap::new()))); + SegmentReader { si, live_docs, From a76f3de513b125f3aab71f437681273757ba32cc Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Tue, 8 Sep 2020 15:50:41 +0800 Subject: [PATCH 46/49] min_should_match for pure shoulds --- src/core/search/query/boolean_query.rs | 39 +++++++++++------ src/core/search/query/query_string.rs | 28 ++++++------ src/core/search/scorer/disjunction_scorer.rs | 45 ++++++++++++++------ src/core/search/scorer/req_not_scorer.rs | 4 +- src/core/search/scorer/req_opt_scorer.rs | 2 +- 5 files changed, 75 insertions(+), 43 deletions(-) diff --git a/src/core/search/query/boolean_query.rs b/src/core/search/query/boolean_query.rs index 1c83506..5445f1c 100644 --- a/src/core/search/query/boolean_query.rs +++ b/src/core/search/query/boolean_query.rs @@ -31,7 +31,7 @@ pub struct BooleanQuery { should_queries: Vec>>, filter_queries: Vec>>, must_not_queries: Vec>>, - minimum_should_match: i32, + min_should_match: i32, } pub const BOOLEAN: &str = "boolean"; @@ -42,8 +42,18 @@ impl BooleanQuery { shoulds: Vec>>, filters: Vec>>, must_nots: Vec>>, + min_should_match: i32, ) -> Result>> { - let minimum_should_match = if musts.is_empty() { 1 } else { 0 }; + let min_should_match = if min_should_match > 0 { + min_should_match + } else { + if musts.is_empty() { + 1 + } else { + 0 + } + }; + let mut musts = musts; let mut shoulds = shoulds; let mut filters = filters; @@ -72,7 +82,7 @@ impl BooleanQuery { should_queries: shoulds, filter_queries: filters, must_not_queries: must_nots, - minimum_should_match, + min_should_match, })) } @@ -110,6 +120,7 @@ impl Query for BooleanQuery { should_weights, must_not_weights, needs_scores, + self.min_should_match, ))) } @@ -145,7 +156,7 @@ impl fmt::Display for BooleanQuery { write!( f, "BooleanQuery(must: [{}], should: [{}], filters: [{}], must_not: [{}], match: {})", - must_str, should_str, filters_str, must_not_str, self.minimum_should_match + must_str, should_str, filters_str, must_not_str, self.min_should_match ) } } @@ -154,8 +165,7 @@ struct BooleanWeight { must_weights: Vec>>, should_weights: Vec>>, must_not_weights: Vec>>, - #[allow(dead_code)] - minimum_should_match: i32, + min_should_match: i32, needs_scores: bool, } @@ -165,13 +175,13 @@ impl BooleanWeight { shoulds: Vec>>, must_nots: Vec>>, needs_scores: bool, + min_should_match: i32, ) -> BooleanWeight { - let minimum_should_match = if musts.is_empty() { 1 } else { 0 }; BooleanWeight { must_weights: musts, should_weights: shoulds, must_not_weights: must_nots, - minimum_should_match, + min_should_match, needs_scores, } } @@ -217,6 +227,7 @@ impl Weight for BooleanWeight { _ => Some(Box::new(DisjunctionSumScorer::new( scorers, self.needs_scores, + self.min_should_match, ))), } }; @@ -230,7 +241,11 @@ impl Weight for BooleanWeight { match scorers.len() { 0 => None, 1 => Some(scorers.remove(0)), - _ => Some(Box::new(DisjunctionSumScorer::new(scorers, false))), + _ => Some(Box::new(DisjunctionSumScorer::new( + scorers, + false, + self.min_should_match, + ))), } }; @@ -348,13 +363,13 @@ impl Weight for BooleanWeight { "No matching clauses".to_string(), subs, )) - } else if should_match_count < self.minimum_should_match { + } else if should_match_count < self.min_should_match { Ok(Explanation::new( false, 0.0f32, format!( "Failure to match minimum number of optional clauses: {}<{}", - should_match_count, self.minimum_should_match + should_match_count, self.min_should_match ), subs, )) @@ -394,7 +409,7 @@ impl fmt::Display for BooleanWeight { f, "BooleanWeight(must: [{}], should: [{}], must_not: [{}], min match: {}, needs score: \ {})", - must_str, should_str, must_not_str, self.minimum_should_match, self.needs_scores + must_str, should_str, must_not_str, self.min_should_match, self.needs_scores ) } } diff --git a/src/core/search/query/query_string.rs b/src/core/search/query/query_string.rs index c4c696d..a6e6d82 100644 --- a/src/core/search/query/query_string.rs +++ b/src/core/search/query/query_string.rs @@ -29,7 +29,7 @@ pub struct QueryStringQueryBuilder { query_string: String, fields: Vec<(String, f32)>, #[allow(dead_code)] - minimum_should_match: i32, + min_should_match: i32, #[allow(dead_code)] boost: f32, } @@ -38,13 +38,13 @@ impl QueryStringQueryBuilder { pub fn new( query_string: String, fields: Vec<(String, f32)>, - minimum_should_match: i32, + min_should_match: i32, boost: f32, ) -> QueryStringQueryBuilder { QueryStringQueryBuilder { query_string, fields, - minimum_should_match, + min_should_match, boost, } } @@ -171,7 +171,7 @@ impl QueryStringQueryBuilder { shoulds.remove(0) } } else { - BooleanQuery::build(musts, shoulds, vec![], vec![])? + BooleanQuery::build(musts, shoulds, vec![], vec![], self.min_should_match)? }; Ok(Some(query)) } @@ -190,7 +190,7 @@ impl QueryStringQueryBuilder { let res = if queries.len() == 1 { queries.remove(0) } else { - BooleanQuery::build(Vec::new(), queries, vec![], vec![])? + BooleanQuery::build(Vec::new(), queries, vec![], vec![], self.min_should_match)? }; Ok(res) } @@ -259,7 +259,7 @@ mod tests { let term = String::from("test"); let field = String::from("title"); let q: Box> = - QueryStringQueryBuilder::new(term.clone(), vec![(field, 1.0)], 1, 1.0) + QueryStringQueryBuilder::new(term.clone(), vec![(field, 1.0)], 0, 1.0) .build() .unwrap(); let term_str: String = q.to_string(); @@ -271,7 +271,7 @@ mod tests { let term = String::from("(test^0.2 | 测试^2)"); let field = String::from("title"); let q: Box> = - QueryStringQueryBuilder::new(term.clone(), vec![(field, 1.0)], 1, 2.0) + QueryStringQueryBuilder::new(term.clone(), vec![(field, 1.0)], 0, 2.0) .build() .unwrap(); let term_str: String = q.to_string(); @@ -287,7 +287,7 @@ mod tests { let term = String::from("test^0.2 \"测试\"^2"); let field = String::from("title"); let q: Box> = - QueryStringQueryBuilder::new(term.clone(), vec![(field, 1.0)], 1, 2.0) + QueryStringQueryBuilder::new(term.clone(), vec![(field, 1.0)], 0, 2.0) .build() .unwrap(); let term_str: String = q.to_string(); @@ -302,7 +302,7 @@ mod tests { let field = String::from("title"); let q: Box> = - QueryStringQueryBuilder::new(String::from("+test"), vec![(field, 1.0)], 1, 1.0) + QueryStringQueryBuilder::new(String::from("+test"), vec![(field, 1.0)], 0, 1.0) .build() .unwrap(); let term_str: String = q.to_string(); @@ -314,7 +314,7 @@ mod tests { let query_string = String::from("test search"); let field = String::from("title"); let q: Box> = - QueryStringQueryBuilder::new(query_string.clone(), vec![(field, 1.0)], 1, 1.0) + QueryStringQueryBuilder::new(query_string.clone(), vec![(field, 1.0)], 0, 1.0) .build() .unwrap(); let term_str: String = q.to_string(); @@ -330,7 +330,7 @@ mod tests { let query_string = String::from("test +search"); let field = String::from("title"); let q: Box> = - QueryStringQueryBuilder::new(query_string.clone(), vec![(field, 1.0)], 1, 1.0) + QueryStringQueryBuilder::new(query_string.clone(), vec![(field, 1.0)], 0, 1.0) .build() .unwrap(); let term_str: String = q.to_string(); @@ -346,7 +346,7 @@ mod tests { let query_string = String::from("test +(search 搜索)"); let field = String::from("title"); let q: Box> = - QueryStringQueryBuilder::new(query_string.clone(), vec![(field, 1.0)], 1, 1.0) + QueryStringQueryBuilder::new(query_string.clone(), vec![(field, 1.0)], 0, 1.0) .build() .unwrap(); let term_str: String = q.to_string(); @@ -364,7 +364,7 @@ mod tests { let q: Box> = QueryStringQueryBuilder::new( query_string.clone(), vec![("title".to_string(), 1.0), ("content".to_string(), 1.0)], - 1, + 0, 1.0, ) .build() @@ -387,7 +387,7 @@ mod tests { ); let field = String::from("title"); let q: Box> = - QueryStringQueryBuilder::new(query_string.clone(), vec![(field, 1.0)], 1, 1.0) + QueryStringQueryBuilder::new(query_string.clone(), vec![(field, 1.0)], 0, 1.0) .build() .unwrap(); let term_str: String = q.to_string(); diff --git a/src/core/search/scorer/disjunction_scorer.rs b/src/core/search/scorer/disjunction_scorer.rs index 749adbc..6da3644 100644 --- a/src/core/search/scorer/disjunction_scorer.rs +++ b/src/core/search/scorer/disjunction_scorer.rs @@ -23,15 +23,20 @@ pub struct DisjunctionSumScorer { sub_scorers: SubScorers, needs_scores: bool, cost: usize, + min_should_match: i32, } impl DisjunctionSumScorer { - pub fn new(children: Vec, needs_scores: bool) -> DisjunctionSumScorer { + pub fn new( + children: Vec, + needs_scores: bool, + min_should_match: i32, + ) -> DisjunctionSumScorer { assert!(children.len() > 1); let cost = children.iter().map(|w| w.cost()).sum(); - let sub_scorers = if children.len() < 10 { + let sub_scorers = if children.len() < 10 || min_should_match > 1 { SubScorers::SQ(SimpleQueue::new(children)) } else { SubScorers::DPQ(DisiPriorityQueue::new(children)) @@ -41,6 +46,7 @@ impl DisjunctionSumScorer { sub_scorers, needs_scores, cost, + min_should_match, } } } @@ -81,7 +87,8 @@ impl DocIterator for DisjunctionSumScorer { } fn approximate_next(&mut self) -> Result { - self.sub_scorers.approximate_next() + self.sub_scorers + .approximate_next(Some(self.min_should_match)) } fn approximate_advance(&mut self, target: DocId) -> Result { @@ -162,7 +169,7 @@ impl DocIterator for DisjunctionMaxScorer { } fn approximate_next(&mut self) -> Result { - self.sub_scorers.approximate_next() + self.sub_scorers.approximate_next(None) } fn approximate_advance(&mut self, target: DocId) -> Result { @@ -278,23 +285,33 @@ impl SubScorers { } } - fn approximate_next(&mut self) -> Result { + fn approximate_next(&mut self, min_should_match: Option) -> Result { + let min_should_match = min_should_match.unwrap_or(0); + match self { SubScorers::SQ(sq) => { - let curr_doc = sq.curr_doc; - let mut min_doc = NO_MORE_DOCS; - for s in sq.scorers.iter_mut() { - if s.doc_id() == curr_doc { - s.approximate_next()?; + loop { + // curr_doc = current min_doc, (not -1) + let curr_doc = sq.curr_doc; + let mut min_doc = NO_MORE_DOCS; + let mut should_count = 0; + for s in sq.scorers.iter_mut() { + if s.doc_id() == curr_doc { + should_count += 1; + s.approximate_next()?; + } + + min_doc = min_doc.min(s.doc_id()); } - min_doc = min_doc.min(s.doc_id()); + sq.curr_doc = min_doc; + if should_count >= min_should_match || sq.curr_doc == NO_MORE_DOCS { + return Ok(sq.curr_doc); + } } - - sq.curr_doc = min_doc; - Ok(sq.curr_doc) } SubScorers::DPQ(dbq) => { + // reset with -1, @posting_reader.rs#1208 let doc = dbq.peek().doc(); loop { diff --git a/src/core/search/scorer/req_not_scorer.rs b/src/core/search/scorer/req_not_scorer.rs index 6f0c9b1..f86ba7e 100644 --- a/src/core/search/scorer/req_not_scorer.rs +++ b/src/core/search/scorer/req_not_scorer.rs @@ -133,7 +133,7 @@ mod tests { let conjunction_scorer: Box = Box::new(ConjunctionScorer::new(vec![s1, s2])); let disjunction_scorer: Box = - Box::new(DisjunctionSumScorer::new(vec![s3, s4], true)); + Box::new(DisjunctionSumScorer::new(vec![s3, s4], true, 0)); let mut scorer = ReqNotScorer::new(conjunction_scorer, disjunction_scorer); assert_eq!(scorer.doc_id(), -1); @@ -154,7 +154,7 @@ mod tests { let conjunction_scorer: Box = Box::new(ConjunctionScorer::new(vec![s1, s2])); let disjunction_scorer: Box = - Box::new(DisjunctionSumScorer::new(vec![s3, s4], true)); + Box::new(DisjunctionSumScorer::new(vec![s3, s4], true, 0)); let mut scorer = ReqNotScorer::new(conjunction_scorer, disjunction_scorer); // 2, 3, 5, 7, 9 diff --git a/src/core/search/scorer/req_opt_scorer.rs b/src/core/search/scorer/req_opt_scorer.rs index 0b93238..61b65b1 100644 --- a/src/core/search/scorer/req_opt_scorer.rs +++ b/src/core/search/scorer/req_opt_scorer.rs @@ -111,7 +111,7 @@ mod tests { let conjunction_scorer: Box = Box::new(ConjunctionScorer::new(vec![s1, s2])); let disjunction_scorer: Box = - Box::new(DisjunctionSumScorer::new(vec![s3, s4], true)); + Box::new(DisjunctionSumScorer::new(vec![s3, s4], true, 0)); let mut scorer = ReqOptScorer::new(conjunction_scorer, disjunction_scorer); assert_eq!(scorer.doc_id(), -1); From 82c3668f9596ec5d9ec66a1e2f8d86ce90857199 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Wed, 16 Sep 2020 18:40:44 +0800 Subject: [PATCH 47/49] drop order leads to panic? --- src/core/index/reader/segment_reader.rs | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/core/index/reader/segment_reader.rs b/src/core/index/reader/segment_reader.rs index b009648..5606ac0 100644 --- a/src/core/index/reader/segment_reader.rs +++ b/src/core/index/reader/segment_reader.rs @@ -489,15 +489,6 @@ pub struct SegmentReader { unsafe impl Sync for SegmentReader {} -impl Drop for SegmentReader { - fn drop(&mut self) { - self.doc_values_producer_preload.write().unwrap().clear(); - self.doc_values_local_preload.write().unwrap().clear(); - self.doc_values_producer.clear(); - self.doc_values_local.clear(); - } -} - /// IndexReader implementation over a single segment. /// Instances pointing to the same segment (but with different deletes, etc) /// may share the same core data. From c9f4283c6238cf9c69f547612bd7d08ad4613162 Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Tue, 29 Sep 2020 16:51:57 +0800 Subject: [PATCH 48/49] fix: approximate_next should matches --- src/core/search/query/filter_query.rs | 30 ++++++++++++++++++++------- 1 file changed, 22 insertions(+), 8 deletions(-) diff --git a/src/core/search/query/filter_query.rs b/src/core/search/query/filter_query.rs index 00144c2..6d3af84 100644 --- a/src/core/search/query/filter_query.rs +++ b/src/core/search/query/filter_query.rs @@ -15,9 +15,9 @@ use core::codec::Codec; use core::index::reader::LeafReaderContext; use core::search::explanation::Explanation; use core::search::query::{Query, TermQuery, Weight}; -use core::search::scorer::{two_phase_next, FeatureResult, Scorer}; +use core::search::scorer::{FeatureResult, Scorer}; use core::search::searcher::SearchPlanBuilder; -use core::search::DocIterator; +use core::search::{DocIterator, NO_MORE_DOCS}; use core::util::DocId; use core::util::IndexedContext; use error::Result; @@ -157,19 +157,31 @@ struct FilterScorer { filters: Vec>, } +impl FilterScorer { + fn two_phase_next(&mut self) -> Result { + let mut doc = self.doc_id(); + loop { + if doc == NO_MORE_DOCS { + return Ok(NO_MORE_DOCS); + } else if self.matches()? { + return Ok(doc); + } + doc = self.scorer.approximate_next()?; + } + } +} + impl DocIterator for FilterScorer { fn doc_id(&self) -> DocId { self.scorer.doc_id() } fn next(&mut self) -> Result { - self.approximate_next()?; - two_phase_next(self) + self.approximate_next() } fn advance(&mut self, target: DocId) -> Result { - self.approximate_advance(target)?; - two_phase_next(self) + self.approximate_advance(target) } fn cost(&self) -> usize { @@ -196,11 +208,13 @@ impl DocIterator for FilterScorer { } fn approximate_next(&mut self) -> Result { - self.scorer.approximate_next() + self.scorer.approximate_next()?; + self.two_phase_next() } fn approximate_advance(&mut self, target: DocId) -> Result { - self.scorer.approximate_advance(target) + self.scorer.approximate_advance(target)?; + self.two_phase_next() } } From 7a75268c96f786916d61b0a2d193f9cfe9d5a79a Mon Sep 17 00:00:00 2001 From: tongjianlin Date: Fri, 9 Oct 2020 14:14:35 +0800 Subject: [PATCH 49/49] Bump version in order to make sure people use cargo dependency can get the latest version. --- joker.yml | 0 src/core/codec/doc_values/doc_values_iterator.rs | 13 +++++++++++++ src/core/codec/postings/partial_block_decoder.rs | 13 +++++++++++++ src/core/codec/postings/simd_block_decoder.rs | 13 +++++++++++++ .../codec/stored_fields/stored_fields_consumer.rs | 13 +++++++++++++ src/core/index/writer/doc_values_update.rs | 13 +++++++++++++ src/core/search/query/boosting_query.rs | 13 +++++++++++++ src/core/search/query/exists_query.rs | 13 +++++++++++++ src/core/search/scorer/boosting_scorer.rs | 13 +++++++++++++ src/core/util/packed/packed_simd.rs | 13 +++++++++++++ 10 files changed, 117 insertions(+) delete mode 100644 joker.yml diff --git a/joker.yml b/joker.yml deleted file mode 100644 index e69de29..0000000 diff --git a/src/core/codec/doc_values/doc_values_iterator.rs b/src/core/codec/doc_values/doc_values_iterator.rs index 1d6061d..7aca84e 100644 --- a/src/core/codec/doc_values/doc_values_iterator.rs +++ b/src/core/codec/doc_values/doc_values_iterator.rs @@ -1,3 +1,16 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + use core::codec::doc_values::{ BinaryDocValues, NumericDocValues, SortedDocValues, SortedNumericDocValues, SortedSetDocValues, }; diff --git a/src/core/codec/postings/partial_block_decoder.rs b/src/core/codec/postings/partial_block_decoder.rs index 350b15f..87cf705 100644 --- a/src/core/codec/postings/partial_block_decoder.rs +++ b/src/core/codec/postings/partial_block_decoder.rs @@ -1,3 +1,16 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + use core::codec::postings::MAX_ENCODED_SIZE; use core::store::io::IndexInput; use core::util::packed::Format; diff --git a/src/core/codec/postings/simd_block_decoder.rs b/src/core/codec/postings/simd_block_decoder.rs index 2a08b43..0b83522 100644 --- a/src/core/codec/postings/simd_block_decoder.rs +++ b/src/core/codec/postings/simd_block_decoder.rs @@ -1,3 +1,16 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + use core::codec::postings::BLOCK_SIZE; use core::search::NO_MORE_DOCS; use core::store::io::IndexInput; diff --git a/src/core/codec/stored_fields/stored_fields_consumer.rs b/src/core/codec/stored_fields/stored_fields_consumer.rs index a0c1578..9add173 100644 --- a/src/core/codec/stored_fields/stored_fields_consumer.rs +++ b/src/core/codec/stored_fields/stored_fields_consumer.rs @@ -1,3 +1,16 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + use core::analysis::TokenStream; use core::codec::field_infos::FieldInfo; use core::codec::segment_infos::SegmentWriteState; diff --git a/src/core/index/writer/doc_values_update.rs b/src/core/index/writer/doc_values_update.rs index 1a1fd3f..73c2edc 100644 --- a/src/core/index/writer/doc_values_update.rs +++ b/src/core/index/writer/doc_values_update.rs @@ -1,3 +1,16 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + use core::codec::{Codec, PackedLongDocMap, PostingIteratorFlags}; use core::codec::{Fields, SorterDocMap, TermIterator, Terms}; use core::doc::{DocValuesType, Term}; diff --git a/src/core/search/query/boosting_query.rs b/src/core/search/query/boosting_query.rs index 38717dc..1486be2 100644 --- a/src/core/search/query/boosting_query.rs +++ b/src/core/search/query/boosting_query.rs @@ -1,3 +1,16 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + use std::any::Any; use std::fmt; diff --git a/src/core/search/query/exists_query.rs b/src/core/search/query/exists_query.rs index fa0d50b..a9f094d 100644 --- a/src/core/search/query/exists_query.rs +++ b/src/core/search/query/exists_query.rs @@ -1,3 +1,16 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + use std::any::Any; use std::fmt; diff --git a/src/core/search/scorer/boosting_scorer.rs b/src/core/search/scorer/boosting_scorer.rs index cbd0e3b..c9f9a90 100644 --- a/src/core/search/scorer/boosting_scorer.rs +++ b/src/core/search/scorer/boosting_scorer.rs @@ -1,3 +1,16 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + use core::search::scorer::Scorer; use core::search::DocIterator; use core::util::DocId; diff --git a/src/core/util/packed/packed_simd.rs b/src/core/util/packed/packed_simd.rs index 85681a8..ff1e713 100644 --- a/src/core/util/packed/packed_simd.rs +++ b/src/core/util/packed/packed_simd.rs @@ -1,3 +1,16 @@ +// Copyright 2019 Zhizhesihai (Beijing) Technology Limited. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + use std::arch::x86_64 as simd; use std::slice::{from_raw_parts, from_raw_parts_mut};