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:
Adam Reichold
2023-01-04 10:11:27 +01:00
committed by GitHub
parent b22f96624e
commit 2080c370c2
14 changed files with 137 additions and 39 deletions

View File

@@ -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(())
}

View File

@@ -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));
}

View File

@@ -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,

View File

@@ -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));
}
}

View File

@@ -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);

View File

@@ -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,

View File

@@ -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);

View File

@@ -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(")")?;

View File

@@ -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 })] }"
);
}
}
}

View File

@@ -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"));
}

View File

@@ -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> {

View File

@@ -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())

View File

@@ -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())

View File

@@ -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())