Merge branch 'master' of github.com:apache/lucene-solr

This commit is contained in:
noblepaul 2020-09-14 18:47:20 +10:00
commit a8099d6367
17 changed files with 159 additions and 152 deletions

View File

@ -182,6 +182,10 @@ API Changes
is now public, making it easier for applications to decode facet is now public, making it easier for applications to decode facet
ordinals into their corresponding labels (Ankur) ordinals into their corresponding labels (Ankur)
* LUCENE-9515: IndexingChain now accepts individual primitives rather than a
DocumentsWriterPerThread instance in order to create a new DocConsumer.
(Simon Willnauer)
New Features New Features
--------------------- ---------------------

View File

@ -19,6 +19,7 @@ package org.apache.lucene.index;
import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.ByteBlockPool; import org.apache.lucene.util.ByteBlockPool;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
/** /**
@ -28,6 +29,10 @@ import org.apache.lucene.util.ByteBlockPool;
*/ */
final class ByteSliceWriter extends DataOutput { final class ByteSliceWriter extends DataOutput {
/* Initial chunks size of the shared byte[] blocks used to
store postings data */
private final static int BYTE_BLOCK_NOT_MASK = ~BYTE_BLOCK_MASK;
private byte[] slice; private byte[] slice;
private int upto; private int upto;
private final ByteBlockPool pool; private final ByteBlockPool pool;
@ -80,6 +85,6 @@ final class ByteSliceWriter extends DataOutput {
} }
public int getAddress() { public int getAddress() {
return upto + (offset0 & DocumentsWriterPerThread.BYTE_BLOCK_NOT_MASK); return upto + (offset0 & BYTE_BLOCK_NOT_MASK);
} }
} }

View File

