mirror of https://github.com/apache/lucene.git
Merge branch 'apache-https-master' into jira/solr-8593
This commit is contained in:
commit
db6a3393a9
|
@ -63,16 +63,42 @@ Other
|
|||
|
||||
======================= Lucene 6.5.0 =======================
|
||||
|
||||
API Changes
|
||||
|
||||
* LUCENE-7624: TermsQuery has been renamed as TermInSetQuery and moved to core.
|
||||
(Alan Woodward)
|
||||
|
||||
* LUCENE-7637: TermInSetQuery requires that all terms come from the same field.
|
||||
(Adrien Grand)
|
||||
|
||||
New Features
|
||||
|
||||
* LUCENE-7623: Add FunctionScoreQuery and FunctionMatchQuery (Alan Woodward,
|
||||
Adrien Grand, David Smiley)
|
||||
|
||||
* LUCENE-7619: Add WordDelimiterGraphFilter, just like
|
||||
WordDelimiterFilter except it produces correct token graphs so that
|
||||
proximity queries at search time will produce correct results (Mike
|
||||
McCandless)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-7630: Fix (Edge)NGramTokenFilter to no longer drop payloads
|
||||
and preserve all attributes. (Nathan Gass via Uwe Schindler)
|
||||
|
||||
Improvements
|
||||
|
||||
* LUCENE-7055: Added Weight#scorerSupplier, which allows to estimate the cost
|
||||
of a Scorer before actually building it, in order to optimize how the query
|
||||
should be run, eg. using points or doc values depending on costs of other
|
||||
parts of the query. (Adrien Grand)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-7641: Optimized point range queries to compute documents that do not
|
||||
match the range on single-valued fields when more than half the documents in
|
||||
the index would match. (Adrien Grand)
|
||||
|
||||
======================= Lucene 6.4.0 =======================
|
||||
|
||||
API Changes
|
||||
|
@ -100,9 +126,6 @@ API Changes
|
|||
* LUCENE-7611: DocumentValueSourceDictionary now takes a LongValuesSource
|
||||
as a parameter, and the ValueSource equivalent is deprecated (Alan Woodward)
|
||||
|
||||
* LUCENE-7624: TermsQuery has been renamed as TermInSetQuery and moved to core.
|
||||
(Alan Woodward)
|
||||
|
||||
New features
|
||||
|
||||
* LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)
|
||||
|
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.analysis.synonym;
|
||||
package org.apache.lucene.analysis.core;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -23,6 +23,7 @@ import java.util.List;
|
|||
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.synonym.SynonymGraphFilter;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
||||
|
@ -410,8 +411,8 @@ public final class FlattenGraphFilter extends TokenFilter {
|
|||
maxLookaheadUsed = 0;
|
||||
}
|
||||
|
||||
// for testing
|
||||
int getMaxLookaheadUsed() {
|
||||
/** For testing */
|
||||
public int getMaxLookaheadUsed() {
|
||||
return maxLookaheadUsed;
|
||||
}
|
||||
}
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.analysis.synonym;
|
||||
package org.apache.lucene.analysis.core;
|
||||
|
||||
import java.util.Map;
|
||||
|
|
@ -28,6 +28,7 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.InPlaceMergeSorter;
|
||||
|
@ -80,7 +81,12 @@ import org.apache.lucene.util.InPlaceMergeSorter;
|
|||
* the current {@link StandardTokenizer} immediately removes many intra-word
|
||||
* delimiters, it is recommended that this filter be used after a tokenizer that
|
||||
* does not do this (such as {@link WhitespaceTokenizer}).
|
||||
*
|
||||
* @deprecated Use {@link WordDelimiterGraphFilter} instead: it produces a correct
|
||||
* token graph so that e.g. {@link PhraseQuery} works correctly when it's used in
|
||||
* the search time analyzer.
|
||||
*/
|
||||
@Deprecated
|
||||
public final class WordDelimiterFilter extends TokenFilter {
|
||||
|
||||
public static final int LOWER = 0x01;
|
||||
|
@ -116,7 +122,7 @@ public final class WordDelimiterFilter extends TokenFilter {
|
|||
/**
|
||||
* Causes maximum runs of word parts to be catenated:
|
||||
* <p>
|
||||
* "wi-fi" => "wifi"
|
||||
* "500-42" => "50042"
|
||||
*/
|
||||
public static final int CATENATE_NUMBERS = 8;
|
||||
|
||||
|
@ -494,7 +500,6 @@ public final class WordDelimiterFilter extends TokenFilter {
|
|||
private void generatePart(boolean isSingleWord) {
|
||||
clearAttributes();
|
||||
termAttribute.copyBuffer(savedBuffer, iterator.current, iterator.end - iterator.current);
|
||||
|
||||
int startOffset = savedStartOffset + iterator.current;
|
||||
int endOffset = savedStartOffset + iterator.end;
|
||||
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.lucene.analysis.TokenStream;
|
|||
import org.apache.lucene.analysis.util.ResourceLoader;
|
||||
import org.apache.lucene.analysis.util.ResourceLoaderAware;
|
||||
import org.apache.lucene.analysis.util.TokenFilterFactory;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
|
||||
import static org.apache.lucene.analysis.miscellaneous.WordDelimiterFilter.*;
|
||||
|
||||
|
@ -47,7 +48,12 @@ import static org.apache.lucene.analysis.miscellaneous.WordDelimiterFilter.*;
|
|||
* types="wdfftypes.txt" />
|
||||
* </analyzer>
|
||||
* </fieldType></pre>
|
||||
*
|
||||
* @deprecated Use {@link WordDelimiterGraphFilterFactory} instead: it produces a correct
|
||||
* token graph so that e.g. {@link PhraseQuery} works correctly when it's used in
|
||||
* the search time analyzer.
|
||||
*/
|
||||
@Deprecated
|
||||
public class WordDelimiterFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
|
||||
public static final String PROTECTED_TOKENS = "protected";
|
||||
public static final String TYPES = "types";
|
||||
|
|
|
@ -0,0 +1,706 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You 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,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.miscellaneous;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.CharArraySet;
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.core.WhitespaceTokenizer;
|
||||
import org.apache.lucene.analysis.standard.StandardTokenizer;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.InPlaceMergeSorter;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* Splits words into subwords and performs optional transformations on subword
|
||||
* groups, producing a correct token graph so that e.g. {@link PhraseQuery} can
|
||||
* work correctly when this filter is used in the search-time analyzer. Unlike
|
||||
* the deprecated {@link WordDelimiterFilter}, this token filter produces a
|
||||
* correct token graph as output. However, it cannot consume an input token
|
||||
* graph correctly.
|
||||
*
|
||||
* <p>
|
||||
* Words are split into subwords with the following rules:
|
||||
* <ul>
|
||||
* <li>split on intra-word delimiters (by default, all non alpha-numeric
|
||||
* characters): <code>"Wi-Fi"</code> → <code>"Wi", "Fi"</code></li>
|
||||
* <li>split on case transitions: <code>"PowerShot"</code> →
|
||||
* <code>"Power", "Shot"</code></li>
|
||||
* <li>split on letter-number transitions: <code>"SD500"</code> →
|
||||
* <code>"SD", "500"</code></li>
|
||||
* <li>leading and trailing intra-word delimiters on each subword are ignored:
|
||||
* <code>"//hello---there, 'dude'"</code> →
|
||||
* <code>"hello", "there", "dude"</code></li>
|
||||
* <li>trailing "'s" are removed for each subword: <code>"O'Neil's"</code>
|
||||
* → <code>"O", "Neil"</code>
|
||||
* <ul>
|
||||
* <li>Note: this step isn't performed in a separate filter because of possible
|
||||
* subword combinations.</li>
|
||||
* </ul>
|
||||
* </li>
|
||||
* </ul>
|
||||
*
|
||||
* The <b>combinations</b> parameter affects how subwords are combined:
|
||||
* <ul>
|
||||
* <li>combinations="0" causes no subword combinations: <code>"PowerShot"</code>
|
||||
* → <code>0:"Power", 1:"Shot"</code> (0 and 1 are the token positions)</li>
|
||||
* <li>combinations="1" means that in addition to the subwords, maximum runs of
|
||||
* non-numeric subwords are catenated and produced at the same position of the
|
||||
* last subword in the run:
|
||||
* <ul>
|
||||
* <li><code>"PowerShot"</code> →
|
||||
* <code>0:"Power", 1:"Shot" 1:"PowerShot"</code></li>
|
||||
* <li><code>"A's+B's&C's"</code> > <code>0:"A", 1:"B", 2:"C", 2:"ABC"</code>
|
||||
* </li>
|
||||
* <li><code>"Super-Duper-XL500-42-AutoCoder!"</code> →
|
||||
* <code>0:"Super", 1:"Duper", 2:"XL", 2:"SuperDuperXL", 3:"500" 4:"42", 5:"Auto", 6:"Coder", 6:"AutoCoder"</code>
|
||||
* </li>
|
||||
* </ul>
|
||||
* </li>
|
||||
* </ul>
|
||||
* One use for {@link WordDelimiterGraphFilter} is to help match words with different
|
||||
* subword delimiters. For example, if the source text contained "wi-fi" one may
|
||||
* want "wifi" "WiFi" "wi-fi" "wi+fi" queries to all match. One way of doing so
|
||||
* is to specify combinations="1" in the analyzer used for indexing, and
|
||||
* combinations="0" (the default) in the analyzer used for querying. Given that
|
||||
* the current {@link StandardTokenizer} immediately removes many intra-word
|
||||
* delimiters, it is recommended that this filter be used after a tokenizer that
|
||||
* does not do this (such as {@link WhitespaceTokenizer}).
|
||||
*/
|
||||
|
||||
public final class WordDelimiterGraphFilter extends TokenFilter {
|
||||
|
||||
/**
|
||||
* Causes parts of words to be generated:
|
||||
* <p>
|
||||
* "PowerShot" => "Power" "Shot"
|
||||
*/
|
||||
public static final int GENERATE_WORD_PARTS = 1;
|
||||
|
||||
/**
|
||||
* Causes number subwords to be generated:
|
||||
* <p>
|
||||
* "500-42" => "500" "42"
|
||||
*/
|
||||
public static final int GENERATE_NUMBER_PARTS = 2;
|
||||
|
||||
/**
|
||||
* Causes maximum runs of word parts to be catenated:
|
||||
* <p>
|
||||
* "wi-fi" => "wifi"
|
||||
*/
|
||||
public static final int CATENATE_WORDS = 4;
|
||||
|
||||
/**
|
||||
* Causes maximum runs of number parts to be catenated:
|
||||
* <p>
|
||||
* "500-42" => "50042"
|
||||
*/
|
||||
public static final int CATENATE_NUMBERS = 8;
|
||||
|
||||
/**
|
||||
* Causes all subword parts to be catenated:
|
||||
* <p>
|
||||
* "wi-fi-4000" => "wifi4000"
|
||||
*/
|
||||
public static final int CATENATE_ALL = 16;
|
||||
|
||||
/**
|
||||
* Causes original words are preserved and added to the subword list (Defaults to false)
|
||||
* <p>
|
||||
* "500-42" => "500" "42" "500-42"
|
||||
*/
|
||||
public static final int PRESERVE_ORIGINAL = 32;
|
||||
|
||||
/**
|
||||
* Causes lowercase -> uppercase transition to start a new subword.
|
||||
*/
|
||||
public static final int SPLIT_ON_CASE_CHANGE = 64;
|
||||
|
||||
/**
|
||||
* If not set, causes numeric changes to be ignored (subwords will only be generated
|
||||
* given SUBWORD_DELIM tokens).
|
||||
*/
|
||||
public static final int SPLIT_ON_NUMERICS = 128;
|
||||
|
||||
/**
|
||||
* Causes trailing "'s" to be removed for each subword
|
||||
* <p>
|
||||
* "O'Neil's" => "O", "Neil"
|
||||
*/
|
||||
public static final int STEM_ENGLISH_POSSESSIVE = 256;
|
||||
|
||||
/**
|
||||
* If not null is the set of tokens to protect from being delimited
|
||||
*
|
||||
*/
|
||||
final CharArraySet protWords;
|
||||
|
||||
private final int flags;
|
||||
|
||||
// packs start pos, end pos, start part, end part (= slice of the term text) for each buffered part:
|
||||
private int[] bufferedParts = new int[16];
|
||||
private int bufferedLen;
|
||||
private int bufferedPos;
|
||||
|
||||
// holds text for each buffered part, or null if it's a simple slice of the original term
|
||||
private char[][] bufferedTermParts = new char[4][];
|
||||
|
||||
private final CharTermAttribute termAttribute = addAttribute(CharTermAttribute.class);
|
||||
private final OffsetAttribute offsetAttribute = addAttribute(OffsetAttribute.class);
|
||||
private final PositionIncrementAttribute posIncAttribute = addAttribute(PositionIncrementAttribute.class);
|
||||
private final PositionLengthAttribute posLenAttribute = addAttribute(PositionLengthAttribute.class);
|
||||
private final TypeAttribute typeAttribute = addAttribute(TypeAttribute.class);
|
||||
|
||||
// used for iterating word delimiter breaks
|
||||
private final WordDelimiterIterator iterator;
|
||||
|
||||
// used for concatenating runs of similar typed subwords (word,number)
|
||||
private final WordDelimiterConcatenation concat = new WordDelimiterConcatenation();
|
||||
|
||||
// number of subwords last output by concat.
|
||||
private int lastConcatCount;
|
||||
|
||||
// used for catenate all
|
||||
private final WordDelimiterConcatenation concatAll = new WordDelimiterConcatenation();
|
||||
|
||||
// used for accumulating position increment gaps so that we preserve incoming holes:
|
||||
private int accumPosInc;
|
||||
|
||||
private char[] savedTermBuffer = new char[16];
|
||||
private int savedTermLength;
|
||||
private int savedStartOffset;
|
||||
private int savedEndOffset;
|
||||
private AttributeSource.State savedState;
|
||||
private int lastStartOffset;
|
||||
|
||||
// if length by start + end offsets doesn't match the term text then assume
|
||||
// this is a synonym and don't adjust the offsets.
|
||||
private boolean hasIllegalOffsets;
|
||||
|
||||
private int wordPos;
|
||||
|
||||
/**
|
||||
* Creates a new WordDelimiterGraphFilter
|
||||
*
|
||||
* @param in TokenStream to be filtered
|
||||
* @param charTypeTable table containing character types
|
||||
* @param configurationFlags Flags configuring the filter
|
||||
* @param protWords If not null is the set of tokens to protect from being delimited
|
||||
*/
|
||||
public WordDelimiterGraphFilter(TokenStream in, byte[] charTypeTable, int configurationFlags, CharArraySet protWords) {
|
||||
super(in);
|
||||
if ((configurationFlags &
|
||||
~(GENERATE_WORD_PARTS |
|
||||
GENERATE_NUMBER_PARTS |
|
||||
CATENATE_WORDS |
|
||||
CATENATE_NUMBERS |
|
||||
CATENATE_ALL |
|
||||
PRESERVE_ORIGINAL |
|
||||
SPLIT_ON_CASE_CHANGE |
|
||||
SPLIT_ON_NUMERICS |
|
||||
STEM_ENGLISH_POSSESSIVE)) != 0) {
|
||||
throw new IllegalArgumentException("flags contains unrecognized flag: " + configurationFlags);
|
||||
}
|
||||
this.flags = configurationFlags;
|
||||
this.protWords = protWords;
|
||||
this.iterator = new WordDelimiterIterator(
|
||||
charTypeTable, has(SPLIT_ON_CASE_CHANGE), has(SPLIT_ON_NUMERICS), has(STEM_ENGLISH_POSSESSIVE));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new WordDelimiterGraphFilter using {@link WordDelimiterIterator#DEFAULT_WORD_DELIM_TABLE}
|
||||
* as its charTypeTable
|
||||
*
|
||||
* @param in TokenStream to be filtered
|
||||
* @param configurationFlags Flags configuring the filter
|
||||
* @param protWords If not null is the set of tokens to protect from being delimited
|
||||
*/
|
||||
public WordDelimiterGraphFilter(TokenStream in, int configurationFlags, CharArraySet protWords) {
|
||||
this(in, WordDelimiterIterator.DEFAULT_WORD_DELIM_TABLE, configurationFlags, protWords);
|
||||
}
|
||||
|
||||
/** Iterates all words parts and concatenations, buffering up the term parts we should return. */
|
||||
private void bufferWordParts() throws IOException {
|
||||
|
||||
saveState();
|
||||
|
||||
// if length by start + end offsets doesn't match the term's text then set offsets for all our word parts/concats to the incoming
|
||||
// offsets. this can happen if WDGF is applied to an injected synonym, or to a stem'd form, etc:
|
||||
hasIllegalOffsets = (savedEndOffset - savedStartOffset != savedTermLength);
|
||||
|
||||
bufferedLen = 0;
|
||||
lastConcatCount = 0;
|
||||
wordPos = 0;
|
||||
|
||||
if (iterator.isSingleWord()) {
|
||||
buffer(wordPos, wordPos+1, iterator.current, iterator.end);
|
||||
wordPos++;
|
||||
iterator.next();
|
||||
} else {
|
||||
|
||||
// iterate all words parts, possibly buffering them, building up concatenations and possibly buffering them too:
|
||||
while (iterator.end != WordDelimiterIterator.DONE) {
|
||||
int wordType = iterator.type();
|
||||
|
||||
// do we already have queued up incompatible concatenations?
|
||||
if (concat.isNotEmpty() && (concat.type & wordType) == 0) {
|
||||
flushConcatenation(concat);
|
||||
}
|
||||
|
||||
// add subwords depending upon options
|
||||
if (shouldConcatenate(wordType)) {
|
||||
concatenate(concat);
|
||||
}
|
||||
|
||||
// add all subwords (catenateAll)
|
||||
if (has(CATENATE_ALL)) {
|
||||
concatenate(concatAll);
|
||||
}
|
||||
|
||||
// if we should output the word or number part
|
||||
if (shouldGenerateParts(wordType)) {
|
||||
buffer(wordPos, wordPos+1, iterator.current, iterator.end);
|
||||
wordPos++;
|
||||
}
|
||||
iterator.next();
|
||||
}
|
||||
|
||||
if (concat.isNotEmpty()) {
|
||||
// flush final concatenation
|
||||
flushConcatenation(concat);
|
||||
}
|
||||
|
||||
if (concatAll.isNotEmpty()) {
|
||||
// only if we haven't output this same combo above, e.g. PowerShot with CATENATE_WORDS:
|
||||
if (concatAll.subwordCount > lastConcatCount) {
|
||||
if (wordPos == concatAll.startPos) {
|
||||
// we are not generating parts, so we must advance wordPos now
|
||||
wordPos++;
|
||||
}
|
||||
concatAll.write();
|
||||
}
|
||||
concatAll.clear();
|
||||
}
|
||||
}
|
||||
|
||||
if (has(PRESERVE_ORIGINAL)) {
|
||||
if (wordPos == 0) {
|
||||
// can happen w/ strange flag combos and inputs :)
|
||||
wordPos++;
|
||||
}
|
||||
// add the original token now so that we can set the correct end position
|
||||
buffer(0, wordPos, 0, savedTermLength);
|
||||
}
|
||||
|
||||
sorter.sort(0, bufferedLen);
|
||||
wordPos = 0;
|
||||
|
||||
// set back to 0 for iterating from the buffer
|
||||
bufferedPos = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
while (true) {
|
||||
if (savedState == null) {
|
||||
|
||||
// process a new input token
|
||||
if (input.incrementToken() == false) {
|
||||
return false;
|
||||
}
|
||||
|
||||
int termLength = termAttribute.length();
|
||||
char[] termBuffer = termAttribute.buffer();
|
||||
|
||||
accumPosInc += posIncAttribute.getPositionIncrement();
|
||||
|
||||
// iterate & cache all word parts up front:
|
||||
iterator.setText(termBuffer, termLength);
|
||||
iterator.next();
|
||||
|
||||
// word of no delimiters, or protected word: just return it
|
||||
if ((iterator.current == 0 && iterator.end == termLength) ||
|
||||
(protWords != null && protWords.contains(termBuffer, 0, termLength))) {
|
||||
posIncAttribute.setPositionIncrement(accumPosInc);
|
||||
accumPosInc = 0;
|
||||
return true;
|
||||
}
|
||||
|
||||
// word of simply delimiters: swallow this token, creating a hole, and move on to next token
|
||||
if (iterator.end == WordDelimiterIterator.DONE) {
|
||||
if (has(PRESERVE_ORIGINAL) == false) {
|
||||
continue;
|
||||
} else {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
// otherwise, we have delimiters, process & buffer all parts:
|
||||
bufferWordParts();
|
||||
}
|
||||
|
||||
if (bufferedPos < bufferedLen) {
|
||||
clearAttributes();
|
||||
restoreState(savedState);
|
||||
|
||||
char[] termPart = bufferedTermParts[bufferedPos];
|
||||
int startPos = bufferedParts[4*bufferedPos];
|
||||
int endPos = bufferedParts[4*bufferedPos+1];
|
||||
int startPart = bufferedParts[4*bufferedPos+2];
|
||||
int endPart = bufferedParts[4*bufferedPos+3];
|
||||
bufferedPos++;
|
||||
|
||||
int startOffset;
|
||||
int endOffset;
|
||||
|
||||
if (hasIllegalOffsets) {
|
||||
startOffset = savedStartOffset;
|
||||
endOffset = savedEndOffset;
|
||||
} else {
|
||||
startOffset = savedStartOffset + startPart;
|
||||
endOffset = savedStartOffset + endPart;
|
||||
}
|
||||
|
||||
// never let offsets go backwards:
|
||||
startOffset = Math.max(startOffset, lastStartOffset);
|
||||
endOffset = Math.max(endOffset, lastStartOffset);
|
||||
|
||||
offsetAttribute.setOffset(startOffset, endOffset);
|
||||
lastStartOffset = startOffset;
|
||||
|
||||
if (termPart == null) {
|
||||
termAttribute.copyBuffer(savedTermBuffer, startPart, endPart - startPart);
|
||||
} else {
|
||||
termAttribute.copyBuffer(termPart, 0, termPart.length);
|
||||
}
|
||||
|
||||
posIncAttribute.setPositionIncrement(accumPosInc + startPos - wordPos);
|
||||
accumPosInc = 0;
|
||||
posLenAttribute.setPositionLength(endPos - startPos);
|
||||
wordPos = startPos;
|
||||
return true;
|
||||
}
|
||||
|
||||
// no saved concatenations, on to the next input word
|
||||
savedState = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
super.reset();
|
||||
accumPosInc = 0;
|
||||
savedState = null;
|
||||
lastStartOffset = 0;
|
||||
concat.clear();
|
||||
concatAll.clear();
|
||||
}
|
||||
|
||||
// ================================================= Helper Methods ================================================
|
||||
|
||||
private class PositionSorter extends InPlaceMergeSorter {
|
||||
@Override
|
||||
protected int compare(int i, int j) {
|
||||
// sort by smaller start position
|
||||
int iPosStart = bufferedParts[4*i];
|
||||
int jPosStart = bufferedParts[4*j];
|
||||
int cmp = Integer.compare(iPosStart, jPosStart);
|
||||
if (cmp != 0) {
|
||||
return cmp;
|
||||
}
|
||||
|
||||
// tie break by longest pos length:
|
||||
int iPosEnd = bufferedParts[4*i+1];
|
||||
int jPosEnd = bufferedParts[4*j+1];
|
||||
return Integer.compare(jPosEnd, iPosEnd);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void swap(int i, int j) {
|
||||
int iOffset = 4*i;
|
||||
int jOffset = 4*j;
|
||||
for(int x=0;x<4;x++) {
|
||||
int tmp = bufferedParts[iOffset+x];
|
||||
bufferedParts[iOffset+x] = bufferedParts[jOffset+x];
|
||||
bufferedParts[jOffset+x] = tmp;
|
||||
}
|
||||
|
||||
char[] tmp2 = bufferedTermParts[i];
|
||||
bufferedTermParts[i] = bufferedTermParts[j];
|
||||
bufferedTermParts[j] = tmp2;
|
||||
}
|
||||
}
|
||||
|
||||
final PositionSorter sorter = new PositionSorter();
|
||||
|
||||
/**
|
||||
* startPos, endPos -> graph start/end position
|
||||
* startPart, endPart -> slice of the original term for this part
|
||||
*/
|
||||
|
||||
void buffer(int startPos, int endPos, int startPart, int endPart) {
|
||||
buffer(null, startPos, endPos, startPart, endPart);
|
||||
}
|
||||
|
||||
/**
|
||||
* a null termPart means it's a simple slice of the original term
|
||||
*/
|
||||
void buffer(char[] termPart, int startPos, int endPos, int startPart, int endPart) {
|
||||
/*
|
||||
System.out.println("buffer: pos=" + startPos + "-" + endPos + " part=" + startPart + "-" + endPart);
|
||||
if (termPart != null) {
|
||||
System.out.println(" termIn=" + new String(termPart));
|
||||
} else {
|
||||
System.out.println(" term=" + new String(savedTermBuffer, startPart, endPart-startPart));
|
||||
}
|
||||
*/
|
||||
assert endPos > startPos: "startPos=" + startPos + " endPos=" + endPos;
|
||||
assert endPart > startPart || (endPart == 0 && startPart == 0 && savedTermLength == 0): "startPart=" + startPart + " endPart=" + endPart;
|
||||
if ((bufferedLen+1)*4 > bufferedParts.length) {
|
||||
bufferedParts = ArrayUtil.grow(bufferedParts, (bufferedLen+1)*4);
|
||||
}
|
||||
if (bufferedTermParts.length == bufferedLen) {
|
||||
int newSize = ArrayUtil.oversize(bufferedLen+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
|
||||
char[][] newArray = new char[newSize][];
|
||||
System.arraycopy(bufferedTermParts, 0, newArray, 0, bufferedTermParts.length);
|
||||
bufferedTermParts = newArray;
|
||||
}
|
||||
bufferedTermParts[bufferedLen] = termPart;
|
||||
bufferedParts[bufferedLen*4] = startPos;
|
||||
bufferedParts[bufferedLen*4+1] = endPos;
|
||||
bufferedParts[bufferedLen*4+2] = startPart;
|
||||
bufferedParts[bufferedLen*4+3] = endPart;
|
||||
bufferedLen++;
|
||||
}
|
||||
|
||||
/**
|
||||
* Saves the existing attribute states
|
||||
*/
|
||||
private void saveState() {
|
||||
savedTermLength = termAttribute.length();
|
||||
savedStartOffset = offsetAttribute.startOffset();
|
||||
savedEndOffset = offsetAttribute.endOffset();
|
||||
savedState = captureState();
|
||||
|
||||
if (savedTermBuffer.length < savedTermLength) {
|
||||
savedTermBuffer = new char[ArrayUtil.oversize(savedTermLength, Character.BYTES)];
|
||||
}
|
||||
|
||||
System.arraycopy(termAttribute.buffer(), 0, savedTermBuffer, 0, savedTermLength);
|
||||
}
|
||||
|
||||
/**
|
||||
* Flushes the given WordDelimiterConcatenation by either writing its concat and then clearing, or just clearing.
|
||||
*
|
||||
* @param concat WordDelimiterConcatenation that will be flushed
|
||||
*/
|
||||
private void flushConcatenation(WordDelimiterConcatenation concat) {
|
||||
if (wordPos == concat.startPos) {
|
||||
// we are not generating parts, so we must advance wordPos now
|
||||
wordPos++;
|
||||
}
|
||||
lastConcatCount = concat.subwordCount;
|
||||
if (concat.subwordCount != 1 || shouldGenerateParts(concat.type) == false) {
|
||||
concat.write();
|
||||
}
|
||||
concat.clear();
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether to concatenate a word or number if the current word is the given type
|
||||
*
|
||||
* @param wordType Type of the current word used to determine if it should be concatenated
|
||||
* @return {@code true} if concatenation should occur, {@code false} otherwise
|
||||
*/
|
||||
private boolean shouldConcatenate(int wordType) {
|
||||
return (has(CATENATE_WORDS) && WordDelimiterIterator.isAlpha(wordType)) || (has(CATENATE_NUMBERS) && WordDelimiterIterator.isDigit(wordType));
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether a word/number part should be generated for a word of the given type
|
||||
*
|
||||
* @param wordType Type of the word used to determine if a word/number part should be generated
|
||||
* @return {@code true} if a word/number part should be generated, {@code false} otherwise
|
||||
*/
|
||||
private boolean shouldGenerateParts(int wordType) {
|
||||
return (has(GENERATE_WORD_PARTS) && WordDelimiterIterator.isAlpha(wordType)) || (has(GENERATE_NUMBER_PARTS) && WordDelimiterIterator.isDigit(wordType));
|
||||
}
|
||||
|
||||
/**
|
||||
* Concatenates the saved buffer to the given WordDelimiterConcatenation
|
||||
*
|
||||
* @param concatenation WordDelimiterConcatenation to concatenate the buffer to
|
||||
*/
|
||||
private void concatenate(WordDelimiterConcatenation concatenation) {
|
||||
if (concatenation.isEmpty()) {
|
||||
concatenation.type = iterator.type();
|
||||
concatenation.startPart = iterator.current;
|
||||
concatenation.startPos = wordPos;
|
||||
}
|
||||
concatenation.append(savedTermBuffer, iterator.current, iterator.end - iterator.current);
|
||||
concatenation.endPart = iterator.end;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the given flag is set
|
||||
*
|
||||
* @param flag Flag to see if set
|
||||
* @return {@code true} if flag is set
|
||||
*/
|
||||
private boolean has(int flag) {
|
||||
return (flags & flag) != 0;
|
||||
}
|
||||
|
||||
// ================================================= Inner Classes =================================================
|
||||
|
||||
/**
|
||||
* A WDF concatenated 'run'
|
||||
*/
|
||||
final class WordDelimiterConcatenation {
|
||||
final StringBuilder buffer = new StringBuilder();
|
||||
int startPart;
|
||||
int endPart;
|
||||
int startPos;
|
||||
int type;
|
||||
int subwordCount;
|
||||
|
||||
/**
|
||||
* Appends the given text of the given length, to the concetenation at the given offset
|
||||
*
|
||||
* @param text Text to append
|
||||
* @param offset Offset in the concetenation to add the text
|
||||
* @param length Length of the text to append
|
||||
*/
|
||||
void append(char text[], int offset, int length) {
|
||||
buffer.append(text, offset, length);
|
||||
subwordCount++;
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes the concatenation to part buffer
|
||||
*/
|
||||
void write() {
|
||||
char[] termPart = new char[buffer.length()];
|
||||
buffer.getChars(0, buffer.length(), termPart, 0);
|
||||
buffer(termPart, startPos, wordPos, startPart, endPart);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines if the concatenation is empty
|
||||
*
|
||||
* @return {@code true} if the concatenation is empty, {@code false} otherwise
|
||||
*/
|
||||
boolean isEmpty() {
|
||||
return buffer.length() == 0;
|
||||
}
|
||||
|
||||
boolean isNotEmpty() {
|
||||
return isEmpty() == false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Clears the concatenation and resets its state
|
||||
*/
|
||||
void clear() {
|
||||
buffer.setLength(0);
|
||||
startPart = endPart = type = subwordCount = 0;
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns string representation of configuration flags */
|
||||
public static String flagsToString(int flags) {
|
||||
StringBuilder b = new StringBuilder();
|
||||
if ((flags & GENERATE_WORD_PARTS) != 0) {
|
||||
b.append("GENERATE_WORD_PARTS");
|
||||
}
|
||||
if ((flags & GENERATE_NUMBER_PARTS) != 0) {
|
||||
if (b.length() > 0) {
|
||||
b.append(" | ");
|
||||
}
|
||||
b.append("GENERATE_NUMBER_PARTS");
|
||||
}
|
||||
if ((flags & CATENATE_WORDS) != 0) {
|
||||
if (b.length() > 0) {
|
||||
b.append(" | ");
|
||||
}
|
||||
b.append("CATENATE_WORDS");
|
||||
}
|
||||
if ((flags & CATENATE_NUMBERS) != 0) {
|
||||
if (b.length() > 0) {
|
||||
b.append(" | ");
|
||||
}
|
||||
b.append("CATENATE_NUMBERS");
|
||||
}
|
||||
if ((flags & CATENATE_ALL) != 0) {
|
||||
if (b.length() > 0) {
|
||||
b.append(" | ");
|
||||
}
|
||||
b.append("CATENATE_ALL");
|
||||
}
|
||||
if ((flags & PRESERVE_ORIGINAL) != 0) {
|
||||
if (b.length() > 0) {
|
||||
b.append(" | ");
|
||||
}
|
||||
b.append("PRESERVE_ORIGINAL");
|
||||
}
|
||||
if ((flags & SPLIT_ON_CASE_CHANGE) != 0) {
|
||||
if (b.length() > 0) {
|
||||
b.append(" | ");
|
||||
}
|
||||
b.append("SPLIT_ON_CASE_CHANGE");
|
||||
}
|
||||
if ((flags & SPLIT_ON_NUMERICS) != 0) {
|
||||
if (b.length() > 0) {
|
||||
b.append(" | ");
|
||||
}
|
||||
b.append("SPLIT_ON_NUMERICS");
|
||||
}
|
||||
if ((flags & STEM_ENGLISH_POSSESSIVE) != 0) {
|
||||
if (b.length() > 0) {
|
||||
b.append(" | ");
|
||||
}
|
||||
b.append("STEM_ENGLISH_POSSESSIVE");
|
||||
}
|
||||
|
||||
return b.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder b = new StringBuilder();
|
||||
b.append("WordDelimiterGraphFilter(flags=");
|
||||
b.append(flagsToString(flags));
|
||||
b.append(')');
|
||||
return b.toString();
|
||||
}
|
||||
|
||||
// questions:
|
||||
// negative numbers? -42 indexed as just 42?
|
||||
// dollar sign? $42
|
||||
// percent sign? 33%
|
||||
// downsides: if source text is "powershot" then a query of "PowerShot" won't match!
|
||||
}
|
|
@ -0,0 +1,199 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You 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,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.miscellaneous;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.lucene.analysis.CharArraySet;
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.util.ResourceLoader;
|
||||
import org.apache.lucene.analysis.util.ResourceLoaderAware;
|
||||
import org.apache.lucene.analysis.util.TokenFilterFactory;
|
||||
|
||||
import static org.apache.lucene.analysis.miscellaneous.WordDelimiterGraphFilter.*;
|
||||
import static org.apache.lucene.analysis.miscellaneous.WordDelimiterIterator.*;
|
||||
|
||||
/**
|
||||
* Factory for {@link WordDelimiterGraphFilter}.
|
||||
* <pre class="prettyprint">
|
||||
* <fieldType name="text_wd" class="solr.TextField" positionIncrementGap="100">
|
||||
* <analyzer>
|
||||
* <tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
* <filter class="solr.WordDelimiterGraphFilterFactory" protected="protectedword.txt"
|
||||
* preserveOriginal="0" splitOnNumerics="1" splitOnCaseChange="1"
|
||||
* catenateWords="0" catenateNumbers="0" catenateAll="0"
|
||||
* generateWordParts="1" generateNumberParts="1" stemEnglishPossessive="1"
|
||||
* types="wdfftypes.txt" />
|
||||
* </analyzer>
|
||||
* </fieldType></pre>
|
||||
*/
|
||||
public class WordDelimiterGraphFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
|
||||
public static final String PROTECTED_TOKENS = "protected";
|
||||
public static final String TYPES = "types";
|
||||
|
||||
private final String wordFiles;
|
||||
private final String types;
|
||||
private final int flags;
|
||||
byte[] typeTable = null;
|
||||
private CharArraySet protectedWords = null;
|
||||
|
||||
/** Creates a new WordDelimiterGraphFilterFactory */
|
||||
public WordDelimiterGraphFilterFactory(Map<String, String> args) {
|
||||
super(args);
|
||||
int flags = 0;
|
||||
if (getInt(args, "generateWordParts", 1) != 0) {
|
||||
flags |= GENERATE_WORD_PARTS;
|
||||
}
|
||||
if (getInt(args, "generateNumberParts", 1) != 0) {
|
||||
flags |= GENERATE_NUMBER_PARTS;
|
||||
}
|
||||
if (getInt(args, "catenateWords", 0) != 0) {
|
||||
flags |= CATENATE_WORDS;
|
||||
}
|
||||
if (getInt(args, "catenateNumbers", 0) != 0) {
|
||||
flags |= CATENATE_NUMBERS;
|
||||
}
|
||||
if (getInt(args, "catenateAll", 0) != 0) {
|
||||
flags |= CATENATE_ALL;
|
||||
}
|
||||
if (getInt(args, "splitOnCaseChange", 1) != 0) {
|
||||
flags |= SPLIT_ON_CASE_CHANGE;
|
||||
}
|
||||
if (getInt(args, "splitOnNumerics", 1) != 0) {
|
||||
flags |= SPLIT_ON_NUMERICS;
|
||||
}
|
||||
if (getInt(args, "preserveOriginal", 0) != 0) {
|
||||
flags |= PRESERVE_ORIGINAL;
|
||||
}
|
||||
if (getInt(args, "stemEnglishPossessive", 1) != 0) {
|
||||
flags |= STEM_ENGLISH_POSSESSIVE;
|
||||
}
|
||||
wordFiles = get(args, PROTECTED_TOKENS);
|
||||
types = get(args, TYPES);
|
||||
this.flags = flags;
|
||||
if (!args.isEmpty()) {
|
||||
throw new IllegalArgumentException("Unknown parameters: " + args);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inform(ResourceLoader loader) throws IOException {
|
||||
if (wordFiles != null) {
|
||||
protectedWords = getWordSet(loader, wordFiles, false);
|
||||
}
|
||||
if (types != null) {
|
||||
List<String> files = splitFileNames( types );
|
||||
List<String> wlist = new ArrayList<>();
|
||||
for( String file : files ){
|
||||
List<String> lines = getLines(loader, file.trim());
|
||||
wlist.addAll( lines );
|
||||
}
|
||||
typeTable = parseTypes(wlist);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TokenFilter create(TokenStream input) {
|
||||
return new WordDelimiterGraphFilter(input, typeTable == null ? WordDelimiterIterator.DEFAULT_WORD_DELIM_TABLE : typeTable,
|
||||
flags, protectedWords);
|
||||
}
|
||||
|
||||
// source => type
|
||||
private static Pattern typePattern = Pattern.compile( "(.*)\\s*=>\\s*(.*)\\s*$" );
|
||||
|
||||
// parses a list of MappingCharFilter style rules into a custom byte[] type table
|
||||
private byte[] parseTypes(List<String> rules) {
|
||||
SortedMap<Character,Byte> typeMap = new TreeMap<>();
|
||||
for( String rule : rules ){
|
||||
Matcher m = typePattern.matcher(rule);
|
||||
if( !m.find() )
|
||||
throw new IllegalArgumentException("Invalid Mapping Rule : [" + rule + "]");
|
||||
String lhs = parseString(m.group(1).trim());
|
||||
Byte rhs = parseType(m.group(2).trim());
|
||||
if (lhs.length() != 1)
|
||||
throw new IllegalArgumentException("Invalid Mapping Rule : [" + rule + "]. Only a single character is allowed.");
|
||||
if (rhs == null)
|
||||
throw new IllegalArgumentException("Invalid Mapping Rule : [" + rule + "]. Illegal type.");
|
||||
typeMap.put(lhs.charAt(0), rhs);
|
||||
}
|
||||
|
||||
// ensure the table is always at least as big as DEFAULT_WORD_DELIM_TABLE for performance
|
||||
byte types[] = new byte[Math.max(typeMap.lastKey()+1, WordDelimiterIterator.DEFAULT_WORD_DELIM_TABLE.length)];
|
||||
for (int i = 0; i < types.length; i++)
|
||||
types[i] = WordDelimiterIterator.getType(i);
|
||||
for (Map.Entry<Character,Byte> mapping : typeMap.entrySet())
|
||||
types[mapping.getKey()] = mapping.getValue();
|
||||
return types;
|
||||
}
|
||||
|
||||
private Byte parseType(String s) {
|
||||
if (s.equals("LOWER"))
|
||||
return LOWER;
|
||||
else if (s.equals("UPPER"))
|
||||
return UPPER;
|
||||
else if (s.equals("ALPHA"))
|
||||
return ALPHA;
|
||||
else if (s.equals("DIGIT"))
|
||||
return DIGIT;
|
||||
else if (s.equals("ALPHANUM"))
|
||||
return ALPHANUM;
|
||||
else if (s.equals("SUBWORD_DELIM"))
|
||||
return SUBWORD_DELIM;
|
||||
else
|
||||
return null;
|
||||
}
|
||||
|
||||
char[] out = new char[256];
|
||||
|
||||
private String parseString(String s){
|
||||
int readPos = 0;
|
||||
int len = s.length();
|
||||
int writePos = 0;
|
||||
while( readPos < len ){
|
||||
char c = s.charAt( readPos++ );
|
||||
if( c == '\\' ){
|
||||
if( readPos >= len )
|
||||
throw new IllegalArgumentException("Invalid escaped char in [" + s + "]");
|
||||
c = s.charAt( readPos++ );
|
||||
switch( c ) {
|
||||
case '\\' : c = '\\'; break;
|
||||
case 'n' : c = '\n'; break;
|
||||
case 't' : c = '\t'; break;
|
||||
case 'r' : c = '\r'; break;
|
||||
case 'b' : c = '\b'; break;
|
||||
case 'f' : c = '\f'; break;
|
||||
case 'u' :
|
||||
if( readPos + 3 >= len )
|
||||
throw new IllegalArgumentException("Invalid escaped char in [" + s + "]");
|
||||
c = (char)Integer.parseInt( s.substring( readPos, readPos + 4 ), 16 );
|
||||
readPos += 4;
|
||||
break;
|
||||
}
|
||||
}
|
||||
out[writePos++] = c;
|
||||
}
|
||||
return new String( out, 0, writePos );
|
||||
}
|
||||
}
|
|
@ -16,15 +16,21 @@
|
|||
*/
|
||||
package org.apache.lucene.analysis.miscellaneous;
|
||||
|
||||
|
||||
import static org.apache.lucene.analysis.miscellaneous.WordDelimiterFilter.*;
|
||||
|
||||
/**
|
||||
* A BreakIterator-like API for iterating over subwords in text, according to WordDelimiterFilter rules.
|
||||
* A BreakIterator-like API for iterating over subwords in text, according to WordDelimiterGraphFilter rules.
|
||||
* @lucene.internal
|
||||
*/
|
||||
public final class WordDelimiterIterator {
|
||||
|
||||
static final int LOWER = 0x01;
|
||||
static final int UPPER = 0x02;
|
||||
static final int DIGIT = 0x04;
|
||||
static final int SUBWORD_DELIM = 0x08;
|
||||
|
||||
// combinations: for testing, not for setting bits
|
||||
public static final int ALPHA = 0x03;
|
||||
public static final int ALPHANUM = 0x07;
|
||||
|
||||
/** Indicates the end of iteration */
|
||||
public static final int DONE = -1;
|
||||
|
||||
|
@ -97,7 +103,7 @@ public final class WordDelimiterIterator {
|
|||
* Create a new WordDelimiterIterator operating with the supplied rules.
|
||||
*
|
||||
* @param charTypeTable table containing character types
|
||||
* @param splitOnCaseChange if true, causes "PowerShot" to be two tokens; ("Power-Shot" remains two parts regards)
|
||||
* @param splitOnCaseChange if true, causes "PowerShot" to be two tokens; ("Power-Shot" remains two parts regardless)
|
||||
* @param splitOnNumerics if true, causes "j2se" to be three tokens; "j" "2" "se"
|
||||
* @param stemEnglishPossessive if true, causes trailing "'s" to be removed for each subword: "O'Neil's" => "O", "Neil"
|
||||
*/
|
||||
|
@ -323,4 +329,45 @@ public final class WordDelimiterIterator {
|
|||
default: return SUBWORD_DELIM;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the given word type includes {@link #ALPHA}
|
||||
*
|
||||
* @param type Word type to check
|
||||
* @return {@code true} if the type contains ALPHA, {@code false} otherwise
|
||||
*/
|
||||
static boolean isAlpha(int type) {
|
||||
return (type & ALPHA) != 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the given word type includes {@link #DIGIT}
|
||||
*
|
||||
* @param type Word type to check
|
||||
* @return {@code true} if the type contains DIGIT, {@code false} otherwise
|
||||
*/
|
||||
static boolean isDigit(int type) {
|
||||
return (type & DIGIT) != 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the given word type includes {@link #SUBWORD_DELIM}
|
||||
*
|
||||
* @param type Word type to check
|
||||
* @return {@code true} if the type contains SUBWORD_DELIM, {@code false} otherwise
|
||||
*/
|
||||
static boolean isSubwordDelim(int type) {
|
||||
return (type & SUBWORD_DELIM) != 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the given word type includes {@link #UPPER}
|
||||
*
|
||||
* @param type Word type to check
|
||||
* @return {@code true} if the type contains UPPER, {@code false} otherwise
|
||||
*/
|
||||
static boolean isUpper(int type) {
|
||||
return (type & UPPER) != 0;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.util.Arrays;
|
|||
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.core.FlattenGraphFilter;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.lucene.analysis.Analyzer;
|
|||
import org.apache.lucene.analysis.LowerCaseFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.analysis.core.FlattenGraphFilterFactory;
|
||||
import org.apache.lucene.analysis.core.WhitespaceTokenizer;
|
||||
import org.apache.lucene.analysis.util.ResourceLoader;
|
||||
import org.apache.lucene.analysis.util.ResourceLoaderAware;
|
||||
|
|
|
@ -17,8 +17,14 @@
|
|||
|
||||
package org.apache.lucene.analysis.synonym;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.core.FlattenGraphFilter;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
|
@ -31,11 +37,6 @@ import org.apache.lucene.util.CharsRefBuilder;
|
|||
import org.apache.lucene.util.RollingBuffer;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
// TODO: maybe we should resolve token -> wordID then run
|
||||
// FST on wordIDs, for better perf?
|
||||
|
||||
|
|
|
@ -78,6 +78,7 @@ org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilterFactory
|
|||
org.apache.lucene.analysis.miscellaneous.TrimFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.TruncateTokenFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.WordDelimiterFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.WordDelimiterGraphFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.ScandinavianFoldingFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.ScandinavianNormalizationFilterFactory
|
||||
org.apache.lucene.analysis.ngram.EdgeNGramFilterFactory
|
||||
|
@ -103,6 +104,6 @@ org.apache.lucene.analysis.standard.StandardFilterFactory
|
|||
org.apache.lucene.analysis.sv.SwedishLightStemFilterFactory
|
||||
org.apache.lucene.analysis.synonym.SynonymFilterFactory
|
||||
org.apache.lucene.analysis.synonym.SynonymGraphFilterFactory
|
||||
org.apache.lucene.analysis.synonym.FlattenGraphFilterFactory
|
||||
org.apache.lucene.analysis.core.FlattenGraphFilterFactory
|
||||
org.apache.lucene.analysis.tr.TurkishLowerCaseFilterFactory
|
||||
org.apache.lucene.analysis.util.ElisionFilterFactory
|
||||
|
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.analysis.synonym;
|
||||
package org.apache.lucene.analysis.core;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
|
@ -446,4 +446,73 @@ public class TestWordDelimiterFilter extends BaseTokenStreamTestCase {
|
|||
a.close();
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
public void testToDot() throws Exception {
|
||||
int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE | PRESERVE_ORIGINAL | CATENATE_WORDS | CATENATE_NUMBERS | STEM_ENGLISH_POSSESSIVE;
|
||||
String text = "PowerSystem2000-5-Shot's";
|
||||
WordDelimiterFilter wdf = new WordDelimiterFilter(new CannedTokenStream(new Token(text, 0, text.length())), DEFAULT_WORD_DELIM_TABLE, flags, null);
|
||||
//StringWriter sw = new StringWriter();
|
||||
// TokenStreamToDot toDot = new TokenStreamToDot(text, wdf, new PrintWriter(sw));
|
||||
PrintWriter pw = new PrintWriter("/x/tmp/before.dot");
|
||||
TokenStreamToDot toDot = new TokenStreamToDot(text, wdf, pw);
|
||||
toDot.toDot();
|
||||
pw.close();
|
||||
System.out.println("TEST DONE");
|
||||
//System.out.println("DOT:\n" + sw.toString());
|
||||
}
|
||||
*/
|
||||
|
||||
public void testOnlyNumbers() throws Exception {
|
||||
int flags = GENERATE_WORD_PARTS | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS;
|
||||
Analyzer a = new Analyzer() {
|
||||
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
return new TokenStreamComponents(tokenizer, new WordDelimiterFilter(tokenizer, flags, null));
|
||||
}
|
||||
};
|
||||
|
||||
assertAnalyzesTo(a, "7-586",
|
||||
new String[] {},
|
||||
new int[] {},
|
||||
new int[] {},
|
||||
null,
|
||||
new int[] {},
|
||||
null,
|
||||
false);
|
||||
}
|
||||
|
||||
public void testNumberPunct() throws Exception {
|
||||
int flags = GENERATE_WORD_PARTS | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS;
|
||||
Analyzer a = new Analyzer() {
|
||||
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
return new TokenStreamComponents(tokenizer, new WordDelimiterFilter(tokenizer, flags, null));
|
||||
}
|
||||
};
|
||||
|
||||
assertAnalyzesTo(a, "6-",
|
||||
new String[] {"6"},
|
||||
new int[] {0},
|
||||
new int[] {1},
|
||||
null,
|
||||
new int[] {1},
|
||||
null,
|
||||
false);
|
||||
}
|
||||
|
||||
private Analyzer getAnalyzer(final int flags) {
|
||||
return new Analyzer() {
|
||||
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
return new TokenStreamComponents(tokenizer, new WordDelimiterFilter(tokenizer, flags, null));
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,897 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You 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,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.miscellaneous;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.lucene.analysis.*;
|
||||
import org.apache.lucene.analysis.CharArraySet;
|
||||
import org.apache.lucene.analysis.StopFilter;
|
||||
import org.apache.lucene.analysis.core.KeywordTokenizer;
|
||||
import org.apache.lucene.analysis.standard.StandardAnalyzer;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
import static org.apache.lucene.analysis.miscellaneous.WordDelimiterGraphFilter.*;
|
||||
import static org.apache.lucene.analysis.miscellaneous.WordDelimiterIterator.DEFAULT_WORD_DELIM_TABLE;
|
||||
|
||||
/**
|
||||
* New WordDelimiterGraphFilter tests... most of the tests are in ConvertedLegacyTest
|
||||
* TODO: should explicitly test things like protWords and not rely on
|
||||
* the factory tests in Solr.
|
||||
*/
|
||||
public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase {
|
||||
|
||||
public void testOffsets() throws IOException {
|
||||
int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
|
||||
// test that subwords and catenated subwords have
|
||||
// the correct offsets.
|
||||
WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("foo-bar", 5, 12)), DEFAULT_WORD_DELIM_TABLE, flags, null);
|
||||
|
||||
assertTokenStreamContents(wdf,
|
||||
new String[] { "foobar", "foo", "bar" },
|
||||
new int[] { 5, 5, 9 },
|
||||
new int[] { 12, 8, 12 });
|
||||
|
||||
// with illegal offsets:
|
||||
wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("foo-bar", 5, 6)), DEFAULT_WORD_DELIM_TABLE, flags, null);
|
||||
assertTokenStreamContents(wdf,
|
||||
new String[] { "foobar", "foo", "bar" },
|
||||
new int[] { 5, 5, 5 },
|
||||
new int[] { 6, 6, 6 });
|
||||
}
|
||||
|
||||
public void testOffsetChange() throws Exception {
|
||||
int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
|
||||
WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("übelkeit)", 7, 16)), DEFAULT_WORD_DELIM_TABLE, flags, null);
|
||||
|
||||
assertTokenStreamContents(wdf,
|
||||
new String[] { "übelkeit" },
|
||||
new int[] { 7 },
|
||||
new int[] { 15 });
|
||||
}
|
||||
|
||||
public void testOffsetChange2() throws Exception {
|
||||
int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
|
||||
WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("(übelkeit", 7, 17)), DEFAULT_WORD_DELIM_TABLE, flags, null);
|
||||
// illegal offsets:
|
||||
assertTokenStreamContents(wdf,
|
||||
new String[] { "übelkeit" },
|
||||
new int[] { 7 },
|
||||
new int[] { 17 });
|
||||
}
|
||||
|
||||
public void testOffsetChange3() throws Exception {
|
||||
int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
|
||||
WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("(übelkeit", 7, 16)), DEFAULT_WORD_DELIM_TABLE, flags, null);
|
||||
assertTokenStreamContents(wdf,
|
||||
new String[] { "übelkeit" },
|
||||
new int[] { 8 },
|
||||
new int[] { 16 });
|
||||
}
|
||||
|
||||
public void testOffsetChange4() throws Exception {
|
||||
int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
|
||||
WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("(foo,bar)", 7, 16)), DEFAULT_WORD_DELIM_TABLE, flags, null);
|
||||
|
||||
assertTokenStreamContents(wdf,
|
||||
new String[] { "foobar", "foo", "bar"},
|
||||
new int[] { 8, 8, 12 },
|
||||
new int[] { 15, 11, 15 });
|
||||
}
|
||||
|
||||
public void doSplit(final String input, String... output) throws Exception {
|
||||
int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
|
||||
WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(keywordMockTokenizer(input),
|
||||
WordDelimiterIterator.DEFAULT_WORD_DELIM_TABLE, flags, null);
|
||||
|
||||
assertTokenStreamContents(wdf, output);
|
||||
}
|
||||
|
||||
public void testSplits() throws Exception {
|
||||
doSplit("basic-split","basic","split");
|
||||
doSplit("camelCase","camel","Case");
|
||||
|
||||
// non-space marking symbol shouldn't cause split
|
||||
// this is an example in Thai
|
||||
doSplit("\u0e1a\u0e49\u0e32\u0e19","\u0e1a\u0e49\u0e32\u0e19");
|
||||
// possessive followed by delimiter
|
||||
doSplit("test's'", "test");
|
||||
|
||||
// some russian upper and lowercase
|
||||
doSplit("Роберт", "Роберт");
|
||||
// now cause a split (russian camelCase)
|
||||
doSplit("РобЕрт", "Роб", "Ерт");
|
||||
|
||||
// a composed titlecase character, don't split
|
||||
doSplit("aDžungla", "aDžungla");
|
||||
|
||||
// a modifier letter, don't split
|
||||
doSplit("ســـــــــــــــــلام", "ســـــــــــــــــلام");
|
||||
|
||||
// enclosing mark, don't split
|
||||
doSplit("test⃝", "test⃝");
|
||||
|
||||
// combining spacing mark (the virama), don't split
|
||||
doSplit("हिन्दी", "हिन्दी");
|
||||
|
||||
// don't split non-ascii digits
|
||||
doSplit("١٢٣٤", "١٢٣٤");
|
||||
|
||||
// don't split supplementaries into unpaired surrogates
|
||||
doSplit("𠀀𠀀", "𠀀𠀀");
|
||||
}
|
||||
|
||||
public void doSplitPossessive(int stemPossessive, final String input, final String... output) throws Exception {
|
||||
int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS;
|
||||
flags |= (stemPossessive == 1) ? STEM_ENGLISH_POSSESSIVE : 0;
|
||||
WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(keywordMockTokenizer(input), flags, null);
|
||||
|
||||
assertTokenStreamContents(wdf, output);
|
||||
}
|
||||
|
||||
/*
|
||||
* Test option that allows disabling the special "'s" stemming, instead treating the single quote like other delimiters.
|
||||
*/
|
||||
public void testPossessives() throws Exception {
|
||||
doSplitPossessive(1, "ra's", "ra");
|
||||
doSplitPossessive(0, "ra's", "ra", "s");
|
||||
}
|
||||
|
||||
/*
|
||||
* Set a large position increment gap of 10 if the token is "largegap" or "/"
|
||||
*/
|
||||
private final class LargePosIncTokenFilter extends TokenFilter {
|
||||
private CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
|
||||
private PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
|
||||
|
||||
protected LargePosIncTokenFilter(TokenStream input) {
|
||||
super(input);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
if (input.incrementToken()) {
|
||||
if (termAtt.toString().equals("largegap") || termAtt.toString().equals("/"))
|
||||
posIncAtt.setPositionIncrement(10);
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testPositionIncrements() throws Exception {
|
||||
final int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
|
||||
final CharArraySet protWords = new CharArraySet(new HashSet<>(Arrays.asList("NUTCH")), false);
|
||||
|
||||
/* analyzer that uses whitespace + wdf */
|
||||
Analyzer a = new Analyzer() {
|
||||
@Override
|
||||
public TokenStreamComponents createComponents(String field) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(
|
||||
tokenizer,
|
||||
flags, protWords));
|
||||
}
|
||||
};
|
||||
|
||||
/* in this case, works as expected. */
|
||||
assertAnalyzesTo(a, "LUCENE / SOLR", new String[] { "LUCENE", "SOLR" },
|
||||
new int[] { 0, 9 },
|
||||
new int[] { 6, 13 },
|
||||
null,
|
||||
new int[] { 1, 2 },
|
||||
null,
|
||||
false);
|
||||
|
||||
/* only in this case, posInc of 2 ?! */
|
||||
assertAnalyzesTo(a, "LUCENE / solR", new String[] { "LUCENE", "solR", "sol", "R" },
|
||||
new int[] { 0, 9, 9, 12 },
|
||||
new int[] { 6, 13, 12, 13 },
|
||||
null,
|
||||
new int[] { 1, 2, 0, 1 },
|
||||
null,
|
||||
false);
|
||||
|
||||
assertAnalyzesTo(a, "LUCENE / NUTCH SOLR", new String[] { "LUCENE", "NUTCH", "SOLR" },
|
||||
new int[] { 0, 9, 15 },
|
||||
new int[] { 6, 14, 19 },
|
||||
null,
|
||||
new int[] { 1, 2, 1 },
|
||||
null,
|
||||
false);
|
||||
|
||||
/* analyzer that will consume tokens with large position increments */
|
||||
Analyzer a2 = new Analyzer() {
|
||||
@Override
|
||||
public TokenStreamComponents createComponents(String field) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(
|
||||
new LargePosIncTokenFilter(tokenizer),
|
||||
flags, protWords));
|
||||
}
|
||||
};
|
||||
|
||||
/* increment of "largegap" is preserved */
|
||||
assertAnalyzesTo(a2, "LUCENE largegap SOLR", new String[] { "LUCENE", "largegap", "SOLR" },
|
||||
new int[] { 0, 7, 16 },
|
||||
new int[] { 6, 15, 20 },
|
||||
null,
|
||||
new int[] { 1, 10, 1 },
|
||||
null,
|
||||
false);
|
||||
|
||||
/* the "/" had a position increment of 10, where did it go?!?!! */
|
||||
assertAnalyzesTo(a2, "LUCENE / SOLR", new String[] { "LUCENE", "SOLR" },
|
||||
new int[] { 0, 9 },
|
||||
new int[] { 6, 13 },
|
||||
null,
|
||||
new int[] { 1, 11 },
|
||||
null,
|
||||
false);
|
||||
|
||||
/* in this case, the increment of 10 from the "/" is carried over */
|
||||
assertAnalyzesTo(a2, "LUCENE / solR", new String[] { "LUCENE", "solR", "sol", "R" },
|
||||
new int[] { 0, 9, 9, 12 },
|
||||
new int[] { 6, 13, 12, 13 },
|
||||
null,
|
||||
new int[] { 1, 11, 0, 1 },
|
||||
null,
|
||||
false);
|
||||
|
||||
assertAnalyzesTo(a2, "LUCENE / NUTCH SOLR", new String[] { "LUCENE", "NUTCH", "SOLR" },
|
||||
new int[] { 0, 9, 15 },
|
||||
new int[] { 6, 14, 19 },
|
||||
null,
|
||||
new int[] { 1, 11, 1 },
|
||||
null,
|
||||
false);
|
||||
|
||||
Analyzer a3 = new Analyzer() {
|
||||
@Override
|
||||
public TokenStreamComponents createComponents(String field) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
StopFilter filter = new StopFilter(tokenizer, StandardAnalyzer.STOP_WORDS_SET);
|
||||
return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(filter, flags, protWords));
|
||||
}
|
||||
};
|
||||
|
||||
assertAnalyzesTo(a3, "lucene.solr",
|
||||
new String[] { "lucenesolr", "lucene", "solr" },
|
||||
new int[] { 0, 0, 7 },
|
||||
new int[] { 11, 6, 11 },
|
||||
null,
|
||||
new int[] { 1, 0, 1 },
|
||||
null,
|
||||
false);
|
||||
|
||||
/* the stopword should add a gap here */
|
||||
assertAnalyzesTo(a3, "the lucene.solr",
|
||||
new String[] { "lucenesolr", "lucene", "solr" },
|
||||
new int[] { 4, 4, 11 },
|
||||
new int[] { 15, 10, 15 },
|
||||
null,
|
||||
new int[] { 2, 0, 1 },
|
||||
null,
|
||||
false);
|
||||
|
||||
IOUtils.close(a, a2, a3);
|
||||
}
|
||||
|
||||
/** concat numbers + words + all */
|
||||
public void testLotsOfConcatenating() throws Exception {
|
||||
final int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_WORDS | CATENATE_NUMBERS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
|
||||
|
||||
/* analyzer that uses whitespace + wdf */
|
||||
Analyzer a = new Analyzer() {
|
||||
@Override
|
||||
public TokenStreamComponents createComponents(String field) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(tokenizer, flags, null));
|
||||
}
|
||||
};
|
||||
|
||||
assertAnalyzesTo(a, "abc-def-123-456",
|
||||
new String[] { "abcdef123456", "abcdef", "abc", "def", "123456", "123", "456" },
|
||||
new int[] { 0, 0, 0, 4, 8, 8, 12 },
|
||||
new int[] { 15, 7, 3, 7, 15, 11, 15 },
|
||||
null,
|
||||
new int[] { 1, 0, 0, 1, 1, 0, 1 },
|
||||
null,
|
||||
false);
|
||||
a.close();
|
||||
}
|
||||
|
||||
/** concat numbers + words + all + preserve original */
|
||||
public void testLotsOfConcatenating2() throws Exception {
|
||||
final int flags = PRESERVE_ORIGINAL | GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_WORDS | CATENATE_NUMBERS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE;
|
||||
|
||||
/* analyzer that uses whitespace + wdf */
|
||||
Analyzer a = new Analyzer() {
|
||||
@Override
|
||||
public TokenStreamComponents createComponents(String field) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(tokenizer, flags, null));
|
||||
}
|
||||
};
|
||||
|
||||
assertAnalyzesTo(a, "abc-def-123-456",
|
||||
new String[] { "abcdef123456", "abc-def-123-456", "abcdef", "abc", "def", "123456", "123", "456" },
|
||||
new int[] { 0, 0, 0, 0, 4, 8, 8, 12 },
|
||||
new int[] { 15, 15, 7, 3, 7, 15, 11, 15 },
|
||||
null,
|
||||
new int[] { 1, 0, 0, 0, 1, 1, 0, 1 },
|
||||
null,
|
||||
false);
|
||||
a.close();
|
||||
}
|
||||
|
||||
/** blast some random strings through the analyzer */
|
||||
public void testRandomStrings() throws Exception {
|
||||
int numIterations = atLeast(5);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
final int flags = random().nextInt(512);
|
||||
final CharArraySet protectedWords;
|
||||
if (random().nextBoolean()) {
|
||||
protectedWords = new CharArraySet(new HashSet<>(Arrays.asList("a", "b", "cd")), false);
|
||||
} else {
|
||||
protectedWords = null;
|
||||
}
|
||||
|
||||
Analyzer a = new Analyzer() {
|
||||
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(tokenizer, flags, protectedWords));
|
||||
}
|
||||
};
|
||||
// TODO: properly support positionLengthAttribute
|
||||
checkRandomData(random(), a, 200*RANDOM_MULTIPLIER, 20, false, false);
|
||||
a.close();
|
||||
}
|
||||
}
|
||||
|
||||
/** blast some enormous random strings through the analyzer */
|
||||
public void testRandomHugeStrings() throws Exception {
|
||||
int numIterations = atLeast(5);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
final int flags = random().nextInt(512);
|
||||
final CharArraySet protectedWords;
|
||||
if (random().nextBoolean()) {
|
||||
protectedWords = new CharArraySet(new HashSet<>(Arrays.asList("a", "b", "cd")), false);
|
||||
} else {
|
||||
protectedWords = null;
|
||||
}
|
||||
|
||||
Analyzer a = new Analyzer() {
|
||||
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
TokenStream wdgf = new WordDelimiterGraphFilter(tokenizer, flags, protectedWords);
|
||||
return new TokenStreamComponents(tokenizer, wdgf);
|
||||
}
|
||||
};
|
||||
// TODO: properly support positionLengthAttribute
|
||||
checkRandomData(random(), a, 20*RANDOM_MULTIPLIER, 8192, false, false);
|
||||
a.close();
|
||||
}
|
||||
}
|
||||
|
||||
public void testEmptyTerm() throws IOException {
|
||||
Random random = random();
|
||||
for (int i = 0; i < 512; i++) {
|
||||
final int flags = i;
|
||||
final CharArraySet protectedWords;
|
||||
if (random.nextBoolean()) {
|
||||
protectedWords = new CharArraySet(new HashSet<>(Arrays.asList("a", "b", "cd")), false);
|
||||
} else {
|
||||
protectedWords = null;
|
||||
}
|
||||
|
||||
Analyzer a = new Analyzer() {
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new KeywordTokenizer();
|
||||
return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(tokenizer, flags, protectedWords));
|
||||
}
|
||||
};
|
||||
// depending upon options, this thing may or may not preserve the empty term
|
||||
checkAnalysisConsistency(random, a, random.nextBoolean(), "");
|
||||
a.close();
|
||||
}
|
||||
}
|
||||
|
||||
private Analyzer getAnalyzer(int flags) {
|
||||
return getAnalyzer(flags, null);
|
||||
}
|
||||
|
||||
private Analyzer getAnalyzer(int flags, CharArraySet protectedWords) {
|
||||
return new Analyzer() {
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new KeywordTokenizer();
|
||||
return new TokenStreamComponents(tokenizer, new WordDelimiterGraphFilter(tokenizer, flags, protectedWords));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private static boolean has(int flags, int flag) {
|
||||
return (flags & flag) != 0;
|
||||
}
|
||||
|
||||
private static boolean isEnglishPossessive(String text, int pos) {
|
||||
if (pos > 2) {
|
||||
if ((text.charAt(pos-1) == 's' || text.charAt(pos-1) == 'S') &&
|
||||
(pos == text.length() || text.charAt(pos) != '-')) {
|
||||
text = text.substring(0, text.length()-2);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private static class WordPart {
|
||||
final String part;
|
||||
final int startOffset;
|
||||
final int endOffset;
|
||||
final int type;
|
||||
|
||||
public WordPart(String text, int startOffset, int endOffset) {
|
||||
this.part = text.substring(startOffset, endOffset);
|
||||
this.startOffset = startOffset;
|
||||
this.endOffset = endOffset;
|
||||
this.type = toType(part.charAt(0));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "WordPart(" + part + " " + startOffset + "-" + endOffset + ")";
|
||||
}
|
||||
}
|
||||
|
||||
private static final int NUMBER = 0;
|
||||
private static final int LETTER = 1;
|
||||
private static final int DELIM = 2;
|
||||
|
||||
private static int toType(char ch) {
|
||||
if (Character.isDigit(ch)) {
|
||||
// numbers
|
||||
return NUMBER;
|
||||
} else if (Character.isLetter(ch)) {
|
||||
// letters
|
||||
return LETTER;
|
||||
} else {
|
||||
// delimiter
|
||||
return DELIM;
|
||||
}
|
||||
}
|
||||
|
||||
/** Does (hopefully) the same thing as WordDelimiterGraphFilter, according to the flags, but more slowly, returning all string paths combinations. */
|
||||
private Set<String> slowWDF(String text, int flags) {
|
||||
|
||||
// first make word parts:
|
||||
List<WordPart> wordParts = new ArrayList<>();
|
||||
int lastCH = -1;
|
||||
int wordPartStart = 0;
|
||||
boolean inToken = false;
|
||||
|
||||
for(int i=0;i<text.length();i++) {
|
||||
char ch = text.charAt(i);
|
||||
if (toType(ch) == DELIM) {
|
||||
// delimiter
|
||||
if (inToken) {
|
||||
// end current token
|
||||
wordParts.add(new WordPart(text, wordPartStart, i));
|
||||
inToken = false;
|
||||
}
|
||||
|
||||
// strip english possessive at the end of this token?:
|
||||
if (has(flags, STEM_ENGLISH_POSSESSIVE) &&
|
||||
ch == '\'' && i > 0 &&
|
||||
i < text.length()-1 &&
|
||||
(text.charAt(i+1) == 's' || text.charAt(i+1) == 'S') &&
|
||||
toType(text.charAt(i-1)) == LETTER &&
|
||||
(i+2 == text.length() || toType(text.charAt(i+2)) == DELIM)) {
|
||||
i += 2;
|
||||
}
|
||||
|
||||
} else if (inToken == false) {
|
||||
// start new token
|
||||
inToken = true;
|
||||
wordPartStart = i;
|
||||
} else {
|
||||
boolean newToken = false;
|
||||
if (Character.isLetter(lastCH)) {
|
||||
if (Character.isLetter(ch)) {
|
||||
if (has(flags, SPLIT_ON_CASE_CHANGE) && Character.isLowerCase(lastCH) && Character.isLowerCase(ch) == false) {
|
||||
// start new token on lower -> UPPER case change (but not vice versa!)
|
||||
newToken = true;
|
||||
}
|
||||
} else if (has(flags, SPLIT_ON_NUMERICS) && Character.isDigit(ch)) {
|
||||
// start new token on letter -> number change
|
||||
newToken = true;
|
||||
}
|
||||
} else {
|
||||
assert Character.isDigit(lastCH);
|
||||
if (Character.isLetter(ch) && has(flags, SPLIT_ON_NUMERICS) ) {
|
||||
// start new token on number -> letter change
|
||||
newToken = true;
|
||||
}
|
||||
}
|
||||
if (newToken) {
|
||||
wordParts.add(new WordPart(text, wordPartStart, i));
|
||||
wordPartStart = i;
|
||||
}
|
||||
}
|
||||
lastCH = ch;
|
||||
}
|
||||
|
||||
if (inToken) {
|
||||
// add last token
|
||||
wordParts.add(new WordPart(text, wordPartStart, text.length()));
|
||||
}
|
||||
|
||||
Set<String> paths = new HashSet<>();
|
||||
if (wordParts.isEmpty() == false) {
|
||||
enumerate(flags, 0, text, wordParts, paths, new StringBuilder());
|
||||
}
|
||||
|
||||
if (has(flags, PRESERVE_ORIGINAL)) {
|
||||
paths.add(text);
|
||||
}
|
||||
|
||||
if (has(flags, CATENATE_ALL) && wordParts.isEmpty() == false) {
|
||||
StringBuilder b = new StringBuilder();
|
||||
for(WordPart wordPart : wordParts) {
|
||||
b.append(wordPart.part);
|
||||
}
|
||||
paths.add(b.toString());
|
||||
}
|
||||
|
||||
return paths;
|
||||
}
|
||||
|
||||
private void add(StringBuilder path, String part) {
|
||||
if (path.length() != 0) {
|
||||
path.append(' ');
|
||||
}
|
||||
path.append(part);
|
||||
}
|
||||
|
||||
private void add(StringBuilder path, List<WordPart> wordParts, int from, int to) {
|
||||
if (path.length() != 0) {
|
||||
path.append(' ');
|
||||
}
|
||||
// no spaces:
|
||||
for(int i=from;i<to;i++) {
|
||||
path.append(wordParts.get(i).part);
|
||||
}
|
||||
}
|
||||
|
||||
private void addWithSpaces(StringBuilder path, List<WordPart> wordParts, int from, int to) {
|
||||
for(int i=from;i<to;i++) {
|
||||
add(path, wordParts.get(i).part);
|
||||
}
|
||||
}
|
||||
|
||||
/** Finds the end (exclusive) of the series of part with the same type */
|
||||
private int endOfRun(List<WordPart> wordParts, int start) {
|
||||
int upto = start+1;
|
||||
while(upto < wordParts.size() && wordParts.get(upto).type == wordParts.get(start).type) {
|
||||
upto++;
|
||||
}
|
||||
return upto;
|
||||
}
|
||||
|
||||
/** Recursively enumerates all paths through the word parts */
|
||||
private void enumerate(int flags, int upto, String text, List<WordPart> wordParts, Set<String> paths, StringBuilder path) {
|
||||
if (upto == wordParts.size()) {
|
||||
if (path.length() > 0) {
|
||||
paths.add(path.toString());
|
||||
}
|
||||
} else {
|
||||
int savLength = path.length();
|
||||
int end = endOfRun(wordParts, upto);
|
||||
|
||||
if (wordParts.get(upto).type == NUMBER) {
|
||||
// always output single word, optionally surrounded by delims:
|
||||
if (has(flags, GENERATE_NUMBER_PARTS) || wordParts.size() == 1) {
|
||||
addWithSpaces(path, wordParts, upto, end);
|
||||
if (has(flags, CATENATE_NUMBERS)) {
|
||||
// recurse first with the parts
|
||||
enumerate(flags, end, text, wordParts, paths, path);
|
||||
path.setLength(savLength);
|
||||
// .. and second with the concat
|
||||
add(path, wordParts, upto, end);
|
||||
}
|
||||
} else if (has(flags, CATENATE_NUMBERS)) {
|
||||
add(path, wordParts, upto, end);
|
||||
}
|
||||
enumerate(flags, end, text, wordParts, paths, path);
|
||||
path.setLength(savLength);
|
||||
} else {
|
||||
assert wordParts.get(upto).type == LETTER;
|
||||
// always output single word, optionally surrounded by delims:
|
||||
if (has(flags, GENERATE_WORD_PARTS) || wordParts.size() == 1) {
|
||||
addWithSpaces(path, wordParts, upto, end);
|
||||
if (has(flags, CATENATE_WORDS)) {
|
||||
// recurse first with the parts
|
||||
enumerate(flags, end, text, wordParts, paths, path);
|
||||
path.setLength(savLength);
|
||||
// .. and second with the concat
|
||||
add(path, wordParts, upto, end);
|
||||
}
|
||||
} else if (has(flags, CATENATE_WORDS)) {
|
||||
add(path, wordParts, upto, end);
|
||||
}
|
||||
enumerate(flags, end, text, wordParts, paths, path);
|
||||
path.setLength(savLength);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testBasicGraphSplits() throws Exception {
|
||||
assertGraphStrings(getAnalyzer(0),
|
||||
"PowerShotPlus",
|
||||
"PowerShotPlus");
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS),
|
||||
"PowerShotPlus",
|
||||
"PowerShotPlus");
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | SPLIT_ON_CASE_CHANGE),
|
||||
"PowerShotPlus",
|
||||
"Power Shot Plus");
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | SPLIT_ON_CASE_CHANGE | PRESERVE_ORIGINAL),
|
||||
"PowerShotPlus",
|
||||
"PowerShotPlus",
|
||||
"Power Shot Plus");
|
||||
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS),
|
||||
"Power-Shot-Plus",
|
||||
"Power Shot Plus");
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | SPLIT_ON_CASE_CHANGE),
|
||||
"Power-Shot-Plus",
|
||||
"Power Shot Plus");
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | SPLIT_ON_CASE_CHANGE | PRESERVE_ORIGINAL),
|
||||
"Power-Shot-Plus",
|
||||
"Power-Shot-Plus",
|
||||
"Power Shot Plus");
|
||||
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | SPLIT_ON_CASE_CHANGE),
|
||||
"PowerShotPlus",
|
||||
"Power Shot Plus");
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | SPLIT_ON_CASE_CHANGE),
|
||||
"PowerShot1000Plus",
|
||||
"Power Shot1000Plus");
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | SPLIT_ON_CASE_CHANGE),
|
||||
"Power-Shot-Plus",
|
||||
"Power Shot Plus");
|
||||
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | SPLIT_ON_CASE_CHANGE | CATENATE_WORDS),
|
||||
"PowerShotPlus",
|
||||
"Power Shot Plus",
|
||||
"PowerShotPlus");
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | SPLIT_ON_CASE_CHANGE | CATENATE_WORDS),
|
||||
"PowerShot1000Plus",
|
||||
"Power Shot1000Plus",
|
||||
"PowerShot1000Plus");
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | SPLIT_ON_CASE_CHANGE | CATENATE_WORDS | CATENATE_NUMBERS),
|
||||
"Power-Shot-1000-17-Plus",
|
||||
"Power Shot 1000 17 Plus",
|
||||
"Power Shot 100017 Plus",
|
||||
"PowerShot 1000 17 Plus",
|
||||
"PowerShot 100017 Plus");
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | SPLIT_ON_CASE_CHANGE | CATENATE_WORDS | CATENATE_NUMBERS | PRESERVE_ORIGINAL),
|
||||
"Power-Shot-1000-17-Plus",
|
||||
"Power-Shot-1000-17-Plus",
|
||||
"Power Shot 1000 17 Plus",
|
||||
"Power Shot 100017 Plus",
|
||||
"PowerShot 1000 17 Plus",
|
||||
"PowerShot 100017 Plus");
|
||||
}
|
||||
|
||||
/*
|
||||
public void testToDot() throws Exception {
|
||||
int flags = GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE | PRESERVE_ORIGINAL | CATENATE_WORDS | CATENATE_NUMBERS | STEM_ENGLISH_POSSESSIVE;
|
||||
String text = "PowerSystem2000-5-Shot's";
|
||||
WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token(text, 0, text.length())), DEFAULT_WORD_DELIM_TABLE, flags, null);
|
||||
//StringWriter sw = new StringWriter();
|
||||
// TokenStreamToDot toDot = new TokenStreamToDot(text, wdf, new PrintWriter(sw));
|
||||
PrintWriter pw = new PrintWriter("/tmp/foo2.dot");
|
||||
TokenStreamToDot toDot = new TokenStreamToDot(text, wdf, pw);
|
||||
toDot.toDot();
|
||||
pw.close();
|
||||
//System.out.println("DOT:\n" + sw.toString());
|
||||
}
|
||||
*/
|
||||
|
||||
private String randomWDFText() {
|
||||
StringBuilder b = new StringBuilder();
|
||||
int length = TestUtil.nextInt(random(), 1, 50);
|
||||
for(int i=0;i<length;i++) {
|
||||
int surpriseMe = random().nextInt(37);
|
||||
int lower = -1;
|
||||
int upper = -1;
|
||||
if (surpriseMe < 10) {
|
||||
// lowercase letter
|
||||
lower = 'a';
|
||||
upper = 'z';
|
||||
} else if (surpriseMe < 20) {
|
||||
// uppercase letter
|
||||
lower = 'A';
|
||||
upper = 'Z';
|
||||
} else if (surpriseMe < 30) {
|
||||
// digit
|
||||
lower = '0';
|
||||
upper = '9';
|
||||
} else if (surpriseMe < 35) {
|
||||
// punct
|
||||
lower = '-';
|
||||
upper = '-';
|
||||
} else {
|
||||
b.append("'s");
|
||||
}
|
||||
|
||||
if (lower != -1) {
|
||||
b.append((char) TestUtil.nextInt(random(), lower, upper));
|
||||
}
|
||||
}
|
||||
|
||||
return b.toString();
|
||||
}
|
||||
|
||||
public void testInvalidFlag() throws Exception {
|
||||
expectThrows(IllegalArgumentException.class,
|
||||
() -> {
|
||||
new WordDelimiterGraphFilter(new CannedTokenStream(), 1 << 31, null);
|
||||
});
|
||||
}
|
||||
|
||||
public void testRandomPaths() throws Exception {
|
||||
int iters = atLeast(100);
|
||||
for(int iter=0;iter<iters;iter++) {
|
||||
String text = randomWDFText();
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: text=" + text + " len=" + text.length());
|
||||
}
|
||||
|
||||
int flags = 0;
|
||||
if (random().nextBoolean()) {
|
||||
flags |= GENERATE_WORD_PARTS;
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
flags |= GENERATE_NUMBER_PARTS;
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
flags |= CATENATE_WORDS;
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
flags |= CATENATE_NUMBERS;
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
flags |= CATENATE_ALL;
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
flags |= PRESERVE_ORIGINAL;
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
flags |= SPLIT_ON_CASE_CHANGE;
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
flags |= SPLIT_ON_NUMERICS;
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
flags |= STEM_ENGLISH_POSSESSIVE;
|
||||
}
|
||||
|
||||
verify(text, flags);
|
||||
}
|
||||
}
|
||||
|
||||
/** Runs normal and slow WDGF and compares results */
|
||||
private void verify(String text, int flags) throws IOException {
|
||||
|
||||
Set<String> expected = slowWDF(text, flags);
|
||||
if (VERBOSE) {
|
||||
for(String path : expected) {
|
||||
System.out.println(" " + path);
|
||||
}
|
||||
}
|
||||
|
||||
Set<String> actual = getGraphStrings(getAnalyzer(flags), text);
|
||||
if (actual.equals(expected) == false) {
|
||||
StringBuilder b = new StringBuilder();
|
||||
b.append("\n\nFAIL: text=");
|
||||
b.append(text);
|
||||
b.append(" flags=");
|
||||
b.append(WordDelimiterGraphFilter.flagsToString(flags));
|
||||
b.append('\n');
|
||||
b.append(" expected paths:\n");
|
||||
for (String s : expected) {
|
||||
b.append(" ");
|
||||
b.append(s);
|
||||
if (actual.contains(s) == false) {
|
||||
b.append(" [missing!]");
|
||||
}
|
||||
b.append('\n');
|
||||
}
|
||||
|
||||
b.append(" actual paths:\n");
|
||||
for (String s : actual) {
|
||||
b.append(" ");
|
||||
b.append(s);
|
||||
if (expected.contains(s) == false) {
|
||||
b.append(" [unexpected!]");
|
||||
}
|
||||
b.append('\n');
|
||||
}
|
||||
|
||||
fail(b.toString());
|
||||
}
|
||||
}
|
||||
|
||||
public void testOnlyNumbers() throws Exception {
|
||||
// no token should be produced
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS), "7-586");
|
||||
}
|
||||
|
||||
public void testNoCatenate() throws Exception {
|
||||
// no token should be produced
|
||||
assertGraphStrings(getAnalyzer(GENERATE_WORD_PARTS | GENERATE_NUMBER_PARTS | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS), "a-b-c-9-d", "a b c 9 d");
|
||||
}
|
||||
|
||||
public void testCuriousCase1() throws Exception {
|
||||
verify("u-0L-4836-ip4Gw--13--q7--L07E1", CATENATE_WORDS | CATENATE_ALL | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS | STEM_ENGLISH_POSSESSIVE);
|
||||
}
|
||||
|
||||
public void testCuriousCase2() throws Exception {
|
||||
verify("u-l-p", CATENATE_ALL);
|
||||
}
|
||||
|
||||
public void testOriginalPosLength() throws Exception {
|
||||
verify("Foo-Bar-Baz", CATENATE_WORDS | SPLIT_ON_CASE_CHANGE | PRESERVE_ORIGINAL);
|
||||
}
|
||||
|
||||
public void testCuriousCase3() throws Exception {
|
||||
verify("cQzk4-GL0izl0mKM-J8--4m-'s", GENERATE_NUMBER_PARTS | CATENATE_NUMBERS | SPLIT_ON_CASE_CHANGE | SPLIT_ON_NUMERICS);
|
||||
}
|
||||
|
||||
public void testEmptyString() throws Exception {
|
||||
WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(new CannedTokenStream(new Token("", 0, 0)), DEFAULT_WORD_DELIM_TABLE, GENERATE_WORD_PARTS | CATENATE_ALL | PRESERVE_ORIGINAL, null);
|
||||
wdf.reset();
|
||||
assertTrue(wdf.incrementToken());
|
||||
assertFalse(wdf.incrementToken());
|
||||
wdf.end();
|
||||
wdf.close();
|
||||
}
|
||||
|
||||
public void testProtectedWords() throws Exception {
|
||||
TokenStream tokens = new CannedTokenStream(new Token("foo17-bar", 0, 9),
|
||||
new Token("foo-bar", 0, 7));
|
||||
|
||||
CharArraySet protectedWords = new CharArraySet(new HashSet<>(Arrays.asList("foo17-BAR")), true);
|
||||
WordDelimiterGraphFilter wdf = new WordDelimiterGraphFilter(tokens, DEFAULT_WORD_DELIM_TABLE, GENERATE_WORD_PARTS | PRESERVE_ORIGINAL | CATENATE_ALL, protectedWords);
|
||||
assertGraphStrings(wdf,
|
||||
"foo17-bar foo bar",
|
||||
"foo17-bar foo-bar",
|
||||
"foo17-bar foobar");
|
||||
}
|
||||
}
|
|
@ -17,14 +17,22 @@
|
|||
|
||||
package org.apache.lucene.analysis.synonym;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
import java.text.ParseException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.analysis.MockGraphTokenFilter;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.TokenStreamToAutomaton;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.analysis.core.FlattenGraphFilter;
|
||||
import org.apache.lucene.analysis.tokenattributes.*;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -35,7 +43,6 @@ import org.apache.lucene.search.PhraseQuery;
|
|||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
@ -49,15 +56,6 @@ import org.apache.lucene.util.automaton.TooComplexToDeterminizeException;
|
|||
import org.apache.lucene.util.automaton.Transition;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
import java.text.ParseException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
public class TestSynonymGraphFilter extends BaseTokenStreamTestCase {
|
||||
|
||||
/** Set as a side effect by {@link #getAnalyzer} and {@link #getFlattenAnalyzer}. */
|
||||
|
@ -1832,7 +1830,7 @@ public class TestSynonymGraphFilter extends BaseTokenStreamTestCase {
|
|||
new int[] {1, 1, 0, 0, 0, 1, 0, 0, 1, 0, 1, 1, 1},
|
||||
new int[] {1, 1, 1, 1, 4, 3, 1, 1, 2, 1, 1, 1, 1});
|
||||
|
||||
assertAllStrings(analyzer, "the usa is wealthy", new String[] {
|
||||
assertGraphStrings(analyzer, "the usa is wealthy", new String[] {
|
||||
"the usa is wealthy",
|
||||
"the united states is wealthy",
|
||||
"the u s a is wealthy",
|
||||
|
@ -1924,33 +1922,4 @@ public class TestSynonymGraphFilter extends BaseTokenStreamTestCase {
|
|||
new int[]{1, 1, 0, 1, 1});
|
||||
a.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to validate all strings that can be generated from a token stream.
|
||||
* Uses {@link TokenStreamToAutomaton} to create an automaton. Asserts the finite strings of the automaton are all
|
||||
* and only the given valid strings.
|
||||
* @param analyzer analyzer containing the SynonymFilter under test.
|
||||
* @param text text to be analyzed.
|
||||
* @param expectedStrings all expected finite strings.
|
||||
*/
|
||||
public void assertAllStrings(Analyzer analyzer, String text, String[] expectedStrings) throws IOException {
|
||||
TokenStream tokenStream = analyzer.tokenStream("dummy", text);
|
||||
try {
|
||||
Automaton automaton = new TokenStreamToAutomaton().toAutomaton(tokenStream);
|
||||
Set<IntsRef> finiteStrings = AutomatonTestUtil.getFiniteStringsRecursive(automaton, -1);
|
||||
|
||||
assertEquals("Invalid resulting strings count. Expected " + expectedStrings.length + " was " + finiteStrings.size(),
|
||||
expectedStrings.length, finiteStrings.size());
|
||||
|
||||
Set<String> expectedStringsSet = new HashSet<>(Arrays.asList(expectedStrings));
|
||||
|
||||
BytesRefBuilder scratchBytesRefBuilder = new BytesRefBuilder();
|
||||
for (IntsRef ir: finiteStrings) {
|
||||
String s = Util.toBytesRef(ir, scratchBytesRefBuilder).utf8ToString().replace((char) TokenStreamToAutomaton.POS_SEP, ' ');
|
||||
assertTrue("Unexpected string found: " + s, expectedStringsSet.contains(s));
|
||||
}
|
||||
} finally {
|
||||
tokenStream.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -286,6 +286,56 @@ final class SimpleTextBKDReader extends PointValues implements Accountable {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimatePointCount(IntersectVisitor visitor) {
|
||||
return estimatePointCount(getIntersectState(visitor), 1, minPackedValue, maxPackedValue);
|
||||
}
|
||||
|
||||
private long estimatePointCount(IntersectState state,
|
||||
int nodeID, byte[] cellMinPacked, byte[] cellMaxPacked) {
|
||||
Relation r = state.visitor.compare(cellMinPacked, cellMaxPacked);
|
||||
|
||||
if (r == Relation.CELL_OUTSIDE_QUERY) {
|
||||
// This cell is fully outside of the query shape: stop recursing
|
||||
return 0L;
|
||||
} else if (nodeID >= leafNodeOffset) {
|
||||
// Assume all points match and there are no dups
|
||||
return maxPointsInLeafNode;
|
||||
} else {
|
||||
|
||||
// Non-leaf node: recurse on the split left and right nodes
|
||||
|
||||
int address = nodeID * bytesPerIndexEntry;
|
||||
int splitDim;
|
||||
if (numDims == 1) {
|
||||
splitDim = 0;
|
||||
} else {
|
||||
splitDim = splitPackedValues[address++] & 0xff;
|
||||
}
|
||||
|
||||
assert splitDim < numDims;
|
||||
|
||||
// TODO: can we alloc & reuse this up front?
|
||||
|
||||
byte[] splitPackedValue = new byte[packedBytesLength];
|
||||
|
||||
// Recurse on left sub-tree:
|
||||
System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
|
||||
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
final long leftCost = estimatePointCount(state,
|
||||
2*nodeID,
|
||||
cellMinPacked, splitPackedValue);
|
||||
|
||||
// Recurse on right sub-tree:
|
||||
System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
|
||||
System.arraycopy(splitPackedValues, address, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
final long rightCost = estimatePointCount(state,
|
||||
2*nodeID+1,
|
||||
splitPackedValue, cellMaxPacked);
|
||||
return leftCost + rightCost;
|
||||
}
|
||||
}
|
||||
|
||||
/** Copies the split value for this node into the provided byte array */
|
||||
public void copySplitValue(int nodeID, byte[] splitPackedValue) {
|
||||
int address = nodeID * bytesPerIndexEntry;
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.lucene.util.automaton.Automaton;
|
|||
public class TokenStreamToAutomaton {
|
||||
|
||||
private boolean preservePositionIncrements;
|
||||
private boolean finalOffsetGapAsHole;
|
||||
private boolean unicodeArcs;
|
||||
|
||||
/** Sole constructor. */
|
||||
|
@ -51,6 +52,11 @@ public class TokenStreamToAutomaton {
|
|||
this.preservePositionIncrements = enablePositionIncrements;
|
||||
}
|
||||
|
||||
/** If true, any final offset gaps will result in adding a position hole. */
|
||||
public void setFinalOffsetGapAsHole(boolean finalOffsetGapAsHole) {
|
||||
this.finalOffsetGapAsHole = finalOffsetGapAsHole;
|
||||
}
|
||||
|
||||
/** Whether to make transition labels Unicode code points instead of UTF8 bytes,
|
||||
* <code>false</code> by default */
|
||||
public void setUnicodeArcs(boolean unicodeArcs) {
|
||||
|
@ -118,7 +124,7 @@ public class TokenStreamToAutomaton {
|
|||
int maxOffset = 0;
|
||||
while (in.incrementToken()) {
|
||||
int posInc = posIncAtt.getPositionIncrement();
|
||||
if (!preservePositionIncrements && posInc > 1) {
|
||||
if (preservePositionIncrements == false && posInc > 1) {
|
||||
posInc = 1;
|
||||
}
|
||||
assert pos > -1 || posInc > 0;
|
||||
|
@ -201,10 +207,35 @@ public class TokenStreamToAutomaton {
|
|||
}
|
||||
|
||||
in.end();
|
||||
|
||||
int endState = -1;
|
||||
if (offsetAtt.endOffset() > maxOffset) {
|
||||
|
||||
int endPosInc = posIncAtt.getPositionIncrement();
|
||||
|
||||
if (endPosInc == 0 && finalOffsetGapAsHole && offsetAtt.endOffset() > maxOffset) {
|
||||
endPosInc = 1;
|
||||
}
|
||||
|
||||
if (endPosInc > 0) {
|
||||
// there were hole(s) after the last token
|
||||
endState = builder.createState();
|
||||
builder.setAccept(endState, true);
|
||||
|
||||
// add trailing holes now:
|
||||
int lastState = endState;
|
||||
while (true) {
|
||||
int state1 = builder.createState();
|
||||
builder.addTransition(lastState, state1, HOLE);
|
||||
endPosInc--;
|
||||
if (endPosInc == 0) {
|
||||
builder.setAccept(state1, true);
|
||||
break;
|
||||
}
|
||||
int state2 = builder.createState();
|
||||
builder.addTransition(state1, state2, POS_SEP);
|
||||
lastState = state2;
|
||||
}
|
||||
} else {
|
||||
endState = -1;
|
||||
}
|
||||
|
||||
pos++;
|
||||
|
@ -219,7 +250,7 @@ public class TokenStreamToAutomaton {
|
|||
}
|
||||
pos++;
|
||||
}
|
||||
|
||||
|
||||
return builder.finish();
|
||||
}
|
||||
|
||||
|
|
|
@ -43,7 +43,7 @@ public class OffsetAttributeImpl extends AttributeImpl implements OffsetAttribut
|
|||
// OffsetAtt
|
||||
|
||||
if (startOffset < 0 || endOffset < startOffset) {
|
||||
throw new IllegalArgumentException("startOffset must be non-negative, and endOffset must be >= startOffset, "
|
||||
throw new IllegalArgumentException("startOffset must be non-negative, and endOffset must be >= startOffset; got "
|
||||
+ "startOffset=" + startOffset + ",endOffset=" + endOffset);
|
||||
}
|
||||
|
||||
|
|
|
@ -107,7 +107,7 @@ public class PackedTokenAttributeImpl extends CharTermAttributeImpl
|
|||
@Override
|
||||
public void setOffset(int startOffset, int endOffset) {
|
||||
if (startOffset < 0 || endOffset < startOffset) {
|
||||
throw new IllegalArgumentException("startOffset must be non-negative, and endOffset must be >= startOffset, "
|
||||
throw new IllegalArgumentException("startOffset must be non-negative, and endOffset must be >= startOffset; got "
|
||||
+ "startOffset=" + startOffset + ",endOffset=" + endOffset);
|
||||
}
|
||||
this.startOffset = startOffset;
|
||||
|
|
|
@ -30,8 +30,7 @@ public class PositionIncrementAttributeImpl extends AttributeImpl implements Pos
|
|||
@Override
|
||||
public void setPositionIncrement(int positionIncrement) {
|
||||
if (positionIncrement < 0) {
|
||||
throw new IllegalArgumentException
|
||||
("Increment must be zero or greater: got " + positionIncrement);
|
||||
throw new IllegalArgumentException("Position increment must be zero or greater; got " + positionIncrement);
|
||||
}
|
||||
this.positionIncrement = positionIncrement;
|
||||
}
|
||||
|
|
|
@ -30,8 +30,7 @@ public class PositionLengthAttributeImpl extends AttributeImpl implements Positi
|
|||
@Override
|
||||
public void setPositionLength(int positionLength) {
|
||||
if (positionLength < 1) {
|
||||
throw new IllegalArgumentException
|
||||
("Position length must be 1 or greater: got " + positionLength);
|
||||
throw new IllegalArgumentException("Position length must be 1 or greater; got " + positionLength);
|
||||
}
|
||||
this.positionLength = positionLength;
|
||||
}
|
||||
|
|
|
@ -127,6 +127,11 @@ public abstract class PointsWriter implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimatePointCount(IntersectVisitor visitor) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue() {
|
||||
throw new UnsupportedOperationException();
|
||||
|
|
|
@ -42,6 +42,8 @@ import org.apache.lucene.codecs.TermVectorsReader;
|
|||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DocumentStoredFieldVisitor;
|
||||
import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.Sort;
|
||||
|
@ -1810,6 +1812,19 @@ public final class CheckIndex implements Closeable {
|
|||
long size = values.size();
|
||||
int docCount = values.getDocCount();
|
||||
|
||||
final long crossCost = values.estimatePointCount(new ConstantRelationIntersectVisitor(Relation.CELL_CROSSES_QUERY));
|
||||
if (crossCost < size / 2) {
|
||||
throw new RuntimeException("estimatePointCount should return >= size/2 when all cells match");
|
||||
}
|
||||
final long insideCost = values.estimatePointCount(new ConstantRelationIntersectVisitor(Relation.CELL_INSIDE_QUERY));
|
||||
if (insideCost < size) {
|
||||
throw new RuntimeException("estimatePointCount should return >= size when all cells fully match");
|
||||
}
|
||||
final long outsideCost = values.estimatePointCount(new ConstantRelationIntersectVisitor(Relation.CELL_OUTSIDE_QUERY));
|
||||
if (outsideCost != 0) {
|
||||
throw new RuntimeException("estimatePointCount should return 0 when no cells match");
|
||||
}
|
||||
|
||||
VerifyPointsVisitor visitor = new VerifyPointsVisitor(fieldInfo.name, reader.maxDoc(), values);
|
||||
values.intersect(visitor);
|
||||
|
||||
|
@ -2002,6 +2017,28 @@ public final class CheckIndex implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
private static class ConstantRelationIntersectVisitor implements IntersectVisitor {
|
||||
private final Relation relation;
|
||||
|
||||
ConstantRelationIntersectVisitor(Relation relation) {
|
||||
this.relation = relation;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
return relation;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test stored fields.
|
||||
|
|
|
@ -313,10 +313,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
|
||||
@Override
|
||||
public void abort() {
|
||||
try {
|
||||
storedFieldsConsumer.abort();
|
||||
} catch (Throwable t) {
|
||||
}
|
||||
storedFieldsConsumer.abort();
|
||||
|
||||
try {
|
||||
// E.g. close any open files in the term vectors writer:
|
||||
|
|
|
@ -1034,17 +1034,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
|
||||
/** Confirms that the incoming index sort (if any) matches the existing index sort (if any).
|
||||
* This is unfortunately just best effort, because it could be the old index only has unsorted flushed segments built
|
||||
* before {@link Version#LUCENE_7_0_0} (flushed segments are sorted in Lucene 7.0). */
|
||||
private void validateIndexSort() {
|
||||
* before {@link Version#LUCENE_6_5_0} (flushed segments are sorted in Lucene 7.0). */
|
||||
private void validateIndexSort() throws CorruptIndexException {
|
||||
Sort indexSort = config.getIndexSort();
|
||||
if (indexSort != null) {
|
||||
for(SegmentCommitInfo info : segmentInfos) {
|
||||
Sort segmentIndexSort = info.info.getIndexSort();
|
||||
if (segmentIndexSort != null && indexSort.equals(segmentIndexSort) == false) {
|
||||
throw new IllegalArgumentException("cannot change previous indexSort=" + segmentIndexSort + " (from segment=" + info + ") to new indexSort=" + indexSort);
|
||||
} else if (segmentIndexSort == null) {
|
||||
// Flushed segments are not sorted if they were built with a version prior to 7.0
|
||||
assert info.info.getVersion().onOrAfter(Version.LUCENE_7_0_0) == false;
|
||||
} else if (segmentIndexSort == null && info.info.getVersion().onOrAfter(Version.LUCENE_6_5_0)) {
|
||||
// Flushed segments are not sorted if they were built with a version prior to 6.5.0
|
||||
throw new CorruptIndexException("segment not sorted with indexSort=" + segmentIndexSort, info.info.toString());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.document.Field;
|
|||
import org.apache.lucene.document.FloatPoint;
|
||||
import org.apache.lucene.document.IntPoint;
|
||||
import org.apache.lucene.document.LongPoint;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.bkd.BKDWriter;
|
||||
|
||||
|
@ -220,6 +221,12 @@ public abstract class PointValues {
|
|||
* to test whether each document is deleted, if necessary. */
|
||||
public abstract void intersect(IntersectVisitor visitor) throws IOException;
|
||||
|
||||
/** Estimate the number of points that would be visited by {@link #intersect}
|
||||
* with the given {@link IntersectVisitor}. This should run many times faster
|
||||
* than {@link #intersect(IntersectVisitor)}.
|
||||
* @see DocIdSetIterator#cost */
|
||||
public abstract long estimatePointCount(IntersectVisitor visitor);
|
||||
|
||||
/** Returns minimum value for each dimension, packed, or null if {@link #size} is <code>0</code> */
|
||||
public abstract byte[] getMinPackedValue() throws IOException;
|
||||
|
||||
|
|
|
@ -90,6 +90,11 @@ class PointValuesWriter {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimatePointCount(IntersectVisitor visitor) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue() {
|
||||
throw new UnsupportedOperationException();
|
||||
|
@ -208,6 +213,11 @@ class PointValuesWriter {
|
|||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimatePointCount(IntersectVisitor visitor) {
|
||||
return in.estimatePointCount(visitor);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue() throws IOException {
|
||||
return in.getMinPackedValue();
|
||||
|
|
|
@ -42,7 +42,7 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
|||
|
||||
/**
|
||||
* An {@link org.apache.lucene.index.LeafReader} which supports sorting documents by a given
|
||||
* {@link Sort}. This is package private and is only used by Lucene fo BWC when it needs to merge
|
||||
* {@link Sort}. This is package private and is only used by Lucene for BWC when it needs to merge
|
||||
* an unsorted flushed segment built by an older version (newly flushed segments are sorted since version 7.0).
|
||||
*
|
||||
* @lucene.experimental
|
||||
|
@ -327,6 +327,11 @@ class SortingLeafReader extends FilterLeafReader {
|
|||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimatePointCount(IntersectVisitor visitor) {
|
||||
return in.estimatePointCount(visitor);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue() throws IOException {
|
||||
return in.getMinPackedValue();
|
||||
|
|
|
@ -0,0 +1,217 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You 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,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.search;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
final class Boolean2ScorerSupplier extends ScorerSupplier {
|
||||
|
||||
private final BooleanWeight weight;
|
||||
private final Map<BooleanClause.Occur, Collection<ScorerSupplier>> subs;
|
||||
private final boolean needsScores;
|
||||
private final int minShouldMatch;
|
||||
private long cost = -1;
|
||||
|
||||
Boolean2ScorerSupplier(BooleanWeight weight,
|
||||
Map<Occur, Collection<ScorerSupplier>> subs,
|
||||
boolean needsScores, int minShouldMatch) {
|
||||
if (minShouldMatch < 0) {
|
||||
throw new IllegalArgumentException("minShouldMatch must be positive, but got: " + minShouldMatch);
|
||||
}
|
||||
if (minShouldMatch != 0 && minShouldMatch >= subs.get(Occur.SHOULD).size()) {
|
||||
throw new IllegalArgumentException("minShouldMatch must be strictly less than the number of SHOULD clauses");
|
||||
}
|
||||
if (needsScores == false && minShouldMatch == 0 && subs.get(Occur.SHOULD).size() > 0
|
||||
&& subs.get(Occur.MUST).size() + subs.get(Occur.FILTER).size() > 0) {
|
||||
throw new IllegalArgumentException("Cannot pass purely optional clauses if scores are not needed");
|
||||
}
|
||||
if (subs.get(Occur.SHOULD).size() + subs.get(Occur.MUST).size() + subs.get(Occur.FILTER).size() == 0) {
|
||||
throw new IllegalArgumentException("There should be at least one positive clause");
|
||||
}
|
||||
this.weight = weight;
|
||||
this.subs = subs;
|
||||
this.needsScores = needsScores;
|
||||
this.minShouldMatch = minShouldMatch;
|
||||
}
|
||||
|
||||
private long computeCost() {
|
||||
OptionalLong minRequiredCost = Stream.concat(
|
||||
subs.get(Occur.MUST).stream(),
|
||||
subs.get(Occur.FILTER).stream())
|
||||
.mapToLong(ScorerSupplier::cost)
|
||||
.min();
|
||||
if (minRequiredCost.isPresent() && minShouldMatch == 0) {
|
||||
return minRequiredCost.getAsLong();
|
||||
} else {
|
||||
final Collection<ScorerSupplier> optionalScorers = subs.get(Occur.SHOULD);
|
||||
final long shouldCost = MinShouldMatchSumScorer.cost(
|
||||
optionalScorers.stream().mapToLong(ScorerSupplier::cost),
|
||||
optionalScorers.size(), minShouldMatch);
|
||||
return Math.min(minRequiredCost.orElse(Long.MAX_VALUE), shouldCost);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
if (cost == -1) {
|
||||
cost = computeCost();
|
||||
}
|
||||
return cost;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer get(boolean randomAccess) throws IOException {
|
||||
// three cases: conjunction, disjunction, or mix
|
||||
|
||||
// pure conjunction
|
||||
if (subs.get(Occur.SHOULD).isEmpty()) {
|
||||
return excl(req(subs.get(Occur.FILTER), subs.get(Occur.MUST), randomAccess), subs.get(Occur.MUST_NOT));
|
||||
}
|
||||
|
||||
// pure disjunction
|
||||
if (subs.get(Occur.FILTER).isEmpty() && subs.get(Occur.MUST).isEmpty()) {
|
||||
return excl(opt(subs.get(Occur.SHOULD), minShouldMatch, needsScores, randomAccess), subs.get(Occur.MUST_NOT));
|
||||
}
|
||||
|
||||
// conjunction-disjunction mix:
|
||||
// we create the required and optional pieces, and then
|
||||
// combine the two: if minNrShouldMatch > 0, then it's a conjunction: because the
|
||||
// optional side must match. otherwise it's required + optional
|
||||
|
||||
if (minShouldMatch > 0) {
|
||||
boolean reqRandomAccess = true;
|
||||
boolean msmRandomAccess = true;
|
||||
if (randomAccess == false) {
|
||||
// We need to figure out whether the MUST/FILTER or the SHOULD clauses would lead the iteration
|
||||
final long reqCost = Stream.concat(
|
||||
subs.get(Occur.MUST).stream(),
|
||||
subs.get(Occur.FILTER).stream())
|
||||
.mapToLong(ScorerSupplier::cost)
|
||||
.min().getAsLong();
|
||||
final long msmCost = MinShouldMatchSumScorer.cost(
|
||||
subs.get(Occur.SHOULD).stream().mapToLong(ScorerSupplier::cost),
|
||||
subs.get(Occur.SHOULD).size(), minShouldMatch);
|
||||
reqRandomAccess = reqCost > msmCost;
|
||||
msmRandomAccess = msmCost > reqCost;
|
||||
}
|
||||
Scorer req = excl(req(subs.get(Occur.FILTER), subs.get(Occur.MUST), reqRandomAccess), subs.get(Occur.MUST_NOT));
|
||||
Scorer opt = opt(subs.get(Occur.SHOULD), minShouldMatch, needsScores, msmRandomAccess);
|
||||
return new ConjunctionScorer(weight, Arrays.asList(req, opt), Arrays.asList(req, opt));
|
||||
} else {
|
||||
assert needsScores;
|
||||
return new ReqOptSumScorer(
|
||||
excl(req(subs.get(Occur.FILTER), subs.get(Occur.MUST), randomAccess), subs.get(Occur.MUST_NOT)),
|
||||
opt(subs.get(Occur.SHOULD), minShouldMatch, needsScores, true));
|
||||
}
|
||||
}
|
||||
|
||||
/** Create a new scorer for the given required clauses. Note that
|
||||
* {@code requiredScoring} is a subset of {@code required} containing
|
||||
* required clauses that should participate in scoring. */
|
||||
private Scorer req(Collection<ScorerSupplier> requiredNoScoring, Collection<ScorerSupplier> requiredScoring, boolean randomAccess) throws IOException {
|
||||
if (requiredNoScoring.size() + requiredScoring.size() == 1) {
|
||||
Scorer req = (requiredNoScoring.isEmpty() ? requiredScoring : requiredNoScoring).iterator().next().get(randomAccess);
|
||||
|
||||
if (needsScores == false) {
|
||||
return req;
|
||||
}
|
||||
|
||||
if (requiredScoring.isEmpty()) {
|
||||
// Scores are needed but we only have a filter clause
|
||||
// BooleanWeight expects that calling score() is ok so we need to wrap
|
||||
// to prevent score() from being propagated
|
||||
return new FilterScorer(req) {
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return 0f;
|
||||
}
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
return req;
|
||||
} else {
|
||||
long minCost = Math.min(
|
||||
requiredNoScoring.stream().mapToLong(ScorerSupplier::cost).min().orElse(Long.MAX_VALUE),
|
||||
requiredScoring.stream().mapToLong(ScorerSupplier::cost).min().orElse(Long.MAX_VALUE));
|
||||
List<Scorer> requiredScorers = new ArrayList<>();
|
||||
List<Scorer> scoringScorers = new ArrayList<>();
|
||||
for (ScorerSupplier s : requiredNoScoring) {
|
||||
requiredScorers.add(s.get(randomAccess || s.cost() > minCost));
|
||||
}
|
||||
for (ScorerSupplier s : requiredScoring) {
|
||||
Scorer scorer = s.get(randomAccess || s.cost() > minCost);
|
||||
requiredScorers.add(scorer);
|
||||
scoringScorers.add(scorer);
|
||||
}
|
||||
return new ConjunctionScorer(weight, requiredScorers, scoringScorers);
|
||||
}
|
||||
}
|
||||
|
||||
private Scorer excl(Scorer main, Collection<ScorerSupplier> prohibited) throws IOException {
|
||||
if (prohibited.isEmpty()) {
|
||||
return main;
|
||||
} else {
|
||||
return new ReqExclScorer(main, opt(prohibited, 1, false, true));
|
||||
}
|
||||
}
|
||||
|
||||
private Scorer opt(Collection<ScorerSupplier> optional, int minShouldMatch,
|
||||
boolean needsScores, boolean randomAccess) throws IOException {
|
||||
if (optional.size() == 1) {
|
||||
return optional.iterator().next().get(randomAccess);
|
||||
} else if (minShouldMatch > 1) {
|
||||
final List<Scorer> optionalScorers = new ArrayList<>();
|
||||
final PriorityQueue<ScorerSupplier> pq = new PriorityQueue<ScorerSupplier>(subs.get(Occur.SHOULD).size() - minShouldMatch + 1) {
|
||||
@Override
|
||||
protected boolean lessThan(ScorerSupplier a, ScorerSupplier b) {
|
||||
return a.cost() > b.cost();
|
||||
}
|
||||
};
|
||||
for (ScorerSupplier scorer : subs.get(Occur.SHOULD)) {
|
||||
ScorerSupplier overflow = pq.insertWithOverflow(scorer);
|
||||
if (overflow != null) {
|
||||
optionalScorers.add(overflow.get(true));
|
||||
}
|
||||
}
|
||||
for (ScorerSupplier scorer : pq) {
|
||||
optionalScorers.add(scorer.get(randomAccess));
|
||||
}
|
||||
return new MinShouldMatchSumScorer(weight, optionalScorers, minShouldMatch);
|
||||
} else {
|
||||
final List<Scorer> optionalScorers = new ArrayList<>();
|
||||
for (ScorerSupplier scorer : optional) {
|
||||
optionalScorers.add(scorer.get(randomAccess));
|
||||
}
|
||||
return new DisjunctionSumScorer(weight, optionalScorers, needsScores);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -19,9 +19,11 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.EnumMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
@ -265,7 +267,9 @@ final class BooleanWeight extends Weight {
|
|||
if (prohibited.isEmpty()) {
|
||||
return positiveScorer;
|
||||
} else {
|
||||
Scorer prohibitedScorer = opt(prohibited, 1);
|
||||
Scorer prohibitedScorer = prohibited.size() == 1
|
||||
? prohibited.get(0)
|
||||
: new DisjunctionSumScorer(this, prohibited, false);
|
||||
if (prohibitedScorer.twoPhaseIterator() != null) {
|
||||
// ReqExclBulkScorer can't deal efficiently with two-phased prohibited clauses
|
||||
return null;
|
||||
|
@ -288,50 +292,48 @@ final class BooleanWeight extends Weight {
|
|||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
// initially the user provided value,
|
||||
// but if minNrShouldMatch == optional.size(),
|
||||
// we will optimize and move these to required, making this 0
|
||||
ScorerSupplier scorerSupplier = scorerSupplier(context);
|
||||
if (scorerSupplier == null) {
|
||||
return null;
|
||||
}
|
||||
return scorerSupplier.get(false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
|
||||
int minShouldMatch = query.getMinimumNumberShouldMatch();
|
||||
|
||||
List<Scorer> required = new ArrayList<>();
|
||||
// clauses that are required AND participate in scoring, subset of 'required'
|
||||
List<Scorer> requiredScoring = new ArrayList<>();
|
||||
List<Scorer> prohibited = new ArrayList<>();
|
||||
List<Scorer> optional = new ArrayList<>();
|
||||
final Map<Occur, Collection<ScorerSupplier>> scorers = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
scorers.put(occur, new ArrayList<>());
|
||||
}
|
||||
|
||||
Iterator<BooleanClause> cIter = query.iterator();
|
||||
for (Weight w : weights) {
|
||||
BooleanClause c = cIter.next();
|
||||
Scorer subScorer = w.scorer(context);
|
||||
ScorerSupplier subScorer = w.scorerSupplier(context);
|
||||
if (subScorer == null) {
|
||||
if (c.isRequired()) {
|
||||
return null;
|
||||
}
|
||||
} else if (c.isRequired()) {
|
||||
required.add(subScorer);
|
||||
if (c.isScoring()) {
|
||||
requiredScoring.add(subScorer);
|
||||
}
|
||||
} else if (c.isProhibited()) {
|
||||
prohibited.add(subScorer);
|
||||
} else {
|
||||
optional.add(subScorer);
|
||||
scorers.get(c.getOccur()).add(subScorer);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// scorer simplifications:
|
||||
|
||||
if (optional.size() == minShouldMatch) {
|
||||
if (scorers.get(Occur.SHOULD).size() == minShouldMatch) {
|
||||
// any optional clauses are in fact required
|
||||
required.addAll(optional);
|
||||
requiredScoring.addAll(optional);
|
||||
optional.clear();
|
||||
scorers.get(Occur.MUST).addAll(scorers.get(Occur.SHOULD));
|
||||
scorers.get(Occur.SHOULD).clear();
|
||||
minShouldMatch = 0;
|
||||
}
|
||||
|
||||
if (required.isEmpty() && optional.isEmpty()) {
|
||||
if (scorers.get(Occur.FILTER).isEmpty() && scorers.get(Occur.MUST).isEmpty() && scorers.get(Occur.SHOULD).isEmpty()) {
|
||||
// no required and optional clauses.
|
||||
return null;
|
||||
} else if (optional.size() < minShouldMatch) {
|
||||
} else if (scorers.get(Occur.SHOULD).size() < minShouldMatch) {
|
||||
// either >1 req scorer, or there are 0 req scorers and at least 1
|
||||
// optional scorer. Therefore if there are not enough optional scorers
|
||||
// no documents will be matched by the query
|
||||
|
@ -339,87 +341,11 @@ final class BooleanWeight extends Weight {
|
|||
}
|
||||
|
||||
// we don't need scores, so if we have required clauses, drop optional clauses completely
|
||||
if (!needsScores && minShouldMatch == 0 && required.size() > 0) {
|
||||
optional.clear();
|
||||
if (!needsScores && minShouldMatch == 0 && scorers.get(Occur.MUST).size() + scorers.get(Occur.FILTER).size() > 0) {
|
||||
scorers.get(Occur.SHOULD).clear();
|
||||
}
|
||||
|
||||
// three cases: conjunction, disjunction, or mix
|
||||
|
||||
// pure conjunction
|
||||
if (optional.isEmpty()) {
|
||||
return excl(req(required, requiredScoring), prohibited);
|
||||
}
|
||||
|
||||
// pure disjunction
|
||||
if (required.isEmpty()) {
|
||||
return excl(opt(optional, minShouldMatch), prohibited);
|
||||
}
|
||||
|
||||
// conjunction-disjunction mix:
|
||||
// we create the required and optional pieces, and then
|
||||
// combine the two: if minNrShouldMatch > 0, then it's a conjunction: because the
|
||||
// optional side must match. otherwise it's required + optional
|
||||
|
||||
Scorer req = excl(req(required, requiredScoring), prohibited);
|
||||
Scorer opt = opt(optional, minShouldMatch);
|
||||
|
||||
if (minShouldMatch > 0) {
|
||||
return new ConjunctionScorer(this, Arrays.asList(req, opt), Arrays.asList(req, opt));
|
||||
} else {
|
||||
return new ReqOptSumScorer(req, opt);
|
||||
}
|
||||
return new Boolean2ScorerSupplier(this, scorers, needsScores, minShouldMatch);
|
||||
}
|
||||
|
||||
/** Create a new scorer for the given required clauses. Note that
|
||||
* {@code requiredScoring} is a subset of {@code required} containing
|
||||
* required clauses that should participate in scoring. */
|
||||
private Scorer req(List<Scorer> required, List<Scorer> requiredScoring) {
|
||||
if (required.size() == 1) {
|
||||
Scorer req = required.get(0);
|
||||
|
||||
if (needsScores == false) {
|
||||
return req;
|
||||
}
|
||||
|
||||
if (requiredScoring.isEmpty()) {
|
||||
// Scores are needed but we only have a filter clause
|
||||
// BooleanWeight expects that calling score() is ok so we need to wrap
|
||||
// to prevent score() from being propagated
|
||||
return new FilterScorer(req) {
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return 0f;
|
||||
}
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
return req;
|
||||
} else {
|
||||
return new ConjunctionScorer(this, required, requiredScoring);
|
||||
}
|
||||
}
|
||||
|
||||
private Scorer excl(Scorer main, List<Scorer> prohibited) throws IOException {
|
||||
if (prohibited.isEmpty()) {
|
||||
return main;
|
||||
} else if (prohibited.size() == 1) {
|
||||
return new ReqExclScorer(main, prohibited.get(0));
|
||||
} else {
|
||||
return new ReqExclScorer(main, new DisjunctionSumScorer(this, prohibited, false));
|
||||
}
|
||||
}
|
||||
|
||||
private Scorer opt(List<Scorer> optional, int minShouldMatch) throws IOException {
|
||||
if (optional.size() == 1) {
|
||||
return optional.get(0);
|
||||
} else if (minShouldMatch > 1) {
|
||||
return new MinShouldMatchSumScorer(this, optional, minShouldMatch);
|
||||
} else {
|
||||
return new DisjunctionSumScorer(this, optional, needsScores);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,7 +41,7 @@ public final class ConjunctionDISI extends DocIdSetIterator {
|
|||
* returned {@link DocIdSetIterator} might leverage two-phase iteration in
|
||||
* which case it is possible to retrieve the {@link TwoPhaseIterator} using
|
||||
* {@link TwoPhaseIterator#unwrap}. */
|
||||
public static DocIdSetIterator intersectScorers(List<Scorer> scorers) {
|
||||
public static DocIdSetIterator intersectScorers(Collection<Scorer> scorers) {
|
||||
if (scorers.size() < 2) {
|
||||
throw new IllegalArgumentException("Cannot make a ConjunctionDISI of less than 2 iterators");
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.lucene.search;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
/** Scorer for conjunctions, sets of queries, all of which are required. */
|
||||
class ConjunctionScorer extends Scorer {
|
||||
|
@ -29,7 +28,7 @@ class ConjunctionScorer extends Scorer {
|
|||
final Scorer[] scorers;
|
||||
|
||||
/** Create a new {@link ConjunctionScorer}, note that {@code scorers} must be a subset of {@code required}. */
|
||||
ConjunctionScorer(Weight weight, List<Scorer> required, List<Scorer> scorers) {
|
||||
ConjunctionScorer(Weight weight, Collection<Scorer> required, Collection<Scorer> scorers) {
|
||||
super(weight);
|
||||
assert required.containsAll(scorers);
|
||||
this.disi = ConjunctionDISI.intersectScorers(required);
|
||||
|
|
|
@ -125,28 +125,48 @@ public final class ConstantScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
final Scorer innerScorer = innerWeight.scorer(context);
|
||||
if (innerScorer == null) {
|
||||
public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
|
||||
ScorerSupplier innerScorerSupplier = innerWeight.scorerSupplier(context);
|
||||
if (innerScorerSupplier == null) {
|
||||
return null;
|
||||
}
|
||||
final float score = score();
|
||||
return new FilterScorer(innerScorer) {
|
||||
return new ScorerSupplier() {
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return score;
|
||||
public Scorer get(boolean randomAccess) throws IOException {
|
||||
final Scorer innerScorer = innerScorerSupplier.get(randomAccess);
|
||||
final float score = score();
|
||||
return new FilterScorer(innerScorer) {
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return score;
|
||||
}
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return 1;
|
||||
}
|
||||
@Override
|
||||
public Collection<ChildScorer> getChildren() {
|
||||
return Collections.singleton(new ChildScorer(innerScorer, "constant"));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return 1;
|
||||
}
|
||||
@Override
|
||||
public Collection<ChildScorer> getChildren() {
|
||||
return Collections.singleton(new ChildScorer(innerScorer, "constant"));
|
||||
public long cost() {
|
||||
return innerScorerSupplier.cost();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
ScorerSupplier scorerSupplier = scorerSupplier(context);
|
||||
if (scorerSupplier == null) {
|
||||
return null;
|
||||
}
|
||||
return scorerSupplier.get(false);
|
||||
}
|
||||
|
||||
};
|
||||
} else {
|
||||
return innerWeight;
|
||||
|
|
|
@ -22,6 +22,8 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.stream.LongStream;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
|
@ -47,7 +49,7 @@ import static org.apache.lucene.search.DisiPriorityQueue.rightNode;
|
|||
*/
|
||||
final class MinShouldMatchSumScorer extends Scorer {
|
||||
|
||||
private static long cost(Collection<Scorer> scorers, int minShouldMatch) {
|
||||
static long cost(LongStream costs, int numScorers, int minShouldMatch) {
|
||||
// the idea here is the following: a boolean query c1,c2,...cn with minShouldMatch=m
|
||||
// could be rewritten to:
|
||||
// (c1 AND (c2..cn|msm=m-1)) OR (!c1 AND (c2..cn|msm=m))
|
||||
|
@ -61,20 +63,14 @@ final class MinShouldMatchSumScorer extends Scorer {
|
|||
|
||||
// If we recurse infinitely, we find out that the cost of a msm query is the sum of the
|
||||
// costs of the num_scorers - minShouldMatch + 1 least costly scorers
|
||||
final PriorityQueue<Scorer> pq = new PriorityQueue<Scorer>(scorers.size() - minShouldMatch + 1) {
|
||||
final PriorityQueue<Long> pq = new PriorityQueue<Long>(numScorers - minShouldMatch + 1) {
|
||||
@Override
|
||||
protected boolean lessThan(Scorer a, Scorer b) {
|
||||
return a.iterator().cost() > b.iterator().cost();
|
||||
protected boolean lessThan(Long a, Long b) {
|
||||
return a > b;
|
||||
}
|
||||
};
|
||||
for (Scorer scorer : scorers) {
|
||||
pq.insertWithOverflow(scorer);
|
||||
}
|
||||
long cost = 0;
|
||||
for (Scorer scorer = pq.pop(); scorer != null; scorer = pq.pop()) {
|
||||
cost += scorer.iterator().cost();
|
||||
}
|
||||
return cost;
|
||||
costs.forEach(pq::insertWithOverflow);
|
||||
return StreamSupport.stream(pq.spliterator(), false).mapToLong(Number::longValue).sum();
|
||||
}
|
||||
|
||||
final int minShouldMatch;
|
||||
|
@ -124,7 +120,7 @@ final class MinShouldMatchSumScorer extends Scorer {
|
|||
children.add(new ChildScorer(scorer, "SHOULD"));
|
||||
}
|
||||
this.childScorers = Collections.unmodifiableCollection(children);
|
||||
this.cost = cost(scorers, minShouldMatch);
|
||||
this.cost = cost(scorers.stream().map(Scorer::iterator).mapToLong(DocIdSetIterator::cost), scorers.size(), minShouldMatch);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -26,7 +26,9 @@ import org.apache.lucene.index.PointValues.Relation;
|
|||
import org.apache.lucene.document.IntPoint; // javadocs
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.BitSetIterator;
|
||||
import org.apache.lucene.util.DocIdSetBuilder;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/**
|
||||
|
@ -104,71 +106,125 @@ public abstract class PointRangeQuery extends Query {
|
|||
|
||||
return new ConstantScoreWeight(this, boost) {
|
||||
|
||||
private DocIdSet buildMatchingDocIdSet(LeafReader reader, PointValues values) throws IOException {
|
||||
DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
|
||||
private IntersectVisitor getIntersectVisitor(DocIdSetBuilder result) {
|
||||
return new IntersectVisitor() {
|
||||
|
||||
values.intersect(
|
||||
new IntersectVisitor() {
|
||||
DocIdSetBuilder.BulkAdder adder;
|
||||
|
||||
DocIdSetBuilder.BulkAdder adder;
|
||||
@Override
|
||||
public void grow(int count) {
|
||||
adder = result.grow(count);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void grow(int count) {
|
||||
adder = result.grow(count);
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
adder.add(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int offset = dim*bytesPerDim;
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, lowerPoint, offset) < 0) {
|
||||
// Doc's value is too low, in this dimension
|
||||
return;
|
||||
}
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, upperPoint, offset) > 0) {
|
||||
// Doc's value is too high, in this dimension
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
// Doc is in-bounds
|
||||
adder.add(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
|
||||
boolean crosses = false;
|
||||
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int offset = dim*bytesPerDim;
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, minPackedValue, offset, upperPoint, offset) > 0 ||
|
||||
StringHelper.compare(bytesPerDim, maxPackedValue, offset, lowerPoint, offset) < 0) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
adder.add(docID);
|
||||
crosses |= StringHelper.compare(bytesPerDim, minPackedValue, offset, lowerPoint, offset) < 0 ||
|
||||
StringHelper.compare(bytesPerDim, maxPackedValue, offset, upperPoint, offset) > 0;
|
||||
}
|
||||
|
||||
if (crosses) {
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
} else {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a visitor that clears documents that do NOT match the range.
|
||||
*/
|
||||
private IntersectVisitor getInverseIntersectVisitor(FixedBitSet result, int[] cost) {
|
||||
return new IntersectVisitor() {
|
||||
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
result.clear(docID);
|
||||
cost[0]--;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int offset = dim*bytesPerDim;
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, lowerPoint, offset) < 0) {
|
||||
// Doc's value is too low, in this dimension
|
||||
result.clear(docID);
|
||||
cost[0]--;
|
||||
return;
|
||||
}
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, upperPoint, offset) > 0) {
|
||||
// Doc's value is too high, in this dimension
|
||||
result.clear(docID);
|
||||
cost[0]--;
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
|
||||
boolean crosses = false;
|
||||
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int offset = dim*bytesPerDim;
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, minPackedValue, offset, upperPoint, offset) > 0 ||
|
||||
StringHelper.compare(bytesPerDim, maxPackedValue, offset, lowerPoint, offset) < 0) {
|
||||
// This dim is not in the range
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int offset = dim*bytesPerDim;
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, lowerPoint, offset) < 0) {
|
||||
// Doc's value is too low, in this dimension
|
||||
return;
|
||||
}
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, upperPoint, offset) > 0) {
|
||||
// Doc's value is too high, in this dimension
|
||||
return;
|
||||
}
|
||||
}
|
||||
crosses |= StringHelper.compare(bytesPerDim, minPackedValue, offset, lowerPoint, offset) < 0 ||
|
||||
StringHelper.compare(bytesPerDim, maxPackedValue, offset, upperPoint, offset) > 0;
|
||||
}
|
||||
|
||||
// Doc is in-bounds
|
||||
adder.add(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
|
||||
boolean crosses = false;
|
||||
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int offset = dim*bytesPerDim;
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, minPackedValue, offset, upperPoint, offset) > 0 ||
|
||||
StringHelper.compare(bytesPerDim, maxPackedValue, offset, lowerPoint, offset) < 0) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
|
||||
crosses |= StringHelper.compare(bytesPerDim, minPackedValue, offset, lowerPoint, offset) < 0 ||
|
||||
StringHelper.compare(bytesPerDim, maxPackedValue, offset, upperPoint, offset) > 0;
|
||||
}
|
||||
|
||||
if (crosses) {
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
} else {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
}
|
||||
});
|
||||
return result.build();
|
||||
if (crosses) {
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
} else {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
|
||||
LeafReader reader = context.reader();
|
||||
|
||||
PointValues values = reader.getPointValues(field);
|
||||
|
@ -201,15 +257,69 @@ public abstract class PointRangeQuery extends Query {
|
|||
allDocsMatch = false;
|
||||
}
|
||||
|
||||
DocIdSetIterator iterator;
|
||||
final Weight weight = this;
|
||||
if (allDocsMatch) {
|
||||
// all docs have a value and all points are within bounds, so everything matches
|
||||
iterator = DocIdSetIterator.all(reader.maxDoc());
|
||||
return new ScorerSupplier() {
|
||||
@Override
|
||||
public Scorer get(boolean randomAccess) {
|
||||
return new ConstantScoreScorer(weight, score(),
|
||||
DocIdSetIterator.all(reader.maxDoc()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return reader.maxDoc();
|
||||
}
|
||||
};
|
||||
} else {
|
||||
iterator = buildMatchingDocIdSet(reader, values).iterator();
|
||||
}
|
||||
return new ScorerSupplier() {
|
||||
|
||||
return new ConstantScoreScorer(this, score(), iterator);
|
||||
final DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
|
||||
final IntersectVisitor visitor = getIntersectVisitor(result);
|
||||
long cost = -1;
|
||||
|
||||
@Override
|
||||
public Scorer get(boolean randomAccess) throws IOException {
|
||||
if (values.getDocCount() == reader.maxDoc()
|
||||
&& values.getDocCount() == values.size()
|
||||
&& cost() > reader.maxDoc() / 2) {
|
||||
// If all docs have exactly one value and the cost is greater
|
||||
// than half the leaf size then maybe we can make things faster
|
||||
// by computing the set of documents that do NOT match the range
|
||||
final FixedBitSet result = new FixedBitSet(reader.maxDoc());
|
||||
result.set(0, reader.maxDoc());
|
||||
int[] cost = new int[] { reader.maxDoc() };
|
||||
values.intersect(getInverseIntersectVisitor(result, cost));
|
||||
final DocIdSetIterator iterator = new BitSetIterator(result, cost[0]);
|
||||
return new ConstantScoreScorer(weight, score(), iterator);
|
||||
}
|
||||
|
||||
values.intersect(visitor);
|
||||
DocIdSetIterator iterator = result.build().iterator();
|
||||
return new ConstantScoreScorer(weight, score(), iterator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
if (cost == -1) {
|
||||
// Computing the cost may be expensive, so only do it if necessary
|
||||
cost = values.estimatePointCount(visitor);
|
||||
assert cost >= 0;
|
||||
}
|
||||
return cost;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
ScorerSupplier scorerSupplier = scorerSupplier(context);
|
||||
if (scorerSupplier == null) {
|
||||
return null;
|
||||
}
|
||||
return scorerSupplier.get(false);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You 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,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.search;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A supplier of {@link Scorer}. This allows to get an estimate of the cost before
|
||||
* building the {@link Scorer}.
|
||||
*/
|
||||
public abstract class ScorerSupplier {
|
||||
|
||||
/**
|
||||
* Get the {@link Scorer}. This may not return {@code null} and must be called
|
||||
* at most once.
|
||||
* @param randomAccess A hint about the expected usage of the {@link Scorer}.
|
||||
* If {@link DocIdSetIterator#advance} or {@link TwoPhaseIterator} will be
|
||||
* used to check whether given doc ids match, then pass {@code true}.
|
||||
* Otherwise if the {@link Scorer} will be mostly used to lead the iteration
|
||||
* using {@link DocIdSetIterator#nextDoc()}, then {@code false} should be
|
||||
* passed. Under doubt, pass {@code false} which usually has a better
|
||||
* worst-case.
|
||||
*/
|
||||
public abstract Scorer get(boolean randomAccess) throws IOException;
|
||||
|
||||
/**
|
||||
* Get an estimate of the {@link Scorer} that would be returned by {@link #get}.
|
||||
* This may be a costly operation, so it should only be called if necessary.
|
||||
* @see DocIdSetIterator#cost
|
||||
*/
|
||||
public abstract long cost();
|
||||
|
||||
}
|
|
@ -21,7 +21,6 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
@ -73,39 +72,12 @@ public class TermInSetQuery extends Query implements Accountable {
|
|||
// Same threshold as MultiTermQueryConstantScoreWrapper
|
||||
static final int BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD = 16;
|
||||
|
||||
private final boolean singleField; // whether all terms are from the same field
|
||||
private final String field;
|
||||
private final PrefixCodedTerms termData;
|
||||
private final int termDataHashCode; // cached hashcode of termData
|
||||
|
||||
/**
|
||||
* Creates a new {@link TermInSetQuery} from the given collection. It
|
||||
* can contain duplicate terms and multiple fields.
|
||||
*/
|
||||
public TermInSetQuery(Collection<Term> terms) {
|
||||
Term[] sortedTerms = terms.toArray(new Term[terms.size()]);
|
||||
// already sorted if we are a SortedSet with natural order
|
||||
boolean sorted = terms instanceof SortedSet && ((SortedSet<Term>)terms).comparator() == null;
|
||||
if (!sorted) {
|
||||
ArrayUtil.timSort(sortedTerms);
|
||||
}
|
||||
PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
|
||||
Set<String> fields = new HashSet<>();
|
||||
Term previous = null;
|
||||
for (Term term : sortedTerms) {
|
||||
if (term.equals(previous) == false) {
|
||||
fields.add(term.field());
|
||||
builder.add(term);
|
||||
}
|
||||
previous = term;
|
||||
}
|
||||
singleField = fields.size() == 1;
|
||||
termData = builder.finish();
|
||||
termDataHashCode = termData.hashCode();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link TermInSetQuery} from the given collection for
|
||||
* a single field. It can contain duplicate terms.
|
||||
* Creates a new {@link TermInSetQuery} from the given collection of terms.
|
||||
*/
|
||||
public TermInSetQuery(String field, Collection<BytesRef> terms) {
|
||||
BytesRef[] sortedTerms = terms.toArray(new BytesRef[terms.size()]);
|
||||
|
@ -125,27 +97,18 @@ public class TermInSetQuery extends Query implements Accountable {
|
|||
builder.add(field, term);
|
||||
previous.copyBytes(term);
|
||||
}
|
||||
singleField = true;
|
||||
this.field = field;
|
||||
termData = builder.finish();
|
||||
termDataHashCode = termData.hashCode();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link TermInSetQuery} from the given {@link BytesRef} array for
|
||||
* a single field.
|
||||
* Creates a new {@link TermInSetQuery} from the given array of terms.
|
||||
*/
|
||||
public TermInSetQuery(String field, BytesRef...terms) {
|
||||
this(field, Arrays.asList(terms));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link TermInSetQuery} from the given array. The array can
|
||||
* contain duplicate terms and multiple fields.
|
||||
*/
|
||||
public TermInSetQuery(final Term... terms) {
|
||||
this(Arrays.asList(terms));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query rewrite(IndexReader reader) throws IOException {
|
||||
final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
|
||||
|
@ -167,6 +130,7 @@ public class TermInSetQuery extends Query implements Accountable {
|
|||
}
|
||||
|
||||
private boolean equalsTo(TermInSetQuery other) {
|
||||
// no need to check 'field' explicitly since it is encoded in 'termData'
|
||||
// termData might be heavy to compare so check the hash code first
|
||||
return termDataHashCode == other.termDataHashCode &&
|
||||
termData.equals(other.termData);
|
||||
|
@ -260,6 +224,15 @@ public class TermInSetQuery extends Query implements Accountable {
|
|||
private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
|
||||
final LeafReader reader = context.reader();
|
||||
|
||||
final Fields fields = reader.fields();
|
||||
Terms terms = fields.terms(field);
|
||||
if (terms == null) {
|
||||
return null;
|
||||
}
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
PostingsEnum docs = null;
|
||||
TermIterator iterator = termData.iterator();
|
||||
|
||||
// We will first try to collect up to 'threshold' terms into 'matchingTerms'
|
||||
// if there are two many terms, we will fall back to building the 'builder'
|
||||
final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
|
||||
|
@ -267,25 +240,9 @@ public class TermInSetQuery extends Query implements Accountable {
|
|||
List<TermAndState> matchingTerms = new ArrayList<>(threshold);
|
||||
DocIdSetBuilder builder = null;
|
||||
|
||||
final Fields fields = reader.fields();
|
||||
String lastField = null;
|
||||
Terms terms = null;
|
||||
TermsEnum termsEnum = null;
|
||||
PostingsEnum docs = null;
|
||||
TermIterator iterator = termData.iterator();
|
||||
for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
|
||||
String field = iterator.field();
|
||||
// comparing references is fine here
|
||||
if (field != lastField) {
|
||||
terms = fields.terms(field);
|
||||
if (terms == null) {
|
||||
termsEnum = null;
|
||||
} else {
|
||||
termsEnum = terms.iterator();
|
||||
}
|
||||
lastField = field;
|
||||
}
|
||||
if (termsEnum != null && termsEnum.seekExact(term)) {
|
||||
assert field.equals(iterator.field());
|
||||
if (termsEnum.seekExact(term)) {
|
||||
if (matchingTerms == null) {
|
||||
docs = termsEnum.postings(docs, PostingsEnum.NONE);
|
||||
builder.add(docs);
|
||||
|
@ -293,15 +250,7 @@ public class TermInSetQuery extends Query implements Accountable {
|
|||
matchingTerms.add(new TermAndState(field, termsEnum));
|
||||
} else {
|
||||
assert matchingTerms.size() == threshold;
|
||||
if (singleField) {
|
||||
// common case: all terms are in the same field
|
||||
// use an optimized builder that leverages terms stats to be more efficient
|
||||
builder = new DocIdSetBuilder(reader.maxDoc(), terms);
|
||||
} else {
|
||||
// corner case: different fields
|
||||
// don't make assumptions about the docs we will get
|
||||
builder = new DocIdSetBuilder(reader.maxDoc());
|
||||
}
|
||||
builder = new DocIdSetBuilder(reader.maxDoc(), terms);
|
||||
docs = termsEnum.postings(docs, PostingsEnum.NONE);
|
||||
builder.add(docs);
|
||||
for (TermAndState t : matchingTerms) {
|
||||
|
@ -344,7 +293,9 @@ public class TermInSetQuery extends Query implements Accountable {
|
|||
@Override
|
||||
public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
|
||||
final WeightOrDocIdSet weightOrBitSet = rewrite(context);
|
||||
if (weightOrBitSet.weight != null) {
|
||||
if (weightOrBitSet == null) {
|
||||
return null;
|
||||
} else if (weightOrBitSet.weight != null) {
|
||||
return weightOrBitSet.weight.bulkScorer(context);
|
||||
} else {
|
||||
final Scorer scorer = scorer(weightOrBitSet.set);
|
||||
|
@ -358,7 +309,9 @@ public class TermInSetQuery extends Query implements Accountable {
|
|||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
final WeightOrDocIdSet weightOrBitSet = rewrite(context);
|
||||
if (weightOrBitSet.weight != null) {
|
||||
if (weightOrBitSet == null) {
|
||||
return null;
|
||||
} else if (weightOrBitSet.weight != null) {
|
||||
return weightOrBitSet.weight.scorer(context);
|
||||
} else {
|
||||
return scorer(weightOrBitSet.set);
|
||||
|
|
|
@ -102,6 +102,31 @@ public abstract class Weight {
|
|||
*/
|
||||
public abstract Scorer scorer(LeafReaderContext context) throws IOException;
|
||||
|
||||
/**
|
||||
* Optional method.
|
||||
* Get a {@link ScorerSupplier}, which allows to know the cost of the {@link Scorer}
|
||||
* before building it. The default implementation calls {@link #scorer} and
|
||||
* builds a {@link ScorerSupplier} wrapper around it.
|
||||
* @see #scorer
|
||||
*/
|
||||
public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
|
||||
final Scorer scorer = scorer(context);
|
||||
if (scorer == null) {
|
||||
return null;
|
||||
}
|
||||
return new ScorerSupplier() {
|
||||
@Override
|
||||
public Scorer get(boolean randomAccess) {
|
||||
return scorer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return scorer.iterator().cost();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Optional method, to return a {@link BulkScorer} to
|
||||
* score the query and send hits to a {@link Collector}.
|
||||
|
|
|
@ -223,6 +223,41 @@ public final class BKDReader extends PointValues implements Accountable {
|
|||
|
||||
/** Only valid after pushLeft or pushRight, not pop! */
|
||||
public abstract long getLeafBlockFP();
|
||||
|
||||
/** Return the number of leaves below the current node. */
|
||||
public int getNumLeaves() {
|
||||
int leftMostLeafNode = nodeID;
|
||||
while (leftMostLeafNode < leafNodeOffset) {
|
||||
leftMostLeafNode = leftMostLeafNode * 2;
|
||||
}
|
||||
int rightMostLeafNode = nodeID;
|
||||
while (rightMostLeafNode < leafNodeOffset) {
|
||||
rightMostLeafNode = rightMostLeafNode * 2 + 1;
|
||||
}
|
||||
final int numLeaves;
|
||||
if (rightMostLeafNode >= leftMostLeafNode) {
|
||||
// both are on the same level
|
||||
numLeaves = rightMostLeafNode - leftMostLeafNode + 1;
|
||||
} else {
|
||||
// left is one level deeper than right
|
||||
numLeaves = rightMostLeafNode - leftMostLeafNode + 1 + leafNodeOffset;
|
||||
}
|
||||
assert numLeaves == getNumLeavesSlow(nodeID) : numLeaves + " " + getNumLeavesSlow(nodeID);
|
||||
return numLeaves;
|
||||
}
|
||||
|
||||
// for assertions
|
||||
private int getNumLeavesSlow(int node) {
|
||||
if (node >= 2 * leafNodeOffset) {
|
||||
return 0;
|
||||
} else if (node >= leafNodeOffset) {
|
||||
return 1;
|
||||
} else {
|
||||
final int leftCount = getNumLeavesSlow(node * 2);
|
||||
final int rightCount = getNumLeavesSlow(node * 2 + 1);
|
||||
return leftCount + rightCount;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Reads the original simple yet heap-heavy index format */
|
||||
|
@ -482,10 +517,16 @@ public final class BKDReader extends PointValues implements Accountable {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void intersect(IntersectVisitor visitor) throws IOException {
|
||||
intersect(getIntersectState(visitor), minPackedValue, maxPackedValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimatePointCount(IntersectVisitor visitor) {
|
||||
return estimatePointCount(getIntersectState(visitor), minPackedValue, maxPackedValue);
|
||||
}
|
||||
|
||||
/** Fast path: this is called when the query box fully encompasses all cells under this node. */
|
||||
private void addAll(IntersectState state) throws IOException {
|
||||
//System.out.println("R: addAll nodeID=" + nodeID);
|
||||
|
@ -696,6 +737,61 @@ public final class BKDReader extends PointValues implements Accountable {
|
|||
}
|
||||
}
|
||||
|
||||
private long estimatePointCount(IntersectState state, byte[] cellMinPacked, byte[] cellMaxPacked) {
|
||||
|
||||
/*
|
||||
System.out.println("\nR: intersect nodeID=" + state.index.getNodeID());
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
System.out.println(" dim=" + dim + "\n cellMin=" + new BytesRef(cellMinPacked, dim*bytesPerDim, bytesPerDim) + "\n cellMax=" + new BytesRef(cellMaxPacked, dim*bytesPerDim, bytesPerDim));
|
||||
}
|
||||
*/
|
||||
|
||||
Relation r = state.visitor.compare(cellMinPacked, cellMaxPacked);
|
||||
|
||||
if (r == Relation.CELL_OUTSIDE_QUERY) {
|
||||
// This cell is fully outside of the query shape: stop recursing
|
||||
return 0L;
|
||||
} else if (r == Relation.CELL_INSIDE_QUERY) {
|
||||
return (long) maxPointsInLeafNode * state.index.getNumLeaves();
|
||||
} else if (state.index.isLeafNode()) {
|
||||
// Assume half the points matched
|
||||
return (maxPointsInLeafNode + 1) / 2;
|
||||
} else {
|
||||
|
||||
// Non-leaf node: recurse on the split left and right nodes
|
||||
int splitDim = state.index.getSplitDim();
|
||||
assert splitDim >= 0: "splitDim=" + splitDim;
|
||||
assert splitDim < numDims;
|
||||
|
||||
byte[] splitPackedValue = state.index.getSplitPackedValue();
|
||||
BytesRef splitDimValue = state.index.getSplitDimValue();
|
||||
assert splitDimValue.length == bytesPerDim;
|
||||
//System.out.println(" splitDimValue=" + splitDimValue + " splitDim=" + splitDim);
|
||||
|
||||
// make sure cellMin <= splitValue <= cellMax:
|
||||
assert StringHelper.compare(bytesPerDim, cellMinPacked, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset) <= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims;
|
||||
assert StringHelper.compare(bytesPerDim, cellMaxPacked, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset) >= 0: "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims;
|
||||
|
||||
// Recurse on left sub-tree:
|
||||
System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
|
||||
System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
state.index.pushLeft();
|
||||
final long leftCost = estimatePointCount(state, cellMinPacked, splitPackedValue);
|
||||
state.index.pop();
|
||||
|
||||
// Restore the split dim value since it may have been overwritten while recursing:
|
||||
System.arraycopy(splitPackedValue, splitDim*bytesPerDim, splitDimValue.bytes, splitDimValue.offset, bytesPerDim);
|
||||
|
||||
// Recurse on right sub-tree:
|
||||
System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
|
||||
System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
|
||||
state.index.pushRight();
|
||||
final long rightCost = estimatePointCount(state, splitPackedValue, cellMaxPacked);
|
||||
state.index.pop();
|
||||
return leftCost + rightCost;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
if (packedIndex != null) {
|
||||
|
|
|
@ -487,7 +487,7 @@ public class BKDWriter implements Closeable {
|
|||
assert Arrays.equals(parentSplits, new int[numDims]);
|
||||
|
||||
long indexFP = out.getFilePointer();
|
||||
writeIndex(out, leafBlockFPs, splitPackedValues);
|
||||
writeIndex(out, Math.toIntExact(countPerLeaf), leafBlockFPs, splitPackedValues);
|
||||
return indexFP;
|
||||
}
|
||||
|
||||
|
@ -645,7 +645,7 @@ public class BKDWriter implements Closeable {
|
|||
for(int i=0;i<leafBlockFPs.size();i++) {
|
||||
arr[i] = leafBlockFPs.get(i);
|
||||
}
|
||||
writeIndex(out, arr, index);
|
||||
writeIndex(out, maxPointsInLeafNode, arr, index);
|
||||
return indexFP;
|
||||
}
|
||||
|
||||
|
@ -1035,7 +1035,7 @@ public class BKDWriter implements Closeable {
|
|||
|
||||
// Write index:
|
||||
long indexFP = out.getFilePointer();
|
||||
writeIndex(out, leafBlockFPs, splitPackedValues);
|
||||
writeIndex(out, Math.toIntExact(countPerLeaf), leafBlockFPs, splitPackedValues);
|
||||
return indexFP;
|
||||
}
|
||||
|
||||
|
@ -1241,16 +1241,16 @@ public class BKDWriter implements Closeable {
|
|||
return result;
|
||||
}
|
||||
|
||||
private void writeIndex(IndexOutput out, long[] leafBlockFPs, byte[] splitPackedValues) throws IOException {
|
||||
private void writeIndex(IndexOutput out, int countPerLeaf, long[] leafBlockFPs, byte[] splitPackedValues) throws IOException {
|
||||
byte[] packedIndex = packIndex(leafBlockFPs, splitPackedValues);
|
||||
writeIndex(out, leafBlockFPs.length, packedIndex);
|
||||
writeIndex(out, countPerLeaf, leafBlockFPs.length, packedIndex);
|
||||
}
|
||||
|
||||
private void writeIndex(IndexOutput out, int numLeaves, byte[] packedIndex) throws IOException {
|
||||
private void writeIndex(IndexOutput out, int countPerLeaf, int numLeaves, byte[] packedIndex) throws IOException {
|
||||
|
||||
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
|
||||
out.writeVInt(numDims);
|
||||
out.writeVInt(maxPointsInLeafNode);
|
||||
out.writeVInt(countPerLeaf);
|
||||
out.writeVInt(bytesPerDim);
|
||||
|
||||
assert numLeaves > 0;
|
||||
|
|
|
@ -21,16 +21,22 @@ import java.io.PrintWriter;
|
|||
import java.io.StringWriter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.automaton.Automata;
|
||||
import org.apache.lucene.util.automaton.Automaton;
|
||||
import org.apache.lucene.util.automaton.AutomatonTestUtil;
|
||||
import org.apache.lucene.util.automaton.Operations;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
|
||||
import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
|
||||
|
||||
|
@ -565,7 +571,13 @@ public class TestGraphTokenizers extends BaseTokenStreamTestCase {
|
|||
assertSameLanguage(join(HOLE_A, SEP_A, s2a("abc")), ts);
|
||||
}
|
||||
|
||||
// TODO: testEndsWithHole... but we need posInc to set in TS.end()
|
||||
public void testEndsWithHole() throws Exception {
|
||||
final TokenStream ts = new CannedTokenStream(1, 0,
|
||||
new Token[] {
|
||||
token("abc", 2, 1),
|
||||
});
|
||||
assertSameLanguage(join(HOLE_A, SEP_A, s2a("abc"), SEP_A, HOLE_A), ts);
|
||||
}
|
||||
|
||||
public void testSynHangingOverEnd() throws Exception {
|
||||
final TokenStream ts = new CannedTokenStream(
|
||||
|
@ -576,14 +588,47 @@ public class TestGraphTokenizers extends BaseTokenStreamTestCase {
|
|||
assertSameLanguage(Operations.union(s2a("a"), s2a("X")), ts);
|
||||
}
|
||||
|
||||
/** Returns all paths */
|
||||
private Set<String> toPathStrings(Automaton a) {
|
||||
BytesRefBuilder scratchBytesRefBuilder = new BytesRefBuilder();
|
||||
Set<String> paths = new HashSet<>();
|
||||
for (IntsRef ir: AutomatonTestUtil.getFiniteStringsRecursive(a, -1)) {
|
||||
paths.add(Util.toBytesRef(ir, scratchBytesRefBuilder).utf8ToString().replace((char) TokenStreamToAutomaton.POS_SEP, ' '));
|
||||
}
|
||||
return paths;
|
||||
}
|
||||
|
||||
private void assertSameLanguage(Automaton expected, TokenStream ts) throws IOException {
|
||||
assertSameLanguage(expected, new TokenStreamToAutomaton().toAutomaton(ts));
|
||||
}
|
||||
|
||||
private void assertSameLanguage(Automaton expected, Automaton actual) {
|
||||
assertTrue(Operations.sameLanguage(
|
||||
Operations.determinize(Operations.removeDeadStates(expected), DEFAULT_MAX_DETERMINIZED_STATES),
|
||||
Operations.determinize(Operations.removeDeadStates(actual), DEFAULT_MAX_DETERMINIZED_STATES)));
|
||||
Automaton expectedDet = Operations.determinize(Operations.removeDeadStates(expected), DEFAULT_MAX_DETERMINIZED_STATES);
|
||||
Automaton actualDet = Operations.determinize(Operations.removeDeadStates(actual), DEFAULT_MAX_DETERMINIZED_STATES);
|
||||
if (Operations.sameLanguage(expectedDet, actualDet) == false) {
|
||||
Set<String> expectedPaths = toPathStrings(expectedDet);
|
||||
Set<String> actualPaths = toPathStrings(actualDet);
|
||||
StringBuilder b = new StringBuilder();
|
||||
b.append("expected:\n");
|
||||
for(String path : expectedPaths) {
|
||||
b.append(" ");
|
||||
b.append(path);
|
||||
if (actualPaths.contains(path) == false) {
|
||||
b.append(" [missing!]");
|
||||
}
|
||||
b.append('\n');
|
||||
}
|
||||
b.append("actual:\n");
|
||||
for(String path : actualPaths) {
|
||||
b.append(" ");
|
||||
b.append(path);
|
||||
if (expectedPaths.contains(path) == false) {
|
||||
b.append(" [unexpected!]");
|
||||
}
|
||||
b.append('\n');
|
||||
}
|
||||
fail("accepted language is different:\n\n" + b.toString());
|
||||
}
|
||||
}
|
||||
|
||||
public void testTokenStreamGraphWithHoles() throws Exception {
|
||||
|
|
|
@ -18,29 +18,43 @@ package org.apache.lucene.codecs.lucene60;
|
|||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.document.BinaryPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.BasePointsFormatTestCase;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.bkd.BKDWriter;
|
||||
|
||||
/**
|
||||
* Tests Lucene60PointsFormat
|
||||
*/
|
||||
public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
|
||||
private final Codec codec;
|
||||
private final int maxPointsInLeafNode;
|
||||
|
||||
public TestLucene60PointsFormat() {
|
||||
// standard issue
|
||||
Codec defaultCodec = TestUtil.getDefaultCodec();
|
||||
if (random().nextBoolean()) {
|
||||
// randomize parameters
|
||||
int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
|
||||
maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
|
||||
double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
|
||||
|
@ -66,6 +80,7 @@ public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
|
|||
} else {
|
||||
// standard issue
|
||||
codec = defaultCodec;
|
||||
maxPointsInLeafNode = BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -79,5 +94,178 @@ public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
|
|||
assumeFalse("TODO: mess with the parameters and test gets angry!", codec instanceof FilterCodec);
|
||||
super.testMergeStability();
|
||||
}
|
||||
|
||||
|
||||
public void testEstimatePointCount() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
|
||||
byte[] pointValue = new byte[3];
|
||||
byte[] uniquePointValue = new byte[3];
|
||||
random().nextBytes(uniquePointValue);
|
||||
final int numDocs = atLeast(10000); // make sure we have several leaves
|
||||
for (int i = 0; i < numDocs; ++i) {
|
||||
Document doc = new Document();
|
||||
if (i == numDocs / 2) {
|
||||
doc.add(new BinaryPoint("f", uniquePointValue));
|
||||
} else {
|
||||
do {
|
||||
random().nextBytes(pointValue);
|
||||
} while (Arrays.equals(pointValue, uniquePointValue));
|
||||
doc.add(new BinaryPoint("f", pointValue));
|
||||
}
|
||||
w.addDocument(doc);
|
||||
}
|
||||
w.forceMerge(1);
|
||||
final IndexReader r = DirectoryReader.open(w);
|
||||
w.close();
|
||||
final LeafReader lr = getOnlyLeafReader(r);
|
||||
PointValues points = lr.getPointValues("f");
|
||||
|
||||
// If all points match, then the point count is numLeaves * maxPointsInLeafNode
|
||||
final int numLeaves = (int) Math.ceil((double) numDocs / maxPointsInLeafNode);
|
||||
assertEquals(numLeaves * maxPointsInLeafNode,
|
||||
points.estimatePointCount(new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
}));
|
||||
|
||||
// Return 0 if no points match
|
||||
assertEquals(0,
|
||||
points.estimatePointCount(new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
}));
|
||||
|
||||
// If only one point matches, then the point count is (maxPointsInLeafNode + 1) / 2
|
||||
assertEquals((maxPointsInLeafNode + 1) / 2,
|
||||
points.estimatePointCount(new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
if (StringHelper.compare(3, uniquePointValue, 0, maxPackedValue, 0) > 0 ||
|
||||
StringHelper.compare(3, uniquePointValue, 0, minPackedValue, 0) < 0) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
}));
|
||||
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// The tree is always balanced in the N dims case, and leaves are
|
||||
// not all full so things are a bit different
|
||||
public void testEstimatePointCount2Dims() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
|
||||
byte[][] pointValue = new byte[2][];
|
||||
pointValue[0] = new byte[3];
|
||||
pointValue[1] = new byte[3];
|
||||
byte[][] uniquePointValue = new byte[2][];
|
||||
uniquePointValue[0] = new byte[3];
|
||||
uniquePointValue[1] = new byte[3];
|
||||
random().nextBytes(uniquePointValue[0]);
|
||||
random().nextBytes(uniquePointValue[1]);
|
||||
final int numDocs = atLeast(10000); // make sure we have several leaves
|
||||
for (int i = 0; i < numDocs; ++i) {
|
||||
Document doc = new Document();
|
||||
if (i == numDocs / 2) {
|
||||
doc.add(new BinaryPoint("f", uniquePointValue));
|
||||
} else {
|
||||
do {
|
||||
random().nextBytes(pointValue[0]);
|
||||
random().nextBytes(pointValue[1]);
|
||||
} while (Arrays.equals(pointValue[0], uniquePointValue[0]) || Arrays.equals(pointValue[1], uniquePointValue[1]));
|
||||
doc.add(new BinaryPoint("f", pointValue));
|
||||
}
|
||||
w.addDocument(doc);
|
||||
}
|
||||
w.forceMerge(1);
|
||||
final IndexReader r = DirectoryReader.open(w);
|
||||
w.close();
|
||||
final LeafReader lr = getOnlyLeafReader(r);
|
||||
PointValues points = lr.getPointValues("f");
|
||||
|
||||
// With >1 dims, the tree is balanced
|
||||
int actualMaxPointsInLeafNode = numDocs;
|
||||
while (actualMaxPointsInLeafNode > maxPointsInLeafNode) {
|
||||
actualMaxPointsInLeafNode = (actualMaxPointsInLeafNode + 1) / 2;
|
||||
}
|
||||
|
||||
// If all points match, then the point count is numLeaves * maxPointsInLeafNode
|
||||
final int numLeaves = Integer.highestOneBit((numDocs - 1) / actualMaxPointsInLeafNode) << 1;
|
||||
assertEquals(numLeaves * actualMaxPointsInLeafNode,
|
||||
points.estimatePointCount(new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
}));
|
||||
|
||||
// Return 0 if no points match
|
||||
assertEquals(0,
|
||||
points.estimatePointCount(new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
}));
|
||||
|
||||
// If only one point matches, then the point count is (actualMaxPointsInLeafNode + 1) / 2
|
||||
assertEquals((actualMaxPointsInLeafNode + 1) / 2,
|
||||
points.estimatePointCount(new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
for (int dim = 0; dim < 2; ++dim) {
|
||||
if (StringHelper.compare(3, uniquePointValue[0], 0, maxPackedValue, dim * 3) > 0 ||
|
||||
StringHelper.compare(3, uniquePointValue[0], 0, minPackedValue, dim * 3) < 0) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
}
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
}));
|
||||
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,15 +18,12 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomStrings;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
|
@ -53,25 +50,25 @@ public class TermInSetQueryTest extends LuceneTestCase {
|
|||
|
||||
public void testDuel() throws IOException {
|
||||
final int iters = atLeast(2);
|
||||
final String field = "f";
|
||||
for (int iter = 0; iter < iters; ++iter) {
|
||||
final List<Term> allTerms = new ArrayList<>();
|
||||
final List<BytesRef> allTerms = new ArrayList<>();
|
||||
final int numTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 10));
|
||||
for (int i = 0; i < numTerms; ++i) {
|
||||
final String field = usually() ? "f" : "g";
|
||||
final String value = TestUtil.randomAnalysisString(random(), 10, true);
|
||||
allTerms.add(new Term(field, value));
|
||||
allTerms.add(new BytesRef(value));
|
||||
}
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
|
||||
final int numDocs = atLeast(100);
|
||||
for (int i = 0; i < numDocs; ++i) {
|
||||
Document doc = new Document();
|
||||
final Term term = allTerms.get(random().nextInt(allTerms.size()));
|
||||
doc.add(new StringField(term.field(), term.text(), Store.NO));
|
||||
final BytesRef term = allTerms.get(random().nextInt(allTerms.size()));
|
||||
doc.add(new StringField(field, term, Store.NO));
|
||||
iw.addDocument(doc);
|
||||
}
|
||||
if (numTerms > 1 && random().nextBoolean()) {
|
||||
iw.deleteDocuments(new TermQuery(allTerms.get(0)));
|
||||
iw.deleteDocuments(new TermQuery(new Term(field, allTerms.get(0))));
|
||||
}
|
||||
iw.commit();
|
||||
final IndexReader reader = iw.getReader();
|
||||
|
@ -87,16 +84,16 @@ public class TermInSetQueryTest extends LuceneTestCase {
|
|||
for (int i = 0; i < 100; ++i) {
|
||||
final float boost = random().nextFloat() * 10;
|
||||
final int numQueryTerms = TestUtil.nextInt(random(), 1, 1 << TestUtil.nextInt(random(), 1, 8));
|
||||
List<Term> queryTerms = new ArrayList<>();
|
||||
List<BytesRef> queryTerms = new ArrayList<>();
|
||||
for (int j = 0; j < numQueryTerms; ++j) {
|
||||
queryTerms.add(allTerms.get(random().nextInt(allTerms.size())));
|
||||
}
|
||||
final BooleanQuery.Builder bq = new BooleanQuery.Builder();
|
||||
for (Term t : queryTerms) {
|
||||
bq.add(new TermQuery(t), Occur.SHOULD);
|
||||
for (BytesRef t : queryTerms) {
|
||||
bq.add(new TermQuery(new Term(field, t)), Occur.SHOULD);
|
||||
}
|
||||
final Query q1 = new ConstantScoreQuery(bq.build());
|
||||
final Query q2 = new TermInSetQuery(queryTerms);
|
||||
final Query q2 = new TermInSetQuery(field, queryTerms);
|
||||
assertSameMatches(searcher, new BoostQuery(q1, boost), new BoostQuery(q2, boost), true);
|
||||
}
|
||||
|
||||
|
@ -118,103 +115,72 @@ public class TermInSetQueryTest extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private TermInSetQuery termsQuery(boolean singleField, Term...terms) {
|
||||
return termsQuery(singleField, Arrays.asList(terms));
|
||||
}
|
||||
|
||||
private TermInSetQuery termsQuery(boolean singleField, Collection<Term> termList) {
|
||||
if (!singleField) {
|
||||
return new TermInSetQuery(new ArrayList<>(termList));
|
||||
}
|
||||
final TermInSetQuery filter;
|
||||
List<BytesRef> bytes = new ArrayList<>();
|
||||
String field = null;
|
||||
for (Term term : termList) {
|
||||
bytes.add(term.bytes());
|
||||
if (field != null) {
|
||||
assertEquals(term.field(), field);
|
||||
}
|
||||
field = term.field();
|
||||
}
|
||||
assertNotNull(field);
|
||||
filter = new TermInSetQuery(field, bytes);
|
||||
return filter;
|
||||
}
|
||||
|
||||
public void testHashCodeAndEquals() {
|
||||
int num = atLeast(100);
|
||||
final boolean singleField = random().nextBoolean();
|
||||
List<Term> terms = new ArrayList<>();
|
||||
Set<Term> uniqueTerms = new HashSet<>();
|
||||
List<BytesRef> terms = new ArrayList<>();
|
||||
Set<BytesRef> uniqueTerms = new HashSet<>();
|
||||
for (int i = 0; i < num; i++) {
|
||||
String field = "field" + (singleField ? "1" : random().nextInt(100));
|
||||
String string = TestUtil.randomRealisticUnicodeString(random());
|
||||
terms.add(new Term(field, string));
|
||||
uniqueTerms.add(new Term(field, string));
|
||||
TermInSetQuery left = termsQuery(singleField ? random().nextBoolean() : false, uniqueTerms);
|
||||
terms.add(new BytesRef(string));
|
||||
uniqueTerms.add(new BytesRef(string));
|
||||
TermInSetQuery left = new TermInSetQuery("field", uniqueTerms);
|
||||
Collections.shuffle(terms, random());
|
||||
TermInSetQuery right = termsQuery(singleField ? random().nextBoolean() : false, terms);
|
||||
TermInSetQuery right = new TermInSetQuery("field", terms);
|
||||
assertEquals(right, left);
|
||||
assertEquals(right.hashCode(), left.hashCode());
|
||||
if (uniqueTerms.size() > 1) {
|
||||
List<Term> asList = new ArrayList<>(uniqueTerms);
|
||||
List<BytesRef> asList = new ArrayList<>(uniqueTerms);
|
||||
asList.remove(0);
|
||||
TermInSetQuery notEqual = termsQuery(singleField ? random().nextBoolean() : false, asList);
|
||||
TermInSetQuery notEqual = new TermInSetQuery("field", asList);
|
||||
assertFalse(left.equals(notEqual));
|
||||
assertFalse(right.equals(notEqual));
|
||||
}
|
||||
}
|
||||
|
||||
TermInSetQuery tq1 = new TermInSetQuery(new Term("thing", "apple"));
|
||||
TermInSetQuery tq2 = new TermInSetQuery(new Term("thing", "orange"));
|
||||
TermInSetQuery tq1 = new TermInSetQuery("thing", new BytesRef("apple"));
|
||||
TermInSetQuery tq2 = new TermInSetQuery("thing", new BytesRef("orange"));
|
||||
assertFalse(tq1.hashCode() == tq2.hashCode());
|
||||
|
||||
// different fields with the same term should have differing hashcodes
|
||||
tq1 = new TermInSetQuery(new Term("thing1", "apple"));
|
||||
tq2 = new TermInSetQuery(new Term("thing2", "apple"));
|
||||
tq1 = new TermInSetQuery("thing", new BytesRef("apple"));
|
||||
tq2 = new TermInSetQuery("thing2", new BytesRef("apple"));
|
||||
assertFalse(tq1.hashCode() == tq2.hashCode());
|
||||
}
|
||||
|
||||
public void testSingleFieldEquals() {
|
||||
public void testSimpleEquals() {
|
||||
// Two terms with the same hash code
|
||||
assertEquals("AaAaBB".hashCode(), "BBBBBB".hashCode());
|
||||
TermInSetQuery left = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "AaAaBB"));
|
||||
TermInSetQuery right = termsQuery(true, new Term("id", "AaAaAa"), new Term("id", "BBBBBB"));
|
||||
TermInSetQuery left = new TermInSetQuery("id", new BytesRef("AaAaAa"), new BytesRef("AaAaBB"));
|
||||
TermInSetQuery right = new TermInSetQuery("id", new BytesRef("AaAaAa"), new BytesRef("BBBBBB"));
|
||||
assertFalse(left.equals(right));
|
||||
}
|
||||
|
||||
public void testToString() {
|
||||
TermInSetQuery termsQuery = new TermInSetQuery(new Term("field1", "a"),
|
||||
new Term("field1", "b"),
|
||||
new Term("field1", "c"));
|
||||
TermInSetQuery termsQuery = new TermInSetQuery("field1",
|
||||
new BytesRef("a"), new BytesRef("b"), new BytesRef("c"));
|
||||
assertEquals("field1:a field1:b field1:c", termsQuery.toString());
|
||||
}
|
||||
|
||||
public void testDedup() {
|
||||
Query query1 = new TermInSetQuery(new Term("foo", "bar"));
|
||||
Query query2 = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "bar"));
|
||||
Query query1 = new TermInSetQuery("foo", new BytesRef("bar"));
|
||||
Query query2 = new TermInSetQuery("foo", new BytesRef("bar"), new BytesRef("bar"));
|
||||
QueryUtils.checkEqual(query1, query2);
|
||||
}
|
||||
|
||||
public void testOrderDoesNotMatter() {
|
||||
// order of terms if different
|
||||
Query query1 = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "baz"));
|
||||
Query query2 = new TermInSetQuery(new Term("foo", "baz"), new Term("foo", "bar"));
|
||||
QueryUtils.checkEqual(query1, query2);
|
||||
|
||||
// order of fields is different
|
||||
query1 = new TermInSetQuery(new Term("foo", "bar"), new Term("bar", "bar"));
|
||||
query2 = new TermInSetQuery(new Term("bar", "bar"), new Term("foo", "bar"));
|
||||
Query query1 = new TermInSetQuery("foo", new BytesRef("bar"), new BytesRef("baz"));
|
||||
Query query2 = new TermInSetQuery("foo", new BytesRef("baz"), new BytesRef("bar"));
|
||||
QueryUtils.checkEqual(query1, query2);
|
||||
}
|
||||
|
||||
public void testRamBytesUsed() {
|
||||
List<Term> terms = new ArrayList<>();
|
||||
List<BytesRef> terms = new ArrayList<>();
|
||||
final int numTerms = 1000 + random().nextInt(1000);
|
||||
for (int i = 0; i < numTerms; ++i) {
|
||||
terms.add(new Term("f", RandomStrings.randomUnicodeOfLength(random(), 10)));
|
||||
terms.add(new BytesRef(RandomStrings.randomUnicodeOfLength(random(), 10)));
|
||||
}
|
||||
TermInSetQuery query = new TermInSetQuery(terms);
|
||||
TermInSetQuery query = new TermInSetQuery("f", terms);
|
||||
final long actualRamBytesUsed = RamUsageTester.sizeOf(query);
|
||||
final long expectedRamBytesUsed = query.ramBytesUsed();
|
||||
// error margin within 5%
|
||||
|
@ -281,43 +247,40 @@ public class TermInSetQueryTest extends LuceneTestCase {
|
|||
|
||||
}
|
||||
|
||||
public void testPullOneTermsEnumPerField() throws Exception {
|
||||
public void testPullOneTermsEnum() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("foo", "1", Store.NO));
|
||||
doc.add(new StringField("bar", "2", Store.NO));
|
||||
doc.add(new StringField("baz", "3", Store.NO));
|
||||
w.addDocument(doc);
|
||||
DirectoryReader reader = w.getReader();
|
||||
w.close();
|
||||
final AtomicInteger counter = new AtomicInteger();
|
||||
DirectoryReader wrapped = new TermsCountingDirectoryReaderWrapper(reader, counter);
|
||||
|
||||
final List<Term> terms = new ArrayList<>();
|
||||
final Set<String> fields = new HashSet<>();
|
||||
final List<BytesRef> terms = new ArrayList<>();
|
||||
// enough terms to avoid the rewrite
|
||||
final int numTerms = TestUtil.nextInt(random(), TermInSetQuery.BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD + 1, 100);
|
||||
for (int i = 0; i < numTerms; ++i) {
|
||||
final String field = RandomPicks.randomFrom(random(), new String[] {"foo", "bar", "baz"});
|
||||
final BytesRef term = new BytesRef(RandomStrings.randomUnicodeOfCodepointLength(random(), 10));
|
||||
fields.add(field);
|
||||
terms.add(new Term(field, term));
|
||||
terms.add(term);
|
||||
}
|
||||
|
||||
new IndexSearcher(wrapped).count(new TermInSetQuery(terms));
|
||||
assertEquals(fields.size(), counter.get());
|
||||
assertEquals(0, new IndexSearcher(wrapped).count(new TermInSetQuery("bar", terms)));
|
||||
assertEquals(0, counter.get()); // missing field
|
||||
new IndexSearcher(wrapped).count(new TermInSetQuery("foo", terms));
|
||||
assertEquals(1, counter.get());
|
||||
wrapped.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testBinaryToString() {
|
||||
TermInSetQuery query = new TermInSetQuery(new Term("field", new BytesRef(new byte[] { (byte) 0xff, (byte) 0xfe })));
|
||||
TermInSetQuery query = new TermInSetQuery("field", new BytesRef(new byte[] { (byte) 0xff, (byte) 0xfe }));
|
||||
assertEquals("field:[ff fe]", query.toString());
|
||||
}
|
||||
|
||||
public void testIsConsideredCostlyByQueryCache() throws IOException {
|
||||
TermInSetQuery query = new TermInSetQuery(new Term("foo", "bar"), new Term("foo", "baz"));
|
||||
TermInSetQuery query = new TermInSetQuery("foo", new BytesRef("bar"), new BytesRef("baz"));
|
||||
UsageTrackingQueryCachingPolicy policy = new UsageTrackingQueryCachingPolicy();
|
||||
assertFalse(policy.shouldCache(query));
|
||||
policy.onUse(query);
|
||||
|
|
|
@ -0,0 +1,332 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You 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,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.search;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.EnumMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
|
||||
public class TestBoolean2ScorerSupplier extends LuceneTestCase {
|
||||
|
||||
private static class FakeScorer extends Scorer {
|
||||
|
||||
private final DocIdSetIterator it;
|
||||
|
||||
FakeScorer(long cost) {
|
||||
super(null);
|
||||
this.it = DocIdSetIterator.all(Math.toIntExact(cost));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return it.docID();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
return it;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "FakeScorer(cost=" + it.cost() + ")";
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class FakeScorerSupplier extends ScorerSupplier {
|
||||
|
||||
private final long cost;
|
||||
private final Boolean randomAccess;
|
||||
|
||||
FakeScorerSupplier(long cost) {
|
||||
this.cost = cost;
|
||||
this.randomAccess = null;
|
||||
}
|
||||
|
||||
FakeScorerSupplier(long cost, boolean randomAccess) {
|
||||
this.cost = cost;
|
||||
this.randomAccess = randomAccess;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer get(boolean randomAccess) throws IOException {
|
||||
if (this.randomAccess != null) {
|
||||
assertEquals(this.toString(), this.randomAccess, randomAccess);
|
||||
}
|
||||
return new FakeScorer(cost);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return cost;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "FakeLazyScorer(cost=" + cost + ",randomAccess=" + randomAccess + ")";
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void testConjunctionCost() {
|
||||
Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
|
||||
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(42));
|
||||
assertEquals(42, new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 0).cost());
|
||||
|
||||
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(12));
|
||||
assertEquals(12, new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 0).cost());
|
||||
|
||||
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(20));
|
||||
assertEquals(12, new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 0).cost());
|
||||
}
|
||||
|
||||
public void testDisjunctionCost() throws IOException {
|
||||
Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42));
|
||||
ScorerSupplier s = new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 0);
|
||||
assertEquals(42, s.cost());
|
||||
assertEquals(42, s.get(random().nextBoolean()).iterator().cost());
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12));
|
||||
s = new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 0);
|
||||
assertEquals(42 + 12, s.cost());
|
||||
assertEquals(42 + 12, s.get(random().nextBoolean()).iterator().cost());
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20));
|
||||
s = new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 0);
|
||||
assertEquals(42 + 12 + 20, s.cost());
|
||||
assertEquals(42 + 12 + 20, s.get(random().nextBoolean()).iterator().cost());
|
||||
}
|
||||
|
||||
public void testDisjunctionWithMinShouldMatchCost() throws IOException {
|
||||
Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12));
|
||||
ScorerSupplier s = new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 1);
|
||||
assertEquals(42 + 12, s.cost());
|
||||
assertEquals(42 + 12, s.get(random().nextBoolean()).iterator().cost());
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20));
|
||||
s = new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 1);
|
||||
assertEquals(42 + 12 + 20, s.cost());
|
||||
assertEquals(42 + 12 + 20, s.get(random().nextBoolean()).iterator().cost());
|
||||
s = new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 2);
|
||||
assertEquals(12 + 20, s.cost());
|
||||
assertEquals(12 + 20, s.get(random().nextBoolean()).iterator().cost());
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30));
|
||||
s = new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 1);
|
||||
assertEquals(42 + 12 + 20 + 30, s.cost());
|
||||
assertEquals(42 + 12 + 20 + 30, s.get(random().nextBoolean()).iterator().cost());
|
||||
s = new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 2);
|
||||
assertEquals(12 + 20 + 30, s.cost());
|
||||
assertEquals(12 + 20 + 30, s.get(random().nextBoolean()).iterator().cost());
|
||||
s = new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 3);
|
||||
assertEquals(12 + 20, s.cost());
|
||||
assertEquals(12 + 20, s.get(random().nextBoolean()).iterator().cost());
|
||||
}
|
||||
|
||||
public void testDuelCost() throws Exception {
|
||||
final int iters = atLeast(1000);
|
||||
for (int iter = 0; iter < iters; ++iter) {
|
||||
Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
int numClauses = TestUtil.nextInt(random(), 1, 10);
|
||||
int numShoulds = 0;
|
||||
int numRequired = 0;
|
||||
for (int j = 0; j < numClauses; ++j) {
|
||||
Occur occur = RandomPicks.randomFrom(random(), Occur.values());
|
||||
subs.get(occur).add(new FakeScorerSupplier(random().nextInt(100)));
|
||||
if (occur == Occur.SHOULD) {
|
||||
++numShoulds;
|
||||
} else if (occur == Occur.FILTER || occur == Occur.MUST) {
|
||||
numRequired++;
|
||||
}
|
||||
}
|
||||
boolean needsScores = random().nextBoolean();
|
||||
if (needsScores == false && numRequired > 0) {
|
||||
numClauses -= numShoulds;
|
||||
numShoulds = 0;
|
||||
subs.get(Occur.SHOULD).clear();
|
||||
}
|
||||
if (numShoulds + numRequired == 0) {
|
||||
// only negative clauses, invalid
|
||||
continue;
|
||||
}
|
||||
int minShouldMatch = numShoulds == 0 ? 0 : TestUtil.nextInt(random(), 0, numShoulds - 1);
|
||||
Boolean2ScorerSupplier supplier = new Boolean2ScorerSupplier(null,
|
||||
subs, needsScores, minShouldMatch);
|
||||
long cost1 = supplier.cost();
|
||||
long cost2 = supplier.get(false).iterator().cost();
|
||||
assertEquals("clauses=" + subs + ", minShouldMatch=" + minShouldMatch, cost1, cost2);
|
||||
}
|
||||
}
|
||||
|
||||
// test the tester...
|
||||
public void testFakeScorerSupplier() {
|
||||
FakeScorerSupplier randomAccessSupplier = new FakeScorerSupplier(random().nextInt(100), true);
|
||||
expectThrows(AssertionError.class, () -> randomAccessSupplier.get(false));
|
||||
FakeScorerSupplier sequentialSupplier = new FakeScorerSupplier(random().nextInt(100), false);
|
||||
expectThrows(AssertionError.class, () -> sequentialSupplier.get(true));
|
||||
}
|
||||
|
||||
public void testConjunctionRandomAccess() throws IOException {
|
||||
Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
|
||||
// If sequential access is required, only the least costly clause does not use random-access
|
||||
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(42, true));
|
||||
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(12, false));
|
||||
new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 0).get(false); // triggers assertions as a side-effect
|
||||
|
||||
subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
|
||||
// If random access is required, then we propagate to sub clauses
|
||||
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(42, true));
|
||||
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))).add(new FakeScorerSupplier(12, true));
|
||||
new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 0).get(true); // triggers assertions as a side-effect
|
||||
}
|
||||
|
||||
public void testDisjunctionRandomAccess() throws IOException {
|
||||
// disjunctions propagate
|
||||
for (boolean randomAccess : new boolean[] {false, true}) {
|
||||
Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, randomAccess));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, randomAccess));
|
||||
new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 0).get(randomAccess); // triggers assertions as a side-effect
|
||||
}
|
||||
}
|
||||
|
||||
public void testDisjunctionWithMinShouldMatchRandomAccess() throws IOException {
|
||||
Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
|
||||
// Only the most costly clause uses random-access in that case:
|
||||
// most of time, we will find agreement between the 2 least costly
|
||||
// clauses and only then check whether the 3rd one matches too
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, true));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, false));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, false));
|
||||
new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 2).get(false); // triggers assertions as a side-effect
|
||||
|
||||
subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
|
||||
// When random-access is true, just propagate
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, true));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, true));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, true));
|
||||
new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 2).get(true); // triggers assertions as a side-effect
|
||||
|
||||
subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, true));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, false));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, false));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20, false));
|
||||
new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 2).get(false); // triggers assertions as a side-effect
|
||||
|
||||
subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, true));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, false));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, true));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20, false));
|
||||
new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 3).get(false); // triggers assertions as a side-effect
|
||||
}
|
||||
|
||||
public void testProhibitedRandomAccess() throws IOException {
|
||||
for (boolean randomAccess : new boolean[] {false, true}) {
|
||||
Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
|
||||
// The MUST_NOT clause always uses random-access
|
||||
subs.get(Occur.MUST).add(new FakeScorerSupplier(42, randomAccess));
|
||||
subs.get(Occur.MUST_NOT).add(new FakeScorerSupplier(TestUtil.nextInt(random(), 1, 100), true));
|
||||
new Boolean2ScorerSupplier(null, subs, random().nextBoolean(), 0).get(randomAccess); // triggers assertions as a side-effect
|
||||
}
|
||||
}
|
||||
|
||||
public void testMixedRandomAccess() throws IOException {
|
||||
for (boolean randomAccess : new boolean[] {false, true}) {
|
||||
Map<Occur, Collection<ScorerSupplier>> subs = new EnumMap<>(Occur.class);
|
||||
for (Occur occur : Occur.values()) {
|
||||
subs.put(occur, new ArrayList<>());
|
||||
}
|
||||
|
||||
// The SHOULD clause always uses random-access if there is a MUST clause
|
||||
subs.get(Occur.MUST).add(new FakeScorerSupplier(42, randomAccess));
|
||||
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(TestUtil.nextInt(random(), 1, 100), true));
|
||||
new Boolean2ScorerSupplier(null, subs, true, 0).get(randomAccess); // triggers assertions as a side-effect
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -206,8 +206,8 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
|
|||
" MUST ConstantScoreScorer\n" +
|
||||
" MUST MinShouldMatchSumScorer\n" +
|
||||
" SHOULD TermScorer body:nutch\n" +
|
||||
" SHOULD TermScorer body:web\n" +
|
||||
" SHOULD TermScorer body:crawler",
|
||||
" SHOULD TermScorer body:crawler\n" +
|
||||
" SHOULD TermScorer body:web",
|
||||
summary);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.lang.reflect.Method;
|
||||
import java.lang.reflect.Modifier;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.junit.Test;
|
||||
|
@ -35,7 +36,7 @@ public class TestFilterWeight extends LuceneTestCase {
|
|||
final int modifiers = superClassMethod.getModifiers();
|
||||
if (Modifier.isFinal(modifiers)) continue;
|
||||
if (Modifier.isStatic(modifiers)) continue;
|
||||
if (superClassMethod.getName().equals("bulkScorer")) {
|
||||
if (Arrays.asList("bulkScorer", "scorerSupplier").contains(superClassMethod.getName())) {
|
||||
try {
|
||||
final Method subClassMethod = subClass.getDeclaredMethod(
|
||||
superClassMethod.getName(),
|
||||
|
|
|
@ -69,6 +69,7 @@ import org.apache.lucene.util.LuceneTestCase;
|
|||
import org.apache.lucene.util.NumericUtils;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.bkd.BKDWriter;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
public class TestPointQueries extends LuceneTestCase {
|
||||
|
@ -2080,4 +2081,38 @@ public class TestPointQueries extends LuceneTestCase {
|
|||
assertTrue(Float.compare(Float.NEGATIVE_INFINITY, FloatPoint.nextDown(Float.NEGATIVE_INFINITY)) == 0);
|
||||
assertTrue(Float.compare(Float.MAX_VALUE, FloatPoint.nextDown(Float.POSITIVE_INFINITY)) == 0);
|
||||
}
|
||||
|
||||
public void testInversePointRange() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
|
||||
final int numDims = TestUtil.nextInt(random(), 1, 3);
|
||||
final int numDocs = atLeast(10 * BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE); // we need multiple leaves to enable this optimization
|
||||
for (int i = 0; i < numDocs; ++i) {
|
||||
Document doc = new Document();
|
||||
int[] values = new int[numDims];
|
||||
Arrays.fill(values, i);
|
||||
doc.add(new IntPoint("f", values));
|
||||
w.addDocument(doc);
|
||||
}
|
||||
w.forceMerge(1);
|
||||
IndexReader r = DirectoryReader.open(w);
|
||||
w.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(r);
|
||||
int[] low = new int[numDims];
|
||||
int[] high = new int[numDims];
|
||||
Arrays.fill(high, numDocs - 2);
|
||||
assertEquals(high[0] - low[0] + 1, searcher.count(IntPoint.newRangeQuery("f", low, high)));
|
||||
Arrays.fill(low, 1);
|
||||
assertEquals(high[0] - low[0] + 1, searcher.count(IntPoint.newRangeQuery("f", low, high)));
|
||||
Arrays.fill(high, numDocs - 1);
|
||||
assertEquals(high[0] - low[0] + 1, searcher.count(IntPoint.newRangeQuery("f", low, high)));
|
||||
Arrays.fill(low, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE + 1);
|
||||
assertEquals(high[0] - low[0] + 1, searcher.count(IntPoint.newRangeQuery("f", low, high)));
|
||||
Arrays.fill(high, numDocs - BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
|
||||
assertEquals(high[0] - low[0] + 1, searcher.count(IntPoint.newRangeQuery("f", low, high)));
|
||||
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -311,6 +311,11 @@ public class TestDocIdSetBuilder extends LuceneTestCase {
|
|||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimatePointCount(IntersectVisitor visitor) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
|
|
|
@ -1104,4 +1104,94 @@ public class TestBKD extends LuceneTestCase {
|
|||
in.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testEstimatePointCount() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
final int numValues = atLeast(10000); // make sure to have multiple leaves
|
||||
final int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
|
||||
final int numBytesPerDim = TestUtil.nextInt(random(), 1, 4);
|
||||
final byte[] pointValue = new byte[numBytesPerDim];
|
||||
final byte[] uniquePointValue = new byte[numBytesPerDim];
|
||||
random().nextBytes(uniquePointValue);
|
||||
|
||||
BKDWriter w = new BKDWriter(numValues, dir, "_temp", 1, numBytesPerDim, maxPointsInLeafNode,
|
||||
BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, numValues, true);
|
||||
for (int i = 0; i < numValues; ++i) {
|
||||
if (i == numValues / 2) {
|
||||
w.add(uniquePointValue, i);
|
||||
} else {
|
||||
do {
|
||||
random().nextBytes(pointValue);
|
||||
} while (Arrays.equals(pointValue, uniquePointValue));
|
||||
w.add(pointValue, i);
|
||||
}
|
||||
}
|
||||
final long indexFP;
|
||||
try (IndexOutput out = dir.createOutput("bkd", IOContext.DEFAULT)) {
|
||||
indexFP = w.finish(out);
|
||||
w.close();
|
||||
}
|
||||
|
||||
IndexInput pointsIn = dir.openInput("bkd", IOContext.DEFAULT);
|
||||
pointsIn.seek(indexFP);
|
||||
BKDReader points = new BKDReader(pointsIn);
|
||||
|
||||
int actualMaxPointsInLeafNode = numValues;
|
||||
while (actualMaxPointsInLeafNode > maxPointsInLeafNode) {
|
||||
actualMaxPointsInLeafNode = (actualMaxPointsInLeafNode + 1) / 2;
|
||||
}
|
||||
|
||||
// If all points match, then the point count is numLeaves * maxPointsInLeafNode
|
||||
final int numLeaves = Integer.highestOneBit((numValues - 1) / actualMaxPointsInLeafNode) << 1;
|
||||
assertEquals(numLeaves * actualMaxPointsInLeafNode,
|
||||
points.estimatePointCount(new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
}));
|
||||
|
||||
// Return 0 if no points match
|
||||
assertEquals(0,
|
||||
points.estimatePointCount(new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
}));
|
||||
|
||||
// If only one point matches, then the point count is (actualMaxPointsInLeafNode + 1) / 2
|
||||
assertEquals((actualMaxPointsInLeafNode + 1) / 2,
|
||||
points.estimatePointCount(new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
if (StringHelper.compare(3, uniquePointValue, 0, maxPackedValue, 0) > 0 ||
|
||||
StringHelper.compare(3, uniquePointValue, 0, minPackedValue, 0) < 0) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
}));
|
||||
|
||||
pointsIn.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -220,6 +220,11 @@ public class TestMutablePointsReaderUtils extends LuceneTestCase {
|
|||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimatePointCount(IntersectVisitor visitor) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
|
|
|
@ -19,9 +19,9 @@ package org.apache.lucene.facet;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TermInSetQuery;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* A multi-terms {@link Query} over a {@link FacetField}.
|
||||
|
@ -38,7 +38,7 @@ public class MultiFacetQuery extends TermInSetQuery {
|
|||
* Creates a new {@code MultiFacetQuery} filtering the query on the given dimension.
|
||||
*/
|
||||
public MultiFacetQuery(final FacetsConfig facetsConfig, final String dimension, final String[]... paths) {
|
||||
super(toTerms(facetsConfig.getDimConfig(dimension), dimension, paths));
|
||||
super(facetsConfig.getDimConfig(dimension).indexFieldName, toTerms(dimension, paths));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -47,14 +47,13 @@ public class MultiFacetQuery extends TermInSetQuery {
|
|||
* <b>NOTE:</b>Uses FacetsConfig.DEFAULT_DIM_CONFIG.
|
||||
*/
|
||||
public MultiFacetQuery(final String dimension, final String[]... paths) {
|
||||
super(toTerms(FacetsConfig.DEFAULT_DIM_CONFIG, dimension, paths));
|
||||
super(FacetsConfig.DEFAULT_DIM_CONFIG.indexFieldName, toTerms(dimension, paths));
|
||||
}
|
||||
|
||||
static Collection<Term> toTerms(final FacetsConfig.DimConfig dimConfig, final String dimension,
|
||||
final String[]... paths) {
|
||||
final Collection<Term> terms = new ArrayList<>(paths.length);
|
||||
static Collection<BytesRef> toTerms(final String dimension, final String[]... paths) {
|
||||
final Collection<BytesRef> terms = new ArrayList<>(paths.length);
|
||||
for (String[] path : paths)
|
||||
terms.add(FacetQuery.toTerm(dimConfig, dimension, path));
|
||||
terms.add(new BytesRef(FacetsConfig.pathToString(dimension, path)));
|
||||
return terms;
|
||||
}
|
||||
|
||||
|
|
|
@ -1521,6 +1521,11 @@ public class MemoryIndex {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimatePointCount(IntersectVisitor visitor) {
|
||||
return 1L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue() throws IOException {
|
||||
BytesRef[] values = info.pointValues;
|
||||
|
|
|
@ -23,8 +23,10 @@ import org.apache.lucene.index.DocValues;
|
|||
import org.apache.lucene.index.DocValuesType;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.index.SortedNumericDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
|
@ -33,10 +35,11 @@ import org.apache.lucene.util.BytesRef;
|
|||
* dense case where most documents match this query, it <b>might</b> be as
|
||||
* fast or faster than a regular {@link PointRangeQuery}.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: be very careful using this query: it is
|
||||
* typically much slower than using {@code TermsQuery},
|
||||
* but in certain specialized cases may be faster.
|
||||
* <b>NOTE:</b> This query is typically best used within a
|
||||
* {@link IndexOrDocValuesQuery} alongside a query that uses an indexed
|
||||
* structure such as {@link PointValues points} or {@link Terms terms},
|
||||
* which allows to run the query on doc values when that would be more
|
||||
* efficient, and using an index otherwise.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
|
|
@ -0,0 +1,116 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You 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,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.search;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
||||
/**
|
||||
* A query that uses either an index (points or terms) or doc values in order
|
||||
* to run a range query, depending which one is more efficient.
|
||||
*/
|
||||
public final class IndexOrDocValuesQuery extends Query {
|
||||
|
||||
private final Query indexQuery, dvQuery;
|
||||
|
||||
/**
|
||||
* Constructor that takes both a query that executes on an index structure
|
||||
* like the inverted index or the points tree, and another query that
|
||||
* executes on doc values. Both queries must match the same documents and
|
||||
* attribute constant scores.
|
||||
*/
|
||||
public IndexOrDocValuesQuery(Query indexQuery, Query dvQuery) {
|
||||
this.indexQuery = indexQuery;
|
||||
this.dvQuery = dvQuery;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return indexQuery.toString(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (sameClassAs(obj) == false) {
|
||||
return false;
|
||||
}
|
||||
IndexOrDocValuesQuery that = (IndexOrDocValuesQuery) obj;
|
||||
return indexQuery.equals(that.indexQuery) && dvQuery.equals(that.dvQuery);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int h = classHash();
|
||||
h = 31 * h + indexQuery.hashCode();
|
||||
h = 31 * h + dvQuery.hashCode();
|
||||
return h;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query rewrite(IndexReader reader) throws IOException {
|
||||
Query indexRewrite = indexQuery.rewrite(reader);
|
||||
Query dvRewrite = dvQuery.rewrite(reader);
|
||||
if (indexQuery != indexRewrite || dvQuery != dvRewrite) {
|
||||
return new IndexOrDocValuesQuery(indexRewrite, dvRewrite);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
|
||||
final Weight indexWeight = indexQuery.createWeight(searcher, needsScores, boost);
|
||||
final Weight dvWeight = dvQuery.createWeight(searcher, needsScores, boost);
|
||||
return new ConstantScoreWeight(this, boost) {
|
||||
@Override
|
||||
public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
|
||||
return indexWeight.bulkScorer(context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
|
||||
final ScorerSupplier indexScorerSupplier = indexWeight.scorerSupplier(context);
|
||||
final ScorerSupplier dvScorerSupplier = dvWeight.scorerSupplier(context);
|
||||
if (indexScorerSupplier == null || dvScorerSupplier == null) {
|
||||
return null;
|
||||
}
|
||||
return new ScorerSupplier() {
|
||||
@Override
|
||||
public Scorer get(boolean randomAccess) throws IOException {
|
||||
return (randomAccess ? dvScorerSupplier : indexScorerSupplier).get(randomAccess);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return Math.min(indexScorerSupplier.cost(), dvScorerSupplier.cost());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
ScorerSupplier scorerSupplier = scorerSupplier(context);
|
||||
if (scorerSupplier == null) {
|
||||
return null;
|
||||
}
|
||||
return scorerSupplier.get(false);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You 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,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.search;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.LongPoint;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
public class TestIndexOrDocValuesQuery extends LuceneTestCase {
|
||||
|
||||
public void testUseIndexForSelectiveQueries() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig()
|
||||
// relies on costs and PointValues.estimateCost so we need the default codec
|
||||
.setCodec(TestUtil.getDefaultCodec()));
|
||||
for (int i = 0; i < 2000; ++i) {
|
||||
Document doc = new Document();
|
||||
if (i == 42) {
|
||||
doc.add(new StringField("f1", "bar", Store.NO));
|
||||
doc.add(new LongPoint("f2", 42L));
|
||||
doc.add(new NumericDocValuesField("f2", 42L));
|
||||
} else if (i == 100) {
|
||||
doc.add(new StringField("f1", "foo", Store.NO));
|
||||
doc.add(new LongPoint("f2", 2L));
|
||||
doc.add(new NumericDocValuesField("f2", 2L));
|
||||
} else {
|
||||
doc.add(new StringField("f1", "bar", Store.NO));
|
||||
doc.add(new LongPoint("f2", 2L));
|
||||
doc.add(new NumericDocValuesField("f2", 2L));
|
||||
}
|
||||
w.addDocument(doc);
|
||||
}
|
||||
w.forceMerge(1);
|
||||
IndexReader reader = DirectoryReader.open(w);
|
||||
IndexSearcher searcher = newSearcher(reader);
|
||||
searcher.setQueryCache(null);
|
||||
|
||||
// The term query is more selective, so the IndexOrDocValuesQuery should use doc values
|
||||
final Query q1 = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("f1", "foo")), Occur.MUST)
|
||||
.add(new IndexOrDocValuesQuery(LongPoint.newExactQuery("f2", 2), new DocValuesNumbersQuery("f2", 2L)), Occur.MUST)
|
||||
.build();
|
||||
|
||||
final Weight w1 = searcher.createNormalizedWeight(q1, random().nextBoolean());
|
||||
final Scorer s1 = w1.scorer(reader.leaves().get(0));
|
||||
assertNotNull(s1.twoPhaseIterator()); // means we use doc values
|
||||
|
||||
// The term query is less selective, so the IndexOrDocValuesQuery should use points
|
||||
final Query q2 = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("f1", "bar")), Occur.MUST)
|
||||
.add(new IndexOrDocValuesQuery(LongPoint.newExactQuery("f2", 42), new DocValuesNumbersQuery("f2", 42L)), Occur.MUST)
|
||||
.build();
|
||||
|
||||
final Weight w2 = searcher.createNormalizedWeight(q2, random().nextBoolean());
|
||||
final Scorer s2 = w2.scorer(reader.leaves().get(0));
|
||||
assertNull(s2.twoPhaseIterator()); // means we use points
|
||||
|
||||
reader.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -24,7 +24,6 @@ import java.util.List;
|
|||
|
||||
import com.carrotsearch.randomizedtesting.annotations.Repeat;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.SimpleCollector;
|
||||
import org.apache.lucene.search.TermInSetQuery;
|
||||
|
@ -36,6 +35,7 @@ import org.apache.lucene.spatial.prefix.tree.DateRangePrefixTree;
|
|||
import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree;
|
||||
import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -127,12 +127,12 @@ public class NumberRangeFacetsTest extends StrategyTestCase {
|
|||
Collections.shuffle(acceptFieldIds, random());
|
||||
acceptFieldIds = acceptFieldIds.subList(0, randomInt(acceptFieldIds.size()));
|
||||
if (!acceptFieldIds.isEmpty()) {
|
||||
List<Term> terms = new ArrayList<>();
|
||||
List<BytesRef> terms = new ArrayList<>();
|
||||
for (Integer acceptDocId : acceptFieldIds) {
|
||||
terms.add(new Term("id", acceptDocId.toString()));
|
||||
terms.add(new BytesRef(acceptDocId.toString()));
|
||||
}
|
||||
|
||||
topAcceptDocs = searchForDocBits(new TermInSetQuery(terms));
|
||||
topAcceptDocs = searchForDocBits(new TermInSetQuery("id", terms));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -332,6 +332,7 @@ public class AnalyzingSuggester extends Lookup implements Accountable {
|
|||
TokenStreamToAutomaton getTokenStreamToAutomaton() {
|
||||
final TokenStreamToAutomaton tsta = new TokenStreamToAutomaton();
|
||||
tsta.setPreservePositionIncrements(preservePositionIncrements);
|
||||
tsta.setFinalOffsetGapAsHole(true);
|
||||
return tsta;
|
||||
}
|
||||
|
||||
|
@ -865,7 +866,7 @@ public class AnalyzingSuggester extends Lookup implements Accountable {
|
|||
// Turn tokenstream into automaton:
|
||||
Automaton automaton = null;
|
||||
try (TokenStream ts = queryAnalyzer.tokenStream("", key.toString())) {
|
||||
automaton = getTokenStreamToAutomaton().toAutomaton(ts);
|
||||
automaton = getTokenStreamToAutomaton().toAutomaton(ts);
|
||||
}
|
||||
|
||||
automaton = replaceSep(automaton);
|
||||
|
|
|
@ -41,11 +41,16 @@ import org.apache.lucene.util.Attribute;
|
|||
import org.apache.lucene.util.AttributeFactory;
|
||||
import org.apache.lucene.util.AttributeImpl;
|
||||
import org.apache.lucene.util.AttributeReflector;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.LineFileDocs;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.Rethrow;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.automaton.Automaton;
|
||||
import org.apache.lucene.util.automaton.AutomatonTestUtil;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
|
||||
/**
|
||||
* Base class for all Lucene unit tests that use TokenStreams.
|
||||
|
@ -166,6 +171,8 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
|
|||
final Map<Integer,Integer> posToStartOffset = new HashMap<>();
|
||||
final Map<Integer,Integer> posToEndOffset = new HashMap<>();
|
||||
|
||||
// TODO: would be nice to be able to assert silly duplicated tokens are not created, but a number of cases do this "legitimately": LUCENE-7622
|
||||
|
||||
ts.reset();
|
||||
int pos = -1;
|
||||
int lastStartOffset = 0;
|
||||
|
@ -182,7 +189,7 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
|
|||
checkClearAtt.getAndResetClearCalled(); // reset it, because we called clearAttribute() before
|
||||
assertTrue("token "+i+" does not exist", ts.incrementToken());
|
||||
assertTrue("clearAttributes() was not called correctly in TokenStream chain", checkClearAtt.getAndResetClearCalled());
|
||||
|
||||
|
||||
assertEquals("term "+i, output[i], termAtt.toString());
|
||||
if (startOffsets != null) {
|
||||
assertEquals("startOffset " + i + " term=" + termAtt, startOffsets[i], offsetAtt.startOffset());
|
||||
|
@ -261,12 +268,12 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
if (posLengthAtt != null) {
|
||||
assertTrue("posLength must be >= 1", posLengthAtt.getPositionLength() >= 1);
|
||||
assertTrue("posLength must be >= 1; got: " + posLengthAtt.getPositionLength(), posLengthAtt.getPositionLength() >= 1);
|
||||
}
|
||||
}
|
||||
|
||||
if (ts.incrementToken()) {
|
||||
fail("TokenStream has more tokens than expected (expected count=" + output.length + "); extra token=" + termAtt);
|
||||
fail("TokenStream has more tokens than expected (expected count=" + output.length + "); extra token=" + ts.getAttribute(CharTermAttribute.class));
|
||||
}
|
||||
|
||||
// repeat our extra safety checks for end()
|
||||
|
@ -977,4 +984,105 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
|
|||
public static AttributeFactory newAttributeFactory() {
|
||||
return newAttributeFactory(random());
|
||||
}
|
||||
|
||||
private static String toString(Set<String> strings) {
|
||||
List<String> stringsList = new ArrayList<>(strings);
|
||||
Collections.sort(stringsList);
|
||||
StringBuilder b = new StringBuilder();
|
||||
for(String s : stringsList) {
|
||||
b.append(" ");
|
||||
b.append(s);
|
||||
b.append('\n');
|
||||
}
|
||||
return b.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Enumerates all accepted strings in the token graph created by the analyzer on the provided text, and then
|
||||
* asserts that it's equal to the expected strings.
|
||||
* Uses {@link TokenStreamToAutomaton} to create an automaton. Asserts the finite strings of the automaton are all
|
||||
* and only the given valid strings.
|
||||
* @param analyzer analyzer containing the SynonymFilter under test.
|
||||
* @param text text to be analyzed.
|
||||
* @param expectedStrings all expected finite strings.
|
||||
*/
|
||||
public static void assertGraphStrings(Analyzer analyzer, String text, String... expectedStrings) throws IOException {
|
||||
checkAnalysisConsistency(random(), analyzer, true, text, true);
|
||||
try (TokenStream tokenStream = analyzer.tokenStream("dummy", text)) {
|
||||
assertGraphStrings(tokenStream, expectedStrings);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Enumerates all accepted strings in the token graph created by the already initialized {@link TokenStream}.
|
||||
*/
|
||||
public static void assertGraphStrings(TokenStream tokenStream, String... expectedStrings) throws IOException {
|
||||
Automaton automaton = new TokenStreamToAutomaton().toAutomaton(tokenStream);
|
||||
Set<IntsRef> actualStringPaths = AutomatonTestUtil.getFiniteStringsRecursive(automaton, -1);
|
||||
|
||||
Set<String> expectedStringsSet = new HashSet<>(Arrays.asList(expectedStrings));
|
||||
|
||||
BytesRefBuilder scratchBytesRefBuilder = new BytesRefBuilder();
|
||||
Set<String> actualStrings = new HashSet<>();
|
||||
for (IntsRef ir: actualStringPaths) {
|
||||
actualStrings.add(Util.toBytesRef(ir, scratchBytesRefBuilder).utf8ToString().replace((char) TokenStreamToAutomaton.POS_SEP, ' '));
|
||||
}
|
||||
for (String s : actualStrings) {
|
||||
assertTrue("Analyzer created unexpected string path: " + s + "\nexpected:\n" + toString(expectedStringsSet) + "\nactual:\n" + toString(actualStrings), expectedStringsSet.contains(s));
|
||||
}
|
||||
for (String s : expectedStrings) {
|
||||
assertTrue("Analyzer created unexpected string path: " + s + "\nexpected:\n" + toString(expectedStringsSet) + "\nactual:\n" + toString(actualStrings), actualStrings.contains(s));
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns all paths accepted by the token stream graph produced by analyzing text with the provided analyzer. The tokens {@link
|
||||
* CharTermAttribute} values are concatenated, and separated with space. */
|
||||
public static Set<String> getGraphStrings(Analyzer analyzer, String text) throws IOException {
|
||||
try(TokenStream tokenStream = analyzer.tokenStream("dummy", text)) {
|
||||
return getGraphStrings(tokenStream);
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns all paths accepted by the token stream graph produced by the already initialized {@link TokenStream}. */
|
||||
public static Set<String> getGraphStrings(TokenStream tokenStream) throws IOException {
|
||||
Automaton automaton = new TokenStreamToAutomaton().toAutomaton(tokenStream);
|
||||
Set<IntsRef> actualStringPaths = AutomatonTestUtil.getFiniteStringsRecursive(automaton, -1);
|
||||
BytesRefBuilder scratchBytesRefBuilder = new BytesRefBuilder();
|
||||
Set<String> paths = new HashSet<>();
|
||||
for (IntsRef ir: actualStringPaths) {
|
||||
paths.add(Util.toBytesRef(ir, scratchBytesRefBuilder).utf8ToString().replace((char) TokenStreamToAutomaton.POS_SEP, ' '));
|
||||
}
|
||||
return paths;
|
||||
}
|
||||
|
||||
/** Returns a {@code String} summary of the tokens this analyzer produces on this text */
|
||||
public static String toString(Analyzer analyzer, String text) throws IOException {
|
||||
try(TokenStream ts = analyzer.tokenStream("field", text)) {
|
||||
StringBuilder b = new StringBuilder();
|
||||
CharTermAttribute termAtt = ts.getAttribute(CharTermAttribute.class);
|
||||
PositionIncrementAttribute posIncAtt = ts.getAttribute(PositionIncrementAttribute.class);
|
||||
PositionLengthAttribute posLengthAtt = ts.getAttribute(PositionLengthAttribute.class);
|
||||
OffsetAttribute offsetAtt = ts.getAttribute(OffsetAttribute.class);
|
||||
assertNotNull(offsetAtt);
|
||||
ts.reset();
|
||||
int pos = -1;
|
||||
while (ts.incrementToken()) {
|
||||
pos += posIncAtt.getPositionIncrement();
|
||||
b.append(termAtt);
|
||||
b.append(" at pos=");
|
||||
b.append(pos);
|
||||
if (posLengthAtt != null) {
|
||||
b.append(" to pos=");
|
||||
b.append(pos + posLengthAtt.getPositionLength());
|
||||
}
|
||||
b.append(" offsets=");
|
||||
b.append(offsetAtt.startOffset());
|
||||
b.append('-');
|
||||
b.append(offsetAtt.endOffset());
|
||||
b.append('\n');
|
||||
}
|
||||
ts.end();
|
||||
return b.toString();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -93,7 +93,10 @@ public class TokenStreamToDot {
|
|||
final int endOffset = offsetAtt.endOffset();
|
||||
//System.out.println("start=" + startOffset + " end=" + endOffset + " len=" + inputText.length());
|
||||
if (inputText != null) {
|
||||
arcLabel += " / " + inputText.substring(startOffset, endOffset);
|
||||
String fragment = inputText.substring(startOffset, endOffset);
|
||||
if (fragment.equals(termAtt.toString()) == false) {
|
||||
arcLabel += " / " + fragment;
|
||||
}
|
||||
} else {
|
||||
arcLabel += " / " + startOffset + "-" + endOffset;
|
||||
}
|
||||
|
|
|
@ -68,13 +68,8 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
|
|||
|
||||
@Override
|
||||
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
|
||||
MutableBits raw = bits;
|
||||
/**
|
||||
* bits is not necessarily an AssertingMutableBits because index sorting needs to wrap it in a sorted view.
|
||||
*/
|
||||
if (bits instanceof AssertingMutableBits) {
|
||||
raw = (MutableBits) ((AssertingMutableBits) bits).in;
|
||||
}
|
||||
assert bits instanceof AssertingMutableBits;
|
||||
MutableBits raw = (MutableBits) ((AssertingMutableBits)bits).in;
|
||||
check(raw, info.info.maxDoc(), info.getDelCount() + newDelCount);
|
||||
in.writeLiveDocs(raw, dir, info, newDelCount, context);
|
||||
}
|
||||
|
|
|
@ -133,6 +133,11 @@ class CrankyPointsFormat extends PointsFormat {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimatePointCount(IntersectVisitor visitor) {
|
||||
return delegate.estimatePointCount(visitor);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue() throws IOException {
|
||||
if (random.nextInt(100) == 0) {
|
||||
|
|
|
@ -883,6 +883,13 @@ public class AssertingLeafReader extends FilterLeafReader {
|
|||
in.intersect(new AssertingIntersectVisitor(in.getNumDimensions(), in.getBytesPerDimension(), visitor));
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimatePointCount(IntersectVisitor visitor) {
|
||||
long cost = in.estimatePointCount(visitor);
|
||||
assert cost >= 0;
|
||||
return cost;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue() throws IOException {
|
||||
return Objects.requireNonNull(in.getMinPackedValue());
|
||||
|
|
|
@ -33,9 +33,45 @@ class AssertingWeight extends FilterWeight {
|
|||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
final Scorer inScorer = in.scorer(context);
|
||||
assert inScorer == null || inScorer.docID() == -1;
|
||||
return AssertingScorer.wrap(new Random(random.nextLong()), inScorer, needsScores);
|
||||
if (random.nextBoolean()) {
|
||||
final Scorer inScorer = in.scorer(context);
|
||||
assert inScorer == null || inScorer.docID() == -1;
|
||||
return AssertingScorer.wrap(new Random(random.nextLong()), inScorer, needsScores);
|
||||
} else {
|
||||
final ScorerSupplier scorerSupplier = scorerSupplier(context);
|
||||
if (scorerSupplier == null) {
|
||||
return null;
|
||||
}
|
||||
if (random.nextBoolean()) {
|
||||
// Evil: make sure computing the cost has no side effects
|
||||
scorerSupplier.cost();
|
||||
}
|
||||
return scorerSupplier.get(false);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
|
||||
final ScorerSupplier inScorerSupplier = in.scorerSupplier(context);
|
||||
if (inScorerSupplier == null) {
|
||||
return null;
|
||||
}
|
||||
return new ScorerSupplier() {
|
||||
private boolean getCalled = false;
|
||||
@Override
|
||||
public Scorer get(boolean randomAccess) throws IOException {
|
||||
assert getCalled == false;
|
||||
getCalled = true;
|
||||
return AssertingScorer.wrap(new Random(random.nextLong()), inScorerSupplier.get(randomAccess), needsScores);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
final long cost = inScorerSupplier.cost();
|
||||
assert cost >= 0;
|
||||
return cost;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -74,9 +74,6 @@ Optimizations
|
|||
* SOLR-9584: Support Solr being proxied with another endpoint than default /solr, by using relative links
|
||||
in AdminUI javascripts (Yun Jie Zhou via janhoy)
|
||||
|
||||
* SOLR-9941: Clear the deletes lists at UpdateLog before replaying from log. This prevents redundantly pre-applying
|
||||
DBQs, during the log replay, to every update in the log as if the DBQs were out of order. (hossman, Ishan Chattopadhyaya)
|
||||
|
||||
================== 6.5.0 ==================
|
||||
|
||||
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
|
||||
|
@ -90,9 +87,22 @@ Apache UIMA 2.3.1
|
|||
Apache ZooKeeper 3.4.6
|
||||
Jetty 9.3.14.v20161028
|
||||
|
||||
Detailed Change List
|
||||
----------------------
|
||||
|
||||
(No Changes)
|
||||
Bug Fixes
|
||||
----------------------
|
||||
|
||||
* SOLR-9976: Fix init bug in SegmentsInfoRequestHandlerTest (hossman)
|
||||
|
||||
* SOLR-9977: Fix config bug in DistribDocExpirationUpdateProcessorTest that allowed false assumptions
|
||||
about when index version changes (hossman)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
|
||||
* SOLR-9941: Clear the deletes lists at UpdateLog before replaying from log. This prevents redundantly pre-applying
|
||||
DBQs, during the log replay, to every update in the log as if the DBQs were out of order. (hossman, Ishan Chattopadhyaya)
|
||||
|
||||
================== 6.4.0 ==================
|
||||
|
||||
|
|
|
@ -295,6 +295,13 @@ public class UnifiedSolrHighlighter extends SolrHighlighter implements PluginInf
|
|||
|
||||
@Override
|
||||
protected BreakIterator getBreakIterator(String field) {
|
||||
// Use a default fragsize the same as the regex Fragmenter (original Highlighter) since we're
|
||||
// both likely shooting for sentence-like patterns.
|
||||
int fragsize = params.getFieldInt(field, HighlightParams.FRAGSIZE, LuceneRegexFragmenter.DEFAULT_FRAGMENT_SIZE);
|
||||
if (fragsize == 0) { // special value; no fragmenting
|
||||
return new WholeBreakIterator();
|
||||
}
|
||||
|
||||
String language = params.getFieldParam(field, HighlightParams.BS_LANGUAGE);
|
||||
String country = params.getFieldParam(field, HighlightParams.BS_COUNTRY);
|
||||
String variant = params.getFieldParam(field, HighlightParams.BS_VARIANT);
|
||||
|
@ -302,9 +309,6 @@ public class UnifiedSolrHighlighter extends SolrHighlighter implements PluginInf
|
|||
String type = params.getFieldParam(field, HighlightParams.BS_TYPE);
|
||||
BreakIterator baseBI = parseBreakIterator(type, locale);
|
||||
|
||||
// Use a default fragsize the same as the regex Fragmenter (original Highlighter) since we're
|
||||
// both likely shooting for sentence-like patterns.
|
||||
int fragsize = params.getFieldInt(field, HighlightParams.FRAGSIZE, LuceneRegexFragmenter.DEFAULT_FRAGMENT_SIZE);
|
||||
if (fragsize <= 1 || baseBI instanceof WholeBreakIterator) { // no real minimum size
|
||||
return baseBI;
|
||||
}
|
||||
|
|
|
@ -1,47 +0,0 @@
|
|||
<?xml version="1.0" ?>
|
||||
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
contributor license agreements. See the NOTICE file distributed with
|
||||
this work for additional information regarding copyright ownership.
|
||||
The ASF licenses this file to You 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,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
<!--
|
||||
|
||||
A solrconfig.xml snippet containing indexConfig settings for randomized testing.
|
||||
|
||||
-->
|
||||
<indexConfig>
|
||||
<!-- this sys property is not set by SolrTestCaseJ4 because we ideally want to use
|
||||
the RandomMergePolicy in all tests - but some tests expect very specific
|
||||
Merge behavior, so those tests can set it as needed.
|
||||
-->
|
||||
<mergePolicy enable="${solr.tests.useMergePolicy:true}" class="${solr.tests.mergePolicy:org.apache.solr.util.RandomMergePolicy}" />
|
||||
<mergePolicyFactory enable="${solr.tests.useMergePolicyFactory:true}" class="${solr.tests.mergePolicyFactory:org.apache.solr.util.RandomMergePolicyFactory}" />
|
||||
|
||||
<useCompoundFile>${useCompoundFile:false}</useCompoundFile>
|
||||
|
||||
<maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
|
||||
<ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
|
||||
|
||||
<mergeScheduler class="${solr.tests.mergeScheduler}" />
|
||||
|
||||
<writeLockTimeout>1000</writeLockTimeout>
|
||||
<commitLockTimeout>10000</commitLockTimeout>
|
||||
|
||||
<!-- this sys property is not set by SolrTestCaseJ4 because almost all tests should
|
||||
use the single process lockType for speed - but tests that explicitly need
|
||||
to vary the lockType canset it as needed.
|
||||
-->
|
||||
<lockType>${solr.tests.lockType:single}</lockType>
|
||||
</indexConfig>
|
|
@ -25,14 +25,25 @@
|
|||
|
||||
<dataDir>${solr.data.dir:}</dataDir>
|
||||
|
||||
<indexConfig>
|
||||
<!-- NOTE: we do *NOT* want Randomized Merging for these tests,
|
||||
because we need to be able to assert that index changes are only happening
|
||||
on the shards we expected them to as a result of our deletes.
|
||||
|
||||
(the random/mock merge classes can cause new readers to be opened after a commit
|
||||
even if the index itself hasn't changed - ex: new segments file listing same exact segments
|
||||
|
||||
Instead use Solr defaults for almost everything
|
||||
-->
|
||||
<lockType>${solr.tests.lockType:single}</lockType>
|
||||
</indexConfig>
|
||||
|
||||
<directoryFactory name="DirectoryFactory"
|
||||
class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
|
||||
<schemaFactory class="ClassicIndexSchemaFactory"/>
|
||||
|
||||
<luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
|
||||
|
||||
<xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
|
||||
|
||||
<updateHandler class="solr.DirectUpdateHandler2">
|
||||
<updateLog>
|
||||
<str name="dir">${solr.ulog.dir:}</str>
|
||||
|
|
|
@ -16,9 +16,11 @@
|
|||
*/
|
||||
package org.apache.solr.handler.admin;
|
||||
|
||||
import org.apache.lucene.index.LogDocMergePolicy;
|
||||
import org.apache.lucene.util.Version;
|
||||
import org.apache.solr.index.LogDocMergePolicyFactory;
|
||||
import org.apache.solr.util.AbstractSolrTestCase;
|
||||
import org.junit.Before;
|
||||
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -32,13 +34,17 @@ public class SegmentsInfoRequestHandlerTest extends AbstractSolrTestCase {
|
|||
|
||||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
System.setProperty("enable.update.log", "false");
|
||||
System.setProperty("solr.tests.useMergePolicy", "false");
|
||||
initCore("solrconfig.xml", "schema12.xml");
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() throws Exception {
|
||||
// we need a consistent segmentation to ensure we don't get a random
|
||||
// merge that reduces the total num docs in all segments, or the number of deletes
|
||||
//
|
||||
systemSetPropertySolrTestsMergePolicy(LogDocMergePolicy.class.getName());
|
||||
systemSetPropertySolrTestsMergePolicyFactory(LogDocMergePolicyFactory.class.getName());
|
||||
|
||||
System.setProperty("enable.update.log", "false"); // no _version_ in our schema
|
||||
initCore("solrconfig.xml", "schema12.xml"); // segments API shouldn't depend on _version_ or ulog
|
||||
|
||||
// build up an index with at least 2 segments and some deletes
|
||||
for (int i = 0; i < DOC_COUNT; i++) {
|
||||
assertU(adoc("id","SOLR100" + i, "name","Apache Solr:" + i));
|
||||
}
|
||||
|
|
|
@ -183,6 +183,42 @@ public class SpellCheckComponentTest extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testCollateExtendedResultsWithJsonNl() throws Exception {
|
||||
final String q = "documemtsss broens";
|
||||
final String jsonNl = "map";
|
||||
final boolean collateExtendedResults = random().nextBoolean();
|
||||
final List<String> testsList = new ArrayList<String>();
|
||||
if (collateExtendedResults) {
|
||||
testsList.add("/spellcheck/collations/collation/collationQuery=='document brown'");
|
||||
testsList.add("/spellcheck/collations/collation/hits==0");
|
||||
switch (jsonNl) {
|
||||
case "map":
|
||||
testsList.add("/spellcheck/collations/collation/misspellingsAndCorrections/documemtsss=='document'");
|
||||
testsList.add("/spellcheck/collations/collation/misspellingsAndCorrections/broens=='brown'");
|
||||
break;
|
||||
default:
|
||||
fail("unexpected json.nl choice: "+jsonNl);
|
||||
break;
|
||||
}
|
||||
} else {
|
||||
testsList.add("/spellcheck/collations/collation=='document brown'");
|
||||
}
|
||||
final String[] testsArray = new String[testsList.size()];
|
||||
implTestCollateExtendedResultsWithJsonNl(q, jsonNl, collateExtendedResults, testsList.toArray(testsArray));
|
||||
}
|
||||
|
||||
private void implTestCollateExtendedResultsWithJsonNl(String q, String jsonNl, boolean collateExtendedResults, String ... tests) throws Exception {
|
||||
final SolrQueryRequest solrQueryRequest = req(
|
||||
CommonParams.QT, rh,
|
||||
CommonParams.Q, q,
|
||||
"json.nl", jsonNl,
|
||||
SpellCheckComponent.COMPONENT_NAME, "true",
|
||||
SpellingParams.SPELLCHECK_COLLATE_EXTENDED_RESULTS, Boolean.toString(collateExtendedResults),
|
||||
SpellingParams.SPELLCHECK_COLLATE, "true");
|
||||
assertJQ(solrQueryRequest, tests);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCorrectSpelling() throws Exception {
|
||||
// Make sure correct spellings are signaled in the response
|
||||
|
|
|
@ -79,7 +79,7 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
|
|||
assertU(commit());
|
||||
assertQ("multiple snippets test",
|
||||
req("q", "text:document", "sort", "id asc", "hl", "true", "hl.snippets", "2", "hl.bs.type", "SENTENCE",
|
||||
"hl.fragsize", "0"),
|
||||
"hl.fragsize", "-1"),
|
||||
"count(//lst[@name='highlighting']/lst[@name='101']/arr[@name='text']/*)=2",
|
||||
"//lst[@name='highlighting']/lst[@name='101']/arr/str[1]='<em>Document</em> snippet one. '",
|
||||
"//lst[@name='highlighting']/lst[@name='101']/arr/str[2]='<em>Document</em> snippet two.'");
|
||||
|
@ -214,9 +214,12 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
|
|||
public void testBreakIteratorWhole() {
|
||||
assertU(adoc("text", "Document one has a first sentence. Document two has a second sentence.", "id", "103"));
|
||||
assertU(commit());
|
||||
assertQ("different breakiterator",
|
||||
assertQ("WHOLE breakiterator",
|
||||
req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "WHOLE", "hl.fragsize", "-1"),
|
||||
"//lst[@name='highlighting']/lst[@name='103']/arr[@name='text']/str='<em>Document</em> one has a first sentence. <em>Document</em> two has a second sentence.'");
|
||||
assertQ("hl.fragsize 0 is equivalent to WHOLE",
|
||||
req("q", "text:document", "sort", "id asc", "hl", "true", "hl.fragsize", "0"),
|
||||
"//lst[@name='highlighting']/lst[@name='103']/arr[@name='text']/str='<em>Document</em> one has a first sentence. <em>Document</em> two has a second sentence.'");
|
||||
}
|
||||
|
||||
public void testFragsize() {
|
||||
|
|
|
@ -259,6 +259,21 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
|
|||
}
|
||||
assertEquals(26, ((TermInSetQuery)qq).getTermData().size());
|
||||
|
||||
// test terms queries of two different fields (LUCENE-7637 changed to require all terms be in the same field)
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (int i=0; i<17; i++) {
|
||||
char letter = (char)('a'+i);
|
||||
sb.append("foo_s:" + letter + " bar_s:" + letter + " ");
|
||||
}
|
||||
qParser = QParser.getParser(sb.toString(), req);
|
||||
qParser.setIsFilter(true); // this may change in the future
|
||||
q = qParser.getQuery();
|
||||
assertEquals(2, ((BooleanQuery)q).clauses().size());
|
||||
for (BooleanClause clause : ((BooleanQuery)q).clauses()) {
|
||||
qq = clause.getQuery();
|
||||
assertEquals(17, ((TermInSetQuery)qq).getTermData().size());
|
||||
}
|
||||
|
||||
req.close();
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue