mirror of https://github.com/apache/lucene.git
Merge remote-tracking branch 'source/master' into remove-constructor-wrapper-classes
This commit is contained in:
commit
78079fc552
|
@ -202,6 +202,18 @@ New Features
|
|||
IndexFileDeleter already accounts for that for existing files which we can
|
||||
now use to also take pending deletes into account which ensures that all file
|
||||
generations per segment always go forward. (Simon Willnauer)
|
||||
|
||||
* LUCENE-7960: Add preserveOriginal option to the NGram and EdgeNGram filters.
|
||||
(Ingomar Wesp, Shawn Heisey via Robert Muir)
|
||||
|
||||
* LUCENE-8335: Enforce soft-deletes field up-front. Soft deletes field must be marked
|
||||
as such once it's introduced and can't be changed after the fact.
|
||||
(Nhat Nguyen via Simon Willnauer)
|
||||
|
||||
* LUCENE-8332: New ConcatenateGraphFilter for concatenating all tokens into one (or more
|
||||
in the event of a graph input). This is useful for fast analyzed exact-match lookup,
|
||||
suggesters, and as a component of a named entity recognition system. This was excised
|
||||
out of CompletionTokenStream in the NRT doc suggester. (David Smiley, Jim Ferenczi)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
|
|
|
@ -0,0 +1,375 @@
|
|||
/*
|
||||
* 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.TokenStream;
|
||||
import org.apache.lucene.analysis.TokenStreamToAutomaton;
|
||||
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.TermToBytesRefAttribute;
|
||||
import org.apache.lucene.util.AttributeImpl;
|
||||
import org.apache.lucene.util.AttributeReflector;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.automaton.Automaton;
|
||||
import org.apache.lucene.util.automaton.LimitedFiniteStringsIterator;
|
||||
import org.apache.lucene.util.automaton.Operations;
|
||||
import org.apache.lucene.util.automaton.TooComplexToDeterminizeException;
|
||||
import org.apache.lucene.util.automaton.Transition;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
|
||||
/**
|
||||
* Concatenates/Joins every incoming token with a separator into one output token for every path through the
|
||||
* token stream (which is a graph). In simple cases this yields one token, but in the presence of any tokens with
|
||||
* a zero positionIncrmeent (e.g. synonyms) it will be more. This filter uses the token bytes, position increment,
|
||||
* and position length of the incoming stream. Other attributes are not used or manipulated.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class ConcatenateGraphFilter extends TokenStream {
|
||||
|
||||
/*
|
||||
* Token stream which converts a provided token stream to an automaton.
|
||||
* The accepted strings enumeration from the automaton are available through the
|
||||
* {@link org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute} attribute
|
||||
* The token stream uses a {@link org.apache.lucene.analysis.tokenattributes.PayloadAttribute} to store
|
||||
* a completion's payload (see {@link ConcatenateGraphFilter#setPayload(org.apache.lucene.util.BytesRef)})
|
||||
*/
|
||||
|
||||
/**
|
||||
* Represents the separation between tokens, if
|
||||
* <code>preserveSep</code> is <code>true</code>.
|
||||
*/
|
||||
public final static int SEP_LABEL = TokenStreamToAutomaton.POS_SEP;
|
||||
public final static int DEFAULT_MAX_GRAPH_EXPANSIONS = Operations.DEFAULT_MAX_DETERMINIZED_STATES;
|
||||
public final static boolean DEFAULT_PRESERVE_SEP = true;
|
||||
public final static boolean DEFAULT_PRESERVE_POSITION_INCREMENTS = true;
|
||||
|
||||
private final BytesRefBuilderTermAttribute bytesAtt = addAttribute(BytesRefBuilderTermAttribute.class);
|
||||
private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
|
||||
private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
|
||||
|
||||
private final TokenStream inputTokenStream;
|
||||
private final boolean preserveSep;
|
||||
private final boolean preservePositionIncrements;
|
||||
private final int maxGraphExpansions;
|
||||
|
||||
private LimitedFiniteStringsIterator finiteStrings;
|
||||
private CharTermAttribute charTermAttribute;
|
||||
private boolean wasReset = false;
|
||||
private int endOffset;
|
||||
|
||||
/**
|
||||
* Creates a token stream to convert <code>input</code> to a token stream
|
||||
* of accepted strings by its token stream graph.
|
||||
* <p>
|
||||
* This constructor uses the default settings of the constants in this class.
|
||||
*/
|
||||
public ConcatenateGraphFilter(TokenStream inputTokenStream) {
|
||||
this(inputTokenStream, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, DEFAULT_MAX_GRAPH_EXPANSIONS);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a token stream to convert <code>input</code> to a token stream
|
||||
* of accepted strings by its token stream graph.
|
||||
*
|
||||
* @param inputTokenStream The input/incoming TokenStream
|
||||
* @param preserveSep Whether {@link #SEP_LABEL} should separate the input tokens in the concatenated token
|
||||
* @param preservePositionIncrements Whether to add an empty token for missing positions.
|
||||
* The effect is a consecutive {@link #SEP_LABEL}.
|
||||
* When false, it's as if there were no missing positions
|
||||
* (we pretend the surrounding tokens were adjacent).
|
||||
* @param maxGraphExpansions If the tokenStream graph has more than this many possible paths through, then we'll throw
|
||||
* {@link TooComplexToDeterminizeException} to preserve the stability and memory of the
|
||||
* machine.
|
||||
* @throws TooComplexToDeterminizeException if the tokenStream graph has more than {@code maxGraphExpansions}
|
||||
* expansions
|
||||
*
|
||||
*/
|
||||
public ConcatenateGraphFilter(TokenStream inputTokenStream, boolean preserveSep, boolean preservePositionIncrements, int maxGraphExpansions) {
|
||||
// Don't call the super(input) ctor - this is a true delegate and has a new attribute source since we consume
|
||||
// the input stream entirely in the first call to incrementToken
|
||||
this.inputTokenStream = inputTokenStream;
|
||||
this.preserveSep = preserveSep;
|
||||
this.preservePositionIncrements = preservePositionIncrements;
|
||||
this.maxGraphExpansions = maxGraphExpansions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
super.reset();
|
||||
// we only capture this if we really need it to save the UTF-8 to UTF-16 conversion
|
||||
charTermAttribute = getAttribute(CharTermAttribute.class); // may return null
|
||||
wasReset = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
if (finiteStrings == null) {
|
||||
if (wasReset == false) {
|
||||
throw new IllegalStateException("reset() missing before incrementToken");
|
||||
}
|
||||
// lazy init/consume
|
||||
Automaton automaton = toAutomaton(); // calls reset(), incrementToken() repeatedly, and end() on inputTokenStream
|
||||
finiteStrings = new LimitedFiniteStringsIterator(automaton, maxGraphExpansions);
|
||||
//note: would be nice to know the startOffset but toAutomaton doesn't capture it. We'll assume 0
|
||||
endOffset = inputTokenStream.getAttribute(OffsetAttribute.class).endOffset();
|
||||
}
|
||||
|
||||
IntsRef string = finiteStrings.next();
|
||||
if (string == null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
clearAttributes();
|
||||
|
||||
if (finiteStrings.size() > 1) { // if number of iterated strings so far is more than one...
|
||||
posIncrAtt.setPositionIncrement(0); // stacked
|
||||
}
|
||||
|
||||
offsetAtt.setOffset(0, endOffset);
|
||||
|
||||
Util.toBytesRef(string, bytesAtt.builder()); // now we have UTF-8
|
||||
if (charTermAttribute != null) {
|
||||
charTermAttribute.setLength(0);
|
||||
charTermAttribute.append(bytesAtt.toUTF16());
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void end() throws IOException {
|
||||
super.end();
|
||||
if (finiteStrings == null) { // thus inputTokenStream hasn't yet received end()
|
||||
inputTokenStream.end(); // the input TS may really want to see "end()" called even if incrementToken hasn't.
|
||||
} // else we already eagerly consumed inputTokenStream including end()
|
||||
if (endOffset != -1) {
|
||||
offsetAtt.setOffset(0, endOffset);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
//delegate lifecycle. Note toAutomaton does not close the stream
|
||||
inputTokenStream.close();
|
||||
finiteStrings = null;
|
||||
wasReset = false;//reset
|
||||
endOffset = -1;//reset
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the tokenStream to an automaton, treating the transition labels as utf-8. Does *not* close it.
|
||||
*/
|
||||
public Automaton toAutomaton() throws IOException {
|
||||
return toAutomaton(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the tokenStream to an automaton. Does *not* close it.
|
||||
*/
|
||||
public Automaton toAutomaton(boolean unicodeAware) throws IOException {
|
||||
// TODO refactor this
|
||||
// maybe we could hook up a modified automaton from TermAutomatonQuery here?
|
||||
|
||||
// Create corresponding automaton: labels are bytes
|
||||
// from each analyzed token, with byte 0 used as
|
||||
// separator between tokens:
|
||||
final TokenStreamToAutomaton tsta;
|
||||
if (preserveSep) {
|
||||
tsta = new EscapingTokenStreamToAutomaton(SEP_LABEL);
|
||||
} else {
|
||||
// When we're not preserving sep, we don't steal 0xff
|
||||
// byte, so we don't need to do any escaping:
|
||||
tsta = new TokenStreamToAutomaton();
|
||||
}
|
||||
tsta.setPreservePositionIncrements(preservePositionIncrements);
|
||||
tsta.setUnicodeArcs(unicodeAware);
|
||||
|
||||
Automaton automaton = tsta.toAutomaton(inputTokenStream);
|
||||
|
||||
// TODO: we can optimize this somewhat by determinizing
|
||||
// while we convert
|
||||
automaton = replaceSep(automaton, preserveSep, SEP_LABEL);
|
||||
// This automaton should not blow up during determinize:
|
||||
return Operations.determinize(automaton, maxGraphExpansions);
|
||||
}
|
||||
|
||||
/**
|
||||
* Just escapes the {@link #SEP_LABEL} byte with an extra.
|
||||
*/
|
||||
private static final class EscapingTokenStreamToAutomaton extends TokenStreamToAutomaton {
|
||||
|
||||
final BytesRefBuilder spare = new BytesRefBuilder();
|
||||
final byte sepLabel;
|
||||
|
||||
public EscapingTokenStreamToAutomaton(int sepLabel) {
|
||||
assert sepLabel <= Byte.MAX_VALUE;
|
||||
this.sepLabel = (byte) sepLabel;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BytesRef changeToken(BytesRef in) {
|
||||
int upto = 0;
|
||||
for (int i = 0; i < in.length; i++) {
|
||||
byte b = in.bytes[in.offset + i];
|
||||
if (b == sepLabel) {
|
||||
spare.grow(upto + 2);
|
||||
spare.setByteAt(upto++, sepLabel);
|
||||
spare.setByteAt(upto++, b);
|
||||
} else {
|
||||
spare.grow(upto + 1);
|
||||
spare.setByteAt(upto++, b);
|
||||
}
|
||||
}
|
||||
spare.setLength(upto);
|
||||
return spare.get();
|
||||
}
|
||||
}
|
||||
|
||||
// Replaces SEP with epsilon or remaps them if
|
||||
// we were asked to preserve them:
|
||||
private static Automaton replaceSep(Automaton a, boolean preserveSep, int sepLabel) {
|
||||
|
||||
Automaton result = new Automaton();
|
||||
|
||||
// Copy all states over
|
||||
int numStates = a.getNumStates();
|
||||
for (int s = 0; s < numStates; s++) {
|
||||
result.createState();
|
||||
result.setAccept(s, a.isAccept(s));
|
||||
}
|
||||
|
||||
// Go in reverse topo sort so we know we only have to
|
||||
// make one pass:
|
||||
Transition t = new Transition();
|
||||
int[] topoSortStates = Operations.topoSortStates(a);
|
||||
for (int i = 0; i < topoSortStates.length; i++) {
|
||||
int state = topoSortStates[topoSortStates.length - 1 - i];
|
||||
int count = a.initTransition(state, t);
|
||||
for (int j = 0; j < count; j++) {
|
||||
a.getNextTransition(t);
|
||||
if (t.min == TokenStreamToAutomaton.POS_SEP) {
|
||||
assert t.max == TokenStreamToAutomaton.POS_SEP;
|
||||
if (preserveSep) {
|
||||
// Remap to SEP_LABEL:
|
||||
result.addTransition(state, t.dest, sepLabel);
|
||||
} else {
|
||||
result.addEpsilon(state, t.dest);
|
||||
}
|
||||
} else if (t.min == TokenStreamToAutomaton.HOLE) {
|
||||
assert t.max == TokenStreamToAutomaton.HOLE;
|
||||
|
||||
// Just remove the hole: there will then be two
|
||||
// SEP tokens next to each other, which will only
|
||||
// match another hole at search time. Note that
|
||||
// it will also match an empty-string token ... if
|
||||
// that's somehow a problem we can always map HOLE
|
||||
// to a dedicated byte (and escape it in the
|
||||
// input).
|
||||
result.addEpsilon(state, t.dest);
|
||||
} else {
|
||||
result.addTransition(state, t.dest, t.min, t.max);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
result.finishState();
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Attribute providing access to the term builder and UTF-16 conversion
|
||||
* @lucene.internal
|
||||
*/
|
||||
public interface BytesRefBuilderTermAttribute extends TermToBytesRefAttribute {
|
||||
/**
|
||||
* Returns the builder from which the term is derived.
|
||||
*/
|
||||
BytesRefBuilder builder();
|
||||
|
||||
/**
|
||||
* Returns the term represented as UTF-16
|
||||
*/
|
||||
CharSequence toUTF16();
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementation of {@link BytesRefBuilderTermAttribute}
|
||||
* @lucene.internal
|
||||
*/
|
||||
public static final class BytesRefBuilderTermAttributeImpl extends AttributeImpl implements BytesRefBuilderTermAttribute, TermToBytesRefAttribute {
|
||||
private final BytesRefBuilder bytes = new BytesRefBuilder();
|
||||
private transient CharsRefBuilder charsRef;
|
||||
|
||||
/**
|
||||
* Sole constructor
|
||||
* no-op
|
||||
*/
|
||||
public BytesRefBuilderTermAttributeImpl() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRefBuilder builder() {
|
||||
return bytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytesRef() {
|
||||
return bytes.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
bytes.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copyTo(AttributeImpl target) {
|
||||
BytesRefBuilderTermAttributeImpl other = (BytesRefBuilderTermAttributeImpl) target;
|
||||
other.bytes.copyBytes(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AttributeImpl clone() {
|
||||
BytesRefBuilderTermAttributeImpl other = new BytesRefBuilderTermAttributeImpl();
|
||||
copyTo(other);
|
||||
return other;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reflectWith(AttributeReflector reflector) {
|
||||
reflector.reflect(TermToBytesRefAttribute.class, "bytes", getBytesRef());
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence toUTF16() {
|
||||
if (charsRef == null) {
|
||||
charsRef = new CharsRefBuilder();
|
||||
}
|
||||
charsRef.copyUTF8Bytes(getBytesRef());
|
||||
return charsRef.get();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,70 @@
|
|||
/*
|
||||
* 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.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.util.TokenFilterFactory;
|
||||
import org.apache.lucene.util.automaton.TooComplexToDeterminizeException;
|
||||
|
||||
/**
|
||||
* Factory for {@link ConcatenateGraphFilter}.
|
||||
*
|
||||
* <ul>
|
||||
* <li><tt>preserveSep</tt>:
|
||||
* Whether {@link ConcatenateGraphFilter#SEP_LABEL}
|
||||
* should separate the input tokens in the concatenated token
|
||||
* </li>
|
||||
* <li><tt>preservePositionIncrements</tt>:
|
||||
* Whether to add an empty token for missing positions.
|
||||
* The effect is a consecutive {@link ConcatenateGraphFilter#SEP_LABEL}.
|
||||
* When false, it's as if there were no missing positions
|
||||
* (we pretend the surrounding tokens were adjacent).
|
||||
* </li>
|
||||
* <li><tt>maxGraphExpansions</tt>:
|
||||
* If the tokenStream graph has more than this many possible paths through, then we'll throw
|
||||
* {@link TooComplexToDeterminizeException} to preserve the stability and memory of the
|
||||
* machine.
|
||||
* </li>
|
||||
* </ul>
|
||||
* @see ConcatenateGraphFilter
|
||||
* @since 7.4.0
|
||||
*/
|
||||
public class ConcatenateGraphFilterFactory extends TokenFilterFactory {
|
||||
|
||||
private boolean preserveSep;
|
||||
private boolean preservePositionIncrements;
|
||||
private int maxGraphExpansions;
|
||||
|
||||
public ConcatenateGraphFilterFactory(Map<String, String> args) {
|
||||
super(args);
|
||||
|
||||
preserveSep = getBoolean(args, "preserveSep", ConcatenateGraphFilter.DEFAULT_PRESERVE_SEP);
|
||||
preservePositionIncrements = getBoolean(args, "preservePositionIncrements", ConcatenateGraphFilter.DEFAULT_PRESERVE_POSITION_INCREMENTS);
|
||||
maxGraphExpansions = getInt(args, "maxGraphExpansions", ConcatenateGraphFilter.DEFAULT_MAX_GRAPH_EXPANSIONS);
|
||||
|
||||
if (!args.isEmpty()) {
|
||||
throw new IllegalArgumentException("Unknown parameters: " + args);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TokenStream create(TokenStream input) {
|
||||
return new ConcatenateGraphFilter(input, preserveSep, preservePositionIncrements, maxGraphExpansions);
|
||||
}
|
||||
}
|
|
@ -81,8 +81,7 @@ public class FingerprintFilter extends TokenFilter {
|
|||
|
||||
@Override
|
||||
public final boolean incrementToken() throws IOException {
|
||||
if (uniqueTerms != null) {
|
||||
// We have already built the single output token - there's no more
|
||||
if (inputEnded) {
|
||||
return false;
|
||||
}
|
||||
boolean result = buildSingleOutputToken();
|
||||
|
@ -177,6 +176,7 @@ public class FingerprintFilter extends TokenFilter {
|
|||
}
|
||||
});
|
||||
|
||||
//TODO lets append directly to termAttribute?
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (Object item : items) {
|
||||
if (sb.length() >= 1) {
|
||||
|
|
|
@ -29,19 +29,21 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
|
|||
* <fieldType name="text_edgngrm" class="solr.TextField" positionIncrementGap="100">
|
||||
* <analyzer>
|
||||
* <tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
* <filter class="solr.EdgeNGramFilterFactory" minGramSize="1" maxGramSize="1"/>
|
||||
* <filter class="solr.EdgeNGramFilterFactory" minGramSize="1" maxGramSize="2" preserveOriginal="true"/>
|
||||
* </analyzer>
|
||||
* </fieldType></pre>
|
||||
*/
|
||||
public class EdgeNGramFilterFactory extends TokenFilterFactory {
|
||||
private final int maxGramSize;
|
||||
private final int minGramSize;
|
||||
private final boolean preserveOriginal;
|
||||
|
||||
/** Creates a new EdgeNGramFilterFactory */
|
||||
public EdgeNGramFilterFactory(Map<String, String> args) {
|
||||
super(args);
|
||||
minGramSize = getInt(args, "minGramSize", EdgeNGramTokenFilter.DEFAULT_MIN_GRAM_SIZE);
|
||||
maxGramSize = getInt(args, "maxGramSize", EdgeNGramTokenFilter.DEFAULT_MAX_GRAM_SIZE);
|
||||
minGramSize = requireInt(args, "minGramSize");
|
||||
maxGramSize = requireInt(args, "maxGramSize");
|
||||
preserveOriginal = getBoolean(args, "preserveOriginal", EdgeNGramTokenFilter.DEFAULT_PRESERVE_ORIGINAL);
|
||||
if (!args.isEmpty()) {
|
||||
throw new IllegalArgumentException("Unknown parameters: " + args);
|
||||
}
|
||||
|
@ -49,6 +51,6 @@ public class EdgeNGramFilterFactory extends TokenFilterFactory {
|
|||
|
||||
@Override
|
||||
public TokenFilter create(TokenStream input) {
|
||||
return new EdgeNGramTokenFilter(input, minGramSize, maxGramSize);
|
||||
return new EdgeNGramTokenFilter(input, minGramSize, maxGramSize, preserveOriginal);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,29 +32,36 @@ import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
|||
* supplementary characters.
|
||||
*/
|
||||
public final class EdgeNGramTokenFilter extends TokenFilter {
|
||||
public static final int DEFAULT_MAX_GRAM_SIZE = 1;
|
||||
public static final int DEFAULT_MIN_GRAM_SIZE = 1;
|
||||
public static final boolean DEFAULT_PRESERVE_ORIGINAL = false;
|
||||
|
||||
private final int minGram;
|
||||
private final int maxGram;
|
||||
private final boolean preserveOriginal;
|
||||
|
||||
private char[] curTermBuffer;
|
||||
private int curTermLength;
|
||||
private int curCodePointCount;
|
||||
private int curTermCodePointCount;
|
||||
private int curGramSize;
|
||||
private int savePosIncr;
|
||||
private int curPosIncr;
|
||||
private State state;
|
||||
|
||||
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
|
||||
private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
|
||||
|
||||
/**
|
||||
* Creates EdgeNGramTokenFilter that can generate n-grams in the sizes of the given range
|
||||
*
|
||||
* Creates an EdgeNGramTokenFilter that, for a given input term, produces all
|
||||
* edge n-grams with lengths >= minGram and <= maxGram. Will
|
||||
* optionally preserve the original term when its length is outside of the
|
||||
* defined range.
|
||||
*
|
||||
* @param input {@link TokenStream} holding the input to be tokenized
|
||||
* @param minGram the smallest n-gram to generate
|
||||
* @param maxGram the largest n-gram to generate
|
||||
* @param minGram the minimum length of the generated n-grams
|
||||
* @param maxGram the maximum length of the generated n-grams
|
||||
* @param preserveOriginal Whether or not to keep the original term when it
|
||||
* is outside the min/max size range.
|
||||
*/
|
||||
public EdgeNGramTokenFilter(TokenStream input, int minGram, int maxGram) {
|
||||
public EdgeNGramTokenFilter(
|
||||
TokenStream input, int minGram, int maxGram, boolean preserveOriginal) {
|
||||
super(input);
|
||||
|
||||
if (minGram < 1) {
|
||||
|
@ -67,6 +74,18 @@ public final class EdgeNGramTokenFilter extends TokenFilter {
|
|||
|
||||
this.minGram = minGram;
|
||||
this.maxGram = maxGram;
|
||||
this.preserveOriginal = preserveOriginal;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an EdgeNGramTokenFilter that produces edge n-grams of the given
|
||||
* size.
|
||||
*
|
||||
* @param input {@link TokenStream} holding the input to be tokenized
|
||||
* @param gramSize the n-gram size to generate.
|
||||
*/
|
||||
public EdgeNGramTokenFilter(TokenStream input, int gramSize) {
|
||||
this(input, gramSize, gramSize, DEFAULT_PRESERVE_ORIGINAL);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -75,32 +94,46 @@ public final class EdgeNGramTokenFilter extends TokenFilter {
|
|||
if (curTermBuffer == null) {
|
||||
if (!input.incrementToken()) {
|
||||
return false;
|
||||
} else {
|
||||
curTermBuffer = termAtt.buffer().clone();
|
||||
curTermLength = termAtt.length();
|
||||
curCodePointCount = Character.codePointCount(termAtt, 0, termAtt.length());
|
||||
curGramSize = minGram;
|
||||
state = captureState();
|
||||
savePosIncr += posIncrAtt.getPositionIncrement();
|
||||
}
|
||||
state = captureState();
|
||||
|
||||
curTermLength = termAtt.length();
|
||||
curTermCodePointCount = Character.codePointCount(termAtt, 0, curTermLength);
|
||||
curPosIncr += posIncrAtt.getPositionIncrement();
|
||||
|
||||
if (preserveOriginal && curTermCodePointCount < minGram) {
|
||||
// Token is shorter than minGram, but we'd still like to keep it.
|
||||
posIncrAtt.setPositionIncrement(curPosIncr);
|
||||
curPosIncr = 0;
|
||||
return true;
|
||||
}
|
||||
|
||||
curTermBuffer = termAtt.buffer().clone();
|
||||
curGramSize = minGram;
|
||||
}
|
||||
if (curGramSize <= maxGram) { // if we have hit the end of our n-gram size range, quit
|
||||
if (curGramSize <= curCodePointCount) { // if the remaining input is too short, we can't generate any n-grams
|
||||
// grab gramSize chars from front or back
|
||||
|
||||
if (curGramSize <= curTermCodePointCount) {
|
||||
if (curGramSize <= maxGram) { // curGramSize is between minGram and maxGram
|
||||
restoreState(state);
|
||||
// first ngram gets increment, others don't
|
||||
if (curGramSize == minGram) {
|
||||
posIncrAtt.setPositionIncrement(savePosIncr);
|
||||
savePosIncr = 0;
|
||||
} else {
|
||||
posIncrAtt.setPositionIncrement(0);
|
||||
}
|
||||
posIncrAtt.setPositionIncrement(curPosIncr);
|
||||
curPosIncr = 0;
|
||||
|
||||
final int charLength = Character.offsetByCodePoints(curTermBuffer, 0, curTermLength, 0, curGramSize);
|
||||
termAtt.copyBuffer(curTermBuffer, 0, charLength);
|
||||
curGramSize++;
|
||||
return true;
|
||||
}
|
||||
else if (preserveOriginal) {
|
||||
// Token is longer than maxGram, but we'd still like to keep it.
|
||||
restoreState(state);
|
||||
posIncrAtt.setPositionIncrement(0);
|
||||
termAtt.copyBuffer(curTermBuffer, 0, curTermLength);
|
||||
curTermBuffer = null;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
// Done with this input token, get next token on the next iteration.
|
||||
curTermBuffer = null;
|
||||
}
|
||||
}
|
||||
|
@ -109,6 +142,6 @@ public final class EdgeNGramTokenFilter extends TokenFilter {
|
|||
public void reset() throws IOException {
|
||||
super.reset();
|
||||
curTermBuffer = null;
|
||||
savePosIncr = 0;
|
||||
curPosIncr = 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,19 +29,21 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
|
|||
* <fieldType name="text_ngrm" class="solr.TextField" positionIncrementGap="100">
|
||||
* <analyzer>
|
||||
* <tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
* <filter class="solr.NGramFilterFactory" minGramSize="1" maxGramSize="2"/>
|
||||
* <filter class="solr.NGramFilterFactory" minGramSize="1" maxGramSize="2" preserveOriginal="true"/>
|
||||
* </analyzer>
|
||||
* </fieldType></pre>
|
||||
*/
|
||||
public class NGramFilterFactory extends TokenFilterFactory {
|
||||
private final int maxGramSize;
|
||||
private final int minGramSize;
|
||||
private final boolean preserveOriginal;
|
||||
|
||||
/** Creates a new NGramFilterFactory */
|
||||
public NGramFilterFactory(Map<String, String> args) {
|
||||
super(args);
|
||||
minGramSize = getInt(args, "minGramSize", NGramTokenFilter.DEFAULT_MIN_NGRAM_SIZE);
|
||||
maxGramSize = getInt(args, "maxGramSize", NGramTokenFilter.DEFAULT_MAX_NGRAM_SIZE);
|
||||
minGramSize = requireInt(args, "minGramSize");
|
||||
maxGramSize = requireInt(args, "maxGramSize");
|
||||
preserveOriginal = getBoolean(args, "keepShortTerm", NGramTokenFilter.DEFAULT_PRESERVE_ORIGINAL);
|
||||
if (!args.isEmpty()) {
|
||||
throw new IllegalArgumentException("Unknown parameters: " + args);
|
||||
}
|
||||
|
@ -49,6 +51,6 @@ public class NGramFilterFactory extends TokenFilterFactory {
|
|||
|
||||
@Override
|
||||
public TokenFilter create(TokenStream input) {
|
||||
return new NGramTokenFilter(input, minGramSize, maxGramSize);
|
||||
return new NGramTokenFilter(input, minGramSize, maxGramSize, preserveOriginal);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.miscellaneous.CodepointCountFilter;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
|
||||
|
@ -40,30 +39,41 @@ import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
|||
* override {@link NGramTokenizer#isTokenChar(int)} to perform pre-tokenization.
|
||||
*/
|
||||
public final class NGramTokenFilter extends TokenFilter {
|
||||
public static final int DEFAULT_MIN_NGRAM_SIZE = 1;
|
||||
public static final int DEFAULT_MAX_NGRAM_SIZE = 2;
|
||||
public static final boolean DEFAULT_PRESERVE_ORIGINAL = false;
|
||||
|
||||
private final int minGram, maxGram;
|
||||
private final int minGram;
|
||||
private final int maxGram;
|
||||
private final boolean preserveOriginal;
|
||||
|
||||
private char[] curTermBuffer;
|
||||
private int curTermLength;
|
||||
private int curCodePointCount;
|
||||
private int curTermCodePointCount;
|
||||
private int curGramSize;
|
||||
private int curPos;
|
||||
private int curPosInc;
|
||||
private int curPosIncr;
|
||||
private State state;
|
||||
|
||||
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
|
||||
private final PositionIncrementAttribute posIncAtt;
|
||||
private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
|
||||
|
||||
/**
|
||||
* Creates NGramTokenFilter with given min and max n-grams.
|
||||
* Creates an NGramTokenFilter that, for a given input term, produces all
|
||||
* contained n-grams with lengths >= minGram and <= maxGram. Will
|
||||
* optionally preserve the original term when its length is outside of the
|
||||
* defined range.
|
||||
*
|
||||
* Note: Care must be taken when choosing minGram and maxGram; depending
|
||||
* on the input token size, this filter potentially produces a huge number
|
||||
* of terms.
|
||||
*
|
||||
* @param input {@link TokenStream} holding the input to be tokenized
|
||||
* @param minGram the smallest n-gram to generate
|
||||
* @param maxGram the largest n-gram to generate
|
||||
* @param minGram the minimum length of the generated n-grams
|
||||
* @param maxGram the maximum length of the generated n-grams
|
||||
* @param preserveOriginal Whether or not to keep the original term when it
|
||||
* is shorter than minGram or longer than maxGram
|
||||
*/
|
||||
public NGramTokenFilter(TokenStream input, int minGram, int maxGram) {
|
||||
super(new CodepointCountFilter(input, minGram, Integer.MAX_VALUE));
|
||||
public NGramTokenFilter(TokenStream input, int minGram, int maxGram, boolean preserveOriginal) {
|
||||
super(input);
|
||||
if (minGram < 1) {
|
||||
throw new IllegalArgumentException("minGram must be greater than zero");
|
||||
}
|
||||
|
@ -72,51 +82,69 @@ public final class NGramTokenFilter extends TokenFilter {
|
|||
}
|
||||
this.minGram = minGram;
|
||||
this.maxGram = maxGram;
|
||||
|
||||
posIncAtt = addAttribute(PositionIncrementAttribute.class);
|
||||
this.preserveOriginal = preserveOriginal;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates NGramTokenFilter with default min and max n-grams.
|
||||
* Creates an NGramTokenFilter that produces n-grams of the indicated size.
|
||||
*
|
||||
* @param input {@link TokenStream} holding the input to be tokenized
|
||||
* @param gramSize the size of n-grams to generate.
|
||||
*/
|
||||
public NGramTokenFilter(TokenStream input) {
|
||||
this(input, DEFAULT_MIN_NGRAM_SIZE, DEFAULT_MAX_NGRAM_SIZE);
|
||||
public NGramTokenFilter(TokenStream input, int gramSize) {
|
||||
this(input, gramSize, gramSize, DEFAULT_PRESERVE_ORIGINAL);
|
||||
}
|
||||
|
||||
/** Returns the next token in the stream, or null at EOS. */
|
||||
@Override
|
||||
public final boolean incrementToken() throws IOException {
|
||||
while (true) {
|
||||
if (curTermBuffer == null) {
|
||||
if (!input.incrementToken()) {
|
||||
return false;
|
||||
} else {
|
||||
curTermBuffer = termAtt.buffer().clone();
|
||||
curTermLength = termAtt.length();
|
||||
curCodePointCount = Character.codePointCount(termAtt, 0, termAtt.length());
|
||||
curGramSize = minGram;
|
||||
curPos = 0;
|
||||
curPosInc = posIncAtt.getPositionIncrement();
|
||||
state = captureState();
|
||||
}
|
||||
state = captureState();
|
||||
|
||||
curTermLength = termAtt.length();
|
||||
curTermCodePointCount = Character.codePointCount(termAtt, 0, termAtt.length());
|
||||
curPosIncr += posIncrAtt.getPositionIncrement();
|
||||
curPos = 0;
|
||||
|
||||
if (preserveOriginal && curTermCodePointCount < minGram) {
|
||||
// Token is shorter than minGram, but we'd still like to keep it.
|
||||
posIncrAtt.setPositionIncrement(curPosIncr);
|
||||
curPosIncr = 0;
|
||||
return true;
|
||||
}
|
||||
|
||||
curTermBuffer = termAtt.buffer().clone();
|
||||
curGramSize = minGram;
|
||||
}
|
||||
|
||||
if (curGramSize > maxGram || (curPos + curGramSize) > curCodePointCount) {
|
||||
if (curGramSize > maxGram || (curPos + curGramSize) > curTermCodePointCount) {
|
||||
++curPos;
|
||||
curGramSize = minGram;
|
||||
}
|
||||
if ((curPos + curGramSize) <= curCodePointCount) {
|
||||
if ((curPos + curGramSize) <= curTermCodePointCount) {
|
||||
restoreState(state);
|
||||
final int start = Character.offsetByCodePoints(curTermBuffer, 0, curTermLength, 0, curPos);
|
||||
final int end = Character.offsetByCodePoints(curTermBuffer, 0, curTermLength, start, curGramSize);
|
||||
termAtt.copyBuffer(curTermBuffer, start, end - start);
|
||||
posIncAtt.setPositionIncrement(curPosInc);
|
||||
curPosInc = 0;
|
||||
posIncrAtt.setPositionIncrement(curPosIncr);
|
||||
curPosIncr = 0;
|
||||
curGramSize++;
|
||||
return true;
|
||||
}
|
||||
curTermBuffer = null;
|
||||
else if (preserveOriginal && curTermCodePointCount > maxGram) {
|
||||
// Token is longer than maxGram, but we'd still like to keep it.
|
||||
restoreState(state);
|
||||
posIncrAtt.setPositionIncrement(0);
|
||||
termAtt.copyBuffer(curTermBuffer, 0, curTermLength);
|
||||
curTermBuffer = null;
|
||||
return true;
|
||||
}
|
||||
|
||||
// Done with this input token, get next token on next iteration.
|
||||
curTermBuffer = null;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -124,5 +152,6 @@ public final class NGramTokenFilter extends TokenFilter {
|
|||
public void reset() throws IOException {
|
||||
super.reset();
|
||||
curTermBuffer = null;
|
||||
curPosIncr = 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -64,6 +64,7 @@ org.apache.lucene.analysis.minhash.MinHashFilterFactory
|
|||
org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.CapitalizationFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.CodepointCountFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.DateRecognizerFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.FingerprintFilterFactory
|
||||
|
|
|
@ -236,7 +236,7 @@ public class TestBugInSomething extends BaseTokenStreamTestCase {
|
|||
//TokenStream stream = new SopTokenFilter(tokenizer);
|
||||
TokenStream stream = new ShingleFilter(tokenizer, 5);
|
||||
//stream = new SopTokenFilter(stream);
|
||||
stream = new NGramTokenFilter(stream, 55, 83);
|
||||
stream = new NGramTokenFilter(stream, 55, 83, false);
|
||||
//stream = new SopTokenFilter(stream);
|
||||
return new TokenStreamComponents(tokenizer, stream);
|
||||
}
|
||||
|
|
|
@ -72,6 +72,7 @@ import org.apache.lucene.analysis.compound.hyphenation.HyphenationTree;
|
|||
import org.apache.lucene.analysis.hunspell.Dictionary;
|
||||
import org.apache.lucene.analysis.hunspell.TestHunspellStemFilter;
|
||||
import org.apache.lucene.analysis.minhash.MinHashFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.ConditionalTokenFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.FingerprintFilter;
|
||||
|
@ -119,10 +120,10 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
|
|||
|
||||
private static final Set<Class<?>> avoidConditionals = new HashSet<>();
|
||||
static {
|
||||
// Fingerprint filter needs to consume the whole tokenstream, so conditionals don't make sense here
|
||||
// These filters needs to consume the whole tokenstream, so conditionals don't make sense here
|
||||
avoidConditionals.add(FingerprintFilter.class);
|
||||
// Ditto MinHashFilter
|
||||
avoidConditionals.add(MinHashFilter.class);
|
||||
avoidConditionals.add(ConcatenateGraphFilter.class);
|
||||
}
|
||||
|
||||
private static final Map<Constructor<?>,Predicate<Object[]>> brokenConstructors = new HashMap<>();
|
||||
|
@ -156,7 +157,7 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
|
|||
return !((Boolean) args[2]); // args are broken if consumeAllTokens is false
|
||||
});
|
||||
for (Class<?> c : Arrays.<Class<?>>asList(
|
||||
// doesn't actual reset itself!
|
||||
// doesn't actual reset itself! TODO this statement is probably obsolete as of LUCENE-6121 ?
|
||||
CachingTokenFilter.class,
|
||||
// LUCENE-8092: doesn't handle graph inputs
|
||||
CJKBigramFilter.class,
|
||||
|
|
|
@ -14,50 +14,42 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.search.suggest.document;
|
||||
package org.apache.lucene.analysis.miscellaneous;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
|
||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.StopFilter;
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.analysis.synonym.SynonymFilter;
|
||||
import org.apache.lucene.analysis.synonym.SynonymMap;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.junit.Test;
|
||||
|
||||
public class CompletionTokenStreamTest extends BaseTokenStreamTestCase {
|
||||
public class TestConcatenateGraphFilter extends BaseTokenStreamTestCase {
|
||||
|
||||
private static final char SEP_LABEL = (char) ConcatenateGraphFilter.SEP_LABEL;
|
||||
|
||||
@Test
|
||||
public void testBasic() throws Exception {
|
||||
Tokenizer tokenStream = new MockTokenizer(MockTokenizer.WHITESPACE, true);
|
||||
String input = "mykeyword";
|
||||
BytesRef payload = new BytesRef("payload");
|
||||
tokenStream.setReader(new StringReader(input));
|
||||
CompletionTokenStream completionTokenStream = new CompletionTokenStream(tokenStream);
|
||||
completionTokenStream.setPayload(payload);
|
||||
PayloadAttrToTypeAttrFilter stream = new PayloadAttrToTypeAttrFilter(completionTokenStream);
|
||||
assertTokenStreamContents(stream, new String[] {input}, null, null, new String[] {payload.utf8ToString()}, new int[] { 1 }, null, null);
|
||||
ConcatenateGraphFilter stream = new ConcatenateGraphFilter(tokenStream);
|
||||
assertTokenStreamContents(stream, new String[] {input}, null, null, new int[] { 1 });
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithNoPreserveSep() throws Exception {
|
||||
Tokenizer tokenStream = new MockTokenizer(MockTokenizer.WHITESPACE, true);
|
||||
String input = "mykeyword another keyword";
|
||||
BytesRef payload = new BytesRef("payload");
|
||||
tokenStream.setReader(new StringReader(input));
|
||||
CompletionTokenStream completionTokenStream = new CompletionTokenStream(tokenStream, false, false, 100);
|
||||
completionTokenStream.setPayload(payload);
|
||||
PayloadAttrToTypeAttrFilter stream = new PayloadAttrToTypeAttrFilter(completionTokenStream);
|
||||
assertTokenStreamContents(stream, new String[] {"mykeywordanotherkeyword"}, null, null, new String[] {payload.utf8ToString()}, new int[] { 1 }, null, null);
|
||||
ConcatenateGraphFilter stream = new ConcatenateGraphFilter(tokenStream, false, false, 100);
|
||||
assertTokenStreamContents(stream, new String[] {"mykeywordanotherkeyword"}, null, null, new int[] { 1 });
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -65,17 +57,14 @@ public class CompletionTokenStreamTest extends BaseTokenStreamTestCase {
|
|||
Tokenizer tokenStream = new MockTokenizer(MockTokenizer.WHITESPACE, true);
|
||||
String input = "mykeyword another keyword";
|
||||
tokenStream.setReader(new StringReader(input));
|
||||
BytesRef payload = new BytesRef("payload");
|
||||
CompletionTokenStream completionTokenStream = new CompletionTokenStream(tokenStream);
|
||||
completionTokenStream.setPayload(payload);
|
||||
PayloadAttrToTypeAttrFilter stream = new PayloadAttrToTypeAttrFilter(completionTokenStream);
|
||||
ConcatenateGraphFilter stream = new ConcatenateGraphFilter(tokenStream);
|
||||
CharsRefBuilder builder = new CharsRefBuilder();
|
||||
builder.append("mykeyword");
|
||||
builder.append(((char) CompletionAnalyzer.SEP_LABEL));
|
||||
builder.append(SEP_LABEL);
|
||||
builder.append("another");
|
||||
builder.append(((char) CompletionAnalyzer.SEP_LABEL));
|
||||
builder.append(SEP_LABEL);
|
||||
builder.append("keyword");
|
||||
assertTokenStreamContents(stream, new String[]{builder.toCharsRef().toString()}, null, null, new String[]{payload.utf8ToString()}, new int[]{1}, null, null);
|
||||
assertTokenStreamContents(stream, new String[]{builder.toCharsRef().toString()}, null, null, new int[]{1});
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -85,11 +74,8 @@ public class CompletionTokenStreamTest extends BaseTokenStreamTestCase {
|
|||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, true);
|
||||
tokenizer.setReader(new StringReader("mykeyword"));
|
||||
SynonymFilter filter = new SynonymFilter(tokenizer, builder.build(), true);
|
||||
CompletionTokenStream completionTokenStream = new CompletionTokenStream(filter);
|
||||
BytesRef payload = new BytesRef("payload");
|
||||
completionTokenStream.setPayload(payload);
|
||||
PayloadAttrToTypeAttrFilter stream = new PayloadAttrToTypeAttrFilter(completionTokenStream);
|
||||
assertTokenStreamContents(stream, new String[] {"mykeyword", "mysynonym"}, null, null, new String[] {payload.utf8ToString(), payload.utf8ToString()}, new int[] { 1, 1 }, null, null);
|
||||
ConcatenateGraphFilter stream = new ConcatenateGraphFilter(filter);
|
||||
assertTokenStreamContents(stream, new String[] {"mykeyword", "mysynonym"}, null, null, new int[] { 1, 0 });
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -100,26 +86,48 @@ public class CompletionTokenStreamTest extends BaseTokenStreamTestCase {
|
|||
String input = "mykeyword another keyword";
|
||||
tokenStream.setReader(new StringReader(input));
|
||||
SynonymFilter filter = new SynonymFilter(tokenStream, builder.build(), true);
|
||||
BytesRef payload = new BytesRef("payload");
|
||||
CompletionTokenStream completionTokenStream = new CompletionTokenStream(filter, true, false, 100);
|
||||
completionTokenStream.setPayload(payload);
|
||||
PayloadAttrToTypeAttrFilter stream = new PayloadAttrToTypeAttrFilter(completionTokenStream);
|
||||
ConcatenateGraphFilter stream = new ConcatenateGraphFilter(filter, true, false, 100);
|
||||
String[] expectedOutputs = new String[2];
|
||||
CharsRefBuilder expectedOutput = new CharsRefBuilder();
|
||||
expectedOutput.append("mykeyword");
|
||||
expectedOutput.append(((char) CompletionAnalyzer.SEP_LABEL));
|
||||
expectedOutput.append(SEP_LABEL);
|
||||
expectedOutput.append("another");
|
||||
expectedOutput.append(((char) CompletionAnalyzer.SEP_LABEL));
|
||||
expectedOutput.append(SEP_LABEL);
|
||||
expectedOutput.append("keyword");
|
||||
expectedOutputs[0] = expectedOutput.toCharsRef().toString();
|
||||
expectedOutput.clear();
|
||||
expectedOutput.append("mysynonym");
|
||||
expectedOutput.append(((char) CompletionAnalyzer.SEP_LABEL));
|
||||
expectedOutput.append(SEP_LABEL);
|
||||
expectedOutput.append("another");
|
||||
expectedOutput.append(((char) CompletionAnalyzer.SEP_LABEL));
|
||||
expectedOutput.append(SEP_LABEL);
|
||||
expectedOutput.append("keyword");
|
||||
expectedOutputs[1] = expectedOutput.toCharsRef().toString();
|
||||
assertTokenStreamContents(stream, expectedOutputs, null, null, new String[]{payload.utf8ToString(), payload.utf8ToString()}, new int[]{1, 1}, null, null);
|
||||
assertTokenStreamContents(stream, expectedOutputs, null, null, new int[]{1, 0});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithStopword() throws Exception {
|
||||
for (boolean preservePosInc : new boolean[]{true, false}) {
|
||||
Tokenizer tokenStream = new MockTokenizer(MockTokenizer.WHITESPACE, true);
|
||||
String input = "a mykeyword a keyword"; //LUCENE-8344 add "a"
|
||||
tokenStream.setReader(new StringReader(input));
|
||||
TokenFilter tokenFilter = new StopFilter(tokenStream, StopFilter.makeStopSet("a"));
|
||||
ConcatenateGraphFilter concatStream = new ConcatenateGraphFilter(tokenFilter, true, preservePosInc, 10);
|
||||
CharsRefBuilder builder = new CharsRefBuilder();
|
||||
if (preservePosInc) {
|
||||
builder.append(SEP_LABEL);
|
||||
}
|
||||
builder.append("mykeyword");
|
||||
builder.append(SEP_LABEL);
|
||||
if (preservePosInc) {
|
||||
builder.append(SEP_LABEL);
|
||||
}
|
||||
builder.append("keyword");
|
||||
// if (preservePosInc) { LUCENE-8344 uncomment
|
||||
// builder.append(SEP_LABEL);
|
||||
// }
|
||||
assertTokenStreamContents(concatStream, new String[]{builder.toCharsRef().toString()});
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -137,41 +145,24 @@ public class CompletionTokenStreamTest extends BaseTokenStreamTestCase {
|
|||
tokenizer.setReader(new StringReader(valueBuilder.toString()));
|
||||
SynonymFilter filter = new SynonymFilter(tokenizer, builder.build(), true);
|
||||
|
||||
CompletionTokenStream completionTokenStream = new CompletionTokenStream(filter);
|
||||
completionTokenStream.setPayload(new BytesRef());
|
||||
PayloadAttrToTypeAttrFilter stream = new PayloadAttrToTypeAttrFilter(completionTokenStream);
|
||||
stream.reset();
|
||||
CompletionTokenStream.BytesRefBuilderTermAttribute attr = stream.addAttribute(CompletionTokenStream.BytesRefBuilderTermAttribute.class);
|
||||
PositionIncrementAttribute posAttr = stream.addAttribute(PositionIncrementAttribute.class);
|
||||
int maxPos = 0;
|
||||
int count = 0;
|
||||
while(stream.incrementToken()) {
|
||||
count++;
|
||||
assertNotNull(attr.getBytesRef());
|
||||
assertTrue(attr.getBytesRef().length > 0);
|
||||
maxPos += posAttr.getPositionIncrement();
|
||||
}
|
||||
stream.close();
|
||||
assertEquals(count, 256);
|
||||
assertEquals(count, maxPos);
|
||||
}
|
||||
|
||||
public final static class PayloadAttrToTypeAttrFilter extends TokenFilter {
|
||||
private PayloadAttribute payload = addAttribute(PayloadAttribute.class);
|
||||
private TypeAttribute type = addAttribute(TypeAttribute.class);
|
||||
|
||||
protected PayloadAttrToTypeAttrFilter(TokenStream input) {
|
||||
super(input);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
if (input.incrementToken()) {
|
||||
// we move them over so we can assert them more easily in the tests
|
||||
type.setType(payload.getPayload().utf8ToString());
|
||||
return true;
|
||||
int count;
|
||||
try (ConcatenateGraphFilter stream = new ConcatenateGraphFilter(filter)) {
|
||||
stream.reset();
|
||||
ConcatenateGraphFilter.BytesRefBuilderTermAttribute attr = stream.addAttribute(ConcatenateGraphFilter.BytesRefBuilderTermAttribute.class);
|
||||
count = 0;
|
||||
while (stream.incrementToken()) {
|
||||
count++;
|
||||
assertNotNull(attr.getBytesRef());
|
||||
assertTrue(attr.getBytesRef().length > 0);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
assertEquals(count, 256);
|
||||
}
|
||||
|
||||
public void testEmpty() throws IOException {
|
||||
Tokenizer tokenizer = whitespaceMockTokenizer("");
|
||||
ConcatenateGraphFilter filter = new ConcatenateGraphFilter(tokenizer);
|
||||
assertTokenStreamContents(filter, new String[0]);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,83 @@
|
|||
/*
|
||||
* 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.Reader;
|
||||
import java.io.StringReader;
|
||||
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.StopFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.util.BaseTokenStreamFactoryTestCase;
|
||||
|
||||
public class TestConcatenateGraphFilterFactory extends BaseTokenStreamFactoryTestCase {
|
||||
public void test() throws Exception {
|
||||
for (final boolean consumeAll : new boolean[]{true, false}) {
|
||||
final String input = "A1 B2 A1 D4 C3";
|
||||
Reader reader = new StringReader(input);
|
||||
MockTokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
tokenizer.setReader(reader);
|
||||
tokenizer.setEnableChecks(consumeAll);
|
||||
TokenStream stream = tokenizer;
|
||||
stream = tokenFilterFactory("ConcatenateGraph").create(stream);
|
||||
assertTokenStreamContents(stream, new String[]{input.replace(' ', (char) ConcatenateGraphFilter.SEP_LABEL)});
|
||||
}
|
||||
}
|
||||
|
||||
public void testPreserveSep() throws Exception {
|
||||
final String input = "A1 B2 A1 D4 C3";
|
||||
final String output = "A1A1D4C3";
|
||||
Reader reader = new StringReader(input);
|
||||
MockTokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
tokenizer.setReader(reader);
|
||||
TokenStream stream = tokenizer;
|
||||
stream = new StopFilter(stream, StopFilter.makeStopSet("B2"));
|
||||
stream = tokenFilterFactory("ConcatenateGraph",
|
||||
"preserveSep", "false"
|
||||
).create(stream);
|
||||
assertTokenStreamContents(stream, new String[]{output});
|
||||
}
|
||||
|
||||
public void testPreservePositionIncrements() throws Exception {
|
||||
final String input = "A1 B2 A1 D4 C3";
|
||||
final String output = "A1 A1 D4 C3";
|
||||
Reader reader = new StringReader(input);
|
||||
MockTokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
tokenizer.setReader(reader);
|
||||
TokenStream stream = tokenizer;
|
||||
stream = new StopFilter(stream, StopFilter.makeStopSet("B2"));
|
||||
stream = tokenFilterFactory("ConcatenateGraph",
|
||||
"preservePositionIncrements", "false"
|
||||
).create(stream);
|
||||
assertTokenStreamContents(stream, new String[]{output.replace(' ', (char) ConcatenateGraphFilter.SEP_LABEL)});
|
||||
}
|
||||
|
||||
public void testRequired() throws Exception {
|
||||
// no params are required
|
||||
tokenFilterFactory("ConcatenateGraph");
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that bogus arguments result in exception
|
||||
*/
|
||||
public void testBogusArguments() throws Exception {
|
||||
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () ->
|
||||
tokenFilterFactory("ConcatenateGraph", "bogusArg", "bogusValue"));
|
||||
assertTrue(expected.getMessage().contains("Unknown parameters"));
|
||||
}
|
||||
}
|
|
@ -69,4 +69,13 @@ public class TestFingerprintFilter extends BaseTokenStreamTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testEmpty() throws Exception {
|
||||
for (final boolean consumeAll : new boolean[] { true, false }) {
|
||||
MockTokenizer tokenizer = whitespaceMockTokenizer("");
|
||||
tokenizer.setEnableChecks(consumeAll);
|
||||
TokenStream stream = new FingerprintFilter(tokenizer);
|
||||
assertTokenStreamContents(stream, new String[0]);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -50,49 +50,73 @@ public class EdgeNGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
|
||||
public void testInvalidInput() throws Exception {
|
||||
expectThrows(IllegalArgumentException.class, () -> {
|
||||
new EdgeNGramTokenFilter(input, 0, 0);
|
||||
new EdgeNGramTokenFilter(input, 0, 0, false);
|
||||
});
|
||||
}
|
||||
|
||||
public void testInvalidInput2() throws Exception {
|
||||
expectThrows(IllegalArgumentException.class, () -> {
|
||||
new EdgeNGramTokenFilter(input, 2, 1);
|
||||
new EdgeNGramTokenFilter(input, 2, 1, false);
|
||||
});
|
||||
}
|
||||
|
||||
public void testInvalidInput3() throws Exception {
|
||||
expectThrows(IllegalArgumentException.class, () -> {
|
||||
new EdgeNGramTokenFilter(input, -1, 2);
|
||||
new EdgeNGramTokenFilter(input, -1, 2, false);
|
||||
});
|
||||
}
|
||||
|
||||
public void testFrontUnigram() throws Exception {
|
||||
EdgeNGramTokenFilter tokenizer = new EdgeNGramTokenFilter(input, 1, 1);
|
||||
EdgeNGramTokenFilter tokenizer = new EdgeNGramTokenFilter(input, 1, 1, false);
|
||||
assertTokenStreamContents(tokenizer, new String[]{"a"}, new int[]{0}, new int[]{5});
|
||||
}
|
||||
|
||||
public void testOversizedNgrams() throws Exception {
|
||||
EdgeNGramTokenFilter tokenizer = new EdgeNGramTokenFilter(input, 6, 6);
|
||||
EdgeNGramTokenFilter tokenizer = new EdgeNGramTokenFilter(input, 6, 6, false);
|
||||
assertTokenStreamContents(tokenizer, new String[0], new int[0], new int[0]);
|
||||
}
|
||||
|
||||
public void testOversizedNgramsPreserveOriginal() throws Exception {
|
||||
EdgeNGramTokenFilter tokenizer = new EdgeNGramTokenFilter(input, 6, 6, true);
|
||||
assertTokenStreamContents(tokenizer, new String[] {"abcde"}, new int[] {0}, new int[] {5});
|
||||
}
|
||||
|
||||
public void testPreserveOriginal() throws Exception {
|
||||
final String inputString = "a bcd efghi jk";
|
||||
|
||||
{ // preserveOriginal = false
|
||||
TokenStream ts = whitespaceMockTokenizer(inputString);
|
||||
EdgeNGramTokenFilter filter = new EdgeNGramTokenFilter(ts, 2, 3, false);
|
||||
assertTokenStreamContents(filter,
|
||||
new String[] { "bc", "bcd", "ef", "efg", "jk" },
|
||||
new int[] { 2, 2, 6, 6, 12 },
|
||||
new int[] { 5, 5, 11, 11, 14 },
|
||||
new int[] { 2, 0, 1, 0, 1 });
|
||||
}
|
||||
|
||||
{ // preserveOriginal = true
|
||||
TokenStream ts = whitespaceMockTokenizer(inputString);
|
||||
EdgeNGramTokenFilter filter = new EdgeNGramTokenFilter(ts, 2, 3, true);
|
||||
assertTokenStreamContents(filter,
|
||||
new String[] { "a", "bc", "bcd", "ef", "efg", "efghi", "jk" },
|
||||
new int[] { 0, 2, 2, 6, 6, 6, 12 },
|
||||
new int[] { 1, 5, 5, 11, 11, 11, 14 },
|
||||
new int[] { 1, 1, 0, 1, 0, 0, 1 });
|
||||
}
|
||||
}
|
||||
|
||||
public void testFrontRangeOfNgrams() throws Exception {
|
||||
EdgeNGramTokenFilter tokenizer = new EdgeNGramTokenFilter(input, 1, 3);
|
||||
EdgeNGramTokenFilter tokenizer = new EdgeNGramTokenFilter(input, 1, 3, false);
|
||||
assertTokenStreamContents(tokenizer, new String[]{"a","ab","abc"}, new int[]{0,0,0}, new int[]{5,5,5});
|
||||
}
|
||||
|
||||
public void testFilterPositions() throws Exception {
|
||||
TokenStream ts = whitespaceMockTokenizer("abcde vwxyz");
|
||||
EdgeNGramTokenFilter tokenizer = new EdgeNGramTokenFilter(ts, 1, 3);
|
||||
EdgeNGramTokenFilter tokenizer = new EdgeNGramTokenFilter(ts, 1, 3, false);
|
||||
assertTokenStreamContents(tokenizer,
|
||||
new String[]{"a","ab","abc","v","vw","vwx"},
|
||||
new int[]{0,0,0,6,6,6},
|
||||
new int[]{5,5,5,11,11,11},
|
||||
null,
|
||||
new int[]{1,0,0,1,0,0},
|
||||
null,
|
||||
null,
|
||||
false);
|
||||
new String[] {"a","ab","abc","v","vw","vwx"},
|
||||
new int[] {0, 0, 0, 6, 6, 6},
|
||||
new int[] {5, 5, 5, 11, 11, 11});
|
||||
}
|
||||
|
||||
private static class PositionFilter extends TokenFilter {
|
||||
|
@ -128,7 +152,7 @@ public class EdgeNGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
public void testFirstTokenPositionIncrement() throws Exception {
|
||||
TokenStream ts = whitespaceMockTokenizer("a abc");
|
||||
ts = new PositionFilter(ts); // All but first token will get 0 position increment
|
||||
EdgeNGramTokenFilter filter = new EdgeNGramTokenFilter(ts, 2, 3);
|
||||
EdgeNGramTokenFilter filter = new EdgeNGramTokenFilter(ts, 2, 3, false);
|
||||
// The first token "a" will not be output, since it's smaller than the mingram size of 2.
|
||||
// The second token on input to EdgeNGramTokenFilter will have position increment of 0,
|
||||
// which should be increased to 1, since this is the first output token in the stream.
|
||||
|
@ -142,14 +166,14 @@ public class EdgeNGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
|
||||
public void testSmallTokenInStream() throws Exception {
|
||||
input = whitespaceMockTokenizer("abc de fgh");
|
||||
EdgeNGramTokenFilter tokenizer = new EdgeNGramTokenFilter(input, 3, 3);
|
||||
EdgeNGramTokenFilter tokenizer = new EdgeNGramTokenFilter(input, 3, 3, false);
|
||||
assertTokenStreamContents(tokenizer, new String[]{"abc","fgh"}, new int[]{0,7}, new int[]{3,10});
|
||||
}
|
||||
|
||||
public void testReset() throws Exception {
|
||||
WhitespaceTokenizer tokenizer = new WhitespaceTokenizer();
|
||||
tokenizer.setReader(new StringReader("abcde"));
|
||||
EdgeNGramTokenFilter filter = new EdgeNGramTokenFilter(tokenizer, 1, 3);
|
||||
EdgeNGramTokenFilter filter = new EdgeNGramTokenFilter(tokenizer, 1, 3, false);
|
||||
assertTokenStreamContents(filter, new String[]{"a","ab","abc"}, new int[]{0,0,0}, new int[]{5,5,5});
|
||||
tokenizer.setReader(new StringReader("abcde"));
|
||||
assertTokenStreamContents(filter, new String[]{"a","ab","abc"}, new int[]{0,0,0}, new int[]{5,5,5});
|
||||
|
@ -160,13 +184,14 @@ public class EdgeNGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
for (int i = 0; i < 10; i++) {
|
||||
final int min = TestUtil.nextInt(random(), 2, 10);
|
||||
final int max = TestUtil.nextInt(random(), min, 20);
|
||||
final boolean preserveOriginal = TestUtil.nextInt(random(), 0, 1) % 2 == 0;
|
||||
|
||||
Analyzer a = new Analyzer() {
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
return new TokenStreamComponents(tokenizer,
|
||||
new EdgeNGramTokenFilter(tokenizer, min, max));
|
||||
new EdgeNGramTokenFilter(tokenizer, min, max, preserveOriginal));
|
||||
}
|
||||
};
|
||||
checkRandomData(random(), a, 100*RANDOM_MULTIPLIER);
|
||||
|
@ -181,7 +206,7 @@ public class EdgeNGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new KeywordTokenizer();
|
||||
return new TokenStreamComponents(tokenizer,
|
||||
new EdgeNGramTokenFilter(tokenizer, 2, 15));
|
||||
new EdgeNGramTokenFilter(tokenizer, 2, 15, false));
|
||||
}
|
||||
};
|
||||
checkAnalysisConsistency(random, a, random.nextBoolean(), "");
|
||||
|
@ -192,7 +217,7 @@ public class EdgeNGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
TokenStream tk = new LetterTokenizer();
|
||||
((Tokenizer)tk).setReader(new StringReader("abc d efgh ij klmno p q"));
|
||||
tk = new ShingleFilter(tk);
|
||||
tk = new EdgeNGramTokenFilter(tk, 7, 10);
|
||||
tk = new EdgeNGramTokenFilter(tk, 7, 10, false);
|
||||
assertTokenStreamContents(tk,
|
||||
new String[] { "efgh ij", "ij klmn", "ij klmno", "klmno p" },
|
||||
new int[] { 6,11,11,14 },
|
||||
|
@ -204,23 +229,44 @@ public class EdgeNGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
}
|
||||
|
||||
public void testSupplementaryCharacters() throws IOException {
|
||||
final String s = TestUtil.randomUnicodeString(random(), 10);
|
||||
final int codePointCount = s.codePointCount(0, s.length());
|
||||
final int minGram = TestUtil.nextInt(random(), 1, 3);
|
||||
final int maxGram = TestUtil.nextInt(random(), minGram, 10);
|
||||
TokenStream tk = new KeywordTokenizer();
|
||||
((Tokenizer)tk).setReader(new StringReader(s));
|
||||
tk = new EdgeNGramTokenFilter(tk, minGram, maxGram);
|
||||
final CharTermAttribute termAtt = tk.addAttribute(CharTermAttribute.class);
|
||||
final OffsetAttribute offsetAtt = tk.addAttribute(OffsetAttribute.class);
|
||||
tk.reset();
|
||||
for (int i = minGram; i <= Math.min(codePointCount, maxGram); ++i) {
|
||||
assertTrue(tk.incrementToken());
|
||||
assertEquals(0, offsetAtt.startOffset());
|
||||
assertEquals(s.length(), offsetAtt.endOffset());
|
||||
final int end = Character.offsetByCodePoints(s, 0, i);
|
||||
assertEquals(s.substring(0, end), termAtt.toString());
|
||||
for (int i = 0; i < 20; i++) {
|
||||
final String s = TestUtil.randomUnicodeString(random(), 10);
|
||||
final int codePointCount = s.codePointCount(0, s.length());
|
||||
final int minGram = TestUtil.nextInt(random(), 1, 3);
|
||||
final int maxGram = TestUtil.nextInt(random(), minGram, 10);
|
||||
final boolean preserveOriginal = TestUtil.nextInt(random(), 0, 1) % 2 == 0;
|
||||
|
||||
TokenStream tk = new KeywordTokenizer();
|
||||
((Tokenizer)tk).setReader(new StringReader(s));
|
||||
tk = new EdgeNGramTokenFilter(tk, minGram, maxGram, preserveOriginal);
|
||||
final CharTermAttribute termAtt = tk.addAttribute(CharTermAttribute.class);
|
||||
final OffsetAttribute offsetAtt = tk.addAttribute(OffsetAttribute.class);
|
||||
tk.reset();
|
||||
|
||||
if (codePointCount < minGram && preserveOriginal) {
|
||||
assertTrue(tk.incrementToken());
|
||||
assertEquals(0, offsetAtt.startOffset());
|
||||
assertEquals(s.length(), offsetAtt.endOffset());
|
||||
assertEquals(s, termAtt.toString());
|
||||
}
|
||||
|
||||
for (int j = minGram; j <= Math.min(codePointCount, maxGram); j++) {
|
||||
assertTrue(tk.incrementToken());
|
||||
assertEquals(0, offsetAtt.startOffset());
|
||||
assertEquals(s.length(), offsetAtt.endOffset());
|
||||
final int end = Character.offsetByCodePoints(s, 0, j);
|
||||
assertEquals(s.substring(0, end), termAtt.toString());
|
||||
}
|
||||
|
||||
if (codePointCount > maxGram && preserveOriginal) {
|
||||
assertTrue(tk.incrementToken());
|
||||
assertEquals(0, offsetAtt.startOffset());
|
||||
assertEquals(s.length(), offsetAtt.endOffset());
|
||||
assertEquals(s, termAtt.toString());
|
||||
}
|
||||
|
||||
assertFalse(tk.incrementToken());
|
||||
tk.close();
|
||||
}
|
||||
assertFalse(tk.incrementToken());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,28 +48,28 @@ public class NGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
|
||||
public void testInvalidInput() throws Exception {
|
||||
expectThrows(IllegalArgumentException.class, () -> {
|
||||
new NGramTokenFilter(input, 2, 1);
|
||||
new NGramTokenFilter(input, 2, 1, false);
|
||||
});
|
||||
}
|
||||
|
||||
public void testInvalidInput2() throws Exception {
|
||||
expectThrows(IllegalArgumentException.class, () -> {
|
||||
new NGramTokenFilter(input, 0, 1);
|
||||
new NGramTokenFilter(input, 0, 1, false);
|
||||
});
|
||||
}
|
||||
|
||||
public void testUnigrams() throws Exception {
|
||||
NGramTokenFilter filter = new NGramTokenFilter(input, 1, 1);
|
||||
NGramTokenFilter filter = new NGramTokenFilter(input, 1, 1, false);
|
||||
assertTokenStreamContents(filter, new String[]{"a","b","c","d","e"}, new int[]{0,0,0,0,0}, new int[]{5,5,5,5,5}, new int[]{1,0,0,0,0});
|
||||
}
|
||||
|
||||
public void testBigrams() throws Exception {
|
||||
NGramTokenFilter filter = new NGramTokenFilter(input, 2, 2);
|
||||
NGramTokenFilter filter = new NGramTokenFilter(input, 2, 2, false);
|
||||
assertTokenStreamContents(filter, new String[]{"ab","bc","cd","de"}, new int[]{0,0,0,0}, new int[]{5,5,5,5}, new int[]{1,0,0,0});
|
||||
}
|
||||
|
||||
public void testNgrams() throws Exception {
|
||||
NGramTokenFilter filter = new NGramTokenFilter(input, 1, 3);
|
||||
NGramTokenFilter filter = new NGramTokenFilter(input, 1, 3, false);
|
||||
assertTokenStreamContents(filter,
|
||||
new String[]{"a","ab","abc","b","bc","bcd","c","cd","cde","d","de","e"},
|
||||
new int[]{0,0,0,0,0,0,0,0,0,0,0,0},
|
||||
|
@ -81,7 +81,7 @@ public class NGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
}
|
||||
|
||||
public void testNgramsNoIncrement() throws Exception {
|
||||
NGramTokenFilter filter = new NGramTokenFilter(input, 1, 3);
|
||||
NGramTokenFilter filter = new NGramTokenFilter(input, 1, 3, false);
|
||||
assertTokenStreamContents(filter,
|
||||
new String[]{"a","ab","abc","b","bc","bcd","c","cd","cde","d","de","e"},
|
||||
new int[]{0,0,0,0,0,0,0,0,0,0,0,0},
|
||||
|
@ -93,25 +93,61 @@ public class NGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
}
|
||||
|
||||
public void testOversizedNgrams() throws Exception {
|
||||
NGramTokenFilter filter = new NGramTokenFilter(input, 6, 7);
|
||||
NGramTokenFilter filter = new NGramTokenFilter(input, 6, 7, false);
|
||||
assertTokenStreamContents(filter, new String[0], new int[0], new int[0]);
|
||||
}
|
||||
|
||||
public void testOversizedNgramsPreserveOriginal() throws Exception {
|
||||
NGramTokenFilter tokenizer = new NGramTokenFilter(input, 6, 6, true);
|
||||
assertTokenStreamContents(tokenizer, new String[] {"abcde"}, new int[] {0}, new int[] {5});
|
||||
}
|
||||
|
||||
public void testSmallTokenInStream() throws Exception {
|
||||
input = whitespaceMockTokenizer("abc de fgh");
|
||||
NGramTokenFilter filter = new NGramTokenFilter(input, 3, 3);
|
||||
assertTokenStreamContents(filter, new String[]{"abc","fgh"}, new int[]{0,7}, new int[]{3,10}, new int[] {1, 2});
|
||||
NGramTokenFilter tokenizer = new NGramTokenFilter(input, 3, 3, false);
|
||||
assertTokenStreamContents(tokenizer, new String[]{"abc","fgh"}, new int[]{0,7}, new int[]{3,10}, new int[] {1, 2});
|
||||
}
|
||||
|
||||
public void testSmallTokenInStreamPreserveOriginal() throws Exception {
|
||||
input = whitespaceMockTokenizer("abc de fgh");
|
||||
NGramTokenFilter tokenizer = new NGramTokenFilter(input, 3, 3, true);
|
||||
assertTokenStreamContents(tokenizer, new String[]{"abc","de","fgh"}, new int[]{0,4,7}, new int[]{3,6,10}, new int[] {1, 1, 1});
|
||||
|
||||
}
|
||||
|
||||
public void testReset() throws Exception {
|
||||
WhitespaceTokenizer tokenizer = new WhitespaceTokenizer();
|
||||
tokenizer.setReader(new StringReader("abcde"));
|
||||
NGramTokenFilter filter = new NGramTokenFilter(tokenizer, 1, 1);
|
||||
NGramTokenFilter filter = new NGramTokenFilter(tokenizer, 1, 1, false);
|
||||
assertTokenStreamContents(filter, new String[]{"a","b","c","d","e"}, new int[]{0,0,0,0,0}, new int[]{5,5,5,5,5}, new int[]{1,0,0,0,0});
|
||||
tokenizer.setReader(new StringReader("abcde"));
|
||||
assertTokenStreamContents(filter, new String[]{"a","b","c","d","e"}, new int[]{0,0,0,0,0}, new int[]{5,5,5,5,5}, new int[]{1,0,0,0,0});
|
||||
}
|
||||
|
||||
public void testKeepShortTermKeepLongTerm() throws Exception {
|
||||
final String inputString = "a bcd efghi jk";
|
||||
|
||||
{ // preserveOriginal = false
|
||||
TokenStream ts = whitespaceMockTokenizer(inputString);
|
||||
NGramTokenFilter filter = new NGramTokenFilter(ts, 2, 3, false);
|
||||
assertTokenStreamContents(filter,
|
||||
new String[] { "bc", "bcd", "cd", "ef", "efg", "fg", "fgh", "gh", "ghi", "hi", "jk" },
|
||||
new int[] { 2, 2, 2, 6, 6, 6, 6, 6, 6, 6, 12 },
|
||||
new int[] { 5, 5, 5, 11, 11, 11, 11, 11, 11, 11, 14 },
|
||||
new int[] { 2, 0, 0, 1, 0, 0, 0, 0, 0, 0, 1 });
|
||||
}
|
||||
|
||||
{ // preserveOriginal = true
|
||||
TokenStream ts = whitespaceMockTokenizer(inputString);
|
||||
NGramTokenFilter filter = new NGramTokenFilter(ts, 2, 3, true);
|
||||
assertTokenStreamContents(filter,
|
||||
new String[] { "a", "bc", "bcd", "cd", "ef", "efg", "fg", "fgh", "gh", "ghi", "hi", "efghi", "jk" },
|
||||
new int[] { 0, 2, 2, 2, 6, 6, 6, 6, 6, 6, 6, 6, 12 },
|
||||
new int[] { 1, 5, 5, 5, 11, 11, 11, 11, 11, 11, 11, 11, 14 },
|
||||
new int[] { 1, 1, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 1 });
|
||||
}
|
||||
}
|
||||
|
||||
// LUCENE-3642
|
||||
// EdgeNgram blindly adds term length to offset, but this can take things out of bounds
|
||||
// wrt original text if a previous filter increases the length of the word (in this case æ -> ae)
|
||||
|
@ -122,7 +158,7 @@ public class NGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
TokenFilter filters = new ASCIIFoldingFilter(tokenizer);
|
||||
filters = new NGramTokenFilter(filters, 2, 2);
|
||||
filters = new NGramTokenFilter(filters, 2, 2, false);
|
||||
return new TokenStreamComponents(tokenizer, filters);
|
||||
}
|
||||
};
|
||||
|
@ -139,12 +175,14 @@ public class NGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
for (int i = 0; i < 10; i++) {
|
||||
final int min = TestUtil.nextInt(random(), 2, 10);
|
||||
final int max = TestUtil.nextInt(random(), min, 20);
|
||||
final boolean preserveOriginal = TestUtil.nextInt(random(), 0, 1) % 2 == 0;
|
||||
|
||||
Analyzer a = new Analyzer() {
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
return new TokenStreamComponents(tokenizer,
|
||||
new NGramTokenFilter(tokenizer, min, max));
|
||||
new NGramTokenFilter(tokenizer, min, max, preserveOriginal));
|
||||
}
|
||||
};
|
||||
checkRandomData(random(), a, 200*RANDOM_MULTIPLIER, 20);
|
||||
|
@ -159,7 +197,7 @@ public class NGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new KeywordTokenizer();
|
||||
return new TokenStreamComponents(tokenizer,
|
||||
new NGramTokenFilter(tokenizer, 2, 15));
|
||||
new NGramTokenFilter(tokenizer, 2, 15, false));
|
||||
}
|
||||
};
|
||||
checkAnalysisConsistency(random, a, random.nextBoolean(), "");
|
||||
|
@ -167,27 +205,47 @@ public class NGramTokenFilterTest extends BaseTokenStreamTestCase {
|
|||
}
|
||||
|
||||
public void testSupplementaryCharacters() throws IOException {
|
||||
final String s = TestUtil.randomUnicodeString(random(), 10);
|
||||
final int codePointCount = s.codePointCount(0, s.length());
|
||||
final int minGram = TestUtil.nextInt(random(), 1, 3);
|
||||
final int maxGram = TestUtil.nextInt(random(), minGram, 10);
|
||||
TokenStream tk = new KeywordTokenizer();
|
||||
((Tokenizer)tk).setReader(new StringReader(s));
|
||||
tk = new NGramTokenFilter(tk, minGram, maxGram);
|
||||
final CharTermAttribute termAtt = tk.addAttribute(CharTermAttribute.class);
|
||||
final OffsetAttribute offsetAtt = tk.addAttribute(OffsetAttribute.class);
|
||||
tk.reset();
|
||||
for (int start = 0; start < codePointCount; ++start) {
|
||||
for (int end = start + minGram; end <= Math.min(codePointCount, start + maxGram); ++end) {
|
||||
for (int i = 0; i < 20; i++) {
|
||||
final String s = TestUtil.randomUnicodeString(random(), 10);
|
||||
final int codePointCount = s.codePointCount(0, s.length());
|
||||
final int minGram = TestUtil.nextInt(random(), 1, 3);
|
||||
final int maxGram = TestUtil.nextInt(random(), minGram, 10);
|
||||
final boolean preserveOriginal = TestUtil.nextInt(random(), 0, 1) % 2 == 0;
|
||||
|
||||
TokenStream tk = new KeywordTokenizer();
|
||||
((Tokenizer)tk).setReader(new StringReader(s));
|
||||
tk = new NGramTokenFilter(tk, minGram, maxGram, preserveOriginal);
|
||||
final CharTermAttribute termAtt = tk.addAttribute(CharTermAttribute.class);
|
||||
final OffsetAttribute offsetAtt = tk.addAttribute(OffsetAttribute.class);
|
||||
tk.reset();
|
||||
|
||||
if (codePointCount < minGram && preserveOriginal) {
|
||||
assertTrue(tk.incrementToken());
|
||||
assertEquals(0, offsetAtt.startOffset());
|
||||
assertEquals(s.length(), offsetAtt.endOffset());
|
||||
final int startIndex = Character.offsetByCodePoints(s, 0, start);
|
||||
final int endIndex = Character.offsetByCodePoints(s, 0, end);
|
||||
assertEquals(s.substring(startIndex, endIndex), termAtt.toString());
|
||||
assertEquals(s, termAtt.toString());
|
||||
}
|
||||
|
||||
for (int start = 0; start < codePointCount; ++start) {
|
||||
for (int end = start + minGram; end <= Math.min(codePointCount, start + maxGram); ++end) {
|
||||
assertTrue(tk.incrementToken());
|
||||
assertEquals(0, offsetAtt.startOffset());
|
||||
assertEquals(s.length(), offsetAtt.endOffset());
|
||||
final int startIndex = Character.offsetByCodePoints(s, 0, start);
|
||||
final int endIndex = Character.offsetByCodePoints(s, 0, end);
|
||||
assertEquals(s.substring(startIndex, endIndex), termAtt.toString());
|
||||
}
|
||||
}
|
||||
|
||||
if (codePointCount > maxGram && preserveOriginal) {
|
||||
assertTrue(tk.incrementToken());
|
||||
assertEquals(0, offsetAtt.startOffset());
|
||||
assertEquals(s.length(), offsetAtt.endOffset());
|
||||
assertEquals(s, termAtt.toString());
|
||||
}
|
||||
|
||||
assertFalse(tk.incrementToken());
|
||||
tk.close();
|
||||
}
|
||||
assertFalse(tk.incrementToken());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -56,12 +56,14 @@ public class TestNGramFilters extends BaseTokenStreamFactoryTestCase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Test the NGramFilterFactory
|
||||
* Test the NGramFilterFactory with old defaults
|
||||
*/
|
||||
public void testNGramFilter() throws Exception {
|
||||
Reader reader = new StringReader("test");
|
||||
TokenStream stream = whitespaceMockTokenizer(reader);
|
||||
stream = tokenFilterFactory("NGram").create(stream);
|
||||
stream = tokenFilterFactory("NGram",
|
||||
"minGramSize", "1",
|
||||
"maxGramSize", "2").create(stream);
|
||||
assertTokenStreamContents(stream,
|
||||
new String[] { "t", "te", "e", "es", "s", "st", "t" });
|
||||
}
|
||||
|
@ -126,12 +128,13 @@ public class TestNGramFilters extends BaseTokenStreamFactoryTestCase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Test EdgeNGramFilterFactory
|
||||
* Test EdgeNGramFilterFactory with old defaults
|
||||
*/
|
||||
public void testEdgeNGramFilter() throws Exception {
|
||||
Reader reader = new StringReader("test");
|
||||
TokenStream stream = whitespaceMockTokenizer(reader);
|
||||
stream = tokenFilterFactory("EdgeNGram").create(stream);
|
||||
stream = tokenFilterFactory("EdgeNGram", "minGramSize", "1",
|
||||
"maxGramSize", "1").create(stream);
|
||||
assertTokenStreamContents(stream,
|
||||
new String[] { "t" });
|
||||
}
|
||||
|
@ -173,7 +176,8 @@ public class TestNGramFilters extends BaseTokenStreamFactoryTestCase {
|
|||
|
||||
/** Test that bogus arguments result in exception */
|
||||
public void testBogusArguments() throws Exception {
|
||||
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
|
||||
IllegalArgumentException expected = null;
|
||||
expected = expectThrows(IllegalArgumentException.class, () -> {
|
||||
tokenizerFactory("NGram", "bogusArg", "bogusValue");
|
||||
});
|
||||
assertTrue(expected.getMessage().contains("Unknown parameters"));
|
||||
|
@ -184,12 +188,12 @@ public class TestNGramFilters extends BaseTokenStreamFactoryTestCase {
|
|||
assertTrue(expected.getMessage().contains("Unknown parameters"));
|
||||
|
||||
expected = expectThrows(IllegalArgumentException.class, () -> {
|
||||
tokenFilterFactory("NGram", "bogusArg", "bogusValue");
|
||||
tokenFilterFactory("NGram", "minGramSize", "2", "maxGramSize", "5", "bogusArg", "bogusValue");
|
||||
});
|
||||
assertTrue(expected.getMessage().contains("Unknown parameters"));
|
||||
|
||||
expected = expectThrows(IllegalArgumentException.class, () -> {
|
||||
tokenFilterFactory("EdgeNGram", "bogusArg", "bogusValue");
|
||||
tokenFilterFactory("EdgeNGram", "minGramSize", "2", "maxGramSize", "5", "bogusArg", "bogusValue");
|
||||
});
|
||||
assertTrue(expected.getMessage().contains("Unknown parameters"));
|
||||
}
|
||||
|
|
|
@ -87,7 +87,7 @@ public class BM25NBClassifierTest extends ClassificationTestBase<BytesRef> {
|
|||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
final Tokenizer tokenizer = new KeywordTokenizer();
|
||||
return new TokenStreamComponents(tokenizer, new ReverseStringFilter(new EdgeNGramTokenFilter(new ReverseStringFilter(tokenizer), 10, 20)));
|
||||
return new TokenStreamComponents(tokenizer, new ReverseStringFilter(new EdgeNGramTokenFilter(new ReverseStringFilter(tokenizer), 10, 20, false)));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -86,7 +86,7 @@ public class CachingNaiveBayesClassifierTest extends ClassificationTestBase<Byte
|
|||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
final Tokenizer tokenizer = new KeywordTokenizer();
|
||||
return new TokenStreamComponents(tokenizer, new ReverseStringFilter(new EdgeNGramTokenFilter(new ReverseStringFilter(tokenizer), 10, 20)));
|
||||
return new TokenStreamComponents(tokenizer, new ReverseStringFilter(new EdgeNGramTokenFilter(new ReverseStringFilter(tokenizer), 10, 20, false)));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -89,7 +89,7 @@ public class SimpleNaiveBayesClassifierTest extends ClassificationTestBase<Bytes
|
|||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
final Tokenizer tokenizer = new KeywordTokenizer();
|
||||
return new TokenStreamComponents(tokenizer, new ReverseStringFilter(new EdgeNGramTokenFilter(new ReverseStringFilter(tokenizer), 10, 20)));
|
||||
return new TokenStreamComponents(tokenizer, new ReverseStringFilter(new EdgeNGramTokenFilter(new ReverseStringFilter(tokenizer), 10, 20, false)));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -66,6 +66,7 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
|
|||
static final BytesRef ATT_VALUE = new BytesRef(" value ");
|
||||
static final BytesRef DIM_COUNT = new BytesRef(" dimensional count ");
|
||||
static final BytesRef DIM_NUM_BYTES = new BytesRef(" dimensional num bytes ");
|
||||
static final BytesRef SOFT_DELETES = new BytesRef(" soft-deletes ");
|
||||
|
||||
@Override
|
||||
public FieldInfos read(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, IOContext iocontext) throws IOException {
|
||||
|
@ -140,9 +141,13 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
|
|||
assert StringHelper.startsWith(scratch.get(), DIM_NUM_BYTES);
|
||||
int dimensionalNumBytes = Integer.parseInt(readString(DIM_NUM_BYTES.length, scratch));
|
||||
|
||||
SimpleTextUtil.readLine(input, scratch);
|
||||
assert StringHelper.startsWith(scratch.get(), SOFT_DELETES);
|
||||
boolean isSoftDeletesField = Boolean.parseBoolean(readString(SOFT_DELETES.length, scratch));
|
||||
|
||||
infos[i] = new FieldInfo(name, fieldNumber, storeTermVector,
|
||||
omitNorms, storePayloads, indexOptions, docValuesType, dvGen, Collections.unmodifiableMap(atts),
|
||||
dimensionalCount, dimensionalNumBytes);
|
||||
dimensionalCount, dimensionalNumBytes, isSoftDeletesField);
|
||||
}
|
||||
|
||||
SimpleTextUtil.checkFooter(input);
|
||||
|
@ -238,6 +243,10 @@ public class SimpleTextFieldInfosFormat extends FieldInfosFormat {
|
|||
SimpleTextUtil.write(out, DIM_NUM_BYTES);
|
||||
SimpleTextUtil.write(out, Integer.toString(fi.getPointNumBytes()), scratch);
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
|
||||
SimpleTextUtil.write(out, SOFT_DELETES);
|
||||
SimpleTextUtil.write(out, Boolean.toString(fi.isSoftDeletesField()), scratch);
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
}
|
||||
SimpleTextUtil.writeChecksum(out, scratch);
|
||||
success = true;
|
||||
|
|
|
@ -148,7 +148,7 @@ public final class Lucene50FieldInfosFormat extends FieldInfosFormat {
|
|||
lastAttributes = attributes;
|
||||
try {
|
||||
infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads,
|
||||
indexOptions, docValuesType, dvGen, attributes, 0, 0);
|
||||
indexOptions, docValuesType, dvGen, attributes, 0, 0, false);
|
||||
infos[i].checkConsistency();
|
||||
} catch (IllegalStateException e) {
|
||||
throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
|
||||
|
|
|
@ -136,6 +136,7 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
|
|||
boolean storeTermVector = (bits & STORE_TERMVECTOR) != 0;
|
||||
boolean omitNorms = (bits & OMIT_NORMS) != 0;
|
||||
boolean storePayloads = (bits & STORE_PAYLOADS) != 0;
|
||||
boolean isSoftDeletesField = (bits & SOFT_DELETES_FIELD) != 0;
|
||||
|
||||
final IndexOptions indexOptions = getIndexOptions(input, input.readByte());
|
||||
|
||||
|
@ -159,7 +160,7 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
|
|||
try {
|
||||
infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads,
|
||||
indexOptions, docValuesType, dvGen, attributes,
|
||||
pointDimensionCount, pointNumBytes);
|
||||
pointDimensionCount, pointNumBytes, isSoftDeletesField);
|
||||
infos[i].checkConsistency();
|
||||
} catch (IllegalStateException e) {
|
||||
throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
|
||||
|
@ -277,6 +278,7 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
|
|||
if (fi.hasVectors()) bits |= STORE_TERMVECTOR;
|
||||
if (fi.omitsNorms()) bits |= OMIT_NORMS;
|
||||
if (fi.hasPayloads()) bits |= STORE_PAYLOADS;
|
||||
if (fi.isSoftDeletesField()) bits |= SOFT_DELETES_FIELD;
|
||||
output.writeByte(bits);
|
||||
|
||||
output.writeByte(indexOptionsByte(fi.getIndexOptions()));
|
||||
|
@ -301,10 +303,12 @@ public final class Lucene60FieldInfosFormat extends FieldInfosFormat {
|
|||
// Codec header
|
||||
static final String CODEC_NAME = "Lucene60FieldInfos";
|
||||
static final int FORMAT_START = 0;
|
||||
static final int FORMAT_CURRENT = FORMAT_START;
|
||||
static final int FORMAT_SOFT_DELETES = 1;
|
||||
static final int FORMAT_CURRENT = FORMAT_SOFT_DELETES;
|
||||
|
||||
// Field flags
|
||||
static final byte STORE_TERMVECTOR = 0x1;
|
||||
static final byte OMIT_NORMS = 0x2;
|
||||
static final byte STORE_PAYLOADS = 0x4;
|
||||
static final byte SOFT_DELETES_FIELD = 0x8;
|
||||
}
|
||||
|
|
|
@ -200,4 +200,9 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
return value.get();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
DocIdSetIterator getDocIdSet() {
|
||||
return docsWithField.iterator();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -259,10 +259,10 @@ final class BufferedUpdatesStream implements Accountable {
|
|||
|
||||
SegmentState(ReadersAndUpdates rld, IOUtils.IOConsumer<ReadersAndUpdates> onClose, SegmentCommitInfo info) throws IOException {
|
||||
this.rld = rld;
|
||||
reader = rld.getReader(IOContext.READ);
|
||||
startDelCount = rld.getDelCount();
|
||||
delGen = info.getBufferedDeletesGen();
|
||||
this.onClose = onClose;
|
||||
reader = rld.getReader(IOContext.READ);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -46,6 +46,7 @@ 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.DocValuesFieldExistsQuery;
|
||||
import org.apache.lucene.search.LeafFieldComparator;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
@ -411,7 +412,7 @@ public final class CheckIndex implements Closeable {
|
|||
* that would otherwise be more complicated to debug if they had to close the writer
|
||||
* for each check.
|
||||
*/
|
||||
public CheckIndex(Directory dir, Lock writeLock) throws IOException {
|
||||
public CheckIndex(Directory dir, Lock writeLock) {
|
||||
this.dir = dir;
|
||||
this.writeLock = writeLock;
|
||||
this.infoStream = null;
|
||||
|
@ -781,7 +782,10 @@ public final class CheckIndex implements Closeable {
|
|||
throw new RuntimeException("Points test failed");
|
||||
}
|
||||
}
|
||||
|
||||
final String softDeletesField = reader.getFieldInfos().getSoftDeletesField();
|
||||
if (softDeletesField != null) {
|
||||
checkSoftDeletes(softDeletesField, info, reader, infoStream, failFast);
|
||||
}
|
||||
msg(infoStream, "");
|
||||
|
||||
if (verbose) {
|
||||
|
@ -3049,6 +3053,25 @@ public final class CheckIndex implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
private static void checkSoftDeletes(String softDeletesField, SegmentCommitInfo info, SegmentReader reader, PrintStream infoStream, boolean failFast) throws IOException {
|
||||
if (infoStream != null)
|
||||
infoStream.print(" test: check soft deletes.....");
|
||||
try {
|
||||
int softDeletes = PendingSoftDeletes.countSoftDeletes(DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(softDeletesField, reader), reader.getLiveDocs());
|
||||
if (softDeletes != info.getSoftDelCount()) {
|
||||
throw new RuntimeException("actual soft deletes: " + softDeletes + " but expected: " +info.getSoftDelCount());
|
||||
}
|
||||
} catch (Exception e) {
|
||||
if (failFast) {
|
||||
throw IOUtils.rethrowAlways(e);
|
||||
}
|
||||
msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
|
||||
if (infoStream != null) {
|
||||
e.printStackTrace(infoStream);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static double nsToSec(long ns) {
|
||||
return ns/1000000000.0;
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.codecs.NormsProducer;
|
|||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
|
@ -841,4 +842,19 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
DocIdSetIterator getHasDocValues(String field) {
|
||||
PerField perField = getPerField(field);
|
||||
if (perField != null) {
|
||||
if (perField.docValuesWriter != null) {
|
||||
if (perField.fieldInfo.getDocValuesType() == DocValuesType.NONE) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return perField.docValuesWriter.getDocIdSet();
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,8 +19,17 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
|
||||
abstract class DocConsumer {
|
||||
abstract void processDocument() throws IOException;
|
||||
abstract Sorter.DocMap flush(final SegmentWriteState state) throws IOException;
|
||||
abstract void abort() throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a {@link DocIdSetIterator} for the given field or null if the field doesn't have
|
||||
* doc values.
|
||||
*/
|
||||
abstract DocIdSetIterator getHasDocValues(String field);
|
||||
|
||||
}
|
||||
|
|
|
@ -20,10 +20,13 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
||||
abstract class DocValuesWriter {
|
||||
abstract void finish(int numDoc);
|
||||
abstract void flush(SegmentWriteState state, Sorter.DocMap sortMap, DocValuesConsumer consumer) throws IOException;
|
||||
abstract Sorter.DocComparator getDocComparator(int numDoc, SortField sortField) throws IOException;
|
||||
abstract DocIdSetIterator getDocIdSet();
|
||||
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FlushInfo;
|
||||
|
@ -460,14 +461,27 @@ final class DocumentsWriterPerThread {
|
|||
}
|
||||
final Sorter.DocMap sortMap;
|
||||
try {
|
||||
DocIdSetIterator softDeletedDocs;
|
||||
if (indexWriterConfig.getSoftDeletesField() != null) {
|
||||
softDeletedDocs = consumer.getHasDocValues(indexWriterConfig.getSoftDeletesField());
|
||||
} else {
|
||||
softDeletedDocs = null;
|
||||
}
|
||||
sortMap = consumer.flush(flushState);
|
||||
if (softDeletedDocs == null) {
|
||||
flushState.softDelCountOnFlush = 0;
|
||||
} else {
|
||||
flushState.softDelCountOnFlush = PendingSoftDeletes.countSoftDeletes(softDeletedDocs, flushState.liveDocs);
|
||||
assert flushState.segmentInfo.maxDoc() >= flushState.softDelCountOnFlush + flushState.delCountOnFlush;
|
||||
}
|
||||
// We clear this here because we already resolved them (private to this segment) when writing postings:
|
||||
pendingUpdates.clearDeleteTerms();
|
||||
segmentInfo.setFiles(new HashSet<>(directory.getCreatedFiles()));
|
||||
|
||||
final SegmentCommitInfo segmentInfoPerCommit = new SegmentCommitInfo(segmentInfo, 0, -1L, -1L, -1L);
|
||||
final SegmentCommitInfo segmentInfoPerCommit = new SegmentCommitInfo(segmentInfo, 0, flushState.softDelCountOnFlush, -1L, -1L, -1L);
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : flushState.delCountOnFlush) + " deleted docs");
|
||||
infoStream.message("DWPT", "new segment has " + flushState.softDelCountOnFlush + " soft-deleted docs");
|
||||
infoStream.message("DWPT", "new segment has " +
|
||||
(flushState.fieldInfos.hasVectors() ? "vectors" : "no vectors") + "; " +
|
||||
(flushState.fieldInfos.hasNorms() ? "norms" : "no norms") + "; " +
|
||||
|
@ -497,8 +511,7 @@ final class DocumentsWriterPerThread {
|
|||
assert segmentInfo != null;
|
||||
|
||||
FlushedSegment fs = new FlushedSegment(infoStream, segmentInfoPerCommit, flushState.fieldInfos,
|
||||
segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush,
|
||||
sortMap);
|
||||
segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush, sortMap);
|
||||
sealFlushedSegment(fs, sortMap, flushNotifications);
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "flush time " + ((System.nanoTime() - t0) / 1000000.0) + " msec");
|
||||
|
|
|
@ -53,14 +53,17 @@ public final class FieldInfo {
|
|||
private int pointDimensionCount;
|
||||
private int pointNumBytes;
|
||||
|
||||
// whether this field is used as the soft-deletes field
|
||||
private final boolean softDeletesField;
|
||||
|
||||
/**
|
||||
* Sole constructor.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public FieldInfo(String name, int number, boolean storeTermVector, boolean omitNorms,
|
||||
boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues,
|
||||
long dvGen, Map<String,String> attributes, int pointDimensionCount, int pointNumBytes) {
|
||||
public FieldInfo(String name, int number, boolean storeTermVector, boolean omitNorms, boolean storePayloads,
|
||||
IndexOptions indexOptions, DocValuesType docValues, long dvGen, Map<String,String> attributes,
|
||||
int pointDimensionCount, int pointNumBytes, boolean softDeletesField) {
|
||||
this.name = Objects.requireNonNull(name);
|
||||
this.number = number;
|
||||
this.docValuesType = Objects.requireNonNull(docValues, "DocValuesType must not be null (field: \"" + name + "\")");
|
||||
|
@ -78,6 +81,7 @@ public final class FieldInfo {
|
|||
this.attributes = Objects.requireNonNull(attributes);
|
||||
this.pointDimensionCount = pointDimensionCount;
|
||||
this.pointNumBytes = pointNumBytes;
|
||||
this.softDeletesField = softDeletesField;
|
||||
assert checkConsistency();
|
||||
}
|
||||
|
||||
|
@ -332,4 +336,12 @@ public final class FieldInfo {
|
|||
public Map<String,String> attributes() {
|
||||
return attributes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if this field is configured and used as the soft-deletes field.
|
||||
* See {@link IndexWriterConfig#softDeletesField}
|
||||
*/
|
||||
public boolean isSoftDeletesField() {
|
||||
return softDeletesField;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -43,6 +43,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
private final boolean hasNorms;
|
||||
private final boolean hasDocValues;
|
||||
private final boolean hasPointValues;
|
||||
private final String softDeletesField;
|
||||
|
||||
// used only by fieldInfo(int)
|
||||
private final FieldInfo[] byNumber;
|
||||
|
@ -62,6 +63,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
boolean hasNorms = false;
|
||||
boolean hasDocValues = false;
|
||||
boolean hasPointValues = false;
|
||||
String softDeletesField = null;
|
||||
|
||||
int size = 0; // number of elements in byNumberTemp, number of used array slots
|
||||
FieldInfo[] byNumberTemp = new FieldInfo[10]; // initial array capacity of 10
|
||||
|
@ -92,6 +94,12 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
hasDocValues |= info.getDocValuesType() != DocValuesType.NONE;
|
||||
hasPayloads |= info.hasPayloads();
|
||||
hasPointValues |= (info.getPointDimensionCount() != 0);
|
||||
if (info.isSoftDeletesField()) {
|
||||
if (softDeletesField != null && softDeletesField.equals(info.name) == false) {
|
||||
throw new IllegalArgumentException("multiple soft-deletes fields [" + info.name + ", " + softDeletesField + "]");
|
||||
}
|
||||
softDeletesField = info.name;
|
||||
}
|
||||
}
|
||||
|
||||
this.hasVectors = hasVectors;
|
||||
|
@ -102,6 +110,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
this.hasNorms = hasNorms;
|
||||
this.hasDocValues = hasDocValues;
|
||||
this.hasPointValues = hasPointValues;
|
||||
this.softDeletesField = softDeletesField;
|
||||
|
||||
List<FieldInfo> valuesTemp = new ArrayList<>();
|
||||
byNumber = new FieldInfo[size];
|
||||
|
@ -153,6 +162,11 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
public boolean hasPointValues() {
|
||||
return hasPointValues;
|
||||
}
|
||||
|
||||
/** Returns the soft-deletes field name if exists; otherwise returns null */
|
||||
public String getSoftDeletesField() {
|
||||
return softDeletesField;
|
||||
}
|
||||
|
||||
/** Returns the number of fields */
|
||||
public int size() {
|
||||
|
@ -221,13 +235,17 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
// norms back on after they were already ommitted; today
|
||||
// we silently discard the norm but this is badly trappy
|
||||
private int lowestUnassignedFieldNumber = -1;
|
||||
|
||||
// The soft-deletes field from IWC to enforce a single soft-deletes field
|
||||
private final String softDeletesFieldName;
|
||||
|
||||
FieldNumbers() {
|
||||
FieldNumbers(String softDeletesFieldName) {
|
||||
this.nameToNumber = new HashMap<>();
|
||||
this.numberToName = new HashMap<>();
|
||||
this.indexOptions = new HashMap<>();
|
||||
this.docValuesType = new HashMap<>();
|
||||
this.dimensions = new HashMap<>();
|
||||
this.softDeletesFieldName = softDeletesFieldName;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -236,7 +254,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
* number assigned if possible otherwise the first unassigned field number
|
||||
* is used as the field number.
|
||||
*/
|
||||
synchronized int addOrGet(String fieldName, int preferredFieldNumber, IndexOptions indexOptions, DocValuesType dvType, int dimensionCount, int dimensionNumBytes) {
|
||||
synchronized int addOrGet(String fieldName, int preferredFieldNumber, IndexOptions indexOptions, DocValuesType dvType, int dimensionCount, int dimensionNumBytes, boolean isSoftDeletesField) {
|
||||
if (indexOptions != IndexOptions.NONE) {
|
||||
IndexOptions currentOpts = this.indexOptions.get(fieldName);
|
||||
if (currentOpts == null) {
|
||||
|
@ -284,6 +302,16 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
nameToNumber.put(fieldName, fieldNumber);
|
||||
}
|
||||
|
||||
if (isSoftDeletesField) {
|
||||
if (softDeletesFieldName == null) {
|
||||
throw new IllegalArgumentException("this index has [" + fieldName + "] as soft-deletes already but soft-deletes field is not configured in IWC");
|
||||
} else if (fieldName.equals(softDeletesFieldName) == false) {
|
||||
throw new IllegalArgumentException("cannot configure [" + softDeletesFieldName + "] as soft-deletes; this index uses [" + fieldName + "] as soft-deletes already");
|
||||
}
|
||||
} else if (fieldName.equals(softDeletesFieldName)) {
|
||||
throw new IllegalArgumentException("cannot configure [" + softDeletesFieldName + "] as soft-deletes; this index uses [" + fieldName + "] as non-soft-deletes already");
|
||||
}
|
||||
|
||||
return fieldNumber.intValue();
|
||||
}
|
||||
|
||||
|
@ -383,11 +411,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
private final HashMap<String,FieldInfo> byName = new HashMap<>();
|
||||
final FieldNumbers globalFieldNumbers;
|
||||
private boolean finished;
|
||||
|
||||
Builder() {
|
||||
this(new FieldNumbers());
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a new instance with the given {@link FieldNumbers}.
|
||||
*/
|
||||
|
@ -413,8 +437,9 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
// number for this field. If the field was seen
|
||||
// before then we'll get the same name and number,
|
||||
// else we'll allocate a new one:
|
||||
final int fieldNumber = globalFieldNumbers.addOrGet(name, -1, IndexOptions.NONE, DocValuesType.NONE, 0, 0);
|
||||
fi = new FieldInfo(name, fieldNumber, false, false, false, IndexOptions.NONE, DocValuesType.NONE, -1, new HashMap<>(), 0, 0);
|
||||
final boolean isSoftDeletesField = name.equals(globalFieldNumbers.softDeletesFieldName);
|
||||
final int fieldNumber = globalFieldNumbers.addOrGet(name, -1, IndexOptions.NONE, DocValuesType.NONE, 0, 0, isSoftDeletesField);
|
||||
fi = new FieldInfo(name, fieldNumber, false, false, false, IndexOptions.NONE, DocValuesType.NONE, -1, new HashMap<>(), 0, 0, isSoftDeletesField);
|
||||
assert !byName.containsKey(fi.name);
|
||||
globalFieldNumbers.verifyConsistent(Integer.valueOf(fi.number), fi.name, DocValuesType.NONE);
|
||||
byName.put(fi.name, fi);
|
||||
|
@ -427,7 +452,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
boolean storeTermVector,
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions,
|
||||
DocValuesType docValues, long dvGen,
|
||||
int dimensionCount, int dimensionNumBytes) {
|
||||
int dimensionCount, int dimensionNumBytes, boolean isSoftDeletesField) {
|
||||
assert assertNotFinished();
|
||||
if (docValues == null) {
|
||||
throw new NullPointerException("DocValuesType must not be null");
|
||||
|
@ -439,8 +464,8 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
// number for this field. If the field was seen
|
||||
// before then we'll get the same name and number,
|
||||
// else we'll allocate a new one:
|
||||
final int fieldNumber = globalFieldNumbers.addOrGet(name, preferredFieldNumber, indexOptions, docValues, dimensionCount, dimensionNumBytes);
|
||||
fi = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, dvGen, new HashMap<>(), dimensionCount, dimensionNumBytes);
|
||||
final int fieldNumber = globalFieldNumbers.addOrGet(name, preferredFieldNumber, indexOptions, docValues, dimensionCount, dimensionNumBytes, isSoftDeletesField);
|
||||
fi = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, dvGen, new HashMap<>(), dimensionCount, dimensionNumBytes, isSoftDeletesField);
|
||||
assert !byName.containsKey(fi.name);
|
||||
globalFieldNumbers.verifyConsistent(Integer.valueOf(fi.number), fi.name, fi.getDocValuesType());
|
||||
byName.put(fi.name, fi);
|
||||
|
@ -473,7 +498,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
return addOrUpdateInternal(fi.name, fi.number, fi.hasVectors(),
|
||||
fi.omitsNorms(), fi.hasPayloads(),
|
||||
fi.getIndexOptions(), fi.getDocValuesType(), dvGen,
|
||||
fi.getPointDimensionCount(), fi.getPointNumBytes());
|
||||
fi.getPointDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField());
|
||||
}
|
||||
|
||||
public FieldInfo fieldInfo(String fieldName) {
|
||||
|
|
|
@ -49,6 +49,7 @@ import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
|||
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
||||
import org.apache.lucene.index.FieldInfos.FieldNumbers;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.search.DocValuesFieldExistsQuery;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Sort;
|
||||
|
@ -347,6 +348,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
* much like how hotels place an "authorization hold" on your credit
|
||||
* card to make sure they can later charge you when you check out. */
|
||||
final AtomicLong pendingNumDocs = new AtomicLong();
|
||||
final boolean softDeletesEnabled;
|
||||
|
||||
private final DocumentsWriter.FlushNotifications flushNotifications = new DocumentsWriter.FlushNotifications() {
|
||||
@Override
|
||||
|
@ -639,7 +641,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
if (rld != null) {
|
||||
return rld.getDelCount(); // get the full count from here since SCI might change concurrently
|
||||
} else {
|
||||
int delCount = info.getDelCount();
|
||||
final int delCount = info.getDelCount(softDeletesEnabled);
|
||||
assert delCount <= info.info.maxDoc(): "delCount: " + delCount + " maxDoc: " + info.info.maxDoc();
|
||||
return delCount;
|
||||
}
|
||||
|
@ -703,7 +705,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
conf.setIndexWriter(this); // prevent reuse by other instances
|
||||
config = conf;
|
||||
infoStream = config.getInfoStream();
|
||||
|
||||
softDeletesEnabled = config.getSoftDeletesField() != null;
|
||||
// obtain the write.lock. If the user configured a timeout,
|
||||
// we wrap with a sleeper and this might take some time.
|
||||
writeLock = d.obtainLock(WRITE_LOCK_NAME);
|
||||
|
@ -960,12 +962,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
* If this {@link SegmentInfos} has no global field number map the returned instance is empty
|
||||
*/
|
||||
private FieldNumbers getFieldNumberMap() throws IOException {
|
||||
final FieldNumbers map = new FieldNumbers();
|
||||
final FieldNumbers map = new FieldNumbers(config.softDeletesField);
|
||||
|
||||
for(SegmentCommitInfo info : segmentInfos) {
|
||||
FieldInfos fis = readFieldInfos(info);
|
||||
for(FieldInfo fi : fis) {
|
||||
map.addOrGet(fi.name, fi.number, fi.getIndexOptions(), fi.getDocValuesType(), fi.getPointDimensionCount(), fi.getPointNumBytes());
|
||||
map.addOrGet(fi.name, fi.number, fi.getIndexOptions(), fi.getDocValuesType(), fi.getPointDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1154,7 +1156,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
if (docWriter.anyDeletions()) {
|
||||
return true;
|
||||
}
|
||||
if (readerPool.anyPendingDeletes()) {
|
||||
if (readerPool.anyDeletions()) {
|
||||
return true;
|
||||
}
|
||||
for (final SegmentCommitInfo info : segmentInfos) {
|
||||
|
@ -1787,7 +1789,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
if (globalFieldNumberMap.contains(f.name(), dvType) == false) {
|
||||
// if this field doesn't exists we try to add it. if it exists and the DV type doesn't match we
|
||||
// get a consistent error message as if you try to do that during an indexing operation.
|
||||
globalFieldNumberMap.addOrGet(f.name(), -1, IndexOptions.NONE, dvType, 0, 0);
|
||||
globalFieldNumberMap.addOrGet(f.name(), -1, IndexOptions.NONE, dvType, 0, 0, f.name().equals(config.softDeletesField));
|
||||
assert globalFieldNumberMap.contains(f.name(), dvType);
|
||||
}
|
||||
if (config.getIndexSortFields().contains(f.name())) {
|
||||
|
@ -2824,7 +2826,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
FieldInfos fis = readFieldInfos(info);
|
||||
for(FieldInfo fi : fis) {
|
||||
// This will throw exceptions if any of the incoming fields have an illegal schema change:
|
||||
globalFieldNumberMap.addOrGet(fi.name, fi.number, fi.getIndexOptions(), fi.getDocValuesType(), fi.getPointDimensionCount(), fi.getPointNumBytes());
|
||||
globalFieldNumberMap.addOrGet(fi.name, fi.number, fi.getIndexOptions(), fi.getDocValuesType(), fi.getPointDimensionCount(), fi.getPointNumBytes(), fi.isSoftDeletesField());
|
||||
}
|
||||
infos.add(copySegmentAsIs(info, newSegName, context));
|
||||
}
|
||||
|
@ -2939,11 +2941,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
|
||||
// long so we can detect int overflow:
|
||||
long numDocs = 0;
|
||||
|
||||
Sort indexSort = config.getIndexSort();
|
||||
|
||||
long seqNo;
|
||||
|
||||
try {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "flush at addIndexes(CodecReader...)");
|
||||
|
@ -2951,10 +2949,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
flush(false, true);
|
||||
|
||||
String mergedName = newSegmentName();
|
||||
|
||||
int numSoftDeleted = 0;
|
||||
for (CodecReader leaf : readers) {
|
||||
numDocs += leaf.numDocs();
|
||||
validateMergeReader(leaf);
|
||||
if (softDeletesEnabled) {
|
||||
Bits liveDocs = leaf.getLiveDocs();
|
||||
numSoftDeleted += PendingSoftDeletes.countSoftDeletes(
|
||||
DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(config.getSoftDeletesField(), leaf), liveDocs);
|
||||
}
|
||||
}
|
||||
|
||||
// Best-effort up front check:
|
||||
|
@ -2979,8 +2982,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
}
|
||||
|
||||
merger.merge(); // merge 'em
|
||||
|
||||
SegmentCommitInfo infoPerCommit = new SegmentCommitInfo(info, 0, -1L, -1L, -1L);
|
||||
SegmentCommitInfo infoPerCommit = new SegmentCommitInfo(info, 0, numSoftDeleted, -1L, -1L, -1L);
|
||||
|
||||
info.setFiles(new HashSet<>(trackingDir.getCreatedFiles()));
|
||||
trackingDir.clearCreatedFiles();
|
||||
|
@ -3057,7 +3059,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
SegmentInfo newInfo = new SegmentInfo(directoryOrig, info.info.getVersion(), info.info.getMinVersion(), segName, info.info.maxDoc(),
|
||||
info.info.getUseCompoundFile(), info.info.getCodec(),
|
||||
info.info.getDiagnostics(), info.info.getId(), info.info.getAttributes(), info.info.getIndexSort());
|
||||
SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getDelGen(),
|
||||
SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getSoftDelCount(), info.getDelGen(),
|
||||
info.getFieldInfosGen(), info.getDocValuesGen());
|
||||
|
||||
newInfo.setFiles(info.info.files());
|
||||
|
@ -4249,7 +4251,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
details.put("mergeMaxNumSegments", "" + merge.maxNumSegments);
|
||||
details.put("mergeFactor", Integer.toString(merge.segments.size()));
|
||||
setDiagnostics(si, SOURCE_MERGE, details);
|
||||
merge.setMergeInfo(new SegmentCommitInfo(si, 0, -1L, -1L, -1L));
|
||||
merge.setMergeInfo(new SegmentCommitInfo(si, 0, 0, -1L, -1L, -1L));
|
||||
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merge seg=" + merge.info.info.name + " " + segString(merge.segments));
|
||||
|
@ -4373,16 +4375,25 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
|
||||
// Let the merge wrap readers
|
||||
List<CodecReader> mergeReaders = new ArrayList<>();
|
||||
int numSoftDeleted = 0;
|
||||
for (SegmentReader reader : merge.readers) {
|
||||
CodecReader wrappedReader = merge.wrapForMerge(reader);
|
||||
validateMergeReader(wrappedReader);
|
||||
mergeReaders.add(wrappedReader);
|
||||
if (softDeletesEnabled) {
|
||||
if (reader != wrappedReader) { // if we don't have a wrapped reader we won't preserve any soft-deletes
|
||||
Bits liveDocs = wrappedReader.getLiveDocs();
|
||||
numSoftDeleted += PendingSoftDeletes.countSoftDeletes(
|
||||
DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(config.getSoftDeletesField(), wrappedReader),
|
||||
liveDocs);
|
||||
}
|
||||
}
|
||||
}
|
||||
final SegmentMerger merger = new SegmentMerger(mergeReaders,
|
||||
merge.info.info, infoStream, dirWrapper,
|
||||
globalFieldNumberMap,
|
||||
context);
|
||||
|
||||
merge.info.setSoftDelCount(numSoftDeleted);
|
||||
merge.checkAborted();
|
||||
|
||||
merge.mergeStartNS = System.nanoTime();
|
||||
|
@ -4604,7 +4615,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
*
|
||||
* @lucene.internal */
|
||||
private synchronized String segString(SegmentCommitInfo info) {
|
||||
return info.toString(numDeletedDocs(info) - info.getDelCount());
|
||||
return info.toString(numDeletedDocs(info) - info.getDelCount(softDeletesEnabled));
|
||||
}
|
||||
|
||||
private synchronized void doWait() {
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.HashSet;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -263,7 +264,10 @@ public final class MultiFields extends Fields {
|
|||
* will be unavailable.
|
||||
*/
|
||||
public static FieldInfos getMergedFieldInfos(IndexReader reader) {
|
||||
final FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
final String softDeletesField = reader.leaves().stream()
|
||||
.map(l -> l.reader().getFieldInfos().getSoftDeletesField())
|
||||
.filter(Objects::nonNull).findAny().orElse(null);
|
||||
final FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(softDeletesField));
|
||||
for(final LeafReaderContext ctx : reader.leaves()) {
|
||||
builder.add(ctx.reader().getFieldInfos());
|
||||
}
|
||||
|
|
|
@ -82,6 +82,11 @@ class NumericDocValuesWriter extends DocValuesWriter {
|
|||
return Sorter.getDocComparator(maxDoc, sortField, () -> null, () -> docValues);
|
||||
}
|
||||
|
||||
@Override
|
||||
DocIdSetIterator getDocIdSet() {
|
||||
return docsWithField.iterator();
|
||||
}
|
||||
|
||||
static SortingLeafReader.CachedNumericDVs sortDocValues(int maxDoc, Sorter.DocMap sortMap, NumericDocValues oldDocValues) throws IOException {
|
||||
FixedBitSet docsWithField = new FixedBitSet(maxDoc);
|
||||
long[] values = new long[maxDoc];
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.HashMap;
|
|||
import java.util.IdentityHashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
@ -101,9 +102,11 @@ public class ParallelLeafReader extends LeafReader {
|
|||
throw new IllegalArgumentException("All readers must have same maxDoc: "+maxDoc+"!="+reader.maxDoc());
|
||||
}
|
||||
}
|
||||
|
||||
final String softDeletesField = completeReaderSet.stream()
|
||||
.map(r -> r.getFieldInfos().getSoftDeletesField())
|
||||
.filter(Objects::nonNull).findAny().orElse(null);
|
||||
// TODO: make this read-only in a cleaner way?
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(softDeletesField));
|
||||
|
||||
Sort indexSort = null;
|
||||
int createdVersionMajor = -1;
|
||||
|
|
|
@ -220,7 +220,7 @@ class PendingDeletes {
|
|||
* Returns <code>true</code> iff the segment represented by this {@link PendingDeletes} is fully deleted
|
||||
*/
|
||||
boolean isFullyDeleted(IOSupplier<CodecReader> readerIOSupplier) throws IOException {
|
||||
return info.getDelCount() + numPendingDeletes() == info.info.maxDoc();
|
||||
return getDelCount() == info.info.maxDoc();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -246,7 +246,8 @@ class PendingDeletes {
|
|||
* Returns the number of deleted docs in the segment.
|
||||
*/
|
||||
final int getDelCount() {
|
||||
return info.getDelCount() + numPendingDeletes();
|
||||
int delCount = info.getDelCount() + info.getSoftDelCount() + numPendingDeletes();
|
||||
return delCount;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -270,7 +271,8 @@ class PendingDeletes {
|
|||
count = info.info.maxDoc();
|
||||
}
|
||||
assert numDocs() == count: "info.maxDoc=" + info.info.maxDoc() + " info.getDelCount()=" + info.getDelCount() +
|
||||
" pendingDeletes=" + toString() + " count=" + count;
|
||||
" info.getSoftDelCount()=" + info.getSoftDelCount() +
|
||||
" pendingDeletes=" + toString() + " count=" + count + " numDocs: " + numDocs();
|
||||
assert reader.numDocs() == numDocs() : "reader.numDocs() = " + reader.numDocs() + " numDocs() " + numDocs();
|
||||
assert reader.numDeletedDocs() <= info.info.maxDoc(): "delCount=" + reader.numDeletedDocs() + " info.maxDoc=" +
|
||||
info.info.maxDoc() + " rld.pendingDeleteCount=" + numPendingDeletes() +
|
||||
|
|
|
@ -58,7 +58,7 @@ final class PendingSoftDeletes extends PendingDeletes {
|
|||
} else {
|
||||
// if it was deleted subtract the delCount
|
||||
pendingDeleteCount--;
|
||||
assert pendingDeleteCount >= 0 : " illegal pending delete count: " + pendingDeleteCount;
|
||||
assert assertPendingDeletes();
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
@ -76,11 +76,15 @@ final class PendingSoftDeletes extends PendingDeletes {
|
|||
hardDeletes.onNewReader(reader, info);
|
||||
if (dvGeneration < info.getDocValuesGen()) { // only re-calculate this if we haven't seen this generation
|
||||
final DocIdSetIterator iterator = DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(field, reader);
|
||||
int newDelCount;
|
||||
if (iterator != null) { // nothing is deleted we don't have a soft deletes field in this segment
|
||||
assert info.info.maxDoc() > 0 : "maxDoc is 0";
|
||||
pendingDeleteCount += applySoftDeletes(iterator, getMutableBits());
|
||||
assert pendingDeleteCount >= 0 : " illegal pending delete count: " + pendingDeleteCount;
|
||||
newDelCount = applySoftDeletes(iterator, getMutableBits());
|
||||
assert newDelCount >= 0 : " illegal pending delete count: " + newDelCount;
|
||||
} else {
|
||||
newDelCount = 0;
|
||||
}
|
||||
assert info.getSoftDelCount() == newDelCount : "softDeleteCount doesn't match " + info.getSoftDelCount() + " != " + newDelCount;
|
||||
dvGeneration = info.getDocValuesGen();
|
||||
}
|
||||
assert getDelCount() <= info.info.maxDoc() : getDelCount() + " > " + info.info.maxDoc();
|
||||
|
@ -88,8 +92,15 @@ final class PendingSoftDeletes extends PendingDeletes {
|
|||
|
||||
@Override
|
||||
boolean writeLiveDocs(Directory dir) throws IOException {
|
||||
// we need to set this here to make sure our stats in SCI are up-to-date otherwise we might hit an assertion
|
||||
// when the hard deletes are set since we need to account for docs that used to be only soft-delete but now hard-deleted
|
||||
this.info.setSoftDelCount(this.info.getSoftDelCount() + pendingDeleteCount);
|
||||
super.dropChanges();
|
||||
// delegate the write to the hard deletes - it will only write if somebody used it.
|
||||
return hardDeletes.writeLiveDocs(dir);
|
||||
if (hardDeletes.writeLiveDocs(dir)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -134,13 +145,21 @@ final class PendingSoftDeletes extends PendingDeletes {
|
|||
void onDocValuesUpdate(FieldInfo info, DocValuesFieldUpdates.Iterator iterator) throws IOException {
|
||||
if (this.field.equals(info.name)) {
|
||||
pendingDeleteCount += applySoftDeletes(iterator, getMutableBits());
|
||||
assert pendingDeleteCount >= 0 : " illegal pending delete count: " + pendingDeleteCount;
|
||||
assert assertPendingDeletes();
|
||||
assert dvGeneration < info.getDocValuesGen() : "we have seen this generation update already: " + dvGeneration + " vs. " + info.getDocValuesGen();
|
||||
assert dvGeneration != -2 : "docValues generation is still uninitialized";
|
||||
dvGeneration = info.getDocValuesGen();
|
||||
this.info.setSoftDelCount(this.info.getSoftDelCount() + pendingDeleteCount);
|
||||
super.dropChanges();
|
||||
}
|
||||
}
|
||||
|
||||
private boolean assertPendingDeletes() {
|
||||
assert pendingDeleteCount + info.getSoftDelCount() >= 0 : " illegal pending delete count: " + pendingDeleteCount + info.getSoftDelCount();
|
||||
assert info.info.maxDoc() >= getDelCount();
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
|
@ -210,4 +229,17 @@ final class PendingSoftDeletes extends PendingDeletes {
|
|||
Bits getHardLiveDocs() {
|
||||
return hardDeletes.getLiveDocs();
|
||||
}
|
||||
|
||||
static int countSoftDeletes(DocIdSetIterator softDeletedDocs, Bits hardDeletes) throws IOException {
|
||||
int count = 0;
|
||||
if (softDeletedDocs != null) {
|
||||
int doc;
|
||||
while ((doc = softDeletedDocs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
if (hardDeletes == null || hardDeletes.get(doc)) {
|
||||
count++;
|
||||
}
|
||||
}
|
||||
}
|
||||
return count;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -130,9 +130,9 @@ final class ReaderPool implements Closeable {
|
|||
/**
|
||||
* Returns <code>true</code> iff any of the buffered readers and updates has at least one pending delete
|
||||
*/
|
||||
synchronized boolean anyPendingDeletes() {
|
||||
synchronized boolean anyDeletions() {
|
||||
for(ReadersAndUpdates rld : readerMap.values()) {
|
||||
if (rld.anyPendingDeletes()) {
|
||||
if (rld.getDelCount() > 0) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -406,10 +406,6 @@ final class ReadersAndUpdates {
|
|||
}
|
||||
}
|
||||
|
||||
synchronized boolean anyPendingDeletes() {
|
||||
return pendingDeletes.numPendingDeletes() != 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* This class merges the current on-disk DV with an incoming update DV instance and merges the two instances
|
||||
* giving the incoming update precedence in terms of values, in other words the values of the update always
|
||||
|
@ -713,8 +709,6 @@ final class ReadersAndUpdates {
|
|||
reader = createNewReaderWithLatestLiveDocs(reader);
|
||||
}
|
||||
assert pendingDeletes.verifyDocCounts(reader);
|
||||
|
||||
|
||||
return new MergeReader(reader, pendingDeletes.getHardLiveDocs());
|
||||
}
|
||||
|
||||
|
|
|
@ -38,6 +38,9 @@ public class SegmentCommitInfo {
|
|||
// How many deleted docs in the segment:
|
||||
private int delCount;
|
||||
|
||||
// How many soft-deleted docs in the segment that are not also hard-deleted:
|
||||
private int softDelCount;
|
||||
|
||||
// Generation number of the live docs file (-1 if there
|
||||
// are no deletes yet):
|
||||
private long delGen;
|
||||
|
@ -73,7 +76,7 @@ public class SegmentCommitInfo {
|
|||
// NOTE: only used in-RAM by IW to track buffered deletes;
|
||||
// this is never written to/read from the Directory
|
||||
private long bufferedDeletesGen = -1;
|
||||
|
||||
|
||||
/**
|
||||
* Sole constructor.
|
||||
*
|
||||
|
@ -88,9 +91,10 @@ public class SegmentCommitInfo {
|
|||
* @param docValuesGen
|
||||
* DocValues generation number (used to name doc-values updates files)
|
||||
*/
|
||||
public SegmentCommitInfo(SegmentInfo info, int delCount, long delGen, long fieldInfosGen, long docValuesGen) {
|
||||
public SegmentCommitInfo(SegmentInfo info, int delCount, int softDelCount, long delGen, long fieldInfosGen, long docValuesGen) {
|
||||
this.info = info;
|
||||
this.delCount = delCount;
|
||||
this.softDelCount = softDelCount;
|
||||
this.delGen = delGen;
|
||||
this.nextWriteDelGen = delGen == -1 ? 1 : delGen + 1;
|
||||
this.fieldInfosGen = fieldInfosGen;
|
||||
|
@ -313,13 +317,29 @@ public class SegmentCommitInfo {
|
|||
return delCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of only soft-deleted docs.
|
||||
*/
|
||||
public int getSoftDelCount() {
|
||||
return softDelCount;
|
||||
}
|
||||
|
||||
void setDelCount(int delCount) {
|
||||
if (delCount < 0 || delCount > info.maxDoc()) {
|
||||
throw new IllegalArgumentException("invalid delCount=" + delCount + " (maxDoc=" + info.maxDoc() + ")");
|
||||
}
|
||||
assert softDelCount + delCount <= info.maxDoc();
|
||||
this.delCount = delCount;
|
||||
}
|
||||
|
||||
void setSoftDelCount(int softDelCount) {
|
||||
if (softDelCount < 0 || softDelCount > info.maxDoc()) {
|
||||
throw new IllegalArgumentException("invalid softDelCount=" + softDelCount + " (maxDoc=" + info.maxDoc() + ")");
|
||||
}
|
||||
assert softDelCount + delCount <= info.maxDoc();
|
||||
this.softDelCount = softDelCount;
|
||||
}
|
||||
|
||||
/** Returns a description of this segment. */
|
||||
public String toString(int pendingDelCount) {
|
||||
String s = info.toString(delCount + pendingDelCount);
|
||||
|
@ -332,6 +352,10 @@ public class SegmentCommitInfo {
|
|||
if (docValuesGen != -1) {
|
||||
s += ":dvGen=" + docValuesGen;
|
||||
}
|
||||
if (softDelCount > 0) {
|
||||
s += " :softDel=" + softDelCount;
|
||||
}
|
||||
|
||||
return s;
|
||||
}
|
||||
|
||||
|
@ -342,7 +366,7 @@ public class SegmentCommitInfo {
|
|||
|
||||
@Override
|
||||
public SegmentCommitInfo clone() {
|
||||
SegmentCommitInfo other = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, docValuesGen);
|
||||
SegmentCommitInfo other = new SegmentCommitInfo(info, delCount, softDelCount, delGen, fieldInfosGen, docValuesGen);
|
||||
// Not clear that we need to carry over nextWriteDelGen
|
||||
// (i.e. do we ever clone after a failed write and
|
||||
// before the next successful write?), but just do it to
|
||||
|
@ -360,4 +384,8 @@ public class SegmentCommitInfo {
|
|||
|
||||
return other;
|
||||
}
|
||||
|
||||
final int getDelCount(boolean includeSoftDeletes) {
|
||||
return includeSoftDeletes ? getDelCount() + getSoftDelCount() : getDelCount();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -122,8 +122,9 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
public static final int VERSION_70 = 7;
|
||||
/** The version that updated segment name counter to be long instead of int. */
|
||||
public static final int VERSION_72 = 8;
|
||||
|
||||
static final int VERSION_CURRENT = VERSION_72;
|
||||
/** The version that recorded softDelCount */
|
||||
public static final int VERSION_74 = 9;
|
||||
static final int VERSION_CURRENT = VERSION_74;
|
||||
|
||||
/** Used to name new segments. */
|
||||
public long counter;
|
||||
|
@ -359,7 +360,14 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
}
|
||||
long fieldInfosGen = input.readLong();
|
||||
long dvGen = input.readLong();
|
||||
SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, dvGen);
|
||||
int softDelCount = format > VERSION_72 ? input.readInt() : 0;
|
||||
if (softDelCount < 0 || softDelCount > info.maxDoc()) {
|
||||
throw new CorruptIndexException("invalid deletion count: " + softDelCount + " vs maxDoc=" + info.maxDoc(), input);
|
||||
}
|
||||
if (softDelCount + delCount > info.maxDoc()) {
|
||||
throw new CorruptIndexException("invalid deletion count: " + softDelCount + delCount + " vs maxDoc=" + info.maxDoc(), input);
|
||||
}
|
||||
SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, softDelCount, delGen, fieldInfosGen, dvGen);
|
||||
siPerCommit.setFieldInfosFiles(input.readSetOfStrings());
|
||||
final Map<Integer,Set<String>> dvUpdateFiles;
|
||||
final int numDVFields = input.readInt();
|
||||
|
@ -517,6 +525,11 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
out.writeInt(delCount);
|
||||
out.writeLong(siPerCommit.getFieldInfosGen());
|
||||
out.writeLong(siPerCommit.getDocValuesGen());
|
||||
int softDelCount = siPerCommit.getSoftDelCount();
|
||||
if (softDelCount < 0 || softDelCount > si.maxDoc()) {
|
||||
throw new IllegalStateException("cannot write segment: invalid maxDoc segment=" + si.name + " maxDoc=" + si.maxDoc() + " softDelCount=" + softDelCount);
|
||||
}
|
||||
out.writeInt(softDelCount);
|
||||
out.writeSetOfStrings(siPerCommit.getFieldInfosFiles());
|
||||
final Map<Integer,Set<String>> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles();
|
||||
out.writeInt(dvUpdatesFiles.size());
|
||||
|
|
|
@ -71,6 +71,7 @@ final class SegmentMerger {
|
|||
if (minVersion.onOrAfter(leafMinVersion)) {
|
||||
minVersion = leafMinVersion;
|
||||
}
|
||||
|
||||
}
|
||||
assert segmentInfo.minVersion == null : "The min version should be set by SegmentMerger for merged segments";
|
||||
segmentInfo.minVersion = minVersion;
|
||||
|
|
|
@ -47,7 +47,9 @@ public class SegmentWriteState {
|
|||
/** Number of deleted documents set while flushing the
|
||||
* segment. */
|
||||
public int delCountOnFlush;
|
||||
|
||||
/** Number of only soft deleted documents set while flushing the
|
||||
* segment. */
|
||||
public int softDelCountOnFlush;
|
||||
/**
|
||||
* Deletes and updates to apply while we are flushing the segment. A Term is
|
||||
* enrolled in here if it was deleted/updated at one point, and it's mapped to
|
||||
|
|
|
@ -62,6 +62,8 @@ public final class SoftDeletesDirectoryReaderWrapper extends FilterDirectoryRead
|
|||
// we try to reuse the life docs instances here if the reader cache key didn't change
|
||||
if (reader instanceof SoftDeletesFilterLeafReader && reader.getReaderCacheHelper() != null) {
|
||||
readerCache.put(((SoftDeletesFilterLeafReader) reader).reader.getReaderCacheHelper().getKey(), reader);
|
||||
} else if (reader instanceof SoftDeletesFilterCodecReader && reader.getReaderCacheHelper() != null) {
|
||||
readerCache.put(((SoftDeletesFilterCodecReader) reader).reader.getReaderCacheHelper().getKey(), reader);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -112,9 +114,35 @@ public final class SoftDeletesDirectoryReaderWrapper extends FilterDirectoryRead
|
|||
bits = new FixedBitSet(reader.maxDoc());
|
||||
bits.set(0, reader.maxDoc());
|
||||
}
|
||||
int numDeletes = reader.numDeletedDocs() + PendingSoftDeletes.applySoftDeletes(iterator, bits);
|
||||
int numSoftDeletes = PendingSoftDeletes.applySoftDeletes(iterator, bits);
|
||||
int numDeletes = reader.numDeletedDocs() + numSoftDeletes;
|
||||
int numDocs = reader.maxDoc() - numDeletes;
|
||||
return new SoftDeletesFilterLeafReader(reader, bits, numDocs);
|
||||
assert assertDocCounts(numDocs, numSoftDeletes, reader);
|
||||
return reader instanceof CodecReader ? new SoftDeletesFilterCodecReader((CodecReader) reader, bits, numDocs)
|
||||
: new SoftDeletesFilterLeafReader(reader, bits, numDocs);
|
||||
}
|
||||
|
||||
private static boolean assertDocCounts(int expectedNumDocs, int numSoftDeletes, LeafReader reader) {
|
||||
if (reader instanceof SegmentReader) {
|
||||
SegmentReader segmentReader = (SegmentReader) reader;
|
||||
SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
|
||||
if (segmentReader.isNRT == false) {
|
||||
int numDocs = segmentInfo.info.maxDoc() - segmentInfo.getSoftDelCount() - segmentInfo.getDelCount();
|
||||
assert numDocs == expectedNumDocs : "numDocs: " + numDocs + " expected: " + expectedNumDocs
|
||||
+ " maxDoc: " + segmentInfo.info.maxDoc()
|
||||
+ " getDelCount: " + segmentInfo.getDelCount()
|
||||
+ " getSoftDelCount: " + segmentInfo.getSoftDelCount()
|
||||
+ " numSoftDeletes: " + numSoftDeletes
|
||||
+ " reader.numDeletedDocs(): " + reader.numDeletedDocs();
|
||||
}
|
||||
// in the NRT case we don't have accurate numbers for getDelCount and getSoftDelCount since they might not be
|
||||
// flushed to disk when this reader is opened. We don't necessarily flush deleted doc on reopen but
|
||||
// we do for docValues.
|
||||
|
||||
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
static final class SoftDeletesFilterLeafReader extends FilterLeafReader {
|
||||
|
@ -153,6 +181,42 @@ public final class SoftDeletesDirectoryReaderWrapper extends FilterDirectoryRead
|
|||
}
|
||||
}
|
||||
|
||||
final static class SoftDeletesFilterCodecReader extends FilterCodecReader {
|
||||
private final LeafReader reader;
|
||||
private final FixedBitSet bits;
|
||||
private final int numDocs;
|
||||
private final CacheHelper readerCacheHelper;
|
||||
|
||||
private SoftDeletesFilterCodecReader(CodecReader reader, FixedBitSet bits, int numDocs) {
|
||||
super(reader);
|
||||
this.reader = reader;
|
||||
this.bits = bits;
|
||||
this.numDocs = numDocs;
|
||||
this.readerCacheHelper = reader.getReaderCacheHelper() == null ? null :
|
||||
new DelegatingCacheHelper(reader.getReaderCacheHelper());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getLiveDocs() {
|
||||
return bits;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numDocs() {
|
||||
return numDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getCoreCacheHelper() {
|
||||
return reader.getCoreCacheHelper();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheHelper getReaderCacheHelper() {
|
||||
return readerCacheHelper;
|
||||
}
|
||||
}
|
||||
|
||||
private static class DelegatingCacheHelper implements CacheHelper {
|
||||
private final CacheHelper delegate;
|
||||
private final CacheKey cacheKey = new CacheKey();
|
||||
|
|
|
@ -175,7 +175,7 @@ public final class SoftDeletesRetentionMergePolicy extends OneMergeWrappingMerge
|
|||
@Override
|
||||
public int numDeletesToMerge(SegmentCommitInfo info, int delCount, IOSupplier<CodecReader> readerSupplier) throws IOException {
|
||||
final int numDeletesToMerge = super.numDeletesToMerge(info, delCount, readerSupplier);
|
||||
if (numDeletesToMerge != 0) {
|
||||
if (numDeletesToMerge != 0 && info.getSoftDelCount() > 0) {
|
||||
final CodecReader reader = readerSupplier.get();
|
||||
if (reader.getLiveDocs() != null) {
|
||||
BooleanQuery.Builder builder = new BooleanQuery.Builder();
|
||||
|
|
|
@ -244,4 +244,9 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
return valueCount;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
DocIdSetIterator getDocIdSet() {
|
||||
return docsWithField.iterator();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -231,4 +231,9 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
|
|||
return docsWithField.cost();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
DocIdSetIterator getDocIdSet() {
|
||||
return docsWithField.iterator();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -315,4 +315,8 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
|
|||
return scratch;
|
||||
}
|
||||
}
|
||||
@Override
|
||||
DocIdSetIterator getDocIdSet() {
|
||||
return docsWithField.iterator();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1410,4 +1410,50 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
dir1.close();
|
||||
dir2.close();
|
||||
}
|
||||
|
||||
public void testAddIndicesWithSoftDeletes() throws IOException {
|
||||
Directory dir1 = newDirectory();
|
||||
IndexWriterConfig iwc1 = newIndexWriterConfig(new MockAnalyzer(random())).setSoftDeletesField("soft_delete");
|
||||
IndexWriter writer = new IndexWriter(dir1, iwc1);
|
||||
for (int i = 0; i < 30; i++) {
|
||||
Document doc = new Document();
|
||||
int docID = random().nextInt(5);
|
||||
doc.add(new StringField("id", "" + docID, Field.Store.YES));
|
||||
writer.softUpdateDocument(new Term("id", "" + docID), doc, new NumericDocValuesField("soft_delete", 1));
|
||||
if (random().nextBoolean()) {
|
||||
writer.flush();
|
||||
}
|
||||
}
|
||||
writer.commit();
|
||||
writer.close();
|
||||
DirectoryReader reader = DirectoryReader.open(dir1);
|
||||
DirectoryReader wrappedReader = new SoftDeletesDirectoryReaderWrapper(reader, "soft_delete");
|
||||
Directory dir2 = newDirectory();
|
||||
int numDocs = reader.numDocs();
|
||||
int maxDoc = reader.maxDoc();
|
||||
assertEquals(numDocs, maxDoc);
|
||||
iwc1 = newIndexWriterConfig(new MockAnalyzer(random())).setSoftDeletesField("soft_delete");
|
||||
writer = new IndexWriter(dir2, iwc1);
|
||||
CodecReader[] readers = new CodecReader[reader.leaves().size()];
|
||||
for (int i = 0; i < readers.length; i++) {
|
||||
readers[i] = (CodecReader)reader.leaves().get(i).reader();
|
||||
}
|
||||
writer.addIndexes(readers);
|
||||
assertEquals(wrappedReader.numDocs(), writer.numDocs());
|
||||
assertEquals(maxDoc, writer.maxDoc());
|
||||
writer.commit();
|
||||
SegmentCommitInfo commitInfo = writer.segmentInfos.asList().get(0);
|
||||
assertEquals(maxDoc-wrappedReader.numDocs(), commitInfo.getSoftDelCount());
|
||||
writer.close();
|
||||
Directory dir3 = newDirectory();
|
||||
iwc1 = newIndexWriterConfig(new MockAnalyzer(random())).setSoftDeletesField("soft_delete");
|
||||
writer = new IndexWriter(dir3, iwc1);
|
||||
for (int i = 0; i < readers.length; i++) {
|
||||
readers[i] = (CodecReader)wrappedReader.leaves().get(i).reader();
|
||||
}
|
||||
writer.addIndexes(readers);
|
||||
assertEquals(wrappedReader.numDocs(), writer.numDocs());
|
||||
assertEquals(wrappedReader.numDocs(), writer.maxDoc());
|
||||
IOUtils.close(reader, writer, dir3, dir2, dir1);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -212,7 +212,7 @@ public class TestCodecs extends LuceneTestCase {
|
|||
terms[i] = new TermData(text, docs, null);
|
||||
}
|
||||
|
||||
final FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
final FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
|
||||
|
||||
final FieldData field = new FieldData("field", builder, terms, true, false);
|
||||
final FieldData[] fields = new FieldData[] {field};
|
||||
|
@ -259,7 +259,7 @@ public class TestCodecs extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testRandomPostings() throws Throwable {
|
||||
final FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
final FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
|
||||
|
||||
final FieldData[] fields = new FieldData[NUM_FIELDS];
|
||||
for(int i=0;i<NUM_FIELDS;i++) {
|
||||
|
|
|
@ -222,7 +222,7 @@ public class TestDoc extends LuceneTestCase {
|
|||
|
||||
SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(r1, r2),
|
||||
si, InfoStream.getDefault(), trackingDir,
|
||||
new FieldInfos.FieldNumbers(), context);
|
||||
new FieldInfos.FieldNumbers(null), context);
|
||||
|
||||
MergeState mergeState = merger.merge();
|
||||
r1.close();
|
||||
|
@ -238,7 +238,7 @@ public class TestDoc extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
return new SegmentCommitInfo(si, 0, -1L, -1L, -1L);
|
||||
return new SegmentCommitInfo(si, 0, 0, -1L, -1L, -1L);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -44,7 +44,7 @@ public class TestFieldsReader extends LuceneTestCase {
|
|||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
testDoc = new Document();
|
||||
fieldInfos = new FieldInfos.Builder();
|
||||
fieldInfos = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
|
||||
DocHelper.setupDoc(testDoc);
|
||||
for (IndexableField field : testDoc.getFields()) {
|
||||
FieldInfo fieldInfo = fieldInfos.getOrAdd(field.name());
|
||||
|
|
|
@ -3137,7 +3137,11 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
searcher = new IndexSearcher(reader);
|
||||
topDocs = searcher.search(new TermQuery(new Term("id", "1")), 10);
|
||||
assertEquals(0, topDocs.totalHits);
|
||||
|
||||
int numSoftDeleted = 0;
|
||||
for (SegmentCommitInfo info : writer.segmentInfos) {
|
||||
numSoftDeleted += info.getSoftDelCount();
|
||||
}
|
||||
assertEquals(writer.maxDoc() - writer.numDocs(), numSoftDeleted);
|
||||
writer.close();
|
||||
reader.close();
|
||||
dir.close();
|
||||
|
@ -3267,6 +3271,20 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
assertEquals(1, reader.docFreq(new Term("id", id)));
|
||||
}
|
||||
}
|
||||
int numSoftDeleted = 0;
|
||||
for (SegmentCommitInfo info : writer.segmentInfos) {
|
||||
numSoftDeleted += info.getSoftDelCount() + info.getDelCount();
|
||||
}
|
||||
assertEquals(writer.maxDoc() - writer.numDocs(), numSoftDeleted);
|
||||
writer.commit();
|
||||
try (DirectoryReader dirReader = DirectoryReader.open(dir)) {
|
||||
int delCount = 0;
|
||||
for (LeafReaderContext ctx : dirReader.leaves()) {
|
||||
SegmentCommitInfo segmentInfo = ((SegmentReader) ctx.reader()).getSegmentInfo();
|
||||
delCount += segmentInfo.getSoftDelCount() + segmentInfo.getDelCount();
|
||||
}
|
||||
assertEquals(numSoftDeleted, delCount);
|
||||
}
|
||||
IOUtils.close(reader, writer, dir);
|
||||
}
|
||||
|
||||
|
@ -3376,4 +3394,110 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
IOUtils.close(reader, writer, dir);
|
||||
}
|
||||
|
||||
public void testPreventChangingSoftDeletesField() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig().setSoftDeletesField("my_deletes"));
|
||||
Document v1 = new Document();
|
||||
v1.add(new StringField("id", "1", Field.Store.YES));
|
||||
v1.add(new StringField("version", "1", Field.Store.YES));
|
||||
writer.addDocument(v1);
|
||||
Document v2 = new Document();
|
||||
v2.add(new StringField("id", "1", Field.Store.YES));
|
||||
v2.add(new StringField("version", "2", Field.Store.YES));
|
||||
writer.softUpdateDocument(new Term("id", "1"), v2, new NumericDocValuesField("my_deletes", 1));
|
||||
writer.commit();
|
||||
writer.close();
|
||||
for (SegmentCommitInfo si : SegmentInfos.readLatestCommit(dir)) {
|
||||
FieldInfos fieldInfos = IndexWriter.readFieldInfos(si);
|
||||
assertEquals("my_deletes", fieldInfos.getSoftDeletesField());
|
||||
assertTrue(fieldInfos.fieldInfo("my_deletes").isSoftDeletesField());
|
||||
}
|
||||
|
||||
IllegalArgumentException illegalError = expectThrows(IllegalArgumentException.class, () -> {
|
||||
new IndexWriter(dir, newIndexWriterConfig().setSoftDeletesField("your_deletes"));
|
||||
});
|
||||
assertEquals("cannot configure [your_deletes] as soft-deletes; " +
|
||||
"this index uses [my_deletes] as soft-deletes already", illegalError.getMessage());
|
||||
|
||||
IndexWriterConfig softDeleteConfig = newIndexWriterConfig().setSoftDeletesField("my_deletes")
|
||||
.setMergePolicy(new SoftDeletesRetentionMergePolicy("my_deletes", () -> new MatchAllDocsQuery(), newMergePolicy()));
|
||||
writer = new IndexWriter(dir, softDeleteConfig);
|
||||
Document tombstone = new Document();
|
||||
tombstone.add(new StringField("id", "tombstone", Field.Store.YES));
|
||||
tombstone.add(new NumericDocValuesField("my_deletes", 1));
|
||||
writer.addDocument(tombstone);
|
||||
writer.flush();
|
||||
for (SegmentCommitInfo si : writer.segmentInfos) {
|
||||
FieldInfos fieldInfos = IndexWriter.readFieldInfos(si);
|
||||
assertEquals("my_deletes", fieldInfos.getSoftDeletesField());
|
||||
assertTrue(fieldInfos.fieldInfo("my_deletes").isSoftDeletesField());
|
||||
}
|
||||
writer.close();
|
||||
// reopen writer without soft-deletes field should be prevented
|
||||
IllegalArgumentException reopenError = expectThrows(IllegalArgumentException.class, () -> {
|
||||
new IndexWriter(dir, newIndexWriterConfig());
|
||||
});
|
||||
assertEquals("this index has [my_deletes] as soft-deletes already" +
|
||||
" but soft-deletes field is not configured in IWC", reopenError.getMessage());
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testPreventAddingIndexesWithDifferentSoftDeletesField() throws Exception {
|
||||
Directory dir1 = newDirectory();
|
||||
IndexWriter w1 = new IndexWriter(dir1, newIndexWriterConfig().setSoftDeletesField("soft_deletes_1"));
|
||||
for (int i = 0; i < 2; i++) {
|
||||
Document d = new Document();
|
||||
d.add(new StringField("id", "1", Field.Store.YES));
|
||||
d.add(new StringField("version", Integer.toString(i), Field.Store.YES));
|
||||
w1.softUpdateDocument(new Term("id", "1"), d, new NumericDocValuesField("soft_deletes_1", 1));
|
||||
}
|
||||
w1.commit();
|
||||
w1.close();
|
||||
|
||||
Directory dir2 = newDirectory();
|
||||
IndexWriter w2 = new IndexWriter(dir2, newIndexWriterConfig().setSoftDeletesField("soft_deletes_2"));
|
||||
IllegalArgumentException error = expectThrows(IllegalArgumentException.class, () -> w2.addIndexes(dir1));
|
||||
assertEquals("cannot configure [soft_deletes_2] as soft-deletes; this index uses [soft_deletes_1] as soft-deletes already",
|
||||
error.getMessage());
|
||||
w2.close();
|
||||
|
||||
Directory dir3 = newDirectory();
|
||||
IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField("soft_deletes_1");
|
||||
IndexWriter w3 = new IndexWriter(dir3, config);
|
||||
w3.addIndexes(dir1);
|
||||
for (SegmentCommitInfo si : w3.segmentInfos) {
|
||||
FieldInfo softDeleteField = IndexWriter.readFieldInfos(si).fieldInfo("soft_deletes_1");
|
||||
assertTrue(softDeleteField.isSoftDeletesField());
|
||||
}
|
||||
w3.close();
|
||||
IOUtils.close(dir1, dir2, dir3);
|
||||
}
|
||||
|
||||
public void testNotAllowUsingExistingFieldAsSoftDeletes() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
|
||||
for (int i = 0; i < 2; i++) {
|
||||
Document d = new Document();
|
||||
d.add(new StringField("id", "1", Field.Store.YES));
|
||||
if (random().nextBoolean()) {
|
||||
d.add(new NumericDocValuesField("dv_field", 1));
|
||||
w.updateDocument(new Term("id", "1"), d);
|
||||
} else {
|
||||
w.softUpdateDocument(new Term("id", "1"), d, new NumericDocValuesField("dv_field", 1));
|
||||
}
|
||||
}
|
||||
w.commit();
|
||||
w.close();
|
||||
String softDeletesField = random().nextBoolean() ? "id" : "dv_field";
|
||||
IllegalArgumentException error = expectThrows(IllegalArgumentException.class, () -> {
|
||||
IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField(softDeletesField);
|
||||
new IndexWriter(dir, config);
|
||||
});
|
||||
assertEquals("cannot configure [" + softDeletesField + "] as soft-deletes;" +
|
||||
" this index uses [" + softDeletesField + "] as non-soft-deletes already", error.getMessage());
|
||||
IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField("non-existing-field");
|
||||
w = new IndexWriter(dir, config);
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -331,7 +331,7 @@ public class TestIndexWriterThreadsToSegments extends LuceneTestCase {
|
|||
byte id[] = readSegmentInfoID(dir, fileName);
|
||||
SegmentInfo si = TestUtil.getDefaultCodec().segmentInfoFormat().read(dir, segName, id, IOContext.DEFAULT);
|
||||
si.setCodec(codec);
|
||||
SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, 0, -1, -1, -1);
|
||||
SegmentReader sr = new SegmentReader(sci, Version.LATEST.major, IOContext.DEFAULT);
|
||||
try {
|
||||
thread0Count += sr.docFreq(new Term("field", "threadID0"));
|
||||
|
|
|
@ -137,7 +137,7 @@ public class TestOneMergeWrappingMergePolicy extends LuceneTestCase {
|
|||
Collections.emptyMap(), // attributes
|
||||
null /* indexSort */);
|
||||
final List<SegmentCommitInfo> segments = new LinkedList<SegmentCommitInfo>();
|
||||
segments.add(new SegmentCommitInfo(si, 0, 0, 0, 0));
|
||||
segments.add(new SegmentCommitInfo(si, 0, 0, 0, 0, 0));
|
||||
ms.add(new MergePolicy.OneMerge(segments));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,7 +40,7 @@ public class TestPendingDeletes extends LuceneTestCase {
|
|||
RAMDirectory dir = new RAMDirectory();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 10, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, -1, -1, -1);
|
||||
PendingDeletes deletes = newPendingDeletes(commitInfo);
|
||||
assertNull(deletes.getLiveDocs());
|
||||
int docToDelete = TestUtil.nextInt(random(), 0, 7);
|
||||
|
@ -74,7 +74,7 @@ public class TestPendingDeletes extends LuceneTestCase {
|
|||
RAMDirectory dir = new RAMDirectory();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 6, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, -1, -1, -1);
|
||||
PendingDeletes deletes = newPendingDeletes(commitInfo);
|
||||
assertFalse(deletes.writeLiveDocs(dir));
|
||||
assertEquals(0, dir.listAll().length);
|
||||
|
@ -131,7 +131,7 @@ public class TestPendingDeletes extends LuceneTestCase {
|
|||
RAMDirectory dir = new RAMDirectory();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 3, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, -1, -1, -1);
|
||||
FieldInfos fieldInfos = new FieldInfos(new FieldInfo[0]);
|
||||
si.getCodec().fieldInfosFormat().write(dir, si, "", fieldInfos, IOContext.DEFAULT);
|
||||
PendingDeletes deletes = newPendingDeletes(commitInfo);
|
||||
|
|
|
@ -44,6 +44,45 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
return new PendingSoftDeletes("_soft_deletes", commitInfo);
|
||||
}
|
||||
|
||||
public void testHardDeleteSoftDeleted() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()
|
||||
.setSoftDeletesField("_soft_deletes")
|
||||
// make sure all docs will end up in the same segment
|
||||
.setMaxBufferedDocs(10)
|
||||
.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH));
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("id", "1", Field.Store.YES));
|
||||
writer.softUpdateDocument(new Term("id", "1"), doc,
|
||||
new NumericDocValuesField("_soft_deletes", 1));
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", "2", Field.Store.YES));
|
||||
writer.softUpdateDocument(new Term("id", "2"), doc,
|
||||
new NumericDocValuesField("_soft_deletes", 1));
|
||||
doc = new Document();
|
||||
doc.add(new StringField("id", "2", Field.Store.YES));
|
||||
writer.softUpdateDocument(new Term("id", "2"), doc,
|
||||
new NumericDocValuesField("_soft_deletes", 1));
|
||||
writer.commit();
|
||||
DirectoryReader reader = writer.getReader();
|
||||
assertEquals(1, reader.leaves().size());
|
||||
SegmentReader segmentReader = (SegmentReader) reader.leaves().get(0).reader();
|
||||
SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
|
||||
PendingSoftDeletes pendingSoftDeletes = newPendingDeletes(segmentInfo);
|
||||
pendingSoftDeletes.onNewReader(segmentReader, segmentInfo);
|
||||
assertEquals(0, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(1, pendingSoftDeletes.getDelCount());
|
||||
assertTrue(pendingSoftDeletes.getLiveDocs().get(0));
|
||||
assertFalse(pendingSoftDeletes.getLiveDocs().get(1));
|
||||
assertTrue(pendingSoftDeletes.getLiveDocs().get(2));
|
||||
assertNull(pendingSoftDeletes.getHardLiveDocs());
|
||||
assertTrue(pendingSoftDeletes.delete(1));
|
||||
assertEquals(0, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(-1, pendingSoftDeletes.pendingDeleteCount); // transferred the delete
|
||||
assertEquals(1, pendingSoftDeletes.getDelCount());
|
||||
IOUtils.close(reader, writer, dir);
|
||||
}
|
||||
|
||||
public void testDeleteSoft() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()
|
||||
|
@ -70,7 +109,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
|
||||
PendingSoftDeletes pendingSoftDeletes = newPendingDeletes(segmentInfo);
|
||||
pendingSoftDeletes.onNewReader(segmentReader, segmentInfo);
|
||||
assertEquals(1, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(0, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(1, pendingSoftDeletes.getDelCount());
|
||||
assertTrue(pendingSoftDeletes.getLiveDocs().get(0));
|
||||
assertFalse(pendingSoftDeletes.getLiveDocs().get(1));
|
||||
assertTrue(pendingSoftDeletes.getLiveDocs().get(2));
|
||||
|
@ -78,7 +118,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
// pass reader again
|
||||
Bits liveDocs = pendingSoftDeletes.getLiveDocs();
|
||||
pendingSoftDeletes.onNewReader(segmentReader, segmentInfo);
|
||||
assertEquals(1, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(0, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(1, pendingSoftDeletes.getDelCount());
|
||||
assertSame(liveDocs, pendingSoftDeletes.getLiveDocs());
|
||||
|
||||
// now apply a hard delete
|
||||
|
@ -91,7 +132,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
segmentInfo = segmentReader.getSegmentInfo();
|
||||
pendingSoftDeletes = newPendingDeletes(segmentInfo);
|
||||
pendingSoftDeletes.onNewReader(segmentReader, segmentInfo);
|
||||
assertEquals(1, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(0, pendingSoftDeletes.numPendingDeletes());
|
||||
assertEquals(2, pendingSoftDeletes.getDelCount());
|
||||
assertFalse(pendingSoftDeletes.getLiveDocs().get(0));
|
||||
assertFalse(pendingSoftDeletes.getLiveDocs().get(1));
|
||||
assertTrue(pendingSoftDeletes.getLiveDocs().get(2));
|
||||
|
@ -106,7 +148,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
RAMDirectory dir = new RAMDirectory();
|
||||
SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 10, false, Codec.getDefault(),
|
||||
Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, -1, -1, -1);
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig());
|
||||
for (int i = 0; i < si.maxDoc(); i++) {
|
||||
writer.addDocument(new Document());
|
||||
|
@ -120,13 +162,14 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
deletes.onNewReader(segmentReader, commitInfo);
|
||||
reader.close();
|
||||
writer.close();
|
||||
FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 0, Collections.emptyMap(), 0, 0);
|
||||
FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 0, Collections.emptyMap(), 0, 0, true);
|
||||
List<Integer> docsDeleted = Arrays.asList(1, 3, 7, 8, DocIdSetIterator.NO_MORE_DOCS);
|
||||
List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 10, true));
|
||||
for (DocValuesFieldUpdates update : updates) {
|
||||
deletes.onDocValuesUpdate(fieldInfo, update.iterator());
|
||||
}
|
||||
assertEquals(4, deletes.numPendingDeletes());
|
||||
assertEquals(0, deletes.numPendingDeletes());
|
||||
assertEquals(4, deletes.getDelCount());
|
||||
assertTrue(deletes.getLiveDocs().get(0));
|
||||
assertFalse(deletes.getLiveDocs().get(1));
|
||||
assertTrue(deletes.getLiveDocs().get(2));
|
||||
|
@ -140,11 +183,12 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
|
||||
docsDeleted = Arrays.asList(1, 2, DocIdSetIterator.NO_MORE_DOCS);
|
||||
updates = Arrays.asList(singleUpdate(docsDeleted, 10, true));
|
||||
fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 1, Collections.emptyMap(), 0, 0);
|
||||
fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 1, Collections.emptyMap(), 0, 0, true);
|
||||
for (DocValuesFieldUpdates update : updates) {
|
||||
deletes.onDocValuesUpdate(fieldInfo, update.iterator());
|
||||
}
|
||||
assertEquals(5, deletes.numPendingDeletes());
|
||||
assertEquals(0, deletes.numPendingDeletes());
|
||||
assertEquals(5, deletes.getDelCount());
|
||||
assertTrue(deletes.getLiveDocs().get(0));
|
||||
assertFalse(deletes.getLiveDocs().get(1));
|
||||
assertFalse(deletes.getLiveDocs().get(2));
|
||||
|
@ -182,13 +226,14 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
|
||||
PendingDeletes deletes = newPendingDeletes(segmentInfo);
|
||||
deletes.onNewReader(segmentReader, segmentInfo);
|
||||
FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0);
|
||||
FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0, true);
|
||||
List<Integer> docsDeleted = Arrays.asList(1, DocIdSetIterator.NO_MORE_DOCS);
|
||||
List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 3, true));
|
||||
for (DocValuesFieldUpdates update : updates) {
|
||||
deletes.onDocValuesUpdate(fieldInfo, update.iterator());
|
||||
}
|
||||
assertEquals(1, deletes.numPendingDeletes());
|
||||
assertEquals(0, deletes.numPendingDeletes());
|
||||
assertEquals(1, deletes.getDelCount());
|
||||
assertTrue(deletes.getLiveDocs().get(0));
|
||||
assertFalse(deletes.getLiveDocs().get(1));
|
||||
assertTrue(deletes.getLiveDocs().get(2));
|
||||
|
@ -199,7 +244,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
assertTrue(deletes.getLiveDocs().get(0));
|
||||
assertFalse(deletes.getLiveDocs().get(1));
|
||||
assertTrue(deletes.getLiveDocs().get(2));
|
||||
assertEquals(1, deletes.numPendingDeletes());
|
||||
assertEquals(0, deletes.numPendingDeletes());
|
||||
assertEquals(1, deletes.getDelCount());
|
||||
IOUtils.close(reader, writer, dir);
|
||||
}
|
||||
|
||||
|
@ -228,7 +274,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
|
||||
PendingDeletes deletes = newPendingDeletes(segmentInfo);
|
||||
deletes.onNewReader(segmentReader, segmentInfo);
|
||||
FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0);
|
||||
FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0, true);
|
||||
List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(Arrays.asList(0, 1, DocIdSetIterator.NO_MORE_DOCS), 3, false));
|
||||
for (DocValuesFieldUpdates update : updates) {
|
||||
deletes.onDocValuesUpdate(fieldInfo, update.iterator());
|
||||
|
@ -247,7 +293,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
assertEquals(0, deletes.numPendingDeletes());
|
||||
|
||||
segmentInfo.advanceDocValuesGen();
|
||||
fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0);
|
||||
fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0, true);
|
||||
updates = Arrays.asList(singleUpdate(Arrays.asList(1, DocIdSetIterator.NO_MORE_DOCS), 3, true));
|
||||
for (DocValuesFieldUpdates update : updates) {
|
||||
deletes.onDocValuesUpdate(fieldInfo, update.iterator());
|
||||
|
@ -257,7 +303,8 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
|
|||
assertTrue(deletes.getLiveDocs().get(0));
|
||||
assertFalse(deletes.getLiveDocs().get(1));
|
||||
assertTrue(deletes.getLiveDocs().get(2));
|
||||
assertEquals(1, deletes.numPendingDeletes());
|
||||
assertEquals(0, deletes.numPendingDeletes());
|
||||
assertEquals(1, deletes.getDelCount());
|
||||
IOUtils.close(reader, writer, dir);
|
||||
|
||||
}
|
||||
|
|
|
@ -60,7 +60,7 @@ public class TestSegmentInfos extends LuceneTestCase {
|
|||
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
|
||||
info.setFiles(Collections.<String>emptySet());
|
||||
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, 0, -1, -1, -1);
|
||||
|
||||
sis.add(commitInfo);
|
||||
sis.commit(dir);
|
||||
|
@ -82,14 +82,14 @@ public class TestSegmentInfos extends LuceneTestCase {
|
|||
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
|
||||
info.setFiles(Collections.<String>emptySet());
|
||||
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
|
||||
SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, 0, -1, -1, -1);
|
||||
sis.add(commitInfo);
|
||||
|
||||
info = new SegmentInfo(dir, Version.LUCENE_8_0_0, Version.LUCENE_8_0_0, "_1", 1, false, Codec.getDefault(),
|
||||
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
|
||||
info.setFiles(Collections.<String>emptySet());
|
||||
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
|
||||
commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
|
||||
commitInfo = new SegmentCommitInfo(info, 0, 0,-1, -1, -1);
|
||||
sis.add(commitInfo);
|
||||
|
||||
sis.commit(dir);
|
||||
|
|
|
@ -88,7 +88,7 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
|
||||
SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(reader1, reader2),
|
||||
si, InfoStream.getDefault(), mergedDir,
|
||||
new FieldInfos.FieldNumbers(),
|
||||
new FieldInfos.FieldNumbers(null),
|
||||
newIOContext(random(), new IOContext(new MergeInfo(-1, -1, false, -1))));
|
||||
MergeState mergeState = merger.merge();
|
||||
int docsMerged = mergeState.segmentInfo.maxDoc();
|
||||
|
@ -96,7 +96,7 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
//Should be able to open a new SegmentReader against the new directory
|
||||
SegmentReader mergedReader = new SegmentReader(new SegmentCommitInfo(
|
||||
mergeState.segmentInfo,
|
||||
0, -1L, -1L, -1L),
|
||||
0, 0, -1L, -1L, -1L),
|
||||
Version.LATEST.major,
|
||||
newIOContext(random()));
|
||||
assertTrue(mergedReader != null);
|
||||
|
|
|
@ -104,7 +104,8 @@ public class TestSoftDeletesDirectoryReaderWrapper extends LuceneTestCase {
|
|||
}
|
||||
|
||||
private boolean isWrapped(LeafReader reader) {
|
||||
return reader instanceof SoftDeletesDirectoryReaderWrapper.SoftDeletesFilterLeafReader;
|
||||
return reader instanceof SoftDeletesDirectoryReaderWrapper.SoftDeletesFilterLeafReader
|
||||
|| reader instanceof SoftDeletesDirectoryReaderWrapper.SoftDeletesFilterCodecReader;
|
||||
}
|
||||
|
||||
public void testMixSoftAndHardDeletes() throws IOException {
|
||||
|
|
|
@ -81,7 +81,7 @@ public class TermVectorLeafReader extends LeafReader {
|
|||
}
|
||||
FieldInfo fieldInfo = new FieldInfo(field, 0,
|
||||
true, true, terms.hasPayloads(),
|
||||
indexOptions, DocValuesType.NONE, -1, Collections.emptyMap(), 0, 0);
|
||||
indexOptions, DocValuesType.NONE, -1, Collections.emptyMap(), 0, 0, false);
|
||||
fieldInfos = new FieldInfos(new FieldInfo[]{fieldInfo});
|
||||
}
|
||||
|
||||
|
|
|
@ -501,7 +501,7 @@ public class MemoryIndex {
|
|||
IndexOptions indexOptions = storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
|
||||
return new FieldInfo(fieldName, ord, fieldType.storeTermVectors(), fieldType.omitNorms(), storePayloads,
|
||||
indexOptions, fieldType.docValuesType(), -1, Collections.emptyMap(),
|
||||
fieldType.pointDimensionCount(), fieldType.pointNumBytes());
|
||||
fieldType.pointDimensionCount(), fieldType.pointNumBytes(), false);
|
||||
}
|
||||
|
||||
private void storePointValues(Info info, BytesRef pointValue) {
|
||||
|
@ -520,7 +520,7 @@ public class MemoryIndex {
|
|||
info.fieldInfo = new FieldInfo(
|
||||
info.fieldInfo.name, info.fieldInfo.number, info.fieldInfo.hasVectors(), info.fieldInfo.hasPayloads(),
|
||||
info.fieldInfo.hasPayloads(), info.fieldInfo.getIndexOptions(), docValuesType, -1, info.fieldInfo.attributes(),
|
||||
info.fieldInfo.getPointDimensionCount(), info.fieldInfo.getPointNumBytes()
|
||||
info.fieldInfo.getPointDimensionCount(), info.fieldInfo.getPointNumBytes(), info.fieldInfo.isSoftDeletesField()
|
||||
);
|
||||
} else if (existingDocValuesType != docValuesType) {
|
||||
throw new IllegalArgumentException("Can't add [" + docValuesType + "] doc values field [" + fieldName + "], because [" + existingDocValuesType + "] doc values field already exists");
|
||||
|
|
|
@ -141,7 +141,7 @@ public class IndexSplitter {
|
|||
// Same info just changing the dir:
|
||||
SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.getMinVersion(), info.name, info.maxDoc(),
|
||||
info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId(), new HashMap<>(), null);
|
||||
destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(),
|
||||
destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getSoftDelCount(),
|
||||
infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen(),
|
||||
infoPerCommit.getDocValuesGen()));
|
||||
// now copy files over
|
||||
|
|
|
@ -359,7 +359,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
: "no need \"textgrams\" when minPrefixChars="+minPrefixChars;
|
||||
if (fieldName.equals(TEXTGRAMS_FIELD_NAME) && minPrefixChars > 0) {
|
||||
// TODO: should use an EdgeNGramTokenFilterFactory here
|
||||
TokenFilter filter = new EdgeNGramTokenFilter(components.getTokenStream(), 1, minPrefixChars);
|
||||
TokenFilter filter = new EdgeNGramTokenFilter(components.getTokenStream(), 1, minPrefixChars, false);
|
||||
return new TokenStreamComponents(components.getTokenizer(), filter);
|
||||
} else {
|
||||
return components;
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search.suggest.document;
|
|||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.AnalyzerWrapper;
|
||||
import org.apache.lucene.analysis.TokenStreamToAutomaton;
|
||||
import org.apache.lucene.util.automaton.Operations;
|
||||
import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
|
||||
|
||||
/**
|
||||
* Wraps an {@link org.apache.lucene.analysis.Analyzer}
|
||||
|
@ -37,24 +37,11 @@ import org.apache.lucene.util.automaton.Operations;
|
|||
*/
|
||||
public final class CompletionAnalyzer extends AnalyzerWrapper {
|
||||
|
||||
/**
|
||||
* Represents the separation between tokens, if
|
||||
* <code>preserveSep</code> is <code>true</code>
|
||||
* <p>
|
||||
* Same label is used as a delimiter in the {@link org.apache.lucene.search.suggest.document.CompletionTokenStream}
|
||||
* payload
|
||||
*/
|
||||
final static int SEP_LABEL = NRTSuggesterBuilder.PAYLOAD_SEP;
|
||||
|
||||
/**
|
||||
* Represent a hole character, inserted by {@link org.apache.lucene.analysis.TokenStreamToAutomaton}
|
||||
*/
|
||||
final static int HOLE_CHARACTER = TokenStreamToAutomaton.HOLE;
|
||||
|
||||
final static int DEFAULT_MAX_GRAPH_EXPANSIONS = Operations.DEFAULT_MAX_DETERMINIZED_STATES;
|
||||
final static boolean DEFAULT_PRESERVE_SEP = true;
|
||||
final static boolean DEFAULT_PRESERVE_POSITION_INCREMENTS = true;
|
||||
|
||||
private final Analyzer analyzer;
|
||||
|
||||
/**
|
||||
|
@ -101,7 +88,7 @@ public final class CompletionAnalyzer extends AnalyzerWrapper {
|
|||
* preserving token separation, position increments and no limit on graph expansions
|
||||
*/
|
||||
public CompletionAnalyzer(Analyzer analyzer) {
|
||||
this(analyzer, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, DEFAULT_MAX_GRAPH_EXPANSIONS);
|
||||
this(analyzer, ConcatenateGraphFilter.DEFAULT_PRESERVE_SEP, ConcatenateGraphFilter.DEFAULT_PRESERVE_POSITION_INCREMENTS, ConcatenateGraphFilter.DEFAULT_MAX_GRAPH_EXPANSIONS);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -109,7 +96,7 @@ public final class CompletionAnalyzer extends AnalyzerWrapper {
|
|||
* with no limit on graph expansions
|
||||
*/
|
||||
public CompletionAnalyzer(Analyzer analyzer, boolean preserveSep, boolean preservePositionIncrements) {
|
||||
this(analyzer, preserveSep, preservePositionIncrements, DEFAULT_MAX_GRAPH_EXPANSIONS);
|
||||
this(analyzer, preserveSep, preservePositionIncrements, ConcatenateGraphFilter.DEFAULT_MAX_GRAPH_EXPANSIONS);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -117,7 +104,7 @@ public final class CompletionAnalyzer extends AnalyzerWrapper {
|
|||
* preserving token separation and position increments
|
||||
*/
|
||||
public CompletionAnalyzer(Analyzer analyzer, int maxGraphExpansions) {
|
||||
this(analyzer, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, maxGraphExpansions);
|
||||
this(analyzer, ConcatenateGraphFilter.DEFAULT_PRESERVE_SEP, ConcatenateGraphFilter.DEFAULT_PRESERVE_POSITION_INCREMENTS, maxGraphExpansions);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.lucene.search.Query;
|
|||
import org.apache.lucene.search.suggest.BitsProducer;
|
||||
|
||||
import static org.apache.lucene.search.suggest.document.CompletionAnalyzer.HOLE_CHARACTER;
|
||||
import static org.apache.lucene.search.suggest.document.CompletionAnalyzer.SEP_LABEL;
|
||||
import static org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter.SEP_LABEL;
|
||||
|
||||
/**
|
||||
* Abstract {@link Query} that match documents containing terms with a specified prefix
|
||||
|
|
|
@ -14,71 +14,43 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.search.suggest.document;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.TokenStreamToAutomaton;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
||||
import org.apache.lucene.util.AttributeImpl;
|
||||
import org.apache.lucene.util.AttributeReflector;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.automaton.Automaton;
|
||||
import org.apache.lucene.util.automaton.FiniteStringsIterator;
|
||||
import org.apache.lucene.util.automaton.LimitedFiniteStringsIterator;
|
||||
import org.apache.lucene.util.automaton.Operations;
|
||||
import org.apache.lucene.util.automaton.Transition;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
|
||||
import static org.apache.lucene.search.suggest.document.CompletionAnalyzer.DEFAULT_MAX_GRAPH_EXPANSIONS;
|
||||
import static org.apache.lucene.search.suggest.document.CompletionAnalyzer.DEFAULT_PRESERVE_POSITION_INCREMENTS;
|
||||
import static org.apache.lucene.search.suggest.document.CompletionAnalyzer.DEFAULT_PRESERVE_SEP;
|
||||
import static org.apache.lucene.search.suggest.document.CompletionAnalyzer.SEP_LABEL;
|
||||
|
||||
/**
|
||||
* Token stream which converts a provided token stream to an automaton.
|
||||
* The accepted strings enumeration from the automaton are available through the
|
||||
* {@link org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute} attribute
|
||||
* The token stream uses a {@link org.apache.lucene.analysis.tokenattributes.PayloadAttribute} to store
|
||||
* a completion's payload (see {@link CompletionTokenStream#setPayload(org.apache.lucene.util.BytesRef)})
|
||||
*
|
||||
* A {@link ConcatenateGraphFilter} but we can set the payload and provide access to config options.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class CompletionTokenStream extends TokenStream {
|
||||
public final class CompletionTokenStream extends TokenFilter {
|
||||
|
||||
private final PayloadAttribute payloadAttr = addAttribute(PayloadAttribute.class);
|
||||
private final BytesRefBuilderTermAttribute bytesAtt = addAttribute(BytesRefBuilderTermAttribute.class);
|
||||
|
||||
// package accessible on purpose
|
||||
final TokenStream inputTokenStream;
|
||||
final boolean preserveSep;
|
||||
final boolean preservePositionIncrements;
|
||||
final int maxGraphExpansions;
|
||||
|
||||
private FiniteStringsIterator finiteStrings;
|
||||
private BytesRef payload;
|
||||
private CharTermAttribute charTermAttribute;
|
||||
private BytesRef payload; // note doesn't participate in TokenStream lifecycle; it's effectively constant
|
||||
|
||||
/**
|
||||
* Creates a token stream to convert <code>input</code> to a token stream
|
||||
* of accepted strings by its automaton.
|
||||
* <p>
|
||||
* The token stream <code>input</code> is converted to an automaton
|
||||
* with the default settings of {@link org.apache.lucene.search.suggest.document.CompletionAnalyzer}
|
||||
*/
|
||||
CompletionTokenStream(TokenStream inputTokenStream) {
|
||||
this(inputTokenStream, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, DEFAULT_MAX_GRAPH_EXPANSIONS);
|
||||
this(inputTokenStream,
|
||||
ConcatenateGraphFilter.DEFAULT_PRESERVE_SEP,
|
||||
ConcatenateGraphFilter.DEFAULT_PRESERVE_POSITION_INCREMENTS,
|
||||
ConcatenateGraphFilter.DEFAULT_MAX_GRAPH_EXPANSIONS);
|
||||
}
|
||||
|
||||
CompletionTokenStream(TokenStream inputTokenStream, boolean preserveSep, boolean preservePositionIncrements, int maxGraphExpansions) {
|
||||
// Don't call the super(input) ctor - this is a true delegate and has a new attribute source since we consume
|
||||
// the input stream entirely in the first call to incrementToken
|
||||
super(new ConcatenateGraphFilter(inputTokenStream, preserveSep, preservePositionIncrements, maxGraphExpansions));
|
||||
this.inputTokenStream = inputTokenStream;
|
||||
this.preserveSep = preserveSep;
|
||||
this.preservePositionIncrements = preservePositionIncrements;
|
||||
|
@ -94,248 +66,23 @@ public final class CompletionTokenStream extends TokenStream {
|
|||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
clearAttributes();
|
||||
if (finiteStrings == null) {
|
||||
Automaton automaton = toAutomaton();
|
||||
finiteStrings = new LimitedFiniteStringsIterator(automaton, maxGraphExpansions);
|
||||
}
|
||||
|
||||
IntsRef string = finiteStrings.next();
|
||||
if (string == null) {
|
||||
if (input.incrementToken()) {
|
||||
payloadAttr.setPayload(payload);
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
|
||||
Util.toBytesRef(string, bytesAtt.builder()); // now we have UTF-8
|
||||
if (charTermAttribute != null) {
|
||||
charTermAttribute.setLength(0);
|
||||
charTermAttribute.append(bytesAtt.toUTF16());
|
||||
}
|
||||
if (payload != null) {
|
||||
payloadAttr.setPayload(this.payload);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void end() throws IOException {
|
||||
super.end();
|
||||
if (finiteStrings == null) {
|
||||
inputTokenStream.end();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (finiteStrings == null) {
|
||||
inputTokenStream.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
super.reset();
|
||||
if (hasAttribute(CharTermAttribute.class)) {
|
||||
// we only create this if we really need it to safe the UTF-8 to UTF-16 conversion
|
||||
charTermAttribute = getAttribute(CharTermAttribute.class);
|
||||
}
|
||||
finiteStrings = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the token stream to an automaton,
|
||||
* treating the transition labels as utf-8
|
||||
*/
|
||||
/** Delegates to...At
|
||||
* @see ConcatenateGraphFilter#toAutomaton() */
|
||||
public Automaton toAutomaton() throws IOException {
|
||||
return toAutomaton(false);
|
||||
return ((ConcatenateGraphFilter)input).toAutomaton();
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the tokenStream to an automaton
|
||||
*/
|
||||
/** Delegates to...
|
||||
* @see ConcatenateGraphFilter#toAutomaton(boolean) */
|
||||
public Automaton toAutomaton(boolean unicodeAware) throws IOException {
|
||||
// TODO refactor this
|
||||
// maybe we could hook up a modified automaton from TermAutomatonQuery here?
|
||||
Automaton automaton = null;
|
||||
try {
|
||||
// Create corresponding automaton: labels are bytes
|
||||
// from each analyzed token, with byte 0 used as
|
||||
// separator between tokens:
|
||||
final TokenStreamToAutomaton tsta;
|
||||
if (preserveSep) {
|
||||
tsta = new EscapingTokenStreamToAutomaton((char) SEP_LABEL);
|
||||
} else {
|
||||
// When we're not preserving sep, we don't steal 0xff
|
||||
// byte, so we don't need to do any escaping:
|
||||
tsta = new TokenStreamToAutomaton();
|
||||
}
|
||||
tsta.setPreservePositionIncrements(preservePositionIncrements);
|
||||
tsta.setUnicodeArcs(unicodeAware);
|
||||
|
||||
automaton = tsta.toAutomaton(inputTokenStream);
|
||||
} finally {
|
||||
IOUtils.closeWhileHandlingException(inputTokenStream);
|
||||
}
|
||||
|
||||
// TODO: we can optimize this somewhat by determinizing
|
||||
// while we convert
|
||||
automaton = replaceSep(automaton, preserveSep, SEP_LABEL);
|
||||
// This automaton should not blow up during determinize:
|
||||
return Operations.determinize(automaton, maxGraphExpansions);
|
||||
}
|
||||
|
||||
/**
|
||||
* Just escapes the 0xff byte (which we still for SEP).
|
||||
*/
|
||||
private static final class EscapingTokenStreamToAutomaton extends TokenStreamToAutomaton {
|
||||
|
||||
final BytesRefBuilder spare = new BytesRefBuilder();
|
||||
private char sepLabel;
|
||||
|
||||
public EscapingTokenStreamToAutomaton(char sepLabel) {
|
||||
this.sepLabel = sepLabel;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BytesRef changeToken(BytesRef in) {
|
||||
int upto = 0;
|
||||
for (int i = 0; i < in.length; i++) {
|
||||
byte b = in.bytes[in.offset + i];
|
||||
if (b == (byte) sepLabel) {
|
||||
spare.grow(upto + 2);
|
||||
spare.setByteAt(upto++, (byte) sepLabel);
|
||||
spare.setByteAt(upto++, b);
|
||||
} else {
|
||||
spare.grow(upto + 1);
|
||||
spare.setByteAt(upto++, b);
|
||||
}
|
||||
}
|
||||
spare.setLength(upto);
|
||||
return spare.get();
|
||||
}
|
||||
}
|
||||
|
||||
// Replaces SEP with epsilon or remaps them if
|
||||
// we were asked to preserve them:
|
||||
private static Automaton replaceSep(Automaton a, boolean preserveSep, int sepLabel) {
|
||||
|
||||
Automaton result = new Automaton();
|
||||
|
||||
// Copy all states over
|
||||
int numStates = a.getNumStates();
|
||||
for (int s = 0; s < numStates; s++) {
|
||||
result.createState();
|
||||
result.setAccept(s, a.isAccept(s));
|
||||
}
|
||||
|
||||
// Go in reverse topo sort so we know we only have to
|
||||
// make one pass:
|
||||
Transition t = new Transition();
|
||||
int[] topoSortStates = Operations.topoSortStates(a);
|
||||
for (int i = 0; i < topoSortStates.length; i++) {
|
||||
int state = topoSortStates[topoSortStates.length - 1 - i];
|
||||
int count = a.initTransition(state, t);
|
||||
for (int j = 0; j < count; j++) {
|
||||
a.getNextTransition(t);
|
||||
if (t.min == TokenStreamToAutomaton.POS_SEP) {
|
||||
assert t.max == TokenStreamToAutomaton.POS_SEP;
|
||||
if (preserveSep) {
|
||||
// Remap to SEP_LABEL:
|
||||
result.addTransition(state, t.dest, sepLabel);
|
||||
} else {
|
||||
result.addEpsilon(state, t.dest);
|
||||
}
|
||||
} else if (t.min == TokenStreamToAutomaton.HOLE) {
|
||||
assert t.max == TokenStreamToAutomaton.HOLE;
|
||||
|
||||
// Just remove the hole: there will then be two
|
||||
// SEP tokens next to each other, which will only
|
||||
// match another hole at search time. Note that
|
||||
// it will also match an empty-string token ... if
|
||||
// that's somehow a problem we can always map HOLE
|
||||
// to a dedicated byte (and escape it in the
|
||||
// input).
|
||||
result.addEpsilon(state, t.dest);
|
||||
} else {
|
||||
result.addTransition(state, t.dest, t.min, t.max);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
result.finishState();
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Attribute providing access to the term builder and UTF-16 conversion
|
||||
*/
|
||||
public interface BytesRefBuilderTermAttribute extends TermToBytesRefAttribute {
|
||||
/**
|
||||
* Returns the builder from which the term is derived.
|
||||
*/
|
||||
BytesRefBuilder builder();
|
||||
|
||||
/**
|
||||
* Returns the term represented as UTF-16
|
||||
*/
|
||||
CharSequence toUTF16();
|
||||
}
|
||||
|
||||
/**
|
||||
* Custom attribute implementation for completion token stream
|
||||
*/
|
||||
public static final class BytesRefBuilderTermAttributeImpl extends AttributeImpl implements BytesRefBuilderTermAttribute, TermToBytesRefAttribute {
|
||||
private final BytesRefBuilder bytes = new BytesRefBuilder();
|
||||
private transient CharsRefBuilder charsRef;
|
||||
|
||||
/**
|
||||
* Sole constructor
|
||||
* no-op
|
||||
*/
|
||||
public BytesRefBuilderTermAttributeImpl() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRefBuilder builder() {
|
||||
return bytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytesRef() {
|
||||
return bytes.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
bytes.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copyTo(AttributeImpl target) {
|
||||
BytesRefBuilderTermAttributeImpl other = (BytesRefBuilderTermAttributeImpl) target;
|
||||
other.bytes.copyBytes(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AttributeImpl clone() {
|
||||
BytesRefBuilderTermAttributeImpl other = new BytesRefBuilderTermAttributeImpl();
|
||||
copyTo(other);
|
||||
return other;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reflectWith(AttributeReflector reflector) {
|
||||
reflector.reflect(TermToBytesRefAttribute.class, "bytes", getBytesRef());
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence toUTF16() {
|
||||
if (charsRef == null) {
|
||||
charsRef = new CharsRefBuilder();
|
||||
}
|
||||
charsRef.copyUTF8Bytes(getBytesRef());
|
||||
return charsRef.get();
|
||||
}
|
||||
return ((ConcatenateGraphFilter)input).toAutomaton(unicodeAware);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.Iterator;
|
|||
import java.util.Map;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.ScoreMode;
|
||||
import org.apache.lucene.search.Weight;
|
||||
|
@ -178,7 +179,7 @@ public class ContextQuery extends CompletionQuery {
|
|||
// if separators are preserved the fst contains a SEP_LABEL
|
||||
// behind each gap. To have a matching automaton, we need to
|
||||
// include the SEP_LABEL in the query as well
|
||||
Automaton optionalSepLabel = Operations.optional(Automata.makeChar(CompletionAnalyzer.SEP_LABEL));
|
||||
Automaton optionalSepLabel = Operations.optional(Automata.makeChar(ConcatenateGraphFilter.SEP_LABEL));
|
||||
Automaton prefixAutomaton = Operations.concatenate(optionalSepLabel, innerAutomaton);
|
||||
Automaton contextsAutomaton = Operations.concatenate(toContextAutomaton(contexts, matchAllContexts), prefixAutomaton);
|
||||
contextsAutomaton = Operations.determinize(contextsAutomaton, Operations.DEFAULT_MAX_DETERMINIZED_STATES);
|
||||
|
@ -302,7 +303,7 @@ public class ContextQuery extends CompletionQuery {
|
|||
}
|
||||
ref.offset = ++i;
|
||||
assert ref.offset < ref.length : "input should not end with the context separator";
|
||||
if (ref.ints[i] == CompletionAnalyzer.SEP_LABEL) {
|
||||
if (ref.ints[i] == ConcatenateGraphFilter.SEP_LABEL) {
|
||||
ref.offset++;
|
||||
assert ref.offset < ref.length : "input should not end with a context separator followed by SEP_LABEL";
|
||||
}
|
||||
|
|
|
@ -90,6 +90,7 @@ public class ContextSuggestField extends SuggestField {
|
|||
}
|
||||
CompletionTokenStream completionTokenStream;
|
||||
if (stream instanceof CompletionTokenStream) {
|
||||
//TODO this is awkward; is there a better way avoiding re-creating the chain?
|
||||
completionTokenStream = (CompletionTokenStream) stream;
|
||||
PrefixTokenFilter prefixTokenFilter = new PrefixTokenFilter(completionTokenStream.inputTokenStream, (char) CONTEXT_SEPARATOR, contexts);
|
||||
completionTokenStream = new CompletionTokenStream(prefixTokenFilter,
|
||||
|
|
|
@ -144,9 +144,12 @@ public class FuzzyCompletionQuery extends PrefixCompletionQuery {
|
|||
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
|
||||
CompletionTokenStream stream = (CompletionTokenStream) analyzer.tokenStream(getField(), getTerm().text());
|
||||
final Automaton originalAutomata;
|
||||
try (CompletionTokenStream stream = (CompletionTokenStream) analyzer.tokenStream(getField(), getTerm().text()) ) {
|
||||
originalAutomata = stream.toAutomaton(unicodeAware);
|
||||
}
|
||||
Set<IntsRef> refs = new HashSet<>();
|
||||
Automaton automaton = toLevenshteinAutomata(stream.toAutomaton(unicodeAware), refs);
|
||||
Automaton automaton = toLevenshteinAutomata(originalAutomata, refs);
|
||||
if (unicodeAware) {
|
||||
Automaton utf8automaton = new UTF32ToUTF8().convert(automaton);
|
||||
utf8automaton = Operations.determinize(utf8automaton, maxDeterminizedStates);
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.search.suggest.document;
|
|||
import java.io.IOException;
|
||||
import java.util.PriorityQueue;
|
||||
|
||||
import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
|
@ -42,7 +43,7 @@ final class NRTSuggesterBuilder {
|
|||
* Label used to separate surface form and docID
|
||||
* in the output
|
||||
*/
|
||||
public static final int PAYLOAD_SEP = '\u001F';
|
||||
public static final int PAYLOAD_SEP = ConcatenateGraphFilter.SEP_LABEL;
|
||||
|
||||
/**
|
||||
* Marks end of the analyzed input and start of dedup
|
||||
|
|
|
@ -68,8 +68,9 @@ public class PrefixCompletionQuery extends CompletionQuery {
|
|||
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
|
||||
CompletionTokenStream stream = (CompletionTokenStream) analyzer.tokenStream(getField(), getTerm().text());
|
||||
return new CompletionWeight(this, stream.toAutomaton());
|
||||
try (CompletionTokenStream stream = (CompletionTokenStream) analyzer.tokenStream(getField(), getTerm().text())) {
|
||||
return new CompletionWeight(this, stream.toAutomaton());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
|
@ -140,7 +141,7 @@ public class SuggestField extends Field {
|
|||
|
||||
private boolean isReserved(char c) {
|
||||
switch (c) {
|
||||
case CompletionAnalyzer.SEP_LABEL:
|
||||
case ConcatenateGraphFilter.SEP_LABEL:
|
||||
case CompletionAnalyzer.HOLE_CHARACTER:
|
||||
case NRTSuggesterBuilder.END_BYTE:
|
||||
return true;
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.ByteArrayOutputStream;
|
|||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
|
||||
import org.apache.lucene.analysis.standard.StandardAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
|
@ -109,21 +110,21 @@ public class TestContextSuggestField extends LuceneTestCase {
|
|||
CharsRefBuilder builder = new CharsRefBuilder();
|
||||
builder.append("context1");
|
||||
builder.append(((char) ContextSuggestField.CONTEXT_SEPARATOR));
|
||||
builder.append(((char) CompletionAnalyzer.SEP_LABEL));
|
||||
builder.append((char) ConcatenateGraphFilter.SEP_LABEL);
|
||||
builder.append("input");
|
||||
expectedOutputs[0] = builder.toCharsRef().toString();
|
||||
builder.clear();
|
||||
builder.append("context2");
|
||||
builder.append(((char) ContextSuggestField.CONTEXT_SEPARATOR));
|
||||
builder.append(((char) CompletionAnalyzer.SEP_LABEL));
|
||||
builder.append((char) ConcatenateGraphFilter.SEP_LABEL);
|
||||
builder.append("input");
|
||||
expectedOutputs[1] = builder.toCharsRef().toString();
|
||||
TokenStream stream = new CompletionTokenStreamTest.PayloadAttrToTypeAttrFilter(field.tokenStream(analyzer, null));
|
||||
assertTokenStreamContents(stream, expectedOutputs, null, null, new String[]{payload.utf8ToString(), payload.utf8ToString()}, new int[]{1, 1}, null, null);
|
||||
TokenStream stream = new TestSuggestField.PayloadAttrToTypeAttrFilter(field.tokenStream(analyzer, null));
|
||||
assertTokenStreamContents(stream, expectedOutputs, null, null, new String[]{payload.utf8ToString(), payload.utf8ToString()}, new int[]{1, 0}, null, null);
|
||||
|
||||
CompletionAnalyzer completionAnalyzer = new CompletionAnalyzer(analyzer);
|
||||
stream = new CompletionTokenStreamTest.PayloadAttrToTypeAttrFilter(field.tokenStream(completionAnalyzer, null));
|
||||
assertTokenStreamContents(stream, expectedOutputs, null, null, new String[]{payload.utf8ToString(), payload.utf8ToString()}, new int[]{1, 1}, null, null);
|
||||
stream = new TestSuggestField.PayloadAttrToTypeAttrFilter(field.tokenStream(completionAnalyzer, null));
|
||||
assertTokenStreamContents(stream, expectedOutputs, null, null, new String[]{payload.utf8ToString(), payload.utf8ToString()}, new int[]{1, 0}, null, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -32,7 +32,11 @@ import java.util.concurrent.CyclicBarrier;
|
|||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.miscellaneous.ConcatenateGraphFilter;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
|
||||
|
@ -99,7 +103,7 @@ public class TestSuggestField extends LuceneTestCase {
|
|||
public void testReservedChars() throws Exception {
|
||||
CharsRefBuilder charsRefBuilder = new CharsRefBuilder();
|
||||
charsRefBuilder.append("sugg");
|
||||
charsRefBuilder.setCharAt(2, (char) CompletionAnalyzer.SEP_LABEL);
|
||||
charsRefBuilder.setCharAt(2, (char) ConcatenateGraphFilter.SEP_LABEL);
|
||||
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
|
||||
new SuggestField("name", charsRefBuilder.toString(), 1);
|
||||
});
|
||||
|
@ -144,11 +148,11 @@ public class TestSuggestField extends LuceneTestCase {
|
|||
output.writeByte(SuggestField.TYPE);
|
||||
}
|
||||
BytesRef payload = new BytesRef(byteArrayOutputStream.toByteArray());
|
||||
TokenStream stream = new CompletionTokenStreamTest.PayloadAttrToTypeAttrFilter(suggestField.tokenStream(analyzer, null));
|
||||
TokenStream stream = new PayloadAttrToTypeAttrFilter(suggestField.tokenStream(analyzer, null));
|
||||
assertTokenStreamContents(stream, new String[] {"input"}, null, null, new String[]{payload.utf8ToString()}, new int[]{1}, null, null);
|
||||
|
||||
CompletionAnalyzer completionAnalyzer = new CompletionAnalyzer(analyzer);
|
||||
stream = new CompletionTokenStreamTest.PayloadAttrToTypeAttrFilter(suggestField.tokenStream(completionAnalyzer, null));
|
||||
stream = new PayloadAttrToTypeAttrFilter(suggestField.tokenStream(completionAnalyzer, null));
|
||||
assertTokenStreamContents(stream, new String[] {"input"}, null, null, new String[]{payload.utf8ToString()}, new int[]{1}, null, null);
|
||||
}
|
||||
|
||||
|
@ -894,4 +898,23 @@ public class TestSuggestField extends LuceneTestCase {
|
|||
iwc.setCodec(filterCodec);
|
||||
return iwc;
|
||||
}
|
||||
|
||||
public final static class PayloadAttrToTypeAttrFilter extends TokenFilter {
|
||||
private PayloadAttribute payload = addAttribute(PayloadAttribute.class);
|
||||
private TypeAttribute type = addAttribute(TypeAttribute.class);
|
||||
|
||||
protected PayloadAttrToTypeAttrFilter(TokenStream input) {
|
||||
super(input);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
if (input.incrementToken()) {
|
||||
// we move them over so we can assert them more easily in the tests
|
||||
type.setType(payload.getPayload().utf8ToString());
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -53,7 +53,7 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
|
|||
Directory dir = newDirectory();
|
||||
Codec codec = getCodec();
|
||||
SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
|
||||
FieldInfo fi = builder.getOrAdd("field");
|
||||
fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
|
||||
addAttributes(fi);
|
||||
|
@ -75,7 +75,7 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
|
|||
Directory dir = newDirectory();
|
||||
Codec codec = getCodec();
|
||||
SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
|
||||
FieldInfo fi = builder.getOrAdd("field");
|
||||
fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
|
||||
addAttributes(fi);
|
||||
|
@ -115,7 +115,7 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
|
|||
dir.failOn(fail);
|
||||
Codec codec = getCodec();
|
||||
SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
|
||||
FieldInfo fi = builder.getOrAdd("field");
|
||||
fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
|
||||
addAttributes(fi);
|
||||
|
@ -150,7 +150,7 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
|
|||
dir.failOn(fail);
|
||||
Codec codec = getCodec();
|
||||
SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
|
||||
FieldInfo fi = builder.getOrAdd("field");
|
||||
fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
|
||||
addAttributes(fi);
|
||||
|
@ -185,7 +185,7 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
|
|||
dir.failOn(fail);
|
||||
Codec codec = getCodec();
|
||||
SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
|
||||
FieldInfo fi = builder.getOrAdd("field");
|
||||
fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
|
||||
addAttributes(fi);
|
||||
|
@ -221,7 +221,7 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
|
|||
dir.failOn(fail);
|
||||
Codec codec = getCodec();
|
||||
SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
|
||||
FieldInfo fi = builder.getOrAdd("field");
|
||||
fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
|
||||
addAttributes(fi);
|
||||
|
@ -251,7 +251,7 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
|
|||
for (int i = 0; i < numFields; i++) {
|
||||
fieldNames.add(TestUtil.randomUnicodeString(random()));
|
||||
}
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder(new FieldInfos.FieldNumbers(null));
|
||||
for (String field : fieldNames) {
|
||||
IndexableFieldType fieldType = randomFieldType(random());
|
||||
FieldInfo fi = builder.getOrAdd(field);
|
||||
|
|
|
@ -323,7 +323,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
|
|||
FieldInfo proto = oneDocReader.getFieldInfos().fieldInfo("field");
|
||||
FieldInfo field = new FieldInfo(proto.name, proto.number, proto.hasVectors(), proto.omitsNorms(), proto.hasPayloads(),
|
||||
proto.getIndexOptions(), proto.getDocValuesType(), proto.getDocValuesGen(), new HashMap<>(),
|
||||
proto.getPointDimensionCount(), proto.getPointNumBytes());
|
||||
proto.getPointDimensionCount(), proto.getPointNumBytes(), proto.isSoftDeletesField());
|
||||
|
||||
FieldInfos fieldInfos = new FieldInfos(new FieldInfo[] { field } );
|
||||
|
||||
|
|
|
@ -125,10 +125,10 @@ public abstract class BaseLiveDocsFormatTestCase extends LuceneTestCase {
|
|||
final Directory dir = newDirectory();
|
||||
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "foo", maxDoc, random().nextBoolean(),
|
||||
codec, Collections.emptyMap(), StringHelper.randomId(), Collections.emptyMap(), null);
|
||||
SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, 0, -1, -1);
|
||||
SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, 0, 0, -1, -1);
|
||||
format.writeLiveDocs(bits, dir, sci, maxDoc - numLiveDocs, IOContext.DEFAULT);
|
||||
|
||||
sci = new SegmentCommitInfo(si, maxDoc - numLiveDocs, 1, -1, -1);
|
||||
sci = new SegmentCommitInfo(si, maxDoc - numLiveDocs, 0, 1, -1, -1);
|
||||
final Bits bits2 = format.readLiveDocs(dir, sci, IOContext.READONCE);
|
||||
assertEquals(maxDoc, bits2.length());
|
||||
for (int i = 0; i < maxDoc; ++i) {
|
||||
|
|
|
@ -116,7 +116,7 @@ public abstract class BaseMergePolicyTestCase extends LuceneTestCase {
|
|||
Collections.emptyMap(), // attributes
|
||||
null /* indexSort */);
|
||||
info.setFiles(Collections.emptyList());
|
||||
infos.add(new SegmentCommitInfo(info, random().nextInt(1), -1, -1, -1));
|
||||
infos.add(new SegmentCommitInfo(info, random().nextInt(1), 0, -1, -1, -1));
|
||||
}
|
||||
MergePolicy.MergeSpecification forcedDeletesMerges = mp.findForcedDeletesMerges(infos, context);
|
||||
if (forcedDeletesMerges != null) {
|
||||
|
|
|
@ -77,7 +77,8 @@ public class MismatchedLeafReader extends FilterLeafReader {
|
|||
oldInfo.getDocValuesGen(), // dvGen
|
||||
oldInfo.attributes(), // attributes
|
||||
oldInfo.getPointDimensionCount(), // dimension count
|
||||
oldInfo.getPointNumBytes()); // dimension numBytes
|
||||
oldInfo.getPointNumBytes(), // dimension numBytes
|
||||
oldInfo.isSoftDeletesField()); // used as soft-deletes field
|
||||
shuffled.set(i, newInfo);
|
||||
}
|
||||
|
||||
|
|
|
@ -130,7 +130,7 @@ public class RandomPostingsTester {
|
|||
fieldInfoArray[fieldUpto] = new FieldInfo(field, fieldUpto, false, false, true,
|
||||
IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
|
||||
DocValuesType.NONE, -1, new HashMap<>(),
|
||||
0, 0);
|
||||
0, 0, false);
|
||||
fieldUpto++;
|
||||
|
||||
SortedMap<BytesRef,SeedAndOrd> postings = new TreeMap<>();
|
||||
|
@ -651,7 +651,7 @@ public class RandomPostingsTester {
|
|||
DocValuesType.NONE,
|
||||
-1,
|
||||
new HashMap<>(),
|
||||
0, 0);
|
||||
0, 0, false);
|
||||
}
|
||||
|
||||
FieldInfos newFieldInfos = new FieldInfos(newFieldInfoArray);
|
||||
|
|
|
@ -804,7 +804,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
|
|||
}
|
||||
|
||||
// NOTE: This is off by default; see LUCENE-5574
|
||||
private boolean assertNoUnreferencedFilesOnClose;
|
||||
private volatile boolean assertNoUnreferencedFilesOnClose;
|
||||
|
||||
public void setAssertNoUnrefencedFilesOnClose(boolean v) {
|
||||
assertNoUnreferencedFilesOnClose = v;
|
||||
|
|
|
@ -70,8 +70,8 @@ Upgrade Notes
|
|||
To return the previous behavior pass false to skipCommitOnMasterVersionZero in slave section of replication
|
||||
handler configuration, or pass it to the fetchindex command.
|
||||
|
||||
* SOLR-11453: Configuring slowQueryThresholdMillis now logs slow requests to a separate file - solr_slow_requests.log .
|
||||
Previously they would get logged in the solr.xml file
|
||||
* SOLR-11453: Configuring slowQueryThresholdMillis now logs slow requests to a separate file - solr_slow_requests.log.
|
||||
Previously they would get logged in the solr.log file.
|
||||
|
||||
New Features
|
||||
----------------------
|
||||
|
@ -140,7 +140,7 @@ New Features
|
|||
* SOLR-12328: JSON Facet API: Domain change with graph query.
|
||||
(Daniel Meehl, Kevin Watters, yonik)
|
||||
|
||||
* SOLR-11453: Configuring slowQueryThresholdMillis logs slow requests to a separate file - solr_slow_requests.log .
|
||||
* SOLR-11453: Configuring slowQueryThresholdMillis logs slow requests to a separate file - solr_slow_requests.log.
|
||||
(Shawn Heisey, Remko Popma, Varun Thacker)
|
||||
|
||||
* SOLR-12401: Add getValue() and setValue() Stream Evaluators (Joel Bernstein, janhoy)
|
||||
|
@ -154,6 +154,9 @@ New Features
|
|||
|
||||
* SOLR-12389: support deeply nested json objects in clusterprops.json (noble)
|
||||
|
||||
* SOLR-12376: Added the TaggerRequestHandler (AKA SolrTextTagger) for tagging text. It's used as a component of
|
||||
NER/ERD systems including query-understanding. See the ref guide for more info. (David Smiley)
|
||||
|
||||
Bug Fixes
|
||||
----------------------
|
||||
|
||||
|
@ -283,13 +286,15 @@ Bug Fixes
|
|||
* SOLR-12374: SnapShooter.getIndexCommit can forget to decref the searcher; though it's not clear in practice when.
|
||||
(David Smiley)
|
||||
|
||||
* SOLR-12417: velocity response writer should enforce valid function name for v.json parameter (yonik)
|
||||
* SOLR-12417: velocity response writer should enforce valid function name for v.json parameter (Mano Kovacs, yonik)
|
||||
|
||||
* SOLR-12271: Fixed bug in how Analytics component reads negative values from float and double fields. (Houston Putman)
|
||||
|
||||
* SOLR-12433: Recovering flag of a replica is set equals to leader even it failed to receive update
|
||||
on recovering. (Cao Manh Dat)
|
||||
|
||||
* SOLR-12354: Register the /admin/info/key end-point at the startup time to avoid 404 (noble)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
|
||||
|
@ -325,6 +330,12 @@ Optimizations
|
|||
SolrConstantScoreQuery as well. QWF since v5.4.0 sometimes needlessly internally executed and cached the query.
|
||||
Affects ExpandComponent, ChildDocTransformer, CurrencyFieldType, TermsQParser. (David Smiley)
|
||||
|
||||
* SOLR-9922: Write buffering updates to another tlog. (Cao Manh Dat)
|
||||
|
||||
* SOLR-12233: QParserPlugin's built-in static registry now holds actual QParserPlugin instances instead of class
|
||||
references. This is consistent with other plugin registries and allows a SolrCore to load faster.
|
||||
(Jeff Miller, David Smiley)
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
|
||||
|
@ -1308,6 +1319,8 @@ Bug Fixes
|
|||
* SOLR-11477: Disallow resolving of external entities in the XML query parser (defType=xmlparser).
|
||||
(Michael Stepankin, Olga Barinova, Uwe Schindler, Christine Poerschke)
|
||||
|
||||
* SOLR-12444: Updating a cluster policy fails (noble)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -537,3 +537,17 @@ See http://www.restlet.org/
|
|||
Protocol Buffers - Google's data interchange format
|
||||
Copyright 2008 Google Inc.
|
||||
http://code.google.com/apis/protocolbuffers/
|
||||
|
||||
=========================================================================
|
||||
== SolrTextTagger Notice ==
|
||||
=========================================================================
|
||||
|
||||
The TaggerRequestHandler and related classes in its package came from the
|
||||
OpenSextant Solr Text Tagger,
|
||||
Copyright 2013 The MITRE Corporation. All Rights Reserved.
|
||||
|
||||
This software was produced for the U. S. Government
|
||||
under Contract No. W15P7T-11-C-F600, and is
|
||||
subject to the Rights in Noncommercial Computer Software
|
||||
and Noncommercial Computer Software Documentation
|
||||
Clause 252.227-7014 (JUN 1995)
|
|
@ -449,7 +449,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
|
|||
|
||||
// TODO: perhaps make this grab a new core each time through the loop to handle core reloads?
|
||||
final public void doSyncOrReplicateRecovery(SolrCore core) throws Exception {
|
||||
boolean replayed = false;
|
||||
boolean successfulRecovery = false;
|
||||
|
||||
UpdateLog ulog;
|
||||
|
@ -500,8 +499,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
|
|||
// when we went down. We may have received updates since then.
|
||||
recentVersions = startingVersions;
|
||||
try {
|
||||
if ((ulog.getStartingOperation() & UpdateLog.FLAG_GAP) != 0) {
|
||||
// last operation at the time of startup had the GAP flag set...
|
||||
if (ulog.existOldBufferLog()) {
|
||||
// this means we were previously doing a full index replication
|
||||
// that probably didn't complete and buffering updates in the
|
||||
// meantime.
|
||||
|
@ -542,9 +540,9 @@ public class RecoveryStrategy implements Runnable, Closeable {
|
|||
}
|
||||
|
||||
LOG.info("Begin buffering updates. core=[{}]", coreName);
|
||||
// recalling buffer updates will drop the old buffer tlog
|
||||
ulog.bufferUpdates();
|
||||
replayed = false;
|
||||
|
||||
|
||||
LOG.info("Publishing state of core [{}] as recovering, leader is [{}] and I am [{}]", core.getName(), leader.getCoreUrl(),
|
||||
ourUrl);
|
||||
zkController.publish(core.getCoreDescriptor(), Replica.State.RECOVERING);
|
||||
|
@ -603,8 +601,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
|
|||
|
||||
LOG.info("Replaying updates buffered during PeerSync.");
|
||||
replay(core);
|
||||
replayed = true;
|
||||
|
||||
|
||||
// sync success
|
||||
successfulRecovery = true;
|
||||
return;
|
||||
|
@ -630,8 +627,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
|
|||
}
|
||||
|
||||
replayFuture = replay(core);
|
||||
replayed = true;
|
||||
|
||||
|
||||
if (isClosed()) {
|
||||
LOG.info("RecoveryStrategy has been closed");
|
||||
break;
|
||||
|
@ -650,21 +646,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
|
|||
} catch (Exception e) {
|
||||
SolrException.log(LOG, "Error while trying to recover. core=" + coreName, e);
|
||||
} finally {
|
||||
if (!replayed) {
|
||||
// dropBufferedUpdate()s currently only supports returning to ACTIVE state, which risks additional updates
|
||||
// being added w/o UpdateLog.FLAG_GAP, hence losing the info on restart that we are not up-to-date.
|
||||
// For now, ulog will simply remain in BUFFERING state, and an additional call to bufferUpdates() will
|
||||
// reset our starting point for playback.
|
||||
LOG.info("Replay not started, or was not successful... still buffering updates.");
|
||||
|
||||
/** this prev code is retained in case we want to switch strategies.
|
||||
try {
|
||||
ulog.dropBufferedUpdates();
|
||||
} catch (Exception e) {
|
||||
SolrException.log(log, "", e);
|
||||
}
|
||||
**/
|
||||
}
|
||||
if (successfulRecovery) {
|
||||
LOG.info("Registering as Active after recovery.");
|
||||
try {
|
||||
|
|
|
@ -97,7 +97,7 @@ public class ReplicateFromLeader {
|
|||
new ModifiableSolrParams());
|
||||
CommitUpdateCommand cuc = new CommitUpdateCommand(req, false);
|
||||
cuc.setVersion(Long.parseLong(commitVersion));
|
||||
updateLog.copyOverOldUpdates(cuc);
|
||||
updateLog.commitAndSwitchToNewTlog(cuc);
|
||||
lastVersion = Long.parseLong(commitVersion);
|
||||
}
|
||||
});
|
||||
|
|
|
@ -168,7 +168,13 @@ public class ComputePlanAction extends TriggerActionBase {
|
|||
// estimate a maximum default limit that should be sufficient for most purposes:
|
||||
// number of nodes * total number of replicas * 3
|
||||
AtomicInteger totalRF = new AtomicInteger();
|
||||
clusterState.forEachCollection(coll -> totalRF.addAndGet(coll.getReplicationFactor() * coll.getSlices().size()));
|
||||
clusterState.forEachCollection(coll -> {
|
||||
Integer rf = coll.getReplicationFactor();
|
||||
if (rf == null) {
|
||||
rf = coll.getReplicas().size() / coll.getSlices().size();
|
||||
}
|
||||
totalRF.addAndGet(rf * coll.getSlices().size());
|
||||
});
|
||||
int totalMax = clusterState.getLiveNodes().size() * totalRF.get() * 3;
|
||||
int maxOp = (Integer) autoScalingConfig.getProperties().getOrDefault(AutoScalingParams.MAX_COMPUTE_OPERATIONS, totalMax);
|
||||
Object o = event.getProperty(AutoScalingParams.MAX_COMPUTE_OPERATIONS, maxOp);
|
||||
|
|
|
@ -16,20 +16,6 @@
|
|||
*/
|
||||
package org.apache.solr.core;
|
||||
|
||||
import static java.util.Objects.requireNonNull;
|
||||
import static org.apache.solr.common.params.CommonParams.AUTHC_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.AUTHZ_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.AUTOSCALING_HISTORY_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.HEALTH_CHECK_HANDLER_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.INFO_HANDLER_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.METRICS_HISTORY_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.METRICS_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.ZK_PATH;
|
||||
import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGIN_PROP;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.nio.file.Path;
|
||||
|
@ -64,15 +50,15 @@ import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder;
|
|||
import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder.AuthSchemeRegistryProvider;
|
||||
import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder.CredentialsProviderProvider;
|
||||
import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
|
||||
import org.apache.solr.cloud.autoscaling.AutoScalingHandler;
|
||||
import org.apache.solr.cloud.CloudDescriptor;
|
||||
import org.apache.solr.cloud.Overseer;
|
||||
import org.apache.solr.cloud.ZkController;
|
||||
import org.apache.solr.cloud.autoscaling.AutoScalingHandler;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.cloud.DocCollection;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.Replica.State;
|
||||
import org.apache.solr.common.cloud.DocCollection;
|
||||
import org.apache.solr.common.util.ExecutorUtil;
|
||||
import org.apache.solr.common.util.IOUtils;
|
||||
import org.apache.solr.common.util.Utils;
|
||||
|
@ -106,6 +92,7 @@ import org.apache.solr.security.AuthenticationPlugin;
|
|||
import org.apache.solr.security.AuthorizationPlugin;
|
||||
import org.apache.solr.security.HttpClientBuilderPlugin;
|
||||
import org.apache.solr.security.PKIAuthenticationPlugin;
|
||||
import org.apache.solr.security.PublicKeyHandler;
|
||||
import org.apache.solr.security.SecurityPluginHolder;
|
||||
import org.apache.solr.update.SolrCoreState;
|
||||
import org.apache.solr.update.UpdateShardHandler;
|
||||
|
@ -116,7 +103,20 @@ import org.apache.zookeeper.KeeperException;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static java.util.Objects.requireNonNull;
|
||||
import static org.apache.solr.common.params.CommonParams.AUTHC_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.AUTHZ_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.AUTOSCALING_HISTORY_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.HEALTH_CHECK_HANDLER_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.INFO_HANDLER_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.METRICS_HISTORY_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.METRICS_PATH;
|
||||
import static org.apache.solr.common.params.CommonParams.ZK_PATH;
|
||||
import static org.apache.solr.core.CorePropertiesLocator.PROPERTIES_FILENAME;
|
||||
import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGIN_PROP;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -301,6 +301,7 @@ public class CoreContainer {
|
|||
public CoreContainer(NodeConfig config, Properties properties, CoresLocator locator, boolean asyncSolrCoreLoad) {
|
||||
this.loader = config.getSolrResourceLoader();
|
||||
this.solrHome = loader.getInstancePath().toString();
|
||||
containerHandlers.put(PublicKeyHandler.PATH, new PublicKeyHandler());
|
||||
this.cfg = requireNonNull(config);
|
||||
this.coresLocator = locator;
|
||||
this.containerProperties = new Properties(properties);
|
||||
|
@ -548,7 +549,8 @@ public class CoreContainer {
|
|||
hostName = cfg.getNodeName();
|
||||
|
||||
zkSys.initZooKeeper(this, solrHome, cfg.getCloudConfig());
|
||||
if(isZooKeeperAware()) pkiAuthenticationPlugin = new PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName());
|
||||
if(isZooKeeperAware()) pkiAuthenticationPlugin = new PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName(),
|
||||
(PublicKeyHandler) containerHandlers.get(PublicKeyHandler.PATH));
|
||||
|
||||
MDCLoggingContext.setNode(this);
|
||||
|
||||
|
@ -592,8 +594,7 @@ public class CoreContainer {
|
|||
containerHandlers.put(AUTHZ_PATH, securityConfHandler);
|
||||
securityConfHandler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), metricTag, AUTHZ_PATH);
|
||||
containerHandlers.put(AUTHC_PATH, securityConfHandler);
|
||||
if(pkiAuthenticationPlugin != null)
|
||||
containerHandlers.put(PKIAuthenticationPlugin.PATH, pkiAuthenticationPlugin.getRequestHandler());
|
||||
|
||||
|
||||
PluginInfo[] metricReporters = cfg.getMetricsConfig().getMetricReporters();
|
||||
metricManager.loadReporters(metricReporters, loader, this, null, null, SolrInfoBean.Group.node);
|
||||
|
|
|
@ -958,7 +958,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
|
|||
initIndex(prev != null, reload);
|
||||
|
||||
initWriters();
|
||||
qParserPlugins.init(createInstances(QParserPlugin.standardPlugins), this);
|
||||
qParserPlugins.init(QParserPlugin.standardPlugins, this);
|
||||
valueSourceParsers.init(ValueSourceParser.standardValueSourceParsers, this);
|
||||
transformerFactories.init(TransformerFactory.defaultFactories, this);
|
||||
loadSearchComponents();
|
||||
|
|
|
@ -16,6 +16,10 @@
|
|||
*/
|
||||
package org.apache.solr.core;
|
||||
|
||||
import javax.naming.Context;
|
||||
import javax.naming.InitialContext;
|
||||
import javax.naming.NamingException;
|
||||
import javax.naming.NoInitialContextException;
|
||||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
|
@ -47,10 +51,6 @@ import java.util.concurrent.ConcurrentSkipListSet;
|
|||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
import javax.naming.Context;
|
||||
import javax.naming.InitialContext;
|
||||
import javax.naming.NamingException;
|
||||
import javax.naming.NoInitialContextException;
|
||||
|
||||
import org.apache.lucene.analysis.WordlistLoader;
|
||||
import org.apache.lucene.analysis.util.CharFilterFactory;
|
||||
|
@ -88,9 +88,9 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
|
|||
static final String project = "solr";
|
||||
static final String base = "org.apache" + "." + project;
|
||||
static final String[] packages = {
|
||||
"", "analysis.", "schema.", "handler.", "search.", "update.", "core.", "response.", "request.",
|
||||
"", "analysis.", "schema.", "handler.", "handler.tagger.", "search.", "update.", "core.", "response.", "request.",
|
||||
"update.processor.", "util.", "spelling.", "handler.component.", "handler.dataimport.",
|
||||
"spelling.suggest.", "spelling.suggest.fst.", "rest.schema.analysis.", "security.","handler.admin.",
|
||||
"spelling.suggest.", "spelling.suggest.fst.", "rest.schema.analysis.", "security.", "handler.admin.",
|
||||
"cloud.autoscaling."
|
||||
};
|
||||
private static final java.lang.String SOLR_CORE_NAME = "solr.core.name";
|
||||
|
|
|
@ -206,9 +206,10 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
|
|||
return this.coreContainer;
|
||||
}
|
||||
|
||||
protected void copyFromClusterProp(Map<String, Object> props, String prop) {
|
||||
protected void copyFromClusterProp(Map<String, Object> props, String prop) throws IOException {
|
||||
if (props.get(prop) != null) return;//if it's already specified , return
|
||||
Object defVal = coreContainer.getZkController().getZkStateReader().getClusterProperty(ImmutableList.of(COLLECTION_DEF, prop), null);
|
||||
Object defVal = new ClusterProperties(coreContainer.getZkController().getZkStateReader().getZkClient())
|
||||
.getClusterProperty(ImmutableList.of(COLLECTION_DEF, prop), null);
|
||||
if (defVal != null) props.put(prop, String.valueOf(defVal));
|
||||
}
|
||||
|
||||
|
|
|
@ -797,7 +797,8 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
|
|||
fieldInfo.getDocValuesGen(),
|
||||
fieldInfo.attributes(),
|
||||
fieldInfo.getPointDimensionCount(),
|
||||
fieldInfo.getPointNumBytes());
|
||||
fieldInfo.getPointNumBytes(),
|
||||
fieldInfo.isSoftDeletesField());
|
||||
newInfos.add(f);
|
||||
|
||||
} else {
|
||||
|
|
|
@ -0,0 +1,178 @@
|
|||
/*
|
||||
* This software was produced for the U. S. Government
|
||||
* under Contract No. W15P7T-11-C-F600, and is
|
||||
* subject to the Rights in Noncommercial Computer Software
|
||||
* and Noncommercial Computer Software Documentation
|
||||
* Clause 252.227-7014 (JUN 1995)
|
||||
*
|
||||
* Copyright 2013 The MITRE Corporation. All Rights Reserved.
|
||||
*
|
||||
* Licensed 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.solr.handler.tagger;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
import com.carrotsearch.hppc.IntArrayList;
|
||||
|
||||
public abstract class OffsetCorrector {
|
||||
|
||||
//TODO support a streaming style of consuming input text so that we need not take a
|
||||
// String. Trickier because we need to keep more information as we parse to know when tags
|
||||
// are adjacent with/without whitespace
|
||||
|
||||
//Data structure requirements:
|
||||
// Given a character offset:
|
||||
// * determine what tagId is it's parent.
|
||||
// * determine if it is adjacent to the parent open tag, ignoring whitespace
|
||||
// * determine if it is adjacent to the parent close tag, ignoring whitespace
|
||||
// Given a tagId:
|
||||
// * What is it's parent tagId
|
||||
// * What's the char offset of the start and end of the open tag
|
||||
// * What's the char offset of the start and end of the close tag
|
||||
|
||||
/** Document text. */
|
||||
protected final String docText;
|
||||
|
||||
/** Array of tag info comprised of 5 int fields:
|
||||
* [int parentTag, int openStartOff, int openEndOff, int closeStartOff, int closeEndOff].
|
||||
* It's size indicates how many tags there are. Tag's are ID'ed sequentially from 0. */
|
||||
protected final IntArrayList tagInfo;
|
||||
|
||||
/** offsets of parent tag id change (ascending order) */
|
||||
protected final IntArrayList parentChangeOffsets;
|
||||
/** tag id; parallel array to parentChangeOffsets */
|
||||
protected final IntArrayList parentChangeIds;
|
||||
|
||||
protected final int[] offsetPair = new int[] { -1, -1};//non-thread-safe state
|
||||
|
||||
/** Disjoint start and end span offsets (inclusive) of non-taggable sections. Null if none. */
|
||||
protected final IntArrayList nonTaggableOffsets;
|
||||
|
||||
/**
|
||||
* Initialize based on the document text.
|
||||
* @param docText non-null structured content.
|
||||
* @param hasNonTaggable if there may be "non-taggable" tags to track
|
||||
*/
|
||||
protected OffsetCorrector(String docText, boolean hasNonTaggable) {
|
||||
this.docText = docText;
|
||||
final int guessNumElements = Math.max(docText.length() / 20, 4);
|
||||
|
||||
tagInfo = new IntArrayList(guessNumElements * 5);
|
||||
parentChangeOffsets = new IntArrayList(guessNumElements * 2);
|
||||
parentChangeIds = new IntArrayList(guessNumElements * 2);
|
||||
nonTaggableOffsets = hasNonTaggable ? new IntArrayList(guessNumElements / 5) : null;
|
||||
}
|
||||
|
||||
/** Corrects the start and end offset pair. It will return null if it can't
|
||||
* due to a failure to keep the offsets balance-able, or if it spans "non-taggable" tags.
|
||||
* The start (left) offset is pulled left as needed over whitespace and opening tags. The end
|
||||
* (right) offset is pulled right as needed over whitespace and closing tags. It's returned as
|
||||
* a 2-element array.
|
||||
* <p>Note that the returned array is internally reused; just use it to examine the response.
|
||||
*/
|
||||
public int[] correctPair(int leftOffset, int rightOffset) {
|
||||
rightOffset = correctEndOffsetForCloseElement(rightOffset);
|
||||
if (spansNonTaggable(leftOffset, rightOffset))
|
||||
return null;
|
||||
|
||||
int startTag = lookupTag(leftOffset);
|
||||
//offsetPair[0] = Math.max(offsetPair[0], getOpenStartOff(startTag));
|
||||
int endTag = lookupTag(rightOffset-1);
|
||||
//offsetPair[1] = Math.min(offsetPair[1], getCloseStartOff(endTag));
|
||||
|
||||
// Find the ancestor tag enclosing offsetPair. And bump out left offset along the way.
|
||||
int iTag = startTag;
|
||||
for (; !tagEnclosesOffset(iTag, rightOffset); iTag = getParentTag(iTag)) {
|
||||
//Ensure there is nothing except whitespace thru OpenEndOff
|
||||
int tagOpenEndOff = getOpenEndOff(iTag);
|
||||
if (hasNonWhitespace(tagOpenEndOff, leftOffset))
|
||||
return null;
|
||||
leftOffset = getOpenStartOff(iTag);
|
||||
}
|
||||
final int ancestorTag = iTag;
|
||||
// Bump out rightOffset until we get to ancestorTag.
|
||||
for (iTag = endTag; iTag != ancestorTag; iTag = getParentTag(iTag)) {
|
||||
//Ensure there is nothing except whitespace thru CloseStartOff
|
||||
int tagCloseStartOff = getCloseStartOff(iTag);
|
||||
if (hasNonWhitespace(rightOffset, tagCloseStartOff))
|
||||
return null;
|
||||
rightOffset = getCloseEndOff(iTag);
|
||||
}
|
||||
|
||||
offsetPair[0] = leftOffset;
|
||||
offsetPair[1] = rightOffset;
|
||||
return offsetPair;
|
||||
}
|
||||
|
||||
/** Correct endOffset for adjacent element at the right side. E.g. offsetPair might point to:
|
||||
* <pre>
|
||||
* foo</tag>
|
||||
* </pre>
|
||||
* and this method pulls the end offset left to the '<'. This is necessary for use with
|
||||
* {@link org.apache.lucene.analysis.charfilter.HTMLStripCharFilter}.
|
||||
*
|
||||
* See https://issues.apache.org/jira/browse/LUCENE-5734 */
|
||||
protected int correctEndOffsetForCloseElement(int endOffset) {
|
||||
if (docText.charAt(endOffset-1) == '>') {
|
||||
final int newEndOffset = docText.lastIndexOf('<', endOffset - 2);
|
||||
if (newEndOffset > offsetPair[0])//just to be sure
|
||||
return newEndOffset;
|
||||
}
|
||||
return endOffset;
|
||||
}
|
||||
|
||||
protected boolean hasNonWhitespace(int start, int end) {
|
||||
for (int i = start; i < end; i++) {
|
||||
if (!Character.isWhitespace(docText.charAt(i)))
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
protected boolean tagEnclosesOffset(int tag, int off) {
|
||||
return off >= getOpenStartOff(tag) && off < getCloseEndOff(tag);
|
||||
}
|
||||
|
||||
protected int getParentTag(int tag) { return tagInfo.get(tag * 5 + 0); }
|
||||
protected int getOpenStartOff(int tag) { return tagInfo.get(tag * 5 + 1); }
|
||||
protected int getOpenEndOff(int tag) { return tagInfo.get(tag * 5 + 2); }
|
||||
protected int getCloseStartOff(int tag) { return tagInfo.get(tag * 5 + 3); }
|
||||
protected int getCloseEndOff(int tag) { return tagInfo.get(tag * 5 + 4); }
|
||||
|
||||
protected int lookupTag(int off) {
|
||||
int idx = Arrays.binarySearch(parentChangeOffsets.buffer, 0, parentChangeOffsets.size(), off);
|
||||
if (idx < 0)
|
||||
idx = (-idx - 1) - 1;//round down
|
||||
return parentChangeIds.get(idx);
|
||||
}
|
||||
|
||||
protected boolean spansNonTaggable(int startOff, int endOff) {
|
||||
if (nonTaggableOffsets == null)
|
||||
return false;
|
||||
int idx = Arrays.binarySearch(nonTaggableOffsets.buffer, 0, nonTaggableOffsets.size(), startOff);
|
||||
//if tag start coincides with first or last char of non-taggable span then result is true.
|
||||
// (probably never happens since those characters are actual element markup)
|
||||
if (idx >= 0)
|
||||
return true;
|
||||
idx = -idx - 1;//modify for where we would insert
|
||||
//if idx is odd then our span intersects a non-taggable span; return true
|
||||
if ((idx & 1) == 1)
|
||||
return true;
|
||||
//it's non-taggable if the next non-taggable start span is before our endOff
|
||||
if (idx == nonTaggableOffsets.size())
|
||||
return false;
|
||||
return nonTaggableOffsets.get(idx) < endOff;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,103 @@
|
|||
/*
|
||||
* This software was produced for the U. S. Government
|
||||
* under Contract No. W15P7T-11-C-F600, and is
|
||||
* subject to the Rights in Noncommercial Computer Software
|
||||
* and Noncommercial Computer Software Documentation
|
||||
* Clause 252.227-7014 (JUN 1995)
|
||||
*
|
||||
* Copyright 2013 The MITRE Corporation. All Rights Reserved.
|
||||
*
|
||||
* Licensed 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.solr.handler.tagger;
|
||||
|
||||
public interface TagClusterReducer {
|
||||
/**
|
||||
* Reduces the linked-list to only those tags that should be emitted
|
||||
* @param head not null; 1-element array to head which isn't null either
|
||||
*/
|
||||
void reduce(TagLL[] head);
|
||||
|
||||
static final TagClusterReducer ALL = new TagClusterReducer() {
|
||||
@Override
|
||||
public void reduce(TagLL[] head) {
|
||||
}
|
||||
};
|
||||
|
||||
static final TagClusterReducer NO_SUB = new TagClusterReducer() {
|
||||
@Override
|
||||
public void reduce(TagLL[] head) {
|
||||
//loop forward over all tags
|
||||
for (TagLL tag = head[0].nextTag; tag != null; tag = tag.nextTag) {
|
||||
//loop backwards over prev tags from this tag
|
||||
for (TagLL tPrev = tag.prevTag; tPrev != null; tPrev = tPrev.prevTag) {
|
||||
assert tPrev.startOffset <= tag.startOffset;
|
||||
//if a previous tag's endOffset is <= this one's, tForward can be removed
|
||||
if (tPrev.endOffset >= tag.endOffset) {
|
||||
tag.removeLL();
|
||||
break;
|
||||
} else if (tPrev.startOffset == tag.startOffset) {
|
||||
tPrev.removeLL();
|
||||
//continue; 'tag' is still valid
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
static final TagClusterReducer LONGEST_DOMINANT_RIGHT = new TagClusterReducer() {
|
||||
@Override
|
||||
public void reduce(TagLL[] head) {
|
||||
|
||||
//--Optimize for common single-tag case
|
||||
if (head[0].nextTag == null)
|
||||
return;
|
||||
|
||||
while (true) {
|
||||
//--Find longest not already marked
|
||||
TagLL longest = null;
|
||||
for (TagLL t = head[0]; t != null; t = t.nextTag) {
|
||||
if (!t.mark && (longest == null || t.charLen() >= longest.charLen()))
|
||||
longest = t;
|
||||
}
|
||||
if (longest == null)
|
||||
break;
|
||||
//--Mark longest (so we return it eventually)
|
||||
longest.mark = true;
|
||||
//--Remove tags overlapping this longest
|
||||
for (TagLL t = head[0]; t != null; t = t.nextTag) {
|
||||
if (t.mark)
|
||||
continue;
|
||||
|
||||
if (t.overlaps(longest)) {
|
||||
t.removeLL();
|
||||
} else if (t.startOffset >= longest.endOffset) {
|
||||
break;//no subsequent can possibly overlap
|
||||
}
|
||||
}
|
||||
}//loop
|
||||
|
||||
//all-remaining should be marked
|
||||
// for (TagLL t = head; t != null; t = t.nextTag) {
|
||||
// assert t.mark;
|
||||
//// if (!t.mark) {
|
||||
//// t.removeLL();
|
||||
//// if (head == t)
|
||||
//// head = t.nextTag;
|
||||
//// }
|
||||
// }
|
||||
assert head[0].mark;
|
||||
}
|
||||
};
|
||||
}
|
|
@ -0,0 +1,176 @@
|
|||
/*
|
||||
* This software was produced for the U. S. Government
|
||||
* under Contract No. W15P7T-11-C-F600, and is
|
||||
* subject to the Rights in Noncommercial Computer Software
|
||||
* and Noncommercial Computer Software Documentation
|
||||
* Clause 252.227-7014 (JUN 1995)
|
||||
*
|
||||
* Copyright 2013 The MITRE Corporation. All Rights Reserved.
|
||||
*
|
||||
* Licensed 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.solr.handler.tagger;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* This is a Tag -- a startOffset, endOffset and value.
|
||||
* <p>
|
||||
* A Tag starts without a value in an
|
||||
* "advancing" state. {@link #advance(org.apache.lucene.util.BytesRef, int)}
|
||||
* is called with subsequent words and then eventually it won't advance any
|
||||
* more, and value is set (could be null).
|
||||
* <p>
|
||||
* A Tag is also a doubly-linked-list (hence the LL in the name). All tags share
|
||||
* a reference to the head via a 1-element array, which is potentially modified
|
||||
* if any of the linked-list methods are called. Tags in the list should have
|
||||
* equal or increasing start offsets.
|
||||
*/
|
||||
public class TagLL{
|
||||
|
||||
private final TagLL[] head;//a shared pointer to the head; 1 element
|
||||
TagLL prevTag, nextTag; // linked list
|
||||
|
||||
private TermPrefixCursor cursor;
|
||||
|
||||
final int startOffset;//inclusive
|
||||
int endOffset;//exclusive
|
||||
Object value;//null means unset
|
||||
|
||||
/** optional boolean used by some TagClusterReducer's */
|
||||
boolean mark = false;
|
||||
|
||||
TagLL(TagLL[] head, TermPrefixCursor cursor, int startOffset, int endOffset, Object value) {
|
||||
this.head = head;
|
||||
this.cursor = cursor;
|
||||
this.startOffset = startOffset;
|
||||
this.endOffset = endOffset;
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Advances this tag with "word" at offset "offset". If this tag is not in
|
||||
* an advancing state then it does nothing. If it is advancing and prior to
|
||||
* advancing further it sees a value, then a non-advancing tag may be inserted
|
||||
* into the LL as side-effect. If this returns false (it didn't advance) and
|
||||
* if there is no value, then it will also be removed.
|
||||
*
|
||||
*
|
||||
* @param word The next word or null if at an end
|
||||
* @param offset The last character in word's offset in the underlying
|
||||
* stream. If word is null then it's meaningless.
|
||||
*
|
||||
* @return Whether it advanced or not.
|
||||
*/
|
||||
boolean advance(BytesRef word, int offset) throws IOException {
|
||||
if (!isAdvancing())
|
||||
return false;
|
||||
|
||||
Object iVal = cursor.getDocIds();
|
||||
|
||||
if (word != null && cursor.advance(word)) {
|
||||
|
||||
if (iVal != null) {
|
||||
addBeforeLL(new TagLL(head, null, startOffset, endOffset, iVal));
|
||||
}
|
||||
|
||||
assert offset >= endOffset;
|
||||
endOffset = offset;
|
||||
return true;
|
||||
} else {
|
||||
this.value = iVal;
|
||||
this.cursor = null;
|
||||
if (iVal == null)
|
||||
removeLL();
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/** Removes this tag from the chain, connecting prevTag and nextTag. Does not
|
||||
* modify "this" object's pointers, so the caller can refer to nextTag after
|
||||
* removing it. */
|
||||
public void removeLL() {
|
||||
if (head[0] == this)
|
||||
head[0] = nextTag;
|
||||
if (prevTag != null) {
|
||||
prevTag.nextTag = nextTag;
|
||||
}
|
||||
if (nextTag != null) {
|
||||
nextTag.prevTag = prevTag;
|
||||
}
|
||||
}
|
||||
|
||||
void addBeforeLL(TagLL tag) {
|
||||
assert tag.startOffset <= startOffset;
|
||||
if (prevTag != null) {
|
||||
assert prevTag.startOffset <= tag.startOffset;
|
||||
prevTag.nextTag = tag;
|
||||
tag.prevTag = prevTag;
|
||||
} else {
|
||||
assert head[0] == this;
|
||||
head[0] = tag;
|
||||
}
|
||||
prevTag = tag;
|
||||
tag.nextTag = this;
|
||||
}
|
||||
|
||||
void addAfterLL(TagLL tag) {
|
||||
assert tag.startOffset >= startOffset;
|
||||
if (nextTag != null) {
|
||||
assert nextTag.startOffset >= tag.startOffset;
|
||||
nextTag.prevTag = tag;
|
||||
tag.nextTag = nextTag;
|
||||
}
|
||||
nextTag = tag;
|
||||
tag.prevTag = this;
|
||||
}
|
||||
|
||||
public int charLen() {
|
||||
return endOffset - startOffset;
|
||||
}
|
||||
|
||||
public TagLL getNextTag() {
|
||||
return nextTag;
|
||||
}
|
||||
|
||||
public TagLL getPrevTag() {
|
||||
return prevTag;
|
||||
}
|
||||
|
||||
public int getStartOffset() {
|
||||
return startOffset;
|
||||
}
|
||||
public int getEndOffset() {
|
||||
return endOffset;
|
||||
}
|
||||
public boolean overlaps(TagLL other) {
|
||||
//don't use >= or <= because startOffset is inclusive while endOffset is exclusive
|
||||
if (startOffset < other.startOffset)
|
||||
return endOffset > other.startOffset;
|
||||
else
|
||||
return startOffset < other.endOffset;
|
||||
}
|
||||
|
||||
boolean isAdvancing() {
|
||||
return cursor != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return (prevTag != null ? '*' : '-') + "|" + (nextTag != null ? '*' : '-') +
|
||||
" " + startOffset + " to " + endOffset + (isAdvancing() ? '+' : " #" + value);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,230 @@
|
|||
/*
|
||||
* This software was produced for the U. S. Government
|
||||
* under Contract No. W15P7T-11-C-F600, and is
|
||||
* subject to the Rights in Noncommercial Computer Software
|
||||
* and Noncommercial Computer Software Documentation
|
||||
* Clause 252.227-7014 (JUN 1995)
|
||||
*
|
||||
* Copyright 2013 The MITRE Corporation. All Rights Reserved.
|
||||
*
|
||||
* Licensed 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.solr.handler.tagger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Tags maximum string of words in a corpus. This is a callback-style API
|
||||
* in which you implement {@link #tagCallback(int, int, Object)}.
|
||||
*
|
||||
* This class should be independently usable outside Solr.
|
||||
*/
|
||||
public abstract class Tagger {
|
||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||
|
||||
private final TokenStream tokenStream;
|
||||
private final TermToBytesRefAttribute byteRefAtt;
|
||||
private final PositionIncrementAttribute posIncAtt;
|
||||
private final OffsetAttribute offsetAtt;
|
||||
private final TaggingAttribute taggingAtt;
|
||||
|
||||
private final TagClusterReducer tagClusterReducer;
|
||||
private final Terms terms;
|
||||
private final Bits liveDocs;
|
||||
private final boolean skipAltTokens;
|
||||
private final boolean ignoreStopWords;
|
||||
|
||||
private Map<BytesRef, IntsRef> docIdsCache;
|
||||
|
||||
/** Whether the WARNING about skipped tokens was already logged. */
|
||||
private boolean loggedSkippedAltTokenWarning = false;
|
||||
|
||||
public Tagger(Terms terms, Bits liveDocs, TokenStream tokenStream,
|
||||
TagClusterReducer tagClusterReducer, boolean skipAltTokens,
|
||||
boolean ignoreStopWords) throws IOException {
|
||||
this.terms = terms;
|
||||
this.liveDocs = liveDocs;
|
||||
this.tokenStream = tokenStream;
|
||||
this.skipAltTokens = skipAltTokens;
|
||||
this.ignoreStopWords = ignoreStopWords;
|
||||
byteRefAtt = tokenStream.addAttribute(TermToBytesRefAttribute.class);
|
||||
posIncAtt = tokenStream.addAttribute(PositionIncrementAttribute.class);
|
||||
offsetAtt = tokenStream.addAttribute(OffsetAttribute.class);
|
||||
taggingAtt = tokenStream.addAttribute(TaggingAttribute.class);
|
||||
tokenStream.reset();
|
||||
|
||||
this.tagClusterReducer = tagClusterReducer;
|
||||
}
|
||||
|
||||
public void enableDocIdsCache(int initSize) {
|
||||
if (initSize > 0)
|
||||
docIdsCache = new HashMap<>(initSize);
|
||||
}
|
||||
|
||||
public void process() throws IOException {
|
||||
if (terms == null)
|
||||
return;
|
||||
|
||||
//a shared pointer to the head used by this method and each Tag instance.
|
||||
final TagLL[] head = new TagLL[1];
|
||||
|
||||
TermPrefixCursor cursor = null;//re-used
|
||||
|
||||
//boolean switch used to log warnings in case tokens where skipped during tagging.
|
||||
boolean skippedTokens = false;
|
||||
|
||||
while (tokenStream.incrementToken()) {
|
||||
if (log.isTraceEnabled()) {
|
||||
log.trace("Token: {}, posInc: {}, offset: [{},{}]",
|
||||
byteRefAtt, posIncAtt.getPositionIncrement(),
|
||||
offsetAtt.startOffset(), offsetAtt.endOffset());
|
||||
}
|
||||
//check for posInc < 1 (alternate Tokens, such as expanded Synonyms)
|
||||
if (posIncAtt.getPositionIncrement() < 1) {
|
||||
//(a) Deal with this as a configuration issue and throw an exception
|
||||
if (!skipAltTokens) {
|
||||
//TODO throw UnsupportedTokenException when PhraseBuilder is ported
|
||||
throw new IllegalStateException("Query Analyzer generates alternate "
|
||||
+ "Tokens (posInc == 0). Please adapt your Analyzer configuration or "
|
||||
+ "enable '" + TaggerRequestHandler.SKIP_ALT_TOKENS + "' to skip such "
|
||||
+ "tokens. NOTE: enabling '" + TaggerRequestHandler.SKIP_ALT_TOKENS
|
||||
+ "' might result in wrong tagging results if the index time analyzer "
|
||||
+ "is not configured accordingly. For detailed information see "
|
||||
+ "https://github.com/OpenSextant/SolrTextTagger/pull/11#issuecomment-24936225");
|
||||
} else {
|
||||
//(b) In case the index time analyser had indexed all variants (users
|
||||
// need to ensure that) processing of alternate tokens can be skipped
|
||||
// as anyways all alternatives will be contained in the FST.
|
||||
skippedTokens = true;
|
||||
log.trace(" ... ignored token");
|
||||
continue;
|
||||
}
|
||||
}
|
||||
//-- If PositionIncrement > 1 (stopwords)
|
||||
if (!ignoreStopWords && posIncAtt.getPositionIncrement() > 1) {
|
||||
log.trace(" - posInc > 1 ... mark cluster as done");
|
||||
advanceTagsAndProcessClusterIfDone(head, null);
|
||||
}
|
||||
|
||||
final BytesRef term;
|
||||
//NOTE: we need to lookup tokens if
|
||||
// * the LookupAtt is true OR
|
||||
// * there are still advancing tags (to find the longest possible match)
|
||||
if(taggingAtt.isTaggable() || head[0] != null){
|
||||
//-- Lookup the term id from the next token
|
||||
term = byteRefAtt.getBytesRef();
|
||||
if (term.length == 0) {
|
||||
throw new IllegalArgumentException("term: " + term.utf8ToString() + " analyzed to a zero-length token");
|
||||
}
|
||||
} else { //no current cluster AND lookup == false ...
|
||||
term = null; //skip this token
|
||||
}
|
||||
|
||||
//-- Process tag
|
||||
advanceTagsAndProcessClusterIfDone(head, term);
|
||||
|
||||
//-- only create new Tags for Tokens we need to lookup
|
||||
if (taggingAtt.isTaggable() && term != null) {
|
||||
|
||||
//determine if the terms index has a term starting with the provided term
|
||||
// TODO create a pool of these cursors to reuse them more? could be trivial impl
|
||||
if (cursor == null)// (else the existing cursor will be re-used)
|
||||
cursor = new TermPrefixCursor(terms.iterator(), liveDocs, docIdsCache);
|
||||
if (cursor.advance(term)) {
|
||||
TagLL newTail = new TagLL(head, cursor, offsetAtt.startOffset(), offsetAtt.endOffset(), null);
|
||||
cursor = null;//because the new tag now "owns" this instance
|
||||
//and add it to the end
|
||||
if (head[0] == null) {
|
||||
head[0] = newTail;
|
||||
} else {
|
||||
for (TagLL t = head[0]; true; t = t.nextTag) {
|
||||
if (t.nextTag == null) {
|
||||
t.addAfterLL(newTail);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}//if termId >= 0
|
||||
}//end while(incrementToken())
|
||||
|
||||
//-- Finish all tags
|
||||
advanceTagsAndProcessClusterIfDone(head, null);
|
||||
assert head[0] == null;
|
||||
|
||||
if(!loggedSkippedAltTokenWarning && skippedTokens){
|
||||
loggedSkippedAltTokenWarning = true; //only log once
|
||||
log.warn("The Tagger skipped some alternate tokens (tokens with posInc == 0) "
|
||||
+ "while processing text. This may cause problems with some Analyzer "
|
||||
+ "configurations (e.g. query time synonym expansion). For details see "
|
||||
+ "https://github.com/OpenSextant/SolrTextTagger/pull/11#issuecomment-24936225");
|
||||
}
|
||||
|
||||
tokenStream.end();
|
||||
//tokenStream.close(); caller closes because caller acquired it
|
||||
}
|
||||
|
||||
private void advanceTagsAndProcessClusterIfDone(TagLL[] head, BytesRef term) throws IOException {
|
||||
//-- Advance tags
|
||||
final int endOffset = term != null ? offsetAtt.endOffset() : -1;
|
||||
boolean anyAdvance = false;
|
||||
for (TagLL t = head[0]; t != null; t = t.nextTag) {
|
||||
anyAdvance |= t.advance(term, endOffset);
|
||||
}
|
||||
|
||||
//-- Process cluster if done
|
||||
if (!anyAdvance && head[0] != null) {
|
||||
tagClusterReducer.reduce(head);
|
||||
for (TagLL t = head[0]; t != null; t = t.nextTag) {
|
||||
assert t.value != null;
|
||||
tagCallback(t.startOffset, t.endOffset, t.value);
|
||||
}
|
||||
head[0] = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Invoked by {@link #process()} for each tag found. endOffset is always >= the endOffset
|
||||
* given in the previous call.
|
||||
*
|
||||
* @param startOffset The character offset of the original stream where the tag starts.
|
||||
* @param endOffset One more than the character offset of the original stream where the tag ends.
|
||||
* @param docIdsKey A reference to the matching docIds that can be resolved via {@link #lookupDocIds(Object)}.
|
||||
*/
|
||||
protected abstract void tagCallback(int startOffset, int endOffset, Object docIdsKey);
|
||||
|
||||
/**
|
||||
* Returns a sorted array of integer docIds given the corresponding key.
|
||||
* @param docIdsKey The lookup key.
|
||||
* @return Not null
|
||||
*/
|
||||
protected IntsRef lookupDocIds(Object docIdsKey) {
|
||||
return (IntsRef) docIdsKey;
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,397 @@
|
|||
/*
|
||||
* This software was produced for the U. S. Government
|
||||
* under Contract No. W15P7T-11-C-F600, and is
|
||||
* subject to the Rights in Noncommercial Computer Software
|
||||
* and Noncommercial Computer Software Documentation
|
||||
* Clause 252.227-7014 (JUN 1995)
|
||||
*
|
||||
* Copyright 2013 The MITRE Corporation. All Rights Reserved.
|
||||
*
|
||||
* Licensed 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.solr.handler.tagger;
|
||||
|
||||
import javax.xml.stream.XMLStreamException;
|
||||
import java.io.IOException;
|
||||
import java.io.Reader;
|
||||
import java.io.StringReader;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Callable;
|
||||
|
||||
import com.google.common.io.CharStreams;
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.core.StopFilterFactory;
|
||||
import org.apache.lucene.analysis.util.TokenFilterFactory;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.ReaderUtil;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.BitSetIterator;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.solr.analysis.TokenizerChain;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.ContentStream;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.handler.RequestHandlerBase;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.response.SolrQueryResponse;
|
||||
import org.apache.solr.schema.FieldType;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.search.BitDocSet;
|
||||
import org.apache.solr.search.DocList;
|
||||
import org.apache.solr.search.DocSet;
|
||||
import org.apache.solr.search.DocSlice;
|
||||
import org.apache.solr.search.QParser;
|
||||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.search.SolrReturnFields;
|
||||
import org.apache.solr.search.SyntaxError;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Scans posted text, looking for matching strings in the Solr index.
|
||||
* The public static final String members are request parameters.
|
||||
* This handler is also called the "SolrTextTagger".
|
||||
*
|
||||
* @since 7.4.0
|
||||
*/
|
||||
public class TaggerRequestHandler extends RequestHandlerBase {
|
||||
|
||||
/** Request parameter. */
|
||||
public static final String OVERLAPS = "overlaps";
|
||||
/** Request parameter. */
|
||||
public static final String TAGS_LIMIT = "tagsLimit";
|
||||
/** Request parameter. */
|
||||
public static final String MATCH_TEXT = "matchText";
|
||||
/** Request parameter. */
|
||||
public static final String SKIP_ALT_TOKENS = "skipAltTokens";
|
||||
/** Request parameter. */
|
||||
public static final String IGNORE_STOPWORDS = "ignoreStopwords";
|
||||
/** Request parameter. */
|
||||
public static final String XML_OFFSET_ADJUST = "xmlOffsetAdjust";
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Processes input text to find matching tokens stored in the index.";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
|
||||
|
||||
//--Read params
|
||||
final String indexedField = req.getParams().get("field");
|
||||
if (indexedField == null)
|
||||
throw new RuntimeException("required param 'field'");
|
||||
|
||||
final TagClusterReducer tagClusterReducer =
|
||||
chooseTagClusterReducer(req.getParams().get(OVERLAPS));
|
||||
final int rows = req.getParams().getInt(CommonParams.ROWS, 10000);
|
||||
final int tagsLimit = req.getParams().getInt(TAGS_LIMIT, 1000);
|
||||
final boolean addMatchText = req.getParams().getBool(MATCH_TEXT, false);
|
||||
final SchemaField idSchemaField = req.getSchema().getUniqueKeyField();
|
||||
if (idSchemaField == null) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The tagger requires a" +
|
||||
"uniqueKey in the schema.");//TODO this could be relaxed
|
||||
}
|
||||
final boolean skipAltTokens = req.getParams().getBool(SKIP_ALT_TOKENS, false);
|
||||
final boolean ignoreStopWords = req.getParams().getBool(IGNORE_STOPWORDS,
|
||||
fieldHasIndexedStopFilter(indexedField, req));
|
||||
|
||||
//--Get posted data
|
||||
Reader inputReader = null;
|
||||
Iterable<ContentStream> streams = req.getContentStreams();
|
||||
if (streams != null) {
|
||||
Iterator<ContentStream> iter = streams.iterator();
|
||||
if (iter.hasNext()) {
|
||||
inputReader = iter.next().getReader();
|
||||
}
|
||||
if (iter.hasNext()) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
getClass().getSimpleName()+" does not support multiple ContentStreams"); //TODO support bulk tagging?
|
||||
}
|
||||
}
|
||||
if (inputReader == null) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
getClass().getSimpleName()+" requires text to be POSTed to it");
|
||||
}
|
||||
|
||||
// We may or may not need to read the input into a string
|
||||
final InputStringLazy inputStringFuture = new InputStringLazy(inputReader);
|
||||
|
||||
final OffsetCorrector offsetCorrector = getOffsetCorrector(req.getParams(), inputStringFuture);
|
||||
|
||||
final String inputString;//only populated if needed
|
||||
if (addMatchText || inputStringFuture.inputString != null) {
|
||||
//Read the input fully into a String buffer that we'll need later,
|
||||
// then replace the input with a reader wrapping the buffer.
|
||||
inputString = inputStringFuture.call();
|
||||
inputReader.close();
|
||||
inputReader = new StringReader(inputString);
|
||||
} else {
|
||||
inputString = null;//not used
|
||||
}
|
||||
|
||||
final SolrIndexSearcher searcher = req.getSearcher();
|
||||
final FixedBitSet matchDocIdsBS = new FixedBitSet(searcher.maxDoc());
|
||||
final List tags = new ArrayList(2000);
|
||||
|
||||
try {
|
||||
Analyzer analyzer = req.getSchema().getField(indexedField).getType().getQueryAnalyzer();
|
||||
try (TokenStream tokenStream = analyzer.tokenStream("", inputReader)) {
|
||||
Terms terms = searcher.getSlowAtomicReader().terms(indexedField);
|
||||
if (terms == null)
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
"field " + indexedField + " has no indexed data");
|
||||
Tagger tagger = new Tagger(terms, computeDocCorpus(req), tokenStream, tagClusterReducer,
|
||||
skipAltTokens, ignoreStopWords) {
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
protected void tagCallback(int startOffset, int endOffset, Object docIdsKey) {
|
||||
if (tags.size() >= tagsLimit)
|
||||
return;
|
||||
if (offsetCorrector != null) {
|
||||
int[] offsetPair = offsetCorrector.correctPair(startOffset, endOffset);
|
||||
if (offsetPair == null) {
|
||||
log.debug("Discarded offsets [{}, {}] because couldn't balance XML.",
|
||||
startOffset, endOffset);
|
||||
return;
|
||||
}
|
||||
startOffset = offsetPair[0];
|
||||
endOffset = offsetPair[1];
|
||||
}
|
||||
|
||||
NamedList tag = new NamedList();
|
||||
tag.add("startOffset", startOffset);
|
||||
tag.add("endOffset", endOffset);
|
||||
if (addMatchText)
|
||||
tag.add("matchText", inputString.substring(startOffset, endOffset));
|
||||
//below caches, and also flags matchDocIdsBS
|
||||
tag.add("ids", lookupSchemaDocIds(docIdsKey));
|
||||
tags.add(tag);
|
||||
}
|
||||
|
||||
Map<Object, List> docIdsListCache = new HashMap<>(2000);
|
||||
|
||||
ValueSourceAccessor uniqueKeyCache = new ValueSourceAccessor(searcher,
|
||||
idSchemaField.getType().getValueSource(idSchemaField, null));
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private List lookupSchemaDocIds(Object docIdsKey) {
|
||||
List schemaDocIds = docIdsListCache.get(docIdsKey);
|
||||
if (schemaDocIds != null)
|
||||
return schemaDocIds;
|
||||
IntsRef docIds = lookupDocIds(docIdsKey);
|
||||
//translate lucene docIds to schema ids
|
||||
schemaDocIds = new ArrayList(docIds.length);
|
||||
for (int i = docIds.offset; i < docIds.offset + docIds.length; i++) {
|
||||
int docId = docIds.ints[i];
|
||||
assert i == docIds.offset || docIds.ints[i - 1] < docId : "not sorted?";
|
||||
matchDocIdsBS.set(docId);//also, flip docid in bitset
|
||||
try {
|
||||
schemaDocIds.add(uniqueKeyCache.objectVal(docId));//translates here
|
||||
} catch (IOException e) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
|
||||
}
|
||||
}
|
||||
assert !schemaDocIds.isEmpty();
|
||||
|
||||
docIdsListCache.put(docIds, schemaDocIds);
|
||||
return schemaDocIds;
|
||||
}
|
||||
|
||||
};
|
||||
tagger.enableDocIdsCache(2000);//TODO configurable
|
||||
tagger.process();
|
||||
}
|
||||
} finally {
|
||||
inputReader.close();
|
||||
}
|
||||
rsp.add("tagsCount",tags.size());
|
||||
rsp.add("tags", tags);
|
||||
|
||||
rsp.setReturnFields(new SolrReturnFields( req ));
|
||||
|
||||
//Solr's standard name for matching docs in response
|
||||
rsp.add("response", getDocList(rows, matchDocIdsBS));
|
||||
}
|
||||
|
||||
private static class InputStringLazy implements Callable<String> {
|
||||
final Reader inputReader;
|
||||
String inputString;
|
||||
|
||||
InputStringLazy(Reader inputReader) {
|
||||
this.inputReader = inputReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String call() throws IOException {
|
||||
if (inputString == null) {
|
||||
inputString = CharStreams.toString(inputReader);
|
||||
}
|
||||
return inputString;
|
||||
}
|
||||
}
|
||||
|
||||
protected OffsetCorrector getOffsetCorrector(SolrParams params, Callable<String> inputStringProvider) throws Exception {
|
||||
final boolean xmlOffsetAdjust = params.getBool(XML_OFFSET_ADJUST, false);
|
||||
if (!xmlOffsetAdjust) {
|
||||
return null;
|
||||
}
|
||||
try {
|
||||
return new XmlOffsetCorrector(inputStringProvider.call());
|
||||
} catch (XMLStreamException e) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
"Expecting XML but wasn't: " + e, e);
|
||||
}
|
||||
}
|
||||
|
||||
private DocList getDocList(int rows, FixedBitSet matchDocIdsBS) throws IOException {
|
||||
//Now we must supply a Solr DocList and add it to the response.
|
||||
// Typically this is gotten via a SolrIndexSearcher.search(), but in this case we
|
||||
// know exactly what documents to return, the order doesn't matter nor does
|
||||
// scoring.
|
||||
// Ideally an implementation of DocList could be directly implemented off
|
||||
// of a BitSet, but there are way too many methods to implement for a minor
|
||||
// payoff.
|
||||
int matchDocs = matchDocIdsBS.cardinality();
|
||||
int[] docIds = new int[ Math.min(rows, matchDocs) ];
|
||||
DocIdSetIterator docIdIter = new BitSetIterator(matchDocIdsBS, 1);
|
||||
for (int i = 0; i < docIds.length; i++) {
|
||||
docIds[i] = docIdIter.nextDoc();
|
||||
}
|
||||
return new DocSlice(0, docIds.length, docIds, null, matchDocs, 1f);
|
||||
}
|
||||
|
||||
private TagClusterReducer chooseTagClusterReducer(String overlaps) {
|
||||
TagClusterReducer tagClusterReducer;
|
||||
if (overlaps == null || overlaps.equals("NO_SUB")) {
|
||||
tagClusterReducer = TagClusterReducer.NO_SUB;
|
||||
} else if (overlaps.equals("ALL")) {
|
||||
tagClusterReducer = TagClusterReducer.ALL;
|
||||
} else if (overlaps.equals("LONGEST_DOMINANT_RIGHT")) {
|
||||
tagClusterReducer = TagClusterReducer.LONGEST_DOMINANT_RIGHT;
|
||||
} else {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
"unknown tag overlap mode: "+overlaps);
|
||||
}
|
||||
return tagClusterReducer;
|
||||
}
|
||||
|
||||
/**
|
||||
* The set of documents matching the provided 'fq' (filter query). Don't include deleted docs
|
||||
* either. If null is returned, then all docs are available.
|
||||
*/
|
||||
private Bits computeDocCorpus(SolrQueryRequest req) throws SyntaxError, IOException {
|
||||
final String[] corpusFilterQueries = req.getParams().getParams("fq");
|
||||
final SolrIndexSearcher searcher = req.getSearcher();
|
||||
final Bits docBits;
|
||||
if (corpusFilterQueries != null && corpusFilterQueries.length > 0) {
|
||||
List<Query> filterQueries = new ArrayList<Query>(corpusFilterQueries.length);
|
||||
for (String corpusFilterQuery : corpusFilterQueries) {
|
||||
QParser qParser = QParser.getParser(corpusFilterQuery, null, req);
|
||||
try {
|
||||
filterQueries.add(qParser.parse());
|
||||
} catch (SyntaxError e) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
|
||||
}
|
||||
}
|
||||
|
||||
final DocSet docSet = searcher.getDocSet(filterQueries);//hopefully in the cache
|
||||
//note: before Solr 4.7 we could call docSet.getBits() but no longer.
|
||||
if (docSet instanceof BitDocSet) {
|
||||
docBits = ((BitDocSet)docSet).getBits();
|
||||
} else {
|
||||
docBits = new Bits() {
|
||||
|
||||
@Override
|
||||
public boolean get(int index) {
|
||||
return docSet.exists(index);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return searcher.maxDoc();
|
||||
}
|
||||
};
|
||||
}
|
||||
} else {
|
||||
docBits = searcher.getSlowAtomicReader().getLiveDocs();
|
||||
}
|
||||
return docBits;
|
||||
}
|
||||
|
||||
private boolean fieldHasIndexedStopFilter(String field, SolrQueryRequest req) {
|
||||
FieldType fieldType = req.getSchema().getFieldType(field);
|
||||
Analyzer analyzer = fieldType.getIndexAnalyzer();//index analyzer
|
||||
if (analyzer instanceof TokenizerChain) {
|
||||
TokenizerChain tokenizerChain = (TokenizerChain) analyzer;
|
||||
TokenFilterFactory[] tokenFilterFactories = tokenizerChain.getTokenFilterFactories();
|
||||
for (TokenFilterFactory tokenFilterFactory : tokenFilterFactories) {
|
||||
if (tokenFilterFactory instanceof StopFilterFactory)
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/** See LUCENE-4541 or {@link org.apache.solr.response.transform.ValueSourceAugmenter}. */
|
||||
static class ValueSourceAccessor {
|
||||
private final List<LeafReaderContext> readerContexts;
|
||||
private final ValueSource valueSource;
|
||||
private final Map fContext;
|
||||
private final FunctionValues[] functionValuesPerSeg;
|
||||
private final int[] functionValuesDocIdPerSeg;
|
||||
|
||||
ValueSourceAccessor(IndexSearcher searcher, ValueSource valueSource) {
|
||||
readerContexts = searcher.getIndexReader().leaves();
|
||||
this.valueSource = valueSource;
|
||||
fContext = ValueSource.newContext(searcher);
|
||||
functionValuesPerSeg = new FunctionValues[readerContexts.size()];
|
||||
functionValuesDocIdPerSeg = new int[readerContexts.size()];
|
||||
}
|
||||
|
||||
Object objectVal(int topDocId) throws IOException {
|
||||
// lookup segment level stuff:
|
||||
int segIdx = ReaderUtil.subIndex(topDocId, readerContexts);
|
||||
LeafReaderContext rcontext = readerContexts.get(segIdx);
|
||||
int segDocId = topDocId - rcontext.docBase;
|
||||
// unfortunately Lucene 7.0 requires forward only traversal (with no reset method).
|
||||
// So we need to track our last docId (per segment) and re-fetch the FunctionValues. :-(
|
||||
FunctionValues functionValues = functionValuesPerSeg[segIdx];
|
||||
if (functionValues == null || segDocId < functionValuesDocIdPerSeg[segIdx]) {
|
||||
functionValues = functionValuesPerSeg[segIdx] = valueSource.getValues(fContext, rcontext);
|
||||
}
|
||||
functionValuesDocIdPerSeg[segIdx] = segDocId;
|
||||
|
||||
// get value:
|
||||
return functionValues.objectVal(segDocId);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
/*
|
||||
* This software was produced for the U. S. Government
|
||||
* under Contract No. W15P7T-11-C-F600, and is
|
||||
* subject to the Rights in Noncommercial Computer Software
|
||||
* and Noncommercial Computer Software Documentation
|
||||
* Clause 252.227-7014 (JUN 1995)
|
||||
*
|
||||
* Copyright 2013 The MITRE Corporation. All Rights Reserved.
|
||||
*
|
||||
* Licensed 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.solr.handler.tagger;
|
||||
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.util.Attribute;
|
||||
|
||||
/**
|
||||
* Attribute used by the {@link Tagger} to decide if a token can start a
|
||||
* new {@link TagLL tag}.
|
||||
* <p>
|
||||
* By default this Attribute will return <code>true</code>, but it might be
|
||||
* reset by some {@link TokenFilter} added to the {@link TokenStream} used
|
||||
* to analyze the parsed text. Typically this will be done based on NLP
|
||||
* processing results (e.g. to only lookup Named Entities).
|
||||
* <p>
|
||||
* NOTE: that all Tokens are used to advance existing {@link TagLL tags}.
|
||||
*/
|
||||
public interface TaggingAttribute extends Attribute {
|
||||
|
||||
/**
|
||||
* By default this Attribute will be initialised with <code>true</code>.
|
||||
* This ensures that all tokens are taggable by default (especially if
|
||||
* the {@link TaggingAttribute} is not set by any component in the configured
|
||||
* {@link TokenStream}
|
||||
*/
|
||||
public static final boolean DEFAULT_TAGGABLE = true;
|
||||
|
||||
/**
|
||||
* Getter for the taggable state of the current Token
|
||||
*
|
||||
* @return the state
|
||||
*/
|
||||
public boolean isTaggable();
|
||||
|
||||
/**
|
||||
* Setter for the taggable state. Typically called by code within
|
||||
* {@link TokenFilter#incrementToken()}.
|
||||
*
|
||||
* @param lookup the state
|
||||
*/
|
||||
public void setTaggable(boolean lookup);
|
||||
|
||||
}
|
|
@ -0,0 +1,79 @@
|
|||
/*
|
||||
* This software was produced for the U. S. Government
|
||||
* under Contract No. W15P7T-11-C-F600, and is
|
||||
* subject to the Rights in Noncommercial Computer Software
|
||||
* and Noncommercial Computer Software Documentation
|
||||
* Clause 252.227-7014 (JUN 1995)
|
||||
*
|
||||
* Copyright 2013 The MITRE Corporation. All Rights Reserved.
|
||||
*
|
||||
* Licensed 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.solr.handler.tagger;
|
||||
|
||||
import org.apache.lucene.util.AttributeImpl;
|
||||
import org.apache.lucene.util.AttributeReflector;
|
||||
|
||||
/**
|
||||
* Implementation of the {@link TaggingAttribute}
|
||||
*/
|
||||
public class TaggingAttributeImpl extends AttributeImpl implements TaggingAttribute {
|
||||
|
||||
/**
|
||||
* the private field initialised with {@link TaggingAttribute#DEFAULT_TAGGABLE}
|
||||
*/
|
||||
private boolean taggable = TaggingAttribute.DEFAULT_TAGGABLE;
|
||||
|
||||
/*
|
||||
* (non-Javadoc)
|
||||
* @see org.opensextant.solrtexttagger.LookupAttribute#isLookup()
|
||||
*/
|
||||
@Override
|
||||
public boolean isTaggable() {
|
||||
return taggable;
|
||||
}
|
||||
|
||||
/*
|
||||
* (non-Javadoc)
|
||||
* @see org.opensextant.solrtexttagger.LookupAttribute#setLookup(boolean)
|
||||
*/
|
||||
@Override
|
||||
public void setTaggable(boolean lookup) {
|
||||
this.taggable = lookup;
|
||||
}
|
||||
|
||||
/*
|
||||
* (non-Javadoc)
|
||||
* @see org.apache.lucene.util.AttributeImpl#clear()
|
||||
*/
|
||||
@Override
|
||||
public void clear() {
|
||||
taggable = DEFAULT_TAGGABLE;
|
||||
}
|
||||
|
||||
/*
|
||||
* (non-Javadoc)
|
||||
* @see org.apache.lucene.util.AttributeImpl#copyTo(org.apache.lucene.util.AttributeImpl)
|
||||
*/
|
||||
@Override
|
||||
public void copyTo(AttributeImpl target) {
|
||||
((TaggingAttribute) target).setTaggable(taggable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reflectWith(AttributeReflector reflector) {
|
||||
reflector.reflect(TaggingAttribute.class, "taggable", isTaggable());
|
||||
}
|
||||
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue