LUCENE-8753: New UniformSplit and SharedTermsUniformSplit PostingsFormats

This commit is contained in:
David Smiley 2019-09-06 11:48:32 -04:00
parent 6574ae63d4
commit b963b7c3db
48 changed files with 7136 additions and 1 deletions

View File

@ -95,6 +95,10 @@ New Features
* LUCENE-8960: Introduce LatLonDocValuesPointInPolygonQuery for LatLonDocValuesField (Ignacio Vera)
* LUCENE-8753: New UniformSplitPostingsFormat (name "UniformSplit") primarily benefiting in simplicity and
extensibility. New STUniformSplitPostingsFormat (name "SharedTermsUniformSplit") that shares a single internal
term dictionary across fields. (Bruno Roustant, Juan Rodriguez, David Smiley)
Improvements
* LUCENE-8874: Show SPI names instead of class names in Luke Analysis tab. (Tomoko Uchida)

View File

@ -0,0 +1,48 @@
/*
* 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 org.apache.lucene.store.DataInput;
import org.apache.lucene.util.BytesRef;
/**
* Decodes the raw bytes of a block when the index is read, according to the
* {@link BlockEncoder} used during the writing of the index.
* <p>
* For example, implementations may decompress or decrypt.
*
* @see BlockEncoder
* @lucene.experimental
*/
public interface BlockDecoder {
/**
* Decodes all the bytes of one block in a single operation. The decoding is per block.
* @param blockBytes The input block bytes to read.
* @param length The number of bytes to read from the input.
* @return The decoded block bytes.
* @throws IOException If a decoding error occurs.
*/
// TODO: this method could return a new interface ReadableBytes which
// would provide the size and create PositionableDataInput (get/set position)
// implemented by ByteArrayDataInput and ByteBuffersDataInput.
// Big length could be supported (> Integer.MAX_VALUE).
BytesRef decode(DataInput blockBytes, long length) throws IOException;
}

View File

@ -0,0 +1,59 @@
/*
* 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 org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
/**
* Encodes the raw bytes of a block when the index is written.
* <p>
* For example, implementations may compress or encrypt.
*
* @see BlockDecoder
* @lucene.experimental
*/
public interface BlockEncoder {
/**
* Encodes all the bytes of one block in a single operation. The encoding is per block.
* @param blockBytes The input block bytes to read.
* @param length The number of bytes to read from the input.
* @return The encoded block bytes.
* @throws IOException If an encoding error occurs.
*/
WritableBytes encode(DataInput blockBytes, long length) throws IOException;
/**
* Writable byte buffer.
*/
interface WritableBytes {
/**
* Gets the number of bytes.
*/
long size();
/**
* Writes the bytes to the provided {@link DataOutput}.
*/
void writeTo(DataOutput dataOutput) throws IOException;
}
}

View File

@ -0,0 +1,167 @@
/*
* 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 org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Block header containing block metadata.
* <p>
* Holds the number of lines in the block.
* <p>
* Holds the base file pointers to apply delta base encoding to all the file
* pointers in the block with {@link DeltaBaseTermStateSerializer}.
* <p>
* Holds the offset to the details region of the block (the term states).
* <p>
* Holds the offset to the middle term of the block to divide the number
* of terms to scan by 2.
*
* @lucene.experimental
*/
public class BlockHeader implements Accountable {
private static final long RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(BlockHeader.class);
protected int linesCount;
protected long baseDocsFP;
protected long basePositionsFP;
protected long basePayloadsFP;
protected int termStatesBaseOffset;
protected int middleLineIndex;
protected int middleLineOffset;
/**
* @param linesCount Number of lines in the block.
* @param baseDocsFP File pointer to the docs of the first term with docs in the block.
* @param basePositionsFP File pointer to the positions of the first term with positions in the block.
* @param basePayloadsFP File pointer to the payloads of the first term with payloads in the block.
* @param termStatesBaseOffset Offset to the details region of the block (the term states), relative to the block start.
* @param middleLineOffset Offset to the middle term of the block, relative to the block start.
*/
protected BlockHeader(int linesCount, long baseDocsFP, long basePositionsFP, long basePayloadsFP,
int termStatesBaseOffset, int middleLineOffset) {
reset(linesCount, baseDocsFP, basePositionsFP, basePayloadsFP, termStatesBaseOffset, middleLineOffset);
}
/**
* Empty constructor. {@link #reset} must be called before writing.
*/
protected BlockHeader() {
}
protected BlockHeader reset(int linesCount, long baseDocsFP, long basePositionsFP,
long basePayloadsFP, int termStatesBaseOffset, int middleTermOffset) {
this.baseDocsFP = baseDocsFP;
this.basePositionsFP = basePositionsFP;
this.basePayloadsFP = basePayloadsFP;
this.linesCount = linesCount;
this.middleLineIndex = linesCount >> 1;
this.termStatesBaseOffset = termStatesBaseOffset;
this.middleLineOffset = middleTermOffset;
return this;
}
/**
* @return The number of lines in the block.
*/
public int getLinesCount() {
return linesCount;
}
/**
* @return The index of the middle line of the block.
*/
public int getMiddleLineIndex() {
return middleLineIndex;
}
/**
* @return The offset to the middle line of the block, relative to the block start.
*/
public int getMiddleLineOffset() {
return middleLineOffset;
}
/**
* @return The offset to the details region of the block (the term states), relative to the block start.
*/
public int getTermStatesBaseOffset() {
return termStatesBaseOffset;
}
/**
* @return The file pointer to the docs of the first term with docs in the block.
*/
public long getBaseDocsFP() {
return baseDocsFP;
}
/**
* @return The file pointer to the positions of the first term with positions in the block.
*/
public long getBasePositionsFP() {
return basePositionsFP;
}
/**
* @return The file pointer to the payloads of the first term with payloads in the block.
*/
public long getBasePayloadsFP() {
return basePayloadsFP;
}
public void write(DataOutput output) throws IOException {
assert linesCount > 0 : "block header does not seem to be initialized";
output.writeVInt(linesCount);
output.writeVLong(baseDocsFP);
output.writeVLong(basePositionsFP);
output.writeVLong(basePayloadsFP);
output.writeVInt(termStatesBaseOffset);
output.writeVInt(middleLineOffset);
}
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;
long baseDocsFP = input.readVLong();
long basePositionsFP = input.readVLong();
long basePayloadsFP = input.readVLong();
int termStatesBaseOffset = input.readVInt();
int middleTermOffset = input.readVInt();
BlockHeader blockHeader = reuse == null ? new BlockHeader() : reuse;
return blockHeader.reset(linesCount, baseDocsFP, basePositionsFP, basePayloadsFP, termStatesBaseOffset, middleTermOffset);
}
@Override
public long ramBytesUsed() {
return RAM_USAGE;
}
}

View File

@ -0,0 +1,260 @@
/*
* 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 org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
/**
* One term block line.
* <p>
* Contains a term and its details as a {@link BlockTermState}.
* <p>
* The line is written to the {@link UniformSplitPostingsFormat#TERMS_BLOCKS_EXTENSION block file}
* in two parts. The first part is the term followed by an offset to the details
* region. The second part is the term {@link BlockTermState}, written in
* the details region, after all the terms of the block.
* <p>
* The separate details region allows fast scan of the terms without having
* to decode the details for each term. At read time, the {@link BlockLine.Serializer#readLine}
* only reads the term and its offset to the details. The corresponding {@link BlockTermState}
* is decoded on demand in the {@link BlockReader} (see {@link BlockReader#readTermStateIfNotRead}).
*
* @lucene.experimental
*/
public class BlockLine implements Accountable {
private static final long BASE_RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(BlockLine.class);
protected TermBytes termBytes;
protected int termStateRelativeOffset;
/**
* Only used for writing.
*/
protected final BlockTermState termState;
/**
* Constructor used for writing a {@link BlockLine}.
*/
protected BlockLine(TermBytes termBytes, BlockTermState termState) {
this(termBytes, -1, termState);
}
/**
* Constructor used for reading a {@link BlockLine}.
*/
protected BlockLine(TermBytes termBytes, int termStateRelativeOffset) {
this(termBytes, termStateRelativeOffset, null);
}
private BlockLine(TermBytes termBytes, int termStateRelativeOffset, BlockTermState termState) {
reset(termBytes, termStateRelativeOffset);
this.termState = termState;
}
/**
* Resets this {@link BlockLine} to reuse it when reading.
*/
protected BlockLine reset(TermBytes termBytes, int termStateRelativeOffset) {
assert termState == null;
this.termBytes = termBytes;
this.termStateRelativeOffset = termStateRelativeOffset;
return this;
}
public TermBytes getTermBytes() {
return termBytes;
}
/**
* @return The offset of the {@link org.apache.lucene.index.TermState}
* bytes in the block, relatively to the term states base offset.
*/
public int getTermStateRelativeOffset() {
return termStateRelativeOffset;
}
@Override
public long ramBytesUsed() {
return BASE_RAM_USAGE
+ termBytes.ramBytesUsed()
+ RamUsageUtil.ramBytesUsed(termState);
}
/**
* Reads block lines with terms encoded incrementally inside a block.
* This class keeps a state of the previous term read to decode the next term.
*/
public static class Serializer implements Accountable {
private static final long BASE_RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(Serializer.class);
protected final BytesRef currentTerm;
public Serializer() {
currentTerm = new BytesRef(64);
}
/**
* Reads the current line.
*
* @param isIncrementalEncodingSeed Whether the term is a seed of the
* incremental encoding. {@code true} for the first and
* middle term, {@code false} for other terms.
* @param reuse A {@link BlockLine} instance to reuse; or null if none.
*/
public BlockLine readLine(DataInput blockInput, boolean isIncrementalEncodingSeed, BlockLine reuse) throws IOException {
int termStateRelativeOffset = blockInput.readVInt();
return reuse == null ?
new BlockLine(readIncrementallyEncodedTerm(blockInput, isIncrementalEncodingSeed, null), termStateRelativeOffset)
: reuse.reset(readIncrementallyEncodedTerm(blockInput, isIncrementalEncodingSeed, reuse.termBytes), termStateRelativeOffset);
}
/**
* Writes a line and its offset to the corresponding term state details in
* the details region.
*
* @param blockOutput The output pointing to the block terms region.
* @param termStateRelativeOffset The offset to the corresponding term
* state details in the details region.
* @param isIncrementalEncodingSeed Whether the term is a seed of
* the incremental encoding. {@code true} for the first
* and middle term, {@code false} for other terms.
*/
public static void writeLine(DataOutput blockOutput, BlockLine line, BlockLine previousLine,
int termStateRelativeOffset, boolean isIncrementalEncodingSeed) throws IOException {
blockOutput.writeVInt(termStateRelativeOffset);
writeIncrementallyEncodedTerm(line.getTermBytes(), previousLine == null ? null : previousLine.getTermBytes(),
isIncrementalEncodingSeed, blockOutput);
}
/**
* Writes the term state details of a line in the details region.
*
* @param termStatesOutput The output pointing to the details region.
*/
protected static void writeLineTermState(DataOutput termStatesOutput, BlockLine line,
FieldInfo fieldInfo, DeltaBaseTermStateSerializer encoder) throws IOException {
assert line.termState != null;
encoder.writeTermState(termStatesOutput, fieldInfo, line.termState);
}
protected static void writeIncrementallyEncodedTerm(TermBytes termBytes, TermBytes previousTermBytes,
boolean isIncrementalEncodingSeed, DataOutput blockOutput) throws IOException {
BytesRef term = termBytes.getTerm();
assert term.offset == 0;
if (isIncrementalEncodingSeed) {
// Mdp length is always 1 for an incremental encoding seed.
blockOutput.writeVLong(term.length);
blockOutput.writeBytes(term.bytes, 0, term.length);
return;
}
if (term.length == 0) {
// Empty term.
blockOutput.writeVLong(0);
return;
}
// For other lines we store:
// - Mdp length.
// - Suffix length.
// - Suffix bytes.
// Instead of writing mdp length and suffix length with 2 VInt, we can compress the storage
// by merging them in a single VLong. The idea is to leverage the information we have about
// the previous line. We know the previous line term length. And we know that
// new line mdp length <= (previous line term length + 1)
// So if numMdpBits = numBitsToEncode(previous line term length),
// then we know we can encode (new line mdp length - 1) in numMdpBits.
// Hence we encode (new line mdp length - 1) in the rightmost numMdpBits of the VLong.
// And we encode new line suffix length in the remaining left bits of the VLong.
// Most of the time both values will be encoded in a single byte.
assert previousTermBytes != null;
assert termBytes.getMdpLength() >= 1;
int numMdpBits = numBitsToEncode(previousTermBytes.getTerm().length);
assert numBitsToEncode(termBytes.getMdpLength() - 1) <= numMdpBits;
long mdpAndSuffixLengths = (((long) termBytes.getSuffixLength()) << numMdpBits) | (termBytes.getMdpLength() - 1);
assert mdpAndSuffixLengths != 0;
blockOutput.writeVLong(mdpAndSuffixLengths);
blockOutput.writeBytes(term.bytes, termBytes.getSuffixOffset(), termBytes.getSuffixLength());
}
protected TermBytes readIncrementallyEncodedTerm(DataInput blockInput, boolean isIncrementalEncodingSeed, TermBytes reuse) throws IOException {
assert currentTerm.offset == 0;
int mdpLength;
if (isIncrementalEncodingSeed) {
int length = (int) blockInput.readVLong();
mdpLength = length == 0 ? 0 : 1;
readBytes(blockInput, currentTerm, 0, length);
} else {
long mdpAndSuffixLengths = blockInput.readVLong();
if (mdpAndSuffixLengths == 0) {
// Empty term.
mdpLength = 0;
currentTerm.length = 0;
} else {
int numMdpBits = numBitsToEncode(currentTerm.length);
mdpLength = (int) (mdpAndSuffixLengths & ((1 << numMdpBits) - 1)) + 1; // Get rightmost numMdpBits.
int suffixLength = (int) (mdpAndSuffixLengths >>> numMdpBits); // Get remaining left bits.
assert mdpLength >= 1;
assert suffixLength >= 1;
readBytes(blockInput, currentTerm, mdpLength - 1, suffixLength);
}
}
return reuse == null ?
new TermBytes(mdpLength, currentTerm)
: reuse.reset(mdpLength, currentTerm);
}
/**
* Reads {@code length} bytes from the given {@link DataInput} and stores
* them at {@code offset} in {@code bytes.bytes}.
*/
protected static void readBytes(DataInput input, BytesRef bytes, int offset, int length) throws IOException {
assert bytes.offset == 0;
bytes.length = offset + length;
bytes.bytes = ArrayUtil.grow(bytes.bytes, bytes.length);
input.readBytes(bytes.bytes, offset, length);
}
@Override
public long ramBytesUsed() {
return BASE_RAM_USAGE
+ RamUsageUtil.ramBytesUsed(currentTerm);
}
/**
* Gets the number of bits required to encode the value of the provided int.
* Returns 0 for int value 0. Equivalent to (log2(i) + 1).
*/
protected static int numBitsToEncode(int i) {
return 32 - Integer.numberOfLeadingZeros(i);
}
}
}

View File

@ -0,0 +1,537 @@
/*
* 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.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.BaseTermsEnum;
import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Seeks the block corresponding to a given term, read the block bytes, and
* scans the block terms.
* <p>
* Reads fully the block in {@link #blockReadBuffer}. Then scans the block
* terms in memory. The details region is lazily decoded with {@link #termStatesReadBuffer}
* which shares the same byte array with {@link #blockReadBuffer}.
* See {@link BlockWriter} and {@link BlockLine} for the block format.
*
* @lucene.experimental
*/
public class BlockReader extends BaseTermsEnum implements Accountable {
private static final long BASE_RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(BlockReader.class)
+ RamUsageEstimator.shallowSizeOfInstance(IndexInput .class)
+RamUsageEstimator.shallowSizeOfInstance(ByteArrayDataInput .class)*2;
/**
* {@link IndexInput} on the {@link UniformSplitPostingsFormat#TERMS_BLOCKS_EXTENSION block file}.
*/
protected IndexInput blockInput;
protected final PostingsReaderBase postingsReader;
protected final FieldMetadata fieldMetadata;
protected final BlockDecoder blockDecoder;
protected BlockLine.Serializer blockLineReader;
/**
* In-memory read buffer for the current block.
*/
protected ByteArrayDataInput blockReadBuffer;
/**
* In-memory read buffer for the details region of the current block.
* It shares the same byte array as {@link #blockReadBuffer}, with a
* different position.
*/
protected ByteArrayDataInput termStatesReadBuffer;
protected DeltaBaseTermStateSerializer termStateSerializer;
/**
* {@link IndexDictionary.Browser} supplier for lazy loading.
*/
protected final Supplier<IndexDictionary.Browser> dictionaryBrowserSupplier;
/**
* Holds the {@link IndexDictionary.Browser} once loaded.
*/
protected IndexDictionary.Browser dictionaryBrowser;
/**
* Current block start file pointer, absolute in the
* {@link UniformSplitPostingsFormat#TERMS_BLOCKS_EXTENSION block file}.
*/
protected long blockStartFP;
/**
* Current block header.
*/
protected BlockHeader blockHeader;
/**
* Current block line.
*/
protected BlockLine blockLine;
/**
* Current block line details.
*/
protected BlockTermState termState;
/**
* Offset of the start of the first line of the current block (just after the header), relative to the block start.
*/
protected int blockFirstLineStart;
/**
* Current line index in the block.
*/
protected int lineIndexInBlock;
/**
* Whether the current {@link TermState} has been forced with a call to
* {@link #seekExact(BytesRef, TermState)}.
*
* @see #forcedTerm
*/
protected boolean termStateForced;
/**
* Set when {@link #seekExact(BytesRef, TermState)} is called.
* <p>
* This optimizes the use-case when the caller calls first {@link #seekExact(BytesRef, TermState)}
* and then {@link #postings(PostingsEnum, int)}. In this case we don't access
* the terms block file (we don't seek) but directly the postings file because
* we already have the {@link TermState} with the file pointers to the postings
* file.
*/
protected BytesRefBuilder forcedTerm;
// Scratch objects to avoid object reallocation.
protected BytesRef scratchBlockBytes;
protected final BlockTermState scratchTermState;
/**
* @param dictionaryBrowserSupplier to load the {@link IndexDictionary.Browser}
* lazily in {@link #seekCeil(BytesRef)}.
* @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,
PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
BlockDecoder blockDecoder) throws IOException {
this.dictionaryBrowserSupplier = dictionaryBrowserSupplier;
this.blockInput = blockInput;
this.postingsReader = postingsReader;
this.fieldMetadata = fieldMetadata;
this.blockDecoder = blockDecoder;
this.blockStartFP = -1;
scratchTermState = postingsReader.newTermState();
}
@Override
public SeekStatus seekCeil(BytesRef searchedTerm) throws IOException {
if (isCurrentTerm(searchedTerm)) {
return SeekStatus.FOUND;
}
clearTermState();
long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(searchedTerm);
blockStartFP = Math.max(blockStartFP, fieldMetadata.getFirstBlockStartFP());
if (isBeyondLastTerm(searchedTerm, blockStartFP)) {
return SeekStatus.END;
}
SeekStatus seekStatus = seekInBlock(searchedTerm, blockStartFP);
if (seekStatus != SeekStatus.END) {
return seekStatus;
}
// Go to next block.
return nextTerm() == null ? SeekStatus.END : SeekStatus.NOT_FOUND;
}
@Override
public boolean seekExact(BytesRef searchedTerm) throws IOException {
if (isCurrentTerm(searchedTerm)) {
return true;
}
clearTermState();
long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(searchedTerm);
if (blockStartFP < fieldMetadata.getFirstBlockStartFP() || isBeyondLastTerm(searchedTerm, blockStartFP)) {
return false;
}
return seekInBlock(searchedTerm, blockStartFP) == SeekStatus.FOUND;
}
protected boolean isCurrentTerm(BytesRef searchedTerm) {
// Optimization and also required to not search with the same BytesRef
// instance as the BytesRef used to read the block line (BlockLine.Serializer).
// Indeed getCurrentTerm() is allowed to return the same BytesRef instance.
return searchedTerm.equals(term());
}
/**
* Indicates whether the searched term is beyond the last term of the field.
*
* @param blockStartFP The current block start file pointer.
*/
protected boolean isBeyondLastTerm(BytesRef searchedTerm, long blockStartFP) {
return blockStartFP == fieldMetadata.getLastBlockStartFP()
&& searchedTerm.compareTo(fieldMetadata.getLastTerm()) > 0;
}
/**
* Seeks to the provided term in the block starting at the provided file pointer.
* Does not exceed the block.
*/
protected SeekStatus seekInBlock(BytesRef searchedTerm, long blockStartFP) throws IOException {
initializeHeader(searchedTerm, blockStartFP);
assert blockHeader != null;
return seekInBlock(searchedTerm);
}
/**
* Seeks to the provided term in this block.
* <p>
* Does not exceed this block; {@link org.apache.lucene.index.TermsEnum.SeekStatus#END}
* is returned if it follows the block.
* <p>
* Compares the line terms with the <code>searchedTerm</code>, taking
* advantage of the incremental encoding properties.
* <p>
* Scans linearly the terms. Updates the current block line with the current
* term.
*/
protected SeekStatus seekInBlock(BytesRef searchedTerm) throws IOException {
if (compareToMiddleAndJump(searchedTerm) == 0) {
return SeekStatus.FOUND;
}
int comparisonOffset = 0;
while (true) {
if (readLineInBlock() == null) {
// No more terms for the block.
return SeekStatus.END;
}
TermBytes lineTermBytes = blockLine.getTermBytes();
BytesRef lineTerm = lineTermBytes.getTerm();
assert lineTerm.offset == 0;
// Equivalent to comparing with BytesRef.compareTo(),
// but faster since we start comparing from min(comparisonOffset, suffixOffset).
int suffixOffset = lineTermBytes.getSuffixOffset();
int start = Math.min(comparisonOffset, suffixOffset);
int end = Math.min(searchedTerm.length, lineTerm.length);
int comparison = searchedTerm.length - lineTerm.length;
for (int i = start; i < end; i++) {
// Compare unsigned bytes.
int byteDiff = (searchedTerm.bytes[i + searchedTerm.offset] & 0xFF) - (lineTerm.bytes[i] & 0xFF);
if (byteDiff != 0) {
comparison = byteDiff;
break;
}
comparisonOffset = i + 1;
}
if (comparison == 0) {
return SeekStatus.FOUND;
} else if (comparison < 0) {
return SeekStatus.NOT_FOUND;
}
}
}
/**
* Compares the searched term to the middle term of the block.
* If the searched term is lexicographically equal or after the middle term
* then jumps to the second half of the block directly.
*
* @return The comparison between the searched term and the middle term.
*/
protected int compareToMiddleAndJump(BytesRef searchedTerm) throws IOException {
if (lineIndexInBlock != 0) {
// Don't try to compare and jump if we are not positioned at the first line.
// This can happen if we seek in the same current block and we continue
// scanning from the current line (see initializeHeader()).
return -1;
}
blockReadBuffer.skipBytes(blockHeader.getMiddleLineOffset());
lineIndexInBlock = blockHeader.getMiddleLineIndex();
readLineInBlock();
assert blockLine != null;
int compare = searchedTerm.compareTo(term());
if (compare < 0) {
blockReadBuffer.setPosition(blockFirstLineStart);
lineIndexInBlock = 0;
}
return compare;
}
/**
* Reads the current block line.
* Sets {@link #blockLine} and increments {@link #lineIndexInBlock}.
*
* @return The {@link BlockLine}; or null if there no more line in the block.
*/
protected BlockLine readLineInBlock() throws IOException {
if (lineIndexInBlock >= blockHeader.getLinesCount()) {
return blockLine = null;
}
boolean isIncrementalEncodingSeed = lineIndexInBlock == 0 || lineIndexInBlock == blockHeader.getMiddleLineIndex();
lineIndexInBlock++;
return blockLine = blockLineReader.readLine(blockReadBuffer, isIncrementalEncodingSeed, blockLine);
}
/**
* Positions this {@link BlockReader} without re-seeking the term dictionary.
* <p>
* The block containing the term is not read by this method. It will be read
* lazily only if needed, for example if {@link #next()} is called.
* Calling {@link #postings} after this method does require the block to be read.
*/
@Override
public void seekExact(BytesRef term, TermState state) {
termStateForced = true;
termState = scratchTermState;
termState.copyFrom(state);
if (forcedTerm == null) {
forcedTerm = new BytesRefBuilder();
}
forcedTerm.copyBytes(term);
}
/**
* Not supported.
*/
@Override
public void seekExact(long ord) {
throw new UnsupportedOperationException();
}
@Override
public BytesRef next() throws IOException {
if (termStateForced) {
initializeHeader(forcedTerm.get(), termState.blockFilePointer);
assert blockHeader != null;
for (int i = lineIndexInBlock; i < termState.termBlockOrd; i++) {
readLineInBlock();
}
assert blockLine.getTermBytes().getTerm().equals(forcedTerm.get());
}
clearTermState();
return nextTerm();
}
/**
* Moves to the next term line and reads it, it may be in the next block.
* The term details are not read yet. They will be read only when needed
* with {@link #readTermStateIfNotRead()}.
*
* @return The read term bytes; or null if there is no more term for the field.
*/
protected BytesRef nextTerm() throws IOException {
if (blockHeader == null) {
// Read the first block for the field.
initializeHeader(null, fieldMetadata.getFirstBlockStartFP());
assert blockHeader != null;
}
if (readLineInBlock() == null) {
// No more line in the current block.
// Read the next block starting at the current file pointer in the block file.
initializeHeader(null, blockInput.getFilePointer());
if (blockHeader == null) {
// No more block for the field.
return null;
}
readLineInBlock();
}
return term();
}
/**
* Reads and sets {@link #blockHeader}. Sets null if there is no block for the field anymore.
*
* @param searchedTerm The searched term; or null if none.
* @param targetBlockStartFP The file pointer of the block to read.
*/
protected void initializeHeader(BytesRef searchedTerm, long targetBlockStartFP) throws IOException {
initializeBlockReadLazily();
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 (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.
// If the searched term equals the current term, we also need to reset
// to scan again the current line.
blockReadBuffer.setPosition(blockFirstLineStart);
lineIndexInBlock = 0;
}
} else {
blockInput.seek(targetBlockStartFP);
blockStartFP = targetBlockStartFP;
readHeader();
blockFirstLineStart = blockReadBuffer.getPosition();
lineIndexInBlock = 0;
}
}
protected void initializeBlockReadLazily() {
if (blockStartFP == -1) {
blockInput = blockInput.clone();
blockLineReader = new BlockLine.Serializer();
blockReadBuffer = new ByteArrayDataInput();
termStatesReadBuffer = new ByteArrayDataInput();
termStateSerializer = new DeltaBaseTermStateSerializer();
scratchBlockBytes = new BytesRef();
}
}
/**
* Reads the block header.
* Sets {@link #blockHeader}.
*
* @return The block header; or null if there is no block for the field anymore.
*/
protected BlockHeader readHeader() throws IOException {
if (blockInput.getFilePointer() > fieldMetadata.getLastBlockStartFP()) {
return blockHeader = null;
}
int numBlockBytes = blockInput.readVInt();
BytesRef blockBytesRef = decodeBlockBytesIfNeeded(numBlockBytes);
blockReadBuffer.reset(blockBytesRef.bytes, blockBytesRef.offset, blockBytesRef.length);
termStatesReadBuffer.reset(blockBytesRef.bytes, blockBytesRef.offset, blockBytesRef.length);
return blockHeader = BlockHeader.read(blockReadBuffer, blockHeader);
}
protected BytesRef decodeBlockBytesIfNeeded(int numBlockBytes) throws IOException {
scratchBlockBytes.bytes = ArrayUtil.grow(scratchBlockBytes.bytes, numBlockBytes);
blockInput.readBytes(scratchBlockBytes.bytes, 0, numBlockBytes);
scratchBlockBytes.length = numBlockBytes;
if (blockDecoder == null) {
return scratchBlockBytes;
}
blockReadBuffer.reset(scratchBlockBytes.bytes, 0, numBlockBytes);
return blockDecoder.decode(blockReadBuffer, numBlockBytes);
}
/**
* Reads the {@link BlockTermState} if it is not already set.
* Sets {@link #termState}.
*/
protected BlockTermState readTermStateIfNotRead() throws IOException {
if (termState == null) {
termState = readTermState();
if (termState != null) {
termState.termBlockOrd = lineIndexInBlock;
termState.blockFilePointer = blockStartFP;
}
}
return termState;
}
/**
* Reads the {@link BlockTermState} on the current line.
* Sets {@link #termState}.
* <p>
* Overriding method may return null if there is no {@link BlockTermState}
* (in this case the extending class must support a null {@link #termState}).
*
* @return The {@link BlockTermState}; or null if none.
*/
protected BlockTermState readTermState() throws IOException {
// We reuse scratchTermState safely as the read TermState is cloned in the termState() method.
termStatesReadBuffer.setPosition(blockFirstLineStart + blockHeader.getTermStatesBaseOffset() + blockLine.getTermStateRelativeOffset());
return termState = termStateSerializer.readTermState(
blockHeader.getBaseDocsFP(), blockHeader.getBasePositionsFP(), blockHeader.getBasePayloadsFP(),
termStatesReadBuffer, fieldMetadata.getFieldInfo(), scratchTermState
);
}
@Override
public BytesRef term() {
if (termStateForced) {
return forcedTerm.get();
}
return blockLine == null ? null : blockLine.getTermBytes().getTerm();
}
@Override
public long ord() {
throw new UnsupportedOperationException();
}
@Override
public int docFreq() throws IOException {
readTermStateIfNotRead();
return termState.docFreq;
}
@Override
public long totalTermFreq() throws IOException {
readTermStateIfNotRead();
return termState.totalTermFreq;
}
@Override
public TermState termState() throws IOException {
readTermStateIfNotRead();
return termState.clone();
}
@Override
public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException {
readTermStateIfNotRead();
return postingsReader.postings(fieldMetadata.getFieldInfo(), termState, reuse, flags);
}
@Override
public ImpactsEnum impacts(int flags) throws IOException {
readTermStateIfNotRead();
return postingsReader.impacts(fieldMetadata.getFieldInfo(), termState, flags);
}
@Override
public long ramBytesUsed() {
return BASE_RAM_USAGE
+ (blockLineReader == null ? 0 : blockLineReader.ramBytesUsed())
+ (blockReadBuffer == null ? 0 : RamUsageUtil.ramBytesUsedByByteArrayOfLength(blockReadBuffer.length()))
+ (termStateSerializer == null ? 0 : termStateSerializer.ramBytesUsed())
+ (forcedTerm == null ? 0 : RamUsageUtil.ramBytesUsed(forcedTerm))
+ (blockHeader == null ? 0 : blockHeader.ramBytesUsed())
+ (blockLine == null ? 0 : blockLine.ramBytesUsed())
+ (termState == null ? 0 : RamUsageUtil.ramBytesUsed(termState));
}
protected IndexDictionary.Browser getOrCreateDictionaryBrowser() {
if (dictionaryBrowser == null) {
dictionaryBrowser = dictionaryBrowserSupplier.get();
}
return dictionaryBrowser;
}
/**
* Called by the primary {@link TermsEnum} methods to clear the previous {@link TermState}.
*/
protected void clearTermState() {
termState = null;
termStateForced = false;
}
}

