Merge remote-tracking branch 'upstream/master' into LUCENE-8343

This commit is contained in:
Alessandro Benedetti 2018-06-07 16:50:58 +01:00
commit cef9a2283e
241 changed files with 6740 additions and 2078 deletions

View File

@ -107,6 +107,9 @@ Optimizations
or phrase queries as sub queries, which know how to leverage this information
to run faster. (Adrien Grand)
======================= Lucene 7.5.0 =======================
(No Changes)
======================= Lucene 7.4.0 =======================
API Changes
@ -202,6 +205,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
@ -255,6 +270,9 @@ Other
* LUCENE-8301: Update randomizedtesting to 2.6.0. (Dawid Weiss)
* LUCENE-8299: Geo3D wrapper uses new polygon method factory that gives better
support for polygons with many points (>100). (Ignacio vera)
* LUCENE-8261: InterpolatedProperties.interpolate and recursive property
references. (Steve Rowe, Dawid Weiss)

View File

@ -62,7 +62,6 @@ import org.apache.lucene.util.IntsRefBuilder;
import org.apache.lucene.util.OfflineSorter.ByteSequencesReader;
import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
import org.apache.lucene.util.OfflineSorter;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
import org.apache.lucene.util.automaton.RegExp;
import org.apache.lucene.util.fst.Builder;
@ -927,10 +926,7 @@ public class Dictionary {
if (hasStemExceptions && end+1 < line.length()) {
String stemException = parseStemException(line.substring(end+1));
if (stemException != null) {
if (stemExceptionCount == stemExceptions.length) {
int newSize = ArrayUtil.oversize(stemExceptionCount+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
stemExceptions = Arrays.copyOf(stemExceptions, newSize);
}
stemExceptions = ArrayUtil.grow(stemExceptions, stemExceptionCount+1);
stemExceptionID = stemExceptionCount+1; // we use '0' to indicate no exception for the form
stemExceptions[stemExceptionCount++] = stemException;
}
@ -1125,7 +1121,7 @@ public class Dictionary {
}
if (upto < flags.length) {
flags = Arrays.copyOf(flags, upto);
flags = ArrayUtil.copyOfSubArray(flags, 0, upto);
}
return flags;
}

View File

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

View File

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

View File

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

View File

@ -17,7 +17,6 @@
package org.apache.lucene.analysis.miscellaneous;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.TokenFilter;
@ -427,9 +426,9 @@ public final class WordDelimiterFilter extends TokenFilter {
private void buffer() {
if (bufferedLen == buffered.length) {
int newSize = ArrayUtil.oversize(bufferedLen+1, 8);
buffered = Arrays.copyOf(buffered, newSize);
startOff = Arrays.copyOf(startOff, newSize);
posInc = Arrays.copyOf(posInc, newSize);
buffered = ArrayUtil.growExact(buffered, newSize);
startOff = ArrayUtil.growExact(startOff, newSize);
posInc = ArrayUtil.growExact(posInc, newSize);
}
startOff[bufferedLen] = offsetAttribute.startOffset();
posInc[bufferedLen] = posIncAttribute.getPositionIncrement();

View File

@ -29,19 +29,21 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
* &lt;fieldType name="text_edgngrm" class="solr.TextField" positionIncrementGap="100"&gt;
* &lt;analyzer&gt;
* &lt;tokenizer class="solr.WhitespaceTokenizerFactory"/&gt;
* &lt;filter class="solr.EdgeNGramFilterFactory" minGramSize="1" maxGramSize="1"/&gt;
* &lt;filter class="solr.EdgeNGramFilterFactory" minGramSize="1" maxGramSize="2" preserveOriginal="true"/&gt;
* &lt;/analyzer&gt;
* &lt;/fieldType&gt;</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);
}
}

View File

@ -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 &gt;= minGram and &lt;= 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;
}
}

View File

@ -29,19 +29,21 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
* &lt;fieldType name="text_ngrm" class="solr.TextField" positionIncrementGap="100"&gt;
* &lt;analyzer&gt;
* &lt;tokenizer class="solr.WhitespaceTokenizerFactory"/&gt;
* &lt;filter class="solr.NGramFilterFactory" minGramSize="1" maxGramSize="2"/&gt;
* &lt;filter class="solr.NGramFilterFactory" minGramSize="1" maxGramSize="2" preserveOriginal="true"/&gt;
* &lt;/analyzer&gt;
* &lt;/fieldType&gt;</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);
}
}

View File

@ -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 &gt;= minGram and &lt;= 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;
}
}

View File