@ -25,6 +25,7 @@ import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.function.Consumer;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.TokenStream;
@ -40,20 +41,24 @@ import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Sort; import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField; import org.apache.lucene.search.SortField;
import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash.MaxBytesLengthExceededException; import org.apache.lucene.util.BytesRefHash.MaxBytesLengthExceededException;
import org.apache.lucene.util.Counter; import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.IntBlockPool;
import org.apache.lucene.util.RamUsageEstimator; import org.apache.lucene.util.RamUsageEstimator;
/** Default general purpose indexing chain, which handles /** Default general purpose indexing chain, which handles
* indexing all types of fields. */ * indexing all types of fields. */
final class DefaultIndexingChain extends DocConsumer { final class DefaultIndexingChain extends DocConsumer {
final Counter bytesUsed = Counter.newCounter(); final Counter bytesUsed = Counter.newCounter();
final DocumentsWriterPerThread docWriter;
final FieldInfos.Builder fieldInfos; final FieldInfos.Builder fieldInfos;
// Writes postings and term vectors: // Writes postings and term vectors:
@ -73,21 +78,38 @@ final class DefaultIndexingChain extends DocConsumer {
// Holds fields seen in each document // Holds fields seen in each document
private PerField[] fields = new PerField[1]; private PerField[] fields = new PerField[1];
private final InfoStream infoStream; private final InfoStream infoStream;
private final ByteBlockPool.Allocator byteBlockAllocator;
private final LiveIndexWriterConfig indexWriterConfig;
private final int indexCreatedVersionMajor;
private final Consumer<Throwable> abortingExceptionConsumer;
private boolean hasHitAbortingException;
DefaultIndexingChain(DocumentsWriterPerThread docWriter) { DefaultIndexingChain(int indexCreatedVersionMajor, SegmentInfo segmentInfo, Directory directory, FieldInfos.Builder fieldInfos, LiveIndexWriterConfig indexWriterConfig,
this.docWriter = docWriter; Consumer<Throwable> abortingExceptionConsumer) {
this.fieldInfos = docWriter.getFieldInfosBuilder(); this.indexCreatedVersionMajor = indexCreatedVersionMajor;
this.infoStream = docWriter.getIndexWriterConfig().getInfoStream(); byteBlockAllocator = new ByteBlockPool.DirectTrackingAllocator(bytesUsed);
IntBlockPool.Allocator intBlockAllocator = new IntBlockAllocator(bytesUsed);
this.indexWriterConfig = indexWriterConfig;
assert segmentInfo.getIndexSort() == indexWriterConfig.getIndexSort();
this.fieldInfos = fieldInfos;
this.infoStream = indexWriterConfig.getInfoStream();
this.abortingExceptionConsumer = abortingExceptionConsumer;
final TermsHash termVectorsWriter; final TermsHash termVectorsWriter;
if (docWriter.getSegmentInfo().getIndexSort() == null) { if (segmentInfo.getIndexSort() == null) {
storedFieldsConsumer = new StoredFieldsConsumer(docWriter.codec, docWriter.directory, docWriter.getSegmentInfo()); storedFieldsConsumer = new StoredFieldsConsumer(indexWriterConfig.getCodec(), directory, segmentInfo);
termVectorsWriter = new TermVectorsConsumer(docWriter); termVectorsWriter = new TermVectorsConsumer(intBlockAllocator, byteBlockAllocator, directory, segmentInfo, indexWriterConfig.getCodec());
} else { } else {
storedFieldsConsumer = new SortingStoredFieldsConsumer(docWriter.codec, docWriter.directory, docWriter.getSegmentInfo()); storedFieldsConsumer = new SortingStoredFieldsConsumer(indexWriterConfig.getCodec(), directory, segmentInfo);
termVectorsWriter = new SortingTermVectorsConsumer(docWriter); termVectorsWriter = new SortingTermVectorsConsumer(intBlockAllocator, byteBlockAllocator, directory, segmentInfo, indexWriterConfig.getCodec());
} }
termsHash = new FreqProxTermsWriter(docWriter, bytesUsed, termVectorsWriter); termsHash = new FreqProxTermsWriter(intBlockAllocator, byteBlockAllocator, bytesUsed, termVectorsWriter);
}
private void onAbortingException(Throwable th) {
assert th != null;
this.hasHitAbortingException = true;
abortingExceptionConsumer.accept(th);
} }
private LeafReader getDocValuesLeafReader() { private LeafReader getDocValuesLeafReader() {
@ -247,7 +269,7 @@ final class DefaultIndexingChain extends DocConsumer {
// FreqProxTermsWriter does this with // FreqProxTermsWriter does this with
// FieldInfo.storePayload. // FieldInfo.storePayload.
t0 = System.nanoTime(); t0 = System.nanoTime();
docWriter.codec.fieldInfosFormat().write(state.directory, state.segmentInfo, "", state.fieldInfos, IOContext.DEFAULT); indexWriterConfig.getCodec().fieldInfosFormat().write(state.directory, state.segmentInfo, "", state.fieldInfos, IOContext.DEFAULT);
if (infoStream.isEnabled("IW")) { if (infoStream.isEnabled("IW")) {
infoStream.message("IW", ((System.nanoTime()-t0)/1000000) + " msec to write fieldInfos"); infoStream.message("IW", ((System.nanoTime()-t0)/1000000) + " msec to write fieldInfos");
} }
@ -423,7 +445,7 @@ final class DefaultIndexingChain extends DocConsumer {
try { try {
storedFieldsConsumer.startDocument(docID); storedFieldsConsumer.startDocument(docID);
} catch (Throwable th) { } catch (Throwable th) {
docWriter.onAbortingException(th); onAbortingException(th);
throw th; throw th;
} }
} }
@ -434,7 +456,7 @@ final class DefaultIndexingChain extends DocConsumer {
try { try {
storedFieldsConsumer.finishDocument(); storedFieldsConsumer.finishDocument();
} catch (Throwable th) { } catch (Throwable th) {
docWriter.onAbortingException(th); onAbortingException(th);
throw th; throw th;
} }
} }
@ -463,7 +485,7 @@ final class DefaultIndexingChain extends DocConsumer {
fieldCount = processField(docID, field, fieldGen, fieldCount); fieldCount = processField(docID, field, fieldGen, fieldCount);
} }
} finally { } finally {
if (docWriter.hasHitAbortingException() == false) { if (hasHitAbortingException == false) {
// Finish each indexed field name seen in the document: // Finish each indexed field name seen in the document:
for (int i=0;i<fieldCount;i++) { for (int i=0;i<fieldCount;i++) {
fields[i].finish(docID); fields[i].finish(docID);
@ -477,7 +499,7 @@ final class DefaultIndexingChain extends DocConsumer {
} catch (Throwable th) { } catch (Throwable th) {
// Must abort, on the possibility that on-disk term // Must abort, on the possibility that on-disk term
// vectors are now corrupt: // vectors are now corrupt:
docWriter.onAbortingException(th); abortingExceptionConsumer.accept(th);
throw th; throw th;
} }
} }
@ -519,7 +541,7 @@ final class DefaultIndexingChain extends DocConsumer {
try { try {
storedFieldsConsumer.writeField(fp.fieldInfo, field); storedFieldsConsumer.writeField(fp.fieldInfo, field);
} catch (Throwable th) { } catch (Throwable th) {
docWriter.onAbortingException(th); onAbortingException(th);
throw th; throw th;
} }
} }
@ -580,7 +602,7 @@ final class DefaultIndexingChain extends DocConsumer {
fp.fieldInfo.setPointDimensions(pointDimensionCount, pointIndexDimensionCount, dimensionNumBytes); fp.fieldInfo.setPointDimensions(pointDimensionCount, pointIndexDimensionCount, dimensionNumBytes);
if (fp.pointValuesWriter == null) { if (fp.pointValuesWriter == null) {
fp.pointValuesWriter = new PointValuesWriter(docWriter.byteBlockAllocator, bytesUsed, fp.fieldInfo); fp.pointValuesWriter = new PointValuesWriter(byteBlockAllocator, bytesUsed, fp.fieldInfo);
} }
fp.pointValuesWriter.addPackedValue(docID, field.binaryValue()); fp.pointValuesWriter.addPackedValue(docID, field.binaryValue());
} }
@ -647,8 +669,8 @@ final class DefaultIndexingChain extends DocConsumer {
// This is the first time we are seeing this field indexed with doc values, so we // This is the first time we are seeing this field indexed with doc values, so we
// now record the DV type so that any future attempt to (illegally) change // now record the DV type so that any future attempt to (illegally) change
// the DV type of this field, will throw an IllegalArgExc: // the DV type of this field, will throw an IllegalArgExc:
if (docWriter.getSegmentInfo().getIndexSort() != null) { if (indexWriterConfig.getIndexSort() != null) {
final Sort indexSort = docWriter.getSegmentInfo().getIndexSort(); final Sort indexSort = indexWriterConfig.getIndexSort();
validateIndexSortDVType(indexSort, fp.fieldInfo.name, dvType); validateIndexSortDVType(indexSort, fp.fieldInfo.name, dvType);
} }
fieldInfos.globalFieldNumbers.setDocValuesType(fp.fieldInfo.number, fp.fieldInfo.name, dvType); fieldInfos.globalFieldNumbers.setDocValuesType(fp.fieldInfo.number, fp.fieldInfo.name, dvType);
@ -735,8 +757,7 @@ final class DefaultIndexingChain extends DocConsumer {
attributes.forEach((k, v) -> fi.putAttribute(k, v)); attributes.forEach((k, v) -> fi.putAttribute(k, v));
} }
LiveIndexWriterConfig indexWriterConfig = docWriter.getIndexWriterConfig(); fp = new PerField(indexCreatedVersionMajor, fi, invert,
fp = new PerField(docWriter.getIndexCreatedVersionMajor(), fi, invert,
indexWriterConfig.getSimilarity(), indexWriterConfig.getInfoStream(), indexWriterConfig.getAnalyzer()); indexWriterConfig.getSimilarity(), indexWriterConfig.getInfoStream(), indexWriterConfig.getAnalyzer());
fp.next = fieldHash[hashPos]; fp.next = fieldHash[hashPos];
fieldHash[hashPos] = fp; fieldHash[hashPos] = fp;
@ -943,14 +964,14 @@ final class DefaultIndexingChain extends DocConsumer {
byte[] prefix = new byte[30]; byte[] prefix = new byte[30];
BytesRef bigTerm = invertState.termAttribute.getBytesRef(); BytesRef bigTerm = invertState.termAttribute.getBytesRef();
System.arraycopy(bigTerm.bytes, bigTerm.offset, prefix, 0, 30); System.arraycopy(bigTerm.bytes, bigTerm.offset, prefix, 0, 30);
String msg = "Document contains at least one immense term in field=\"" + fieldInfo.name + "\" (whose UTF8 encoding is longer than the max length " + DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8 + "), all of which were skipped. Please correct the analyzer to not produce such terms. The prefix of the first immense term is: '" + Arrays.toString(prefix) + "...', original message: " + e.getMessage(); String msg = "Document contains at least one immense term in field=\"" + fieldInfo.name + "\" (whose UTF8 encoding is longer than the max length " + IndexWriter.MAX_TERM_LENGTH + "), all of which were skipped. Please correct the analyzer to not produce such terms. The prefix of the first immense term is: '" + Arrays.toString(prefix) + "...', original message: " + e.getMessage();
if (infoStream.isEnabled("IW")) { if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "ERROR: " + msg); infoStream.message("IW", "ERROR: " + msg);
} }
// Document will be deleted above: // Document will be deleted above:
throw new IllegalArgumentException(msg, e); throw new IllegalArgumentException(msg, e);
} catch (Throwable th) { } catch (Throwable th) {
docWriter.onAbortingException(th); onAbortingException(th);
throw th; throw th;
} }
} }
@ -992,4 +1013,28 @@ final class DefaultIndexingChain extends DocConsumer {
} }
return null; return null;
} }
private static class IntBlockAllocator extends IntBlockPool.Allocator {
private final Counter bytesUsed;
IntBlockAllocator(Counter bytesUsed) {
super(IntBlockPool.INT_BLOCK_SIZE);
this.bytesUsed = bytesUsed;
}
/* Allocate another int[] from the shared pool */
@Override
public int[] getIntBlock() {
int[] b = new int[IntBlockPool.INT_BLOCK_SIZE];
bytesUsed.addAndGet(IntBlockPool.INT_BLOCK_SIZE * Integer.BYTES);
return b;
}
@Override
public void recycleIntBlocks(int[][] blocks, int offset, int length) {
bytesUsed.addAndGet(-(length * (IntBlockPool.INT_BLOCK_SIZE * Integer.BYTES)));
}
}
} }