View File

@ -0,0 +1,258 @@
/*
* 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.ArrayList;
import java.util.List;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
/**
* Writes blocks in the block file.
* <p>
* According the Uniform Split technique, the writing combines three steps
* per block, and it is repeated for all the field blocks:
* <ol>
* <li>Select the term with the shortest {@link TermBytes minimal distinguishing prefix}
* (MDP) in the neighborhood of the {@link #targetNumBlockLines target block size}
* (+- {@link #deltaNumLines delta size})</li>
* <li>The selected term becomes the first term of the next block, and its
* MDP is the next block key.</li>
* <li>The current block is written to the {@link UniformSplitPostingsFormat#TERMS_BLOCKS_EXTENSION block file}.
* And its block key is {@link IndexDictionary.Builder#add(BytesRef, long) added}
* to the {@link IndexDictionary index dictionary}.</li>
* </ol>
* <p>
* This stateful {@link BlockWriter} is called repeatedly to
* {@link #addLine(BytesRef, BlockTermState, IndexDictionary.Builder) add}
* all the {@link BlockLine} terms of a field. Then {@link #finishLastBlock}
* is called. And then this {@link BlockWriter} can be reused to add the terms
* of another field.
*
* @lucene.experimental
*/
public class BlockWriter {
protected final int targetNumBlockLines;
protected final int deltaNumLines;
protected final List<BlockLine> blockLines;
protected final IndexOutput blockOutput;
protected final ByteBuffersDataOutput blockLinesWriteBuffer;
protected final ByteBuffersDataOutput termStatesWriteBuffer;
protected final DeltaBaseTermStateSerializer termStateSerializer;
protected final BlockEncoder blockEncoder;
protected final ByteBuffersDataOutput blockWriteBuffer;
protected FieldMetadata fieldMetadata;
protected BytesRef lastTerm;
protected final BlockHeader reusableBlockHeader;
protected BytesRef scratchBytesRef;
protected BlockWriter(IndexOutput blockOutput, int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder) {
assert blockOutput != null;
assert targetNumBlockLines > 0;
assert deltaNumLines > 0;
assert deltaNumLines < targetNumBlockLines;
this.blockOutput = blockOutput;
this.targetNumBlockLines = targetNumBlockLines;
this.deltaNumLines = deltaNumLines;
this.blockEncoder = blockEncoder;
this.blockLines = new ArrayList<>(targetNumBlockLines);
this.termStateSerializer = new DeltaBaseTermStateSerializer();
this.blockLinesWriteBuffer = ByteBuffersDataOutput.newResettableInstance();
this.termStatesWriteBuffer = ByteBuffersDataOutput.newResettableInstance();
this.blockWriteBuffer = ByteBuffersDataOutput.newResettableInstance();
this.reusableBlockHeader = new BlockHeader();
this.scratchBytesRef = new BytesRef();
}
/**
* Adds a new {@link BlockLine} term for the current field.
* <p>
* This method determines whether the new term is part of the current block,
* or if it is part of the next block. In the latter case, a new block is started
* (including one or more of the lastly added lines), the current block is
* written to the block file, and the current block key is added to the
* {@link IndexDictionary.Builder}.
*
* @param term The block line term. The {@link BytesRef} instance is used directly,
* the caller is responsible to make a deep copy if needed. This is required
* because we keep a list of block lines until we decide to write the
* current block, and each line must have a different term instance.
* @param blockTermState Block line details.
* @param dictionaryBuilder to which the block keys are added.
*/
protected void addLine(BytesRef term, BlockTermState blockTermState, IndexDictionary.Builder dictionaryBuilder) throws IOException {
assert term != null;
assert blockTermState != null;
int mdpLength = TermBytes.computeMdpLength(lastTerm, term);
blockLines.add(new BlockLine(new TermBytes(mdpLength, term), blockTermState));
lastTerm = term;
if (blockLines.size() >= targetNumBlockLines + deltaNumLines) {
splitAndWriteBlock(dictionaryBuilder);
}
}
/**
* This method is called when there is no more term for the field. It writes
* the remaining lines added with {@link #addLine} as the last block of the
* field and resets this {@link BlockWriter} state. Then this {@link BlockWriter}
* can be used for another field.
*/
protected void finishLastBlock(IndexDictionary.Builder dictionaryBuilder) throws IOException {
while (!blockLines.isEmpty()) {
splitAndWriteBlock(dictionaryBuilder);
}
fieldMetadata = null;
lastTerm = null;
}
/**
* Defines the new block start according to {@link #targetNumBlockLines}
* and {@link #deltaNumLines}.
* The new block is started (including one or more of the lastly added lines),
* the current block is written to the block file, and the current block key
* is added to the {@link IndexDictionary.Builder}.
*/
protected void splitAndWriteBlock(IndexDictionary.Builder dictionaryBuilder) throws IOException {
assert !blockLines.isEmpty();
int numLines = blockLines.size();
if (numLines <= targetNumBlockLines - deltaNumLines) {
writeBlock(blockLines, dictionaryBuilder);
blockLines.clear();
return;
}
int deltaStart = numLines - deltaNumLines * 2;
assert deltaStart >= 1 : "blockLines size: " + numLines;
int minMdpLength = Integer.MAX_VALUE;
int minMdpEndIndex = 0;
for (int i = deltaStart; i < numLines; i++) {
TermBytes term = blockLines.get(i).getTermBytes();
int mdpLength = term.getMdpLength();
if (mdpLength <= minMdpLength) {
minMdpLength = mdpLength;
minMdpEndIndex = i;
}
}
List<BlockLine> subList = blockLines.subList(0, minMdpEndIndex);
writeBlock(subList, dictionaryBuilder);
// Clear the written block lines to keep only the lines composing the next block.
// ArrayList.subList().clear() is O(N) but still fast since we work on a small list.
// It is internally an array copy and an iteration to set array refs to null.
// For clarity we keep that until the day a CircularArrayList is available in the jdk.
subList.clear();
}
/**
* Writes a block and adds its block key to the dictionary builder.
*/
protected void writeBlock(List<BlockLine> blockLines, IndexDictionary.Builder dictionaryBuilder) throws IOException {
long blockStartFP = blockOutput.getFilePointer();
addBlockKey(blockLines, dictionaryBuilder);
int middle = blockLines.size() >> 1;
int middleOffset = -1;
BlockLine previousLine = null;
for (int i = 0, size = blockLines.size(); i < size; i++) {
boolean isIncrementalEncodingSeed = i == 0;
if (i == middle) {
middleOffset = Math.toIntExact(blockLinesWriteBuffer.size());
isIncrementalEncodingSeed = true;
}
BlockLine line = blockLines.get(i);
writeBlockLine(isIncrementalEncodingSeed, line, previousLine);
previousLine = line;
}
reusableBlockHeader.reset(blockLines.size(), termStateSerializer.getBaseDocStartFP(), termStateSerializer.getBasePosStartFP(),
termStateSerializer.getBasePayStartFP(), Math.toIntExact(blockLinesWriteBuffer.size()), middleOffset);
reusableBlockHeader.write(blockWriteBuffer);
blockLinesWriteBuffer.copyTo(blockWriteBuffer);
termStatesWriteBuffer.copyTo(blockWriteBuffer);
if (blockEncoder == null) {
blockOutput.writeVInt(Math.toIntExact(blockWriteBuffer.size()));
blockWriteBuffer.copyTo(blockOutput);
} else {
BlockEncoder.WritableBytes encodedBytes = blockEncoder.encode(blockWriteBuffer.toDataInput(), blockWriteBuffer.size());
blockOutput.writeVInt(Math.toIntExact(encodedBytes.size()));
encodedBytes.writeTo(blockOutput);
}
blockLinesWriteBuffer.reset();
termStatesWriteBuffer.reset();
blockWriteBuffer.reset();
termStateSerializer.resetBaseStartFP();
updateFieldMetadata(blockStartFP);
}
/**
* updates the field metadata after all lines were written for the block.
*/
protected void updateFieldMetadata(long blockStartFP) {
assert fieldMetadata != null;
if (fieldMetadata.getFirstBlockStartFP() == -1) {
fieldMetadata.setFirstBlockStartFP(blockStartFP);
}
fieldMetadata.setLastBlockStartFP(blockStartFP);
}
void setField(FieldMetadata fieldMetadata) {
this.fieldMetadata = fieldMetadata;
}
protected void writeBlockLine(boolean isIncrementalEncodingSeed, BlockLine line, BlockLine previousLine) throws IOException {
assert fieldMetadata != null;
BlockLine.Serializer.writeLine(blockLinesWriteBuffer, line, previousLine, Math.toIntExact(termStatesWriteBuffer.size()), isIncrementalEncodingSeed);
BlockLine.Serializer.writeLineTermState(termStatesWriteBuffer, line, fieldMetadata.getFieldInfo(), termStateSerializer);
}
/**
* Adds a new block key with its corresponding block file pointer to the
* {@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) {
assert !blockLines.isEmpty();
assert dictionaryBuilder != null;
TermBytes firstTerm = blockLines.get(0).getTermBytes();
assert firstTerm.getTerm().offset == 0;
assert scratchBytesRef.offset == 0;
scratchBytesRef.bytes = firstTerm.getTerm().bytes;
scratchBytesRef.length = firstTerm.getMdpLength();
dictionaryBuilder.add(scratchBytesRef, blockOutput.getFilePointer());
}
}

View File

@ -0,0 +1,219 @@
/*
* 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 org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.TermState;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.IntBlockTermState;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.RamUsageEstimator;
import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.BLOCK_SIZE;
/**
* {@link TermState} serializer which encodes each file pointer as a delta relative
* to a base file pointer. It differs from {@link Lucene50PostingsWriter#encodeTerm}
* which encodes each file pointer as a delta relative to the previous file pointer.
* <p>
* It automatically sets the base file pointer to the first valid file pointer for
* doc start FP, pos start FP, pay start FP. These base file pointers have to be
* {@link #resetBaseStartFP() reset} by the caller before starting to write a new block.
*
* @lucene.experimental
*/
public class DeltaBaseTermStateSerializer implements Accountable {
private static final long RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(DeltaBaseTermStateSerializer.class);
private static final long INT_BLOCK_TERM_STATE_RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(IntBlockTermState.class);
protected long baseDocStartFP;
protected long basePosStartFP;
protected long basePayStartFP;
public DeltaBaseTermStateSerializer() {
resetBaseStartFP();
}
/**
* Resets the base file pointers to 0.
* This method has to be called before starting to write a new block.
*/
public void resetBaseStartFP() {
this.baseDocStartFP = 0;
this.basePosStartFP = 0;
this.basePayStartFP = 0;
}
/**
* @return The base doc start file pointer. It is the file pointer of the first
* {@link TermState} written after {@link #resetBaseStartFP()} is called.
*/
public long getBaseDocStartFP() {
return baseDocStartFP;
}
/**
* @return The base position start file pointer. It is the file pointer of the first
* {@link TermState} written after {@link #resetBaseStartFP()} is called.
*/
public long getBasePosStartFP() {
return basePosStartFP;
}
/**
* @return The base payload start file pointer. It is the file pointer of the first
* {@link TermState} written after {@link #resetBaseStartFP()} is called.
*/
public long getBasePayStartFP() {
return basePayStartFP;
}
/**
* Writes a {@link BlockTermState} to the provided {@link DataOutput}.
* <p>
* Simpler variant of {@link Lucene50PostingsWriter#encodeTerm(long[], DataOutput, FieldInfo, BlockTermState, boolean)}.
*/
public void writeTermState(DataOutput termStatesOutput, FieldInfo fieldInfo, BlockTermState termState) throws IOException {
IndexOptions indexOptions = fieldInfo.getIndexOptions();
boolean hasFreqs = indexOptions != IndexOptions.DOCS;
boolean hasPositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
boolean hasOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
boolean hasPayloads = fieldInfo.hasPayloads();
IntBlockTermState intTermState = (IntBlockTermState) termState;
termStatesOutput.writeVInt(intTermState.docFreq);
if (hasFreqs) {
assert intTermState.totalTermFreq >= intTermState.docFreq;
termStatesOutput.writeVLong(intTermState.totalTermFreq - intTermState.docFreq);
}
if (intTermState.singletonDocID != -1) {
termStatesOutput.writeVInt(intTermState.singletonDocID);
} else {
if (baseDocStartFP == 0) {
baseDocStartFP = intTermState.docStartFP;
}
termStatesOutput.writeVLong(intTermState.docStartFP - baseDocStartFP);
}
if (hasPositions) {
if (basePosStartFP == 0) {
basePosStartFP = intTermState.posStartFP;
}
termStatesOutput.writeVLong(intTermState.posStartFP - basePosStartFP);
if (hasPayloads || hasOffsets) {
if (basePayStartFP == 0) {
basePayStartFP = intTermState.payStartFP;
}
termStatesOutput.writeVLong(intTermState.payStartFP - basePayStartFP);
}
if (intTermState.lastPosBlockOffset != -1) {
termStatesOutput.writeVLong(intTermState.lastPosBlockOffset);
}
}
if (intTermState.skipOffset != -1) {
termStatesOutput.writeVLong(intTermState.skipOffset);
}
}
/**
* Reads a {@link BlockTermState} from the provided {@link DataInput}.
* <p>
* Simpler variant of {@link Lucene50PostingsReader#decodeTerm(long[], DataInput, FieldInfo, BlockTermState, boolean)}.
*
* @param reuse {@link BlockTermState} to reuse; or null to create a new one.
*/
public BlockTermState readTermState(long baseDocStartFP, long basePosStartFP, long basePayStartFP,
DataInput termStatesInput, FieldInfo fieldInfo, BlockTermState reuse) throws IOException {
IndexOptions indexOptions = fieldInfo.getIndexOptions();
boolean hasFreqs = indexOptions != IndexOptions.DOCS;
boolean hasPositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
IntBlockTermState intTermState = reuse != null ? reset((IntBlockTermState) reuse) : new IntBlockTermState();
intTermState.docFreq = termStatesInput.readVInt();
intTermState.totalTermFreq = hasFreqs ?
intTermState.docFreq + termStatesInput.readVLong() : intTermState.docFreq;
assert intTermState.totalTermFreq >= intTermState.docFreq;
if (intTermState.docFreq == 1) {
intTermState.singletonDocID = termStatesInput.readVInt();
} else {
intTermState.docStartFP = baseDocStartFP + termStatesInput.readVLong();
}
if (hasPositions) {
intTermState.posStartFP = basePosStartFP + termStatesInput.readVLong();
boolean hasOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
if (hasOffsets || fieldInfo.hasPayloads()) {
intTermState.payStartFP = basePayStartFP + termStatesInput.readVLong();
}
if (intTermState.totalTermFreq > BLOCK_SIZE) {
intTermState.lastPosBlockOffset = termStatesInput.readVLong();
}
}
if (intTermState.docFreq > BLOCK_SIZE) {
intTermState.skipOffset = termStatesInput.readVLong();
}
return intTermState;
}
protected IntBlockTermState reset(IntBlockTermState termState) {
// OrdTermState.
termState.ord = 0;
// BlockTermState.
termState.docFreq = 0;
termState.totalTermFreq = 0;
termState.termBlockOrd = 0;
termState.blockFilePointer = 0;
// IntBlockTermState.
termState.docStartFP = 0;
termState.posStartFP = 0;
termState.payStartFP = 0;
termState.skipOffset = -1;
termState.lastPosBlockOffset = -1;
termState.singletonDocID = -1;
return termState;
}
@Override
public long ramBytesUsed() {
return RAM_USAGE;
}
/**
* @return The estimated RAM usage of the given {@link TermState}.
*/
public static long ramBytesUsed(TermState termState) {
return termState instanceof IntBlockTermState ?
INT_BLOCK_TERM_STATE_RAM_USAGE
: RamUsageEstimator.shallowSizeOf(termState);
}
}

View File

@ -0,0 +1,81 @@
/*
* 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

@ -0,0 +1,234 @@
/*
* 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 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.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IntsRefBuilder;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util;
/**
* Immutable stateless {@link FST}-based index dictionary kept in memory.
* <p>
* Use {@link IndexDictionary.Builder} to build the {@link IndexDictionary}.
* <p>
* Create a stateful {@link IndexDictionary.Browser} to seek a term in this
* {@link IndexDictionary} and get its corresponding block file pointer to
* the terms block file.
* <p>
* Its greatest advantage is to be very compact in memory thanks to both
* the compaction of the {@link FST} as a byte array, and the incremental
* encoding of the leaves block pointer values, which are long integers in
* increasing order, with {@link PositiveIntOutputs}.<br>
* With a compact dictionary in memory we can increase the number of blocks.
* This allows us to reduce the average block size, which means faster scan
* inside a block.
*
* @lucene.experimental
*/
public class FSTDictionary implements IndexDictionary {
private static final long BASE_RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(FSTDictionary.class);
protected final FST<Long> dictionary;
protected FSTDictionary(FST<Long> dictionary) {
this.dictionary = dictionary;
}
@Override
public long ramBytesUsed() {
return BASE_RAM_USAGE + dictionary.ramBytesUsed();
}
@Override
public void write(DataOutput output, BlockEncoder blockEncoder) throws IOException {
if (blockEncoder == null) {
dictionary.save(output);
} else {
ByteBuffersDataOutput bytesDataOutput = ByteBuffersDataOutput.newResettableInstance();
dictionary.save(bytesDataOutput);
BlockEncoder.WritableBytes encodedBytes = blockEncoder.encode(bytesDataOutput.toDataInput(), bytesDataOutput.size());
output.writeVLong(encodedBytes.size());
encodedBytes.writeTo(output);
}
}
/**
* Reads a {@link FSTDictionary} from the provided input.
* @param blockDecoder The {@link BlockDecoder} to use for specific decoding; or null if none.
*/
protected static FSTDictionary read(DataInput input, BlockDecoder blockDecoder) throws IOException {
DataInput fstDataInput;
if (blockDecoder == null) {
fstDataInput = input;
} else {
long numBytes = input.readVLong();
BytesRef decodedBytes = blockDecoder.decode(input, numBytes);
fstDataInput = new ByteArrayDataInput(decodedBytes.bytes, 0, decodedBytes.length);
}
PositiveIntOutputs fstOutputs = PositiveIntOutputs.getSingleton();
FST<Long> dictionary = new FST<>(fstDataInput, fstOutputs);
return new FSTDictionary(dictionary);
}
@Override
public Browser browser() {
return new Browser();
}
protected class Browser implements IndexDictionary.Browser {
protected final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<>(dictionary);
protected static final int STATE_SEEK = 0, STATE_NEXT = 1, STATE_END = 2;
protected int state = STATE_SEEK;
// Note: key and pointer are one position prior to the current fstEnum position,
// since we need need the fstEnum to be one ahead to calculate the prefix.
protected final BytesRefBuilder keyBuilder = new BytesRefBuilder();
protected int blockPrefixLen = 0;
protected long blockFilePointer = -1;
@Override
public long seekBlock(BytesRef term) {
state = STATE_SEEK;
try {
BytesRefFSTEnum.InputOutput<Long> seekFloor = fstEnum.seekFloor(term);
if (seekFloor == null) {
blockFilePointer = -1;
} else {
blockFilePointer = seekFloor.output;
}
return blockFilePointer;
} catch (IOException e) {
// Should never happen.
throw new RuntimeException(e);
}
}
@Override
public BytesRef nextKey() {
try {
if (state == STATE_END) {
// 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;
}
}
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);
}
}
@Override
public BytesRef peekKey() {
assert state != STATE_SEEK;
return (state == STATE_END) ? null : fstEnum.current().input;
}
@Override
public int getBlockPrefixLen() {
assert state != STATE_SEEK;
assert blockPrefixLen >= 0;
return blockPrefixLen;
}
@Override
public long getBlockFilePointer() {
assert state != STATE_SEEK;
assert blockFilePointer >= 0;
return blockFilePointer;
}
}
/**
* Builds an immutable {@link FSTDictionary}.
*
* @lucene.experimental
*/
public static class Builder implements IndexDictionary.Builder {
protected final org.apache.lucene.util.fst.Builder<Long> fstBuilder;
protected final IntsRefBuilder scratchInts;
public Builder() {
PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
fstBuilder = new org.apache.lucene.util.fst.Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
scratchInts = new IntsRefBuilder();
}
@Override
public void add(BytesRef blockKey, long blockFilePointer) {
try {
fstBuilder.add(Util.toIntsRef(blockKey, scratchInts), blockFilePointer);
} catch (IOException e) {
// Should never happen.
throw new RuntimeException(e);
}
}
@Override
public FSTDictionary build() {
try {
return new FSTDictionary(fstBuilder.finish());
} catch (IOException e) {
// Should never happen.
throw new RuntimeException(e);
}
}
}
}

