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
LuceneTestCase methods (Uwe Schindler)
* LUCENE-9105: UniformSplit postings format detects corrupted index and better handles IO exceptions. (Bruno Roustant)
Optimizations
---------------------
(No changes)

View File

@ -19,6 +19,7 @@ package org.apache.lucene.codecs.uniformsplit;
import java.io.IOException;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
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 {
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 basePositionsFP = input.readVLong();
long basePayloadsFP = input.readVLong();
int termStatesBaseOffset = input.readVInt();
if (termStatesBaseOffset < 0) {
throw new CorruptIndexException("Illegal termStatesBaseOffset= " + termStatesBaseOffset, input);
}
int middleTermOffset = input.readVInt();
if (middleTermOffset < 0) {
throw new CorruptIndexException("Illegal middleTermOffset= " + middleTermOffset, input);
}
BlockHeader blockHeader = reuse == null ? new BlockHeader() : reuse;
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 org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.store.DataInput;
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 {
int termStateRelativeOffset = blockInput.readVInt();
if (termStateRelativeOffset < 0) {
throw new CorruptIndexException("Illegal termStateRelativeOffset= " + termStateRelativeOffset, blockInput);
}
return reuse == null ?
new BlockLine(readIncrementallyEncodedTerm(blockInput, isIncrementalEncodingSeed, null), termStateRelativeOffset)
: reuse.reset(readIncrementallyEncodedTerm(blockInput, isIncrementalEncodingSeed, reuse.termBytes), termStateRelativeOffset);

View File

@ -18,11 +18,11 @@
package org.apache.lucene.codecs.uniformsplit;
import java.io.IOException;
import java.util.function.Supplier;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.BaseTermsEnum;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.TermState;
@ -76,7 +76,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
/**
* {@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.
*/
@ -135,7 +135,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
* @param blockDecoder Optional block decoder, may be null if none.
* 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,
BlockDecoder blockDecoder) throws IOException {
this.dictionaryBrowserSupplier = dictionaryBrowserSupplier;
@ -204,7 +204,9 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
*/
protected SeekStatus seekInBlock(BytesRef searchedTerm, long blockStartFP) throws IOException {
initializeHeader(searchedTerm, blockStartFP);
assert blockHeader != null;
if (blockHeader == null) {
throw newCorruptIndexException("Illegal absence of block", blockStartFP);
}
return seekInBlock(searchedTerm);
}
@ -274,7 +276,9 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
blockReadBuffer.skipBytes(blockHeader.getMiddleLineOffset());
lineIndexInBlock = blockHeader.getMiddleLineIndex();
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());
if (compare < 0) {
blockReadBuffer.setPosition(blockFirstLineStart);
@ -328,7 +332,9 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
public BytesRef next() throws IOException {
if (termStateForced) {
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++) {
readLineInBlock();
}
@ -349,7 +355,9 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
if (blockHeader == null) {
// Read the first block for the field.
initializeHeader(null, fieldMetadata.getFirstBlockStartFP());
assert blockHeader != null;
if (blockHeader == null) {
throw newCorruptIndexException("Illegal absence of first block", fieldMetadata.getFirstBlockStartFP());
}
}
if (readLineInBlock() == null) {
// No more line in the current block.
@ -375,7 +383,9 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
if (blockStartFP == targetBlockStartFP) {
// Optimization: If the block to read is already the current block, then
// 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 the searched term precedes lexicographically the current term,
// 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) {
blockInput = blockInput.clone();
blockLineReader = new BlockLine.Serializer();
@ -520,7 +530,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
+ (termState == null ? 0 : RamUsageUtil.ramBytesUsed(termState));
}
protected IndexDictionary.Browser getOrCreateDictionaryBrowser() {
protected IndexDictionary.Browser getOrCreateDictionaryBrowser() throws IOException {
if (dictionaryBrowser == null) {
dictionaryBrowser = dictionaryBrowserSupplier.get();
}
@ -534,4 +544,10 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
termState = null;
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) {
assert blockOutput != null;
assert targetNumBlockLines > 0;
assert deltaNumLines > 0;
assert deltaNumLines >= 0;
assert deltaNumLines < targetNumBlockLines;
this.blockOutput = blockOutput;
this.targetNumBlockLines = targetNumBlockLines;
@ -245,7 +245,7 @@ public class BlockWriter {
* {@link IndexDictionary.Builder} .
* 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 dictionaryBuilder != null;
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.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IntsRefBuilder;
@ -104,6 +105,10 @@ public class FSTDictionary implements IndexDictionary {
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 final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<>(dictionary);
@ -118,9 +123,8 @@ public class FSTDictionary implements IndexDictionary {
protected long blockFilePointer = -1;
@Override
public long seekBlock(BytesRef term) {
public long seekBlock(BytesRef term) throws IOException {
state = STATE_SEEK;
try {
BytesRefFSTEnum.InputOutput<Long> seekFloor = fstEnum.seekFloor(term);
if (seekFloor == null) {
blockFilePointer = -1;
@ -128,15 +132,10 @@ public class FSTDictionary implements IndexDictionary {
blockFilePointer = seekFloor.output;
}
return blockFilePointer;
} catch (IOException e) {
// Should never happen.
throw new RuntimeException(e);
}
}
@Override
public BytesRef nextKey() {
try {
public BytesRef nextKey() throws IOException {
if (state == STATE_END) {
// if fstEnum is at end, then that's it.
return null;
@ -169,10 +168,6 @@ public class FSTDictionary implements IndexDictionary {
blockPrefixLen = sortKeyLength - 1;
}
return key;
} catch (IOException e) {
// Should never happen.
throw new RuntimeException(e);
}
}
@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}.
*
@ -213,23 +250,13 @@ public class FSTDictionary implements IndexDictionary {
}
@Override
public void add(BytesRef blockKey, long blockFilePointer) {
try {
public void add(BytesRef blockKey, long blockFilePointer) throws IOException {
fstCompiler.add(Util.toIntsRef(blockKey, scratchInts), blockFilePointer);
} catch (IOException e) {
// Should never happen.
throw new RuntimeException(e);
}
}
@Override
public FSTDictionary build() {
try {
public FSTDictionary build() throws IOException {
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 org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexOptions;
@ -193,33 +194,57 @@ public class FieldMetadata implements Accountable {
+ (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();
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();
stats.sumDocFreq = input.readVInt();
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;
fieldMetadata.numTerms = input.readVInt();
if (fieldMetadata.numTerms <= 0) {
throw new CorruptIndexException("Illegal number of terms= " + fieldMetadata.numTerms + " for field= " + fieldId, input);
}
stats.docCount = input.readVInt();
stats.dictionaryStartFP = input.readVLong();
stats.firstBlockStartFP = input.readVLong();
stats.lastBlockStartFP = input.readVLong();
int len = input.readVInt();
BytesRef lastTerm = new BytesRef(len);
if (len > 0) {
input.readBytes(lastTerm.bytes, 0, len);
lastTerm.length = len;
fieldMetadata.sumDocFreq = input.readVInt();
fieldMetadata.sumTotalTermFreq = fieldMetadata.sumDocFreq;
if (fieldMetadata.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0) {
fieldMetadata.sumTotalTermFreq += input.readVInt();
if (fieldMetadata.sumTotalTermFreq < fieldMetadata.sumDocFreq) {
// #positions must be >= #postings.
throw new CorruptIndexException("Illegal sumTotalTermFreq= " + fieldMetadata.sumTotalTermFreq
+ " sumDocFreq= " + fieldMetadata.sumDocFreq + " for field= " + fieldId, input);
}
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 {

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOSupplier;
/**
* Immutable stateless index dictionary kept in RAM.
@ -52,7 +53,7 @@ public interface IndexDictionary extends Accountable {
/**
* Creates a new {@link IndexDictionary.Browser}.
*/
Browser browser();
Browser browser() throws IOException;
/**
* 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
* 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
* 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.
* A key is a prefix of a term in the dictionary.
* If seekBlock was just called then this is the current block key.
*/
BytesRef nextKey();
BytesRef nextKey() throws IOException;
/**
* Returns the next key without advancing.
* 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.
* 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.
* 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;
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 {
super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);

View File

@ -47,7 +47,7 @@ public class UniformSplitTerms extends Terms implements Accountable {
protected final FieldMetadata fieldMetadata;
protected final PostingsReaderBase postingsReader;
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.
@ -55,7 +55,7 @@ public class UniformSplitTerms extends Terms implements Accountable {
protected UniformSplitTerms(IndexInput dictionaryInput, IndexInput blockInput, FieldMetadata fieldMetadata,
PostingsReaderBase postingsReader, BlockDecoder blockDecoder) throws IOException {
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,
PostingsReaderBase postingsReader, BlockDecoder blockDecoder,
DictionaryBrowserSupplier dictionaryBrowserSupplier) {
IndexDictionary.BrowserSupplier dictionaryBrowserSupplier) {
assert fieldMetadata != null;
assert fieldMetadata.getFieldInfo() != 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.FieldsProducer;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
@ -99,7 +100,7 @@ public class UniformSplitTermsReader extends FieldsProducer {
CodecUtil.retrieveChecksum(blockInput);
seekFieldsMetadata(blockInput);
Collection<FieldMetadata> fieldMetadataCollection = parseFieldsMetadata(blockInput, state.fieldInfos);
Collection<FieldMetadata> fieldMetadataCollection = parseFieldsMetadata(blockInput, state.fieldInfos, state.segmentInfo.maxDoc());
fieldToTermsMap = new HashMap<>();
this.blockInput = blockInput;
@ -132,11 +133,14 @@ public class UniformSplitTermsReader extends FieldsProducer {
* @param indexInput {@link IndexInput} must be positioned to the fields metadata
* details by calling {@link #seekFieldsMetadata(IndexInput)} before this call.
*/
protected static Collection<FieldMetadata> parseFieldsMetadata(IndexInput indexInput, FieldInfos fieldInfos) throws IOException {
Collection<FieldMetadata> fieldMetadataCollection = new ArrayList<>();
int fieldsNumber = indexInput.readVInt();
for (int i = 0; i < fieldsNumber; i++) {
fieldMetadataCollection.add(FieldMetadata.read(indexInput, fieldInfos));
protected static Collection<FieldMetadata> parseFieldsMetadata(IndexInput indexInput, FieldInfos fieldInfos, int maxNumDocs) throws IOException {
int numFields = indexInput.readVInt();
if (numFields < 0) {
throw new CorruptIndexException("Illegal number of fields= " + numFields, indexInput);
}
Collection<FieldMetadata> fieldMetadataCollection = new ArrayList<>(numFields);
for (int i = 0; i < numFields; i++) {
fieldMetadataCollection.add(FieldMetadata.read(indexInput, fieldInfos, maxNumDocs));
}
return fieldMetadataCollection;
}

View File

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

View File

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

View File

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

View File

@ -22,8 +22,9 @@ import java.util.Collection;
import org.apache.lucene.codecs.PostingsReaderBase;
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.IndexDictionary;
import org.apache.lucene.codecs.uniformsplit.UniformSplitTerms;
import org.apache.lucene.codecs.uniformsplit.UniformSplitTermsReader;
import org.apache.lucene.index.FieldInfos;
@ -62,7 +63,7 @@ public class STUniformSplitTermsReader extends UniformSplitTermsReader {
if (!fieldMetadataCollection.isEmpty()) {
FieldMetadata unionFieldMetadata = createUnionFieldMetadata(fieldMetadataCollection);
// 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) {
fieldToTermsMap.put(fieldMetadata.getFieldInfo().name,
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 void testEmptyTermSupported() {
public void testEmptyTermSupported() throws Exception {
FSTDictionary indexDictionary = createFSTDictionary(Collections.singletonList(new BytesRef()), new int[]{588});
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")};
FSTDictionary indexDictionary = createFSTDictionary(Arrays.asList(terms), new int[]{588, 588});
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<>();
vocab.add("aswoon");
vocab.add("asyl");
@ -121,7 +121,7 @@ public class TestFSTDictionary extends LuceneTestCase {
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();
for (int i = 0; i < blockKeys.size(); i++) {
builder.add(blockKeys.get(i), blockFPs[i]);
@ -129,7 +129,7 @@ public class TestFSTDictionary extends LuceneTestCase {
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();
for (int i = 0; i < vocab.size(); 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.Map;
import java.util.Set;
import java.util.function.Supplier;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase;
@ -58,7 +57,7 @@ public class STBlockReaderTest extends LuceneTestCase {
private FieldInfos fieldInfos;
private List<MockSTBlockLine> blockLines;
private Supplier<IndexDictionary.Browser> supplier;
private IndexDictionary.BrowserSupplier supplier;
private ByteBuffersDirectory directory;
@Override
@ -83,7 +82,16 @@ public class STBlockReaderTest extends LuceneTestCase {
IndexDictionary.Builder builder = new FSTDictionary.Builder();
builder.add(new BytesRef("a"), 0);
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
@ -242,7 +250,7 @@ public class STBlockReaderTest extends LuceneTestCase {
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),
getMockPostingReaderBase(), mockFieldMetadata(fieldInfo, getLastTermForField(lines, fieldInfo.name)), null, fieldInfos);
this.lines = lines;