Skip to content

Commit

Permalink
Implement matches() on SourceConfirmedTextQuery
Browse files Browse the repository at this point in the history
  • Loading branch information
romseygeek committed Oct 2, 2023
1 parent c24cc0f commit fa83b97
Show file tree
Hide file tree
Showing 4 changed files with 172 additions and 2 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/

package org.elasticsearch.index.mapper;

import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.index.mapper.extras.MapperExtrasPlugin;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xcontent.XContentBuilder;
import org.elasticsearch.xcontent.XContentFactory;

import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;

import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder;
import static org.hamcrest.Matchers.containsString;

public class MatchOnlyTextMapperIntegrationIT extends ESIntegTestCase {

protected Collection<Class<? extends Plugin>> nodePlugins() {
return Arrays.asList(MapperExtrasPlugin.class);
}

public void testHighlightingWithMatchOnlyTextFieldMatchPhrase() throws IOException {
XContentBuilder mappings = jsonBuilder();
mappings.startObject().startObject("properties").startObject("message").field("type", "match_only_text").endObject().endObject();
mappings.endObject();
assertAcked(prepareCreate("test").setMapping(mappings));
BulkRequestBuilder bulk = client().prepareBulk("test").setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
for (int i = 0; i < 2000; i++) {
bulk.add(
client().prepareIndex()
.setSource(
XContentFactory.jsonBuilder()
.startObject()
.field(
"message",
"[.ds-.slm-history-5-2023.09.20-"
+ randomInt()
+ "][0] marking and sending shard failed due to [failed recovery]"
)
.endObject()
)
);
}
BulkResponse bulkItemResponses = bulk.get();
assertNoFailures(bulkItemResponses);

SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(QueryBuilders.matchPhraseQuery("message", "marking and sending shard"))
.setSize(500)
.highlighter(new HighlightBuilder().field("message"))
.get();
assertNoFailures(searchResponse);
assertThat(
searchResponse.getHits().getAt(0).getHighlightFields().get("message").fragments()[0].string(),
containsString("<em>marking and sending shard</em>")
);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -204,9 +204,9 @@ private IOFunction<LeafReaderContext, CheckedIntFunction<List<Object>, IOExcepti
};
};
}
ValueFetcher valueFetcher = valueFetcher(searchExecutionContext, null);
SourceProvider sourceProvider = searchExecutionContext.lookup();
return context -> {
ValueFetcher valueFetcher = valueFetcher(searchExecutionContext, null);
SourceProvider sourceProvider = searchExecutionContext.lookup();
valueFetcher.setNextReader(context);
return docID -> {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.lucene.search.LeafSimScorer;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Matches;
import org.apache.lucene.search.MultiPhraseQuery;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.PrefixQuery;
Expand Down Expand Up @@ -293,6 +294,14 @@ public RuntimePhraseScorer scorer(LeafReaderContext context) throws IOException
return new RuntimePhraseScorer(this, approximation, leafSimScorer, valueFetcher, field, in);
}

@Override
public Matches matches(LeafReaderContext context, int doc) throws IOException {
RuntimePhraseScorer scorer = scorer(context);
if (scorer == null || scorer.iterator().advance(doc) != doc) {
return null;
}
return scorer.matches();
}
};
}

Expand Down Expand Up @@ -385,6 +394,20 @@ private float computeFreq() throws IOException {
}
return frequency;
}

private Matches matches() throws IOException {
MemoryIndex index = new MemoryIndex(true, false);
List<Object> values = valueFetcher.apply(docID());
for (Object value : values) {
if (value == null) {
continue;
}
index.addField(field, value.toString(), indexAnalyzer);
}
IndexSearcher searcher = index.createSearcher();
Weight w = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1);
return w.matches(searcher.getLeafContexts().get(0), 0);
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -10,11 +10,13 @@