View File

@ -0,0 +1,256 @@
/*
* 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 org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Metadata and stats for one field in the index.
* <p>
* There is only one instance of {@link FieldMetadata} per {@link FieldInfo}.
*
* @lucene.experimental
*/
public class FieldMetadata implements Accountable {
private static final long BASE_RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(FieldMetadata.class);
protected final FieldInfo fieldInfo;
protected final boolean isMutable;
protected final FixedBitSet docsSeen;
protected int sumDocFreq;
protected int numTerms;
protected int sumTotalTermFreq;
protected int docCount;
protected long dictionaryStartFP;
protected long firstBlockStartFP;
protected long lastBlockStartFP;
protected BytesRef lastTerm;
/**
* Constructs a {@link FieldMetadata} used for writing the index. This {@link FieldMetadata} is mutable.
*
* @param maxDoc The total number of documents in the segment being written.
*/
public FieldMetadata(FieldInfo fieldInfo, int maxDoc) {
this(fieldInfo, maxDoc, true);
}
public FieldMetadata(FieldInfo fieldInfo, int maxDoc, boolean isMutable) {
this(fieldInfo, maxDoc, isMutable, -1, -1, null);
}
/**
* @param isMutable Set true if this FieldMetadata is created for writing the index. Set false if it is used for reading the index.
*/
public FieldMetadata(FieldInfo fieldInfo, int maxDoc, boolean isMutable, long firstBlockStartFP, long lastBlockStartFP, BytesRef lastTerm) {
assert isMutable || maxDoc == 0;
this.fieldInfo = fieldInfo;
this.isMutable = isMutable;
// docsSeen must not be set if this FieldMetadata is immutable, that means it is used for reading the index.
this.docsSeen = isMutable ? new FixedBitSet(maxDoc) : null;
this.dictionaryStartFP = -1;
this.firstBlockStartFP = firstBlockStartFP;
this.lastBlockStartFP = lastBlockStartFP;
this.lastTerm = lastTerm;
}
/**
* Updates the field stats with the given {@link BlockTermState} for the current
* block line (for one term).
*/
public void updateStats(BlockTermState state) {
assert isMutable;
assert state.docFreq > 0;
sumDocFreq += state.docFreq;
if (state.totalTermFreq > 0) {
sumTotalTermFreq += state.totalTermFreq;
}
numTerms++;
}
/**
* Provides the {@link FixedBitSet} to keep track of the docs seen when calling
* {@link org.apache.lucene.codecs.PostingsWriterBase#writeTerm(BytesRef, TermsEnum, FixedBitSet, org.apache.lucene.codecs.NormsProducer)}.
* <p>
* The returned {@link FixedBitSet} is created once in this {@link FieldMetadata}
* constructor.
*
* @return The {@link FixedBitSet} for the docs seen, during segment writing;
* or null if this {@link FieldMetadata} is created immutable during segment reading.
*/
public FixedBitSet getDocsSeen() {
return docsSeen;
}
public FieldInfo getFieldInfo() {
return fieldInfo;
}
public int getSumDocFreq() {
return sumDocFreq;
}
public int getNumTerms() {
return numTerms;
}
public int getSumTotalTermFreq() {
return sumTotalTermFreq;
}
public int getDocCount() {
return isMutable ? docsSeen.cardinality() : docCount;
}
/**
* @return The file pointer to the start of the first block of the field.
*/
public long getFirstBlockStartFP() {
return firstBlockStartFP;
}
/**
* Sets the file pointer to the start of the first block of the field.
*/
public void setFirstBlockStartFP(long firstBlockStartFP) {
assert isMutable;
this.firstBlockStartFP = firstBlockStartFP;
}
/**
* @return The start file pointer for the last block of the field.
*/
public long getLastBlockStartFP() {
return lastBlockStartFP;
}
/**
* Sets the file pointer after the end of the last block of the field.
*/
public void setLastBlockStartFP(long lastBlockStartFP) {
assert isMutable;
this.lastBlockStartFP = lastBlockStartFP;
}
/**
* @return The file pointer to the start of the dictionary of the field.
*/
public long getDictionaryStartFP() {
return dictionaryStartFP;
}
/**
* Sets the file pointer to the start of the dictionary of the field.
*/
public void setDictionaryStartFP(long dictionaryStartFP) {
assert isMutable;
this.dictionaryStartFP = dictionaryStartFP;
}
public void setLastTerm(BytesRef lastTerm) {
assert lastTerm != null;
this.lastTerm = lastTerm;
}
public BytesRef getLastTerm() {
return lastTerm;
}
@Override
public long ramBytesUsed() {
return BASE_RAM_USAGE
+ (docsSeen == null ? 0 : docsSeen.ramBytesUsed());
}
public static FieldMetadata read(DataInput input, FieldInfos fieldInfos) throws IOException {
int fieldId = input.readVInt();
FieldMetadata stats = new FieldMetadata(fieldInfos.fieldInfo(fieldId), 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;
}
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;
}
stats.setLastTerm(lastTerm);
return stats;
}
public void write(DataOutput output) throws IOException {
assert dictionaryStartFP >= 0;
assert firstBlockStartFP >= 0;
assert lastBlockStartFP >= 0;
assert numTerms > 0 : "There should be at least one term for field " + fieldInfo.name + ": " + numTerms;
assert firstBlockStartFP <= lastBlockStartFP : "start: " + firstBlockStartFP + " end: " + lastBlockStartFP;
assert lastTerm != null : "you must set the last term";
output.writeVInt(fieldInfo.number);
output.writeVInt(numTerms);
output.writeVInt(sumDocFreq);
if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0) {
assert sumTotalTermFreq >= sumDocFreq : "sumTotalFQ: " + sumTotalTermFreq + " sumDocFQ: " + sumDocFreq;
output.writeVInt(sumTotalTermFreq - sumDocFreq);
}
output.writeVInt(getDocCount());
output.writeVLong(dictionaryStartFP);
output.writeVLong(firstBlockStartFP);
output.writeVLong(lastBlockStartFP);
if (lastTerm.length > 0) {
output.writeVInt(lastTerm.length);
output.writeBytes(lastTerm.bytes, lastTerm.offset, lastTerm.length);
} else {
output.writeVInt(0);
}
}
}

View File

@ -0,0 +1,127 @@
/*
* 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 org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
/**
* Immutable stateless index dictionary kept in RAM.
* <p>
* Implementations must be immutable.
* <p>
* Use {@link IndexDictionary.Builder} to build the {@link IndexDictionary}.
* <p>
* Create a stateful {@link IndexDictionary.Browser} to seek a term in this
* {@link IndexDictionary} and get its corresponding block file pointer to
* the terms block file.
* <p>
* There is a single implementation of this interface, {@link FSTDictionary}.
* However this interface allows you to plug easily a new kind of index dictionary
* to experiment and improve the existing one.
*
* @lucene.experimental
*/
public interface IndexDictionary extends Accountable {
/**
* Writes this dictionary to the provided output.
* @param blockEncoder The {@link BlockEncoder} for specific encoding of this index dictionary;
* or null if none.
*/
void write(DataOutput output, BlockEncoder blockEncoder) throws IOException;
/**
* Creates a new {@link IndexDictionary.Browser}.
*/
Browser browser();
/**
* Builds an immutable {@link IndexDictionary}.
*/
interface Builder {
/**
* Adds a [block key - block file pointer] entry to the dictionary.
* <p>
* The Uniform Split technique adds block keys in the dictionary. See
* {@link BlockReader} and {@link TermBytes} for more info about block
* key and minimal distinguishing prefix (MDP).
* <p>
* All block keys are added in strictly increasing order of the block file pointers,
* this allows long encoding optimizations such as with {@link org.apache.lucene.util.fst.PositiveIntOutputs}
* for {@link org.apache.lucene.util.fst.FST}.
*
* @param blockKey The block key which is the minimal distinguishing
* prefix (MDP) of the first term of a block.
* @param blockFilePointer Non-negative file pointer to the start of the
* block in the block file.
*/
void add(BytesRef blockKey, long blockFilePointer);
IndexDictionary build();
}
/**
* Stateful {@link IndexDictionary.Browser} to seek a term in this {@link IndexDictionary}
* and get its corresponding block file pointer in the block file.
*/
interface Browser {
/**
* Seeks the given term in the {@link IndexDictionary} and returns its corresponding
* block file pointer.
*
* @return The block file pointer corresponding to the term if it matches
* exactly a block key in the dictionary.
* Otherwise the floor block key, which is the greatest block key present
* in the dictionary that is alphabetically preceding the searched term.
* 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);
/**
* 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();
/**
* Returns the next key without advancing.
* Only call this after {@link #nextKey()} returns a non-null result.
*/
BytesRef peekKey();
/**
* 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();
/**
* Returns the block file pointer associated with the key returned.
* Only call this after {@link #nextKey()} returns a non-null result.
*/
long getBlockFilePointer();
}
}

View File

@ -0,0 +1,558 @@
/*
* 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.Objects;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IntsRefBuilder;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.Operations;
import org.apache.lucene.util.automaton.Transition;
/**
* The "intersect" {@link TermsEnum} response to {@link UniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
* intersecting the terms with an automaton.
*/
public class IntersectBlockReader extends BlockReader {
protected final AutomatonNextTermCalculator nextStringCalculator;
protected final ByteRunAutomaton runAutomaton;
protected final BytesRef commonSuffixRef; // maybe null
protected final BytesRef commonPrefixRef;
protected final BytesRef startTerm; // maybe null
/** Set this when our current mode is seeking to this term. Set to null after. */
protected BytesRef seekTerm;
protected int blockPrefixRunAutomatonState;
protected int blockPrefixLen;
/**
* Number of bytes accepted by the last call to {@link #runAutomatonForState}.
*/
protected int numBytesAccepted;
/**
* Whether the current term is beyond the automaton common prefix.
* If true this means the enumeration should stop immediately.
*/
protected boolean beyondCommonPrefix;
public IntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
DictionaryBrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
FieldMetadata fieldMetadata, BlockDecoder blockDecoder) throws IOException {
super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
this.nextStringCalculator = new AutomatonNextTermCalculator(compiled);
Automaton automaton = Objects.requireNonNull(compiled.automaton);
this.runAutomaton = Objects.requireNonNull(compiled.runAutomaton);
this.commonSuffixRef = compiled.commonSuffixRef; // maybe null
this.commonPrefixRef = Operations.getCommonPrefixBytesRef(automaton); // never null
this.startTerm = startTerm;
assert startTerm == null || StringHelper.startsWith(startTerm, commonPrefixRef);
// it is thus also true that startTerm >= commonPrefixRef
this.seekTerm = startTerm != null ? startTerm : commonPrefixRef;
}
@Override
public BytesRef next() throws IOException {
clearTermState();
if (blockHeader == null) { // initial state
// note: don't call super.seekCeil here; we have our own logic
// Set the browser position to the block having the seek term.
// Even if -1, it's okay since we'll soon call nextKey().
long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(seekTerm);
if (isBeyondLastTerm(seekTerm, blockStartFP)) {
return null; // EOF
}
// Starting at this block find and load the next matching block.
// note: Since seekBlock was just called, we actually consider the current block as "next".
if (nextBlockMatchingPrefix() == false) { // note: starts at seek'ed block, which may have a match
return null; // EOF
}
}
do {
// look in the rest of this block.
BytesRef term = nextTermInBlockMatching();
if (term != null) {
return term;
}
// next term dict matching prefix
} while (nextBlockMatchingPrefix());
return null; // EOF
}
/**
* Find the next block that appears to contain terms that could match the automata.
* The prefix is the primary clue. Returns true if at one, or false for no more (EOF).
*/
protected boolean nextBlockMatchingPrefix() throws IOException {
if (beyondCommonPrefix) {
return false; // EOF
}
IndexDictionary.Browser browser = getOrCreateDictionaryBrowser();
do {
// Get next block key (becomes in effect the current blockKey)
BytesRef blockKey = browser.nextKey();
if (blockKey == null) {
return false; // EOF
}
blockPrefixLen = browser.getBlockPrefixLen();
blockPrefixRunAutomatonState = runAutomatonForState(blockKey.bytes, blockKey.offset, blockPrefixLen, 0);
// We may have passed commonPrefix (a short-circuit optimization).
if (isBeyondCommonPrefix(blockKey)) {
return false; // EOF
}
if (blockPrefixRunAutomatonState >= 0) {
break; // a match
}
//
// This block doesn't match.
//
seekTerm = null; // we're moving on to another block, and seekTerm is before it.
// Should we simply get the next key (linear mode) or try to seek?
if (nextStringCalculator.isLinearState(blockKey)) {
continue;
}
// Maybe the next block's key matches? We have to check this before calling nextStringCalculator.
BytesRef peekKey = browser.peekKey();
if (peekKey == null) {
return false; // EOF
}
if (runAutomatonForState(peekKey.bytes, peekKey.offset, peekKey.length, 0) >= 0) {
continue; // yay; it matched. Continue to actually advance to it. This is rare?
}
// Seek to a block by calculating the next term to match the automata *following* peekKey.
this.seekTerm = nextStringCalculator.nextSeekTerm(browser.peekKey());
if (seekTerm == null) {
return false; // EOF
}
browser.seekBlock(seekTerm);
//continue
} while (true); // while not a match
// A match!
//NOTE: we could determine if this automata has a prefix for this specific block (longer than the commonPrefix).
// If we see it, we could set it as the seekTerm and we could also exit the block early if we get past this prefix
// and runAutomatonFromPrefix would start from this prefix. Smiley tried but benchmarks were not favorable to it.
initializeHeader(null, browser.getBlockFilePointer());
return true;
}
/**
* Find the next block line that matches, or null when at end of block.
*/
protected BytesRef nextTermInBlockMatching() throws IOException {
do {
// if seekTerm is set, then we seek into this block instead of starting with the first blindly.
if (seekTerm != null) {
assert blockLine == null;
boolean moveBeyondIfFound = seekTerm == startTerm; // for startTerm, we want to get the following term
SeekStatus seekStatus = seekInBlock(seekTerm);
seekTerm = null;// reset.
if (seekStatus == SeekStatus.END) {
return null;
} else if (seekStatus == SeekStatus.FOUND && moveBeyondIfFound) {
if (readLineInBlock() == null) {
return null;
}
}
assert blockLine != null;
} else {
if (readLineInBlock() == null) {
return null;
}
}
TermBytes lineTermBytes = blockLine.getTermBytes();
BytesRef lineTerm = lineTermBytes.getTerm();
if (commonSuffixRef == null || StringHelper.endsWith(lineTerm, commonSuffixRef)) {
if (runAutomatonFromPrefix(lineTerm)) {
return lineTerm;
} else if (beyondCommonPrefix) {
return null;
}
}
} while (true);
}
protected boolean runAutomatonFromPrefix(BytesRef term) {
int state = runAutomatonForState(term.bytes, term.offset + blockPrefixLen, term.length - blockPrefixLen, blockPrefixRunAutomatonState);
if (state >= 0 && runAutomaton.isAccept(state)) {
return true;
}
if (isBeyondCommonPrefix(term)) {
// If the automaton rejects early the term, before the common prefix length,
// and if the term rejected byte is lexicographically after the same byte in the common prefix,
// then it means the current term is beyond the common prefix.
// Exit immediately the enumeration.
beyondCommonPrefix = true;
}
return false;
}
/**
* Run the automaton and return the final state (not necessary accepted). -1 signifies no state / no match.
* Sets {@link #numBytesAccepted} with the offset of the first byte rejected by the automaton;
* or (offset + length) if no byte is rejected.
*/
protected int runAutomatonForState(byte[] s, int offset, int length, int initialState) {
//see ByteRunAutomaton.run(); similar
int state = initialState;
int index = 0;
while (index < length) {
state = runAutomaton.step(state, s[index + offset] & 0xFF);
if (state == -1) {
break;
}
index++;
}
numBytesAccepted = index;
return state;
}
/**
* Determines if the provided {@link BytesRef} is beyond the automaton common prefix.
* This method must be called after a call to {@link #runAutomatonForState} because
* it uses {@link #numBytesAccepted} value.
*/
protected boolean isBeyondCommonPrefix(BytesRef bytesRef) {
// If the automaton rejects early the bytes, before the common prefix length,
// and if the rejected byte is lexicographically after the same byte in the common prefix,
// then it means the bytes are beyond the common prefix.
return numBytesAccepted < commonPrefixRef.length
&& numBytesAccepted < bytesRef.length
&& (bytesRef.bytes[numBytesAccepted + bytesRef.offset] & 0xFF) > (commonPrefixRef.bytes[numBytesAccepted + commonPrefixRef.offset] & 0xFF);
}
@Override
public boolean seekExact(BytesRef text) {
throw new UnsupportedOperationException();
}
@Override
public void seekExact(long ord) {
throw new UnsupportedOperationException();
}
@Override
public SeekStatus seekCeil(BytesRef text) {
throw new UnsupportedOperationException();
}
@Override
public void seekExact(BytesRef term, TermState state) {
throw new UnsupportedOperationException();
}
/**
* This is a copy of AutomatonTermsEnum. Since it's an inner class, the outer class can
* call methods that ATE does not expose. It'd be nice if ATE's logic could be more extensible.
*/
protected static class AutomatonNextTermCalculator {
// a tableized array-based form of the DFA
protected final ByteRunAutomaton runAutomaton;
// common suffix of the automaton
protected final BytesRef commonSuffixRef;
// true if the automaton accepts a finite language
protected final boolean finite;
// array of sorted transitions for each state, indexed by state number
protected final Automaton automaton;
// for path tracking: each long records gen when we last
// visited the state; we use gens to avoid having to clear
protected final long[] visited;
protected long curGen;
// the reference used for seeking forwards through the term dictionary
protected final BytesRefBuilder seekBytesRef = new BytesRefBuilder();
// true if we are enumerating an infinite portion of the DFA.
// in this case it is faster to drive the query based on the terms dictionary.
// when this is true, linearUpperBound indicate the end of range
// of terms where we should simply do sequential reads instead.
protected boolean linear = false;
protected final BytesRef linearUpperBound = new BytesRef(10);
protected Transition transition = new Transition();
protected final IntsRefBuilder savedStates = new IntsRefBuilder();
protected AutomatonNextTermCalculator(CompiledAutomaton compiled) {
if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
}
this.finite = compiled.finite;
this.runAutomaton = compiled.runAutomaton;
assert this.runAutomaton != null;
this.commonSuffixRef = compiled.commonSuffixRef;
this.automaton = compiled.automaton;
// used for path tracking, where each bit is a numbered state.
visited = new long[runAutomaton.getSize()];
}
/** True if the current state of the automata is best iterated linearly (without seeking). */
protected boolean isLinearState(BytesRef term) {
return linear && term.compareTo(linearUpperBound) < 0;
}
/** @see org.apache.lucene.index.FilteredTermsEnum#nextSeekTerm(BytesRef) */
protected BytesRef nextSeekTerm(final BytesRef term) throws IOException {
//System.out.println("ATE.nextSeekTerm term=" + term);
if (term == null) {
assert seekBytesRef.length() == 0;
// return the empty term, as it's valid
if (runAutomaton.isAccept(0)) {
return seekBytesRef.get();
}
} else {
seekBytesRef.copyBytes(term);
}
// seek to the next possible string;
if (nextString()) {
return seekBytesRef.get(); // reposition
} else {
return null; // no more possible strings can match
}
}
/**
* Sets the enum to operate in linear fashion, as we have found
* a looping transition at position: we set an upper bound and
* act like a TermRangeQuery for this portion of the term space.
*/
protected void setLinear(int position) {
assert linear == false;
int state = 0;
assert state == 0;
int maxInterval = 0xff;
//System.out.println("setLinear pos=" + position + " seekbytesRef=" + seekBytesRef);
for (int i = 0; i < position; i++) {
state = runAutomaton.step(state, seekBytesRef.byteAt(i) & 0xff);
assert state >= 0: "state=" + state;
}
final int numTransitions = automaton.getNumTransitions(state);
automaton.initTransition(state, transition);
for (int i = 0; i < numTransitions; i++) {
automaton.getNextTransition(transition);
if (transition.min <= (seekBytesRef.byteAt(position) & 0xff) &&
(seekBytesRef.byteAt(position) & 0xff) <= transition.max) {
maxInterval = transition.max;
break;
}
}
// 0xff terms don't get the optimization... not worth the trouble.
if (maxInterval != 0xff)
maxInterval++;
int length = position + 1; /* position + maxTransition */
if (linearUpperBound.bytes.length < length)
linearUpperBound.bytes = new byte[length];
System.arraycopy(seekBytesRef.bytes(), 0, linearUpperBound.bytes, 0, position);
linearUpperBound.bytes[position] = (byte) maxInterval;
linearUpperBound.length = length;
linear = true;
}
/**
* Increments the byte buffer to the next String in binary order after s that will not put
* the machine into a reject state. If such a string does not exist, returns
* false.
*
* The correctness of this method depends upon the automaton being deterministic,
* and having no transitions to dead states.
*
* @return true if more possible solutions exist for the DFA
*/
protected boolean nextString() {
int state;
int pos = 0;
savedStates.grow(seekBytesRef.length()+1);
savedStates.setIntAt(0, 0);
while (true) {
curGen++;
linear = false;
// walk the automaton until a character is rejected.
for (state = savedStates.intAt(pos); pos < seekBytesRef.length(); pos++) {
visited[state] = curGen;
int nextState = runAutomaton.step(state, seekBytesRef.byteAt(pos) & 0xff);
if (nextState == -1)
break;
savedStates.setIntAt(pos+1, nextState);
// we found a loop, record it for faster enumeration
if (!finite && !linear && visited[nextState] == curGen) {
setLinear(pos);
}
state = nextState;
}
// take the useful portion, and the last non-reject state, and attempt to
// append characters that will match.
if (nextString(state, pos)) {
return true;
} else { /* no more solutions exist from this useful portion, backtrack */
if ((pos = backtrack(pos)) < 0) /* no more solutions at all */
return false;
final int newState = runAutomaton.step(savedStates.intAt(pos), seekBytesRef.byteAt(pos) & 0xff);
if (newState >= 0 && runAutomaton.isAccept(newState))
/* String is good to go as-is */
return true;
/* else advance further */
// TODO: paranoia? if we backtrack thru an infinite DFA, the loop detection is important!
// for now, restart from scratch for all infinite DFAs
if (!finite) pos = 0;
}
}
}
/**
* Returns the next String in lexicographic order that will not put
* the machine into a reject state.
*
* This method traverses the DFA from the given position in the String,
* starting at the given state.
*
* If this cannot satisfy the machine, returns false. This method will
* walk the minimal path, in lexicographic order, as long as possible.
*
* If this method returns false, then there might still be more solutions,
* it is necessary to backtrack to find out.
*
* @param state current non-reject state
* @param position useful portion of the string
* @return true if more possible solutions exist for the DFA from this
* position
*/
protected boolean nextString(int state, int position) {
/*
* the next lexicographic character must be greater than the existing
* character, if it exists.
*/
int c = 0;
if (position < seekBytesRef.length()) {
c = seekBytesRef.byteAt(position) & 0xff;
// if the next byte is 0xff and is not part of the useful portion,
// then by definition it puts us in a reject state, and therefore this
// path is dead. there cannot be any higher transitions. backtrack.
if (c++ == 0xff)
return false;
}
seekBytesRef.setLength(position);
visited[state] = curGen;
final int numTransitions = automaton.getNumTransitions(state);
automaton.initTransition(state, transition);
// find the minimal path (lexicographic order) that is >= c
for (int i = 0; i < numTransitions; i++) {
automaton.getNextTransition(transition);
if (transition.max >= c) {
int nextChar = Math.max(c, transition.min);
// append either the next sequential char, or the minimum transition
seekBytesRef.grow(seekBytesRef.length() + 1);
seekBytesRef.append((byte) nextChar);
state = transition.dest;
/*
* as long as is possible, continue down the minimal path in
* lexicographic order. if a loop or accept state is encountered, stop.
*/
while (visited[state] != curGen && !runAutomaton.isAccept(state)) {
visited[state] = curGen;
/*
* Note: we work with a DFA with no transitions to dead states.
* so the below is ok, if it is not an accept state,
* then there MUST be at least one transition.
*/
automaton.initTransition(state, transition);
automaton.getNextTransition(transition);
state = transition.dest;
// append the minimum transition
seekBytesRef.grow(seekBytesRef.length() + 1);
seekBytesRef.append((byte) transition.min);
// we found a loop, record it for faster enumeration
if (!finite && !linear && visited[state] == curGen) {
setLinear(seekBytesRef.length()-1);
}
}
return true;
}
}
return false;
}
/**
* Attempts to backtrack thru the string after encountering a dead end
* at some given position. Returns false if no more possible strings
* can match.
*
* @param position current position in the input String
* @return {@code position >= 0} if more possible solutions exist for the DFA
*/
protected int backtrack(int position) {
while (position-- > 0) {
int nextChar = seekBytesRef.byteAt(position) & 0xff;
// if a character is 0xff it's a dead-end too,
// because there is no higher character in binary sort order.
if (nextChar++ != 0xff) {
seekBytesRef.setByteAt(position, (byte) nextChar);
seekBytesRef.setLength(position+1);
return position;
}
}
return -1; /* all solutions exhausted */
}
}
}

