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

This commit is contained in:
Dawid Weiss 2019-12-31 10:08:07 +01:00
commit a40b3e755b
34 changed files with 267 additions and 243 deletions

View File

@ -1357,7 +1357,7 @@ groups:
title: Setup your mail client for PGP
description: |
The announce mail to `announce@apache.org` should be cryptographically signed.
Make sure you have a PGP enabled email client with your apache ket installed.
Make sure you have a PGP enabled email client with your apache key installed.
There are plugins for popular email programs, as well as browser plugins for webmail.
See links for help on how to setup your email client for PGP.
links:
@ -1370,7 +1370,7 @@ groups:
id: announce_lucene_sig
title: Announce the Lucene release (announce@a.o)
description: |
(( template=announce_solr_sign_mail ))
(( template=announce_lucene_sign_mail ))
- !Todo
id: announce_solr_sig
title: Announce the Solr release (announce@a.o)
@ -1667,7 +1667,7 @@ groups:
*Solr*
. Open https://issues.apache.org/jira/issues/?jql=project+=+LUCENE+AND+resolution=Unresolved+AND+fixVersion={{ release_version }}
. Open https://issues.apache.org/jira/issues/?jql=project+=+SOLR+AND+resolution=Unresolved+AND+fixVersion={{ release_version }}
. In the `Tools` menu, start a bulk change - operation="Edit issues"
. Identify issues that *are included* in the release, but are unresolved e.g. due to being REOPENED. These shall *not* be bulk changed!
. Check the box next to `Change Fix Version/s` and in the dropdown `Find and remove these`, selecting v {{ release_version }}
@ -1675,7 +1675,7 @@ groups:
. Click `Next`, review the changes and click `Confirm`
links:
- https://issues.apache.org/jira/issues/?jql=project+=+LUCENE+AND+resolution=Unresolved+AND+fixVersion={{ release_version }}
- https://issues.apache.org/jira/issues/?jql=project+=+LUCENE+AND+resolution=Unresolved+AND+fixVersion={{ release_version }}
- https://issues.apache.org/jira/issues/?jql=project+=+SOLR+AND+resolution=Unresolved+AND+fixVersion={{ release_version }}
- !Todo
id: jira_clear_security
title: Clear Security Level of Public Solr JIRA Issues

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -4,7 +4,7 @@
/com.adobe.xmp/xmpcore = 5.1.3
com.carrotsearch.randomizedtesting.version = 2.7.5
com.carrotsearch.randomizedtesting.version = 2.7.6
/com.carrotsearch.randomizedtesting/junit4-ant = ${com.carrotsearch.randomizedtesting.version}
/com.carrotsearch.randomizedtesting/randomizedtesting-runner = ${com.carrotsearch.randomizedtesting.version}

View File

@ -1 +0,0 @@
967e89a9a46a5b15ce8bef199f01be4830af1a31

View File

@ -0,0 +1 @@
17894fe98cce53f9bd994044d9bea7bd7801cafa

View File

@ -182,6 +182,8 @@ Improvements
* SOLR-13808: filter in BoolQParser and {"bool":{"filter":..}} in Query DSL are cached by default (Mikhail Khludnev)
* SOLR-14129: Reuse Jackson ObjectMapper in AuditLoggerPlugin (janhoy)
Optimizations
---------------------
(No changes)
@ -195,6 +197,8 @@ Bug Fixes
* SOLR-14106: Cleanup Jetty SslContextFactory usage (Ryan Rockenbaugh, Jan Hoydahl, Kevin Risden)
* SOLR-14109: Always log to stdout from server/scripts/cloud-scripts/zkcli.{bat|sh} (janhoy)
Other Changes
---------------------

View File

@ -283,14 +283,15 @@ public abstract class AuditLoggerPlugin implements Closeable, Runnable, SolrInfo
* Event formatter that returns event as JSON string
*/
public static class JSONAuditEventFormatter implements AuditEventFormatter {
private static ObjectMapper mapper = new ObjectMapper()
.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false)
.setSerializationInclusion(Include.NON_NULL);
/**
* Formats an audit event as a JSON string
*/
@Override
public String formatEvent(AuditEvent event) {
ObjectMapper mapper = new ObjectMapper();
mapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false);
mapper.setSerializationInclusion(Include.NON_NULL);
try {
StringWriter sw = new StringWriter();
mapper.writeValue(sw, event);

View File

@ -1 +0,0 @@
e044cdd1aeaafe12df7808a3d9295a03a2717f12

View File

@ -0,0 +1 @@
38416b709b9d7604cd2b65e5e032b61b5d32e9f2

View File

@ -1 +0,0 @@
967e89a9a46a5b15ce8bef199f01be4830af1a31

View File

@ -0,0 +1 @@
17894fe98cce53f9bd994044d9bea7bd7801cafa

View File

@ -9,11 +9,7 @@ REM Find location of this script
set SDIR=%~dp0
if "%SDIR:~-1%"=="\" set SDIR=%SDIR:~0,-1%
if defined LOG4J_PROPS (
set "LOG4J_CONFIG=file:///%LOG4J_PROPS%"
) else (
set "LOG4J_CONFIG=file:///%SDIR%\..\..\resources\log4j2-console.xml"
)
REM Settings for ZK ACL
REM set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider ^

View File

@ -9,11 +9,7 @@ JVM="java"
sdir="`dirname \"$0\"`"
if [ -n "$LOG4J_PROPS" ]; then
log4j_config="file:$LOG4J_PROPS"
else
log4j_config="file:$sdir/../../resources/log4j2-console.xml"
fi
# Settings for ZK ACL
#SOLR_ZK_CREDS_AND_ACLS="-DzkACLProvider=org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider \

View File

@ -23,7 +23,7 @@ This page highlights the biggest changes, including new features you may want to
== Solr 8 Upgrade Planning
Before staring an upgrade to Solr 8, please take the time to review all information about changes from the version you are currently on up to Solr 8.
Before starting an upgrade to Solr 8, please take the time to review all information about changes from the version you are currently on up to Solr 8.
You should also consider all changes that have been made to Solr in any version you have not upgraded to already. For example, if you are currently using Solr 7.4, you should review changes made in all subsequent 7.x releases in addition to changes for 8.0.

View File

@ -23,7 +23,7 @@ This page highlights the biggest changes, including new features you may want to
== Solr 9 Upgrade Planning
Before staring an upgrade to Solr 9, please take the time to review all information about changes from the version you are currently on up to Solr 9.
Before starting an upgrade to Solr 9, please take the time to review all information about changes from the version you are currently on up to Solr 9.
You should also consider all changes that have been made to Solr in any version you have not upgraded to already. For example, if you are currently using Solr 8.1, you should review changes made in all subsequent 8.x releases in addition to changes for 9.0.

View File

@ -38,7 +38,7 @@ If you are upgrading from 7.x, see the section <<Upgrading from 7.x Releases>> b
=== Solr 8.4
See the https://cwiki.apache.org/confluence/display/SOLR/ReleaseNote84[8.4 Release Notes]
for an overview of the main new features of Solr 8.3.
for an overview of the main new features of Solr 8.4.
When upgrading to 8.4.x users should be aware of the following major changes from 8.3.