View File

@ -18,7 +18,6 @@ package org.apache.lucene.index;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.LinkedList; import java.util.LinkedList;
@ -440,7 +439,7 @@ final class DocumentsWriterFlushControl implements Accountable, Closeable {
flushDeletes.set(true); flushDeletes.set(true);
} }
DocumentsWriterPerThread obtainAndLock() throws IOException { DocumentsWriterPerThread obtainAndLock() {
while (closed == false) { while (closed == false) {
final DocumentsWriterPerThread perThread = perThreadPool.getAndLock(); final DocumentsWriterPerThread perThread = perThreadPool.getAndLock();
if (perThread.deleteQueue == documentsWriter.deleteQueue) { if (perThread.deleteQueue == documentsWriter.deleteQueue) {

View File

@ -27,6 +27,7 @@ import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Consumer;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice; import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
@ -38,55 +39,44 @@ import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.Accountable; import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ByteBlockPool.Allocator;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.FixedBitSet; import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.IntBlockPool;
import org.apache.lucene.util.SetOnce; import org.apache.lucene.util.SetOnce;
import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.Version; import org.apache.lucene.util.Version;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
final class DocumentsWriterPerThread implements Accountable { final class DocumentsWriterPerThread implements Accountable {
LiveIndexWriterConfig getIndexWriterConfig() {
return indexWriterConfig;
}
/** /**
* The IndexingChain must define the {@link #getChain(DocumentsWriterPerThread)} method * The IndexingChain must define the {@link #getChain(int, SegmentInfo, Directory, FieldInfos.Builder, LiveIndexWriterConfig, Consumer)} method
* which returns the DocConsumer that the DocumentsWriter calls to process the * which returns the DocConsumer that the DocumentsWriter calls to process the
* documents. * documents.
*/ */
abstract static class IndexingChain { abstract static class IndexingChain {
abstract DocConsumer getChain(DocumentsWriterPerThread documentsWriterPerThread) throws IOException; abstract DocConsumer getChain(int indexCreatedVersionMajor, SegmentInfo segmentInfo, Directory directory,
FieldInfos.Builder fieldInfos, LiveIndexWriterConfig indexWriterConfig,
Consumer<Throwable> abortingExceptionConsumer);
} }
private Throwable abortingException; private Throwable abortingException;
final void onAbortingException(Throwable throwable) { private void onAbortingException(Throwable throwable) {
assert throwable != null : "aborting exception must not be null";
assert abortingException == null: "aborting exception has already been set"; assert abortingException == null: "aborting exception has already been set";
abortingException = throwable; abortingException = throwable;
} }
final boolean hasHitAbortingException() {
return abortingException != null;
}
final boolean isAborted() { final boolean isAborted() {
return aborted; return aborted;
} }
static final IndexingChain defaultIndexingChain = new IndexingChain() { static final IndexingChain defaultIndexingChain = new IndexingChain() {
@Override @Override
DocConsumer getChain(DocumentsWriterPerThread documentsWriterPerThread) { DocConsumer getChain(int indexCreatedVersionMajor, SegmentInfo segmentInfo, Directory directory,
return new DefaultIndexingChain(documentsWriterPerThread); FieldInfos.Builder fieldInfos, LiveIndexWriterConfig indexWriterConfig,
Consumer<Throwable> abortingExceptionConsumer) {
return new DefaultIndexingChain(indexCreatedVersionMajor, segmentInfo, directory, fieldInfos, indexWriterConfig, abortingExceptionConsumer);
} }
}; };
@ -135,7 +125,6 @@ final class DocumentsWriterPerThread implements Accountable {
final Codec codec; final Codec codec;
final TrackingDirectoryWrapper directory; final TrackingDirectoryWrapper directory;
private final DocConsumer consumer; private final DocConsumer consumer;
private final Counter bytesUsed;
// Updates for our still-in-RAM (to be flushed next) segment // Updates for our still-in-RAM (to be flushed next) segment
private final BufferedUpdates pendingUpdates; private final BufferedUpdates pendingUpdates;
@ -151,29 +140,23 @@ final class DocumentsWriterPerThread implements Accountable {
final DocumentsWriterDeleteQueue deleteQueue; final DocumentsWriterDeleteQueue deleteQueue;
private final DeleteSlice deleteSlice; private final DeleteSlice deleteSlice;
private final NumberFormat nf = NumberFormat.getInstance(Locale.ROOT); private final NumberFormat nf = NumberFormat.getInstance(Locale.ROOT);
final Allocator byteBlockAllocator;
final IntBlockPool.Allocator intBlockAllocator;
private final AtomicLong pendingNumDocs; private final AtomicLong pendingNumDocs;
private final LiveIndexWriterConfig indexWriterConfig; private final LiveIndexWriterConfig indexWriterConfig;
private final boolean enableTestPoints; private final boolean enableTestPoints;
private final int indexVersionCreated;
private final ReentrantLock lock = new ReentrantLock(); private final ReentrantLock lock = new ReentrantLock();
private int[] deleteDocIDs = new int[0]; private int[] deleteDocIDs = new int[0];
private int numDeletedDocIds = 0; private int numDeletedDocIds = 0;
DocumentsWriterPerThread(int indexVersionCreated, String segmentName, Directory directoryOrig, Directory directory,
DocumentsWriterPerThread(int indexVersionCreated, String segmentName, Directory directoryOrig, Directory directory, LiveIndexWriterConfig indexWriterConfig, DocumentsWriterDeleteQueue deleteQueue, LiveIndexWriterConfig indexWriterConfig, DocumentsWriterDeleteQueue deleteQueue,
FieldInfos.Builder fieldInfos, AtomicLong pendingNumDocs, boolean enableTestPoints) throws IOException { FieldInfos.Builder fieldInfos, AtomicLong pendingNumDocs, boolean enableTestPoints) {
this.directory = new TrackingDirectoryWrapper(directory); this.directory = new TrackingDirectoryWrapper(directory);
this.fieldInfos = fieldInfos; this.fieldInfos = fieldInfos;
this.indexWriterConfig = indexWriterConfig; this.indexWriterConfig = indexWriterConfig;
this.infoStream = indexWriterConfig.getInfoStream(); this.infoStream = indexWriterConfig.getInfoStream();
this.codec = indexWriterConfig.getCodec(); this.codec = indexWriterConfig.getCodec();
this.pendingNumDocs = pendingNumDocs; this.pendingNumDocs = pendingNumDocs;
bytesUsed = Counter.newCounter();
byteBlockAllocator = new DirectTrackingAllocator(bytesUsed);
pendingUpdates = new BufferedUpdates(segmentName); pendingUpdates = new BufferedUpdates(segmentName);
intBlockAllocator = new IntBlockAllocator(bytesUsed);
this.deleteQueue = Objects.requireNonNull(deleteQueue); this.deleteQueue = Objects.requireNonNull(deleteQueue);
assert numDocsInRAM == 0 : "num docs " + numDocsInRAM; assert numDocsInRAM == 0 : "num docs " + numDocsInRAM;
deleteSlice = deleteQueue.newSlice(); deleteSlice = deleteQueue.newSlice();
@ -184,18 +167,7 @@ final class DocumentsWriterPerThread implements Accountable {
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segmentName + " delQueue=" + deleteQueue); infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segmentName + " delQueue=" + deleteQueue);
} }
this.enableTestPoints = enableTestPoints; this.enableTestPoints = enableTestPoints;
this.indexVersionCreated = indexVersionCreated; consumer = indexWriterConfig.getIndexingChain().getChain(indexVersionCreated, segmentInfo, this.directory, fieldInfos, indexWriterConfig, this::onAbortingException);
// this should be the last call in the ctor
// it really sucks that we need to pull this within the ctor and pass this ref to the chain!
consumer = indexWriterConfig.getIndexingChain().getChain(this);
}
FieldInfos.Builder getFieldInfosBuilder() {
return fieldInfos;
}
int getIndexCreatedVersionMajor() {
return indexVersionCreated;
} }
final void testPoint(String message) { final void testPoint(String message) {
@ -218,7 +190,7 @@ final class DocumentsWriterPerThread implements Accountable {
long updateDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs, DocumentsWriterDeleteQueue.Node<?> deleteNode, DocumentsWriter.FlushNotifications flushNotifications) throws IOException { long updateDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs, DocumentsWriterDeleteQueue.Node<?> deleteNode, DocumentsWriter.FlushNotifications flushNotifications) throws IOException {
try { try {
testPoint("DocumentsWriterPerThread addDocuments start"); testPoint("DocumentsWriterPerThread addDocuments start");
assert hasHitAbortingException() == false: "DWPT has hit aborting exception but is still indexing"; assert abortingException == null: "DWPT has hit aborting exception but is still indexing";
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) { if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + deleteNode + " docID=" + numDocsInRAM + " seg=" + segmentInfo.name); infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + deleteNode + " docID=" + numDocsInRAM + " seg=" + segmentInfo.name);
} }
@ -290,12 +262,10 @@ final class DocumentsWriterPerThread implements Accountable {
private void deleteLastDocs(int docCount) { private void deleteLastDocs(int docCount) {
int from = numDocsInRAM-docCount; int from = numDocsInRAM-docCount;
int to = numDocsInRAM; int to = numDocsInRAM;
int size = deleteDocIDs.length;
deleteDocIDs = ArrayUtil.grow(deleteDocIDs, numDeletedDocIds + (to-from)); deleteDocIDs = ArrayUtil.grow(deleteDocIDs, numDeletedDocIds + (to-from));
for (int docId = from; docId < to; docId++) { for (int docId = from; docId < to; docId++) {
deleteDocIDs[numDeletedDocIds++] = docId; deleteDocIDs[numDeletedDocIds++] = docId;
} }
bytesUsed.addAndGet((deleteDocIDs.length - size) * Integer.BYTES);
// NOTE: we do not trigger flush here. This is // NOTE: we do not trigger flush here. This is
// potentially a RAM leak, if you have an app that tries // potentially a RAM leak, if you have an app that tries
// to add docs but every single doc always hits a // to add docs but every single doc always hits a
@ -354,9 +324,7 @@ final class DocumentsWriterPerThread implements Accountable {
flushState.liveDocs.clear(deleteDocIDs[i]); flushState.liveDocs.clear(deleteDocIDs[i]);
} }
flushState.delCountOnFlush = numDeletedDocIds; flushState.delCountOnFlush = numDeletedDocIds;
bytesUsed.addAndGet(-(deleteDocIDs.length * Integer.BYTES)); deleteDocIDs = new int[0];
deleteDocIDs = null;
} }
if (aborted) { if (aborted) {
@ -374,8 +342,8 @@ final class DocumentsWriterPerThread implements Accountable {
final Sorter.DocMap sortMap; final Sorter.DocMap sortMap;
try { try {
DocIdSetIterator softDeletedDocs; DocIdSetIterator softDeletedDocs;
if (getIndexWriterConfig().getSoftDeletesField() != null) { if (indexWriterConfig.getSoftDeletesField() != null) {
softDeletedDocs = consumer.getHasDocValues(getIndexWriterConfig().getSoftDeletesField()); softDeletedDocs = consumer.getHasDocValues(indexWriterConfig.getSoftDeletesField());
} else { } else {
softDeletedDocs = null; softDeletedDocs = null;
} }
@ -439,7 +407,7 @@ final class DocumentsWriterPerThread implements Accountable {
} }
private void maybeAbort(String location, DocumentsWriter.FlushNotifications flushNotifications) throws IOException { private void maybeAbort(String location, DocumentsWriter.FlushNotifications flushNotifications) throws IOException {
if (hasHitAbortingException() && aborted == false) { if (abortingException != null && aborted == false) {
// if we are already aborted don't do anything here // if we are already aborted don't do anything here
try { try {
abort(); abort();
@ -483,7 +451,7 @@ final class DocumentsWriterPerThread implements Accountable {
boolean success = false; boolean success = false;
try { try {
if (getIndexWriterConfig().getUseCompoundFile()) { if (indexWriterConfig.getUseCompoundFile()) {
Set<String> originalFiles = newSegment.info.files(); Set<String> originalFiles = newSegment.info.files();
// TODO: like addIndexes, we are relying on createCompoundFile to successfully cleanup... // TODO: like addIndexes, we are relying on createCompoundFile to successfully cleanup...
IndexWriter.createCompoundFile(infoStream, new TrackingDirectoryWrapper(directory), newSegment.info, context, flushNotifications::deleteUnusedFiles); IndexWriter.createCompoundFile(infoStream, new TrackingDirectoryWrapper(directory), newSegment.info, context, flushNotifications::deleteUnusedFiles);
@ -550,7 +518,7 @@ final class DocumentsWriterPerThread implements Accountable {
@Override @Override
public long ramBytesUsed() { public long ramBytesUsed() {
return bytesUsed.get() + pendingUpdates.ramBytesUsed() + consumer.ramBytesUsed(); return (deleteDocIDs.length * Integer.BYTES)+ pendingUpdates.ramBytesUsed() + consumer.ramBytesUsed();
} }
@Override @Override
@ -558,38 +526,6 @@ final class DocumentsWriterPerThread implements Accountable {
return List.of(pendingUpdates, consumer); return List.of(pendingUpdates, consumer);
} }
/* Initial chunks size of the shared byte[] blocks used to
store postings data */
final static int BYTE_BLOCK_NOT_MASK = ~BYTE_BLOCK_MASK;
/* if you increase this, you must fix field cache impl for
* getTerms/getTermsIndex requires <= 32768 */
final static int MAX_TERM_LENGTH_UTF8 = BYTE_BLOCK_SIZE-2;
private static class IntBlockAllocator extends IntBlockPool.Allocator {
private final Counter bytesUsed;
public IntBlockAllocator(Counter bytesUsed) {
super(IntBlockPool.INT_BLOCK_SIZE);
this.bytesUsed = bytesUsed;
}
/* Allocate another int[] from the shared pool */
@Override
public int[] getIntBlock() {
int[] b = new int[IntBlockPool.INT_BLOCK_SIZE];
bytesUsed.addAndGet(IntBlockPool.INT_BLOCK_SIZE * Integer.BYTES);
return b;
}
@Override
public void recycleIntBlocks(int[][] blocks, int offset, int length) {
bytesUsed.addAndGet(-(length * (IntBlockPool.INT_BLOCK_SIZE * Integer.BYTES)));
}
}
@Override @Override
public String toString() { public String toString() {
return "DocumentsWriterPerThread [pendingDeletes=" + pendingUpdates return "DocumentsWriterPerThread [pendingDeletes=" + pendingUpdates

View File

@ -17,7 +17,6 @@
package org.apache.lucene.index; package org.apache.lucene.index;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayDeque; import java.util.ArrayDeque;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
@ -27,9 +26,9 @@ import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.function.Predicate; import java.util.function.Predicate;
import java.util.function.Supplier;
import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.util.IOSupplier;
import org.apache.lucene.util.ThreadInterruptedException; import org.apache.lucene.util.ThreadInterruptedException;
/** /**
@ -49,12 +48,12 @@ final class DocumentsWriterPerThreadPool implements Iterable<DocumentsWriterPerT
private final Set<DocumentsWriterPerThread> dwpts = Collections.newSetFromMap(new IdentityHashMap<>()); private final Set<DocumentsWriterPerThread> dwpts = Collections.newSetFromMap(new IdentityHashMap<>());
private final Deque<DocumentsWriterPerThread> freeList = new ArrayDeque<>(); private final Deque<DocumentsWriterPerThread> freeList = new ArrayDeque<>();
private final IOSupplier<DocumentsWriterPerThread> dwptFactory; private final Supplier<DocumentsWriterPerThread> dwptFactory;
private int takenWriterPermits = 0; private int takenWriterPermits = 0;
private boolean closed; private boolean closed;
DocumentsWriterPerThreadPool(IOSupplier<DocumentsWriterPerThread> dwptFactory) { DocumentsWriterPerThreadPool(Supplier<DocumentsWriterPerThread> dwptFactory) {
this.dwptFactory = dwptFactory; this.dwptFactory = dwptFactory;
} }
@ -86,7 +85,7 @@ final class DocumentsWriterPerThreadPool implements Iterable<DocumentsWriterPerT
* *
* @return a new {@link DocumentsWriterPerThread} * @return a new {@link DocumentsWriterPerThread}
*/ */
private synchronized DocumentsWriterPerThread newWriter() throws IOException { private synchronized DocumentsWriterPerThread newWriter() {
assert takenWriterPermits >= 0; assert takenWriterPermits >= 0;
while (takenWriterPermits > 0) { while (takenWriterPermits > 0) {
// we can't create new DWPTs while not all permits are available // we can't create new DWPTs while not all permits are available
@ -110,7 +109,7 @@ final class DocumentsWriterPerThreadPool implements Iterable<DocumentsWriterPerT
// of items (docs, deletes, DV updates) to most take advantage of concurrency while flushing // of items (docs, deletes, DV updates) to most take advantage of concurrency while flushing
/** This method is used by DocumentsWriter/FlushControl to obtain a DWPT to do an indexing operation (add/updateDocument). */ /** This method is used by DocumentsWriter/FlushControl to obtain a DWPT to do an indexing operation (add/updateDocument). */
DocumentsWriterPerThread getAndLock() throws IOException { DocumentsWriterPerThread getAndLock() {
synchronized (this) { synchronized (this) {
ensureOpen(); ensureOpen();
// Important that we are LIFO here! This way if number of concurrent indexing threads was once high, // Important that we are LIFO here! This way if number of concurrent indexing threads was once high,

View File

@ -30,18 +30,20 @@ import org.apache.lucene.store.ByteBuffersDataInput;
import org.apache.lucene.store.ByteBuffersDataOutput; import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.CollectionUtil;
import org.apache.lucene.util.Counter; import org.apache.lucene.util.Counter;
import org.apache.lucene.util.FixedBitSet; import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntBlockPool;
import org.apache.lucene.util.TimSorter; import org.apache.lucene.util.TimSorter;
import org.apache.lucene.util.automaton.CompiledAutomaton; import org.apache.lucene.util.automaton.CompiledAutomaton;
final class FreqProxTermsWriter extends TermsHash { final class FreqProxTermsWriter extends TermsHash {
FreqProxTermsWriter(DocumentsWriterPerThread docWriter, Counter bytesUsed, TermsHash termVectors) { FreqProxTermsWriter(final IntBlockPool.Allocator intBlockAllocator, final ByteBlockPool.Allocator byteBlockAllocator, Counter bytesUsed, TermsHash termVectors) {
super(docWriter, bytesUsed, termVectors); super(intBlockAllocator, byteBlockAllocator, bytesUsed, termVectors);
} }
private void applyDeletes(SegmentWriteState state, Fields fields) throws IOException { private void applyDeletes(SegmentWriteState state, Fields fields) throws IOException {

View File

@ -82,6 +82,7 @@ import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.Version; import org.apache.lucene.util.Version;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
/** /**
An <code>IndexWriter</code> creates and maintains an index. An <code>IndexWriter</code> creates and maintains an index.
@ -272,7 +273,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
* and a message is printed to infoStream, if set (see {@link * and a message is printed to infoStream, if set (see {@link
* IndexWriterConfig#setInfoStream(InfoStream)}). * IndexWriterConfig#setInfoStream(InfoStream)}).
*/ */
public final static int MAX_TERM_LENGTH = DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8; public final static int MAX_TERM_LENGTH = BYTE_BLOCK_SIZE-2;
/** /**
* Maximum length string for a stored field. * Maximum length string for a stored field.

View File

@ -21,20 +21,24 @@ import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import java.util.Map; import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter; import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo; import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntBlockPool;
final class SortingTermVectorsConsumer extends TermVectorsConsumer { final class SortingTermVectorsConsumer extends TermVectorsConsumer {
TrackingTmpOutputDirectoryWrapper tmpDirectory; TrackingTmpOutputDirectoryWrapper tmpDirectory;
SortingTermVectorsConsumer(DocumentsWriterPerThread docWriter) { SortingTermVectorsConsumer(final IntBlockPool.Allocator intBlockAllocator, final ByteBlockPool.Allocator byteBlockAllocator, Directory directory, SegmentInfo info, Codec codec) {
super(docWriter); super(intBlockAllocator, byteBlockAllocator, directory, info, codec);
} }
@Override @Override
@ -48,10 +52,10 @@ final class SortingTermVectorsConsumer extends TermVectorsConsumer {
} }
return; return;
} }
TermVectorsReader reader = docWriter.codec.termVectorsFormat() TermVectorsReader reader = codec.termVectorsFormat()
.vectorsReader(tmpDirectory, state.segmentInfo, state.fieldInfos, IOContext.DEFAULT); .vectorsReader(tmpDirectory, state.segmentInfo, state.fieldInfos, IOContext.DEFAULT);
TermVectorsReader mergeReader = reader.getMergeInstance(); TermVectorsReader mergeReader = reader.getMergeInstance();
TermVectorsWriter writer = docWriter.codec.termVectorsFormat() TermVectorsWriter writer = codec.termVectorsFormat()
.vectorsWriter(state.directory, state.segmentInfo, IOContext.DEFAULT); .vectorsWriter(state.directory, state.segmentInfo, IOContext.DEFAULT);
try { try {
reader.checkIntegrity(); reader.checkIntegrity();
@ -71,9 +75,9 @@ final class SortingTermVectorsConsumer extends TermVectorsConsumer {
@Override @Override
void initTermVectorsWriter() throws IOException { void initTermVectorsWriter() throws IOException {
if (writer == null) { if (writer == null) {
IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.ramBytesUsed())); IOContext context = new IOContext(new FlushInfo(lastDocID, bytesUsed.get()));
tmpDirectory = new TrackingTmpOutputDirectoryWrapper(docWriter.directory); tmpDirectory = new TrackingTmpOutputDirectoryWrapper(directory);
writer = docWriter.codec.termVectorsFormat().vectorsWriter(tmpDirectory, docWriter.getSegmentInfo(), context); writer = codec.termVectorsFormat().vectorsWriter(tmpDirectory, info, context);
lastDocID = 0; lastDocID = 0;
} }
} }

View File

@ -21,23 +21,29 @@ import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Map; import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.TermVectorsWriter; import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo; import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter; import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntBlockPool;
import org.apache.lucene.util.RamUsageEstimator; import org.apache.lucene.util.RamUsageEstimator;
class TermVectorsConsumer extends TermsHash { class TermVectorsConsumer extends TermsHash {
protected final Directory directory;
protected final SegmentInfo info;
protected final Codec codec;
TermVectorsWriter writer; TermVectorsWriter writer;
/** Scratch term used by TermVectorsConsumerPerField.finishDocument. */ /** Scratch term used by TermVectorsConsumerPerField.finishDocument. */
final BytesRef flushTerm = new BytesRef(); final BytesRef flushTerm = new BytesRef();
final DocumentsWriterPerThread docWriter;
/** Used by TermVectorsConsumerPerField when serializing /** Used by TermVectorsConsumerPerField when serializing
* the term vectors. */ * the term vectors. */
@ -49,9 +55,11 @@ class TermVectorsConsumer extends TermsHash {
int lastDocID; int lastDocID;
private TermVectorsConsumerPerField[] perFields = new TermVectorsConsumerPerField[1]; private TermVectorsConsumerPerField[] perFields = new TermVectorsConsumerPerField[1];
TermVectorsConsumer(DocumentsWriterPerThread docWriter) { TermVectorsConsumer(final IntBlockPool.Allocator intBlockAllocator, final ByteBlockPool.Allocator byteBlockAllocator, Directory directory, SegmentInfo info, Codec codec) {
super(docWriter, Counter.newCounter(), null); super(intBlockAllocator, byteBlockAllocator, Counter.newCounter(), null);
this.docWriter = docWriter; this.directory = directory;
this.info = info;
this.codec = codec;
} }
@Override @Override
@ -85,8 +93,8 @@ class TermVectorsConsumer extends TermsHash {
void initTermVectorsWriter() throws IOException { void initTermVectorsWriter() throws IOException {
if (writer == null) { if (writer == null) {
IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.ramBytesUsed())); IOContext context = new IOContext(new FlushInfo(lastDocID, bytesUsed.get()));
writer = docWriter.codec.termVectorsFormat().vectorsWriter(docWriter.directory, docWriter.getSegmentInfo(), context); writer = codec.termVectorsFormat().vectorsWriter(directory, info, context);
lastDocID = 0; lastDocID = 0;
} }
} }

View File

@ -41,11 +41,11 @@ abstract class TermsHash {
ByteBlockPool termBytePool; ByteBlockPool termBytePool;
final Counter bytesUsed; final Counter bytesUsed;
TermsHash(final DocumentsWriterPerThread docWriter, Counter bytesUsed, TermsHash nextTermsHash) { TermsHash(final IntBlockPool.Allocator intBlockAllocator, final ByteBlockPool.Allocator byteBlockAllocator, Counter bytesUsed, TermsHash nextTermsHash) {
this.nextTermsHash = nextTermsHash; this.nextTermsHash = nextTermsHash;
this.bytesUsed = bytesUsed; this.bytesUsed = bytesUsed;
intPool = new IntBlockPool(docWriter.intBlockAllocator); intPool = new IntBlockPool(intBlockAllocator);
bytePool = new ByteBlockPool(docWriter.byteBlockAllocator); bytePool = new ByteBlockPool(byteBlockAllocator);
if (nextTermsHash != null) { if (nextTermsHash != null) {
// We are primary // We are primary

View File

@ -74,8 +74,6 @@ public class TestDocumentsWriterPerThreadPool extends LuceneTestCase {
fail(); fail();
} catch (AlreadyClosedException e) { } catch (AlreadyClosedException e) {
// fine // fine
} catch (IOException e) {
throw new AssertionError(e);
} }
}); });
t.start(); t.start();

View File

@ -1533,7 +1533,7 @@ public class TestIndexWriter extends LuceneTestCase {
Directory dir = newDirectory(); Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir, new StringSplitAnalyzer()); RandomIndexWriter w = new RandomIndexWriter(random(), dir, new StringSplitAnalyzer());
char[] chars = new char[DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8]; char[] chars = new char[IndexWriter.MAX_TERM_LENGTH];
Arrays.fill(chars, 'x'); Arrays.fill(chars, 'x');
Document hugeDoc = new Document(); Document hugeDoc = new Document();
final String bigTerm = new String(chars); final String bigTerm = new String(chars);
@ -3717,8 +3717,6 @@ public class TestIndexWriter extends LuceneTestCase {
states.add(state::unlock); states.add(state::unlock);
state.deleteQueue.getNextSequenceNumber(); state.deleteQueue.getNextSequenceNumber();
} }
} catch (IOException e) {
throw new AssertionError(e);
} finally { } finally {
IOUtils.closeWhileHandlingException(states); IOUtils.closeWhileHandlingException(states);
} }

View File

@ -22,6 +22,7 @@ import java.lang.reflect.Method;
import java.lang.reflect.Modifier; import java.lang.reflect.Modifier;
import java.util.HashSet; import java.util.HashSet;
import java.util.Set; import java.util.Set;
import java.util.function.Consumer;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
@ -43,12 +44,13 @@ public class TestIndexWriterConfig extends LuceneTestCase {
} }
private static final class MyIndexingChain extends IndexingChain { private static final class MyIndexingChain extends IndexingChain {
// Does not implement anything - used only for type checking on IndexWriterConfig.
@Override @Override
DocConsumer getChain(DocumentsWriterPerThread documentsWriter) { DocConsumer getChain(int indexCreatedVersionMajor, SegmentInfo segmentInfo, Directory directory,
FieldInfos.Builder fieldInfos, LiveIndexWriterConfig indexWriterConfig,
Consumer<Throwable> abortingExceptionConsumer) {
return null; return null;
} }
// Does not implement anything - used only for type checking on IndexWriterConfig.
} }

View File

@ -27,6 +27,7 @@ import java.util.Objects;
import static com.carrotsearch.randomizedtesting.RandomizedTest.*; import static com.carrotsearch.randomizedtesting.RandomizedTest.*;
@LuceneTestCase.AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-9521")
public class TestPassageSelector extends LuceneTestCase { public class TestPassageSelector extends LuceneTestCase {
@Test @Test
public void checkEmptyExtra() { public void checkEmptyExtra() {

View File

@ -15,6 +15,8 @@
* limitations under the License. * limitations under the License.
*/ */
rootProject.name = "lucene-solr"
includeBuild("dev-tools/missing-doclet") includeBuild("dev-tools/missing-doclet")
include "lucene:analysis:common" include "lucene:analysis:common"

View File

@ -30,6 +30,7 @@ public class ConfigSet {
private final String name; private final String name;
private final SolrConfig solrconfig; private final SolrConfig solrconfig;
private IndexSchema schema;
private final SchemaSupplier schemaSupplier; private final SchemaSupplier schemaSupplier;
@ -40,10 +41,11 @@ public class ConfigSet {
@SuppressWarnings({"rawtypes"}) @SuppressWarnings({"rawtypes"})
public ConfigSet(String name, SolrConfig solrConfig, SchemaSupplier indexSchemaSupplier, public ConfigSet(String name, SolrConfig solrConfig, SchemaSupplier indexSchemaSupplier,
NamedList properties, boolean trusted) { NamedList properties, boolean trusted) {
this.name = name; this.name = name;
this.solrconfig = solrConfig; this.solrconfig = solrConfig;
this.schemaSupplier = indexSchemaSupplier; this.schemaSupplier = indexSchemaSupplier;
schema = schemaSupplier.get(true);
this.properties = properties; this.properties = properties;
this.trusted = trusted; this.trusted = trusted;
} }
@ -61,10 +63,11 @@ public class ConfigSet {
* @param forceFetch get a fresh value and not cached value * @param forceFetch get a fresh value and not cached value
*/ */
public IndexSchema getIndexSchema(boolean forceFetch) { public IndexSchema getIndexSchema(boolean forceFetch) {
return schemaSupplier.get(forceFetch); if(forceFetch) schema = schemaSupplier.get(true);
return schema;
} }
public IndexSchema getIndexSchema() { public IndexSchema getIndexSchema() {
return schemaSupplier.get(false); return schema;
} }
@SuppressWarnings({"rawtypes"}) @SuppressWarnings({"rawtypes"})
@ -82,7 +85,7 @@ public class ConfigSet {
* So, we may not be able to update the core if we the schema classes are updated * So, we may not be able to update the core if we the schema classes are updated
* */ * */
interface SchemaSupplier { interface SchemaSupplier {
IndexSchema get(boolean forceFetch); IndexSchema get(boolean forceFetch);
} }
} }