View File

@ -0,0 +1,87 @@
/*
* 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.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.index.TermState;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.RamUsageEstimator;
import static org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_ARRAY_HEADER;
import static org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF;
import static org.apache.lucene.util.RamUsageEstimator.alignObjectSize;
import static org.apache.lucene.util.RamUsageEstimator.shallowSizeOfInstance;
/**
* Utility methods to estimate the RAM usage of objects.
* It relies on {@link RamUsageEstimator}.
*
* @lucene.experimental
*/
public class RamUsageUtil {
private static final long BYTES_REF_BASE_RAM_USAGE = shallowSizeOfInstance(BytesRef.class);
private static final long BYTES_REF_BUILDER_BASE_RAM_USAGE = shallowSizeOfInstance(BytesRefBuilder.class);
private static final long HASH_MAP_BASE_RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(HashMap.class);
private static final long HASH_MAP_ENTRY_BASE_RAM_USAGE;
private static final long UNMODIFIABLE_ARRAY_LIST_BASE_RAM_USAGE;
static {
Map<Object, Object> map = new HashMap<>();
map.put(map, map);
HASH_MAP_ENTRY_BASE_RAM_USAGE = RamUsageEstimator.shallowSizeOf(map.entrySet().iterator().next());
UNMODIFIABLE_ARRAY_LIST_BASE_RAM_USAGE = RamUsageEstimator.shallowSizeOf(Collections.unmodifiableList(new ArrayList<>()))
+ RamUsageEstimator.shallowSizeOfInstance(ArrayList.class);
}
public static long ramBytesUsed(BytesRef bytesRef) {
return BYTES_REF_BASE_RAM_USAGE + RamUsageEstimator.sizeOf(bytesRef.bytes);
}
public static long ramBytesUsed(BytesRefBuilder bytesRefBuilder) {
return BYTES_REF_BUILDER_BASE_RAM_USAGE + ramBytesUsed(bytesRefBuilder.get());
}
public static long ramBytesUsed(TermState termState) {
return DeltaBaseTermStateSerializer.ramBytesUsed(termState);
}
public static long ramBytesUsedByByteArrayOfLength(int length) {
return alignObjectSize((long) NUM_BYTES_ARRAY_HEADER + (long) Byte.BYTES * length);
}
public static long ramBytesUsedByHashMapOfSize(int size) {
return HASH_MAP_BASE_RAM_USAGE
+ RamUsageUtil.ramBytesUsedByObjectArrayOfLength((int) (size / 0.6))
+ HASH_MAP_ENTRY_BASE_RAM_USAGE * size;
}
public static long ramBytesUsedByUnmodifiableArrayListOfSize(int size) {
return UNMODIFIABLE_ARRAY_LIST_BASE_RAM_USAGE
+ ramBytesUsedByObjectArrayOfLength(size);
}
public static long ramBytesUsedByObjectArrayOfLength(int length) {
return alignObjectSize((long) NUM_BYTES_ARRAY_HEADER + (long) NUM_BYTES_OBJECT_REF * length);
}
}

View File

@ -0,0 +1,123 @@
/*
* 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 org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
/**
* Term of a block line.
* <p>
* Contains the term bytes and the minimal distinguishing prefix (MDP) length
* of this term.
* <p>
* The MDP is the minimal prefix that distinguishes a term from its immediate
* previous term (terms are alphabetically sorted).
* <p>
* The incremental encoding suffix is the suffix starting at the last byte of
* the MDP (inclusive).
* <p>
* Example:
* For the block
* <pre>
* client
* color
* company
* companies
* </pre>
* "color" - MDP is "co" - incremental encoding suffix is "olor".
* <br>
* "company" - MDP is "com" - incremental encoding suffix is "mpany".
* <br>
* "companies" - MDP is "compani" - incremental encoding suffix is "ies".
*
* @lucene.experimental
*/
public class TermBytes implements Accountable {
private static final long BASE_RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(TermBytes.class);
protected int mdpLength;
protected BytesRef term;
public TermBytes(int mdpLength, BytesRef term) {
reset(mdpLength, term);
}
public TermBytes reset(int mdpLength, BytesRef term) {
assert term.length > 0 && mdpLength > 0 || term.length == 0 && mdpLength == 0 : "Inconsistent mdpLength=" + mdpLength + ", term.length=" + term.length;
assert term.length == 0 || mdpLength <= term.length : "Too large mdpLength=" + mdpLength + ", term.length=" + term.length;
assert term.offset == 0;
this.mdpLength = mdpLength;
this.term = term;
return this;
}
/**
* @return This term MDP length.
* @see TermBytes
*/
public int getMdpLength() {
return mdpLength;
}
/**
* @return This term bytes.
*/
public BytesRef getTerm() {
return term;
}
/**
* @return The offset of this term incremental encoding suffix.
* @see TermBytes
*/
public int getSuffixOffset() {
return Math.max(mdpLength - 1, 0);
}
/**
* @return The length of this term incremental encoding suffix.
* @see TermBytes
*/
public int getSuffixLength() {
return term.length - getSuffixOffset();
}
/**
* Computes the length of the minimal distinguishing prefix (MDP) between
* a current term and its previous term (terms are alphabetically sorted).
* <p>
* Example: If previous="car" and current="cartridge", then MDP length is
* 4. It is the length of the minimal prefix distinguishing "cartridge" from
* "car", that is, the length of "cart".
*
* @see TermBytes
*/
public static int computeMdpLength(BytesRef previousTerm, BytesRef currentTerm) {
int mdpLength = previousTerm == null ? 1 : StringHelper.sortKeyLength(previousTerm, currentTerm);
return Math.min(mdpLength, currentTerm.length);
}
@Override
public long ramBytesUsed() {
return BASE_RAM_USAGE + RamUsageUtil.ramBytesUsed(term);
}
}

View File

@ -0,0 +1,136 @@
/*
* 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 org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
/**
* {@link PostingsFormat} based on the Uniform Split technique.
*
* @see UniformSplitTermsWriter
* @lucene.experimental
*/
public class UniformSplitPostingsFormat extends PostingsFormat {
/**
* Extension of the file containing the terms dictionary (the FST "trie").
*/
public static final String TERMS_DICTIONARY_EXTENSION = "ustd";
/**
* Extension of the file containing the terms blocks for each field and the fields metadata.
*/
public static final String TERMS_BLOCKS_EXTENSION = "ustb";
public static final int VERSION_CURRENT = 0;
public static final String NAME = "UniformSplit";
protected final int targetNumBlockLines;
protected final int deltaNumLines;
protected final BlockEncoder blockEncoder;
protected final BlockDecoder blockDecoder;
public UniformSplitPostingsFormat() {
this(UniformSplitTermsWriter.DEFAULT_TARGET_NUM_BLOCK_LINES, UniformSplitTermsWriter.DEFAULT_DELTA_NUM_LINES, null, null);
}
/**
* @param targetNumBlockLines Target number of lines per block.
* Must be strictly greater than 0.
* The parameters can be pre-validated with {@link UniformSplitTermsWriter#validateSettings(int, int)}.
* There is one term per block line, with its corresponding details ({@link org.apache.lucene.index.TermState}).
* @param deltaNumLines Maximum allowed delta variation of the number of lines per block.
* Must be greater than or equal to 0 and strictly less than {@code targetNumBlockLines}.
* The block size will be {@code targetNumBlockLines}+-{@code deltaNumLines}.
* The block size must always be less than or equal to {@link UniformSplitTermsWriter#MAX_NUM_BLOCK_LINES}.
* @param blockEncoder Optional block encoder, may be null if none.
* It can be used for compression or encryption.
* @param blockDecoder Optional block decoder, may be null if none.
* It can be used for compression or encryption.
*/
public UniformSplitPostingsFormat(int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder, BlockDecoder blockDecoder) {
this(NAME, targetNumBlockLines, deltaNumLines, blockEncoder, blockDecoder);
}
protected UniformSplitPostingsFormat(String name, int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder, BlockDecoder blockDecoder) {
super(name);
UniformSplitTermsWriter.validateSettings(targetNumBlockLines, deltaNumLines);
validateBlockEncoder(blockEncoder, blockDecoder);
this.targetNumBlockLines = targetNumBlockLines;
this.deltaNumLines = deltaNumLines;
this.blockEncoder = blockEncoder;
this.blockDecoder = blockDecoder;
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
boolean success = false;
try {
FieldsConsumer termsWriter = createUniformSplitTermsWriter(postingsWriter, state, targetNumBlockLines, deltaNumLines, blockEncoder);
success = true;
return termsWriter;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsWriter);
}
}
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene50PostingsReader(state);
boolean success = false;
try {
FieldsProducer termsReader = createUniformSplitTermsReader(postingsReader, state, blockDecoder);
success = true;
return termsReader;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsReader);
}
}
}
protected FieldsConsumer createUniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state,
int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder) throws IOException {
return new UniformSplitTermsWriter(postingsWriter, state, targetNumBlockLines, deltaNumLines, blockEncoder);
}
protected FieldsProducer createUniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state,
BlockDecoder blockDecoder) throws IOException {
return new UniformSplitTermsReader(postingsReader, state, blockDecoder);
}
private static void validateBlockEncoder(BlockEncoder blockEncoder, BlockDecoder blockDecoder) {
if (blockEncoder != null && blockDecoder == null || blockEncoder == null && blockDecoder != null) {
throw new IllegalArgumentException("Invalid blockEncoder=" + blockEncoder + " and blockDecoder=" + blockDecoder + ", both must be null or both must be non-null");
}
}
}

View File

@ -0,0 +1,153 @@
/*
* 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 org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.automaton.CompiledAutomaton;
/**
* {@link Terms} based on the Uniform Split technique.
* <p>
* The {@link IndexDictionary index dictionary} is lazy loaded only when
* {@link TermsEnum#seekCeil} or {@link TermsEnum#seekExact} are called
* (it is not loaded for a direct terms enumeration).
*
* @see UniformSplitTermsWriter
* @lucene.experimental
*/
public class UniformSplitTerms extends Terms implements Accountable {
private static final long BASE_RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(UniformSplitTerms.class);
protected final IndexInput blockInput;
protected final FieldMetadata fieldMetadata;
protected final PostingsReaderBase postingsReader;
protected final BlockDecoder blockDecoder;
protected final DictionaryBrowserSupplier dictionaryBrowserSupplier;
/**
* @param blockDecoder Optional block decoder, may be null if none. It can be used for decompression or decryption.
*/
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));
}
/**
* @param blockDecoder Optional block decoder, may be null if none. It can be used for decompression or decryption.
*/
protected UniformSplitTerms(IndexInput blockInput, FieldMetadata fieldMetadata,
PostingsReaderBase postingsReader, BlockDecoder blockDecoder,
DictionaryBrowserSupplier dictionaryBrowserSupplier) {
assert fieldMetadata != null;
assert fieldMetadata.getFieldInfo() != null;
assert fieldMetadata.getLastTerm() != null;
assert dictionaryBrowserSupplier != null;
this.blockInput = blockInput;
this.fieldMetadata = fieldMetadata;
this.postingsReader = postingsReader;
this.blockDecoder = blockDecoder;
this.dictionaryBrowserSupplier = dictionaryBrowserSupplier;
}
@Override
public TermsEnum iterator() throws IOException {
return new BlockReader(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
}
@Override
public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
checkIntersectAutomatonType(compiled);
return new IntersectBlockReader(compiled, startTerm, dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
}
protected void checkIntersectAutomatonType(CompiledAutomaton automaton) {
// This check is consistent with other impls and precondition stated in javadoc.
if (automaton.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
}
}
@Override
public BytesRef getMax() {
return fieldMetadata.getLastTerm();
}
@Override
public long size() {
return fieldMetadata.getNumTerms();
}
@Override
public long getSumTotalTermFreq() {
return fieldMetadata.getSumTotalTermFreq();
}
@Override
public long getSumDocFreq() {
return fieldMetadata.getSumDocFreq();
}
@Override
public int getDocCount() {
return fieldMetadata.getDocCount();
}
@Override
public boolean hasFreqs() {
return fieldMetadata.getFieldInfo().getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
}
@Override
public boolean hasOffsets() {
return fieldMetadata.getFieldInfo().getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
}
@Override
public boolean hasPositions() {
return fieldMetadata.getFieldInfo().getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
}
@Override
public boolean hasPayloads() {
return fieldMetadata.getFieldInfo().hasPayloads();
}
@Override
public long ramBytesUsed() {
return ramBytesUsedWithoutDictionary() + getDictionaryRamBytesUsed();
}
public long ramBytesUsedWithoutDictionary() {
return BASE_RAM_USAGE + fieldMetadata.ramBytesUsed();
}
public long getDictionaryRamBytesUsed() {
return dictionaryBrowserSupplier.ramBytesUsed();
}
}

View File

@ -0,0 +1,204 @@
/*
* 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.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
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.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import static org.apache.lucene.codecs.uniformsplit.UniformSplitPostingsFormat.NAME;
import static org.apache.lucene.codecs.uniformsplit.UniformSplitPostingsFormat.TERMS_BLOCKS_EXTENSION;
import static org.apache.lucene.codecs.uniformsplit.UniformSplitPostingsFormat.TERMS_DICTIONARY_EXTENSION;
import static org.apache.lucene.codecs.uniformsplit.UniformSplitPostingsFormat.VERSION_CURRENT;
/**
* A block-based terms index and dictionary based on the Uniform Split technique.
*
* @see UniformSplitTermsWriter
* @lucene.experimental
*/
public class UniformSplitTermsReader extends FieldsProducer {
protected static final int VERSION_START = 0;
private static final long BASE_RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(UniformSplitTermsReader.class)
+ RamUsageEstimator.shallowSizeOfInstance(IndexInput.class) * 2;
protected final PostingsReaderBase postingsReader;
protected final IndexInput blockInput;
protected final IndexInput dictionaryInput;
protected final Map<String, UniformSplitTerms> fieldToTermsMap;
// Keeps the order of the field names; much more efficient than having a TreeMap for the fieldToTermsMap.
protected final Collection<String> sortedFieldNames;
/**
* @param blockDecoder Optional block decoder, may be null if none.
* It can be used for decompression or decryption.
*/
public UniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state, BlockDecoder blockDecoder) throws IOException {
this(postingsReader, state, blockDecoder, NAME, VERSION_START, VERSION_CURRENT,
TERMS_BLOCKS_EXTENSION, TERMS_DICTIONARY_EXTENSION);
}
/**
* @param blockDecoder Optional block decoder, may be null if none.
* It can be used for decompression or decryption.
*/
protected UniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state, BlockDecoder blockDecoder,
String codecName, int versionStart, int versionCurrent, String termsBlocksExtension, String dictionaryExtension) throws IOException {
IndexInput dictionaryInput = null;
IndexInput blockInput = null;
boolean success = false;
try {
this.postingsReader = postingsReader;
String segmentName = state.segmentInfo.name;
String termsName = IndexFileNames.segmentFileName(segmentName, state.segmentSuffix, termsBlocksExtension);
blockInput = state.directory.openInput(termsName, state.context);
int version = CodecUtil.checkIndexHeader(blockInput, codecName, versionStart,
versionCurrent, state.segmentInfo.getId(), state.segmentSuffix);
String indexName = IndexFileNames.segmentFileName(segmentName, state.segmentSuffix, dictionaryExtension);
dictionaryInput = state.directory.openInput(indexName, state.context);
CodecUtil.checkIndexHeader(dictionaryInput, codecName, version, version, state.segmentInfo.getId(), state.segmentSuffix);
CodecUtil.checksumEntireFile(dictionaryInput);
postingsReader.init(blockInput, state);
CodecUtil.retrieveChecksum(blockInput);
seekFieldsMetadata(blockInput);
Collection<FieldMetadata> fieldMetadataCollection = parseFieldsMetadata(blockInput, state.fieldInfos);
fieldToTermsMap = new HashMap<>();
this.blockInput = blockInput;
this.dictionaryInput = dictionaryInput;
fillFieldMap(postingsReader, blockDecoder, dictionaryInput, blockInput, fieldMetadataCollection, state.fieldInfos);
List<String> fieldNames = new ArrayList<>(fieldToTermsMap.keySet());
Collections.sort(fieldNames);
sortedFieldNames = Collections.unmodifiableList(fieldNames);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(blockInput, dictionaryInput);
}
}
}
protected void fillFieldMap(PostingsReaderBase postingsReader, BlockDecoder blockDecoder,
IndexInput dictionaryInput, IndexInput blockInput,
Collection<FieldMetadata> fieldMetadataCollection, FieldInfos fieldInfos) throws IOException {
for (FieldMetadata fieldMetadata : fieldMetadataCollection) {
fieldToTermsMap.put(fieldMetadata.getFieldInfo().name,
new UniformSplitTerms(dictionaryInput, blockInput, fieldMetadata, postingsReader, blockDecoder));
}
}
/**
* @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));
}
return fieldMetadataCollection;
}
@Override
public void close() throws IOException {
try {
IOUtils.close(blockInput, dictionaryInput, postingsReader);
} finally {
// Clear so refs to terms index is GCable even if app hangs onto us.
fieldToTermsMap.clear();
}
}
@Override
public void checkIntegrity() throws IOException {
// term dictionary
CodecUtil.checksumEntireFile(blockInput);
// postings
postingsReader.checkIntegrity();
}
@Override
public Iterator<String> iterator() {
return sortedFieldNames.iterator();
}
@Override
public Terms terms(String field) {
return fieldToTermsMap.get(field);
}
@Override
public int size() {
return fieldToTermsMap.size();
}
@Override
public long ramBytesUsed() {
long ramUsage = BASE_RAM_USAGE;
ramUsage += postingsReader.ramBytesUsed();
ramUsage += RamUsageUtil.ramBytesUsedByHashMapOfSize(fieldToTermsMap.size());
ramUsage += getTermsRamBytesUsed();
ramUsage += RamUsageUtil.ramBytesUsedByUnmodifiableArrayListOfSize(sortedFieldNames.size());
return ramUsage;
}
protected long getTermsRamBytesUsed() {
long ramUsage = 0L;
for (UniformSplitTerms terms : fieldToTermsMap.values()) {
ramUsage += terms.ramBytesUsed();
}
return ramUsage;
}
/**
* Positions the given {@link IndexInput} at the beginning of the fields metadata.
*/
protected static void seekFieldsMetadata(IndexInput indexInput) throws IOException {
indexInput.seek(indexInput.length() - CodecUtil.footerLength() - 8);
indexInput.seek(indexInput.readLong());
}
}

View File

