LUCENE-9105: UniformSplit postings format detects corrupted index and better handles IO exceptions.

Closes #1105
This commit is contained in:
Bruno Roustant 2019-12-30 12:23:50 +01:00
parent ca6bd364fb
commit bbb6e418e4
No known key found for this signature in database
GPG Key ID: CD28DABB95360525
19 changed files with 242 additions and 215 deletions

View File

@ -102,6 +102,8 @@ Improvements
* LUCENE-9110: Backport refactored stack analysis in tests to use generalized * LUCENE-9110: Backport refactored stack analysis in tests to use generalized
LuceneTestCase methods (Uwe Schindler) LuceneTestCase methods (Uwe Schindler)
* LUCENE-9105: UniformSplit postings format detects corrupted index and better handles IO exceptions. (Bruno Roustant)
Optimizations Optimizations
--------------------- ---------------------
(No changes) (No changes)

View File

@ -19,6 +19,7 @@ package org.apache.lucene.codecs.uniformsplit;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable; import org.apache.lucene.util.Accountable;
@ -147,14 +148,22 @@ public class BlockHeader implements Accountable {
public static BlockHeader read(DataInput input, BlockHeader reuse) throws IOException { public static BlockHeader read(DataInput input, BlockHeader reuse) throws IOException {
int linesCount = input.readVInt(); int linesCount = input.readVInt();
assert linesCount > 0 && linesCount <= UniformSplitTermsWriter.MAX_NUM_BLOCK_LINES : "linesCount=" + linesCount; if (linesCount <= 0 || linesCount > UniformSplitTermsWriter.MAX_NUM_BLOCK_LINES) {
throw new CorruptIndexException("Illegal number of lines in a block: " + linesCount, input);
}
long baseDocsFP = input.readVLong(); long baseDocsFP = input.readVLong();
long basePositionsFP = input.readVLong(); long basePositionsFP = input.readVLong();
long basePayloadsFP = input.readVLong(); long basePayloadsFP = input.readVLong();
int termStatesBaseOffset = input.readVInt(); int termStatesBaseOffset = input.readVInt();
if (termStatesBaseOffset < 0) {
throw new CorruptIndexException("Illegal termStatesBaseOffset= " + termStatesBaseOffset, input);
}
int middleTermOffset = input.readVInt(); int middleTermOffset = input.readVInt();
if (middleTermOffset < 0) {
throw new CorruptIndexException("Illegal middleTermOffset= " + middleTermOffset, input);
}
BlockHeader blockHeader = reuse == null ? new BlockHeader() : reuse; BlockHeader blockHeader = reuse == null ? new BlockHeader() : reuse;
return blockHeader.reset(linesCount, baseDocsFP, basePositionsFP, basePayloadsFP, termStatesBaseOffset, middleTermOffset); return blockHeader.reset(linesCount, baseDocsFP, basePositionsFP, basePayloadsFP, termStatesBaseOffset, middleTermOffset);

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.uniformsplit;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.DataOutput;
@ -129,6 +130,9 @@ public class BlockLine implements Accountable {
*/ */
public BlockLine readLine(DataInput blockInput, boolean isIncrementalEncodingSeed, BlockLine reuse) throws IOException { public BlockLine readLine(DataInput blockInput, boolean isIncrementalEncodingSeed, BlockLine reuse) throws IOException {
int termStateRelativeOffset = blockInput.readVInt(); int termStateRelativeOffset = blockInput.readVInt();
if (termStateRelativeOffset < 0) {
throw new CorruptIndexException("Illegal termStateRelativeOffset= " + termStateRelativeOffset, blockInput);
}
return reuse == null ? return reuse == null ?
new BlockLine(readIncrementallyEncodedTerm(blockInput, isIncrementalEncodingSeed, null), termStateRelativeOffset) new BlockLine(readIncrementallyEncodedTerm(blockInput, isIncrementalEncodingSeed, null), termStateRelativeOffset)
: reuse.reset(readIncrementallyEncodedTerm(blockInput, isIncrementalEncodingSeed, reuse.termBytes), termStateRelativeOffset); : reuse.reset(readIncrementallyEncodedTerm(blockInput, isIncrementalEncodingSeed, reuse.termBytes), termStateRelativeOffset);

View File

@ -18,11 +18,11 @@
package org.apache.lucene.codecs.uniformsplit; package org.apache.lucene.codecs.uniformsplit;
import java.io.IOException; import java.io.IOException;
import java.util.function.Supplier;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.BaseTermsEnum; import org.apache.lucene.index.BaseTermsEnum;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.ImpactsEnum; import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.TermState; import org.apache.lucene.index.TermState;
@ -76,7 +76,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
/** /**
* {@link IndexDictionary.Browser} supplier for lazy loading. * {@link IndexDictionary.Browser} supplier for lazy loading.
*/ */
protected final Supplier<IndexDictionary.Browser> dictionaryBrowserSupplier; protected final IndexDictionary.BrowserSupplier dictionaryBrowserSupplier;
/** /**
* Holds the {@link IndexDictionary.Browser} once loaded. * Holds the {@link IndexDictionary.Browser} once loaded.
*/ */
@ -135,7 +135,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
* @param blockDecoder Optional block decoder, may be null if none. * @param blockDecoder Optional block decoder, may be null if none.
* It can be used for decompression or decryption. * It can be used for decompression or decryption.
*/ */
protected BlockReader(Supplier<IndexDictionary.Browser> dictionaryBrowserSupplier, IndexInput blockInput, protected BlockReader(IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput,
PostingsReaderBase postingsReader, FieldMetadata fieldMetadata, PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
BlockDecoder blockDecoder) throws IOException { BlockDecoder blockDecoder) throws IOException {
this.dictionaryBrowserSupplier = dictionaryBrowserSupplier; this.dictionaryBrowserSupplier = dictionaryBrowserSupplier;
@ -204,7 +204,9 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
*/ */
protected SeekStatus seekInBlock(BytesRef searchedTerm, long blockStartFP) throws IOException { protected SeekStatus seekInBlock(BytesRef searchedTerm, long blockStartFP) throws IOException {
initializeHeader(searchedTerm, blockStartFP); initializeHeader(searchedTerm, blockStartFP);
assert blockHeader != null; if (blockHeader == null) {
throw newCorruptIndexException("Illegal absence of block", blockStartFP);
}
return seekInBlock(searchedTerm); return seekInBlock(searchedTerm);
} }
@ -274,7 +276,9 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
blockReadBuffer.skipBytes(blockHeader.getMiddleLineOffset()); blockReadBuffer.skipBytes(blockHeader.getMiddleLineOffset());
lineIndexInBlock = blockHeader.getMiddleLineIndex(); lineIndexInBlock = blockHeader.getMiddleLineIndex();
readLineInBlock(); readLineInBlock();
assert blockLine != null; if (blockLine == null) {
throw newCorruptIndexException("Illegal absence of line at the middle of the block", null);
}
int compare = searchedTerm.compareTo(term()); int compare = searchedTerm.compareTo(term());
if (compare < 0) { if (compare < 0) {
blockReadBuffer.setPosition(blockFirstLineStart); blockReadBuffer.setPosition(blockFirstLineStart);
@ -328,7 +332,9 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
public BytesRef next() throws IOException { public BytesRef next() throws IOException {
if (termStateForced) { if (termStateForced) {
initializeHeader(forcedTerm.get(), termState.blockFilePointer); initializeHeader(forcedTerm.get(), termState.blockFilePointer);
assert blockHeader != null; if (blockHeader == null) {
throw newCorruptIndexException("Illegal absence of block for TermState", termState.blockFilePointer);
}
for (int i = lineIndexInBlock; i < termState.termBlockOrd; i++) { for (int i = lineIndexInBlock; i < termState.termBlockOrd; i++) {
readLineInBlock(); readLineInBlock();
} }
@ -349,7 +355,9 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
if (blockHeader == null) { if (blockHeader == null) {
// Read the first block for the field. // Read the first block for the field.
initializeHeader(null, fieldMetadata.getFirstBlockStartFP()); initializeHeader(null, fieldMetadata.getFirstBlockStartFP());
assert blockHeader != null; if (blockHeader == null) {
throw newCorruptIndexException("Illegal absence of first block", fieldMetadata.getFirstBlockStartFP());
}
} }
if (readLineInBlock() == null) { if (readLineInBlock() == null) {
// No more line in the current block. // No more line in the current block.
@ -375,7 +383,9 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
if (blockStartFP == targetBlockStartFP) { if (blockStartFP == targetBlockStartFP) {
// Optimization: If the block to read is already the current block, then // Optimization: If the block to read is already the current block, then
// reuse it directly without reading nor decoding the block bytes. // reuse it directly without reading nor decoding the block bytes.
assert blockHeader != null; if (blockHeader == null) {
throw newCorruptIndexException("Illegal absence of block", blockStartFP);
}
if (searchedTerm == null || blockLine == null || searchedTerm.compareTo(blockLine.getTermBytes().getTerm()) <= 0) { if (searchedTerm == null || blockLine == null || searchedTerm.compareTo(blockLine.getTermBytes().getTerm()) <= 0) {
// If the searched term precedes lexicographically the current term, // If the searched term precedes lexicographically the current term,
// then reset the position to the first term line of the block. // then reset the position to the first term line of the block.
@ -393,7 +403,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
} }
} }
protected void initializeBlockReadLazily() { protected void initializeBlockReadLazily() throws IOException {
if (blockStartFP == -1) { if (blockStartFP == -1) {
blockInput = blockInput.clone(); blockInput = blockInput.clone();
blockLineReader = new BlockLine.Serializer(); blockLineReader = new BlockLine.Serializer();
@ -520,7 +530,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
+ (termState == null ? 0 : RamUsageUtil.ramBytesUsed(termState)); + (termState == null ? 0 : RamUsageUtil.ramBytesUsed(termState));
} }
protected IndexDictionary.Browser getOrCreateDictionaryBrowser() { protected IndexDictionary.Browser getOrCreateDictionaryBrowser() throws IOException {
if (dictionaryBrowser == null) { if (dictionaryBrowser == null) {
dictionaryBrowser = dictionaryBrowserSupplier.get(); dictionaryBrowser = dictionaryBrowserSupplier.get();
} }
@ -534,4 +544,10 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
termState = null; termState = null;
termStateForced = false; termStateForced = false;
} }
private CorruptIndexException newCorruptIndexException(String msg, Long fp) {
return new CorruptIndexException(msg
+ (fp == null ? "" : " at FP " + fp)
+ " for field \"" + fieldMetadata.getFieldInfo().name + "\"", blockInput);
}
} }

View File

@ -73,7 +73,7 @@ public class BlockWriter {
protected BlockWriter(IndexOutput blockOutput, int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder) { protected BlockWriter(IndexOutput blockOutput, int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder) {
assert blockOutput != null; assert blockOutput != null;
assert targetNumBlockLines > 0; assert targetNumBlockLines > 0;
assert deltaNumLines > 0; assert deltaNumLines >= 0;
assert deltaNumLines < targetNumBlockLines; assert deltaNumLines < targetNumBlockLines;
this.blockOutput = blockOutput; this.blockOutput = blockOutput;
this.targetNumBlockLines = targetNumBlockLines; this.targetNumBlockLines = targetNumBlockLines;
@ -245,7 +245,7 @@ public class BlockWriter {
* {@link IndexDictionary.Builder} . * {@link IndexDictionary.Builder} .
* The block key is the MDP (see {@link TermBytes}) of the block first term. * The block key is the MDP (see {@link TermBytes}) of the block first term.
*/ */
protected void addBlockKey(List<BlockLine> blockLines, IndexDictionary.Builder dictionaryBuilder) { protected void addBlockKey(List<BlockLine> blockLines, IndexDictionary.Builder dictionaryBuilder) throws IOException {
assert !blockLines.isEmpty(); assert !blockLines.isEmpty();
assert dictionaryBuilder != null; assert dictionaryBuilder != null;
TermBytes firstTerm = blockLines.get(0).getTermBytes(); TermBytes firstTerm = blockLines.get(0).getTermBytes();

View File

@ -1,81 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.uniformsplit;
import java.io.IOException;
import java.util.function.Supplier;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
/**
* Supplier for a new stateful {@link IndexDictionary.Browser} created on
* the immutable {@link IndexDictionary}.
* <p>
* The immutable {@link IndexDictionary} is lazy loaded thread safely. This
* lazy loading allows us to load it only when {@link TermsEnum#seekCeil}
* or {@link TermsEnum#seekExact} are called (it is not loaded for a direct
* all-terms enumeration).
*
* @lucene.experimental
*/
public class DictionaryBrowserSupplier implements Supplier<IndexDictionary.Browser>, Accountable {
protected final IndexInput dictionaryInput;
protected final BlockDecoder blockDecoder;
/**
* Lazy loaded immutable index dictionary (trie hold in RAM).
*/
protected IndexDictionary dictionary;
public DictionaryBrowserSupplier(IndexInput dictionaryInput, long startFilePointer, BlockDecoder blockDecoder) throws IOException {
this.dictionaryInput = dictionaryInput.clone();
this.dictionaryInput.seek(startFilePointer);
this.blockDecoder = blockDecoder;
}
/**
* Gets or lazy loads the immutable {@link IndexDictionary} thread safely
* and creates a new {@link IndexDictionary.Browser}.
*/
@Override
public IndexDictionary.Browser get() {
// This double-check idiom does not require the dictionary to be volatile
// because it is immutable. See section "Double-Checked Locking Immutable Objects"
// of https://www.cs.umd.edu/~pugh/java/memoryModel/DoubleCheckedLocking.html.
if (dictionary == null) {
synchronized (this) {
try {
if (dictionary == null) {
dictionary = FSTDictionary.read(dictionaryInput, blockDecoder);
}
} catch (IOException e) {
throw new IllegalStateException(e);
}
}
}
return dictionary.browser();
}
@Override
public long ramBytesUsed() {
return dictionary == null ? 0 : dictionary.ramBytesUsed();
}
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.ByteBuffersDataOutput; import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IntsRefBuilder; import org.apache.lucene.util.IntsRefBuilder;
@ -104,6 +105,10 @@ public class FSTDictionary implements IndexDictionary {
return new Browser(); return new Browser();
} }
/**
* Stateful {@link Browser} to seek a term in this {@link FSTDictionary}
* and get its corresponding block file pointer in the block file.
*/
protected class Browser implements IndexDictionary.Browser { protected class Browser implements IndexDictionary.Browser {
protected final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<>(dictionary); protected final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<>(dictionary);
@ -118,61 +123,51 @@ public class FSTDictionary implements IndexDictionary {
protected long blockFilePointer = -1; protected long blockFilePointer = -1;
@Override @Override
public long seekBlock(BytesRef term) { public long seekBlock(BytesRef term) throws IOException {
state = STATE_SEEK; state = STATE_SEEK;
try { BytesRefFSTEnum.InputOutput<Long> seekFloor = fstEnum.seekFloor(term);
BytesRefFSTEnum.InputOutput<Long> seekFloor = fstEnum.seekFloor(term); if (seekFloor == null) {
if (seekFloor == null) { blockFilePointer = -1;
blockFilePointer = -1; } else {
} else { blockFilePointer = seekFloor.output;
blockFilePointer = seekFloor.output;
}
return blockFilePointer;
} catch (IOException e) {
// Should never happen.
throw new RuntimeException(e);
} }
return blockFilePointer;
} }
@Override @Override
public BytesRef nextKey() { public BytesRef nextKey() throws IOException {
try { if (state == STATE_END) {
if (state == STATE_END) { // if fstEnum is at end, then that's it.
// if fstEnum is at end, then that's it. return null;
}
if (state == STATE_SEEK && blockFilePointer == -1) { // see seekBlock
if (fstEnum.next() == null) { // advance.
state = STATE_END; // probably never happens (empty FST)? We code defensively.
return null; return null;
} }
if (state == STATE_SEEK && blockFilePointer == -1) { // see seekBlock
if (fstEnum.next() == null) { // advance.
state = STATE_END; // probably never happens (empty FST)? We code defensively.
return null;
}
}
keyBuilder.copyBytes(fstEnum.current().input);
blockFilePointer = fstEnum.current().output;
assert blockFilePointer >= 0;
state = STATE_NEXT;
BytesRef key = keyBuilder.get();
// advance fstEnum
BytesRefFSTEnum.InputOutput<Long> inputOutput = fstEnum.next();
// calc common prefix
if (inputOutput == null) {
state = STATE_END; // for *next* call; current state is good
blockPrefixLen = 0;
} else {
int sortKeyLength = StringHelper.sortKeyLength(key, inputOutput.input);
assert sortKeyLength >= 1;
blockPrefixLen = sortKeyLength - 1;
}
return key;
} catch (IOException e) {
// Should never happen.
throw new RuntimeException(e);
} }
keyBuilder.copyBytes(fstEnum.current().input);
blockFilePointer = fstEnum.current().output;
assert blockFilePointer >= 0;
state = STATE_NEXT;
BytesRef key = keyBuilder.get();
// advance fstEnum
BytesRefFSTEnum.InputOutput<Long> inputOutput = fstEnum.next();
// calc common prefix
if (inputOutput == null) {
state = STATE_END; // for *next* call; current state is good
blockPrefixLen = 0;
} else {
int sortKeyLength = StringHelper.sortKeyLength(key, inputOutput.input);
assert sortKeyLength >= 1;
blockPrefixLen = sortKeyLength - 1;
}
return key;
} }
@Override @Override
@ -196,6 +191,48 @@ public class FSTDictionary implements IndexDictionary {
} }
} }
/**
* Provides stateful {@link Browser} to seek in the {@link FSTDictionary}.
*
* @lucene.experimental
*/
public static class BrowserSupplier implements IndexDictionary.BrowserSupplier {
protected final IndexInput dictionaryInput;
protected final BlockDecoder blockDecoder;
/**
* Lazy loaded immutable index dictionary (trie hold in RAM).
*/
protected IndexDictionary dictionary;
public BrowserSupplier(IndexInput dictionaryInput, long startFilePointer, BlockDecoder blockDecoder) throws IOException {
this.dictionaryInput = dictionaryInput.clone();
this.dictionaryInput.seek(startFilePointer);
this.blockDecoder = blockDecoder;
}
@Override
public IndexDictionary.Browser get() throws IOException {
// This double-check idiom does not require the dictionary to be volatile
// because it is immutable. See section "Double-Checked Locking Immutable Objects"
// of https://www.cs.umd.edu/~pugh/java/memoryModel/DoubleCheckedLocking.html.
if (dictionary == null) {
synchronized (this) {
if (dictionary == null) {
dictionary = read(dictionaryInput, blockDecoder);
}
}
}
return dictionary.browser();
}
@Override
public long ramBytesUsed() {
return dictionary == null ? 0 : dictionary.ramBytesUsed();
}
}
/** /**
* Builds an immutable {@link FSTDictionary}. * Builds an immutable {@link FSTDictionary}.
* *
@ -213,23 +250,13 @@ public class FSTDictionary implements IndexDictionary {
} }
@Override @Override
public void add(BytesRef blockKey, long blockFilePointer) { public void add(BytesRef blockKey, long blockFilePointer) throws IOException {
try { fstCompiler.add(Util.toIntsRef(blockKey, scratchInts), blockFilePointer);
fstCompiler.add(Util.toIntsRef(blockKey, scratchInts), blockFilePointer);
} catch (IOException e) {
// Should never happen.
throw new RuntimeException(e);
}
} }
@Override @Override
public FSTDictionary build() { public FSTDictionary build() throws IOException {
try { return new FSTDictionary(fstCompiler.compile());
return new FSTDictionary(fstCompiler.compile());
} catch (IOException e) {
// Should never happen.
throw new RuntimeException(e);
}
} }
} }
} }

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.uniformsplit;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexOptions;
@ -193,33 +194,57 @@ public class FieldMetadata implements Accountable {
+ (docsSeen == null ? 0 : docsSeen.ramBytesUsed()); + (docsSeen == null ? 0 : docsSeen.ramBytesUsed());
} }
public static FieldMetadata read(DataInput input, FieldInfos fieldInfos) throws IOException { public static FieldMetadata read(DataInput input, FieldInfos fieldInfos, int maxNumDocs) throws IOException {
int fieldId = input.readVInt(); int fieldId = input.readVInt();
FieldMetadata stats = new FieldMetadata(fieldInfos.fieldInfo(fieldId), 0, false); FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldId);
if (fieldInfo == null) {
throw new CorruptIndexException("Illegal field id= " + fieldId, input);
}
FieldMetadata fieldMetadata = new FieldMetadata(fieldInfo, 0, false);
stats.numTerms = input.readVInt(); fieldMetadata.numTerms = input.readVInt();
stats.sumDocFreq = input.readVInt(); if (fieldMetadata.numTerms <= 0) {
throw new CorruptIndexException("Illegal number of terms= " + fieldMetadata.numTerms + " for field= " + fieldId, input);
stats.sumTotalTermFreq = stats.sumDocFreq;
if (stats.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0) {
stats.sumTotalTermFreq += input.readVInt();
assert stats.sumTotalTermFreq >= stats.sumDocFreq : "sumTotalFQ: " + stats.sumTotalTermFreq + " sumDocFQ: " + stats.sumDocFreq;
} }
stats.docCount = input.readVInt(); fieldMetadata.sumDocFreq = input.readVInt();
fieldMetadata.sumTotalTermFreq = fieldMetadata.sumDocFreq;
stats.dictionaryStartFP = input.readVLong(); if (fieldMetadata.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0) {
stats.firstBlockStartFP = input.readVLong(); fieldMetadata.sumTotalTermFreq += input.readVInt();
stats.lastBlockStartFP = input.readVLong(); if (fieldMetadata.sumTotalTermFreq < fieldMetadata.sumDocFreq) {
// #positions must be >= #postings.
int len = input.readVInt(); throw new CorruptIndexException("Illegal sumTotalTermFreq= " + fieldMetadata.sumTotalTermFreq
BytesRef lastTerm = new BytesRef(len); + " sumDocFreq= " + fieldMetadata.sumDocFreq + " for field= " + fieldId, input);
if (len > 0) { }
input.readBytes(lastTerm.bytes, 0, len);
lastTerm.length = len;
} }
stats.setLastTerm(lastTerm);
return stats; fieldMetadata.docCount = input.readVInt();
if (fieldMetadata.docCount < 0 || fieldMetadata.docCount > maxNumDocs) {
// #docs with field must be <= #docs.
throw new CorruptIndexException("Illegal number of docs= " + fieldMetadata.docCount
+ " maxNumDocs= " + maxNumDocs + " for field=" + fieldId, input);
}
if (fieldMetadata.sumDocFreq < fieldMetadata.docCount) {
// #postings must be >= #docs with field.
throw new CorruptIndexException("Illegal sumDocFreq= " + fieldMetadata.sumDocFreq
+ " docCount= " + fieldMetadata.docCount + " for field= " + fieldId, input);
}
fieldMetadata.dictionaryStartFP = input.readVLong();
fieldMetadata.firstBlockStartFP = input.readVLong();
fieldMetadata.lastBlockStartFP = input.readVLong();
int lastTermLength = input.readVInt();
BytesRef lastTerm = new BytesRef(lastTermLength);
if (lastTermLength > 0) {
input.readBytes(lastTerm.bytes, 0, lastTermLength);
lastTerm.length = lastTermLength;
} else if (lastTermLength < 0) {
throw new CorruptIndexException("Illegal last term length= " + lastTermLength + " for field= " + fieldId, input);
}
fieldMetadata.setLastTerm(lastTerm);
return fieldMetadata;
} }
public void write(DataOutput output) throws IOException { public void write(DataOutput output) throws IOException {

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable; import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOSupplier;
/** /**
* Immutable stateless index dictionary kept in RAM. * Immutable stateless index dictionary kept in RAM.
@ -52,7 +53,7 @@ public interface IndexDictionary extends Accountable {
/** /**
* Creates a new {@link IndexDictionary.Browser}. * Creates a new {@link IndexDictionary.Browser}.
*/ */
Browser browser(); Browser browser() throws IOException;
/** /**
* Builds an immutable {@link IndexDictionary}. * Builds an immutable {@link IndexDictionary}.
@ -75,9 +76,12 @@ public interface IndexDictionary extends Accountable {
* @param blockFilePointer Non-negative file pointer to the start of the * @param blockFilePointer Non-negative file pointer to the start of the
* block in the block file. * block in the block file.
*/ */
void add(BytesRef blockKey, long blockFilePointer); void add(BytesRef blockKey, long blockFilePointer) throws IOException;
IndexDictionary build(); /**
* Builds the immutable {@link IndexDictionary} for the added entries.
*/
IndexDictionary build() throws IOException;
} }
/** /**
@ -97,31 +101,41 @@ public interface IndexDictionary extends Accountable {
* Otherwise {@code -1} if there is no floor block key because the searched * Otherwise {@code -1} if there is no floor block key because the searched
* term precedes alphabetically the first block key of the dictionary. * term precedes alphabetically the first block key of the dictionary.
*/ */
long seekBlock(BytesRef term); long seekBlock(BytesRef term) throws IOException;
/** /**
* Returns the next block key and positions the browser at this key. * Returns the next block key and positions the browser at this key.
* A key is a prefix of a term in the dictionary. * A key is a prefix of a term in the dictionary.
* If seekBlock was just called then this is the current block key. * If seekBlock was just called then this is the current block key.
*/ */
BytesRef nextKey(); BytesRef nextKey() throws IOException;
/** /**
* Returns the next key without advancing. * Returns the next key without advancing.
* Only call this after {@link #nextKey()} returns a non-null result. * Only call this after {@link #nextKey()} returns a non-null result.
*/ */
BytesRef peekKey(); BytesRef peekKey() throws IOException;
/** /**
* Returns the number of characters of this block's key that is in common with all terms in this block. * Returns the number of characters of this block's key that is in common with all terms in this block.
* Only call this after {@link #nextKey()} returns a non-null result. * Only call this after {@link #nextKey()} returns a non-null result.
*/ */
int getBlockPrefixLen(); int getBlockPrefixLen() throws IOException;
/** /**
* Returns the block file pointer associated with the key returned. * Returns the block file pointer associated with the key returned.
* Only call this after {@link #nextKey()} returns a non-null result. * Only call this after {@link #nextKey()} returns a non-null result.
*/ */
long getBlockFilePointer(); long getBlockFilePointer() throws IOException;
} }
/**
* Supplier for a new stateful {@link Browser} created on the immutable {@link IndexDictionary}.
* <p>
* The immutable {@link IndexDictionary} is lazy loaded thread safely. This lazy loading allows
* us to load it only when {@link org.apache.lucene.index.TermsEnum#seekCeil} or
* {@link org.apache.lucene.index.TermsEnum#seekExact} are called (it is not loaded for a direct
* all-terms enumeration).
*/
interface BrowserSupplier extends IOSupplier<Browser>, Accountable {}
} }

View File

@ -63,7 +63,7 @@ public class IntersectBlockReader extends BlockReader {
protected boolean beyondCommonPrefix; protected boolean beyondCommonPrefix;
public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm, public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
DictionaryBrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader, IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException { FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder); super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
this.nextStringCalculator = new AutomatonNextTermCalculator(compiled); this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);

View File

@ -47,7 +47,7 @@ public class UniformSplitTerms extends Terms implements Accountable {
protected final FieldMetadata fieldMetadata; protected final FieldMetadata fieldMetadata;
protected final PostingsReaderBase postingsReader; protected final PostingsReaderBase postingsReader;
protected final BlockDecoder blockDecoder; protected final BlockDecoder blockDecoder;
protected final DictionaryBrowserSupplier dictionaryBrowserSupplier; protected final IndexDictionary.BrowserSupplier dictionaryBrowserSupplier;
/** /**
* @param blockDecoder Optional block decoder, may be null if none. It can be used for decompression or decryption. * @param blockDecoder Optional block decoder, may be null if none. It can be used for decompression or decryption.
@ -55,7 +55,7 @@ public class UniformSplitTerms extends Terms implements Accountable {
protected UniformSplitTerms(IndexInput dictionaryInput, IndexInput blockInput, FieldMetadata fieldMetadata, protected UniformSplitTerms(IndexInput dictionaryInput, IndexInput blockInput, FieldMetadata fieldMetadata,
PostingsReaderBase postingsReader, BlockDecoder blockDecoder) throws IOException { PostingsReaderBase postingsReader, BlockDecoder blockDecoder) throws IOException {
this(blockInput, fieldMetadata, postingsReader, blockDecoder, this(blockInput, fieldMetadata, postingsReader, blockDecoder,
new DictionaryBrowserSupplier(dictionaryInput, fieldMetadata.getDictionaryStartFP(), blockDecoder)); new FSTDictionary.BrowserSupplier(dictionaryInput, fieldMetadata.getDictionaryStartFP(), blockDecoder));
} }
/** /**
@ -63,7 +63,7 @@ public class UniformSplitTerms extends Terms implements Accountable {
*/ */
protected UniformSplitTerms(IndexInput blockInput, FieldMetadata fieldMetadata, protected UniformSplitTerms(IndexInput blockInput, FieldMetadata fieldMetadata,
PostingsReaderBase postingsReader, BlockDecoder blockDecoder, PostingsReaderBase postingsReader, BlockDecoder blockDecoder,
DictionaryBrowserSupplier dictionaryBrowserSupplier) { IndexDictionary.BrowserSupplier dictionaryBrowserSupplier) {
assert fieldMetadata != null; assert fieldMetadata != null;
assert fieldMetadata.getFieldInfo() != null; assert fieldMetadata.getFieldInfo() != null;
assert fieldMetadata.getLastTerm() != null; assert fieldMetadata.getLastTerm() != null;

View File

@ -29,6 +29,7 @@ import java.util.Map;
import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
@ -99,7 +100,7 @@ public class UniformSplitTermsReader extends FieldsProducer {
CodecUtil.retrieveChecksum(blockInput); CodecUtil.retrieveChecksum(blockInput);
seekFieldsMetadata(blockInput); seekFieldsMetadata(blockInput);
Collection<FieldMetadata> fieldMetadataCollection = parseFieldsMetadata(blockInput, state.fieldInfos); Collection<FieldMetadata> fieldMetadataCollection = parseFieldsMetadata(blockInput, state.fieldInfos, state.segmentInfo.maxDoc());
fieldToTermsMap = new HashMap<>(); fieldToTermsMap = new HashMap<>();
this.blockInput = blockInput; this.blockInput = blockInput;
@ -132,11 +133,14 @@ public class UniformSplitTermsReader extends FieldsProducer {
* @param indexInput {@link IndexInput} must be positioned to the fields metadata * @param indexInput {@link IndexInput} must be positioned to the fields metadata
* details by calling {@link #seekFieldsMetadata(IndexInput)} before this call. * details by calling {@link #seekFieldsMetadata(IndexInput)} before this call.
*/ */
protected static Collection<FieldMetadata> parseFieldsMetadata(IndexInput indexInput, FieldInfos fieldInfos) throws IOException { protected static Collection<FieldMetadata> parseFieldsMetadata(IndexInput indexInput, FieldInfos fieldInfos, int maxNumDocs) throws IOException {
Collection<FieldMetadata> fieldMetadataCollection = new ArrayList<>(); int numFields = indexInput.readVInt();
int fieldsNumber = indexInput.readVInt(); if (numFields < 0) {
for (int i = 0; i < fieldsNumber; i++) { throw new CorruptIndexException("Illegal number of fields= " + numFields, indexInput);
fieldMetadataCollection.add(FieldMetadata.read(indexInput, fieldInfos)); }
Collection<FieldMetadata> fieldMetadataCollection = new ArrayList<>(numFields);
for (int i = 0; i < numFields; i++) {
fieldMetadataCollection.add(FieldMetadata.read(indexInput, fieldInfos, maxNumDocs));
} }
return fieldMetadataCollection; return fieldMetadataCollection;
} }

View File

@ -18,7 +18,6 @@
package org.apache.lucene.codecs.uniformsplit.sharedterms; package org.apache.lucene.codecs.uniformsplit.sharedterms;
import java.io.IOException; import java.io.IOException;
import java.util.function.Supplier;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
@ -40,7 +39,7 @@ public class STBlockReader extends BlockReader {
protected final FieldInfos fieldInfos; protected final FieldInfos fieldInfos;
public STBlockReader(Supplier<IndexDictionary.Browser> dictionaryBrowserSupplier, public STBlockReader(IndexDictionary.BrowserSupplier dictionaryBrowserSupplier,
IndexInput blockInput, PostingsReaderBase postingsReader, IndexInput blockInput, PostingsReaderBase postingsReader,
FieldMetadata fieldMetadata, BlockDecoder blockDecoder, FieldInfos fieldInfos) throws IOException { FieldMetadata fieldMetadata, BlockDecoder blockDecoder, FieldInfos fieldInfos) throws IOException {
super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder); super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);

View File

@ -22,8 +22,8 @@ import java.io.IOException;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.uniformsplit.BlockDecoder; import org.apache.lucene.codecs.uniformsplit.BlockDecoder;
import org.apache.lucene.codecs.uniformsplit.DictionaryBrowserSupplier;
import org.apache.lucene.codecs.uniformsplit.FieldMetadata; import org.apache.lucene.codecs.uniformsplit.FieldMetadata;
import org.apache.lucene.codecs.uniformsplit.IndexDictionary;
import org.apache.lucene.codecs.uniformsplit.IntersectBlockReader; import org.apache.lucene.codecs.uniformsplit.IntersectBlockReader;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexInput;
@ -42,8 +42,8 @@ public class STIntersectBlockReader extends IntersectBlockReader {
protected final FieldInfos fieldInfos; protected final FieldInfos fieldInfos;
public STIntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm, public STIntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
DictionaryBrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader, IndexDictionary.BrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
FieldMetadata fieldMetadata, BlockDecoder blockDecoder, FieldInfos fieldInfos) throws IOException { FieldMetadata fieldMetadata, BlockDecoder blockDecoder, FieldInfos fieldInfos) throws IOException {
super(compiled, startTerm, dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder); super(compiled, startTerm, dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
this.fieldInfos = fieldInfos; this.fieldInfos = fieldInfos;
} }

View File

@ -19,7 +19,6 @@ package org.apache.lucene.codecs.uniformsplit.sharedterms;
import java.io.IOException; import java.io.IOException;
import java.util.Map; import java.util.Map;
import java.util.function.Supplier;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
@ -44,7 +43,7 @@ import org.apache.lucene.util.BytesRef;
public class STMergingBlockReader extends STBlockReader { public class STMergingBlockReader extends STBlockReader {
public STMergingBlockReader( public STMergingBlockReader(
Supplier<IndexDictionary.Browser> dictionaryBrowserSupplier, IndexDictionary.BrowserSupplier dictionaryBrowserSupplier,
IndexInput blockInput, IndexInput blockInput,
PostingsReaderBase postingsReader, PostingsReaderBase postingsReader,
FieldMetadata fieldMetadata, FieldMetadata fieldMetadata,

View File

@ -21,8 +21,8 @@ import java.io.IOException;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.uniformsplit.BlockDecoder; import org.apache.lucene.codecs.uniformsplit.BlockDecoder;
import org.apache.lucene.codecs.uniformsplit.DictionaryBrowserSupplier;
import org.apache.lucene.codecs.uniformsplit.FieldMetadata; import org.apache.lucene.codecs.uniformsplit.FieldMetadata;
import org.apache.lucene.codecs.uniformsplit.IndexDictionary;
import org.apache.lucene.codecs.uniformsplit.UniformSplitTerms; import org.apache.lucene.codecs.uniformsplit.UniformSplitTerms;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.TermsEnum;
@ -43,7 +43,7 @@ public class STUniformSplitTerms extends UniformSplitTerms {
protected STUniformSplitTerms(IndexInput blockInput, FieldMetadata fieldMetadata, protected STUniformSplitTerms(IndexInput blockInput, FieldMetadata fieldMetadata,
FieldMetadata unionFieldMetadata, PostingsReaderBase postingsReader, FieldMetadata unionFieldMetadata, PostingsReaderBase postingsReader,
BlockDecoder blockDecoder, FieldInfos fieldInfos, DictionaryBrowserSupplier dictionaryBrowserSupplier) { BlockDecoder blockDecoder, FieldInfos fieldInfos, IndexDictionary.BrowserSupplier dictionaryBrowserSupplier) {
super(blockInput, fieldMetadata, postingsReader, blockDecoder, dictionaryBrowserSupplier); super(blockInput, fieldMetadata, postingsReader, blockDecoder, dictionaryBrowserSupplier);
this.unionFieldMetadata = unionFieldMetadata; this.unionFieldMetadata = unionFieldMetadata;
this.fieldInfos = fieldInfos; this.fieldInfos = fieldInfos;

View File

@ -22,8 +22,9 @@ import java.util.Collection;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.uniformsplit.BlockDecoder; import org.apache.lucene.codecs.uniformsplit.BlockDecoder;
import org.apache.lucene.codecs.uniformsplit.DictionaryBrowserSupplier; import org.apache.lucene.codecs.uniformsplit.FSTDictionary;
import org.apache.lucene.codecs.uniformsplit.FieldMetadata; import org.apache.lucene.codecs.uniformsplit.FieldMetadata;
import org.apache.lucene.codecs.uniformsplit.IndexDictionary;
import org.apache.lucene.codecs.uniformsplit.UniformSplitTerms; import org.apache.lucene.codecs.uniformsplit.UniformSplitTerms;
import org.apache.lucene.codecs.uniformsplit.UniformSplitTermsReader; import org.apache.lucene.codecs.uniformsplit.UniformSplitTermsReader;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
@ -62,7 +63,7 @@ public class STUniformSplitTermsReader extends UniformSplitTermsReader {
if (!fieldMetadataCollection.isEmpty()) { if (!fieldMetadataCollection.isEmpty()) {
FieldMetadata unionFieldMetadata = createUnionFieldMetadata(fieldMetadataCollection); FieldMetadata unionFieldMetadata = createUnionFieldMetadata(fieldMetadataCollection);
// Share the same immutable dictionary between all fields. // Share the same immutable dictionary between all fields.
DictionaryBrowserSupplier dictionaryBrowserSupplier = new DictionaryBrowserSupplier(dictionaryInput, fieldMetadataCollection.iterator().next().getDictionaryStartFP(), blockDecoder); IndexDictionary.BrowserSupplier dictionaryBrowserSupplier = new FSTDictionary.BrowserSupplier(dictionaryInput, fieldMetadataCollection.iterator().next().getDictionaryStartFP(), blockDecoder);
for (FieldMetadata fieldMetadata : fieldMetadataCollection) { for (FieldMetadata fieldMetadata : fieldMetadataCollection) {
fieldToTermsMap.put(fieldMetadata.getFieldInfo().name, fieldToTermsMap.put(fieldMetadata.getFieldInfo().name,
new STUniformSplitTerms(blockInput, fieldMetadata, unionFieldMetadata, postingsReader, blockDecoder, fieldInfos, dictionaryBrowserSupplier)); new STUniformSplitTerms(blockInput, fieldMetadata, unionFieldMetadata, postingsReader, blockDecoder, fieldInfos, dictionaryBrowserSupplier));

View File

@ -33,7 +33,7 @@ import org.apache.lucene.util.LuceneTestCase;
*/ */
public class TestFSTDictionary extends LuceneTestCase { public class TestFSTDictionary extends LuceneTestCase {
public void testEmptyTermSupported() { public void testEmptyTermSupported() throws Exception {
FSTDictionary indexDictionary = createFSTDictionary(Collections.singletonList(new BytesRef()), new int[]{588}); FSTDictionary indexDictionary = createFSTDictionary(Collections.singletonList(new BytesRef()), new int[]{588});
assertEquals(588, indexDictionary.browser().seekBlock(new BytesRef())); assertEquals(588, indexDictionary.browser().seekBlock(new BytesRef()));
} }
@ -49,7 +49,7 @@ public class TestFSTDictionary extends LuceneTestCase {
} }
} }
public void testRepeatedOutputAllowed() { public void testRepeatedOutputAllowed() throws Exception {
BytesRef[] terms = {new BytesRef("a"), new BytesRef("b")}; BytesRef[] terms = {new BytesRef("a"), new BytesRef("b")};
FSTDictionary indexDictionary = createFSTDictionary(Arrays.asList(terms), new int[]{588, 588}); FSTDictionary indexDictionary = createFSTDictionary(Arrays.asList(terms), new int[]{588, 588});
assertEquals(588, indexDictionary.browser().seekBlock(new BytesRef("a"))); assertEquals(588, indexDictionary.browser().seekBlock(new BytesRef("a")));
@ -83,7 +83,7 @@ public class TestFSTDictionary extends LuceneTestCase {
} }
} }
public void testCommonPrefixes() { public void testCommonPrefixes() throws Exception {
List<String> vocab = new ArrayList<>(); List<String> vocab = new ArrayList<>();
vocab.add("aswoon"); vocab.add("aswoon");
vocab.add("asyl"); vocab.add("asyl");
@ -121,7 +121,7 @@ public class TestFSTDictionary extends LuceneTestCase {
assertEquals(blockFPs[9], browser.seekBlock(new BytesRef("asymmetriesz"))); assertEquals(blockFPs[9], browser.seekBlock(new BytesRef("asymmetriesz")));
} }
private static FSTDictionary createFSTDictionary(List<BytesRef> blockKeys, int[] blockFPs) { private static FSTDictionary createFSTDictionary(List<BytesRef> blockKeys, int[] blockFPs) throws IOException {
FSTDictionary.Builder builder = new FSTDictionary.Builder(); FSTDictionary.Builder builder = new FSTDictionary.Builder();
for (int i = 0; i < blockKeys.size(); i++) { for (int i = 0; i < blockKeys.size(); i++) {
builder.add(blockKeys.get(i), blockFPs[i]); builder.add(blockKeys.get(i), blockFPs[i]);
@ -129,7 +129,7 @@ public class TestFSTDictionary extends LuceneTestCase {
return builder.build(); return builder.build();
} }
private static FSTDictionary createFSTDictionary(List<String> vocab) { private static FSTDictionary createFSTDictionary(List<String> vocab) throws IOException {
FSTDictionary.Builder builder = new FSTDictionary.Builder(); FSTDictionary.Builder builder = new FSTDictionary.Builder();
for (int i = 0; i < vocab.size(); i++) { for (int i = 0; i < vocab.size(); i++) {
builder.add(new BytesRef(vocab.get(i)), i); builder.add(new BytesRef(vocab.get(i)), i);

View File

@ -25,7 +25,6 @@ import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.function.Supplier;
import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsReaderBase;
@ -58,7 +57,7 @@ public class STBlockReaderTest extends LuceneTestCase {
private FieldInfos fieldInfos; private FieldInfos fieldInfos;
private List<MockSTBlockLine> blockLines; private List<MockSTBlockLine> blockLines;
private Supplier<IndexDictionary.Browser> supplier; private IndexDictionary.BrowserSupplier supplier;
private ByteBuffersDirectory directory; private ByteBuffersDirectory directory;
@Override @Override
@ -83,7 +82,16 @@ public class STBlockReaderTest extends LuceneTestCase {
IndexDictionary.Builder builder = new FSTDictionary.Builder(); IndexDictionary.Builder builder = new FSTDictionary.Builder();
builder.add(new BytesRef("a"), 0); builder.add(new BytesRef("a"), 0);
IndexDictionary indexDictionary = builder.build(); IndexDictionary indexDictionary = builder.build();
supplier = indexDictionary::browser; supplier = new IndexDictionary.BrowserSupplier() {
@Override
public IndexDictionary.Browser get() throws IOException {
return indexDictionary.browser();
}
@Override
public long ramBytesUsed() {
return indexDictionary.ramBytesUsed();
}
};
} }
@Override @Override
@ -242,7 +250,7 @@ public class STBlockReaderTest extends LuceneTestCase {
List<MockSTBlockLine> lines; List<MockSTBlockLine> lines;
MockSTBlockReader(Supplier<IndexDictionary.Browser> supplier, List<MockSTBlockLine> lines, Directory directory, FieldInfo fieldInfo, FieldInfos fieldInfos) throws IOException { MockSTBlockReader(IndexDictionary.BrowserSupplier supplier, List<MockSTBlockLine> lines, Directory directory, FieldInfo fieldInfo, FieldInfos fieldInfos) throws IOException {
super(supplier, directory.openInput(MOCK_BLOCK_OUTPUT_NAME, IOContext.DEFAULT), super(supplier, directory.openInput(MOCK_BLOCK_OUTPUT_NAME, IOContext.DEFAULT),
getMockPostingReaderBase(), mockFieldMetadata(fieldInfo, getLastTermForField(lines, fieldInfo.name)), null, fieldInfos); getMockPostingReaderBase(), mockFieldMetadata(fieldInfo, getLastTermForField(lines, fieldInfo.name)), null, fieldInfos);
this.lines = lines; this.lines = lines;