import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.KeywordField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.Term;
import org.apache.lucene.queries.spans.SpanNearQuery;
import org.apache.lucene.queries.spans.SpanQuery;
Expand All @@ -23,12 +25,19 @@
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Matches;
import org.apache.lucene.search.MatchesIterator;
import org.apache.lucene.search.MultiPhraseQuery;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortedSetSelector;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.Weight;
import org.apache.lucene.store.Directory;
import org.apache.lucene.tests.search.CheckHits;
import org.apache.lucene.util.IOFunction;
Expand All @@ -41,6 +50,8 @@
import java.util.Collections;
import java.util.List;

import static org.hamcrest.Matchers.greaterThan;

public class SourceConfirmedTextQueryTests extends ESTestCase {

private static final IOFunction<LeafReaderContext, CheckedIntFunction<List<Object>, IOException>> SOURCE_FETCHER_PROVIDER =
Expand Down Expand Up @@ -429,4 +440,65 @@ public void testEmptyIndex() throws Exception {
}
}

public void testMatches() throws Exception {
checkMatches(new TermQuery(new Term("body", "d")), "a b c d e", new int[] { 3, 3 });
checkMatches(new PhraseQuery("body", "b", "c"), "a b c d c b c a", new int[] { 1, 2, 5, 6 });
}

private static void checkMatches(Query query, String inputDoc, int[] expectedMatches) throws IOException {
try (Directory dir = newDirectory(); IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(Lucene.STANDARD_ANALYZER))) {
Document doc = new Document();
doc.add(new TextField("body", "xxxxxnomatchxxxx", Store.YES));
doc.add(new KeywordField("sort", "0", Store.NO));
w.addDocument(doc);

doc = new Document();
doc.add(new TextField("body", inputDoc, Store.YES));
doc.add(new KeywordField("sort", "1", Store.NO));
w.addDocument(doc);

doc = new Document();
doc.add(new TextField("body", "xxxx " + inputDoc, Store.YES));
doc.add(new KeywordField("sort", "2", Store.NO));
w.addDocument(doc);

Query sourceConfirmedQuery = new SourceConfirmedTextQuery(query, SOURCE_FETCHER_PROVIDER, Lucene.STANDARD_ANALYZER);

try (IndexReader ir = DirectoryReader.open(w)) {

IndexSearcher searcher = new IndexSearcher(ir);
TopDocs td = searcher.search(
sourceConfirmedQuery,
3,
new Sort(KeywordField.newSortField("sort", false, SortedSetSelector.Type.MAX))
);

Weight weight = searcher.createWeight(searcher.rewrite(sourceConfirmedQuery), ScoreMode.COMPLETE_NO_SCORES, 1);

int firstDoc = td.scoreDocs[0].doc;
LeafReaderContext firstCtx = searcher.getLeafContexts().get(ReaderUtil.subIndex(firstDoc, searcher.getLeafContexts()));
checkMatches(weight, firstCtx, firstDoc - firstCtx.docBase, expectedMatches, 0);

int secondDoc = td.scoreDocs[1].doc;
LeafReaderContext secondCtx = searcher.getLeafContexts().get(ReaderUtil.subIndex(secondDoc, searcher.getLeafContexts()));
checkMatches(weight, secondCtx, secondDoc - secondCtx.docBase, expectedMatches, 1);

}
}
}

private static void checkMatches(Weight w, LeafReaderContext ctx, int doc, int[] expectedMatches, int offset) throws IOException {
Matches matches = w.matches(ctx, doc);
assertNotNull(matches);
MatchesIterator mi = matches.getMatches("body");
int i = 0;
while (mi.next()) {
assertThat(expectedMatches.length, greaterThan(i + 1));
assertEquals(mi.startPosition(), expectedMatches[i] + offset);
assertEquals(mi.endPosition(), expectedMatches[i + 1] + offset);
i += 2;
}
assertEquals(expectedMatches.length, i);
}

}

0 comments on commit fa83b97

Please sign in to comment.