@ -0,0 +1,315 @@
/*
* 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 org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import static org.apache.lucene.codecs.uniformsplit.UniformSplitPostingsFormat.NAME;
import static org.apache.lucene.codecs.uniformsplit.UniformSplitPostingsFormat.TERMS_BLOCKS_EXTENSION;
import static org.apache.lucene.codecs.uniformsplit.UniformSplitPostingsFormat.TERMS_DICTIONARY_EXTENSION;
import static org.apache.lucene.codecs.uniformsplit.UniformSplitPostingsFormat.VERSION_CURRENT;
/**
* A block-based terms index and dictionary that assigns terms to nearly
* uniform length blocks. This technique is called Uniform Split.
* <p>
* The block construction is driven by two parameters, {@code targetNumBlockLines}
* and {@code deltaNumLines}.
* Each block size (number of terms) is {@code targetNumBlockLines}+-{@code deltaNumLines}.
* The algorithm computes the minimal distinguishing prefix (MDP) between
* each term and its previous term (alphabetically ordered). Then it selects
* in the neighborhood of the {@code targetNumBlockLines}, and within the
* {@code deltaNumLines}, the term with the minimal MDP. This term becomes
* the first term of the next block and its MDP is the block key. This block
* key is added to the terms dictionary trie.
* <p>
* We call dictionary the trie structure in memory, and block file the disk file
* containing the block lines, with one term and its corresponding term state
* details per line.
* <p>
* When seeking a term, the dictionary seeks the floor leaf of the trie for the
* searched term and jumps to the corresponding file pointer in the block file.
* There, the block terms are scanned for the exact searched term.
* <p>
* The terms inside a block do not need to share a prefix. Only the block
* key is used to find the block from the dictionary trie. And the block key
* is selected because it is the locally smallest MDP. This makes the dictionary
* trie very compact.
* <p>
* An interesting property of the Uniform Split technique is the very linear
* balance between memory usage and lookup performance. By decreasing
* the target block size, the block scan becomes faster, and since there are
* more blocks, the dictionary trie memory usage increases. Additionally,
* small blocks are faster to read from disk. A good sweet spot for the target
* block size is 32 with delta of 3 (10%) (default values). This can be tuned
* in the constructor.
* <p>
* There are additional optimizations:
* <ul>
* <li>Each block has a header that allows the lookup to jump directly to
* the middle term with a fast comparison. This reduces the linear scan
* by 2 for a small disk size increase.</li>
* <li>Each block term is incrementally encoded according to its previous
* term. This both reduces the disk size and speeds up the block scan.</li>
* <li>All term line details (the terms states) are written after all terms. This
* allows faster term scan without needing to decode the term states.</li>
* <li>All file pointers are base-encoded. Their value is relative to the block
* base file pointer (not to the previous file pointer), this allows to read the
* term state of any term independently.</li>
* </ul>
* <p>
* Blocks can be compressed or encrypted with an optional {@link BlockEncoder}
* provided in the {@link #UniformSplitTermsWriter(PostingsWriterBase, SegmentWriteState, int, int, BlockEncoder) constructor}.
* <p>
* The {@link UniformSplitPostingsFormat#TERMS_BLOCKS_EXTENSION block file}
* contains all the term blocks for each field sequentially. It also contains
* the fields metadata at the end of the file.
* <p>
* The {@link UniformSplitPostingsFormat#TERMS_DICTIONARY_EXTENSION dictionary file}
* contains the trie ({@link org.apache.lucene.util.fst.FST} bytes) for each
* field sequentially.
*
* @lucene.experimental
*/
public class UniformSplitTermsWriter extends FieldsConsumer {
/**
* Default value for the target block size (number of terms per block).
*/
public static final int DEFAULT_TARGET_NUM_BLOCK_LINES = 32;
/**
* Default value for the maximum allowed delta variation of the block size (delta of the number of terms per block).
* The block size will be [target block size]+-[allowed delta].
*/
public static final int DEFAULT_DELTA_NUM_LINES = (int) (DEFAULT_TARGET_NUM_BLOCK_LINES * 0.1);
/**
* Upper limit of the block size (maximum number of terms per block).
*/
protected static final int MAX_NUM_BLOCK_LINES = 1_000;
protected final FieldInfos fieldInfos;
protected final PostingsWriterBase postingsWriter;
protected final int maxDoc;
protected final int targetNumBlockLines;
protected final int deltaNumLines;
protected final BlockEncoder blockEncoder;
protected final IndexOutput blockOutput;
protected final IndexOutput dictionaryOutput;
/**
* @param blockEncoder Optional block encoder, may be null if none.
* It can be used for compression or encryption.
*/
public UniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state,
BlockEncoder blockEncoder) throws IOException {
this(postingsWriter, state, DEFAULT_TARGET_NUM_BLOCK_LINES, DEFAULT_DELTA_NUM_LINES, blockEncoder);
}
/**
* @param blockEncoder Optional block encoder, may be null if none.
* It can be used for compression or encryption.
*/
public UniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state,
int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder) throws IOException {
this(postingsWriter, state, targetNumBlockLines, deltaNumLines, blockEncoder,
NAME, VERSION_CURRENT, TERMS_BLOCKS_EXTENSION, TERMS_DICTIONARY_EXTENSION);
}
/**
* @param targetNumBlockLines Target number of lines per block.
* Must be strictly greater than 0.
* The parameters can be pre-validated with {@link #validateSettings(int, int)}.
* There is one term per block line, with its corresponding details ({@link org.apache.lucene.index.TermState}).
* @param deltaNumLines Maximum allowed delta variation of the number of lines per block.
* Must be greater than or equal to 0 and strictly less than {@code targetNumBlockLines}.
* The block size will be {@code targetNumBlockLines}+-{@code deltaNumLines}.
* The block size must always be less than or equal to {@link #MAX_NUM_BLOCK_LINES}.
* @param blockEncoder Optional block encoder, may be null if none.
* It can be used for compression or encryption.
*/
protected UniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state,
int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder,
String codecName, int versionCurrent, String termsBlocksExtension, String dictionaryExtension) throws IOException {
validateSettings(targetNumBlockLines, deltaNumLines);
IndexOutput blockOutput = null;
IndexOutput dictionaryOutput = null;
boolean success = false;
try {
this.fieldInfos = state.fieldInfos;
this.postingsWriter = postingsWriter;
this.maxDoc = state.segmentInfo.maxDoc();
this.targetNumBlockLines = targetNumBlockLines;
this.deltaNumLines = deltaNumLines;
this.blockEncoder = blockEncoder;
String termsName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, termsBlocksExtension);
blockOutput = state.directory.createOutput(termsName, state.context);
CodecUtil.writeIndexHeader(blockOutput, codecName, versionCurrent, state.segmentInfo.getId(), state.segmentSuffix);
String indexName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dictionaryExtension);
dictionaryOutput = state.directory.createOutput(indexName, state.context);
CodecUtil.writeIndexHeader(dictionaryOutput, codecName, versionCurrent, state.segmentInfo.getId(), state.segmentSuffix);
postingsWriter.init(blockOutput, state);
this.blockOutput = blockOutput;
this.dictionaryOutput = dictionaryOutput;
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(blockOutput, dictionaryOutput);
}
}
}
/**
* Validates the {@link #UniformSplitTermsWriter(PostingsWriterBase, SegmentWriteState, int, int, BlockEncoder) constructor}
* settings.
*
* @param targetNumBlockLines Target number of lines per block.
* Must be strictly greater than 0.
* @param deltaNumLines Maximum allowed delta variation of the number of lines per block.
* Must be greater than or equal to 0 and strictly less than {@code targetNumBlockLines}.
* Additionally, {@code targetNumBlockLines} + {@code deltaNumLines} must be less than
* or equal to {@link #MAX_NUM_BLOCK_LINES}.
*/
protected static void validateSettings(int targetNumBlockLines, int deltaNumLines) {
if (targetNumBlockLines <= 0) {
throw new IllegalArgumentException("Invalid negative or nul targetNumBlockLines=" + targetNumBlockLines);
}
if (deltaNumLines < 0) {
throw new IllegalArgumentException("Invalid negative deltaNumLines=" + deltaNumLines);
}
if (deltaNumLines >= targetNumBlockLines) {
throw new IllegalArgumentException("Invalid too large deltaNumLines=" + deltaNumLines
+ ", it must be < targetNumBlockLines=" + targetNumBlockLines);
}
if (targetNumBlockLines + deltaNumLines > UniformSplitTermsWriter.MAX_NUM_BLOCK_LINES) {
throw new IllegalArgumentException("Invalid (targetNumBlockLines + deltaNumLines)="
+ (targetNumBlockLines + deltaNumLines) + ", it must be <= MAX_NUM_BLOCK_LINES="
+ UniformSplitTermsWriter.MAX_NUM_BLOCK_LINES);
}
}
@Override
public void write(Fields fields, NormsProducer normsProducer) throws IOException {
BlockWriter blockWriter = new BlockWriter(blockOutput, targetNumBlockLines, deltaNumLines, blockEncoder);
ByteBuffersDataOutput fieldsOutput = new ByteBuffersDataOutput();
int fieldsNumber = 0;
for (String field : fields) {
Terms terms = fields.terms(field);
if (terms != null) {
TermsEnum termsEnum = terms.iterator();
FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
fieldsNumber += writeFieldTerms(blockWriter, fieldsOutput, termsEnum, fieldInfo, normsProducer);
}
}
writeFieldsMetadata(fieldsNumber, fieldsOutput);
CodecUtil.writeFooter(dictionaryOutput);
}
protected void writeFieldsMetadata(int fieldsNumber, ByteBuffersDataOutput fieldsOutput) throws IOException {
long fieldsStartPosition = blockOutput.getFilePointer();
blockOutput.writeVInt(fieldsNumber);
fieldsOutput.copyTo(blockOutput);
blockOutput.writeLong(fieldsStartPosition);
CodecUtil.writeFooter(blockOutput);
}
/**
* @return 1 if the field was written; 0 otherwise.
*/
protected int writeFieldTerms(BlockWriter blockWriter, DataOutput fieldsOutput, TermsEnum termsEnum,
FieldInfo fieldInfo, NormsProducer normsProducer) throws IOException {
FieldMetadata fieldMetadata = new FieldMetadata(fieldInfo, maxDoc);
fieldMetadata.setDictionaryStartFP(dictionaryOutput.getFilePointer());
postingsWriter.setField(fieldInfo);
blockWriter.setField(fieldMetadata);
IndexDictionary.Builder dictionaryBuilder = new FSTDictionary.Builder();
BytesRef lastTerm = null;
while (termsEnum.next() != null) {
BlockTermState blockTermState = writePostingLine(termsEnum, fieldMetadata, normsProducer);
if (blockTermState != null) {
lastTerm = BytesRef.deepCopyOf(termsEnum.term());
blockWriter.addLine(lastTerm, blockTermState, dictionaryBuilder);
}
}
// Flush remaining terms.
blockWriter.finishLastBlock(dictionaryBuilder);
if (fieldMetadata.getNumTerms() > 0) {
fieldMetadata.setLastTerm(lastTerm);
fieldMetadata.write(fieldsOutput);
writeDictionary(dictionaryBuilder);
return 1;
}
return 0;
}
/**
* Writes the posting values for the current term in the given {@link TermsEnum}
* and updates the {@link FieldMetadata} stats.
*
* @return the written {@link BlockTermState}; or null if none.
*/
protected BlockTermState writePostingLine(TermsEnum termsEnum, FieldMetadata fieldMetadata, NormsProducer normsProducer) throws IOException {
BlockTermState state = postingsWriter.writeTerm(termsEnum.term(), termsEnum, fieldMetadata.getDocsSeen(), normsProducer);
if (state == null) {
// No doc for this term.
return null;
}
fieldMetadata.updateStats(state);
return state;
}
/**
* Writes the dictionary index (FST) to disk.
*/
protected void writeDictionary(IndexDictionary.Builder dictionaryBuilder) throws IOException {
dictionaryBuilder.build().write(dictionaryOutput, blockEncoder);
}
@Override
public void close() throws IOException {
IOUtils.close(blockOutput, dictionaryOutput, postingsWriter);
}
}

View File

@ -0,0 +1,33 @@
/*
* 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.
*/
/**
* Pluggable term index / block terms dictionary implementations.
* <p>
* Structure similar to {@link org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter}
* with additional optimizations.
* <p>
* <ul>
* <li>Designed to be extensible</li>
* <li>Reduced on-heap memory usage.</li>
* <li>Efficient to seek terms ({@link org.apache.lucene.search.TermQuery}, {@link org.apache.lucene.search.PhraseQuery})</li>
* <li>Quite efficient for {@link org.apache.lucene.search.PrefixQuery}</li>
* <li>Not efficient for spell-check and {@link org.apache.lucene.search.FuzzyQuery}, in this case prefer
* {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat}</li>
* </ul>
*/
package org.apache.lucene.codecs.uniformsplit;

View File

@ -0,0 +1,37 @@
/*
* 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.sharedterms;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.uniformsplit.FieldMetadata;
/**
* Pair of {@link FieldMetadata} and {@link BlockTermState} for a specific field.
*
* @lucene.experimental
*/
public class FieldMetadataTermState {
public final FieldMetadata fieldMetadata;
public final BlockTermState state;
public FieldMetadataTermState(FieldMetadata fieldMetadata, BlockTermState state) {
this.fieldMetadata = fieldMetadata;
this.state = state;
}
}

View File

@ -0,0 +1,194 @@
/*
* 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.sharedterms;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.uniformsplit.BlockHeader;
import org.apache.lucene.codecs.uniformsplit.BlockLine;
import org.apache.lucene.codecs.uniformsplit.DeltaBaseTermStateSerializer;
import org.apache.lucene.codecs.uniformsplit.FieldMetadata;
import org.apache.lucene.codecs.uniformsplit.TermBytes;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
/**
* Represents a term and its details stored in the {@link BlockTermState}.
* It is an extension of {@link BlockLine} for the Shared Terms format. This
* means the line contains a term and all its fields {@link org.apache.lucene.index.TermState}s.
*
* @lucene.experimental
*/
public class STBlockLine extends BlockLine {
/**
* List of the fields ond their TermStates for this block line. Only used for writing.
*/
protected final List<FieldMetadataTermState> termStates;
public STBlockLine(TermBytes termBytes, List<FieldMetadataTermState> termStates) {
super(termBytes, null);
assert !termStates.isEmpty();
this.termStates = new ArrayList<>(termStates);
}
/**
* Collects the {@link FieldMetadata} of all fields listed in this line.
* @param collector Receives the collected {@link FieldMetadata}.
*/
public void collectFields(Collection<FieldMetadata> collector) {
for (FieldMetadataTermState fieldTermState : termStates) {
collector.add(fieldTermState.fieldMetadata);
}
}
/**
* Reads block lines encoded incrementally, with all fields corresponding
* to the term of the line.
* <p>
* This class extends {@link org.apache.lucene.codecs.uniformsplit.BlockLine.Serializer},
* so it keeps a state of the previous term read to decode the next term.
*/
public static class Serializer extends BlockLine.Serializer {
/**
* Writes all the {@link BlockTermState} of the provided {@link STBlockLine} to the given output.
*/
public static void writeLineTermStates(DataOutput termStatesOutput, STBlockLine line,
DeltaBaseTermStateSerializer encoder) throws IOException {
FieldMetadataTermState fieldMetadataTermState;
int size = line.termStates.size();
assert size > 0 : "not valid block line with :" + size + " lines.";
if (size == 1) {
// When there is only 1 field, write its id as negative, followed by the field TermState.
int fieldID = line.termStates.get(0).fieldMetadata.getFieldInfo().number;
termStatesOutput.writeZInt(-fieldID);
fieldMetadataTermState = line.termStates.get(0);
encoder.writeTermState(termStatesOutput, fieldMetadataTermState.fieldMetadata.getFieldInfo(), fieldMetadataTermState.state);
return;
}
termStatesOutput.writeZInt(size);
// First iteration writes the fields ids.
for (int i = 0; i < size; i++) {
fieldMetadataTermState = line.termStates.get(i);
termStatesOutput.writeVInt(fieldMetadataTermState.fieldMetadata.getFieldInfo().number);
}
// Second iteration writes the corresponding field TermStates.
for (int i = 0; i < size; i++) {
fieldMetadataTermState = line.termStates.get(i);
encoder.writeTermState(termStatesOutput, fieldMetadataTermState.fieldMetadata.getFieldInfo(), fieldMetadataTermState.state);
}
}
/**
* Reads a single {@link BlockTermState} for the provided field in the current block line of the provided input.
* @param termStatesInput Data input to read the {@link BlockTermState} from.
* @param blockHeader Current block header.
* @param reuse A previous {@link BlockTermState} to reuse; or null to create a new one.
* @return The {@link BlockTermState} corresponding to the provided field id; or null if the field
* does not occur in the line.
*/
public static BlockTermState readTermStateForField(int fieldId, DataInput termStatesInput,
DeltaBaseTermStateSerializer termStateSerializer,
BlockHeader blockHeader, FieldInfos fieldInfos,
BlockTermState reuse) throws IOException {
assert fieldId >= 0;
int numFields = termStatesInput.readZInt();
if (numFields <= 0) {
int readFieldId = -numFields;
if (fieldId == readFieldId) {
return termStateSerializer.readTermState(blockHeader.getBaseDocsFP(), blockHeader.getBasePositionsFP(),
blockHeader.getBasePayloadsFP(), termStatesInput, fieldInfos.fieldInfo(readFieldId), reuse);
}
return null;
}
// There are multiple fields for the term.
// We have to read all the field ids (aka field numbers) sequentially.
// Then if the required field is in the list, we have to read all the TermState
// sequentially. This could be optimized with a jump-to-middle offset
// for example, but we don't need that currently.
boolean isFieldInList = false;
int[] readFieldIds = new int[numFields];
for (int i = 0; i < numFields; i++) {
int readFieldId = termStatesInput.readVInt();
if (!isFieldInList && readFieldId > fieldId) {
// As the list of fieldIds is sorted we can return early if we find fieldId greater than the seeked one.
// But if we found the seeked one, we have to read all the list to get to the term state part afterward (there is no jump offset).
return null;
}
isFieldInList |= readFieldId == fieldId;
readFieldIds[i] = readFieldId;
}
if (isFieldInList) {
for (int readFieldId : readFieldIds) {
BlockTermState termState = termStateSerializer.readTermState(blockHeader.getBaseDocsFP(), blockHeader.getBasePositionsFP(),
blockHeader.getBasePayloadsFP(), termStatesInput, fieldInfos.fieldInfo(readFieldId), reuse);
if (fieldId == readFieldId) {
return termState;
}
}
}
return null;
}
/**
* Reads all the {@link BlockTermState} of all the field in the current block line of the provided input.
* @param fieldTermStatesMap Map filled with the term states for each field. It is cleared first.
* @see #readTermStateForField
*/
public static void readFieldTermStatesMap(DataInput termStatesInput,
DeltaBaseTermStateSerializer termStateSerializer,
BlockHeader blockHeader,
FieldInfos fieldInfos,
Map<String, BlockTermState> fieldTermStatesMap) throws IOException {
fieldTermStatesMap.clear();
int numFields = termStatesInput.readZInt();
if (numFields <= 0) {
int fieldId = -numFields;
fieldTermStatesMap.put(fieldInfos.fieldInfo(fieldId).name, termStateSerializer.readTermState(blockHeader.getBaseDocsFP(), blockHeader.getBasePositionsFP(),
blockHeader.getBasePayloadsFP(), termStatesInput, fieldInfos.fieldInfo(fieldId), null));
return;
}
for (int fieldId : readFieldIds(termStatesInput, numFields)) {
fieldTermStatesMap.put(fieldInfos.fieldInfo(fieldId).name, termStateSerializer.readTermState(blockHeader.getBaseDocsFP(), blockHeader.getBasePositionsFP(),
blockHeader.getBasePayloadsFP(), termStatesInput, fieldInfos.fieldInfo(fieldId), null));
}
}
/**
* Reads all the field ids in the current block line of the provided input.
*/
public static int[] readFieldIds(DataInput termStatesInput, int numFields) throws IOException {
int[] fieldIds = new int[numFields];
for (int i = 0; i < numFields; i++) {
fieldIds[i] = termStatesInput.readVInt();
}
return fieldIds;
}
}
}

View File

@ -0,0 +1,139 @@
/*
* 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.sharedterms;
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.codecs.uniformsplit.BlockDecoder;
import org.apache.lucene.codecs.uniformsplit.BlockReader;
import org.apache.lucene.codecs.uniformsplit.FieldMetadata;
import org.apache.lucene.codecs.uniformsplit.IndexDictionary;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
/**
* Reads terms blocks with the Shared Terms format.
*
* @see STBlockWriter
* @lucene.experimental
*/
public class STBlockReader extends BlockReader {
protected final FieldInfos fieldInfos;
public STBlockReader(Supplier<IndexDictionary.Browser> dictionaryBrowserSupplier,
IndexInput blockInput, PostingsReaderBase postingsReader,
FieldMetadata fieldMetadata, BlockDecoder blockDecoder, FieldInfos fieldInfos) throws IOException {
super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
this.fieldInfos = fieldInfos;
}
@Override
public BytesRef next() throws IOException {
BytesRef next = super.next();
if (next == null) {
return null;
}
// Check if the term occurs for the searched field.
while (!termOccursInField()) {
next = super.next();
if (next == null) {
// No more term for any field.
return null;
}
}
// The term occurs for the searched field.
return next;
}
private boolean termOccursInField() throws IOException {
readTermStateIfNotRead();
return termState != null;
}
/**
* Moves to the next term line and reads it, whichever are the corresponding fields.
* The term details are not read yet. They will be read only when needed
* with {@link #readTermStateIfNotRead()}.
*
* @return The read term bytes.
*/
@Override
protected BytesRef nextTerm() throws IOException {
BytesRef nextTerm = super.nextTerm();
if (nextTerm != null && super.isBeyondLastTerm(nextTerm, blockStartFP)) {
return null;
}
return nextTerm;
}
@Override
public SeekStatus seekCeil(BytesRef searchedTerm) throws IOException {
SeekStatus seekStatus = seekCeilIgnoreField(searchedTerm);
if (seekStatus != SeekStatus.END) {
if (!termOccursInField()) {
// The term does not occur for the field.
// We have to move the iterator to the next valid term for the field.
BytesRef nextTerm = next();
seekStatus = nextTerm == null ? SeekStatus.END : SeekStatus.NOT_FOUND;
}
}
return seekStatus;
}
// Visible for testing.
SeekStatus seekCeilIgnoreField(BytesRef searchedTerm) throws IOException {
return super.seekCeil(searchedTerm);
}
@Override
public boolean seekExact(BytesRef searchedTerm) throws IOException {
if (super.seekExact(searchedTerm)) {
return termOccursInField();
}
return false;
}
@Override
protected boolean isBeyondLastTerm(BytesRef searchedTerm, long blockStartFP) {
return blockStartFP > fieldMetadata.getLastBlockStartFP() || super.isBeyondLastTerm(searchedTerm, blockStartFP);
}
/**
* Reads the {@link BlockTermState} on the current line for this reader's field.
*
* @return The {@link BlockTermState}; or null if the term does not occur for the field.
*/
@Override
protected BlockTermState readTermState() throws IOException {
termStatesReadBuffer.setPosition(blockFirstLineStart + blockHeader.getTermStatesBaseOffset() + blockLine.getTermStateRelativeOffset());
return termState = STBlockLine.Serializer.readTermStateForField(
fieldMetadata.getFieldInfo().number,
termStatesReadBuffer,
termStateSerializer,
blockHeader,
fieldInfos,
scratchTermState
);
}
}

View File

@ -0,0 +1,105 @@
/*
* 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.sharedterms;
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.lucene.codecs.uniformsplit.BlockEncoder;
import org.apache.lucene.codecs.uniformsplit.BlockLine;
import org.apache.lucene.codecs.uniformsplit.BlockWriter;
import org.apache.lucene.codecs.uniformsplit.FieldMetadata;
import org.apache.lucene.codecs.uniformsplit.IndexDictionary;
import org.apache.lucene.codecs.uniformsplit.TermBytes;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
/**
* Writes terms blocks with the Shared Terms format.
* <p>
* As defined in {@link STUniformSplitTermsWriter}, all the fields terms are
* shared in the same dictionary. Each block line contains a term and all the
* fields {@link org.apache.lucene.index.TermState}s for this term.
*
* @lucene.experimental
*/
public class STBlockWriter extends BlockWriter {
protected final Set<FieldMetadata> fieldsInBlock;
public STBlockWriter(IndexOutput blockOutput, int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder) {
super(blockOutput, targetNumBlockLines, deltaNumLines, blockEncoder);
fieldsInBlock = new HashSet<>();
}
/**
* Adds a new {@link BlockLine} term for the current field.
* <p>
* This method determines whether the new term is part of the current block,
* or if it is part of the next block. In the latter case, a new block is started
* (including one or more of the lastly added lines), the current block is
* written to the block file, and the current block key is added to the
* {@link org.apache.lucene.codecs.uniformsplit.IndexDictionary.Builder}.
*
* @param term The block line term. The {@link BytesRef} instance is used directly,
* the caller is responsible to make a deep copy if needed. This is required
* because we keep a list of block lines until we decide to write the
* current block, and each line must have a different term instance.
* @param termStates Block line details for all fields in the line.
* @param dictionaryBuilder to which the block keys are added.
*/
public void addLine(BytesRef term, List<FieldMetadataTermState> termStates,
IndexDictionary.Builder dictionaryBuilder) throws IOException {
if (termStates.isEmpty()) {
return;
}
int mdpLength = TermBytes.computeMdpLength(lastTerm, term);
blockLines.add(new STBlockLine(new TermBytes(mdpLength, term), termStates));
lastTerm = term;
if (blockLines.size() >= targetNumBlockLines + deltaNumLines) {
splitAndWriteBlock(dictionaryBuilder);
}
}
@Override
protected void finishLastBlock(IndexDictionary.Builder dictionaryBuilder) throws IOException {
// Make this method accessible to package.
super.finishLastBlock(dictionaryBuilder);
}
@Override
protected void writeBlockLine(boolean isIncrementalEncodingSeed, BlockLine line, BlockLine previousLine) throws IOException {
STBlockLine.Serializer.writeLine(blockLinesWriteBuffer, line, previousLine, Math.toIntExact(termStatesWriteBuffer.size()), isIncrementalEncodingSeed);
STBlockLine.Serializer.writeLineTermStates(termStatesWriteBuffer, (STBlockLine) line, termStateSerializer);
((STBlockLine) line).collectFields(fieldsInBlock);
}
@Override
protected void updateFieldMetadata(long blockStartFP) {
assert !fieldsInBlock.isEmpty();
for (FieldMetadata fieldMetadata : fieldsInBlock) {
if (fieldMetadata.getFirstBlockStartFP() == -1) {
fieldMetadata.setFirstBlockStartFP(blockStartFP);
}
fieldMetadata.setLastBlockStartFP(blockStartFP);
}
fieldsInBlock.clear();
}
}

View File

@ -0,0 +1,112 @@
/*
* 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.sharedterms;
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.IntersectBlockReader;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.automaton.CompiledAutomaton;
/**
* The "intersect" {@link org.apache.lucene.index.TermsEnum} response to
* {@link STUniformSplitTerms#intersect(CompiledAutomaton, BytesRef)},
* intersecting the terms with an automaton.
*
* @lucene.experimental
*/
public class STIntersectBlockReader extends IntersectBlockReader {
protected final FieldInfos fieldInfos;
public STIntersectBlockReader(CompiledAutomaton compiled, BytesRef startTerm,
DictionaryBrowserSupplier dictionaryBrowserSupplier, IndexInput blockInput, PostingsReaderBase postingsReader,
FieldMetadata fieldMetadata, BlockDecoder blockDecoder, FieldInfos fieldInfos) throws IOException {
super(compiled, startTerm, dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder);
this.fieldInfos = fieldInfos;
}
// ---------------------------------------------
// The methods below are duplicate from STBlockReader.
//
// This class inherits code from both IntersectBlockReader and STBlockReader.
// We choose to extend IntersectBlockReader because this is the one that
// runs the next(), reads the block lines and keeps the reader state.
// But we still need the STBlockReader logic to skip terms that do not occur
// in this TermsEnum field.
// So we end up having a couple of methods directly duplicate from STBlockReader.
// We tried various different approaches to avoid duplicating the code, but
// actually this becomes difficult to read and to understand. This is simpler
// to duplicate and explain it here.
// ---------------------------------------------
@Override
public BytesRef next() throws IOException {
BytesRef next = super.next();
if (next == null) {
return null;
}
// Check if the term occurs for the searched field.
while (!termOccursInField()) {
next = super.next();
if (next == null) {
// No more term.
return null;
}
}
// The term occurs for the searched field.
return next;
}
private boolean termOccursInField() throws IOException {
readTermStateIfNotRead();
return termState != null;
}
@Override
protected boolean nextBlockMatchingPrefix() throws IOException {
// block header maybe null if we are positioned outside the field block range
return super.nextBlockMatchingPrefix() && blockHeader != null;
}
/**
* Reads the {@link BlockTermState} on the current line for the specific field
* corresponding this this reader.
* Changes the current {@link BlockTermState} to null if the term does not
* occur for the field.
*/
@Override
protected BlockTermState readTermState() throws IOException {
termStatesReadBuffer.setPosition(blockFirstLineStart + blockHeader.getTermStatesBaseOffset() + blockLine.getTermStateRelativeOffset());
return STBlockLine.Serializer.readTermStateForField(
fieldMetadata.getFieldInfo().number,
termStatesReadBuffer,
termStateSerializer,
blockHeader,
fieldInfos,
scratchTermState
);
}
}

