mirror of
https://github.com/quickwit-oss/tantivy.git
synced 2025-12-26 20:19:57 +00:00
Enable usage of FuzzyTermQuery for specific fields via QueryParser (#1750)
* Make nightly Clippy mostly happy. * Document how to produce TermSetQuery queries using QueryParser. * Enable construction of queries using FuzzyTermQuery via the QueryParser * Use FxHashMap instead of HashMap in the QueryParser as these hash tables are not exposed to DoS attacks. * Use a struct instead of a tuple to improve readability.
This commit is contained in:
@@ -232,7 +232,7 @@ impl Directory for RamDirectory {
|
||||
let path_buf = PathBuf::from(path);
|
||||
self.fs.write().unwrap().write(path_buf, data);
|
||||
if path == *META_FILEPATH {
|
||||
let _ = self.fs.write().unwrap().watch_router.broadcast();
|
||||
drop(self.fs.write().unwrap().watch_router.broadcast());
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -168,7 +168,7 @@ mod tests {
|
||||
watch_event_router.broadcast().wait().unwrap();
|
||||
assert_eq!(2, counter.load(Ordering::SeqCst));
|
||||
mem::drop(handle_a);
|
||||
let _ = watch_event_router.broadcast();
|
||||
drop(watch_event_router.broadcast());
|
||||
watch_event_router.broadcast().wait().unwrap();
|
||||
assert_eq!(2, counter.load(Ordering::SeqCst));
|
||||
}
|
||||
|
||||
@@ -89,11 +89,11 @@ pub(crate) fn index_json_values<'a>(
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn index_json_object<'a>(
|
||||
fn index_json_object(
|
||||
doc: DocId,
|
||||
json_value: &serde_json::Map<String, serde_json::Value>,
|
||||
text_analyzer: &TextAnalyzer,
|
||||
json_term_writer: &mut JsonTermWriter<'a>,
|
||||
json_term_writer: &mut JsonTermWriter,
|
||||
postings_writer: &mut dyn PostingsWriter,
|
||||
ctx: &mut IndexingContext,
|
||||
positions_per_path: &mut IndexingPositionsPerPath,
|
||||
@@ -113,11 +113,11 @@ fn index_json_object<'a>(
|
||||
}
|
||||
}
|
||||
|
||||
fn index_json_value<'a>(
|
||||
fn index_json_value(
|
||||
doc: DocId,
|
||||
json_value: &serde_json::Value,
|
||||
text_analyzer: &TextAnalyzer,
|
||||
json_term_writer: &mut JsonTermWriter<'a>,
|
||||
json_term_writer: &mut JsonTermWriter,
|
||||
postings_writer: &mut dyn PostingsWriter,
|
||||
ctx: &mut IndexingContext,
|
||||
positions_per_path: &mut IndexingPositionsPerPath,
|
||||
|
||||
@@ -577,7 +577,7 @@ impl SegmentUpdater {
|
||||
for merge_operation in merge_candidates {
|
||||
// If a merge cannot be started this is not a fatal error.
|
||||
// We do log a warning in `start_merge`.
|
||||
let _ = self.start_merge(merge_operation);
|
||||
drop(self.start_merge(merge_operation));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -291,7 +291,7 @@ pub mod tests {
|
||||
const PADDING_VALUE: u32 = 234_234_345u32;
|
||||
let expected_length = 154;
|
||||
let mut encoder = BlockEncoder::new();
|
||||
let input: Vec<u32> = (0u32..123u32).map(|i| 4 + i * 7 / 2).into_iter().collect();
|
||||
let input: Vec<u32> = (0u32..123u32).map(|i| 4 + i * 7 / 2).collect();
|
||||
for offset in &[0u32, 1u32, 2u32] {
|
||||
let encoded_data = encoder.compress_vint_sorted(&input, *offset);
|
||||
assert!(encoded_data.len() <= expected_length);
|
||||
|
||||
@@ -146,7 +146,7 @@ impl Query for BooleanQuery {
|
||||
let sub_weights = self
|
||||
.subqueries
|
||||
.iter()
|
||||
.map(|&(ref occur, ref subquery)| Ok((*occur, subquery.weight(enable_scoring)?)))
|
||||
.map(|(occur, subquery)| Ok((*occur, subquery.weight(enable_scoring)?)))
|
||||
.collect::<crate::Result<_>>()?;
|
||||
Ok(Box::new(BooleanWeight::new(
|
||||
sub_weights,
|
||||
|
||||
@@ -91,7 +91,7 @@ impl<TScoreCombiner: ScoreCombiner> BooleanWeight<TScoreCombiner> {
|
||||
boost: Score,
|
||||
) -> crate::Result<HashMap<Occur, Vec<Box<dyn Scorer>>>> {
|
||||
let mut per_occur_scorers: HashMap<Occur, Vec<Box<dyn Scorer>>> = HashMap::new();
|
||||
for &(ref occur, ref subweight) in &self.weights {
|
||||
for (occur, subweight) in &self.weights {
|
||||
let sub_scorer: Box<dyn Scorer> = subweight.scorer(reader, boost)?;
|
||||
per_occur_scorers
|
||||
.entry(*occur)
|
||||
@@ -191,7 +191,7 @@ impl<TScoreCombiner: ScoreCombiner + Sync> Weight for BooleanWeight<TScoreCombin
|
||||
}
|
||||
|
||||
let mut explanation = Explanation::new("BooleanClause. Sum of ...", scorer.score());
|
||||
for &(ref occur, ref subweight) in &self.weights {
|
||||
for (occur, subweight) in &self.weights {
|
||||
if is_positive_occur(*occur) {
|
||||
if let Ok(child_explanation) = subweight.explain(reader, doc) {
|
||||
explanation.add_detail(child_explanation);
|
||||
|
||||
@@ -54,9 +54,9 @@ impl fmt::Debug for LogicalAst {
|
||||
if clause.is_empty() {
|
||||
write!(formatter, "<emptyclause>")?;
|
||||
} else {
|
||||
let (ref occur, ref subquery) = clause[0];
|
||||
let (occur, subquery) = &clause[0];
|
||||
write!(formatter, "({}{:?}", occur_letter(*occur), subquery)?;
|
||||
for &(ref occur, ref subquery) in &clause[1..] {
|
||||
for (occur, subquery) in &clause[1..] {
|
||||
write!(formatter, " {}{:?}", occur_letter(*occur), subquery)?;
|
||||
}
|
||||
formatter.write_str(")")?;
|
||||
|
||||
@@ -1,9 +1,9 @@
|
||||
use std::collections::HashMap;
|
||||
use std::net::{AddrParseError, IpAddr};
|
||||
use std::num::{ParseFloatError, ParseIntError};
|
||||
use std::ops::Bound;
|
||||
use std::str::{FromStr, ParseBoolError};
|
||||
|
||||
use rustc_hash::FxHashMap;
|
||||
use tantivy_query_grammar::{UserInputAst, UserInputBound, UserInputLeaf, UserInputLiteral};
|
||||
|
||||
use super::logical_ast::*;
|
||||
@@ -12,8 +12,8 @@ use crate::indexer::{
|
||||
convert_to_fast_value_and_get_term, set_string_and_get_terms, JsonTermWriter,
|
||||
};
|
||||
use crate::query::{
|
||||
AllQuery, BooleanQuery, BoostQuery, EmptyQuery, Occur, PhraseQuery, Query, RangeQuery,
|
||||
TermQuery, TermSetQuery,
|
||||
AllQuery, BooleanQuery, BoostQuery, EmptyQuery, FuzzyTermQuery, Occur, PhraseQuery, Query,
|
||||
RangeQuery, TermQuery, TermSetQuery,
|
||||
};
|
||||
use crate::schema::{
|
||||
Facet, FacetParseError, Field, FieldType, IndexRecordOption, IntoIpv6Addr, JsonObjectOptions,
|
||||
@@ -160,6 +160,10 @@ fn trim_ast(logical_ast: LogicalAst) -> Option<LogicalAst> {
|
||||
/// word lexicographically between `a` and `c` (inclusive lower bound, exclusive upper bound).
|
||||
/// Inclusive bounds are `[]`, exclusive are `{}`.
|
||||
///
|
||||
/// * set terms: Using the `IN` operator, a field can be matched against a set of literals, e.g.
|
||||
/// `title: IN [a b cd]` will match documents where `title` is either `a`, `b` or `cd`, but do so
|
||||
/// more efficiently than the alternative query `title:a OR title:b OR title:c` does.
|
||||
///
|
||||
/// * date values: The query parser supports rfc3339 formatted dates. For example
|
||||
/// `"2002-10-02T15:00:00.05Z"` or `some_date_field:[2002-10-02T15:00:00Z TO
|
||||
/// 2002-10-02T18:00:00Z}`
|
||||
@@ -174,6 +178,9 @@ fn trim_ast(logical_ast: LogicalAst) -> Option<LogicalAst> {
|
||||
/// (See [`set_field_boost(...)`](QueryParser::set_field_boost)). Typically you may want to boost a
|
||||
/// title field.
|
||||
///
|
||||
/// Additionally, specific fields can be marked to use fuzzy term queries for each literal
|
||||
/// via the [`QueryParser::set_field_fuzzy`] method.
|
||||
///
|
||||
/// Phrase terms support the `~` slop operator which allows to set the phrase's matching
|
||||
/// distance in words. `"big wolf"~1` will return documents containing the phrase `"big bad wolf"`.
|
||||
#[derive(Clone)]
|
||||
@@ -182,7 +189,15 @@ pub struct QueryParser {
|
||||
default_fields: Vec<Field>,
|
||||
conjunction_by_default: bool,
|
||||
tokenizer_manager: TokenizerManager,
|
||||
boost: HashMap<Field, Score>,
|
||||
boost: FxHashMap<Field, Score>,
|
||||
fuzzy: FxHashMap<Field, Fuzzy>,
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
struct Fuzzy {
|
||||
prefix: bool,
|
||||
distance: u8,
|
||||
transpose_cost_one: bool,
|
||||
}
|
||||
|
||||
fn all_negative(ast: &LogicalAst) -> bool {
|
||||
@@ -210,6 +225,7 @@ impl QueryParser {
|
||||
tokenizer_manager,
|
||||
conjunction_by_default: false,
|
||||
boost: Default::default(),
|
||||
fuzzy: Default::default(),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -247,6 +263,30 @@ impl QueryParser {
|
||||
self.boost.insert(field, boost);
|
||||
}
|
||||
|
||||
/// Sets the given [field][`Field`] to use [fuzzy term queries][`FuzzyTermQuery`]
|
||||
///
|
||||
/// If set, the parse will produce queries using fuzzy term queries
|
||||
/// with the given parameters for each literal matched against the given field.
|
||||
///
|
||||
/// See the [`FuzzyTermQuery::new`] and [`FuzzyTermQuery::new_prefix`] methods
|
||||
/// for the meaning of the individual parameters.
|
||||
pub fn set_field_fuzzy(
|
||||
&mut self,
|
||||
field: Field,
|
||||
prefix: bool,
|
||||
distance: u8,
|
||||
transpose_cost_one: bool,
|
||||
) {
|
||||
self.fuzzy.insert(
|
||||
field,
|
||||
Fuzzy {
|
||||
prefix,
|
||||
distance,
|
||||
transpose_cost_one,
|
||||
},
|
||||
);
|
||||
}
|
||||
|
||||
/// Parse a query
|
||||
///
|
||||
/// Note that `parse_query` returns an error if the input
|
||||
@@ -259,7 +299,7 @@ impl QueryParser {
|
||||
/// in [Issue 5](https://github.com/fulmicoton/tantivy/issues/5)
|
||||
pub fn parse_query(&self, query: &str) -> Result<Box<dyn Query>, QueryParserError> {
|
||||
let logical_ast = self.parse_query_to_logical_ast(query)?;
|
||||
Ok(convert_to_query(logical_ast))
|
||||
Ok(convert_to_query(&self.fuzzy, logical_ast))
|
||||
}
|
||||
|
||||
/// Parse the user query into an AST.
|
||||
@@ -660,9 +700,30 @@ impl QueryParser {
|
||||
}
|
||||
}
|
||||
|
||||
fn convert_literal_to_query(logical_literal: LogicalLiteral) -> Box<dyn Query> {
|
||||
fn convert_literal_to_query(
|
||||
fuzzy: &FxHashMap<Field, Fuzzy>,
|
||||
logical_literal: LogicalLiteral,
|
||||
) -> Box<dyn Query> {
|
||||
match logical_literal {
|
||||
LogicalLiteral::Term(term) => Box::new(TermQuery::new(term, IndexRecordOption::WithFreqs)),
|
||||
LogicalLiteral::Term(term) => {
|
||||
if let Some(fuzzy) = fuzzy.get(&term.field()) {
|
||||
if fuzzy.prefix {
|
||||
Box::new(FuzzyTermQuery::new_prefix(
|
||||
term,
|
||||
fuzzy.distance,
|
||||
fuzzy.transpose_cost_one,
|
||||
))
|
||||
} else {
|
||||
Box::new(FuzzyTermQuery::new(
|
||||
term,
|
||||
fuzzy.distance,
|
||||
fuzzy.transpose_cost_one,
|
||||
))
|
||||
}
|
||||
} else {
|
||||
Box::new(TermQuery::new(term, IndexRecordOption::WithFreqs))
|
||||
}
|
||||
}
|
||||
LogicalLiteral::Phrase(term_with_offsets, slop) => Box::new(
|
||||
PhraseQuery::new_with_offset_and_slop(term_with_offsets, slop),
|
||||
),
|
||||
@@ -755,12 +816,12 @@ fn generate_literals_for_json_object(
|
||||
Ok(logical_literals)
|
||||
}
|
||||
|
||||
fn convert_to_query(logical_ast: LogicalAst) -> Box<dyn Query> {
|
||||
fn convert_to_query(fuzzy: &FxHashMap<Field, Fuzzy>, logical_ast: LogicalAst) -> Box<dyn Query> {
|
||||
match trim_ast(logical_ast) {
|
||||
Some(LogicalAst::Clause(trimmed_clause)) => {
|
||||
let occur_subqueries = trimmed_clause
|
||||
.into_iter()
|
||||
.map(|(occur, subquery)| (occur, convert_to_query(subquery)))
|
||||
.map(|(occur, subquery)| (occur, convert_to_query(fuzzy, subquery)))
|
||||
.collect::<Vec<_>>();
|
||||
assert!(
|
||||
!occur_subqueries.is_empty(),
|
||||
@@ -769,10 +830,10 @@ fn convert_to_query(logical_ast: LogicalAst) -> Box<dyn Query> {
|
||||
Box::new(BooleanQuery::new(occur_subqueries))
|
||||
}
|
||||
Some(LogicalAst::Leaf(trimmed_logical_literal)) => {
|
||||
convert_literal_to_query(*trimmed_logical_literal)
|
||||
convert_literal_to_query(fuzzy, *trimmed_logical_literal)
|
||||
}
|
||||
Some(LogicalAst::Boost(ast, boost)) => {
|
||||
let query = convert_to_query(*ast);
|
||||
let query = convert_to_query(fuzzy, *ast);
|
||||
let boosted_query = BoostQuery::new(query, boost);
|
||||
Box::new(boosted_query)
|
||||
}
|
||||
@@ -1568,4 +1629,41 @@ mod test {
|
||||
false,
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
pub fn test_set_field_fuzzy() {
|
||||
{
|
||||
let mut query_parser = make_query_parser();
|
||||
query_parser.set_field_fuzzy(
|
||||
query_parser.schema.get_field("title").unwrap(),
|
||||
false,
|
||||
1,
|
||||
true,
|
||||
);
|
||||
let query = query_parser.parse_query("abc").unwrap();
|
||||
assert_eq!(
|
||||
format!("{:?}", query),
|
||||
"BooleanQuery { subqueries: [(Should, FuzzyTermQuery { term: Term(type=Str, \
|
||||
field=0, \"abc\"), distance: 1, transposition_cost_one: true, prefix: false }), \
|
||||
(Should, TermQuery(Term(type=Str, field=1, \"abc\")))] }"
|
||||
);
|
||||
}
|
||||
|
||||
{
|
||||
let mut query_parser = make_query_parser();
|
||||
query_parser.set_field_fuzzy(
|
||||
query_parser.schema.get_field("text").unwrap(),
|
||||
true,
|
||||
2,
|
||||
false,
|
||||
);
|
||||
let query = query_parser.parse_query("abc").unwrap();
|
||||
assert_eq!(
|
||||
format!("{:?}", query),
|
||||
"BooleanQuery { subqueries: [(Should, TermQuery(Term(type=Str, field=0, \
|
||||
\"abc\"))), (Should, FuzzyTermQuery { term: Term(type=Str, field=1, \"abc\"), \
|
||||
distance: 2, transposition_cost_one: false, prefix: true })] }"
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -258,7 +258,7 @@ mod tests {
|
||||
let field = schema.get_field("body").unwrap();
|
||||
let field_entry = schema.get_field_entry(field);
|
||||
assert!(matches!(field_entry.field_type(),
|
||||
&FieldType::Str(ref text_options)
|
||||
FieldType::Str(text_options)
|
||||
if text_options.get_indexing_options().unwrap().tokenizer() == "default"));
|
||||
}
|
||||
|
||||
|
||||
@@ -124,9 +124,9 @@ impl Snippet {
|
||||
///
|
||||
/// Fragments must be valid in the sense that `&text[fragment.start..fragment.stop]`\
|
||||
/// has to be a valid string.
|
||||
fn search_fragments<'a>(
|
||||
fn search_fragments(
|
||||
tokenizer: &TextAnalyzer,
|
||||
text: &'a str,
|
||||
text: &str,
|
||||
terms: &BTreeMap<String, Score>,
|
||||
max_num_chars: usize,
|
||||
) -> Vec<FragmentCandidate> {
|
||||
|
||||
@@ -99,7 +99,7 @@ fn test_term_dictionary_stream() -> crate::Result<()> {
|
||||
.collect();
|
||||
let buffer: Vec<u8> = {
|
||||
let mut term_dictionary_builder = TermDictionaryBuilder::create(vec![]).unwrap();
|
||||
for &(ref id, ref i) in &ids {
|
||||
for (id, i) in &ids {
|
||||
term_dictionary_builder
|
||||
.insert(id.as_bytes(), &make_term_info(*i as u64))
|
||||
.unwrap();
|
||||
@@ -112,14 +112,14 @@ fn test_term_dictionary_stream() -> crate::Result<()> {
|
||||
let mut streamer = term_dictionary.stream()?;
|
||||
let mut i = 0;
|
||||
while let Some((streamer_k, streamer_v)) = streamer.next() {
|
||||
let &(ref key, ref v) = &ids[i];
|
||||
let (key, v) = &ids[i];
|
||||
assert_eq!(streamer_k, key.as_bytes());
|
||||
assert_eq!(streamer_v, &make_term_info(*v as u64));
|
||||
i += 1;
|
||||
}
|
||||
}
|
||||
|
||||
let &(ref key, ref val) = &ids[2047];
|
||||
let (key, val) = &ids[2047];
|
||||
assert_eq!(
|
||||
term_dictionary.get(key.as_bytes())?,
|
||||
Some(make_term_info(*val as u64))
|
||||
@@ -160,7 +160,7 @@ fn test_stream_range() -> crate::Result<()> {
|
||||
.collect();
|
||||
let buffer: Vec<u8> = {
|
||||
let mut term_dictionary_builder = TermDictionaryBuilder::create(vec![]).unwrap();
|
||||
for &(ref id, ref i) in &ids {
|
||||
for (id, i) in &ids {
|
||||
term_dictionary_builder
|
||||
.insert(id.as_bytes(), &make_term_info(*i as u64))
|
||||
.unwrap();
|
||||
@@ -173,14 +173,14 @@ fn test_stream_range() -> crate::Result<()> {
|
||||
let term_dictionary: TermDictionary = TermDictionary::open(file)?;
|
||||
{
|
||||
for i in (0..20).chain(6000..8_000) {
|
||||
let &(ref target_key, _) = &ids[i];
|
||||
let (target_key, _) = &ids[i];
|
||||
let mut streamer = term_dictionary
|
||||
.range()
|
||||
.ge(target_key.as_bytes())
|
||||
.into_stream()?;
|
||||
for j in 0..3 {
|
||||
let (streamer_k, streamer_v) = streamer.next().unwrap();
|
||||
let &(ref key, ref v) = &ids[i + j];
|
||||
let (key, v) = &ids[i + j];
|
||||
assert_eq!(str::from_utf8(streamer_k).unwrap(), key);
|
||||
assert_eq!(streamer_v.doc_freq, *v);
|
||||
assert_eq!(streamer_v, &make_term_info(*v as u64));
|
||||
@@ -190,14 +190,14 @@ fn test_stream_range() -> crate::Result<()> {
|
||||
|
||||
{
|
||||
for i in (0..20).chain(BLOCK_SIZE - 10..BLOCK_SIZE + 10) {
|
||||
let &(ref target_key, _) = &ids[i];
|
||||
let (target_key, _) = &ids[i];
|
||||
let mut streamer = term_dictionary
|
||||
.range()
|
||||
.gt(target_key.as_bytes())
|
||||
.into_stream()?;
|
||||
for j in 0..3 {
|
||||
let (streamer_k, streamer_v) = streamer.next().unwrap();
|
||||
let &(ref key, ref v) = &ids[i + j + 1];
|
||||
let (key, v) = &ids[i + j + 1];
|
||||
assert_eq!(streamer_k, key.as_bytes());
|
||||
assert_eq!(streamer_v.doc_freq, *v);
|
||||
}
|
||||
@@ -207,8 +207,8 @@ fn test_stream_range() -> crate::Result<()> {
|
||||
{
|
||||
for i in (0..20).chain(BLOCK_SIZE - 10..BLOCK_SIZE + 10) {
|
||||
for j in 0..3 {
|
||||
let &(ref fst_key, _) = &ids[i];
|
||||
let &(ref last_key, _) = &ids[i + j];
|
||||
let (fst_key, _) = &ids[i];
|
||||
let (last_key, _) = &ids[i + j];
|
||||
let mut streamer = term_dictionary
|
||||
.range()
|
||||
.ge(fst_key.as_bytes())
|
||||
|
||||
@@ -26,7 +26,7 @@ impl<'a> SimpleTokenStream<'a> {
|
||||
// search for the end of the current token.
|
||||
fn search_token_end(&mut self) -> usize {
|
||||
(&mut self.chars)
|
||||
.filter(|&(_, ref c)| !c.is_alphanumeric())
|
||||
.filter(|(_, c)| !c.is_alphanumeric())
|
||||
.map(|(offset, _)| offset)
|
||||
.next()
|
||||
.unwrap_or(self.text.len())
|
||||
|
||||
@@ -26,7 +26,7 @@ impl<'a> WhitespaceTokenStream<'a> {
|
||||
// search for the end of the current token.
|
||||
fn search_token_end(&mut self) -> usize {
|
||||
(&mut self.chars)
|
||||
.filter(|&(_, ref c)| c.is_ascii_whitespace())
|
||||
.filter(|(_, c)| c.is_ascii_whitespace())
|
||||
.map(|(offset, _)| offset)
|
||||
.next()
|
||||
.unwrap_or(self.text.len())
|
||||
|
||||
Reference in New Issue
Block a user