Merge remote-tracking branch 'origin/master' into gradle-master

This commit is contained in:
Dawid Weiss 2020-01-02 10:49:19 +01:00
commit 1a690d95ad
23 changed files with 497 additions and 250 deletions

View File

@ -83,7 +83,10 @@ Other
API Changes API Changes
--------------------- ---------------------
(No changes)
* LUCENE-9093: Not an API change but a change in behavior of the UnifiedHighlighter's LengthGoalBreakIterator that will
yield Passages sized a little different due to the fact that the sizing pivot is now the center of the first match and
not its left edge.
New Features New Features
--------------------- ---------------------
@ -104,6 +107,12 @@ Improvements
* LUCENE-9105: UniformSplit postings format detects corrupted index and better handles IO exceptions. (Bruno Roustant) * LUCENE-9105: UniformSplit postings format detects corrupted index and better handles IO exceptions. (Bruno Roustant)
* LUCENE-9106: UniformSplit postings format allows extension of block/line serializers. (Bruno Roustant)
* LUCENE-9093: UnifiedHighlighter's LengthGoalBreakIterator has a new fragmentAlignment option to better center the
first match in the passage. Also the sizing point now pivots at the center of the first match term and not its left
edge. This yields Passages that won't be identical to the previous behavior. (Nándor Mátravölgyi, David Smiley)
Optimizations Optimizations
--------------------- ---------------------
(No changes) (No changes)

View File

@ -134,43 +134,49 @@ public class BlockHeader implements Accountable {
return basePayloadsFP; 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();
if (linesCount <= 0 || linesCount > UniformSplitTermsWriter.MAX_NUM_BLOCK_LINES) {
throw new CorruptIndexException("Illegal number of lines in a block: " + linesCount, input);
}
long baseDocsFP = input.readVLong();
long basePositionsFP = input.readVLong();
long basePayloadsFP = input.readVLong();
int termStatesBaseOffset = input.readVInt();
if (termStatesBaseOffset < 0) {
throw new CorruptIndexException("Illegal termStatesBaseOffset= " + termStatesBaseOffset, input);
}
int middleTermOffset = input.readVInt();
if (middleTermOffset < 0) {
throw new CorruptIndexException("Illegal middleTermOffset= " + middleTermOffset, input);
}
BlockHeader blockHeader = reuse == null ? new BlockHeader() : reuse;
return blockHeader.reset(linesCount, baseDocsFP, basePositionsFP, basePayloadsFP, termStatesBaseOffset, middleTermOffset);
}
@Override @Override
public long ramBytesUsed() { public long ramBytesUsed() {
return RAM_USAGE; return RAM_USAGE;
} }
/**
* Reads/writes block header.
*/
public static class Serializer {
public void write(DataOutput output, BlockHeader blockHeader) throws IOException {
assert blockHeader.linesCount > 0 : "Block header is not initialized";
output.writeVInt(blockHeader.linesCount);
output.writeVLong(blockHeader.baseDocsFP);
output.writeVLong(blockHeader.basePositionsFP);
output.writeVLong(blockHeader.basePayloadsFP);
output.writeVInt(blockHeader.termStatesBaseOffset);
output.writeVInt(blockHeader.middleLineOffset);
}
public BlockHeader read(DataInput input, BlockHeader reuse) throws IOException {
int linesCount = input.readVInt();
if (linesCount <= 0 || linesCount > UniformSplitTermsWriter.MAX_NUM_BLOCK_LINES) {
throw new CorruptIndexException("Illegal number of lines in block: " + linesCount, input);
}
long baseDocsFP = input.readVLong();
long basePositionsFP = input.readVLong();
long basePayloadsFP = input.readVLong();
int termStatesBaseOffset = input.readVInt();
if (termStatesBaseOffset < 0) {
throw new CorruptIndexException("Illegal termStatesBaseOffset= " + termStatesBaseOffset, input);
}
int middleTermOffset = input.readVInt();
if (middleTermOffset < 0) {
throw new CorruptIndexException("Illegal middleTermOffset= " + middleTermOffset, input);
}
BlockHeader blockHeader = reuse == null ? new BlockHeader() : reuse;
return blockHeader.reset(linesCount, baseDocsFP, basePositionsFP, basePayloadsFP, termStatesBaseOffset, middleTermOffset);
}
}
} }

View File