View File

@ -0,0 +1,111 @@
/*
* 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.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;
import org.apache.lucene.codecs.uniformsplit.BlockDecoder;
import org.apache.lucene.codecs.uniformsplit.FieldMetadata;
import org.apache.lucene.codecs.uniformsplit.IndexDictionary;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.TermState;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
/**
* {@link org.apache.lucene.index.TermsEnum} used when merging segments,
* to enumerate the terms of an input segment and get all the fields {@link TermState}s
* of each term.
* <p>
* It only supports calls to {@link #next()} and no seek method.
*
* @lucene.experimental
*/
public class STMergingBlockReader extends STBlockReader {
public STMergingBlockReader(
Supplier<IndexDictionary.Browser> dictionaryBrowserSupplier,
IndexInput blockInput,
PostingsReaderBase postingsReader,
FieldMetadata fieldMetadata,
BlockDecoder blockDecoder,
FieldInfos fieldInfos) throws IOException {
super(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder, fieldInfos);
}
@Override
public SeekStatus seekCeil(BytesRef searchedTerm) {
throw new UnsupportedOperationException();
}
@Override
public boolean seekExact(BytesRef searchedTerm) {
throw new UnsupportedOperationException();
}
@Override
public void seekExact(BytesRef term, TermState state) {
throw new UnsupportedOperationException();
}
@Override
public void seekExact(long ord) {
throw new UnsupportedOperationException();
}
@Override
protected BlockTermState readTermStateIfNotRead() {
throw new UnsupportedOperationException();
}
@Override
public BytesRef next() throws IOException {
return nextTerm();
}
/**
* Creates a new {@link PostingsEnum} for the provided field and {@link BlockTermState}.
* @param reuse Previous {@link PostingsEnum} to reuse; or null to create a new one.
* @param flags Postings flags.
*/
public PostingsEnum postings(String fieldName, BlockTermState termState, PostingsEnum reuse, int flags) throws IOException {
return postingsReader.postings(fieldInfos.fieldInfo(fieldName), termState, reuse, flags);
}
/**
* Reads all the fields {@link TermState}s of the current term and put them
* in the provided map. Clears the map first, before putting {@link TermState}s.
*/
public void readFieldTermStatesMap(Map<String, BlockTermState> fieldTermStatesMap) throws IOException {
if (term() != null) {
termStatesReadBuffer.setPosition(blockFirstLineStart + blockHeader.getTermStatesBaseOffset() + blockLine.getTermStateRelativeOffset());
STBlockLine.Serializer.readFieldTermStatesMap(
termStatesReadBuffer,
termStateSerializer,
blockHeader,
fieldInfos,
fieldTermStatesMap
);
}
}
}

View File

@ -0,0 +1,243 @@
/*
* 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.sharedterms;
import java.io.IOException;
import java.util.List;
import java.util.RandomAccess;
import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
/**
* Combines {@link PostingsEnum} for the same term for a given field from
* multiple segments. It is used during segment merging.
*
* @lucene.experimental
*/
public class STMergingTermsEnum extends TermsEnum {
protected final String fieldName;
protected final MultiSegmentsPostingsEnum multiPostingsEnum;
protected BytesRef term;
/**
* Constructs a {@link STMergingTermsEnum} for a given field.
*/
protected STMergingTermsEnum(String fieldName, int numSegments) {
this.fieldName = fieldName;
multiPostingsEnum = new MultiSegmentsPostingsEnum(numSegments);
}
/**
* Resets this {@link STMergingTermsEnum} with a new term and its list of
* {@link STUniformSplitTermsWriter.SegmentPostings} to combine.
*
* @param segmentPostings List sorted by segment index.
*/
protected void reset(BytesRef term, List<STUniformSplitTermsWriter.SegmentPostings> segmentPostings) {
this.term = term;
multiPostingsEnum.reset(segmentPostings);
}
@Override
public AttributeSource attributes() {
throw new UnsupportedOperationException();
}
@Override
public boolean seekExact(BytesRef text) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public SeekStatus seekCeil(BytesRef text) {
throw new UnsupportedOperationException();
}
@Override
public void seekExact(long ord) {
throw new UnsupportedOperationException();
}
@Override
public void seekExact(BytesRef term, TermState state) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public BytesRef term() {
return term;
}
@Override
public long ord() {
throw new UnsupportedOperationException();
}
@Override
public int docFreq() {
throw new UnsupportedOperationException();
}
@Override
public long totalTermFreq() {
throw new UnsupportedOperationException();
}
@Override
public PostingsEnum postings(PostingsEnum reuse, int flags) {
multiPostingsEnum.setPostingFlags(flags);
return multiPostingsEnum;
}
@Override
public ImpactsEnum impacts(int flags) {
throw new UnsupportedOperationException();
}
@Override
public TermState termState() {
throw new UnsupportedOperationException();
}
@Override
public BytesRef next() {
throw new UnsupportedOperationException();
}
/**
* Combines multiple segments {@link PostingsEnum} as a single {@link PostingsEnum},
* for one field and one term.
* <p>
* This {@link PostingsEnum} does not extend {@link org.apache.lucene.index.FilterLeafReader.FilterPostingsEnum}
* because it updates the delegate for each segment.
*/
protected class MultiSegmentsPostingsEnum extends PostingsEnum {
protected final PostingsEnum[] reusablePostingsEnums;
protected List<STUniformSplitTermsWriter.SegmentPostings> segmentPostingsList;
protected int segmentIndex;
protected PostingsEnum postingsEnum;
protected boolean postingsEnumExhausted;
protected MergeState.DocMap docMap;
protected int docId;
protected int postingsFlags;
protected MultiSegmentsPostingsEnum(int numSegments) {
reusablePostingsEnums = new PostingsEnum[numSegments];
}
/**
* Resets/reuse this {@link PostingsEnum}.
* @param segmentPostingsList List of segment postings ordered by segment index.
*/
protected void reset(List<STUniformSplitTermsWriter.SegmentPostings> segmentPostingsList) {
assert segmentPostingsList instanceof RandomAccess
: "We optimize by accessing the list elements instead of creating an Iterator";
this.segmentPostingsList = segmentPostingsList;
segmentIndex = -1;
postingsEnumExhausted = true;
docId = -1;
}
protected void setPostingFlags(int flags) {
this.postingsFlags = flags;
}
@Override
public int freq() throws IOException {
return postingsEnum.freq();
}
@Override
public int nextPosition() throws IOException {
return postingsEnum.nextPosition();
}
@Override
public int startOffset() throws IOException {
return postingsEnum.startOffset();
}
@Override
public int endOffset() throws IOException {
return postingsEnum.endOffset();
}
@Override
public BytesRef getPayload() throws IOException {
return postingsEnum.getPayload();
}
@Override
public int docID() {
return docId;
}
@Override
public int nextDoc() throws IOException {
assert segmentPostingsList != null : "reset not called";
while (true) {
if (postingsEnumExhausted) {
if (segmentIndex == segmentPostingsList.size() - 1) {
return docId = NO_MORE_DOCS;
} else {
segmentIndex++;
STUniformSplitTermsWriter.SegmentPostings segmentPostings =
segmentPostingsList.get(segmentIndex);
postingsEnum = getPostings(segmentPostings);
postingsEnumExhausted = false;
docMap = segmentPostings.docMap;
}
}
int docId = postingsEnum.nextDoc();
if (docId == NO_MORE_DOCS) {
postingsEnumExhausted = true;
} else {
docId = docMap.get(docId);
if (docId != -1) {
assert docId > this.docId : "next docId=" + docId + ", current docId=" + this.docId;
return this.docId = docId;
}
}
}
}
protected PostingsEnum getPostings(STUniformSplitTermsWriter.SegmentPostings segmentPostings) throws IOException {
// The field is present in the segment because it is one of the segments provided in the reset() method.
return reusablePostingsEnums[segmentPostings.segmentIndex] =
segmentPostings.getPostings(fieldName, reusablePostingsEnums[segmentPostings.segmentIndex], postingsFlags);
}
@Override
public int advance(int target) {
throw new UnsupportedOperationException();
}
@Override
public long cost() {
return 0; // Cost is not useful here.
}
}
}

View File

@ -0,0 +1,84 @@
/*
* 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.sharedterms;
import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.uniformsplit.BlockDecoder;
import org.apache.lucene.codecs.uniformsplit.BlockEncoder;
import org.apache.lucene.codecs.uniformsplit.UniformSplitPostingsFormat;
import org.apache.lucene.codecs.uniformsplit.UniformSplitTermsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/**
* {@link PostingsFormat} based on the Uniform Split technique and supporting
* Shared Terms.
* <p>
* Shared Terms means the terms of all fields are stored in the same block file,
* with multiple fields associated to one term (one block line). In the same way,
* the dictionary trie is also shared between all fields. This highly reduces
* the memory required by the field dictionary compared to having one separate
* dictionary per field.
*
* @lucene.experimental
*/
public class STUniformSplitPostingsFormat extends UniformSplitPostingsFormat {
/**
* Extension of the file containing the terms dictionary (the FST "trie").
*/
public static final String TERMS_DICTIONARY_EXTENSION = "stustd";
/**
* Extension of the file containing the terms blocks for each field and the fields metadata.
*/
public static final String TERMS_BLOCKS_EXTENSION = "stustb";
public static final int VERSION_CURRENT = 0;
public static final String NAME = "SharedTermsUniformSplit";
public STUniformSplitPostingsFormat() {
this(UniformSplitTermsWriter.DEFAULT_TARGET_NUM_BLOCK_LINES, UniformSplitTermsWriter.DEFAULT_DELTA_NUM_LINES, null, null);
}
public STUniformSplitPostingsFormat(int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder, BlockDecoder blockDecoder) {
this(NAME, targetNumBlockLines, deltaNumLines, blockEncoder, blockDecoder);
}
protected STUniformSplitPostingsFormat(String name, int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder, BlockDecoder blockDecoder) {
super(name, targetNumBlockLines, deltaNumLines, blockEncoder, blockDecoder);
}
@Override
protected FieldsConsumer createUniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state,
int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder) throws IOException {
return new STUniformSplitTermsWriter(postingsWriter, state, targetNumBlockLines, deltaNumLines, blockEncoder);
}
@Override
protected FieldsProducer createUniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state,
BlockDecoder blockDecoder) throws IOException {
return new STUniformSplitTermsReader(postingsReader, state, blockDecoder);
}
}

View File

@ -0,0 +1,65 @@
/*
* 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.sharedterms;
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.UniformSplitTerms;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.automaton.CompiledAutomaton;
/**
* Extends {@link UniformSplitTerms} for a shared-terms dictionary, with
* all the fields of a term in the same block line.
*
* @lucene.experimental
*/
public class STUniformSplitTerms extends UniformSplitTerms {
protected final FieldMetadata unionFieldMetadata;
protected final FieldInfos fieldInfos;
protected STUniformSplitTerms(IndexInput blockInput, FieldMetadata fieldMetadata,
FieldMetadata unionFieldMetadata, PostingsReaderBase postingsReader,
BlockDecoder blockDecoder, FieldInfos fieldInfos, DictionaryBrowserSupplier dictionaryBrowserSupplier) {
super(blockInput, fieldMetadata, postingsReader, blockDecoder, dictionaryBrowserSupplier);
this.unionFieldMetadata = unionFieldMetadata;
this.fieldInfos = fieldInfos;
}
@Override
public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
return new STIntersectBlockReader(compiled, startTerm, dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder, fieldInfos);
}
@Override
public TermsEnum iterator() throws IOException {
return new STBlockReader(dictionaryBrowserSupplier, blockInput, postingsReader, fieldMetadata, blockDecoder, fieldInfos);
}
STMergingBlockReader createMergingBlockReader() throws IOException {
return new STMergingBlockReader(dictionaryBrowserSupplier, blockInput, postingsReader, unionFieldMetadata, blockDecoder, fieldInfos);
}
}

View File

@ -0,0 +1,98 @@
/*
* 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.sharedterms;
import java.io.IOException;
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.FieldMetadata;
import org.apache.lucene.codecs.uniformsplit.UniformSplitTerms;
import org.apache.lucene.codecs.uniformsplit.UniformSplitTermsReader;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.IndexInput;
import static org.apache.lucene.codecs.uniformsplit.sharedterms.STUniformSplitPostingsFormat.NAME;
import static org.apache.lucene.codecs.uniformsplit.sharedterms.STUniformSplitPostingsFormat.TERMS_BLOCKS_EXTENSION;
import static org.apache.lucene.codecs.uniformsplit.sharedterms.STUniformSplitPostingsFormat.TERMS_DICTIONARY_EXTENSION;
import static org.apache.lucene.codecs.uniformsplit.sharedterms.STUniformSplitPostingsFormat.VERSION_CURRENT;
/**
* A block-based terms index and dictionary based on the Uniform Split technique,
* and sharing all the fields terms in the same dictionary, with all the fields
* of a term in the same block line.
*
* @see STUniformSplitTermsWriter
* @lucene.experimental
*/
public class STUniformSplitTermsReader extends UniformSplitTermsReader {
public STUniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state, BlockDecoder blockDecoder) throws IOException {
super(postingsReader, state, blockDecoder, NAME, VERSION_START,
VERSION_CURRENT, TERMS_BLOCKS_EXTENSION, TERMS_DICTIONARY_EXTENSION);
}
protected STUniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state, BlockDecoder blockDecoder,
String codecName, int versionStart, int versionCurrent, String termsBlocksExtension, String dictionaryExtension) throws IOException {
super(postingsReader, state, blockDecoder, codecName, versionStart, versionCurrent, termsBlocksExtension, dictionaryExtension);
}
@Override
protected void fillFieldMap(PostingsReaderBase postingsReader, BlockDecoder blockDecoder,
IndexInput dictionaryInput, IndexInput blockInput,
Collection<FieldMetadata> fieldMetadataCollection, FieldInfos fieldInfos) throws IOException {
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);
for (FieldMetadata fieldMetadata : fieldMetadataCollection) {
fieldToTermsMap.put(fieldMetadata.getFieldInfo().name,
new STUniformSplitTerms(blockInput, fieldMetadata, unionFieldMetadata, postingsReader, blockDecoder, fieldInfos, dictionaryBrowserSupplier));
}
}
}
@Override
protected long getTermsRamBytesUsed() {
long termsRamUsage = 0L;
long dictionaryRamUsage = 0L;
for (UniformSplitTerms terms : fieldToTermsMap.values()) {
termsRamUsage += terms.ramBytesUsedWithoutDictionary();
dictionaryRamUsage = terms.getDictionaryRamBytesUsed();
}
termsRamUsage += dictionaryRamUsage;
return termsRamUsage;
}
/**
* Creates a virtual {@link FieldMetadata} that is the union of the given {@link FieldMetadata}s.
* Its {@link FieldMetadata#getFirstBlockStartFP}, {@link FieldMetadata#getLastBlockStartFP}
* and {@link FieldMetadata#getLastTerm()} are respectively the min and
* max among the {@link FieldMetadata}s provided as parameter.
*/
protected FieldMetadata createUnionFieldMetadata(Iterable<FieldMetadata> fieldMetadataIterable) {
UnionFieldMetadataBuilder builder = new UnionFieldMetadataBuilder();
for (FieldMetadata fieldMetadata : fieldMetadataIterable) {
builder.addFieldMetadata(fieldMetadata);
}
return builder.build();
}
}

View File