@ -17,7 +17,6 @@
package org.apache.lucene.analysis.synonym;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
@ -33,7 +32,6 @@ import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.fst.FST;
/**
@ -206,7 +204,7 @@ public final class SynonymFilter extends TokenFilter {
public void add(char[] output, int offset, int len, int endOffset, int posLength) {
if (count == outputs.length) {
outputs = Arrays.copyOf(outputs, ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
outputs = ArrayUtil.grow(outputs, count+1);
}
if (count == endOffsets.length) {
final int[] next = new int[ArrayUtil.oversize(1+count, Integer.BYTES)];

View File

@ -21,9 +21,9 @@ import java.io.IOException;
import java.io.LineNumberReader;
import java.io.Reader;
import java.text.ParseException;
import java.util.Arrays;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.CharsRefBuilder;
@ -59,10 +59,7 @@ public class WordnetSynonymParser extends SynonymMap.Parser {
synsetSize = 0;
}
if (synset.length <= synsetSize+1) {
synset = Arrays.copyOf(synset, synset.length * 2);
}
synset = ArrayUtil.grow(synset, synsetSize + 1);
synset[synsetSize] = parseSynonym(line, new CharsRefBuilder());
synsetSize++;
lastSynSetID = synSetID;

View File

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

View File

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

View File

@ -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;
@ -87,6 +88,7 @@ import org.apache.lucene.analysis.path.PathHierarchyTokenizer;
import org.apache.lucene.analysis.path.ReversePathHierarchyTokenizer;
import org.apache.lucene.analysis.payloads.IdentityEncoder;
import org.apache.lucene.analysis.payloads.PayloadEncoder;
import org.apache.lucene.analysis.shingle.ShingleFilter;
import org.apache.lucene.analysis.snowball.TestSnowball;
import org.apache.lucene.analysis.standard.StandardTokenizer;
import org.apache.lucene.analysis.synonym.SynonymMap;
@ -119,10 +121,14 @@ 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);
// ShingleFilter doesn't handle input graphs correctly, so wrapping it in a condition can
// expose inconsistent offsets
// https://issues.apache.org/jira/browse/LUCENE-4170
avoidConditionals.add(ShingleFilter.class);
}
private static final Map<Constructor<?>,Predicate<Object[]>> brokenConstructors = new HashMap<>();
@ -156,7 +162,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,

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -29,6 +29,7 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.TestUtil;
import com.carrotsearch.randomizedtesting.generators.RandomStrings;
@ -179,7 +180,7 @@ public class NGramTokenizerTest extends BaseTokenStreamTestCase {
}
}
assertTrue(grams.incrementToken());
assertArrayEquals(Arrays.copyOfRange(codePoints, start, end), toCodePoints(termAtt));
assertArrayEquals(ArrayUtil.copyOfSubArray(codePoints, start, end), toCodePoints(termAtt));
assertEquals(1, posIncAtt.getPositionIncrement());
assertEquals(1, posLenAtt.getPositionLength());
assertEquals(offsets[start], offsetAtt.startOffset());

View File

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

View File

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

View File

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

View File

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

View File

@ -20,7 +20,6 @@ package org.apache.lucene.codecs.blockterms;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.lucene.codecs.BlockTermState;
@ -44,7 +43,6 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
// TODO: currently we encode all terms between two indexed
// terms as a block; but, we could decouple the two, ie
@ -260,11 +258,9 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
//System.out.println(" index term!");
}
if (pendingTerms.length == pendingCount) {
pendingTerms = Arrays.copyOf(pendingTerms, ArrayUtil.oversize(pendingCount+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
for(int i=pendingCount;i<pendingTerms.length;i++) {
pendingTerms[i] = new TermEntry();
}
pendingTerms = ArrayUtil.grow(pendingTerms, pendingCount + 1);
for (int i = pendingCount; i < pendingTerms.length; i++) {
pendingTerms[i] = new TermEntry();
}
final TermEntry te = pendingTerms[pendingCount];
te.term.copyBytes(text);

View File

@ -505,7 +505,7 @@ final class SimpleTextBKDWriter implements Closeable {
if (leafBlockFPs.size() > 0) {
// Save the first (minimum) value in each leaf block except the first, to build the split value index in the end:
leafBlockStartValues.add(Arrays.copyOf(leafValues, packedBytesLength));
leafBlockStartValues.add(ArrayUtil.copyOfSubArray(leafValues, 0, packedBytesLength));
}
leafBlockFPs.add(out.getFilePointer());
checkMaxLeafNodeCount(leafBlockFPs.size());
@ -539,8 +539,8 @@ final class SimpleTextBKDWriter implements Closeable {
return scratch;
}
};
assert valuesInOrderAndBounds(leafCount, 0, Arrays.copyOf(leafValues, packedBytesLength),
Arrays.copyOfRange(leafValues, (leafCount - 1) * packedBytesLength, leafCount * packedBytesLength),
assert valuesInOrderAndBounds(leafCount, 0, ArrayUtil.copyOfSubArray(leafValues, 0, packedBytesLength),
ArrayUtil.copyOfSubArray(leafValues, (leafCount - 1) * packedBytesLength, leafCount * packedBytesLength),
packedValues, leafDocs, 0);
writeLeafBlockPackedValues(out, commonPrefixLengths, leafCount, 0, packedValues);
}
@ -1206,8 +1206,8 @@ final class SimpleTextBKDWriter implements Closeable {
reader.getValue(mid, scratchBytesRef1);
System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim, splitPackedValues, address + 1, bytesPerDim);
byte[] minSplitPackedValue = Arrays.copyOf(minPackedValue, packedBytesLength);
byte[] maxSplitPackedValue = Arrays.copyOf(maxPackedValue, packedBytesLength);
byte[] minSplitPackedValue = ArrayUtil.copyOfSubArray(minPackedValue, 0, packedBytesLength);
byte[] maxSplitPackedValue = ArrayUtil.copyOfSubArray(maxPackedValue, 0, packedBytesLength);
System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim,
minSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim,

View File

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

View File

@ -42,6 +42,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.StringHelper;
@ -158,7 +159,7 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch.get(), SI_ID);
final byte[] id = Arrays.copyOfRange(scratch.bytes(), SI_ID.length, scratch.length());
final byte[] id = ArrayUtil.copyOfSubArray(scratch.bytes(), SI_ID.length, scratch.length());
if (!Arrays.equals(segmentID, id)) {
throw new CorruptIndexException("file mismatch, expected: " + StringHelper.idToString(segmentID)

View File

@ -21,7 +21,6 @@ import static org.apache.lucene.util.BitUtil.zigZagDecode;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
@ -73,12 +72,12 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable, Acco
}
if (blockCount == docBases.length) {
final int newSize = ArrayUtil.oversize(blockCount + 1, 8);
docBases = Arrays.copyOf(docBases, newSize);
startPointers = Arrays.copyOf(startPointers, newSize);
avgChunkDocs = Arrays.copyOf(avgChunkDocs, newSize);
avgChunkSizes = Arrays.copyOf(avgChunkSizes, newSize);
docBasesDeltas = Arrays.copyOf(docBasesDeltas, newSize);
startPointersDeltas = Arrays.copyOf(startPointersDeltas, newSize);
docBases = ArrayUtil.growExact(docBases, newSize);
startPointers = ArrayUtil.growExact(startPointers, newSize);
avgChunkDocs = ArrayUtil.growExact(avgChunkDocs, newSize);
avgChunkSizes = ArrayUtil.growExact(avgChunkSizes, newSize);
docBasesDeltas = ArrayUtil.growExact(docBasesDeltas, newSize);
startPointersDeltas = ArrayUtil.growExact(startPointersDeltas, newSize);
}
// doc bases
@ -102,12 +101,12 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable, Acco
++blockCount;
}
this.docBases = Arrays.copyOf(docBases, blockCount);
this.startPointers = Arrays.copyOf(startPointers, blockCount);
this.avgChunkDocs = Arrays.copyOf(avgChunkDocs, blockCount);
this.avgChunkSizes = Arrays.copyOf(avgChunkSizes, blockCount);
this.docBasesDeltas = Arrays.copyOf(docBasesDeltas, blockCount);
this.startPointersDeltas = Arrays.copyOf(startPointersDeltas, blockCount);
this.docBases = ArrayUtil.copyOfSubArray(docBases, 0, blockCount);
this.startPointers = ArrayUtil.copyOfSubArray(startPointers, 0, blockCount);
this.avgChunkDocs = ArrayUtil.copyOfSubArray(avgChunkDocs, 0, blockCount);
this.avgChunkSizes = ArrayUtil.copyOfSubArray(avgChunkSizes, 0, blockCount);
this.docBasesDeltas = ArrayUtil.copyOfSubArray(docBasesDeltas, 0, blockCount);
this.startPointersDeltas = ArrayUtil.copyOfSubArray(startPointersDeltas, 0, blockCount);
}
private int block(int docID) {

View File

@ -19,7 +19,6 @@ package org.apache.lucene.codecs.compressing;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.lucene.codecs.CodecUtil;
@ -158,8 +157,8 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
public void finishDocument() throws IOException {
if (numBufferedDocs == this.numStoredFields.length) {
final int newLength = ArrayUtil.oversize(numBufferedDocs + 1, 4);
this.numStoredFields = Arrays.copyOf(this.numStoredFields, newLength);
endOffsets = Arrays.copyOf(endOffsets, newLength);
this.numStoredFields = ArrayUtil.growExact(this.numStoredFields, newLength);
endOffsets = ArrayUtil.growExact(endOffsets, newLength);
}
this.numStoredFields[numBufferedDocs] = numStoredFieldsInDoc;
numStoredFieldsInDoc = 0;

View File

@ -176,8 +176,8 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
if (hasOffsets) {
if (offStart + totalPositions == startOffsetsBuf.length) {
final int newLength = ArrayUtil.oversize(offStart + totalPositions, 4);
startOffsetsBuf = Arrays.copyOf(startOffsetsBuf, newLength);
lengthsBuf = Arrays.copyOf(lengthsBuf, newLength);
startOffsetsBuf = ArrayUtil.growExact(startOffsetsBuf, newLength);
lengthsBuf = ArrayUtil.growExact(lengthsBuf, newLength);
}
startOffsetsBuf[offStart + totalPositions] = startOffset;
lengthsBuf[offStart + totalPositions] = length;
@ -705,8 +705,8 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
final int offStart = curField.offStart + curField.totalPositions;
if (offStart + numProx > startOffsetsBuf.length) {
final int newLength = ArrayUtil.oversize(offStart + numProx, 4);
startOffsetsBuf = Arrays.copyOf(startOffsetsBuf, newLength);
lengthsBuf = Arrays.copyOf(lengthsBuf, newLength);
startOffsetsBuf = ArrayUtil.growExact(startOffsetsBuf, newLength);
lengthsBuf = ArrayUtil.growExact(lengthsBuf, newLength);
}
int lastOffset = 0, startOffset, endOffset;
for (int i = 0; i < numProx; ++i) {

View File

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

View File

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

View File

@ -200,4 +200,9 @@ class BinaryDocValuesWriter extends DocValuesWriter {
return value.get();
}
}
@Override
DocIdSetIterator getDocIdSet() {
return docsWithField.iterator();
}
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -244,4 +244,9 @@ class SortedDocValuesWriter extends DocValuesWriter {
return valueCount;
}
}
@Override
DocIdSetIterator getDocIdSet() {
return docsWithField.iterator();
}
}

View File

@ -231,4 +231,9 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
return docsWithField.cost();
}
}
@Override
DocIdSetIterator getDocIdSet() {
return docsWithField.iterator();
}
}

View File

@ -164,7 +164,7 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
}
docOrds[upto++] = ord;
}
ords[newDocID] = Arrays.copyOfRange(docOrds, 0, upto);
ords[newDocID] = ArrayUtil.copyOfSubArray(docOrds, 0, upto);
}
return ords;
}
@ -315,4 +315,8 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
return scratch;
}
}
@Override
DocIdSetIterator getDocIdSet() {
return docsWithField.iterator();
}
}

View File

@ -890,8 +890,8 @@ class SortingLeafReader extends FilterLeafReader {
while ((doc = in.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
if (i == docs.length) {
final int newLength = ArrayUtil.oversize(i + 1, 4);
docs = Arrays.copyOf(docs, newLength);
offsets = Arrays.copyOf(offsets, newLength);
docs = ArrayUtil.growExact(docs, newLength);
offsets = ArrayUtil.growExact(offsets, newLength);
}
docs[i] = docMap.oldToNew(doc);
offsets[i] = out.getFilePointer();
@ -1230,7 +1230,7 @@ class SortingLeafReader extends FilterLeafReader {
}
docOrds[upto++] = ord;
}
ords[newDocID] = Arrays.copyOfRange(docOrds, 0, upto);
ords[newDocID] = ArrayUtil.copyOfSubArray(docOrds, 0, upto);
}
cachedSortedSetDVs.put(field, ords);
}

View File

@ -102,9 +102,9 @@ public final class BlendedTermQuery extends Query {
/** Build the {@link BlendedTermQuery}. */
public BlendedTermQuery build() {
return new BlendedTermQuery(
Arrays.copyOf(terms, numTerms),
Arrays.copyOf(boosts, numTerms),
Arrays.copyOf(contexts, numTerms),
ArrayUtil.copyOfSubArray(terms, 0, numTerms),
ArrayUtil.copyOfSubArray(boosts, 0, numTerms),
ArrayUtil.copyOfSubArray(contexts, 0, numTerms),
rewriteMethod);
}
@ -263,7 +263,7 @@ public final class BlendedTermQuery extends Query {
@Override
public final Query rewrite(IndexReader reader) throws IOException {
final TermStates[] contexts = Arrays.copyOf(this.contexts, this.contexts.length);
final TermStates[] contexts = ArrayUtil.copyOfSubArray(this.contexts, 0, this.contexts.length);
for (int i = 0; i < contexts.length; ++i) {
if (contexts[i] == null || contexts[i].wasBuiltFor(reader.getContext()) == false) {
contexts[i] = TermStates.build(reader.getContext(), terms[i], true);

View File

@ -18,7 +18,6 @@ package org.apache.lucene.search;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.lucene.index.LeafReaderContext;
@ -213,7 +212,7 @@ public abstract class CachingCollector extends FilterCollector {
}
protected void grow(int newLen) {
docs = Arrays.copyOf(docs, newLen);
docs = ArrayUtil.growExact(docs, newLen);
}
protected void invalidate() {
@ -250,7 +249,7 @@ public abstract class CachingCollector extends FilterCollector {
}
int[] cachedDocs() {
return docs == null ? null : Arrays.copyOf(docs, docCount);
return docs == null ? null : ArrayUtil.copyOfSubArray(docs, 0, docCount);
}
}
@ -274,7 +273,7 @@ public abstract class CachingCollector extends FilterCollector {
@Override
protected void grow(int newLen) {
super.grow(newLen);
scores = Arrays.copyOf(scores, newLen);
scores = ArrayUtil.growExact(scores, newLen);
}
@Override
@ -290,7 +289,7 @@ public abstract class CachingCollector extends FilterCollector {
}
float[] cachedScores() {
return docs == null ? null : Arrays.copyOf(scores, docCount);
return docs == null ? null : ArrayUtil.copyOfSubArray(scores, 0, docCount);
}
}

View File

@ -53,7 +53,7 @@ final class MaxScoreCache {
if (maxScoreCache.length < size) {
int oldLength = maxScoreCache.length;
maxScoreCache = ArrayUtil.grow(maxScoreCache, size);
maxScoreCacheUpTo = Arrays.copyOf(maxScoreCacheUpTo, maxScoreCache.length);
maxScoreCacheUpTo = ArrayUtil.growExact(maxScoreCacheUpTo, maxScoreCache.length);
Arrays.fill(maxScoreCacheUpTo, oldLength, maxScoreCacheUpTo.length, -1);
}
}

View File

@ -413,7 +413,7 @@ public class PhraseQuery extends Query {
}
}
if (termUpTo > 0) {
return similarity.scorer(boost, searcher.collectionStatistics(field), Arrays.copyOf(termStats, termUpTo));
return similarity.scorer(boost, searcher.collectionStatistics(field), ArrayUtil.copyOfSubArray(termStats, 0, termUpTo));
} else {
return null; // no terms at all, we won't use similarity
}

View File

@ -333,7 +333,7 @@ public abstract class PointInSetQuery extends Query {
upto++;
BytesRef next = iterator.next();
return Arrays.copyOfRange(next.bytes, next.offset, next.length);
return BytesRef.deepCopyOf(next).bytes;
}
};
}

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.index.PointValues.Relation;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BitSetIterator;
import org.apache.lucene.util.DocIdSetBuilder;
import org.apache.lucene.util.FixedBitSet;
@ -392,9 +393,9 @@ public abstract class PointRangeQuery extends Query {
int startOffset = bytesPerDim * i;
sb.append('[');
sb.append(toString(i, Arrays.copyOfRange(lowerPoint, startOffset, startOffset + bytesPerDim)));
sb.append(toString(i, ArrayUtil.copyOfSubArray(lowerPoint, startOffset, startOffset + bytesPerDim)));
sb.append(" TO ");
sb.append(toString(i, Arrays.copyOfRange(upperPoint, startOffset, startOffset + bytesPerDim)));
sb.append(toString(i, ArrayUtil.copyOfSubArray(upperPoint, startOffset, startOffset + bytesPerDim)));
sb.append(']');
}

View File

@ -18,7 +18,6 @@ package org.apache.lucene.search.spans;
import java.io.IOException;
import java.util.Arrays;
import java.util.Map;
import org.apache.lucene.index.LeafReaderContext;
@ -32,6 +31,7 @@ import org.apache.lucene.search.LeafSimScorer;
import org.apache.lucene.search.TermStatistics;
import org.apache.lucene.search.Weight;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.ArrayUtil;
/**
* Expert-only. Public for use by other weight implementations
@ -103,7 +103,7 @@ public abstract class SpanWeight extends Weight {
}
CollectionStatistics collectionStats = searcher.collectionStatistics(query.getField());
if (termUpTo > 0) {
return similarity.scorer(boost, collectionStats, Arrays.copyOf(termStats, termUpTo));
return similarity.scorer(boost, collectionStats, ArrayUtil.copyOfSubArray(termStats, 0, termUpTo));
} else {
return null; // no terms at all exist, we won't use similarity
}

View File

@ -16,7 +16,7 @@
*/
package org.apache.lucene.util;
import java.util.Arrays;
import java.lang.reflect.Array;
import java.util.Comparator;
/**
@ -211,94 +211,172 @@ public final class ArrayUtil {
}
}
/** Returns a new array whose size is exact the specified {@code newLength} without over-allocating */
public static <T> T[] growExact(T[] array, int newLength) {
Class<? extends Object[]> type = array.getClass();
@SuppressWarnings("unchecked")
T[] copy = (type == Object[].class)
? (T[]) new Object[newLength]
: (T[]) Array.newInstance(type.getComponentType(), newLength);
System.arraycopy(array, 0, copy, 0, array.length);
return copy;
}
/** Returns an array whose size is at least {@code minSize}, generally over-allocating exponentially */
public static <T> T[] grow(T[] array, int minSize) {
assert minSize >= 0: "size must be positive (got " + minSize + "): likely integer overflow?";
assert minSize >= 0 : "size must be positive (got " + minSize + "): likely integer overflow?";
if (array.length < minSize) {
return Arrays.copyOf(array, oversize(minSize, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
final int newLength = oversize(minSize, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
return growExact(array, newLength);
} else
return array;
}
/** Returns a new array whose size is exact the specified {@code newLength} without over-allocating */
public static short[] growExact(short[] array, int newLength) {
short[] copy = new short[newLength];
System.arraycopy(array, 0, copy, 0, array.length);
return copy;
}
/** Returns an array whose size is at least {@code minSize}, generally over-allocating exponentially */
public static short[] grow(short[] array, int minSize) {
assert minSize >= 0: "size must be positive (got " + minSize + "): likely integer overflow?";
if (array.length < minSize) {
return Arrays.copyOf(array, oversize(minSize, Short.BYTES));
return growExact(array, oversize(minSize, Short.BYTES));
} else
return array;
}
/** Returns a larger array, generally over-allocating exponentially */
public static short[] grow(short[] array) {
return grow(array, 1 + array.length);
}
/** Returns a new array whose size is exact the specified {@code newLength} without over-allocating */
public static float[] growExact(float[] array, int newLength) {
float[] copy = new float[newLength];
System.arraycopy(array, 0, copy, 0, array.length);
return copy;
}
/** Returns an array whose size is at least {@code minSize}, generally over-allocating exponentially */
public static float[] grow(float[] array, int minSize) {
assert minSize >= 0: "size must be positive (got " + minSize + "): likely integer overflow?";
if (array.length < minSize) {
return Arrays.copyOf(array, oversize(minSize, Float.BYTES));
float[] copy = new float[oversize(minSize, Float.BYTES)];
System.arraycopy(array, 0, copy, 0, array.length);
return copy;
} else
return array;
}
/** Returns a larger array, generally over-allocating exponentially */
public static float[] grow(float[] array) {
return grow(array, 1 + array.length);
}
/** Returns a new array whose size is exact the specified {@code newLength} without over-allocating */
public static double[] growExact(double[] array, int newLength) {
double[] copy = new double[newLength];
System.arraycopy(array, 0, copy, 0, array.length);
return copy;
}
/** Returns an array whose size is at least {@code minSize}, generally over-allocating exponentially */
public static double[] grow(double[] array, int minSize) {
assert minSize >= 0: "size must be positive (got " + minSize + "): likely integer overflow?";
if (array.length < minSize) {
return Arrays.copyOf(array, oversize(minSize, Double.BYTES));
return growExact(array, oversize(minSize, Double.BYTES));
} else
return array;
}
/** Returns a larger array, generally over-allocating exponentially */
public static double[] grow(double[] array) {
return grow(array, 1 + array.length);
}
/** Returns a new array whose size is exact the specified {@code newLength} without over-allocating */
public static int[] growExact(int[] array, int newLength) {
int[] copy = new int[newLength];
System.arraycopy(array, 0, copy, 0, array.length);
return copy;
}
/** Returns an array whose size is at least {@code minSize}, generally over-allocating exponentially */
public static int[] grow(int[] array, int minSize) {
assert minSize >= 0: "size must be positive (got " + minSize + "): likely integer overflow?";
if (array.length < minSize) {
return Arrays.copyOf(array, oversize(minSize, Integer.BYTES));
return growExact(array, oversize(minSize, Integer.BYTES));
} else
return array;
}
/** Returns a larger array, generally over-allocating exponentially */
public static int[] grow(int[] array) {
return grow(array, 1 + array.length);
}
/** Returns a new array whose size is exact the specified {@code newLength} without over-allocating */
public static long[] growExact(long[] array, int newLength) {
long[] copy = new long[newLength];
System.arraycopy(array, 0, copy, 0, array.length);
return copy;
}
/** Returns an array whose size is at least {@code minSize}, generally over-allocating exponentially */
public static long[] grow(long[] array, int minSize) {
assert minSize >= 0: "size must be positive (got " + minSize + "): likely integer overflow?";
if (array.length < minSize) {
return Arrays.copyOf(array, oversize(minSize, Long.BYTES));
return growExact(array, oversize(minSize, Long.BYTES));
} else
return array;
}
/** Returns a larger array, generally over-allocating exponentially */
public static long[] grow(long[] array) {
return grow(array, 1 + array.length);
}
/** Returns a new array whose size is exact the specified {@code newLength} without over-allocating */
public static byte[] growExact(byte[] array, int newLength) {
byte[] copy = new byte[newLength];
System.arraycopy(array, 0, copy, 0, array.length);
return copy;
}
/** Returns an array whose size is at least {@code minSize}, generally over-allocating exponentially */
public static byte[] grow(byte[] array, int minSize) {
assert minSize >= 0: "size must be positive (got " + minSize + "): likely integer overflow?";
if (array.length < minSize) {
return Arrays.copyOf(array, oversize(minSize, Byte.BYTES));
return growExact(array, oversize(minSize, Byte.BYTES));
} else
return array;
}
/** Returns a larger array, generally over-allocating exponentially */
public static byte[] grow(byte[] array) {
return grow(array, 1 + array.length);
}
/** Returns a new array whose size is exact the specified {@code newLength} without over-allocating */
public static char[] growExact(char[] array, int newLength) {
char[] copy = new char[newLength];
System.arraycopy(array, 0, copy, 0, array.length);
return copy;
}
/** Returns an array whose size is at least {@code minSize}, generally over-allocating exponentially */
public static char[] grow(char[] array, int minSize) {
assert minSize >= 0: "size must be positive (got " + minSize + "): likely integer overflow?";
if (array.length < minSize) {
return Arrays.copyOf(array, oversize(minSize, Character.BYTES));
return growExact(array, oversize(minSize, Character.BYTES));
} else
return array;
}
/** Returns a larger array, generally over-allocating exponentially */
public static char[] grow(char[] array) {
return grow(array, 1 + array.length);
}
@ -429,4 +507,104 @@ public final class ArrayUtil {
}.select(from, to, k);
}
/**
* Copies the specified range of the given array into a new sub array.
* @param array the input array
* @param from the initial index of range to be copied (inclusive)
* @param to the final index of range to be copied (exclusive)
*/
public static byte[] copyOfSubArray(byte[] array, int from, int to) {
final byte[] copy = new byte[to-from];
System.arraycopy(array, from, copy, 0, to-from);
return copy;
}
/**
* Copies the specified range of the given array into a new sub array.
* @param array the input array
* @param from the initial index of range to be copied (inclusive)
* @param to the final index of range to be copied (exclusive)
*/
public static char[] copyOfSubArray(char[] array, int from, int to) {
final char[] copy = new char[to-from];
System.arraycopy(array, from, copy, 0, to-from);
return copy;
}
/**
* Copies the specified range of the given array into a new sub array.
* @param array the input array
* @param from the initial index of range to be copied (inclusive)
* @param to the final index of range to be copied (exclusive)
*/
public static short[] copyOfSubArray(short[] array, int from, int to) {
final short[] copy = new short[to-from];
System.arraycopy(array, from, copy, 0, to-from);
return copy;
}
/**
* Copies the specified range of the given array into a new sub array.
* @param array the input array
* @param from the initial index of range to be copied (inclusive)
* @param to the final index of range to be copied (exclusive)
*/
public static int[] copyOfSubArray(int[] array, int from, int to) {
final int[] copy = new int[to-from];
System.arraycopy(array, from, copy, 0, to-from);
return copy;
}
/**
* Copies the specified range of the given array into a new sub array.
* @param array the input array
* @param from the initial index of range to be copied (inclusive)
* @param to the final index of range to be copied (exclusive)
*/
public static long[] copyOfSubArray(long[] array, int from, int to) {
final long[] copy = new long[to-from];
System.arraycopy(array, from, copy, 0, to-from);
return copy;
}
/**
* Copies the specified range of the given array into a new sub array.
* @param array the input array
* @param from the initial index of range to be copied (inclusive)
* @param to the final index of range to be copied (exclusive)
*/
public static float[] copyOfSubArray(float[] array, int from, int to) {
final float[] copy = new float[to-from];
System.arraycopy(array, from, copy, 0, to-from);
return copy;
}
/**
* Copies the specified range of the given array into a new sub array.
* @param array the input array
* @param from the initial index of range to be copied (inclusive)
* @param to the final index of range to be copied (exclusive)
*/
public static double[] copyOfSubArray(double[] array, int from, int to) {
final double[] copy = new double[to-from];
System.arraycopy(array, from, copy, 0, to-from);
return copy;
}
/**
* Copies the specified range of the given array into a new sub array.
* @param array the input array
* @param from the initial index of range to be copied (inclusive)
* @param to the final index of range to be copied (exclusive)
*/
public static <T> T[] copyOfSubArray(T[] array, int from, int to) {
final int subLength = to - from;
final Class<? extends Object[]> type = array.getClass();
@SuppressWarnings("unchecked")
final T[] copy = (type == Object[].class)
? (T[]) new Object[subLength]
: (T[]) Array.newInstance(type.getComponentType(), subLength);
System.arraycopy(array, from, copy, 0, subLength);
return copy;
}
}

View File

@ -16,9 +16,6 @@
*/
package org.apache.lucene.util;
import java.util.Arrays;
/** Represents byte[], as a slice (offset + length) into an
* existing byte[]. The {@link #bytes} member should never be null;
* use {@link #EMPTY_BYTES} if necessary.
@ -172,11 +169,7 @@ public final class BytesRef implements Comparable<BytesRef>,Cloneable {
* and an offset of zero.
*/
public static BytesRef deepCopyOf(BytesRef other) {
BytesRef copy = new BytesRef();
copy.bytes = Arrays.copyOfRange(other.bytes, other.offset, other.offset + other.length);
copy.offset = 0;
copy.length = other.length;
return copy;
return new BytesRef(ArrayUtil.copyOfSubArray(other.bytes, other.offset, other.offset + other.length), 0, other.length);
}
/**

View File

@ -17,8 +17,6 @@
package org.apache.lucene.util;
import java.util.Arrays;
/**
* A builder for {@link BytesRef} instances.
* @lucene.internal
@ -170,7 +168,7 @@ public class BytesRefBuilder {
* Build a new {@link BytesRef} that has the same content as this buffer.
*/
public BytesRef toBytesRef() {
return new BytesRef(Arrays.copyOf(ref.bytes, ref.length));
return new BytesRef(ArrayUtil.copyOfSubArray(ref.bytes, 0, ref.length));
}
@Override

View File

@ -17,7 +17,6 @@
package org.apache.lucene.util;
import java.util.Arrays;
import java.util.Comparator;
/**
@ -202,7 +201,7 @@ public final class CharsRef implements Comparable<CharsRef>, CharSequence, Clone
* and an offset of zero.
*/
public static CharsRef deepCopyOf(CharsRef other) {
return new CharsRef(Arrays.copyOfRange(other.chars, other.offset, other.offset + other.length), 0, other.length);
return new CharsRef(ArrayUtil.copyOfSubArray(other.chars, other.offset, other.offset + other.length), 0, other.length);
}
/**

View File

@ -17,8 +17,6 @@
package org.apache.lucene.util;
import java.util.Arrays;
/**
* A builder for {@link CharsRef} instances.
* @lucene.internal
@ -153,7 +151,7 @@ public class CharsRefBuilder implements Appendable {
/** Build a new {@link CharsRef} that has the same content as this builder. */
public CharsRef toCharsRef() {
return new CharsRef(Arrays.copyOf(ref.chars, ref.length), 0, ref.length);
return new CharsRef(ArrayUtil.copyOfSubArray(ref.chars, 0, ref.length), 0, ref.length);
}
@Override

View File

@ -18,7 +18,6 @@ package org.apache.lucene.util;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.lucene.index.PointValues;
@ -226,7 +225,7 @@ public final class DocIdSetBuilder {
}
private void growBuffer(Buffer buffer, int additionalCapacity) {
buffer.array = Arrays.copyOf(buffer.array, buffer.array.length + additionalCapacity);
buffer.array = ArrayUtil.growExact(buffer.array, buffer.array.length + additionalCapacity);
totalAllocated += additionalCapacity;
}
@ -297,7 +296,7 @@ public final class DocIdSetBuilder {
}
int[] docs = largestBuffer.array;
if (docs.length < totalLength + 1) {
docs = Arrays.copyOf(docs, totalLength + 1);
docs = ArrayUtil.growExact(docs, totalLength + 1);
}
totalLength = largestBuffer.length;
for (Buffer buffer : buffers) {

View File

@ -16,8 +16,6 @@
*/
package org.apache.lucene.util;
import java.util.Arrays;
/** Represents int[], as a slice (offset + length) into an
* existing int[]. The {@link #ints} member should never be null; use
@ -127,7 +125,7 @@ public final class IntsRef implements Comparable<IntsRef>, Cloneable {
* and an offset of zero.
*/
public static IntsRef deepCopyOf(IntsRef other) {
return new IntsRef(Arrays.copyOfRange(other.ints, other.offset, other.offset + other.length), 0, other.length);
return new IntsRef(ArrayUtil.copyOfSubArray(other.ints, other.offset, other.offset + other.length), 0, other.length);
}
/**

View File

@ -16,8 +16,6 @@
*/
package org.apache.lucene.util;
import java.util.Arrays;
/** Represents long[], as a slice (offset + length) into an
* existing long[]. The {@link #longs} member should never be null; use
@ -126,7 +124,7 @@ public final class LongsRef implements Comparable<LongsRef>, Cloneable {
* and an offset of zero.
*/
public static LongsRef deepCopyOf(LongsRef other) {
return new LongsRef(Arrays.copyOfRange(other.longs, other.offset, other.offset + other.length), 0, other.length);
return new LongsRef(ArrayUtil.copyOfSubArray(other.longs, other.offset, other.offset + other.length), 0, other.length);
}
/**

View File

@ -18,7 +18,6 @@ package org.apache.lucene.util;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
@ -62,7 +61,7 @@ public final class PagedBytes implements Accountable {
private final long bytesUsedPerBlock;
private Reader(PagedBytes pagedBytes) {
blocks = Arrays.copyOf(pagedBytes.blocks, pagedBytes.numBlocks);
blocks = ArrayUtil.copyOfSubArray(pagedBytes.blocks, 0, pagedBytes.numBlocks);
blockBits = pagedBytes.blockBits;
blockMask = pagedBytes.blockMask;
blockSize = pagedBytes.blockSize;
@ -154,9 +153,7 @@ public final class PagedBytes implements Accountable {
}
private void addBlock(byte[] block) {
if (blocks.length == numBlocks) {
blocks = Arrays.copyOf(blocks, ArrayUtil.oversize(numBlocks, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
}
blocks = ArrayUtil.grow(blocks, numBlocks + 1);
blocks[numBlocks++] = block;
}

View File

@ -346,9 +346,9 @@ public class QueryBuilder {
/**
* Creates a span query from the tokenstream. In the case of a single token, a simple <code>SpanTermQuery</code> is
* returned. When multiple tokens, an ordered <code>SpanNearQuery</code> with slop of 0 is returned.
* returned. When multiple tokens, an ordered <code>SpanNearQuery</code> with slop 0 is returned.
*/
protected final SpanQuery createSpanQuery(TokenStream in, String field) throws IOException {
protected SpanQuery createSpanQuery(TokenStream in, String field) throws IOException {
TermToBytesRefAttribute termAtt = in.getAttribute(TermToBytesRefAttribute.class);
if (termAtt == null) {
return null;

View File

@ -18,7 +18,6 @@ package org.apache.lucene.util;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
@ -73,7 +72,7 @@ public class RoaringDocIdSet extends DocIdSet {
// Use sparse encoding
assert denseBuffer == null;
if (currentBlockCardinality > 0) {
sets[currentBlock] = new ShortArrayDocIdSet(Arrays.copyOf(buffer, currentBlockCardinality));
sets[currentBlock] = new ShortArrayDocIdSet(ArrayUtil.copyOfSubArray(buffer, 0, currentBlockCardinality));
}
} else {
assert denseBuffer != null;

View File

@ -18,7 +18,6 @@ package org.apache.lucene.util;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.search.DocIdSetIterator;
@ -372,7 +371,7 @@ public class SparseFixedBitSet extends BitSet implements Bits, Accountable {
// fast path: if we currently have nothing in the block, just copy the data
// this especially happens all the time if you call OR on an empty set
indices[i4096] = index;
this.bits[i4096] = Arrays.copyOf(bits, nonZeroLongCount);
this.bits[i4096] = ArrayUtil.copyOfSubArray(bits, 0, nonZeroLongCount);
this.nonZeroLongCount += nonZeroLongCount;
return;
}

View File

@ -307,7 +307,7 @@ public abstract class StringHelper {
if (bits.length > ID_LENGTH) {
assert bits.length == ID_LENGTH + 1;
assert bits[0] == 0;
return Arrays.copyOfRange(bits, 1, bits.length);
return ArrayUtil.copyOfSubArray(bits, 1, bits.length);
} else {
byte[] result = new byte[ID_LENGTH];
System.arraycopy(bits, 0, result, result.length - bits.length, bits.length);

View File

@ -89,6 +89,13 @@ public final class Version {
@Deprecated
public static final Version LUCENE_7_4_0 = new Version(7, 4, 0);
/**
* Match settings and bugs in Lucene's 7.5.0 release.
* @deprecated Use latest
*/
@Deprecated
public static final Version LUCENE_7_5_0 = new Version(7, 5, 0);
/**
* Match settings and bugs in Lucene's 8.0.0 release.
* <p>

View File

@ -137,8 +137,8 @@ public final class DaciukMihovAutomatonBuilder {
assert Arrays.binarySearch(labels, label) < 0 : "State already has transition labeled: "
+ label;
labels = Arrays.copyOf(labels, labels.length + 1);
states = Arrays.copyOf(states, states.length + 1);
labels = ArrayUtil.growExact(labels, labels.length + 1);
states = ArrayUtil.growExact(states, states.length + 1);
labels[labels.length - 1] = label;
return states[states.length - 1] = new State();

View File

@ -656,7 +656,7 @@ public class BKDWriter implements Closeable {
if (leafBlockFPs.size() > 0) {
// Save the first (minimum) value in each leaf block except the first, to build the split value index in the end:
leafBlockStartValues.add(Arrays.copyOf(leafValues, packedBytesLength));
leafBlockStartValues.add(ArrayUtil.copyOfSubArray(leafValues, 0, packedBytesLength));
}
leafBlockFPs.add(out.getFilePointer());
checkMaxLeafNodeCount(leafBlockFPs.size());
@ -687,8 +687,8 @@ public class BKDWriter implements Closeable {
return scratchBytesRef1;
}
};
assert valuesInOrderAndBounds(leafCount, 0, Arrays.copyOf(leafValues, packedBytesLength),
Arrays.copyOfRange(leafValues, (leafCount - 1) * packedBytesLength, leafCount * packedBytesLength),
assert valuesInOrderAndBounds(leafCount, 0, ArrayUtil.copyOfSubArray(leafValues, 0, packedBytesLength),
ArrayUtil.copyOfSubArray(leafValues, (leafCount - 1) * packedBytesLength, leafCount * packedBytesLength),
packedValues, leafDocs, 0);
writeLeafBlockPackedValues(scratchOut, commonPrefixLengths, leafCount, 0, packedValues);
out.writeBytes(scratchOut.getBytes(), 0, scratchOut.getPosition());
@ -1591,8 +1591,8 @@ public class BKDWriter implements Closeable {
reader.getValue(mid, scratchBytesRef1);
System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim, splitPackedValues, address + 1, bytesPerDim);
byte[] minSplitPackedValue = Arrays.copyOf(minPackedValue, packedBytesLength);
byte[] maxSplitPackedValue = Arrays.copyOf(maxPackedValue, packedBytesLength);
byte[] minSplitPackedValue = ArrayUtil.copyOfSubArray(minPackedValue, 0, packedBytesLength);
byte[] maxSplitPackedValue = ArrayUtil.copyOfSubArray(maxPackedValue, 0, packedBytesLength);
System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim,
minSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + splitDim * bytesPerDim,

View File

@ -18,7 +18,6 @@ package org.apache.lucene.util.bkd;
import java.io.Closeable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.lucene.util.ArrayUtil;
@ -116,12 +115,12 @@ public final class HeapPointWriter implements PointWriter {
if (docIDs.length == nextWrite) {
int nextSize = Math.min(maxSize, ArrayUtil.oversize(nextWrite+1, Integer.BYTES));
assert nextSize > nextWrite: "nextSize=" + nextSize + " vs nextWrite=" + nextWrite;
docIDs = Arrays.copyOf(docIDs, nextSize);
docIDs = ArrayUtil.growExact(docIDs, nextSize);
if (singleValuePerDoc == false) {
if (ordsLong != null) {
ordsLong = Arrays.copyOf(ordsLong, nextSize);
ordsLong = ArrayUtil.growExact(ordsLong, nextSize);
} else {
ords = Arrays.copyOf(ords, nextSize);
ords = ArrayUtil.growExact(ords, nextSize);
}
}
}

View File

@ -17,8 +17,7 @@
package org.apache.lucene.util.packed;
import java.util.Arrays;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts.Reader;
@ -70,8 +69,8 @@ class DeltaPackedLongValues extends PackedLongValues {
public DeltaPackedLongValues build() {
finish();
pending = null;
final PackedInts.Reader[] values = Arrays.copyOf(this.values, valuesOff);
final long[] mins = Arrays.copyOf(this.mins, valuesOff);
final PackedInts.Reader[] values = ArrayUtil.copyOfSubArray(this.values, 0, valuesOff);
final long[] mins = ArrayUtil.copyOfSubArray(this.mins, 0, valuesOff);
final long ramBytesUsed = DeltaPackedLongValues.BASE_RAM_BYTES_USED
+ RamUsageEstimator.sizeOf(values) + RamUsageEstimator.sizeOf(mins);
return new DeltaPackedLongValues(pageShift, pageMask, values, mins, size, ramBytesUsed);
@ -94,7 +93,7 @@ class DeltaPackedLongValues extends PackedLongValues {
void grow(int newBlockCount) {
super.grow(newBlockCount);
ramBytesUsed -= RamUsageEstimator.sizeOf(mins);
mins = Arrays.copyOf(mins, newBlockCount);
mins = ArrayUtil.growExact(mins, newBlockCount);
ramBytesUsed += RamUsageEstimator.sizeOf(mins);
}

View File

@ -17,10 +17,9 @@
package org.apache.lucene.util.packed;
import java.util.Arrays;
import static org.apache.lucene.util.packed.MonotonicBlockPackedReader.expected;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts.Reader;
@ -72,9 +71,9 @@ class MonotonicLongValues extends DeltaPackedLongValues {
public MonotonicLongValues build() {
finish();
pending = null;
final PackedInts.Reader[] values = Arrays.copyOf(this.values, valuesOff);
final long[] mins = Arrays.copyOf(this.mins, valuesOff);
final float[] averages = Arrays.copyOf(this.averages, valuesOff);
final PackedInts.Reader[] values = ArrayUtil.copyOfSubArray(this.values, 0, valuesOff);
final long[] mins = ArrayUtil.copyOfSubArray(this.mins, 0, valuesOff);
final float[] averages = ArrayUtil.copyOfSubArray(this.averages, 0, valuesOff);
final long ramBytesUsed = MonotonicLongValues.BASE_RAM_BYTES_USED
+ RamUsageEstimator.sizeOf(values) + RamUsageEstimator.sizeOf(mins)
+ RamUsageEstimator.sizeOf(averages);
@ -95,7 +94,7 @@ class MonotonicLongValues extends DeltaPackedLongValues {
void grow(int newBlockCount) {
super.grow(newBlockCount);
ramBytesUsed -= RamUsageEstimator.sizeOf(averages);
averages = Arrays.copyOf(averages, newBlockCount);
averages = ArrayUtil.growExact(averages, newBlockCount);
ramBytesUsed += RamUsageEstimator.sizeOf(averages);
}

View File

@ -19,8 +19,6 @@ package org.apache.lucene.util.packed;
import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
import java.util.Arrays;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.LongValues;
@ -194,7 +192,7 @@ public class PackedLongValues extends LongValues implements Accountable {
public PackedLongValues build() {
finish();
pending = null;
final PackedInts.Reader[] values = Arrays.copyOf(this.values, valuesOff);
final PackedInts.Reader[] values = ArrayUtil.copyOfSubArray(this.values, 0, valuesOff);
final long ramBytesUsed = PackedLongValues.BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(values);
return new PackedLongValues(pageShift, pageMask, values, size, ramBytesUsed);
}
@ -273,7 +271,7 @@ public class PackedLongValues extends LongValues implements Accountable {
void grow(int newBlockCount) {
ramBytesUsed -= RamUsageEstimator.shallowSizeOf(values);
values = Arrays.copyOf(values, newBlockCount);
values = ArrayUtil.growExact(values, newBlockCount);
ramBytesUsed += RamUsageEstimator.shallowSizeOf(values);
}

View File

@ -20,9 +20,9 @@ package org.apache.lucene.analysis;
import java.io.IOException;
import java.io.Reader;
import java.io.StringReader;
import java.util.Arrays;
import org.apache.lucene.analysis.CharacterUtils.CharacterBuffer;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.junit.Test;
@ -42,7 +42,7 @@ public class TestCharacterUtils extends LuceneTestCase {
final int codePointCount = CharacterUtils.toCodePoints(orig, o1, orig.length - o1, buf, o2);
final int charCount = CharacterUtils.toChars(buf, o2, codePointCount, restored, o3);
assertEquals(orig.length - o1, charCount);
assertArrayEquals(Arrays.copyOfRange(orig, o1, o1 + charCount), Arrays.copyOfRange(restored, o3, o3 + charCount));
assertArrayEquals(ArrayUtil.copyOfSubArray(orig, o1, o1 + charCount), ArrayUtil.copyOfSubArray(restored, o3, o3 + charCount));
}
@Test

View File

@ -22,6 +22,7 @@ import java.util.Arrays;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
@ -60,7 +61,7 @@ public abstract class AbstractTestCompressionMode extends LuceneTestCase {
ByteArrayDataOutput out = new ByteArrayDataOutput(compressed);
compressor.compress(decompressed, off, len, out);
final int compressedLen = out.getPosition();
return Arrays.copyOf(compressed, compressedLen);
return ArrayUtil.copyOfSubArray(compressed, 0, compressedLen);
}
byte[] decompress(byte[] compressed, int originalLength) throws IOException {
@ -71,14 +72,14 @@ public abstract class AbstractTestCompressionMode extends LuceneTestCase {
static byte[] decompress(Decompressor decompressor, byte[] compressed, int originalLength) throws IOException {
final BytesRef bytes = new BytesRef();
decompressor.decompress(new ByteArrayDataInput(compressed), originalLength, 0, originalLength, bytes);
return Arrays.copyOfRange(bytes.bytes, bytes.offset, bytes.offset + bytes.length);
return BytesRef.deepCopyOf(bytes).bytes;
}
byte[] decompress(byte[] compressed, int originalLength, int offset, int length) throws IOException {
Decompressor decompressor = mode.newDecompressor();
final BytesRef bytes = new BytesRef();
decompressor.decompress(new ByteArrayDataInput(compressed), originalLength, offset, length, bytes);
return Arrays.copyOfRange(bytes.bytes, bytes.offset, bytes.offset + bytes.length);
return BytesRef.deepCopyOf(bytes).bytes;
}
public void testDecompress() throws IOException {
@ -89,7 +90,7 @@ public abstract class AbstractTestCompressionMode extends LuceneTestCase {
final int len = random().nextBoolean() ? decompressed.length - off : TestUtil.nextInt(random(), 0, decompressed.length - off);
final byte[] compressed = compress(decompressed, off, len);
final byte[] restored = decompress(compressed, len);
assertArrayEquals(Arrays.copyOfRange(decompressed, off, off+len), restored);
assertArrayEquals(ArrayUtil.copyOfSubArray(decompressed, off, off+len), restored);
}
}
@ -106,7 +107,7 @@ public abstract class AbstractTestCompressionMode extends LuceneTestCase {
length = random().nextInt(decompressed.length - offset);
}
final byte[] restored = decompress(compressed, decompressed.length, offset, length);
assertArrayEquals(Arrays.copyOfRange(decompressed, offset, offset + length), restored);
assertArrayEquals(ArrayUtil.copyOfSubArray(decompressed, offset, offset + length), restored);
}
}

View File

@ -22,13 +22,13 @@ import static org.apache.lucene.codecs.lucene50.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene50.ForUtil.MAX_ENCODED_SIZE;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.packed.PackedInts;
@ -39,7 +39,7 @@ public class TestForUtil extends LuceneTestCase {
public void testEncodeDecode() throws IOException {
final int iterations = RandomNumbers.randomIntBetween(random(), 1, 1000);
final float acceptableOverheadRatio = random().nextFloat();
final int[] values = new int[(iterations - 1) * BLOCK_SIZE + ForUtil.MAX_DATA_SIZE];
final int[] values = new int[iterations * BLOCK_SIZE];
for (int i = 0; i < iterations; ++i) {
final int bpv = random().nextInt(32);
if (bpv == 0) {
@ -64,9 +64,9 @@ public class TestForUtil extends LuceneTestCase {
final ForUtil forUtil = new ForUtil(acceptableOverheadRatio, out);
for (int i = 0; i < iterations; ++i) {
forUtil.writeBlock(
Arrays.copyOfRange(values, i * BLOCK_SIZE, values.length),
new byte[MAX_ENCODED_SIZE], out);
// Although values after BLOCK_SIZE are garbage, we need to allocate extra bytes to avoid AIOOBE.
int[] block = ArrayUtil.grow(ArrayUtil.copyOfSubArray(values, i*BLOCK_SIZE, (i+1)*BLOCK_SIZE));
forUtil.writeBlock(ArrayUtil.grow(block, MAX_DATA_SIZE), new byte[MAX_ENCODED_SIZE], out);
}
endPointer = out.getFilePointer();
out.close();
@ -83,8 +83,8 @@ public class TestForUtil extends LuceneTestCase {
}
final int[] restored = new int[MAX_DATA_SIZE];
forUtil.readBlock(in, new byte[MAX_ENCODED_SIZE], restored);
assertArrayEquals(Arrays.copyOfRange(values, i * BLOCK_SIZE, (i + 1) * BLOCK_SIZE),
Arrays.copyOf(restored, BLOCK_SIZE));
assertArrayEquals(ArrayUtil.copyOfSubArray(values, i*BLOCK_SIZE, (i+1)*BLOCK_SIZE),
ArrayUtil.copyOfSubArray(restored, 0, BLOCK_SIZE));
}
assertEquals(endPointer, in.getFilePointer());
in.close();

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

Some files were not shown because too many files have changed in this diff Show More