@ -107,7 +107,7 @@ public class BlockLine implements Accountable {
} }
/** /**
* Reads block lines with terms encoded incrementally inside a block. * Reads/writes block lines with terms encoded incrementally inside a block.
* This class keeps a state of the previous term read to decode the next term. * This class keeps a state of the previous term read to decode the next term.
*/ */
public static class Serializer implements Accountable { public static class Serializer implements Accountable {
@ -149,7 +149,7 @@ public class BlockLine implements Accountable {
* the incremental encoding. {@code true} for the first * the incremental encoding. {@code true} for the first
* and middle term, {@code false} for other terms. * and middle term, {@code false} for other terms.
*/ */
public static void writeLine(DataOutput blockOutput, BlockLine line, BlockLine previousLine, public void writeLine(DataOutput blockOutput, BlockLine line, BlockLine previousLine,
int termStateRelativeOffset, boolean isIncrementalEncodingSeed) throws IOException { int termStateRelativeOffset, boolean isIncrementalEncodingSeed) throws IOException {
blockOutput.writeVInt(termStateRelativeOffset); blockOutput.writeVInt(termStateRelativeOffset);
writeIncrementallyEncodedTerm(line.getTermBytes(), previousLine == null ? null : previousLine.getTermBytes(), writeIncrementallyEncodedTerm(line.getTermBytes(), previousLine == null ? null : previousLine.getTermBytes(),
@ -161,13 +161,13 @@ public class BlockLine implements Accountable {
* *
* @param termStatesOutput The output pointing to the details region. * @param termStatesOutput The output pointing to the details region.
*/ */
protected static void writeLineTermState(DataOutput termStatesOutput, BlockLine line, protected void writeLineTermState(DataOutput termStatesOutput, BlockLine line,
FieldInfo fieldInfo, DeltaBaseTermStateSerializer encoder) throws IOException { FieldInfo fieldInfo, DeltaBaseTermStateSerializer encoder) throws IOException {
assert line.termState != null; assert line.termState != null;
encoder.writeTermState(termStatesOutput, fieldInfo, line.termState); encoder.writeTermState(termStatesOutput, fieldInfo, line.termState);
} }
protected static void writeIncrementallyEncodedTerm(TermBytes termBytes, TermBytes previousTermBytes, protected void writeIncrementallyEncodedTerm(TermBytes termBytes, TermBytes previousTermBytes,
boolean isIncrementalEncodingSeed, DataOutput blockOutput) throws IOException { boolean isIncrementalEncodingSeed, DataOutput blockOutput) throws IOException {
BytesRef term = termBytes.getTerm(); BytesRef term = termBytes.getTerm();
assert term.offset == 0; assert term.offset == 0;
@ -240,7 +240,7 @@ public class BlockLine implements Accountable {
* Reads {@code length} bytes from the given {@link DataInput} and stores * Reads {@code length} bytes from the given {@link DataInput} and stores
* them at {@code offset} in {@code bytes.bytes}. * them at {@code offset} in {@code bytes.bytes}.
*/ */
protected static void readBytes(DataInput input, BytesRef bytes, int offset, int length) throws IOException { protected void readBytes(DataInput input, BytesRef bytes, int offset, int length) throws IOException {
assert bytes.offset == 0; assert bytes.offset == 0;
bytes.length = offset + length; bytes.length = offset + length;
bytes.bytes = ArrayUtil.grow(bytes.bytes, bytes.length); bytes.bytes = ArrayUtil.grow(bytes.bytes, bytes.length);

View File

@ -60,6 +60,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
protected final FieldMetadata fieldMetadata; protected final FieldMetadata fieldMetadata;
protected final BlockDecoder blockDecoder; protected final BlockDecoder blockDecoder;
protected BlockHeader.Serializer blockHeaderReader;
protected BlockLine.Serializer blockLineReader; protected BlockLine.Serializer blockLineReader;
/** /**
* In-memory read buffer for the current block. * In-memory read buffer for the current block.
@ -406,14 +407,27 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
protected void initializeBlockReadLazily() throws IOException { protected void initializeBlockReadLazily() throws IOException {
if (blockStartFP == -1) { if (blockStartFP == -1) {
blockInput = blockInput.clone(); blockInput = blockInput.clone();
blockLineReader = new BlockLine.Serializer(); blockHeaderReader = createBlockHeaderSerializer();
blockLineReader = createBlockLineSerializer();
blockReadBuffer = new ByteArrayDataInput(); blockReadBuffer = new ByteArrayDataInput();
termStatesReadBuffer = new ByteArrayDataInput(); termStatesReadBuffer = new ByteArrayDataInput();
termStateSerializer = new DeltaBaseTermStateSerializer(); termStateSerializer = createDeltaBaseTermStateSerializer();
scratchBlockBytes = new BytesRef(); scratchBlockBytes = new BytesRef();
} }
} }
protected BlockHeader.Serializer createBlockHeaderSerializer() {
return new BlockHeader.Serializer();
}
protected BlockLine.Serializer createBlockLineSerializer() {
return new BlockLine.Serializer();
}
protected DeltaBaseTermStateSerializer createDeltaBaseTermStateSerializer() {
return new DeltaBaseTermStateSerializer();
}
/** /**
* Reads the block header. * Reads the block header.
* Sets {@link #blockHeader}. * Sets {@link #blockHeader}.
@ -428,7 +442,7 @@ public class BlockReader extends BaseTermsEnum implements Accountable {
BytesRef blockBytesRef = decodeBlockBytesIfNeeded(numBlockBytes); BytesRef blockBytesRef = decodeBlockBytesIfNeeded(numBlockBytes);
blockReadBuffer.reset(blockBytesRef.bytes, blockBytesRef.offset, blockBytesRef.length); blockReadBuffer.reset(blockBytesRef.bytes, blockBytesRef.offset, blockBytesRef.length);
termStatesReadBuffer.reset(blockBytesRef.bytes, blockBytesRef.offset, blockBytesRef.length); termStatesReadBuffer.reset(blockBytesRef.bytes, blockBytesRef.offset, blockBytesRef.length);
return blockHeader = BlockHeader.read(blockReadBuffer, blockHeader); return blockHeader = blockHeaderReader.read(blockReadBuffer, blockHeader);
} }
protected BytesRef decodeBlockBytesIfNeeded(int numBlockBytes) throws IOException { protected BytesRef decodeBlockBytesIfNeeded(int numBlockBytes) throws IOException {

View File

@ -60,6 +60,8 @@ public class BlockWriter {
protected final ByteBuffersDataOutput blockLinesWriteBuffer; protected final ByteBuffersDataOutput blockLinesWriteBuffer;
protected final ByteBuffersDataOutput termStatesWriteBuffer; protected final ByteBuffersDataOutput termStatesWriteBuffer;
protected final BlockHeader.Serializer blockHeaderWriter;
protected final BlockLine.Serializer blockLineWriter;
protected final DeltaBaseTermStateSerializer termStateSerializer; protected final DeltaBaseTermStateSerializer termStateSerializer;
protected final BlockEncoder blockEncoder; protected final BlockEncoder blockEncoder;
protected final ByteBuffersDataOutput blockWriteBuffer; protected final ByteBuffersDataOutput blockWriteBuffer;
@ -81,7 +83,9 @@ public class BlockWriter {
this.blockEncoder = blockEncoder; this.blockEncoder = blockEncoder;
this.blockLines = new ArrayList<>(targetNumBlockLines); this.blockLines = new ArrayList<>(targetNumBlockLines);
this.termStateSerializer = new DeltaBaseTermStateSerializer(); this.blockHeaderWriter = createBlockHeaderSerializer();
this.blockLineWriter = createBlockLineSerializer();
this.termStateSerializer = createDeltaBaseTermStateSerializer();
this.blockLinesWriteBuffer = ByteBuffersDataOutput.newResettableInstance(); this.blockLinesWriteBuffer = ByteBuffersDataOutput.newResettableInstance();
this.termStatesWriteBuffer = ByteBuffersDataOutput.newResettableInstance(); this.termStatesWriteBuffer = ByteBuffersDataOutput.newResettableInstance();
@ -91,6 +95,18 @@ public class BlockWriter {
this.scratchBytesRef = new BytesRef(); this.scratchBytesRef = new BytesRef();
} }
protected BlockHeader.Serializer createBlockHeaderSerializer() {
return new BlockHeader.Serializer();
}
protected BlockLine.Serializer createBlockLineSerializer() {
return new BlockLine.Serializer();
}
protected DeltaBaseTermStateSerializer createDeltaBaseTermStateSerializer() {
return new DeltaBaseTermStateSerializer();
}
/** /**
* Adds a new {@link BlockLine} term for the current field. * Adds a new {@link BlockLine} term for the current field.
* <p> * <p>
@ -196,7 +212,7 @@ public class BlockWriter {
reusableBlockHeader.reset(blockLines.size(), termStateSerializer.getBaseDocStartFP(), termStateSerializer.getBasePosStartFP(), reusableBlockHeader.reset(blockLines.size(), termStateSerializer.getBaseDocStartFP(), termStateSerializer.getBasePosStartFP(),
termStateSerializer.getBasePayStartFP(), Math.toIntExact(blockLinesWriteBuffer.size()), middleOffset); termStateSerializer.getBasePayStartFP(), Math.toIntExact(blockLinesWriteBuffer.size()), middleOffset);
reusableBlockHeader.write(blockWriteBuffer); blockHeaderWriter.write(blockWriteBuffer, reusableBlockHeader);
blockLinesWriteBuffer.copyTo(blockWriteBuffer); blockLinesWriteBuffer.copyTo(blockWriteBuffer);
termStatesWriteBuffer.copyTo(blockWriteBuffer); termStatesWriteBuffer.copyTo(blockWriteBuffer);
@ -236,8 +252,8 @@ public class BlockWriter {
protected void writeBlockLine(boolean isIncrementalEncodingSeed, BlockLine line, BlockLine previousLine) throws IOException { protected void writeBlockLine(boolean isIncrementalEncodingSeed, BlockLine line, BlockLine previousLine) throws IOException {
assert fieldMetadata != null; assert fieldMetadata != null;
BlockLine.Serializer.writeLine(blockLinesWriteBuffer, line, previousLine, Math.toIntExact(termStatesWriteBuffer.size()), isIncrementalEncodingSeed); blockLineWriter.writeLine(blockLinesWriteBuffer, line, previousLine, Math.toIntExact(termStatesWriteBuffer.size()), isIncrementalEncodingSeed);
BlockLine.Serializer.writeLineTermState(termStatesWriteBuffer, line, fieldMetadata.getFieldInfo(), termStateSerializer); blockLineWriter.writeLineTermState(termStatesWriteBuffer, line, fieldMetadata.getFieldInfo(), termStateSerializer);
} }
/** /**

View File

@ -194,88 +194,99 @@ public class FieldMetadata implements Accountable {
+ (docsSeen == null ? 0 : docsSeen.ramBytesUsed()); + (docsSeen == null ? 0 : docsSeen.ramBytesUsed());
} }
public static FieldMetadata read(DataInput input, FieldInfos fieldInfos, int maxNumDocs) throws IOException { /**
int fieldId = input.readVInt(); * Reads/writes field metadata.
FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldId); */
if (fieldInfo == null) { public static class Serializer {
throw new CorruptIndexException("Illegal field id= " + fieldId, input);
}
FieldMetadata fieldMetadata = new FieldMetadata(fieldInfo, 0, false);
fieldMetadata.numTerms = input.readVInt(); /**
if (fieldMetadata.numTerms <= 0) { * Stateless singleton.
throw new CorruptIndexException("Illegal number of terms= " + fieldMetadata.numTerms + " for field= " + fieldId, input); */
} public static final Serializer INSTANCE = new Serializer();
fieldMetadata.sumDocFreq = input.readVInt(); public void write(DataOutput output, FieldMetadata fieldMetadata) throws IOException {
fieldMetadata.sumTotalTermFreq = fieldMetadata.sumDocFreq; assert fieldMetadata.dictionaryStartFP >= 0;
if (fieldMetadata.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0) { assert fieldMetadata.firstBlockStartFP >= 0;
fieldMetadata.sumTotalTermFreq += input.readVInt(); assert fieldMetadata.lastBlockStartFP >= 0;
if (fieldMetadata.sumTotalTermFreq < fieldMetadata.sumDocFreq) { assert fieldMetadata.numTerms > 0 : "There should be at least one term for field " + fieldMetadata.fieldInfo.name + ": " + fieldMetadata.numTerms;
// #positions must be >= #postings. assert fieldMetadata.firstBlockStartFP <= fieldMetadata.lastBlockStartFP : "start: " + fieldMetadata.firstBlockStartFP + " end: " + fieldMetadata.lastBlockStartFP;
throw new CorruptIndexException("Illegal sumTotalTermFreq= " + fieldMetadata.sumTotalTermFreq assert fieldMetadata.lastTerm != null : "you must set the last term";
+ " sumDocFreq= " + fieldMetadata.sumDocFreq + " for field= " + fieldId, input);
output.writeVInt(fieldMetadata.fieldInfo.number);
output.writeVInt(fieldMetadata.numTerms);
output.writeVInt(fieldMetadata.sumDocFreq);
if (fieldMetadata.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0) {
assert fieldMetadata.sumTotalTermFreq >= fieldMetadata.sumDocFreq : "sumTotalFQ: " + fieldMetadata.sumTotalTermFreq + " sumDocFQ: " + fieldMetadata.sumDocFreq;
output.writeVInt(fieldMetadata.sumTotalTermFreq - fieldMetadata.sumDocFreq);
}
output.writeVInt(fieldMetadata.getDocCount());
output.writeVLong(fieldMetadata.dictionaryStartFP);
output.writeVLong(fieldMetadata.firstBlockStartFP);
output.writeVLong(fieldMetadata.lastBlockStartFP);
if (fieldMetadata.lastTerm.length > 0) {
output.writeVInt(fieldMetadata.lastTerm.length);
output.writeBytes(fieldMetadata.lastTerm.bytes, fieldMetadata.lastTerm.offset, fieldMetadata.lastTerm.length);
} else {
output.writeVInt(0);
} }
} }
fieldMetadata.docCount = input.readVInt(); public FieldMetadata read(DataInput input, FieldInfos fieldInfos, int maxNumDocs) throws IOException {
if (fieldMetadata.docCount < 0 || fieldMetadata.docCount > maxNumDocs) { int fieldId = input.readVInt();
// #docs with field must be <= #docs. FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldId);
throw new CorruptIndexException("Illegal number of docs= " + fieldMetadata.docCount if (fieldInfo == null) {
+ " maxNumDocs= " + maxNumDocs + " for field=" + fieldId, input); throw new CorruptIndexException("Illegal field id= " + fieldId, input);
} }
if (fieldMetadata.sumDocFreq < fieldMetadata.docCount) { FieldMetadata fieldMetadata = new FieldMetadata(fieldInfo, 0, false);
// #postings must be >= #docs with field.
throw new CorruptIndexException("Illegal sumDocFreq= " + fieldMetadata.sumDocFreq
+ " docCount= " + fieldMetadata.docCount + " for field= " + fieldId, input);
}
fieldMetadata.dictionaryStartFP = input.readVLong(); fieldMetadata.numTerms = input.readVInt();
fieldMetadata.firstBlockStartFP = input.readVLong(); if (fieldMetadata.numTerms <= 0) {
fieldMetadata.lastBlockStartFP = input.readVLong(); throw new CorruptIndexException("Illegal number of terms= " + fieldMetadata.numTerms + " for field= " + fieldId, input);
}
int lastTermLength = input.readVInt(); fieldMetadata.sumDocFreq = input.readVInt();
BytesRef lastTerm = new BytesRef(lastTermLength); fieldMetadata.sumTotalTermFreq = fieldMetadata.sumDocFreq;
if (lastTermLength > 0) { if (fieldMetadata.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0) {
input.readBytes(lastTerm.bytes, 0, lastTermLength); fieldMetadata.sumTotalTermFreq += input.readVInt();
lastTerm.length = lastTermLength; if (fieldMetadata.sumTotalTermFreq < fieldMetadata.sumDocFreq) {
} else if (lastTermLength < 0) { // #positions must be >= #postings.
throw new CorruptIndexException("Illegal last term length= " + lastTermLength + " for field= " + fieldId, input); throw new CorruptIndexException("Illegal sumTotalTermFreq= " + fieldMetadata.sumTotalTermFreq
} + " sumDocFreq= " + fieldMetadata.sumDocFreq + " for field= " + fieldId, input);
fieldMetadata.setLastTerm(lastTerm); }
}
return fieldMetadata; fieldMetadata.docCount = input.readVInt();
} if (fieldMetadata.docCount < 0 || fieldMetadata.docCount > maxNumDocs) {
// #docs with field must be <= #docs.
throw new CorruptIndexException("Illegal number of docs= " + fieldMetadata.docCount
+ " maxNumDocs= " + maxNumDocs + " for field=" + fieldId, input);
}
if (fieldMetadata.sumDocFreq < fieldMetadata.docCount) {
// #postings must be >= #docs with field.
throw new CorruptIndexException("Illegal sumDocFreq= " + fieldMetadata.sumDocFreq
+ " docCount= " + fieldMetadata.docCount + " for field= " + fieldId, input);
}
public void write(DataOutput output) throws IOException { fieldMetadata.dictionaryStartFP = input.readVLong();
assert dictionaryStartFP >= 0; fieldMetadata.firstBlockStartFP = input.readVLong();
assert firstBlockStartFP >= 0; fieldMetadata.lastBlockStartFP = input.readVLong();
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); int lastTermLength = input.readVInt();
BytesRef lastTerm = new BytesRef(lastTermLength);
if (lastTermLength > 0) {
input.readBytes(lastTerm.bytes, 0, lastTermLength);
lastTerm.length = lastTermLength;
} else if (lastTermLength < 0) {
throw new CorruptIndexException("Illegal last term length= " + lastTermLength + " for field= " + fieldId, input);
}
fieldMetadata.setLastTerm(lastTerm);
output.writeVInt(numTerms); return fieldMetadata;
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

@ -69,7 +69,7 @@ public class UniformSplitTermsReader extends FieldsProducer {
* It can be used for decompression or decryption. * It can be used for decompression or decryption.
*/ */
public UniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state, BlockDecoder blockDecoder) throws IOException { public UniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state, BlockDecoder blockDecoder) throws IOException {
this(postingsReader, state, blockDecoder, NAME, VERSION_START, VERSION_CURRENT, this(postingsReader, state, blockDecoder, FieldMetadata.Serializer.INSTANCE, NAME, VERSION_START, VERSION_CURRENT,
TERMS_BLOCKS_EXTENSION, TERMS_DICTIONARY_EXTENSION); TERMS_BLOCKS_EXTENSION, TERMS_DICTIONARY_EXTENSION);
} }
@ -77,8 +77,10 @@ public class UniformSplitTermsReader extends FieldsProducer {
* @param blockDecoder Optional block decoder, may be null if none. * @param blockDecoder Optional block decoder, may be null if none.
* It can be used for decompression or decryption. * It can be used for decompression or decryption.
*/ */
protected UniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state, BlockDecoder blockDecoder, protected UniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state,
String codecName, int versionStart, int versionCurrent, String termsBlocksExtension, String dictionaryExtension) throws IOException { BlockDecoder blockDecoder, FieldMetadata.Serializer fieldMetadataReader,
String codecName, int versionStart, int versionCurrent,
String termsBlocksExtension, String dictionaryExtension) throws IOException {
IndexInput dictionaryInput = null; IndexInput dictionaryInput = null;
IndexInput blockInput = null; IndexInput blockInput = null;
boolean success = false; boolean success = false;
@ -100,7 +102,7 @@ public class UniformSplitTermsReader extends FieldsProducer {
CodecUtil.retrieveChecksum(blockInput); CodecUtil.retrieveChecksum(blockInput);
seekFieldsMetadata(blockInput); seekFieldsMetadata(blockInput);
Collection<FieldMetadata> fieldMetadataCollection = parseFieldsMetadata(blockInput, state.fieldInfos, state.segmentInfo.maxDoc()); Collection<FieldMetadata> fieldMetadataCollection = parseFieldsMetadata(blockInput, state.fieldInfos, fieldMetadataReader, state.segmentInfo.maxDoc());
fieldToTermsMap = new HashMap<>(); fieldToTermsMap = new HashMap<>();
this.blockInput = blockInput; this.blockInput = blockInput;
@ -133,19 +135,19 @@ public class UniformSplitTermsReader extends FieldsProducer {
* @param indexInput {@link IndexInput} must be positioned to the fields metadata * @param indexInput {@link IndexInput} must be positioned to the fields metadata
* details by calling {@link #seekFieldsMetadata(IndexInput)} before this call. * details by calling {@link #seekFieldsMetadata(IndexInput)} before this call.
*/ */
protected static Collection<FieldMetadata> parseFieldsMetadata(IndexInput indexInput, FieldInfos fieldInfos, int maxNumDocs) throws IOException { protected static Collection<FieldMetadata> parseFieldsMetadata(IndexInput indexInput, FieldInfos fieldInfos,
FieldMetadata.Serializer fieldMetadataReader, int maxNumDocs) throws IOException {
int numFields = indexInput.readVInt(); int numFields = indexInput.readVInt();
if (numFields < 0) { if (numFields < 0) {
throw new CorruptIndexException("Illegal number of fields= " + numFields, indexInput); throw new CorruptIndexException("Illegal number of fields= " + numFields, indexInput);
} }
Collection<FieldMetadata> fieldMetadataCollection = new ArrayList<>(numFields); Collection<FieldMetadata> fieldMetadataCollection = new ArrayList<>(numFields);
for (int i = 0; i < numFields; i++) { for (int i = 0; i < numFields; i++) {
fieldMetadataCollection.add(FieldMetadata.read(indexInput, fieldInfos, maxNumDocs)); fieldMetadataCollection.add(fieldMetadataReader.read(indexInput, fieldInfos, maxNumDocs));
} }
return fieldMetadataCollection; return fieldMetadataCollection;
} }
@Override @Override
public void close() throws IOException { public void close() throws IOException {
try { try {

View File

@ -128,6 +128,7 @@ public class UniformSplitTermsWriter extends FieldsConsumer {
protected final int deltaNumLines; protected final int deltaNumLines;
protected final BlockEncoder blockEncoder; protected final BlockEncoder blockEncoder;
protected final FieldMetadata.Serializer fieldMetadataWriter;
protected final IndexOutput blockOutput; protected final IndexOutput blockOutput;
protected final IndexOutput dictionaryOutput; protected final IndexOutput dictionaryOutput;
@ -146,7 +147,7 @@ public class UniformSplitTermsWriter extends FieldsConsumer {
*/ */
public UniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state, public UniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state,
int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder) throws IOException { int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder) throws IOException {
this(postingsWriter, state, targetNumBlockLines, deltaNumLines, blockEncoder, this(postingsWriter, state, targetNumBlockLines, deltaNumLines, blockEncoder, FieldMetadata.Serializer.INSTANCE,
NAME, VERSION_CURRENT, TERMS_BLOCKS_EXTENSION, TERMS_DICTIONARY_EXTENSION); NAME, VERSION_CURRENT, TERMS_BLOCKS_EXTENSION, TERMS_DICTIONARY_EXTENSION);
} }
@ -164,7 +165,7 @@ public class UniformSplitTermsWriter extends FieldsConsumer {
* It can be used for compression or encryption. * It can be used for compression or encryption.
*/ */
protected UniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state, protected UniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state,
int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder, int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder, FieldMetadata.Serializer fieldMetadataWriter,
String codecName, int versionCurrent, String termsBlocksExtension, String dictionaryExtension) throws IOException { String codecName, int versionCurrent, String termsBlocksExtension, String dictionaryExtension) throws IOException {
validateSettings(targetNumBlockLines, deltaNumLines); validateSettings(targetNumBlockLines, deltaNumLines);
IndexOutput blockOutput = null; IndexOutput blockOutput = null;
@ -177,6 +178,7 @@ public class UniformSplitTermsWriter extends FieldsConsumer {
this.targetNumBlockLines = targetNumBlockLines; this.targetNumBlockLines = targetNumBlockLines;
this.deltaNumLines = deltaNumLines; this.deltaNumLines = deltaNumLines;
this.blockEncoder = blockEncoder; this.blockEncoder = blockEncoder;
this.fieldMetadataWriter = fieldMetadataWriter;
String termsName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, termsBlocksExtension); String termsName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, termsBlocksExtension);
blockOutput = state.directory.createOutput(termsName, state.context); blockOutput = state.directory.createOutput(termsName, state.context);
@ -278,7 +280,7 @@ public class UniformSplitTermsWriter extends FieldsConsumer {
if (fieldMetadata.getNumTerms() > 0) { if (fieldMetadata.getNumTerms() > 0) {
fieldMetadata.setLastTerm(lastTerm); fieldMetadata.setLastTerm(lastTerm);
fieldMetadata.write(fieldsOutput); fieldMetadataWriter.write(fieldsOutput, fieldMetadata);
writeDictionary(dictionaryBuilder); writeDictionary(dictionaryBuilder);
return 1; return 1;
} }

View File

@ -75,7 +75,7 @@ public class STBlockLine extends BlockLine {
/** /**
* Writes all the {@link BlockTermState} of the provided {@link STBlockLine} to the given output. * Writes all the {@link BlockTermState} of the provided {@link STBlockLine} to the given output.
*/ */
public static void writeLineTermStates(DataOutput termStatesOutput, STBlockLine line, public void writeLineTermStates(DataOutput termStatesOutput, STBlockLine line,
DeltaBaseTermStateSerializer encoder) throws IOException { DeltaBaseTermStateSerializer encoder) throws IOException {
FieldMetadataTermState fieldMetadataTermState; FieldMetadataTermState fieldMetadataTermState;
@ -111,7 +111,7 @@ public class STBlockLine extends BlockLine {
* @return The {@link BlockTermState} corresponding to the provided field id; or null if the field * @return The {@link BlockTermState} corresponding to the provided field id; or null if the field
* does not occur in the line. * does not occur in the line.
*/ */
public static BlockTermState readTermStateForField(int fieldId, DataInput termStatesInput, public BlockTermState readTermStateForField(int fieldId, DataInput termStatesInput,
DeltaBaseTermStateSerializer termStateSerializer, DeltaBaseTermStateSerializer termStateSerializer,
BlockHeader blockHeader, FieldInfos fieldInfos, BlockHeader blockHeader, FieldInfos fieldInfos,
BlockTermState reuse) throws IOException { BlockTermState reuse) throws IOException {
@ -161,7 +161,7 @@ public class STBlockLine extends BlockLine {
* @param fieldTermStatesMap Map filled with the term states for each field. It is cleared first. * @param fieldTermStatesMap Map filled with the term states for each field. It is cleared first.
* @see #readTermStateForField * @see #readTermStateForField
*/ */
public static void readFieldTermStatesMap(DataInput termStatesInput, public void readFieldTermStatesMap(DataInput termStatesInput,
DeltaBaseTermStateSerializer termStateSerializer, DeltaBaseTermStateSerializer termStateSerializer,
BlockHeader blockHeader, BlockHeader blockHeader,
FieldInfos fieldInfos, FieldInfos fieldInfos,
@ -183,7 +183,7 @@ public class STBlockLine extends BlockLine {
/** /**
* Reads all the field ids in the current block line of the provided input. * Reads all the field ids in the current block line of the provided input.
*/ */
public static int[] readFieldIds(DataInput termStatesInput, int numFields) throws IOException { public int[] readFieldIds(DataInput termStatesInput, int numFields) throws IOException {
int[] fieldIds = new int[numFields]; int[] fieldIds = new int[numFields];
for (int i = 0; i < numFields; i++) { for (int i = 0; i < numFields; i++) {
fieldIds[i] = termStatesInput.readVInt(); fieldIds[i] = termStatesInput.readVInt();

View File

@ -117,6 +117,11 @@ public class STBlockReader extends BlockReader {
return blockStartFP > fieldMetadata.getLastBlockStartFP() || super.isBeyondLastTerm(searchedTerm, blockStartFP); return blockStartFP > fieldMetadata.getLastBlockStartFP() || super.isBeyondLastTerm(searchedTerm, blockStartFP);
} }
@Override
protected STBlockLine.Serializer createBlockLineSerializer() {
return new STBlockLine.Serializer();
}
/** /**
* Reads the {@link BlockTermState} on the current line for this reader's field. * Reads the {@link BlockTermState} on the current line for this reader's field.
* *
@ -125,7 +130,7 @@ public class STBlockReader extends BlockReader {
@Override @Override
protected BlockTermState readTermState() throws IOException { protected BlockTermState readTermState() throws IOException {
termStatesReadBuffer.setPosition(blockFirstLineStart + blockHeader.getTermStatesBaseOffset() + blockLine.getTermStateRelativeOffset()); termStatesReadBuffer.setPosition(blockFirstLineStart + blockHeader.getTermStatesBaseOffset() + blockLine.getTermStateRelativeOffset());
return termState = STBlockLine.Serializer.readTermStateForField( return termState = ((STBlockLine.Serializer) blockLineReader).readTermStateForField(
fieldMetadata.getFieldInfo().number, fieldMetadata.getFieldInfo().number,
termStatesReadBuffer, termStatesReadBuffer,
termStateSerializer, termStateSerializer,

View File

@ -84,10 +84,15 @@ public class STBlockWriter extends BlockWriter {
super.finishLastBlock(dictionaryBuilder); super.finishLastBlock(dictionaryBuilder);
} }
@Override
protected BlockLine.Serializer createBlockLineSerializer() {
return new STBlockLine.Serializer();
}
@Override @Override
protected void writeBlockLine(boolean isIncrementalEncodingSeed, BlockLine line, BlockLine previousLine) throws IOException { protected void writeBlockLine(boolean isIncrementalEncodingSeed, BlockLine line, BlockLine previousLine) throws IOException {
STBlockLine.Serializer.writeLine(blockLinesWriteBuffer, line, previousLine, Math.toIntExact(termStatesWriteBuffer.size()), isIncrementalEncodingSeed); blockLineWriter.writeLine(blockLinesWriteBuffer, line, previousLine, Math.toIntExact(termStatesWriteBuffer.size()), isIncrementalEncodingSeed);
STBlockLine.Serializer.writeLineTermStates(termStatesWriteBuffer, (STBlockLine) line, termStateSerializer); ((STBlockLine.Serializer) blockLineWriter).writeLineTermStates(termStatesWriteBuffer, (STBlockLine) line, termStateSerializer);
((STBlockLine) line).collectFields(fieldsInBlock); ((STBlockLine) line).collectFields(fieldsInBlock);
} }

View File

@ -91,6 +91,11 @@ public class STIntersectBlockReader extends IntersectBlockReader {
return super.nextBlockMatchingPrefix() && blockHeader != null; return super.nextBlockMatchingPrefix() && blockHeader != null;
} }
@Override
protected STBlockLine.Serializer createBlockLineSerializer() {
return new STBlockLine.Serializer();
}
/** /**
* Reads the {@link BlockTermState} on the current line for the specific field * Reads the {@link BlockTermState} on the current line for the specific field
* corresponding this this reader. * corresponding this this reader.
@ -100,7 +105,7 @@ public class STIntersectBlockReader extends IntersectBlockReader {
@Override @Override
protected BlockTermState readTermState() throws IOException { protected BlockTermState readTermState() throws IOException {
termStatesReadBuffer.setPosition(blockFirstLineStart + blockHeader.getTermStatesBaseOffset() + blockLine.getTermStateRelativeOffset()); termStatesReadBuffer.setPosition(blockFirstLineStart + blockHeader.getTermStatesBaseOffset() + blockLine.getTermStateRelativeOffset());
return STBlockLine.Serializer.readTermStateForField( return ((STBlockLine.Serializer) blockLineReader).readTermStateForField(
fieldMetadata.getFieldInfo().number, fieldMetadata.getFieldInfo().number,
termStatesReadBuffer, termStatesReadBuffer,
termStateSerializer, termStateSerializer,

View File

@ -98,7 +98,7 @@ public class STMergingBlockReader extends STBlockReader {
public void readFieldTermStatesMap(Map<String, BlockTermState> fieldTermStatesMap) throws IOException { public void readFieldTermStatesMap(Map<String, BlockTermState> fieldTermStatesMap) throws IOException {
if (term() != null) { if (term() != null) {
termStatesReadBuffer.setPosition(blockFirstLineStart + blockHeader.getTermStatesBaseOffset() + blockLine.getTermStateRelativeOffset()); termStatesReadBuffer.setPosition(blockFirstLineStart + blockHeader.getTermStatesBaseOffset() + blockLine.getTermStateRelativeOffset());
STBlockLine.Serializer.readFieldTermStatesMap( ((STBlockLine.Serializer) blockLineReader).readFieldTermStatesMap(
termStatesReadBuffer, termStatesReadBuffer,
termStateSerializer, termStateSerializer,
blockHeader, blockHeader,

View File

@ -47,13 +47,15 @@ import static org.apache.lucene.codecs.uniformsplit.sharedterms.STUniformSplitPo
public class STUniformSplitTermsReader extends UniformSplitTermsReader { public class STUniformSplitTermsReader extends UniformSplitTermsReader {
public STUniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state, BlockDecoder blockDecoder) throws IOException { public STUniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state, BlockDecoder blockDecoder) throws IOException {
super(postingsReader, state, blockDecoder, NAME, VERSION_START, this(postingsReader, state, blockDecoder, FieldMetadata.Serializer.INSTANCE,
VERSION_CURRENT, TERMS_BLOCKS_EXTENSION, TERMS_DICTIONARY_EXTENSION); NAME, VERSION_START, VERSION_CURRENT, TERMS_BLOCKS_EXTENSION, TERMS_DICTIONARY_EXTENSION);
} }
protected STUniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state, BlockDecoder blockDecoder, protected STUniformSplitTermsReader(PostingsReaderBase postingsReader, SegmentReadState state,
String codecName, int versionStart, int versionCurrent, String termsBlocksExtension, String dictionaryExtension) throws IOException { BlockDecoder blockDecoder, FieldMetadata.Serializer fieldMetadataReader,
super(postingsReader, state, blockDecoder, codecName, versionStart, versionCurrent, termsBlocksExtension, dictionaryExtension); String codecName, int versionStart, int versionCurrent,
String termsBlocksExtension, String dictionaryExtension) throws IOException {
super(postingsReader, state, blockDecoder, fieldMetadataReader, codecName, versionStart, versionCurrent, termsBlocksExtension, dictionaryExtension);
} }
@Override @Override

View File

@ -88,13 +88,14 @@ public class STUniformSplitTermsWriter extends UniformSplitTermsWriter {
public STUniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state, public STUniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state,
int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder) throws IOException { int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder) throws IOException {
this(postingsWriter, state, targetNumBlockLines, deltaNumLines, blockEncoder, NAME, VERSION_CURRENT, TERMS_BLOCKS_EXTENSION, TERMS_DICTIONARY_EXTENSION); this(postingsWriter, state, targetNumBlockLines, deltaNumLines, blockEncoder, FieldMetadata.Serializer.INSTANCE,
NAME, VERSION_CURRENT, TERMS_BLOCKS_EXTENSION, TERMS_DICTIONARY_EXTENSION);
} }
protected STUniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state, protected STUniformSplitTermsWriter(PostingsWriterBase postingsWriter, SegmentWriteState state,
int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder, int targetNumBlockLines, int deltaNumLines, BlockEncoder blockEncoder, FieldMetadata.Serializer fieldMetadataWriter,
String codecName, int versionCurrent, String termsBlocksExtension, String dictionaryExtension) throws IOException { String codecName, int versionCurrent, String termsBlocksExtension, String dictionaryExtension) throws IOException {
super(postingsWriter, state, targetNumBlockLines, deltaNumLines, blockEncoder, codecName, versionCurrent, termsBlocksExtension, dictionaryExtension); super(postingsWriter, state, targetNumBlockLines, deltaNumLines, blockEncoder, fieldMetadataWriter, codecName, versionCurrent, termsBlocksExtension, dictionaryExtension);
} }
@Override @Override
@ -200,7 +201,7 @@ public class STUniformSplitTermsWriter extends UniformSplitTermsWriter {
int fieldsNumber = 0; int fieldsNumber = 0;
for (FieldMetadata fieldMetadata : fieldMetadataList) { for (FieldMetadata fieldMetadata : fieldMetadataList) {
if (fieldMetadata.getNumTerms() > 0) { if (fieldMetadata.getNumTerms() > 0) {
fieldMetadata.write(fieldsOutput); fieldMetadataWriter.write(fieldsOutput, fieldMetadata);
fieldsNumber++; fieldsNumber++;
} }
} }

View File

@ -141,6 +141,7 @@ public class FieldHighlighter {
} }
}); });
Passage passage = new Passage(); // the current passage in-progress. Will either get reset or added to queue. Passage passage = new Passage(); // the current passage in-progress. Will either get reset or added to queue.
int lastPassageEnd = 0;
do { do {
int start = off.startOffset(); int start = off.startOffset();
@ -158,9 +159,12 @@ public class FieldHighlighter {
if (start >= contentLength) { if (start >= contentLength) {
break; break;
} }
// find fragment from the middle of the match, so the result's length may be closer to fragsize
final int center = start + (end - start) / 2;
// advance breakIterator // advance breakIterator
passage.setStartOffset(Math.max(this.breakIterator.preceding(start + 1), 0)); passage.setStartOffset(Math.min(start, Math.max(this.breakIterator.preceding(Math.max(start + 1, center)), lastPassageEnd)));
passage.setEndOffset(Math.min(this.breakIterator.following(start), contentLength)); lastPassageEnd = Math.max(end, Math.min(this.breakIterator.following(Math.min(end - 1, center)), contentLength));
passage.setEndOffset(lastPassageEnd);
} }
// Add this term to the passage. // Add this term to the passage.
BytesRef term = off.getTerm();// a reference; safe to refer to BytesRef term = off.getTerm();// a reference; safe to refer to

View File

@ -33,22 +33,49 @@ public class LengthGoalBreakIterator extends BreakIterator {
private final BreakIterator baseIter; private final BreakIterator baseIter;
private final int lengthGoal; private final int lengthGoal;
private final float fragmentAlignment; // how much text to align before match-fragment, valid in range [0, 1]
private final boolean isMinimumLength; // if false then is "closest to" length private final boolean isMinimumLength; // if false then is "closest to" length
private int currentCache;
/** Breaks will be at least {@code minLength} apart (to the extent possible). */ /**
* Breaks will be at least {@code minLength} apart (to the extent possible),
* while trying to position the match inside the fragment according to {@code fragmentAlignment}.
*/
public static LengthGoalBreakIterator createMinLength(BreakIterator baseIter, int minLength,
float fragmentAlignment) {
return new LengthGoalBreakIterator(baseIter, minLength, fragmentAlignment, true, baseIter.current());
}
/** For backwards compatibility you can initialise the break iterator without fragmentAlignment. */
@Deprecated
public static LengthGoalBreakIterator createMinLength(BreakIterator baseIter, int minLength) { public static LengthGoalBreakIterator createMinLength(BreakIterator baseIter, int minLength) {
return new LengthGoalBreakIterator(baseIter, minLength, true); return createMinLength(baseIter, minLength, 0.f);
} }
/** Breaks will be on average {@code targetLength} apart; the closest break to this target (before or after) /**
* is chosen. */ * Breaks will be on average {@code targetLength} apart; the closest break to this target (before or after)
* is chosen. The match will be positioned according to {@code fragmentAlignment} as much as possible.
*/
public static LengthGoalBreakIterator createClosestToLength(BreakIterator baseIter, int targetLength,
float fragmentAlignment) {
return new LengthGoalBreakIterator(baseIter, targetLength, fragmentAlignment, false, baseIter.current());
}
/** For backwards compatibility you can initialise the break iterator without fragmentAlignment. */
@Deprecated
public static LengthGoalBreakIterator createClosestToLength(BreakIterator baseIter, int targetLength) { public static LengthGoalBreakIterator createClosestToLength(BreakIterator baseIter, int targetLength) {
return new LengthGoalBreakIterator(baseIter, targetLength, false); return createClosestToLength(baseIter, targetLength, 0.f);
} }
private LengthGoalBreakIterator(BreakIterator baseIter, int lengthGoal, boolean isMinimumLength) { private LengthGoalBreakIterator(BreakIterator baseIter, int lengthGoal, float fragmentAlignment,
boolean isMinimumLength, int currentCache) {
this.baseIter = baseIter; this.baseIter = baseIter;
this.currentCache = currentCache;
this.lengthGoal = lengthGoal; this.lengthGoal = lengthGoal;
if (fragmentAlignment < 0.f || fragmentAlignment > 1.f || !Float.isFinite(fragmentAlignment)) {
throw new IllegalArgumentException("fragmentAlignment must be >= zero and <= one");
}
this.fragmentAlignment = fragmentAlignment;
this.isMinimumLength = isMinimumLength; this.isMinimumLength = isMinimumLength;
} }
@ -60,12 +87,15 @@ public class LengthGoalBreakIterator extends BreakIterator {
@Override @Override
public String toString() { public String toString() {
String goalDesc = isMinimumLength ? "minLen" : "targetLen"; String goalDesc = isMinimumLength ? "minLen" : "targetLen";
return getClass().getSimpleName() + "{" + goalDesc + "=" + lengthGoal + ", baseIter=" + baseIter + "}"; return getClass().getSimpleName() + "{" + goalDesc + "=" + lengthGoal + ", fragAlign=" + fragmentAlignment +
", baseIter=" + baseIter + "}";
} }
@Override @Override
public Object clone() { public Object clone() {
return new LengthGoalBreakIterator((BreakIterator) baseIter.clone(), lengthGoal, isMinimumLength); return new LengthGoalBreakIterator(
(BreakIterator) baseIter.clone(), lengthGoal, fragmentAlignment, isMinimumLength, currentCache
);
} }
@Override @Override
@ -76,26 +106,28 @@ public class LengthGoalBreakIterator extends BreakIterator {
@Override @Override
public void setText(String newText) { public void setText(String newText) {
baseIter.setText(newText); baseIter.setText(newText);
currentCache = baseIter.current();
} }
@Override @Override
public void setText(CharacterIterator newText) { public void setText(CharacterIterator newText) {
baseIter.setText(newText); baseIter.setText(newText);
currentCache = baseIter.current();
} }
@Override @Override
public int current() { public int current() {
return baseIter.current(); return currentCache;
} }
@Override @Override
public int first() { public int first() {
return baseIter.first(); return currentCache = baseIter.first();
} }
@Override @Override
public int last() { public int last() {
return baseIter.last(); return currentCache = baseIter.last();
} }
@Override @Override
@ -104,10 +136,10 @@ public class LengthGoalBreakIterator extends BreakIterator {
return baseIter.next(n); // probably wrong return baseIter.next(n); // probably wrong
} }
// called by getSummaryPassagesNoHighlight to generate default summary. // Called by getSummaryPassagesNoHighlight to generate default summary.
@Override @Override
public int next() { public int next() {
return following(current()); return following(currentCache, currentCache + lengthGoal);
} }
@Override @Override
@ -116,65 +148,70 @@ public class LengthGoalBreakIterator extends BreakIterator {
return baseIter.previous(); return baseIter.previous();
} }
// called while the current position is the start of a new passage; find end of passage
@Override @Override
public int following(int followingIdx) { public int following(int matchEndIndex) {
final int startIdx = current(); return following(matchEndIndex, (matchEndIndex + 1) + (int)(lengthGoal * (1.f - fragmentAlignment)));
if (followingIdx < startIdx) {
assert false : "Not supported";
return baseIter.following(followingIdx);
}
final int targetIdx = startIdx + lengthGoal;
// When followingIdx >= targetIdx, we can simply delegate since it will be >= the target
if (followingIdx >= targetIdx - 1) {
return baseIter.following(followingIdx);
}
// If target exceeds the text length, return the last index.
if (targetIdx >= getText().getEndIndex()) {
return baseIter.last();
}
// Find closest break >= the target
final int afterIdx = baseIter.following(targetIdx - 1);
if (afterIdx == DONE) { // we're at the end; can this happen?
return current();
}
if (afterIdx == targetIdx) { // right on the money
return afterIdx;
}
if (isMinimumLength) { // thus never undershoot
return afterIdx;
}
// note: it is a shame that we invoke preceding() *in addition to* following(); BI's are sometimes expensive.
// Find closest break < target
final int beforeIdx = baseIter.preceding(targetIdx); // or could do baseIter.previous() but we hope the BI implements preceding()
if (beforeIdx <= followingIdx) { // too far back
return moveToBreak(afterIdx);
}
if (targetIdx - beforeIdx <= afterIdx - targetIdx) {
return beforeIdx;
}
return moveToBreak(afterIdx);
} }
private int moveToBreak(int idx) { // precondition: idx is a known break private int following(int matchEndIndex, int targetIdx) {
// bi.isBoundary(idx) has side-effect of moving the position. Not obvious! if (targetIdx >= getText().getEndIndex()) {
//boolean moved = baseIter.isBoundary(idx); // probably not particularly expensive if (currentCache == baseIter.last()) {
//assert moved && current() == idx; return DONE;
}
return currentCache = baseIter.last();
}
final int afterIdx = baseIter.following(targetIdx - 1);
if (afterIdx == DONE) {
currentCache = baseIter.last();
return DONE;
}
if (afterIdx == targetIdx) { // right on the money
return currentCache = afterIdx;
}
if (isMinimumLength) { // thus never undershoot
return currentCache = afterIdx;
}
// TODO fix: Would prefer to do "- 1" instead of "- 2" but CustomSeparatorBreakIterator has a bug. // note: it is a shame that we invoke preceding() *one more time*; BI's are sometimes expensive.
int current = baseIter.following(idx - 2);
assert current == idx : "following() didn't move us to the expected index."; // Find closest break to target
return idx; final int beforeIdx = baseIter.preceding(targetIdx);
if (targetIdx - beforeIdx < afterIdx - targetIdx && beforeIdx > matchEndIndex) {
return currentCache = beforeIdx;
}
return currentCache = afterIdx;
} }
// called at start of new Passage given first word start offset // called at start of new Passage given first word start offset
@Override @Override
public int preceding(int offset) { public int preceding(int matchStartIndex) {
return baseIter.preceding(offset); // no change needed final int targetIdx = (matchStartIndex - 1) - (int)(lengthGoal * fragmentAlignment);
if (targetIdx <= 0) {
if (currentCache == baseIter.first()) {
return DONE;
}
return currentCache = baseIter.first();
}
final int beforeIdx = baseIter.preceding(targetIdx + 1);
if (beforeIdx == DONE) {
currentCache = baseIter.first();
return DONE;
}
if (beforeIdx == targetIdx) { // right on the money
return currentCache = beforeIdx;
}
if (isMinimumLength) { // thus never undershoot
return currentCache = beforeIdx;
}
// note: it is a shame that we invoke following() *one more time*; BI's are sometimes expensive.
// Find closest break to target
final int afterIdx = baseIter.following(targetIdx - 1);
if (afterIdx - targetIdx < targetIdx - beforeIdx && afterIdx < matchStartIndex) {
return currentCache = afterIdx;
}
return currentCache = beforeIdx;
} }
@Override @Override

View File

@ -18,6 +18,7 @@
package org.apache.lucene.search.uhighlight; package org.apache.lucene.search.uhighlight;
import java.io.IOException; import java.io.IOException;
import java.text.BreakIterator;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockAnalyzer;
@ -28,6 +29,7 @@ import org.apache.lucene.util.QueryBuilder;
public class LengthGoalBreakIteratorTest extends LuceneTestCase { public class LengthGoalBreakIteratorTest extends LuceneTestCase {
private static final String FIELD = "body"; private static final String FIELD = "body";
private static final float[] ALIGNS = {0.f, 0.5f, 1.f};
// We test LengthGoalBreakIterator as it is used by the UnifiedHighlighter instead of directly, because it is // We test LengthGoalBreakIterator as it is used by the UnifiedHighlighter instead of directly, because it is
// not a general purpose BreakIterator. A unit test of it directly wouldn't give as much confidence. // not a general purpose BreakIterator. A unit test of it directly wouldn't give as much confidence.
@ -39,65 +41,159 @@ public class LengthGoalBreakIteratorTest extends LuceneTestCase {
// 0 1 // 0 1
// 01234567890123456789 // 01234567890123456789
static final String CONTENT = "Aa bb. Cc dd. Ee ff"; static final String CONTENT = "Aa bb. Cc dd. Ee ff";
static final String CONTENT2 = "Aa bb Cc dd X Ee ff Gg hh.";
static final String CONTENT3 = "Aa bbcc ddxyzee ffgg hh.";
public void testFragmentAlignmentConstructor() throws IOException {
BreakIterator baseBI = new CustomSeparatorBreakIterator('.');
// test fragmentAlignment validation
float[] valid_aligns = {0.f, 0.3333f, 0.5f, 0.99f, 1.f};
for (float alignment : valid_aligns) {
LengthGoalBreakIterator.createClosestToLength(baseBI, 50, alignment);
}
float[] invalid_aligns = {-0.01f, -1.f, 1.5f, Float.NaN, Float.NEGATIVE_INFINITY, Float.POSITIVE_INFINITY};
for (float alignment : invalid_aligns) {
expectThrows(IllegalArgumentException.class, () -> {
LengthGoalBreakIterator.createClosestToLength(baseBI, 50, alignment);
});
}
// test backwards compatibility constructors
String backwardCompString = LengthGoalBreakIterator.createClosestToLength(baseBI, 50).toString();
assertTrue(backwardCompString, backwardCompString.contains("fragAlign=0.0"));
backwardCompString = LengthGoalBreakIterator.createMinLength(baseBI, 50).toString();
assertTrue(backwardCompString, backwardCompString.contains("fragAlign=0.0"));
}
public void testTargetLen() throws IOException { public void testTargetLen() throws IOException {
// "goal" means target length goal to find closest break // "goal" means target length goal to find closest break
// at first word: // at first word:
Query query = query("aa"); Query query = query("aa");
assertEquals("almost two sent", assertEquals("almost two sent A",
"<b>Aa</b> bb.", highlightClosestToLen(CONTENT, query, 9)); "<b>Aa</b> bb.", highlightClosestToLen(CONTENT, query, 7, 0.f));
assertEquals( "barely two sent", assertEquals("almost two sent B",
"<b>Aa</b> bb. Cc dd.", highlightClosestToLen(CONTENT, query, 10)); "<b>Aa</b> bb.", highlightClosestToLen(CONTENT, query, 15, 0.5f));
assertEquals("long goal", assertEquals("almost two sent C",
"<b>Aa</b> bb. Cc dd. Ee ff", highlightClosestToLen(CONTENT, query, 17 + random().nextInt(20))); "<b>Aa</b> bb.", highlightClosestToLen(CONTENT, query, 64, 1.f));
assertEquals("barely two sent A",
"<b>Aa</b> bb. Cc dd.", highlightClosestToLen(CONTENT, query, 8, 0.f));
assertEquals("barely two sent B",
"<b>Aa</b> bb. Cc dd.", highlightClosestToLen(CONTENT, query, 16, 0.5f));
assertEquals("long goal A",
"<b>Aa</b> bb. Cc dd. Ee ff", highlightClosestToLen(CONTENT, query, 14 + random().nextInt(20), 0.f));
assertEquals("long goal B",
"<b>Aa</b> bb. Cc dd. Ee ff", highlightClosestToLen(CONTENT, query, 28 + random().nextInt(20), 0.5f));
// at some word not at start of passage // at some word not at start of passage
query = query("dd"); query = query("dd");
assertEquals("short goal", for (float align : ALIGNS) {
" Cc <b>dd</b>.", highlightClosestToLen(CONTENT, query, random().nextInt(5))); // alignment is not meaningful if fragsize is shorter than or closer to match-fragment boundaries
assertEquals("almost two sent", assertEquals("short goal " + align,
" Cc <b>dd</b>.", highlightClosestToLen(CONTENT, query, 10)); " Cc <b>dd</b>.", highlightClosestToLen(CONTENT, query, random().nextInt(4), align));
assertEquals("barely two sent", }
" Cc <b>dd</b>. Ee ff", highlightClosestToLen(CONTENT, query, 11)); // preceding/following inclusion by alignment parameter
assertEquals("long goal", assertEquals("barely two sent A",
" Cc <b>dd</b>. Ee ff", highlightClosestToLen(CONTENT, query, 12 + random().nextInt(20))); " Cc <b>dd</b>. Ee ff", highlightClosestToLen(CONTENT, query, 11, 0.f));
assertEquals("barely two sent B",
" Cc <b>dd</b>. Ee ff", highlightClosestToLen(CONTENT, query, 11, 0.5f));
assertEquals("barely two sent C",
"Aa bb. Cc <b>dd</b>.", highlightClosestToLen(CONTENT, query, 11, 1.f));
assertEquals("long goal A",
" Cc <b>dd</b>. Ee ff", highlightClosestToLen(CONTENT, query, 17 + random().nextInt(20), 0.f));
assertEquals("long goal B",
"Aa bb. Cc <b>dd</b>. Ee ff", highlightClosestToLen(CONTENT, query, 17 + random().nextInt(20), 0.5f));
assertEquals("long goal C",
"Aa bb. Cc <b>dd</b>.", highlightClosestToLen(CONTENT, query, 17 + random().nextInt(20), 1.f));
query = query("ddxyzee");
assertEquals("test fragment search from the middle of the match; almost including",
"<b>ddxyzee</b> ", highlightClosestToLen(CONTENT3, query, 7, 0.5f, 1, ' '));
assertEquals("test fragment search from the middle of the match; barely including",
"bbcc <b>ddxyzee</b> ffgg ", highlightClosestToLen(CONTENT3, query, 14, 0.5f, 1, ' '));
} }
public void testMinLen() throws IOException { public void testMinLen() throws IOException {
// minLen mode is simpler than targetLen... just test a few cases // minLen mode is simpler than targetLen... just test a few cases
Query query = query("dd"); Query query = query("dd");
assertEquals("almost two sent", assertEquals("almost two sent A",
" Cc <b>dd</b>.", highlightMinLen(CONTENT, query, 6)); " Cc <b>dd</b>.", highlightMinLen(CONTENT, query, 0, 0.f));
assertEquals("barely two sent", assertEquals("almost two sent B",
" Cc <b>dd</b>. Ee ff", highlightMinLen(CONTENT, query, 7)); " Cc <b>dd</b>.", highlightMinLen(CONTENT, query, 1, 0.5f));
assertEquals("almost two sent C",
" Cc <b>dd</b>.", highlightMinLen(CONTENT, query, 5, 1.f));
assertEquals("barely two sent A",
" Cc <b>dd</b>. Ee ff", highlightMinLen(CONTENT, query, 1, 0.f));
assertEquals("barely two sent B",
" Cc <b>dd</b>. Ee ff", highlightMinLen(CONTENT, query, 2, 0.5f));
assertEquals("barely two sent C",
"Aa bb. Cc <b>dd</b>.", highlightMinLen(CONTENT, query, 7, 1.f));
assertEquals("barely two sent D/a",
" Cc <b>dd</b>.", highlightMinLen(CONTENT, query, 2, 0.55f));
assertEquals("barely two sent D/b",
" Cc <b>dd</b>. Ee ff", highlightMinLen(CONTENT, query, 3, 0.55f));
assertEquals("barely two sent E/a",
" Cc <b>dd</b>. Ee ff", highlightMinLen(CONTENT, query, 10, 0.5f));
assertEquals("barely two sent E/b",
"Aa bb. Cc <b>dd</b>. Ee ff", highlightMinLen(CONTENT, query, 10, 0.7f));
assertEquals("barely two sent E/c",
"Aa bb. Cc <b>dd</b>.", highlightMinLen(CONTENT, query, 9, 0.9f));
query = query("ddxyzee");
assertEquals("test fragment search from the middle of the match; almost including",
"<b>ddxyzee</b> ", highlightMinLen(CONTENT3, query, 7, 0.5f, ' '));
assertEquals("test fragment search from the middle of the match; barely including",
"bbcc <b>ddxyzee</b> ffgg ", highlightMinLen(CONTENT3, query, 8, 0.5f, ' '));
}
public void testMinLenPrecision() throws IOException {
Query query = query("x");
assertEquals("test absolute minimal length",
"<b>X</b> ", highlightMinLen(CONTENT2, query, 1, 0.5f, ' '));
assertEquals("test slightly above minimal length",
"dd <b>X</b> Ee ", highlightMinLen(CONTENT2, query, 4, 0.5f, ' '));
} }
public void testDefaultSummaryTargetLen() throws IOException { public void testDefaultSummaryTargetLen() throws IOException {
Query query = query("zz"); Query query = query("zz");
assertEquals("Aa bb.", for (float align : ALIGNS) { // alignment is not used for creating default-summary
highlightClosestToLen(CONTENT, query, random().nextInt(10))); // < 10 assertEquals("Aa bb.",
highlightClosestToLen(CONTENT, query, 6 + random().nextInt(4), align));
assertEquals("Aa bb. Cc dd.",
highlightClosestToLen(CONTENT, query, 12 + random().nextInt(4), align));
assertEquals("Aa bb. Cc dd. Ee ff",
highlightClosestToLen(CONTENT, query, 17 + random().nextInt(20), align));
}
assertEquals("Aa bb. Cc dd.", assertEquals("Aa bb. Cc dd.",
highlightClosestToLen(CONTENT, query, 10 + 6)); // cusp of adding 3rd sentence highlightClosestToLen(CONTENT, query, 6 + random().nextInt(4), 0.f, 2));
assertEquals("Aa bb. Cc dd. Ee ff",
highlightClosestToLen(CONTENT, query, 17 + random().nextInt(20))); // >= 14
} }
private Query query(String qStr) { private Query query(String qStr) {
return new QueryBuilder(analyzer).createBooleanQuery(FIELD, qStr); return new QueryBuilder(analyzer).createBooleanQuery(FIELD, qStr);
} }
private String highlightClosestToLen(String content, Query query, int lengthGoal) throws IOException { private String highlightClosestToLen(String content, Query query, int lengthGoal, float fragAlign) throws IOException {
UnifiedHighlighter highlighter = new UnifiedHighlighter(null, analyzer); return highlightClosestToLen(content, query, lengthGoal, fragAlign, 1);
highlighter.setBreakIterator(() -> LengthGoalBreakIterator.createClosestToLength(new CustomSeparatorBreakIterator('.'), lengthGoal));
return highlighter.highlightWithoutSearcher(FIELD, query, content, 1).toString();
} }
private String highlightMinLen(String content, Query query, int lengthGoal) throws IOException { private String highlightClosestToLen(String content, Query query, int lengthGoal, float fragAlign, int maxPassages) throws IOException {
return highlightClosestToLen(content, query, lengthGoal, fragAlign, maxPassages, '.');
}
private String highlightClosestToLen(String content, Query query, int lengthGoal, float fragAlign, int maxPassages, char separator) throws IOException {
UnifiedHighlighter highlighter = new UnifiedHighlighter(null, analyzer);
highlighter.setBreakIterator(() -> LengthGoalBreakIterator.createClosestToLength(new CustomSeparatorBreakIterator(separator), lengthGoal, fragAlign));
return highlighter.highlightWithoutSearcher(FIELD, query, content, maxPassages).toString();
}
private String highlightMinLen(String content, Query query, int lengthGoal, float fragAlign) throws IOException {
return highlightMinLen(content, query, lengthGoal, fragAlign, '.');
}
private String highlightMinLen(String content, Query query, int lengthGoal, float fragAlign, char separator) throws IOException {
// differs from above only by "createMinLength" // differs from above only by "createMinLength"
UnifiedHighlighter highlighter = new UnifiedHighlighter(null, analyzer); UnifiedHighlighter highlighter = new UnifiedHighlighter(null, analyzer);
highlighter.setBreakIterator(() -> LengthGoalBreakIterator.createMinLength(new CustomSeparatorBreakIterator('.'), lengthGoal)); highlighter.setBreakIterator(() -> LengthGoalBreakIterator.createMinLength(new CustomSeparatorBreakIterator(separator), lengthGoal, fragAlign));
return highlighter.highlightWithoutSearcher(FIELD, query, content, 1).toString(); return highlighter.highlightWithoutSearcher(FIELD, query, content, 1).toString();
} }
} }

View File

@ -184,6 +184,10 @@ Improvements
* SOLR-14129: Reuse Jackson ObjectMapper in AuditLoggerPlugin (janhoy) * SOLR-14129: Reuse Jackson ObjectMapper in AuditLoggerPlugin (janhoy)
* LUCENE-9093: The Unified highlighter has two new passage sizing parameters, hl.fragAlignRatio and
hl.fragsizeIsMinimum, with defaults that aim to better center matches in fragments than previously. See the ref guide.
Regardless of the settings, the passages may be sized differently than before. (Nándor Mátravölgyi, David Smiley)
Optimizations Optimizations
--------------------- ---------------------
(No changes) (No changes)

View File

@ -326,8 +326,11 @@ public class UnifiedSolrHighlighter extends SolrHighlighter implements PluginInf
if (fragsize <= 1) { // no real minimum size if (fragsize <= 1) { // no real minimum size
return baseBI; return baseBI;
} }
return LengthGoalBreakIterator.createMinLength(baseBI, fragsize); float fragalign = params.getFieldFloat(field, HighlightParams.FRAGALIGNRATIO, 0.5f);
// TODO option for using createClosestToLength() if (params.getFieldBool(field, HighlightParams.FRAGSIZEISMINIMUM, false)) {
return LengthGoalBreakIterator.createMinLength(baseBI, fragsize, fragalign);
}
return LengthGoalBreakIterator.createClosestToLength(baseBI, fragsize, fragalign);
} }
/** /**

View File

@ -251,8 +251,17 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "SEPARATOR","hl.bs.separator","#","hl.fragsize", "-1"), req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "SEPARATOR","hl.bs.separator","#","hl.fragsize", "-1"),
"//lst[@name='highlighting']/lst[@name='104']/arr[@name='text']/str='While the other <em>document</em> contains the same #'"); "//lst[@name='highlighting']/lst[@name='104']/arr[@name='text']/str='While the other <em>document</em> contains the same #'");
assertQ("CUSTOM breakiterator with fragsize 70 minimum",
req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "SEPARATOR","hl.bs.separator","#","hl.fragsize", "70", "hl.fragsizeIsMinimum", "true"),
"//lst[@name='highlighting']/lst[@name='103']/arr[@name='text']/str='This <em>document</em> contains # special characters, while the other <em>document</em> contains the same #'");
assertQ("CUSTOM breakiterator with fragsize 70", assertQ("CUSTOM breakiterator with fragsize 70",
req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "SEPARATOR","hl.bs.separator","#","hl.fragsize", "70"), req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "SEPARATOR","hl.bs.separator","#","hl.fragsize", "70"),
"//lst[@name='highlighting']/lst[@name='103']/arr[@name='text']/str='This <em>document</em> contains #'");
assertQ("CUSTOM breakiterator with fragsize 90",
req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "SEPARATOR","hl.bs.separator","#","hl.fragsize", "90"),
"//lst[@name='highlighting']/lst[@name='103']/arr[@name='text']/str='This <em>document</em> contains #'");
assertQ("CUSTOM breakiterator with fragsize 100",
req("q", "text:document", "sort", "id asc", "hl", "true", "hl.bs.type", "SEPARATOR","hl.bs.separator","#","hl.fragsize", "100"),
"//lst[@name='highlighting']/lst[@name='103']/arr[@name='text']/str='This <em>document</em> contains # special characters, while the other <em>document</em> contains the same #'"); "//lst[@name='highlighting']/lst[@name='103']/arr[@name='text']/str='This <em>document</em> contains # special characters, while the other <em>document</em> contains the same #'");
} }
@ -262,11 +271,17 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
assertU(adoc("id", "10", "text", "This is a sentence just under seventy chars in length blah blah. Next sentence is here.")); assertU(adoc("id", "10", "text", "This is a sentence just under seventy chars in length blah blah. Next sentence is here."));
assertU(commit()); assertU(commit());
assertQ("default fragsize", assertQ("default fragsize",
req("q", "text:seventy", "hl", "true"), req("q", "text:seventy", "hl", "true", "hl.fragsizeIsMinimum", "true"),
"//lst[@name='highlighting']/lst[@name='10']/arr[@name='text']/str='This is a sentence just under <em>seventy</em> chars in length blah blah. Next sentence is here.'"); "//lst[@name='highlighting']/lst[@name='10']/arr[@name='text']/str='This is a sentence just under <em>seventy</em> chars in length blah blah. Next sentence is here.'");
assertQ("smaller fragsize", assertQ("default fragsize",
req("q", "text:seventy", "hl", "true", "hl.fragsize", "60"), // a bit smaller req("q", "text:seventy", "hl", "true", "hl.fragsizeIsMinimum", "true", "hl.fragsize", "60"),
"//lst[@name='highlighting']/lst[@name='10']/arr[@name='text']/str='This is a sentence just under <em>seventy</em> chars in length blah blah. '"); "//lst[@name='highlighting']/lst[@name='10']/arr[@name='text']/str='This is a sentence just under <em>seventy</em> chars in length blah blah. '");
assertQ("smaller fragsize",
req("q", "text:seventy", "hl", "true"),
"//lst[@name='highlighting']/lst[@name='10']/arr[@name='text']/str='This is a sentence just under <em>seventy</em> chars in length blah blah. '");
assertQ("default fragsize",
req("q", "text:seventy", "hl", "true", "hl.fragsize", "90"),
"//lst[@name='highlighting']/lst[@name='10']/arr[@name='text']/str='This is a sentence just under <em>seventy</em> chars in length blah blah. Next sentence is here.'");
} }
public void testEncoder() { public void testEncoder() {

View File

@ -225,6 +225,14 @@ By default, the Unified Highlighter will usually pick the right offset source (s
+ +
The offset source can be explicitly configured to one of: `ANALYSIS`, `POSTINGS`, `POSTINGS_WITH_TERM_VECTORS`, or `TERM_VECTORS`. The offset source can be explicitly configured to one of: `ANALYSIS`, `POSTINGS`, `POSTINGS_WITH_TERM_VECTORS`, or `TERM_VECTORS`.
`hl.fragAlignRatio`::
Fragment alignment can influence where the match in a passage is positioned. This floating point value is used to break the `hl.fragsize` around the match. The default value of `0.5` means to align the match to the middle. A value of `0.0` would mean to align the match to the left, while `1.0` to align it to the right. _Note: there are situations where the requested alignment is not plausible. This depends on the length of the match, the used breakiterator and the text content around the match._
`hl.fragsizeIsMinimum`::
By default this parameter is `false`. In this case the highlighter will accept a shorter fragment than `hl.fragsize`, if that is closer to the target than what the next longer choice would be.
+
When treating `hl.fragsize` as a strict minimum length is acceptable, turning this on has some performance benefits to consider.
`hl.tag.ellipsis`:: `hl.tag.ellipsis`::
By default, each snippet is returned as a separate value (as is done with the other highlighters). Set this parameter to instead return one string with this text as the delimiter. _Note: this is likely to be removed in the future._ By default, each snippet is returned as a separate value (as is done with the other highlighters). Set this parameter to instead return one string with this text as the delimiter. _Note: this is likely to be removed in the future._

View File

@ -47,6 +47,8 @@ public interface HighlightParams {
// sizing // sizing
public static final String FRAGSIZE = HIGHLIGHT+".fragsize"; // OH, FVH, UH public static final String FRAGSIZE = HIGHLIGHT+".fragsize"; // OH, FVH, UH
public static final String FRAGSIZEISMINIMUM = HIGHLIGHT+".fragsizeIsMinimum"; // UH
public static final String FRAGALIGNRATIO = HIGHLIGHT+".fragAlignRatio"; // UH
public static final String FRAGMENTER = HIGHLIGHT+".fragmenter"; // OH public static final String FRAGMENTER = HIGHLIGHT+".fragmenter"; // OH
public static final String INCREMENT = HIGHLIGHT+".increment"; // OH public static final String INCREMENT = HIGHLIGHT+".increment"; // OH
public static final String REGEX = "regex"; // OH public static final String REGEX = "regex"; // OH