@ -0,0 +1,477 @@
/*
* 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.sharedterms;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.uniformsplit.BlockEncoder;
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.UniformSplitTermsWriter;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.PriorityQueue;
import static org.apache.lucene.codecs.uniformsplit.sharedterms.STUniformSplitPostingsFormat.NAME;
import static org.apache.lucene.codecs.uniformsplit.sharedterms.STUniformSplitPostingsFormat.TERMS_BLOCKS_EXTENSION;
import static org.apache.lucene.codecs.uniformsplit.sharedterms.STUniformSplitPostingsFormat.TERMS_DICTIONARY_EXTENSION;
import static org.apache.lucene.codecs.uniformsplit.sharedterms.STUniformSplitPostingsFormat.VERSION_CURRENT;
/**
* Extends {@link UniformSplitTermsWriter} by sharing all the fields terms
* in the same dictionary and by writing all the fields of a term in the same
* block line.
* <p>
* The {@link STUniformSplitPostingsFormat#TERMS_BLOCKS_EXTENSION block file}
* contains all the term blocks for all fields. Each block line, for a single term,
* may have multiple fields {@link org.apache.lucene.index.TermState}. The
* block file also contains the fields metadata at the end of the file.
* <p>
* The {@link STUniformSplitPostingsFormat#TERMS_DICTIONARY_EXTENSION dictionary file}
* contains a single trie ({@link org.apache.lucene.util.fst.FST} bytes) for all
* fields.
* <p>
* This structure is adapted when there are lots of fields. In this case the shared-terms
* dictionary trie is much smaller.
* <p>
* This {@link org.apache.lucene.codecs.FieldsConsumer} requires a custom
* {@link #merge(MergeState, NormsProducer)} method for efficiency. The
* regular merge would scan all the fields sequentially, which internally would
* scan the whole shared-terms dictionary as many times as there are fields.
* Whereas the custom merge directly scans the internal shared-terms dictionary
* of all segments to merge, thus scanning once whatever the number of fields is.
*
* @lucene.experimental
*/
public class STUniformSplitTermsWriter extends UniformSplitTermsWriter {
public STUniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state,
BlockEncoder blockEncoder) throws IOException {
this(postingsWriter, state, DEFAULT_TARGET_NUM_BLOCK_LINES, DEFAULT_DELTA_NUM_LINES, blockEncoder);
}
public STUniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state,
int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder) throws IOException {
this(postingsWriter, state, targetNumBlockLines, deltaNumLines, blockEncoder, NAME, VERSION_CURRENT, TERMS_BLOCKS_EXTENSION, TERMS_DICTIONARY_EXTENSION);
}
protected STUniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state,
int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder,
String codecName, int versionCurrent, String termsBlocksExtension, String dictionaryExtension) throws IOException {
super(postingsWriter, state, targetNumBlockLines, deltaNumLines, blockEncoder, codecName, versionCurrent, termsBlocksExtension, dictionaryExtension);
}
@Override
public void write(Fields fields, NormsProducer normsProducer) throws IOException {
writeSegment((blockWriter, dictionaryBuilder) -> writeSingleSegment(fields, normsProducer, blockWriter, dictionaryBuilder));
}
/**
* Writes the new segment with the provided {@link SharedTermsWriter},
* which can be either a single segment writer, or a multiple segment merging writer.
*/
private void writeSegment(SharedTermsWriter termsWriter) throws IOException {
STBlockWriter blockWriter = new STBlockWriter(blockOutput, targetNumBlockLines, deltaNumLines, blockEncoder);
IndexDictionary.Builder dictionaryBuilder = new FSTDictionary.Builder();
Collection<FieldMetadata> fieldMetadataList = termsWriter.writeSharedTerms(blockWriter, dictionaryBuilder);
blockWriter.finishLastBlock(dictionaryBuilder);
int fieldsNumber = writeFieldMetadataList(fieldMetadataList);
writeDictionary(fieldsNumber, dictionaryBuilder);
}
private Collection<FieldMetadata> writeSingleSegment(Fields fields, NormsProducer normsProducer, STBlockWriter blockWriter, IndexDictionary.Builder dictionaryBuilder) throws IOException {
List<FieldMetadata> fieldMetadataList = createFieldMetadataList(new FieldsIterator(fields, fieldInfos), maxDoc);
TermIteratorQueue<FieldTerms> fieldTermsQueue = createFieldTermsQueue(fields, fieldMetadataList);
List<TermIterator<FieldTerms>> groupedFieldTerms = new ArrayList<>(fieldTermsQueue.size());
List<FieldMetadataTermState> termStates = new ArrayList<>(fieldTermsQueue.size());
while (fieldTermsQueue.size() != 0) {
TermIterator<FieldTerms> topFieldTerms = fieldTermsQueue.popTerms();
BytesRef term = BytesRef.deepCopyOf(topFieldTerms.term);
groupByTerm(fieldTermsQueue, topFieldTerms, groupedFieldTerms);
writePostingLines(term, groupedFieldTerms, normsProducer, termStates);
blockWriter.addLine(term, termStates, dictionaryBuilder);
nextTermForIterators(groupedFieldTerms, fieldTermsQueue);
}
return fieldMetadataList;
}
private List<FieldMetadata> createFieldMetadataList(Iterator<FieldInfo> fieldInfos, int maxDoc) {
List<FieldMetadata> fieldMetadataList = new ArrayList<>();
while (fieldInfos.hasNext()) {
FieldMetadata fieldMetadata = new FieldMetadata(fieldInfos.next(), maxDoc);
fieldMetadata.setDictionaryStartFP(dictionaryOutput.getFilePointer());
fieldMetadataList.add(fieldMetadata);
}
return fieldMetadataList;
}
private TermIteratorQueue<FieldTerms> createFieldTermsQueue(Fields fields, List<FieldMetadata> fieldMetadataList) throws IOException {
TermIteratorQueue<FieldTerms> fieldQueue = new TermIteratorQueue<>(fieldMetadataList.size());
for (FieldMetadata fieldMetadata : fieldMetadataList) {
Terms terms = fields.terms(fieldMetadata.getFieldInfo().name);
if (terms != null) {
FieldTerms fieldTerms = new FieldTerms(fieldMetadata, terms.iterator());
if (fieldTerms.nextTerm()) {
// There is at least one term for the field.
fieldQueue.add(fieldTerms);
}
}
}
return fieldQueue;
}
private <T> void groupByTerm(TermIteratorQueue<T> termIteratorQueue, TermIterator<T> topTermIterator, List<TermIterator<T>> groupedTermIterators) {
groupedTermIterators.clear();
groupedTermIterators.add(topTermIterator);
while (termIteratorQueue.size() != 0) {
TermIterator<T> termIterator = termIteratorQueue.top();
if (topTermIterator.term.compareTo(termIterator.term) != 0) {
return;
}
// Same term for another iterator. Combine the iterators.
groupedTermIterators.add(termIterator);
termIteratorQueue.pop();
}
}
private void writePostingLines(BytesRef term, List<? extends TermIterator<FieldTerms>> groupedFieldTerms,
NormsProducer normsProducer, List<FieldMetadataTermState> termStates) throws IOException {
termStates.clear();
for (TermIterator<FieldTerms> fieldTermIterator : groupedFieldTerms) {
FieldTerms fieldTerms = (FieldTerms) fieldTermIterator;
postingsWriter.setField(fieldTerms.fieldMetadata.getFieldInfo());
BlockTermState blockTermState = writePostingLine(fieldTerms.termsEnum, fieldTerms.fieldMetadata, normsProducer);
if (blockTermState != null) {
fieldTerms.fieldMetadata.setLastTerm(term);
termStates.add(new FieldMetadataTermState(fieldTerms.fieldMetadata, blockTermState));
}
}
}
private <T> void nextTermForIterators(List<? extends TermIterator<T>> termIterators,
TermIteratorQueue<T> termIteratorQueue) throws IOException {
for (TermIterator<T> termIterator : termIterators) {
if (termIterator.nextTerm()) {
// There is a next term for the iterator. Add it to the priority queue.
termIteratorQueue.add(termIterator);
}
}
}
private int writeFieldMetadataList(Collection<FieldMetadata> fieldMetadataList) throws IOException {
ByteBuffersDataOutput fieldsOutput = new ByteBuffersDataOutput();
int fieldsNumber = 0;
for (FieldMetadata fieldMetadata : fieldMetadataList) {
if (fieldMetadata.getNumTerms() > 0) {
fieldMetadata.write(fieldsOutput);
fieldsNumber++;
}
}
writeFieldsMetadata(fieldsNumber, fieldsOutput);
return fieldsNumber;
}
protected void writeDictionary(int fieldsNumber, IndexDictionary.Builder dictionaryBuilder) throws IOException {
if (fieldsNumber > 0) {
writeDictionary(dictionaryBuilder);
}
CodecUtil.writeFooter(dictionaryOutput);
}
@Override
public void merge(MergeState mergeState, NormsProducer normsProducer) throws IOException {
if (mergeState.needsIndexSort) {
// This custom merging does not support sorted index.
// Fall back to the default merge, which is inefficient for this postings format.
super.merge(mergeState, normsProducer);
return;
}
FieldsProducer[] fieldsProducers = mergeState.fieldsProducers;
List<TermIterator<SegmentTerms>> segmentTermsList = new ArrayList<>(fieldsProducers.length);
for (int segmentIndex = 0; segmentIndex < fieldsProducers.length; segmentIndex++) {
FieldsProducer fieldsProducer = fieldsProducers[segmentIndex];
// Iterate the FieldInfo provided by mergeState.fieldInfos because they may be
// filtered by PerFieldMergeState.
for (FieldInfo fieldInfo : mergeState.fieldInfos[segmentIndex]) {
// Iterate all fields only the get the *first* Terms instanceof STUniformSplitTerms.
// See the break below.
Terms terms = fieldsProducer.terms(fieldInfo.name);
if (terms != null) {
if (!(terms instanceof STUniformSplitTerms)) {
// Terms is not directly an instance of STUniformSplitTerms, it is wrapped/filtered.
// Fall back to the default merge, which is inefficient for this postings format.
super.merge(mergeState, normsProducer);
return;
}
STUniformSplitTerms sharedTerms = (STUniformSplitTerms) terms;
segmentTermsList.add(new SegmentTerms(
segmentIndex, sharedTerms.createMergingBlockReader(), mergeState.docMaps[segmentIndex]));
// We have the STUniformSplitTerms for the segment. Break the field
// loop to iterate the next segment.
break;
}
}
}
writeSegment((blockWriter, dictionaryBuilder) -> mergeSegments(mergeState, normsProducer, segmentTermsList, blockWriter, dictionaryBuilder));
}
private Collection<FieldMetadata> mergeSegments(MergeState mergeState, NormsProducer normsProducer,
List<TermIterator<SegmentTerms>> segmentTermsList,
STBlockWriter blockWriter, IndexDictionary.Builder dictionaryBuilder) throws IOException {
List<FieldMetadata> fieldMetadataList = createFieldMetadataList(mergeState.mergeFieldInfos.iterator(), mergeState.segmentInfo.maxDoc());
Map<String, MergingFieldTerms> fieldTermsMap = createMergingFieldTermsMap(fieldMetadataList, mergeState.fieldsProducers.length);
TermIteratorQueue<SegmentTerms> segmentTermsQueue = createSegmentTermsQueue(segmentTermsList);
List<TermIterator<SegmentTerms>> groupedSegmentTerms = new ArrayList<>(segmentTermsList.size());
Map<String, List<SegmentPostings>> fieldPostingsMap = new HashMap<>(mergeState.fieldInfos.length);
List<MergingFieldTerms> groupedFieldTerms = new ArrayList<>(mergeState.fieldInfos.length);
List<FieldMetadataTermState> termStates = new ArrayList<>(mergeState.fieldInfos.length);
while (segmentTermsQueue.size() != 0) {
TermIterator<SegmentTerms> topSegmentTerms = segmentTermsQueue.popTerms();
BytesRef term = BytesRef.deepCopyOf(topSegmentTerms.term);
groupByTerm(segmentTermsQueue, topSegmentTerms, groupedSegmentTerms);
combineSegmentsFields(groupedSegmentTerms, fieldPostingsMap);
combinePostingsPerField(term, fieldTermsMap, fieldPostingsMap, groupedFieldTerms);
writePostingLines(term, groupedFieldTerms, normsProducer, termStates);
blockWriter.addLine(term, termStates, dictionaryBuilder);
nextTermForIterators(groupedSegmentTerms, segmentTermsQueue);
}
return fieldMetadataList;
}
private Map<String, MergingFieldTerms> createMergingFieldTermsMap(List<FieldMetadata> fieldMetadataList, int numSegments) {
Map<String, MergingFieldTerms> fieldTermsMap = new HashMap<>(fieldMetadataList.size() * 2);
for (FieldMetadata fieldMetadata : fieldMetadataList) {
FieldInfo fieldInfo = fieldMetadata.getFieldInfo();
fieldTermsMap.put(fieldInfo.name, new MergingFieldTerms(fieldMetadata, new STMergingTermsEnum(fieldInfo.name, numSegments)));
}
return fieldTermsMap;
}
private TermIteratorQueue<SegmentTerms> createSegmentTermsQueue(List<TermIterator<SegmentTerms>> segmentTermsList) throws IOException {
TermIteratorQueue<SegmentTerms> segmentQueue = new TermIteratorQueue<>(segmentTermsList.size());
for (TermIterator<SegmentTerms> segmentTerms : segmentTermsList) {
if (segmentTerms.nextTerm()) {
// There is at least one term in the segment
segmentQueue.add(segmentTerms);
}
}
return segmentQueue;
}
private void combineSegmentsFields(List<TermIterator<SegmentTerms>> groupedSegmentTerms, Map<String, List<SegmentPostings>> fieldPostingsMap) {
fieldPostingsMap.clear();
for (TermIterator<SegmentTerms> segmentTermIterator : groupedSegmentTerms) {
SegmentTerms segmentTerms = (SegmentTerms) segmentTermIterator;
for (Map.Entry<String, BlockTermState> fieldTermState : segmentTerms.fieldTermStatesMap.entrySet()) {
List<SegmentPostings> segmentPostingsList = fieldPostingsMap.get(fieldTermState.getKey());
if (segmentPostingsList == null) {
segmentPostingsList = new ArrayList<>(groupedSegmentTerms.size());
fieldPostingsMap.put(fieldTermState.getKey(), segmentPostingsList);
}
segmentPostingsList.add(new SegmentPostings(segmentTerms.segmentIndex, fieldTermState.getValue(), segmentTerms.mergingBlockReader, segmentTerms.docMap));
}
}
}
private void combinePostingsPerField(BytesRef term,
Map<String, MergingFieldTerms> fieldTermsMap,
Map<String, List<SegmentPostings>> fieldPostingsMap,
List<MergingFieldTerms> groupedFieldTerms) {
groupedFieldTerms.clear();
for (Map.Entry<String, List<SegmentPostings>> fieldPostingsEntry : fieldPostingsMap.entrySet()) {
// The field defined in fieldPostingsMap comes from the FieldInfos of the SegmentReadState.
// The fieldTermsMap contains entries for fields coming from the SegmentMergeSate.
// So it is possible that the field is not present in fieldTermsMap because it is removed.
MergingFieldTerms fieldTerms = fieldTermsMap.get(fieldPostingsEntry.getKey());
if (fieldTerms != null) {
fieldTerms.resetIterator(term, fieldPostingsEntry.getValue());
groupedFieldTerms.add(fieldTerms);
}
}
// Keep the fields ordered by their number in the target merge segment.
groupedFieldTerms.sort(Comparator.comparingInt(fieldTerms -> fieldTerms.fieldMetadata.getFieldInfo().number));
}
private interface SharedTermsWriter {
Collection<FieldMetadata> writeSharedTerms(STBlockWriter blockWriter, IndexDictionary.Builder dictionaryBuilder) throws IOException;
}
protected class SegmentPostings {
final int segmentIndex;
final BlockTermState termState;
final STMergingBlockReader mergingBlockReader;
final MergeState.DocMap docMap;
SegmentPostings(int segmentIndex, BlockTermState termState, STMergingBlockReader mergingBlockReader, MergeState.DocMap docMap) {
this.segmentIndex = segmentIndex;
this.termState = termState;
this.mergingBlockReader = mergingBlockReader;
this.docMap = docMap;
}
PostingsEnum getPostings(String fieldName, PostingsEnum reuse, int flags) throws IOException {
return mergingBlockReader.postings(fieldName, termState, reuse, flags);
}
}
private class TermIteratorQueue<T> extends PriorityQueue<TermIterator<T>> {
TermIteratorQueue(int numFields) {
super(numFields);
}
@Override
protected boolean lessThan(TermIterator<T> a, TermIterator<T> b) {
return a.compareTo(b) < 0;
}
TermIterator<T> popTerms() {
TermIterator<T> topTerms = pop();
assert topTerms != null;
assert topTerms.term != null;
return topTerms;
}
}
private abstract class TermIterator<T> implements Comparable<TermIterator<T>> {
BytesRef term;
abstract boolean nextTerm() throws IOException;
@Override
public int compareTo(TermIterator<T> other) {
assert term != null : "Should not be compared when the iterator is exhausted";
int comparison = term.compareTo(other.term);
if (comparison == 0) {
return compareSecondary(other);
}
return comparison;
}
abstract int compareSecondary(TermIterator<T> other);
}
private class FieldTerms extends TermIterator<FieldTerms> {
final FieldMetadata fieldMetadata;
final TermsEnum termsEnum;
FieldTerms(FieldMetadata fieldMetadata, TermsEnum termsEnum) {
this.fieldMetadata = fieldMetadata;
this.termsEnum = termsEnum;
}
@Override
boolean nextTerm() throws IOException {
term = termsEnum.next();
return term != null;
}
@Override
int compareSecondary(TermIterator<FieldTerms> other) {
return Integer.compare(fieldMetadata.getFieldInfo().number, ((FieldTerms) other).fieldMetadata.getFieldInfo().number);
}
}
private class MergingFieldTerms extends FieldTerms {
MergingFieldTerms(FieldMetadata fieldMetadata, STMergingTermsEnum termsEnum) {
super(fieldMetadata, termsEnum);
}
void resetIterator(BytesRef term, List<SegmentPostings> segmentPostingsList) {
((STMergingTermsEnum) termsEnum).reset(term, segmentPostingsList);
}
}
private class SegmentTerms extends TermIterator<SegmentTerms> {
private final Integer segmentIndex;
private final STMergingBlockReader mergingBlockReader;
private final Map<String, BlockTermState> fieldTermStatesMap;
private final MergeState.DocMap docMap;
SegmentTerms(int segmentIndex, STMergingBlockReader mergingBlockReader, MergeState.DocMap docMap) {
this.segmentIndex = segmentIndex;
this.mergingBlockReader = mergingBlockReader;
this.docMap = docMap;
this.fieldTermStatesMap = new HashMap<>();
}
@Override
boolean nextTerm() throws IOException {
term = mergingBlockReader.next();
if (term == null) {
return false;
}
mergingBlockReader.readFieldTermStatesMap(fieldTermStatesMap);
return true;
}
@Override
int compareSecondary(TermIterator<SegmentTerms> other) {
return Integer.compare(segmentIndex, ((SegmentTerms) other).segmentIndex);
}
}
private static class FieldsIterator implements Iterator<FieldInfo> {
private final Iterator<String> fieldNames;
private final FieldInfos fieldInfos;
FieldsIterator(Fields fields, FieldInfos fieldInfos) {
this.fieldNames = fields.iterator();
this.fieldInfos = fieldInfos;
}
@Override
public boolean hasNext() {
return fieldNames.hasNext();
}
@Override
public FieldInfo next() {
return fieldInfos.fieldInfo(fieldNames.next());
}
}
}

View File

@ -0,0 +1,60 @@
/*
* 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.sharedterms;
import org.apache.lucene.codecs.uniformsplit.FieldMetadata;
import org.apache.lucene.util.BytesRef;
/**
* Builds a {@link FieldMetadata} that is the union of multiple {@link FieldMetadata}.
*
* @lucene.experimental
*/
public class UnionFieldMetadataBuilder {
private long minStartBlockFP;
private long maxEndBlockFP;
private BytesRef maxLastTerm;
public UnionFieldMetadataBuilder() {
reset();
}
public UnionFieldMetadataBuilder reset() {
maxEndBlockFP = Long.MIN_VALUE;
minStartBlockFP = Long.MAX_VALUE;
maxLastTerm = null;
return this;
}
public UnionFieldMetadataBuilder addFieldMetadata(FieldMetadata fieldMetadata) {
minStartBlockFP = Math.min(minStartBlockFP, fieldMetadata.getFirstBlockStartFP());
maxEndBlockFP = Math.max(maxEndBlockFP, fieldMetadata.getLastBlockStartFP());
if (maxLastTerm == null || maxLastTerm.compareTo(fieldMetadata.getLastTerm()) < 0) {
maxLastTerm = fieldMetadata.getLastTerm();
}
return this;
}
public FieldMetadata build() {
if (maxLastTerm == null) {
throw new IllegalStateException("no field metadata was provided");
}
return new FieldMetadata(null, 0, false, minStartBlockFP, maxEndBlockFP, maxLastTerm);
}
}

View File

@ -0,0 +1,30 @@
/*
* 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.
*/
/**
* Pluggable term index / block terms dictionary implementations.
* <p>
* Extension of {@link org.apache.lucene.codecs.uniformsplit} with Shared Terms principle:
* Terms are shared between all fields. It is particularly adapted to index a massive number of fields
* because all the terms are stored in a single FST dictionary.
* <p>
* <ul>
* <li>Designed to be extensible</li>
* <li>Highly reduced on-heap memory usage when dealing with a massive number of fields.</li>
* </ul>
*/
package org.apache.lucene.codecs.uniformsplit.sharedterms;

View File

@ -18,4 +18,5 @@ org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
org.apache.lucene.codecs.memory.DirectPostingsFormat
org.apache.lucene.codecs.memory.FSTOrdPostingsFormat
org.apache.lucene.codecs.memory.FSTPostingsFormat
org.apache.lucene.codecs.uniformsplit.UniformSplitPostingsFormat
org.apache.lucene.codecs.uniformsplit.sharedterms.STUniformSplitPostingsFormat

View File

@ -0,0 +1,31 @@
/*
* 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.lucene50;
/**
* Test utility class to create mock {@link Lucene50PostingsFormat.IntBlockTermState}.
*/
public class MockTermStateFactory {
/**
* Creates an empty {@link Lucene50PostingsFormat.IntBlockTermState}.
*/
public static Lucene50PostingsFormat.IntBlockTermState create() {
return new Lucene50PostingsFormat.IntBlockTermState();
}
}

View File

@ -0,0 +1,126 @@
/*
* 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.Collections;
import org.apache.lucene.codecs.lucene50.MockTermStateFactory;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.ByteBuffersIndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
/**
* Tests {@link BlockWriter}.
*/
public class TestBlockWriter extends LuceneTestCase {
private BlockWriter blockWriter;
private ByteBuffersIndexOutput blockOutput;
@Override
public void setUp() throws Exception {
super.setUp();
blockOutput = new ByteBuffersIndexOutput(ByteBuffersDataOutput.newResettableInstance(), "Test", "Test");
blockWriter = new BlockWriter(blockOutput, 10, 2, null);
}
public void testAddLine() throws IOException {
BytesRef term = new BytesRef("mike");
blockWriter.addLine(term, MockTermStateFactory.create(), null);
assertEquals(1, blockWriter.blockLines.size());
assertEquals(term, blockWriter.lastTerm);
}
public void testAddMultipleLinesSingleBlock() throws IOException {
String[] terms = new String[]{
"ana",
"bark",
"condor",
"dice",
"elephant"
};
for (String t : terms) {
blockWriter.addLine(new BytesRef(t), MockTermStateFactory.create(), null);
}
assertEquals(terms.length, blockWriter.blockLines.size());
assertEquals(new BytesRef(terms[terms.length - 1]), blockWriter.lastTerm);
}
public void testAddMultipleLinesMultiBlock() throws IOException {
String[] terms = new String[]{
"ana",
"bark",
"condor",
"dice",
"elephant",
"fork",
"gain",
"hyper",
"identifier",
"judge",
"ko",
"large",
};
// in order to build a block a FieldMetadata must be set
blockWriter.setField(new FieldMetadata(getMockFieldInfo("content", 0), 0));
FSTDictionary.Builder dictionaryBuilder = new FSTDictionary.Builder();
for (String t : terms) {
blockWriter.addLine(new BytesRef(t), MockTermStateFactory.create(), dictionaryBuilder);
}
//at least one block was flushed
assertTrue(blockOutput.getFilePointer() > 0);
// last term is always the last term to be writen
assertEquals(new BytesRef(terms[terms.length - 1]), blockWriter.lastTerm);
// remains 'large' to be flushed
assertEquals(1, blockWriter.blockLines.size());
blockWriter.finishLastBlock(dictionaryBuilder);
// we release memory
assertTrue(blockWriter.blockLines.isEmpty());
assertNull(blockWriter.lastTerm);
assertEquals(0, blockWriter.blockLinesWriteBuffer.size());
assertEquals(0, blockWriter.termStatesWriteBuffer.size());
}
private static FieldInfo getMockFieldInfo(String fieldName, int number) {
return new FieldInfo(fieldName,
number,
false,
false,
true,
IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
DocValuesType.NONE,
-1,
Collections.emptyMap(),
0,
0,
0,
true
);
}
}

View File

@ -0,0 +1,145 @@
/*
* 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.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
/**
* Tests {@link FSTDictionary}.
*/
public class TestFSTDictionary extends LuceneTestCase {
public void testEmptyTermSupported() {
FSTDictionary indexDictionary = createFSTDictionary(Collections.singletonList(new BytesRef()), new int[]{588});
assertEquals(588, indexDictionary.browser().seekBlock(new BytesRef()));
}
public void testRepeatedTermNotAllowed() {
for (BytesRef term : new BytesRef[] {new BytesRef(), new BytesRef("a")}) {
try {
createFSTDictionary(Arrays.asList(term, term), new int[]{0, 1});
fail("Expected exception not thrown");
} catch (Exception e) {
assertSame(UnsupportedOperationException.class, e.getClass());
}
}
}
public void testRepeatedOutputAllowed() {
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")));
assertEquals(588, indexDictionary.browser().seekBlock(new BytesRef("b")));
}
public void testSerialization() throws IOException {
List<String> vocab = Arrays.asList(
"aswoon",
"asyl",
"asyla",
"asyllabic");
for (boolean shouldEncode : new boolean[] {false, true}) {
FSTDictionary srcDictionary = createFSTDictionary(vocab);
FSTDictionary fstDictionary = serializeAndReadDictionary(srcDictionary, shouldEncode);
assertNotSame(srcDictionary, fstDictionary);
assertEquals(-1L, fstDictionary.browser().seekBlock(new BytesRef()));
assertNotSame(-1L, fstDictionary.browser().seekBlock(new BytesRef("aswoon")));
assertNotSame(-1L, fstDictionary.browser().seekBlock(new BytesRef("z")));
}
}
public void testSerializationEmptyTerm() throws IOException {
for (boolean shouldEncode : new boolean[] {false, true}) {
FSTDictionary srcDictionary = createFSTDictionary(Collections.singletonList(new BytesRef()), new int[1]);
FSTDictionary fstDictionary = serializeAndReadDictionary(srcDictionary, shouldEncode);
assertNotSame(srcDictionary, fstDictionary);
assertEquals(0, fstDictionary.browser().seekBlock(new BytesRef()));
}
}
public void testCommonPrefixes() {
List<String> vocab = new ArrayList<>();
vocab.add("aswoon");
vocab.add("asyl");
vocab.add("asyla");
vocab.add("asyllabic");
vocab.add("asylum");
vocab.add("asylums");
vocab.add("asymmetric");
vocab.add("asymmetrical");
vocab.add("asymmetrically");
vocab.add("asymmetries");
vocab.add("asymmetry");
vocab.add("asymptomatic");
vocab.add("asymptomatically");
vocab.add("asymptote");
vocab.add("asymptotes");
vocab.add("asymptotic");
vocab.add("asymptotical");
vocab.add("asymptotically");
vocab.add("asynapses");
vocab.add("asynapsis");
int[] blockFPs = new int[vocab.size()];
for (int i = 0; i < blockFPs.length; i++) {
blockFPs[i] = i;
}
List<BytesRef> blockKeys = vocab.stream().map(BytesRef::new).collect(Collectors.toList());
FSTDictionary indexDictionary = createFSTDictionary(blockKeys, blockFPs);
IndexDictionary.Browser browser = indexDictionary.browser();
for (int i = 0; i < vocab.size(); i++) {
assertEquals(blockFPs[i], browser.seekBlock(blockKeys.get(i)));
}
assertEquals(blockFPs[vocab.size() - 1], browser.seekBlock(new BytesRef("zoo")));
assertEquals(-1, browser.seekBlock(new BytesRef("A")));
assertEquals(blockFPs[9], browser.seekBlock(new BytesRef("asymmetriesz")));
}
private static FSTDictionary createFSTDictionary(List<BytesRef> blockKeys, int[] blockFPs) {
FSTDictionary.Builder builder = new FSTDictionary.Builder();
for (int i = 0; i < blockKeys.size(); i++) {
builder.add(blockKeys.get(i), blockFPs[i]);
}
return builder.build();
}
private static FSTDictionary createFSTDictionary(List<String> vocab) {
FSTDictionary.Builder builder = new FSTDictionary.Builder();
for (int i = 0; i < vocab.size(); i++) {
builder.add(new BytesRef(vocab.get(i)), i);
}
return builder.build();
}
private static FSTDictionary serializeAndReadDictionary(FSTDictionary srcDictionary, boolean shouldEncrypt) throws IOException {
ByteBuffersDataOutput output = ByteBuffersDataOutput.newResettableInstance();
srcDictionary.write(output, shouldEncrypt ? Rot13CypherTestUtil.getBlockEncoder() : null);
return FSTDictionary.read(output.toDataInput(), shouldEncrypt ? Rot13CypherTestUtil.getBlockDecoder() : null);
}
}

View File

@ -0,0 +1,217 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.codecs.uniformsplit;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
/**
* Tests {@link TermBytes}.
*/
public class TestTermBytes extends LuceneTestCase {
public void testMDPA() {
validateExpectedMDP(new String[][]{
{"aa", "a"},
{"abbreviator", "ab"},
{"abidingly", "abi"},
{"aboiteaus", "abo"},
{"abranchiates", "abr"},
{"absentminded", "abs"},
});
}
public void testIncrementalA() {
validateExpectedSuffix(new String[][]{
{"aa", "0aa"},
{"abbreviator", "1bbreviator"},
{"abidingly", "2idingly"},
{"aboiteaus", "2oiteaus"},
{"abranchiates", "2ranchiates"},
{"absentminded", "2sentminded"},
{"rodriguez", "0rodriguez"},
{"romero", "2mero"},
});
}
public void testMDPMIX2() {
validateExpectedMDP(new String[][]{
{"abaco", "a"},
{"amigo", "am"},
{"bloom", "b"},
{"break", "br"},
{"can", "c"},
{"car", "car"},
{"carmagedon", "carm"},
{"danger", "d"},
{"lala", "l"},
{"literature", "li"},
{"lucene", "lu"},
{"nature", "n"},
{"naval", "nav"},
{"rico", "r"},
{"weird", "w"},
{"zoo", "z"},
});
}
public void testMDP() {
validateExpectedMDP(new String[][]{
{"abaco", "a"},
{"amigo", "am"},
{"arco", "ar"},
{"bloom", "b"},
{"frien", "f"},
{"frienchies", "frienc"},
{"friend", "friend"},
{"friendalan", "frienda"},
{"friende", "friende"},
});
}
public void testIncremental() {
validateExpectedSuffix(new String[][]{
{"abaco", "0abaco"},
{"amigo", "1migo"},
{"arco", "1rco"},
{"bloom", "0bloom"},
{"frien", "0frien"},
{"frienchies", "5chies"},
{"friend", "5d"},
{"friendalan", "6alan"},
{"friende", "6e"},
});
}
public void testIncrementalSimple() {
validateExpectedSuffix(new String[][]{
{"abaco", "0abaco"},
{"rodriguez", "0rodriguez"},
{"roma", "2ma"},
{"romero", "3ero"},
});
}
public void testMDPSimple() {
validateExpectedMDP(new String[][]{
{"abaco", "a"},
{"rodriguez", "r"},
{"romero", "rom"},
});
}
public void testMDPMIX() {
validateExpectedMDP(new String[][]{
{"aaab", "a"},
{"arco", "ar"},
{"busqueda", "b"},
{"trabajo", "t"},
{"zufix", "z"},
{"zzfix", "zz"},
});
}
private void validateExpectedSuffix(String[][] vocab) {
Map<String, String> vocabMap = toMap(vocab);
validateExpectedSuffix(vocabMap);
validateIncrementalDecoding(vocabMap);
}
private void validateExpectedSuffix(Map<String, String> vocab) {
List<BytesRef> src = vocab.keySet().stream().sorted().map(BytesRef::new).collect(Collectors.toList());
List<TermBytes> output = compressPrefixes(src);
validateMapList(vocab,
src.stream().map(BytesRef::utf8ToString).collect(Collectors.toList()),
output.stream().map(e -> e.getSuffixOffset() + createSuffixBytes(e).utf8ToString()).collect(Collectors.toList()));
}
private BytesRef createSuffixBytes(TermBytes termBytes) {
return new BytesRef(termBytes.getTerm().bytes, termBytes.getSuffixOffset(), termBytes.getSuffixLength());
}
private void validateExpectedMDP(String[][] vocab) {
Map<String, String> vocabMap = toMap(vocab);
validateExpectedMDP(vocabMap);
validateIncrementalDecoding(vocabMap);
}
private void validateExpectedMDP(Map<String, String> vocab) {
List<BytesRef> src = vocab.keySet().stream().sorted().map(BytesRef::new).collect(Collectors.toList());
List<TermBytes> output = compressPrefixes(src);
validateMapList(vocab,
src.stream().map(BytesRef::utf8ToString).collect(Collectors.toList()),
output.stream().map(e -> new BytesRef(e.getTerm().bytes, 0, e.getMdpLength()).utf8ToString())
.collect(Collectors.toList()));
}
private void validateIncrementalDecoding(Map<String, String> vocab) {
BytesRef previous = new BytesRef(80);
List<BytesRef> src = vocab.keySet().stream().sorted().map(BytesRef::new).collect(Collectors.toList());
List<TermBytes> output = compressPrefixes(src);
for (int i = 0; i < src.size(); i++) {
copyBytes(BytesRef.deepCopyOf(createSuffixBytes(output.get(i))), previous, output.get(i).getSuffixOffset());
assertEquals("Error in line " + i, src.get(i).utf8ToString(), previous.utf8ToString());
}
}
private void validateMapList(Map<String, String> expectedMap, List<String> src, List<String> result) {
for (int i = 0; i < src.size(); i++) {
assertEquals("Error in line " + i, expectedMap.get(src.get(i)), result.get(i));
}
}
private static List<TermBytes> compressPrefixes(List<BytesRef> vocab) {
List<TermBytes> termBytes = new ArrayList<>(vocab.size());
BytesRef last = null;
TermBytes term;
int mdp;
for (BytesRef current : vocab) {
mdp = TermBytes.computeMdpLength(last, current);
term = new TermBytes(mdp, current);
termBytes.add(term);
last = current;
}
return termBytes;
}
private static void copyBytes(BytesRef source, BytesRef target, int targetOffset) {
assert target.offset == 0;
assert source.offset == 0;
int newLength = targetOffset + source.length;
if (newLength > target.bytes.length) {
byte[] copy = new byte[newLength];
System.arraycopy(target.bytes, 0, copy, 0, targetOffset);
target.bytes = copy;
}
target.length = newLength;
System.arraycopy(source.bytes, 0, target.bytes, targetOffset, source.length);
}
private static Map<String, String> toMap(String[][] src) {
assert src.length > 0 : "insert at least one row";
assert src[0].length == 2 : "two columns are mandatory";
return Arrays.stream(src).collect(Collectors.toMap(kv -> kv[0], kv -> kv[1]));
}
}

