Convert limit to long, add new query type.
This commit is contained in:
parent
5d5d506421
commit
575bde94f0
@ -95,7 +95,7 @@ public class LuceneIndex<T, U> implements LLSnapshottable {
|
||||
private SearchResult<T, U> transformLuceneResultWithValues(LLSearchResult llSearchResult,
|
||||
@Nullable MultiSort<SearchResultItem<T, U>> sort,
|
||||
LLScoreMode scoreMode,
|
||||
@Nullable Integer limit,
|
||||
@Nullable Long limit,
|
||||
ValueGetter<T, U> valueGetter) {
|
||||
var mappedKeys = llSearchResult
|
||||
.results()
|
||||
@ -182,7 +182,7 @@ public class LuceneIndex<T, U> implements LLSnapshottable {
|
||||
*/
|
||||
public Mono<SearchResult<T, U>> searchWithValues(@Nullable CompositeSnapshot snapshot,
|
||||
Query query,
|
||||
int limit,
|
||||
long limit,
|
||||
@Nullable MultiSort<SearchResultItem<T, U>> sort,
|
||||
LLScoreMode scoreMode,
|
||||
@Nullable Float minCompetitiveScore,
|
||||
|
@ -33,7 +33,7 @@ public interface LLLuceneIndex extends LLSnapshottable {
|
||||
*/
|
||||
Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
|
||||
Flux<Tuple2<String, Set<String>>> mltDocumentFields,
|
||||
int limit,
|
||||
long limit,
|
||||
@Nullable Float minCompetitiveScore,
|
||||
String keyFieldName);
|
||||
|
||||
@ -46,7 +46,7 @@ public interface LLLuceneIndex extends LLSnapshottable {
|
||||
*/
|
||||
Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot,
|
||||
Query query,
|
||||
int limit,
|
||||
long limit,
|
||||
@Nullable LLSort sort,
|
||||
LLScoreMode scoreMode,
|
||||
@Nullable Float minCompetitiveScore,
|
||||
|
@ -33,6 +33,7 @@ import java.util.concurrent.ArrayBlockingQueue;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.commons.math3.exception.NumberIsTooLargeException;
|
||||
import org.apache.lucene.index.IndexCommit;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
@ -338,7 +339,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
|
||||
@Override
|
||||
public Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
|
||||
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
|
||||
int limit,
|
||||
long limit,
|
||||
@Nullable Float minCompetitiveScore,
|
||||
String keyFieldName) {
|
||||
return moreLikeThis(snapshot, mltDocumentFieldsFlux, limit, minCompetitiveScore, keyFieldName, false, 0, 1);
|
||||
@ -346,7 +347,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
|
||||
|
||||
public Mono<LLSearchResult> distributedMoreLikeThis(@Nullable LLSnapshot snapshot,
|
||||
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
|
||||
int limit,
|
||||
long limit,
|
||||
@Nullable Float minCompetitiveScore,
|
||||
String keyFieldName,
|
||||
long actionId,
|
||||
@ -365,7 +366,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
|
||||
@SuppressWarnings({"Convert2MethodRef", "unchecked", "rawtypes"})
|
||||
private Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
|
||||
Flux<Tuple2<String, Set<String>>> mltDocumentFieldsFlux,
|
||||
int limit,
|
||||
long limit,
|
||||
@Nullable Float minCompetitiveScore,
|
||||
String keyFieldName,
|
||||
boolean doDistributedPre,
|
||||
@ -408,9 +409,12 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
|
||||
allowOnlyQueryParsingCollectorStreamSearcher.search(indexSearcher, query);
|
||||
totalHitsCountSink.tryEmitValue(0L);
|
||||
} else {
|
||||
if (limit > Integer.MAX_VALUE) {
|
||||
throw new NumberIsTooLargeException(limit, Integer.MAX_VALUE, true);
|
||||
}
|
||||
streamSearcher.search(indexSearcher,
|
||||
query,
|
||||
limit,
|
||||
(int) limit,
|
||||
null,
|
||||
ScoreMode.TOP_SCORES,
|
||||
minCompetitiveScore,
|
||||
@ -458,13 +462,13 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot, it.cavallium.dbengine.lucene.serializer.Query query, int limit,
|
||||
public Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot, it.cavallium.dbengine.lucene.serializer.Query query, long limit,
|
||||
@Nullable LLSort sort, @NotNull LLScoreMode scoreMode, @Nullable Float minCompetitiveScore, String keyFieldName) {
|
||||
return search(snapshot, query, limit, sort, scoreMode, minCompetitiveScore,
|
||||
keyFieldName, false, 0, 1);
|
||||
}
|
||||
|
||||
public Mono<LLSearchResult> distributedSearch(@Nullable LLSnapshot snapshot, it.cavallium.dbengine.lucene.serializer.Query query, int limit,
|
||||
public Mono<LLSearchResult> distributedSearch(@Nullable LLSnapshot snapshot, it.cavallium.dbengine.lucene.serializer.Query query, long limit,
|
||||
@Nullable LLSort sort, @NotNull LLScoreMode scoreMode, @Nullable Float minCompetitiveScore, String keyFieldName, long actionId, int scoreDivisor) {
|
||||
return search(snapshot, query, limit, sort, scoreMode, minCompetitiveScore,
|
||||
keyFieldName, false, actionId, scoreDivisor);
|
||||
@ -480,7 +484,7 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
|
||||
|
||||
@SuppressWarnings("Convert2MethodRef")
|
||||
private Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot,
|
||||
it.cavallium.dbengine.lucene.serializer.Query query, int limit,
|
||||
it.cavallium.dbengine.lucene.serializer.Query query, long limit,
|
||||
@Nullable LLSort sort, @NotNull LLScoreMode scoreMode, @Nullable Float minCompetitiveScore, String keyFieldName,
|
||||
boolean doDistributedPre, long actionId, int scoreDivisor) {
|
||||
return acquireSearcherWrapper(snapshot, doDistributedPre, actionId)
|
||||
@ -511,9 +515,12 @@ public class LLLocalLuceneIndex implements LLLuceneIndex {
|
||||
allowOnlyQueryParsingCollectorStreamSearcher.search(indexSearcher, luceneQuery);
|
||||
totalHitsCountSink.tryEmitValue(0L);
|
||||
} else {
|
||||
if (limit > Integer.MAX_VALUE) {
|
||||
throw new NumberIsTooLargeException(limit, Integer.MAX_VALUE, true);
|
||||
}
|
||||
streamSearcher.search(indexSearcher,
|
||||
luceneQuery,
|
||||
limit,
|
||||
(int) limit,
|
||||
luceneSort,
|
||||
luceneScoreMode,
|
||||
minCompetitiveScore,
|
||||
|
@ -203,7 +203,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
|
||||
@Override
|
||||
public Mono<LLSearchResult> moreLikeThis(@Nullable LLSnapshot snapshot,
|
||||
Flux<Tuple2<String, Set<String>>> mltDocumentFields,
|
||||
int limit,
|
||||
long limit,
|
||||
@Nullable Float minCompetitiveScore,
|
||||
String keyFieldName) {
|
||||
long actionId;
|
||||
@ -272,7 +272,7 @@ public class LLLocalMultiLuceneIndex implements LLLuceneIndex {
|
||||
@Override
|
||||
public Mono<LLSearchResult> search(@Nullable LLSnapshot snapshot,
|
||||
Query query,
|
||||
int limit,
|
||||
long limit,
|
||||
@Nullable LLSort sort,
|
||||
LLScoreMode scoreMode,
|
||||
@Nullable Float minCompetitiveScore,
|
||||
|
@ -171,7 +171,7 @@ public class LuceneUtils {
|
||||
*/
|
||||
public static <T> Flux<T> mergeStream(Flux<Flux<T>> mappedMultiResults,
|
||||
@Nullable MultiSort<T> sort,
|
||||
@Nullable Integer limit) {
|
||||
@Nullable Long limit) {
|
||||
if (limit != null && limit == 0) {
|
||||
return mappedMultiResults.flatMap(f -> f).ignoreElements().flux();
|
||||
}
|
||||
@ -183,10 +183,10 @@ public class LuceneUtils {
|
||||
//noinspection unchecked
|
||||
mergedFlux = Flux.mergeOrdered(32, sort.getResultSort(), mappedMultiResultsList.toArray(Flux[]::new));
|
||||
}
|
||||
if (limit == null) {
|
||||
if (limit == null || limit == Long.MAX_VALUE) {
|
||||
return mergedFlux;
|
||||
} else {
|
||||
return mergedFlux.take(limit);
|
||||
return mergedFlux.limitRequest(limit);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -0,0 +1,26 @@
|
||||
package it.cavallium.dbengine.lucene.serializer;
|
||||
|
||||
public class DocValuesFieldExistsQuery implements Query {
|
||||
|
||||
private final String field;
|
||||
|
||||
public DocValuesFieldExistsQuery(String field) {
|
||||
this.field = field;
|
||||
}
|
||||
|
||||
public String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stringify(StringBuilder output) {
|
||||
StringBuilder data = new StringBuilder();
|
||||
StringifyUtils.stringifyString(data, field);
|
||||
StringifyUtils.writeHeader(output, QueryConstructorType.DOC_VALUES_FIELD_EXISTS_QUERY, data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "(existence of field " + field + ")";
|
||||
}
|
||||
}
|
@ -8,6 +8,7 @@ public enum QueryConstructorType {
|
||||
INT_POINT_EXACT_QUERY,
|
||||
LONG_POINT_EXACT_QUERY,
|
||||
TERM_QUERY,
|
||||
DOC_VALUES_FIELD_EXISTS_QUERY,
|
||||
FUZZY_QUERY,
|
||||
PHRASE_QUERY,
|
||||
SYNONYM_QUERY,
|
||||
|
@ -12,6 +12,7 @@ import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.BoostQuery;
|
||||
import org.apache.lucene.search.DocValuesFieldExistsQuery;
|
||||
import org.apache.lucene.search.FuzzyQuery;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
@ -185,6 +186,10 @@ public class QueryParser {
|
||||
Term term2 = (Term) parse(completeText, position);
|
||||
assert term2 != null;
|
||||
return new TermQuery(term2);
|
||||
case DOC_VALUES_FIELD_EXISTS_QUERY:
|
||||
String fieldKey = (String) parse(completeText, position);
|
||||
assert fieldKey != null;
|
||||
return new DocValuesFieldExistsQuery(fieldKey);
|
||||
case FLOAT:
|
||||
position.addAndGet(toParse.length());
|
||||
return Float.parseFloat(toParse);
|
||||
|
Loading…
x
Reference in New Issue
Block a user