View File

@ -0,0 +1,189 @@
/*
* 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.ArrayList;
import java.util.List;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteBuffersDirectory;
import org.apache.lucene.store.DataInput;
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.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
/**
* Tests the {@link TermBytes} comparator.
*/
public class TestTermBytesComparator extends LuceneTestCase {
public void testComparison() throws IOException {
TermBytes[] vocab = new TermBytes[]{
termBytes(1, "abaco"),
termBytes(2, "amiga"),
termBytes(5, "amigo"),
termBytes(2, "arco"),
termBytes(1, "bloom"),
termBytes(1, "frien"),
termBytes(6, "frienchies"),
termBytes(6, "friend"),
termBytes(7, "friendalan"),
termBytes(7, "friende"),
termBytes(8, "friendez"),
};
List<BlockLine> lines = generateBlockLines(vocab);
Directory directory = new ByteBuffersDirectory();
try (IndexOutput indexOutput = directory.createOutput("temp.bin", IOContext.DEFAULT)) {
indexOutput.writeVInt(5);
}
MockBlockReader blockReader = new MockBlockReader(lines, directory);
assertAlwaysGreater(blockReader, new BytesRef("z"));
assertGreaterUntil(1, blockReader, new BytesRef("abacu"));
assertGreaterUntil(4, blockReader, new BytesRef("bar"));
assertGreaterUntil(2, blockReader, new BytesRef("amigas"));
assertGreaterUntil(10, blockReader, new BytesRef("friendez"));
}
private TermsEnum.SeekStatus assertGreaterUntil(int expectedPosition, MockBlockReader blockReader, BytesRef lookedTerm) throws IOException {
TermsEnum.SeekStatus seekStatus = blockReader.seekInBlock(lookedTerm);
assertEquals("looked Term: " + lookedTerm.utf8ToString(), expectedPosition, blockReader.lineIndexInBlock - 1);
//reset the state
blockReader.reset();
return seekStatus;
}
private void assertAlwaysGreater(MockBlockReader blockReader, BytesRef lookedTerm) throws IOException {
TermsEnum.SeekStatus seekStatus = assertGreaterUntil(-1, blockReader, lookedTerm);
assertEquals(TermsEnum.SeekStatus.END, seekStatus);
}
private List<BlockLine> generateBlockLines(TermBytes[] words) {
List<BlockLine> lines = new ArrayList<>(words.length);
for (TermBytes word : words) {
lines.add(new BlockLine(word, null));
}
return lines;
}
class MockBlockReader extends BlockReader {
private List<BlockLine> lines;
MockBlockReader(List<BlockLine> lines, Directory directory) throws IOException {
super(null, directory.openInput("temp.bin", IOContext.DEFAULT),
createMockPostingReaderBase(), new FieldMetadata(null, 1), null);
this.lines = lines;
}
@Override
protected int compareToMiddleAndJump(BytesRef searchedTerm) {
// Do not jump in test.
return -1;
}
@Override
protected BlockLine readLineInBlock() {
if (lineIndexInBlock >= lines.size()) {
lineIndexInBlock = 0;
return blockLine = null;
}
return blockLine = lines.get(lineIndexInBlock++);
}
@Override
protected void initializeHeader(BytesRef searchedTerm, long targetBlockStartFP) throws IOException {
// Force blockStartFP to an impossible value so we never trigger the optimization
// that keeps the current block with our mock block reader.
blockStartFP = Long.MIN_VALUE;
super.initializeHeader(searchedTerm, targetBlockStartFP);
}
@Override
protected BlockHeader readHeader() {
return blockHeader = lineIndexInBlock >= lines.size() ? null : new BlockHeader(lines.size(), 0, 0, 0, 0, 0);
}
void reset() {
lineIndexInBlock = 0;
blockHeader = null;
blockLine = null;
}
}
private static TermBytes termBytes(int mdpLength, String term) {
return new TermBytes(mdpLength, new BytesRef(term));
}
private static PostingsReaderBase createMockPostingReaderBase() {
return new PostingsReaderBase() {
@Override
public void init(IndexInput termsIn, SegmentReadState state) {
}
@Override
public BlockTermState newTermState() {
return null;
}
@Override
public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState state, boolean absolute) {
}
@Override
public PostingsEnum postings(FieldInfo fieldInfo, BlockTermState state, PostingsEnum reuse, int flags) {
return null;
}
@Override
public ImpactsEnum impacts(FieldInfo fieldInfo, BlockTermState state, int flags) {
return null;
}
@Override
public void checkIntegrity() {
}
@Override
public void close() {
}
@Override
public long ramBytesUsed() {
return 0;
}
};
}
}

View File

@ -0,0 +1,59 @@
/*
* 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 org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util.TestUtil;
import org.junit.After;
import org.junit.Before;
/**
* Tests {@link UniformSplitPostingsFormat} with block encoding using ROT13 cypher.
*/
public class TestUniformSplitPostingFormat extends BasePostingsFormatTestCase {
private final Codec codec = TestUtil.alwaysPostingsFormat(new UniformSplitRot13PostingsFormat());
private boolean shouldCheckDecoderWasCalled = true;
@Override
protected Codec getCodec() {
return codec;
}
@Before
public void initialize() {
UniformSplitRot13PostingsFormat.resetEncodingFlags();
}
@After
public void checkEncodingCalled() {
assertTrue(UniformSplitRot13PostingsFormat.blocksEncoded);
assertTrue(UniformSplitRot13PostingsFormat.dictionaryEncoded);
if (shouldCheckDecoderWasCalled) {
assertTrue(UniformSplitRot13PostingsFormat.decoderCalled);
}
}
@Override
public void testRandomExceptions() throws Exception {
shouldCheckDecoderWasCalled = false;
super.testRandomExceptions();
}
}

View File

@ -0,0 +1,352 @@
/*
* 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.sharedterms;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
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;
import org.apache.lucene.codecs.lucene50.MockTermStateFactory;
import org.apache.lucene.codecs.uniformsplit.BlockHeader;
import org.apache.lucene.codecs.uniformsplit.BlockLine;
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.TermBytes;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.ByteBuffersDirectory;
import org.apache.lucene.store.DataInput;
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.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
public class STBlockReaderTest extends LuceneTestCase {
private static final String MOCK_BLOCK_OUTPUT_NAME = "STBlockReaderTest.tmp";
private FieldInfos fieldInfos;
private List<MockSTBlockLine> blockLines;
private Supplier<IndexDictionary.Browser> supplier;
private ByteBuffersDirectory directory;
@Override
public void setUp() throws Exception {
super.setUp();
fieldInfos = mockFieldInfos();
List<BlockLineDefinition> vocab = new ArrayList<>();
vocab.add(blockLineDef(1, "abaco", "f1", "f3"));
vocab.add(blockLineDef(2, "amiga", "f1", "f2", "f4"));
vocab.add(blockLineDef(5, "amigo", "f1", "f2", "f3", "f4"));
vocab.add(blockLineDef(2, "arco", "f1"));
vocab.add(blockLineDef(1, "bloom", "f2"));
vocab.add(blockLineDef(1, "frien", "f2"));
vocab.add(blockLineDef(6, "frienchies", "f3"));
blockLines = generateBlockLines(vocab);
directory = new ByteBuffersDirectory();
try (IndexOutput blockOutput = directory.createOutput(MOCK_BLOCK_OUTPUT_NAME, IOContext.DEFAULT)) {
blockOutput.writeVInt(5);
}
IndexDictionary.Builder builder = new FSTDictionary.Builder();
builder.add(new BytesRef("a"), 0);
IndexDictionary indexDictionary = builder.build();
supplier = indexDictionary::browser;
}
@Override
public void tearDown() throws Exception {
try {
blockLines.clear();
directory.close();
} finally {
super.tearDown();
}
}
public void testSeekExactIgnoreFieldF1() throws IOException {
// when block reader for field 1 -> f1
MockSTBlockReader blockReader = new MockSTBlockReader(
supplier,
blockLines,
directory,
fieldInfos.fieldInfo("f1"), //last term "arco"
fieldInfos
);
// when seekCeil
blockReader.seekCeil(new BytesRef("arco2"));
// then
assertNull(blockReader.term());
// when seekCeilIgnoreField
blockReader.seekCeilIgnoreField(new BytesRef("arco2"));
// then
assertEquals("bloom", blockReader.term().utf8ToString());
}
public void testSeekExactIgnoreFieldF2() throws IOException {
MockSTBlockReader blockReader = new MockSTBlockReader(
supplier,
blockLines,
directory,
fieldInfos.fieldInfo("f2"),//last term "frien"
fieldInfos
);
// when seekCeil
blockReader.seekCeilIgnoreField(new BytesRef("arco2"));
// then
assertEquals("bloom", blockReader.term().utf8ToString());
}
public void testSeekExactIgnoreFieldF3() throws IOException {
MockSTBlockReader blockReader = new MockSTBlockReader(
supplier,
blockLines,
directory,
fieldInfos.fieldInfo("f3"),//last term "frienchies"
fieldInfos
);
// when seekCeilIgnoreField
blockReader.seekCeilIgnoreField(new BytesRef("arco2"));
// then
assertEquals("bloom", blockReader.term().utf8ToString());
// when seekCeil
blockReader.seekCeil(new BytesRef("arco2"));
// then
assertEquals("frienchies", blockReader.term().utf8ToString());
}
public void testSeekExactIgnoreFieldF4() throws IOException {
MockSTBlockReader blockReader = new MockSTBlockReader(
supplier,
blockLines,
directory,
fieldInfos.fieldInfo("f4"),//last term "amigo"
fieldInfos
);
// when seekCeilIgnoreField
blockReader.seekCeilIgnoreField(new BytesRef("abaco"));
// then
assertEquals("abaco", blockReader.term().utf8ToString());
// when seekCeil
blockReader.seekCeil(new BytesRef("abaco"));
// then
assertEquals("amiga", blockReader.term().utf8ToString());
}
private static FieldInfos mockFieldInfos() {
return new FieldInfos(
new FieldInfo[]{
mockFieldInfo("f1", 0),
mockFieldInfo("f2", 1),
mockFieldInfo("f3", 2),
mockFieldInfo("f4", 3),
});
}
private static FieldInfo mockFieldInfo(String fieldName, int number) {
return new FieldInfo(fieldName,
number,
false,
false,
true,
IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
DocValuesType.NONE,
-1,
Collections.emptyMap(),
0,
0,
0,
false
);
}
private BlockLineDefinition blockLineDef(int mdpLength, String term, String... fields) {
return new BlockLineDefinition(new TermBytes(mdpLength, new BytesRef(term)), Arrays.asList(fields));
}
private static List<MockSTBlockLine> generateBlockLines(Iterable<BlockLineDefinition> blockLineDefinitions) {
List<MockSTBlockLine> lines = new ArrayList<>();
for (BlockLineDefinition blockLineDefinition : blockLineDefinitions) {
lines.add(new MockSTBlockLine(blockLineDefinition.termBytes, blockLineDefinition.fields));
}
return lines;
}
private static class BlockLineDefinition {
final TermBytes termBytes;
final List<String> fields;
BlockLineDefinition(TermBytes termBytes, List<String> fields) {
this.termBytes = termBytes;
this.fields = fields;
}
}
private static class MockSTBlockLine extends STBlockLine {
final Map<String, BlockTermState> termStates;
MockSTBlockLine(TermBytes termBytes, List<String> fields) {
super(termBytes, Collections.singletonList(new FieldMetadataTermState(null, null)));
this.termStates = new HashMap<>();
for (String field : fields) {
termStates.put(field, MockTermStateFactory.create());
}
}
Set<String> getFields() {
return termStates.keySet();
}
}
private static class MockSTBlockReader extends STBlockReader {
List<MockSTBlockLine> lines;
MockSTBlockReader(Supplier<IndexDictionary.Browser> 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;
}
static PostingsReaderBase getMockPostingReaderBase() {
return new PostingsReaderBase() {
@Override
public void init(IndexInput termsIn, SegmentReadState state) {
}
@Override
public BlockTermState newTermState() {
return null;
}
@Override
public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState state, boolean absolute) {
}
@Override
public PostingsEnum postings(FieldInfo fieldInfo, BlockTermState state, PostingsEnum reuse, int flags) {
return null;
}
@Override
public ImpactsEnum impacts(FieldInfo fieldInfo, BlockTermState state, int flags) {
return null;
}
@Override
public void checkIntegrity() {
}
@Override
public void close() {
}
@Override
public long ramBytesUsed() {
return 0;
}
};
}
static FieldMetadata mockFieldMetadata(FieldInfo fieldInfo, BytesRef lastTerm) {
FieldMetadata fieldMetadata = new FieldMetadata(fieldInfo, 1);
fieldMetadata.setLastTerm(lastTerm);
fieldMetadata.setLastBlockStartFP(1);
return fieldMetadata;
}
static BytesRef getLastTermForField(List<MockSTBlockLine> lines, String fieldName) {
BytesRef lastTerm = null;
for (MockSTBlockLine line : lines) {
if (line.getFields().contains(fieldName)) {
lastTerm = line.getTermBytes().getTerm();
}
}
return lastTerm;
}
@Override
protected BlockTermState readTermState() {
return termState = lines.get(lineIndexInBlock - 1).termStates.get(fieldMetadata.getFieldInfo().name);
}
@Override
protected int compareToMiddleAndJump(BytesRef searchedTerm) {
blockLine = lines.get(lines.size() >> 1);
lineIndexInBlock = blockHeader.getMiddleLineIndex();
int compare = searchedTerm.compareTo(term());
if (compare < 0) {
lineIndexInBlock = 0;
}
return compare;
}
@Override
protected BlockLine readLineInBlock() {
if (lineIndexInBlock >= lines.size()) {
return blockLine = null;
}
return blockLine = lines.get(lineIndexInBlock++);
}
@Override
protected void initializeHeader(BytesRef searchedTerm, long startBlockLinePos) throws IOException {
// Force blockStartFP to an impossible value so we never trigger the optimization
// that keeps the current block with our mock block reader.
blockStartFP = -1;
super.initializeHeader(searchedTerm, startBlockLinePos);
}
@Override
protected BlockHeader readHeader() {
return blockHeader = lineIndexInBlock >= lines.size() ? null : new MockBlockHeader(lines.size());
}
}
private static class MockBlockHeader extends BlockHeader {
MockBlockHeader(int linesCount) {
super(linesCount, 0, 0, 0, 1, 0);
}
}
}

View File

@ -0,0 +1,35 @@
/*
* 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.sharedterms;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.uniformsplit.TestUniformSplitPostingFormat;
import org.apache.lucene.util.TestUtil;
/**
* Tests {@link STUniformSplitPostingsFormat} with block encoding using ROT13 cypher.
*/
public class TestSTUniformSplitPostingFormat extends TestUniformSplitPostingFormat {
private final Codec codec = TestUtil.alwaysPostingsFormat(new STUniformSplitRot13PostingsFormat());
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -0,0 +1,72 @@
/*
* 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 org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.BytesRef;
/**
* Test utility for simple ROT13 cipher (https://en.wikipedia.org/wiki/ROT13).
*/
public class Rot13CypherTestUtil {
private static final int ENCODING_OFFSET = 7;
private static final int ENCODING_ROTATION = 13;
public static byte[] encode(DataInput bytesInput, int length) throws IOException {
byte[] encodedBytes = new byte[length + ENCODING_OFFSET];
for (int i = 0; i < length; i++) {
encodedBytes[i + ENCODING_OFFSET] = (byte)(bytesInput.readByte() + ENCODING_ROTATION);
}
return encodedBytes;
}
public static byte[] decode(DataInput bytesInput, long length) throws IOException {
length -= ENCODING_OFFSET;
bytesInput.skipBytes(ENCODING_OFFSET);
byte[] decodedBytes = new byte[Math.toIntExact(length)];
for (int i = 0; i < length; i++) {
decodedBytes[i] = (byte)(bytesInput.readByte() - ENCODING_ROTATION);
}
return decodedBytes;
}
public static BlockEncoder getBlockEncoder() {
return (blockBytes, length) -> {
byte[] encodedBytes = Rot13CypherTestUtil.encode(blockBytes, Math.toIntExact(length));
return new BlockEncoder.WritableBytes() {
@Override
public long size() {
return encodedBytes.length;
}
@Override
public void writeTo(DataOutput dataOutput) throws IOException {
dataOutput.writeBytes(encodedBytes, 0, encodedBytes.length);
}
};
};
}
public static BlockDecoder getBlockDecoder() {
return (blockBytes, length) -> new BytesRef(Rot13CypherTestUtil.decode(blockBytes, length));
}
}

View File

@ -0,0 +1,147 @@
/*
* 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 org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/**
* {@link UniformSplitPostingsFormat} with block encoding using ROT13 cypher.
*/
public class UniformSplitRot13PostingsFormat extends PostingsFormat {
public static volatile boolean encoderCalled;
public static volatile boolean decoderCalled;
public static volatile boolean blocksEncoded;
public static volatile boolean dictionaryEncoded;
public UniformSplitRot13PostingsFormat() {
this("UniformSplitRot13");
}
protected UniformSplitRot13PostingsFormat(String name) {
super(name);
}
public static void resetEncodingFlags() {
encoderCalled = false;
decoderCalled = false;
blocksEncoded = false;
dictionaryEncoded = false;
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState segmentWriteState) throws IOException {
PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(segmentWriteState);
boolean success = false;
try {
FieldsConsumer fieldsConsumer = createFieldsConsumer(segmentWriteState, postingsWriter);
success = true;
return fieldsConsumer;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsWriter);
}
}
}
protected FieldsConsumer createFieldsConsumer(SegmentWriteState segmentWriteState, PostingsWriterBase postingsWriter) throws IOException {
return new UniformSplitTermsWriter(postingsWriter, segmentWriteState,
UniformSplitTermsWriter.DEFAULT_TARGET_NUM_BLOCK_LINES,
UniformSplitTermsWriter.DEFAULT_DELTA_NUM_LINES,
getBlockEncoder()
) {
@Override
protected void writeDictionary(IndexDictionary.Builder dictionaryBuilder) throws IOException {
recordBlockEncodingCall();
super.writeDictionary(dictionaryBuilder);
recordDictionaryEncodingCall();
}
};
}
protected void recordBlockEncodingCall() {
if (encoderCalled) {
blocksEncoded = true;
encoderCalled = false;
}
}
protected void recordDictionaryEncodingCall() {
if (encoderCalled) {
dictionaryEncoded = true;
encoderCalled = false;
}
}
protected BlockEncoder getBlockEncoder() {
return (blockBytes, length) -> {
byte[] encodedBytes = Rot13CypherTestUtil.encode(blockBytes, Math.toIntExact(length));
return new BlockEncoder.WritableBytes() {
@Override
public long size() {
return encodedBytes.length;
}
@Override
public void writeTo(DataOutput dataOutput) throws IOException {
encoderCalled = true;
dataOutput.writeBytes(encodedBytes, 0, encodedBytes.length);
}
};
};
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState segmentReadState) throws IOException {
PostingsReaderBase postingsReader = new Lucene50PostingsReader(segmentReadState);
boolean success = false;
try {
FieldsProducer fieldsProducer = createFieldsProducer(segmentReadState, postingsReader);
success = true;
return fieldsProducer;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsReader);
}
}
}
protected FieldsProducer createFieldsProducer(SegmentReadState segmentReadState, PostingsReaderBase postingsReader) throws IOException {
return new UniformSplitTermsReader(postingsReader, segmentReadState, getBlockDecoder());
}
protected BlockDecoder getBlockDecoder() {
return (blockBytes, length) -> {
decoderCalled = true;
return new BytesRef(Rot13CypherTestUtil.decode(blockBytes, length));
};
}
}

View File

@ -0,0 +1,43 @@
<!--
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.
-->
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<!-- not a package-info.java, because we already defined this package in codecs/ -->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Test PostingsFormat for testing {@link org.apache.lucene.codecs.uniformsplit} with block encryption using basic ROT13.
</body>
</html>

View File

@ -0,0 +1,59 @@
/*
* 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.sharedterms;
import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.uniformsplit.IndexDictionary;
import org.apache.lucene.codecs.uniformsplit.UniformSplitRot13PostingsFormat;
import org.apache.lucene.codecs.uniformsplit.UniformSplitTermsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/**
* {@link STUniformSplitPostingsFormat} with block encoding using ROT13 cypher.
*/
public class STUniformSplitRot13PostingsFormat extends UniformSplitRot13PostingsFormat {
public STUniformSplitRot13PostingsFormat() {
super("STUniformSplitRot13");
}
protected FieldsConsumer createFieldsConsumer(SegmentWriteState segmentWriteState, PostingsWriterBase postingsWriter) throws IOException {
return new STUniformSplitTermsWriter(postingsWriter, segmentWriteState,
UniformSplitTermsWriter.DEFAULT_TARGET_NUM_BLOCK_LINES,
UniformSplitTermsWriter.DEFAULT_DELTA_NUM_LINES,
getBlockEncoder()
) {
@Override
protected void writeDictionary(IndexDictionary.Builder dictionaryBuilder) throws IOException {
recordBlockEncodingCall();
super.writeDictionary(dictionaryBuilder);
recordDictionaryEncodingCall();
}
};
}
protected FieldsProducer createFieldsProducer(SegmentReadState segmentReadState, PostingsReaderBase postingsReader) throws IOException {
return new STUniformSplitTermsReader(postingsReader, segmentReadState, getBlockDecoder());
}
}

View File

@ -0,0 +1,43 @@
<!--
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.
-->
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<!-- not a package-info.java, because we already defined this package in codecs/uniformsplit/ -->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Test PostingsFormat for testing {@link org.apache.lucene.codecs.uniformsplit.sharedterms} with block encryption using basic ROT13.
</body>
</html>

View File

@ -20,3 +20,5 @@ org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval
org.apache.lucene.codecs.blockterms.LuceneVarGapDocFreqInterval
org.apache.lucene.codecs.bloom.TestBloomFilteredLucenePostings
org.apache.lucene.codecs.asserting.AssertingPostingsFormat
org.apache.lucene.codecs.uniformsplit.UniformSplitRot13PostingsFormat
org.apache.lucene.codecs.uniformsplit.sharedterms.STUniformSplitRot13PostingsFormat