diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestIndexUpgradeBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestIndexUpgradeBackwardsCompatibility.java index 10099a5b5b59..4df8d6dd00c8 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestIndexUpgradeBackwardsCompatibility.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestIndexUpgradeBackwardsCompatibility.java @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Set; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexUpgrader; @@ -131,7 +132,7 @@ public void testUpgradeOldSingleSegmentIndexWithAdditions() throws Exception { // add dummy segments (which are all in current // version) to single segment index MergePolicy mp = random().nextBoolean() ? newLogMergePolicy() : newTieredMergePolicy(); - IndexWriterConfig iwc = new IndexWriterConfig(null).setMergePolicy(mp); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null).setMergePolicy(mp); IndexWriter w = new IndexWriter(directory, iwc); w.addIndexes(ramDir); try (w) { diff --git a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/tasks/TestAddIndexesTask.java b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/tasks/TestAddIndexesTask.java index 841047f629db..298062ddaefe 100644 --- a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/tasks/TestAddIndexesTask.java +++ b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/tasks/TestAddIndexesTask.java @@ -18,6 +18,7 @@ import java.nio.file.Path; import java.util.Properties; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.benchmark.BenchmarkTestCase; import org.apache.lucene.benchmark.byTask.PerfRunData; import org.apache.lucene.benchmark.byTask.utils.Config; @@ -45,7 +46,7 @@ public static void beforeClassAddIndexesTaskTest() throws Exception { inputDir = testDir.resolve("input"); Directory tmpDir = newFSDirectory(inputDir); try { - IndexWriter writer = new IndexWriter(tmpDir, new IndexWriterConfig(null)); + IndexWriter writer = new IndexWriter(tmpDir, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < 10; i++) { writer.addDocument(new Document()); } diff --git a/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java b/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java index bd09d59b69ca..13c517afdb52 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java @@ -16,6 +16,8 @@ */ package org.apache.lucene.index; +import java.io.IOException; + /** * Default {@link FlushPolicy} implementation that flushes new segments based on RAM used and * document count depending on the IndexWriter's {@link IndexWriterConfig}. It also applies pending @@ -52,6 +54,17 @@ && flushOnDocCount() } } + @Override + public void flushWriter( + IndexWriterRAMManager ramManager, + IndexWriterRAMManager.PerWriterIndexWriterRAMManager perWriterRamManager) + throws IOException { + long totalBytes = perWriterRamManager.getTotalBufferBytesUsed(); + if (totalBytes > ramManager.getRamBufferSizeMB() * 1024 * 1024) { + ramManager.flushRoundRobin(); + } + } + private void flushDeletes(DocumentsWriterFlushControl control) { control.setApplyAllDeletes(); if (infoStream.isEnabled("FP")) { diff --git a/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java b/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java index d2abc28dfa2b..a57f6f78fd4b 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java +++ b/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java @@ -16,6 +16,7 @@ */ package org.apache.lucene.index; +import java.io.IOException; import org.apache.lucene.store.Directory; import org.apache.lucene.util.InfoStream; @@ -57,6 +58,17 @@ abstract class FlushPolicy { public abstract void onChange( DocumentsWriterFlushControl control, DocumentsWriterPerThread perThread); + /** + * Chooses which writer should be flushed. Default implementation chooses the writer with most RAM + * usage + * + * @param ramManager the {@link IndexWriterRAMManager} being used to actually flush the writers + */ + public abstract void flushWriter( + IndexWriterRAMManager ramManager, + IndexWriterRAMManager.PerWriterIndexWriterRAMManager perWriterRamManager) + throws IOException; + /** Called by DocumentsWriter to initialize the FlushPolicy */ protected synchronized void init(LiveIndexWriterConfig indexWriterConfig) { this.indexWriterConfig = indexWriterConfig; diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexUpgrader.java b/lucene/core/src/java/org/apache/lucene/index/IndexUpgrader.java index 17de25e0ed48..82a5a862b024 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexUpgrader.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexUpgrader.java @@ -20,6 +20,7 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.util.Collection; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.util.CommandLineUtil; @@ -135,7 +136,7 @@ static IndexUpgrader parseArgs(String[] args) throws IOException { * {@code matchVersion}. The tool refuses to upgrade indexes with multiple commit points. */ public IndexUpgrader(Directory dir) { - this(dir, new IndexWriterConfig(null), false); + this(dir, new IndexWriterConfig((Analyzer) null), false); } /** @@ -145,7 +146,7 @@ public IndexUpgrader(Directory dir) { * be sent to this stream. */ public IndexUpgrader(Directory dir, InfoStream infoStream, boolean deletePriorCommits) { - this(dir, new IndexWriterConfig(null), deletePriorCommits); + this(dir, new IndexWriterConfig((Analyzer) null), deletePriorCommits); if (null != infoStream) { this.iwc.setInfoStream(infoStream); } diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java index 346da8a907ec..eeff44f840ef 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -462,6 +462,8 @@ public void onTicketBacklog() { } }; + private final IndexWriterRAMManager.PerWriterIndexWriterRAMManager indexWriterRAMManager; + /** * Expert: returns a readonly reader, covering all committed as well as un-committed changes to * the index. This provides "near real-time" searching, in that changes made during an IndexWriter @@ -1211,6 +1213,9 @@ public IndexWriter(Directory d, IndexWriterConfig conf) throws IOException { writeLock = null; } } + this.indexWriterRAMManager = + new IndexWriterRAMManager.PerWriterIndexWriterRAMManager( + this, config.getIndexWriterRAMManager()); } /** Confirms that the incoming index sort (if any) matches the existing index sort (if any). */ @@ -1365,6 +1370,7 @@ private void shutdown() throws IOException { */ @Override public void close() throws IOException { + indexWriterRAMManager.removeWriter(); if (config.getCommitOnClose()) { shutdown(); } else { @@ -2445,6 +2451,7 @@ public void rollback() throws IOException { // Ensure that only one thread actually gets to do the // closing, and make sure no commit is also in progress: if (shouldClose(true)) { + indexWriterRAMManager.removeWriter(); rollbackInternal(); } } @@ -6012,6 +6019,7 @@ private long maybeProcessEvents(long seqNo) throws IOException { seqNo = -seqNo; processEvents(true); } + indexWriterRAMManager.flushIfNecessary(config.flushPolicy); return seqNo; } diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java index e20a6371ef2c..835e0e68632f 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java @@ -142,7 +142,21 @@ public IndexWriterConfig() { * problem you should switch to {@link LogByteSizeMergePolicy} or {@link LogDocMergePolicy}. */ public IndexWriterConfig(Analyzer analyzer) { - super(analyzer); + this(analyzer, new IndexWriterRAMManager(IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB)); + } + + /** + * Creates a new config with the provided {@link IndexWriterRAMManager}. If you want to share a + * buffer between multiple {@link IndexWriter}, you will need to use this constructor as {@link + * IndexWriterConfig} maintains a 1:1 relationship with {@link IndexWriter} + */ + public IndexWriterConfig(IndexWriterRAMManager indexWriterRAMManager) { + this(new StandardAnalyzer(), indexWriterRAMManager); + } + + /** Creates a new config with the provided {@link Analyzer} and {@link IndexWriterRAMManager} */ + public IndexWriterConfig(Analyzer analyzer, IndexWriterRAMManager indexWriterRAMManager) { + super(analyzer, indexWriterRAMManager); } /** @@ -393,6 +407,11 @@ public double getRAMBufferSizeMB() { return super.getRAMBufferSizeMB(); } + @Override + public IndexWriterRAMManager getIndexWriterRAMManager() { + return super.getIndexWriterRAMManager(); + } + /** * Information about merges, deletes and a message when maxFieldLength is reached will be printed * to this. Must not be null, but {@link InfoStream#NO_OUTPUT} may be used to suppress output. diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriterRAMManager.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriterRAMManager.java new file mode 100644 index 000000000000..80852cef00c5 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriterRAMManager.java @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.index; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.ReentrantLock; + +/** + * For managing multiple instances of {@link IndexWriter} sharing the same buffer (configured by + * {@link IndexWriterConfig#setRAMBufferSizeMB}) + */ +public class IndexWriterRAMManager { + private final LinkedIdToWriter idToWriter = new LinkedIdToWriter(); + private final AtomicInteger idGenerator = new AtomicInteger(); + private double ramBufferSizeMB; + + /** + * Default constructor + * + * @param ramBufferSizeMB the RAM buffer size to use between all registered {@link IndexWriter} + * instances + */ + IndexWriterRAMManager(double ramBufferSizeMB) { + if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH && ramBufferSizeMB <= 0.0) { + throw new IllegalArgumentException("ramBufferSize should be > 0.0 MB when enabled"); + } + this.ramBufferSizeMB = ramBufferSizeMB; + } + + /** Set the buffer size for this manager */ + public void setRamBufferSizeMB(double ramBufferSizeMB) { + this.ramBufferSizeMB = ramBufferSizeMB; + } + + /** Get the buffer size assigned to this manager */ + public double getRamBufferSizeMB() { + return ramBufferSizeMB; + } + + /** + * Calls {@link IndexWriter#flushNextBuffer()} in a round-robin fashion starting from the first + * writer added that has not been removed yet. Subsequent calls will flush the next writer in line + * and eventually loop back to the beginning. Returns the flushed writer id for testing + */ + public int flushRoundRobin() throws IOException { + return idToWriter.flushRoundRobin(); + } + + /** Registers a writer and returns the associated ID, protected for testing */ + protected int registerWriter(IndexWriter writer) { + int id = idGenerator.incrementAndGet(); + idToWriter.addWriter(writer, id); + return id; + } + + /** Removes a writer given the writer's ide, protected for testing */ + protected void removeWriter(int id) { + idToWriter.removeWriter(id); + } + + private void flushIfNecessary( + FlushPolicy flushPolicy, PerWriterIndexWriterRAMManager perWriterRAMManager) + throws IOException { + if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH) { + flushPolicy.flushWriter(this, perWriterRAMManager); + } + } + + private long updateAndGetCurrentBytesUsed(int id) { + return idToWriter.getTotalRamTracker(id); + } + + /** + * For use in {@link IndexWriter}, manages communication with the {@link IndexWriterRAMManager} + */ + public static class PerWriterIndexWriterRAMManager { + + private final int id; + private final IndexWriterRAMManager manager; + + PerWriterIndexWriterRAMManager(IndexWriter writer, IndexWriterRAMManager manager) { + id = manager.registerWriter(writer); + this.manager = manager; + } + + void removeWriter() { + manager.removeWriter(id); + } + + void flushIfNecessary(FlushPolicy flushPolicy) throws IOException { + manager.flushIfNecessary(flushPolicy, this); + } + + long getTotalBufferBytesUsed() { + return manager.updateAndGetCurrentBytesUsed(id); + } + } + + private static class LinkedIdToWriter { + private final Map idToWriterNode = new HashMap<>(); + private IndexWriterNode first; + private IndexWriterNode last; + private long totalRamTracker; + + private final ReentrantLock lock = new ReentrantLock(); + + // for round-robin flushing + private int lastIdFlushed = -1; + + void addWriter(IndexWriter writer, int id) { + synchronized (lock) { + IndexWriterNode node = new IndexWriterNode(writer, id); + if (idToWriterNode.isEmpty()) { + first = node; + last = node; + } + node.next = first; + last.next = node; + node.prev = last; + last = node; + first.prev = node; + idToWriterNode.put(id, node); + } + } + + void removeWriter(int id) { + synchronized (lock) { + if (idToWriterNode.containsKey(id)) { + IndexWriterNode nodeToRemove = idToWriterNode.remove(id); + totalRamTracker -= nodeToRemove.ram; + if (idToWriterNode.isEmpty()) { + first = null; + last = null; + lastIdFlushed = -1; + return; + } + if (id == lastIdFlushed) { + lastIdFlushed = nodeToRemove.prev.id; + } + nodeToRemove.prev.next = nodeToRemove.next; + nodeToRemove.next.prev = nodeToRemove.prev; + if (nodeToRemove == first) { + first = nodeToRemove.next; + } + if (nodeToRemove == last) { + last = nodeToRemove.prev; + } + } + } + } + + // Returns the writer id that we attempted to flush (for testing purposes) + int flushRoundRobin() throws IOException { + synchronized (lock) { + if (idToWriterNode.isEmpty()) { + return -1; + } + int idToFlush; + if (lastIdFlushed == -1) { + idToFlush = first.id; + } else { + idToFlush = idToWriterNode.get(lastIdFlushed).next.id; + } + idToWriterNode.get(idToFlush).writer.flushNextBuffer(); + lastIdFlushed = idToFlush; + return idToFlush; + } + } + + long getTotalRamTracker(int id) { + synchronized (lock) { + if (idToWriterNode.isEmpty()) { + return 0; + } + if (idToWriterNode.containsKey(id) == false) { + return totalRamTracker; + } + long oldRAMBytesUsed = idToWriterNode.get(id).ram; + long newRAMBytesUsed = idToWriterNode.get(id).writer.ramBytesUsed(); + idToWriterNode.get(id).ram = newRAMBytesUsed; + totalRamTracker += newRAMBytesUsed - oldRAMBytesUsed; + return totalRamTracker; + } + } + + private static class IndexWriterNode { + IndexWriter writer; + int id; + long ram; + IndexWriterNode next; + IndexWriterNode prev; + + IndexWriterNode(IndexWriter writer, int id) { + this.writer = writer; + this.id = id; + this.ram = writer.ramBytesUsed(); + } + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java index c9db6d0c6f66..698e66924529 100644 --- a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java +++ b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java @@ -40,7 +40,6 @@ public class LiveIndexWriterConfig { private final Analyzer analyzer; private volatile int maxBufferedDocs; - private volatile double ramBufferSizeMB; private volatile IndexReaderWarmer mergedSegmentWarmer; // modified by IndexWriterConfig @@ -116,10 +115,15 @@ public class LiveIndexWriterConfig { /** The IndexWriter event listener to record key events * */ protected IndexWriterEventListener eventListener; + /** + * RAM manager associated with this config, can be shared with other configs if sharing buffers + */ + protected final IndexWriterRAMManager indexWriterRAMManager; + // used by IndexWriterConfig - LiveIndexWriterConfig(Analyzer analyzer) { + LiveIndexWriterConfig(Analyzer analyzer, IndexWriterRAMManager indexWriterRAMManager) { this.analyzer = analyzer; - ramBufferSizeMB = IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB; + this.indexWriterRAMManager = indexWriterRAMManager; maxBufferedDocs = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS; mergedSegmentWarmer = null; delPolicy = new KeepOnlyLastCommitDeletionPolicy(); @@ -181,21 +185,23 @@ public Analyzer getAnalyzer() { * ramBufferSize when maxBufferedDocs is already disabled */ public synchronized LiveIndexWriterConfig setRAMBufferSizeMB(double ramBufferSizeMB) { - if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH && ramBufferSizeMB <= 0.0) { - throw new IllegalArgumentException("ramBufferSize should be > 0.0 MB when enabled"); - } if (ramBufferSizeMB == IndexWriterConfig.DISABLE_AUTO_FLUSH && maxBufferedDocs == IndexWriterConfig.DISABLE_AUTO_FLUSH) { throw new IllegalArgumentException( "at least one of ramBufferSize and maxBufferedDocs must be enabled"); } - this.ramBufferSizeMB = ramBufferSizeMB; + indexWriterRAMManager.setRamBufferSizeMB(ramBufferSizeMB); return this; } /** Returns the value set by {@link #setRAMBufferSizeMB(double)} if enabled. */ public double getRAMBufferSizeMB() { - return ramBufferSizeMB; + return indexWriterRAMManager.getRamBufferSizeMB(); + } + + /** Get the {@link IndexWriterRAMManager} associated with this config */ + public IndexWriterRAMManager getIndexWriterRAMManager() { + return indexWriterRAMManager; } /** @@ -220,7 +226,7 @@ public synchronized LiveIndexWriterConfig setMaxBufferedDocs(int maxBufferedDocs throw new IllegalArgumentException("maxBufferedDocs must at least be 2 when enabled"); } if (maxBufferedDocs == IndexWriterConfig.DISABLE_AUTO_FLUSH - && ramBufferSizeMB == IndexWriterConfig.DISABLE_AUTO_FLUSH) { + && indexWriterRAMManager.getRamBufferSizeMB() == IndexWriterConfig.DISABLE_AUTO_FLUSH) { throw new IllegalArgumentException( "at least one of ramBufferSize and maxBufferedDocs must be enabled"); } diff --git a/lucene/core/src/test/org/apache/lucene/TestMergeSchedulerExternal.java b/lucene/core/src/test/org/apache/lucene/TestMergeSchedulerExternal.java index 99e1e8e1fc0b..ae018ae6a504 100644 --- a/lucene/core/src/test/org/apache/lucene/TestMergeSchedulerExternal.java +++ b/lucene/core/src/test/org/apache/lucene/TestMergeSchedulerExternal.java @@ -24,6 +24,7 @@ import java.io.PrintStream; import java.io.PrintWriter; import java.io.StringWriter; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.index.ConcurrentMergeScheduler; @@ -184,7 +185,7 @@ public void testCustomMergeScheduler() throws Exception { // compiles. But ensure that it can be used as well, e.g., no other hidden // dependencies or something. Therefore, don't use any random API ! Directory dir = new ByteBuffersDirectory(); - IndexWriterConfig conf = new IndexWriterConfig(null); + IndexWriterConfig conf = new IndexWriterConfig((Analyzer) null); conf.setMergeScheduler(new ReportingMergeScheduler()); IndexWriter writer = new IndexWriter(dir, conf); writer.addDocument(new Document()); diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java b/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java index 8b90b1d610b1..b946bfcc720a 100644 --- a/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java +++ b/lucene/core/src/test/org/apache/lucene/index/Test2BPostingsBytes.java @@ -19,6 +19,7 @@ import java.util.Arrays; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; import org.apache.lucene.codecs.Codec; @@ -46,9 +47,9 @@ public class Test2BPostingsBytes extends LuceneTestCase { public void test() throws Exception { - IndexWriterConfig defaultConfig = new IndexWriterConfig(null); + IndexWriterConfig defaultConfig = new IndexWriterConfig((Analyzer) null); Codec defaultCodec = defaultConfig.getCodec(); - if ((new IndexWriterConfig(null)).getCodec() instanceof CompressingCodec) { + if ((new IndexWriterConfig((Analyzer) null)).getCodec() instanceof CompressingCodec) { Pattern regex = Pattern.compile("maxDocsPerChunk=(\\d+), blockSize=(\\d+)"); Matcher matcher = regex.matcher(defaultCodec.toString()); assertTrue( @@ -116,7 +117,7 @@ public void test() throws Exception { if (dir2 instanceof MockDirectoryWrapper) { ((MockDirectoryWrapper) dir2).setThrottling(MockDirectoryWrapper.Throttling.NEVER); } - IndexWriter w2 = new IndexWriter(dir2, new IndexWriterConfig(null)); + IndexWriter w2 = new IndexWriter(dir2, new IndexWriterConfig((Analyzer) null)); TestUtil.addIndexesSlowly(w2, subReaders); w2.forceMerge(1); w2.close(); @@ -129,7 +130,7 @@ public void test() throws Exception { if (dir3 instanceof MockDirectoryWrapper) { ((MockDirectoryWrapper) dir3).setThrottling(MockDirectoryWrapper.Throttling.NEVER); } - IndexWriter w3 = new IndexWriter(dir3, new IndexWriterConfig(null)); + IndexWriter w3 = new IndexWriter(dir3, new IndexWriterConfig((Analyzer) null)); TestUtil.addIndexesSlowly(w3, subReaders); w3.forceMerge(1); w3.close(); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFieldReuse.java b/lucene/core/src/test/org/apache/lucene/index/TestFieldReuse.java index 0dabb5b17f8e..fd44334a4ef3 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestFieldReuse.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestFieldReuse.java @@ -108,7 +108,7 @@ public InvertableType invertableType() { public void testIndexWriterActuallyReuses() throws IOException { Directory dir = newDirectory(); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); IndexWriter iw = new IndexWriter(dir, iwc); final MyField field1 = new MyField(); iw.addDocument(Collections.singletonList(field1)); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java index e62ca8235a24..de430a900ccb 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.simpletext.SimpleTextCodec; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; @@ -238,14 +239,14 @@ public void testVirusScannerDoesntCorruptIndex() throws IOException { TestUtil.disableVirusChecker(dir); // add empty commit - new IndexWriter(dir, new IndexWriterConfig(null)).close(); + new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)).close(); // add a trash unreferenced file dir.createOutput("_0.si", IOContext.DEFAULT).close(); // start virus scanner TestUtil.enableVirusChecker(dir); - IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); iw.addDocument(new Document()); // stop virus scanner TestUtil.disableVirusChecker(dir); @@ -258,7 +259,7 @@ public void testNoSegmentsDotGenInflation() throws IOException { Directory dir = newMockDirectory(); // empty commit - new IndexWriter(dir, new IndexWriterConfig(null)).close(); + new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)).close(); SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(1, sis.getGeneration()); @@ -275,7 +276,7 @@ public void testSegmentsInflation() throws IOException { dir.setCheckIndexOnClose(false); // TODO: allow falling back more than one commit // empty commit - new IndexWriter(dir, new IndexWriterConfig(null)).close(); + new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)).close(); SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(1, sis.getGeneration()); @@ -299,7 +300,7 @@ public void testSegmentNameInflation() throws IOException { MockDirectoryWrapper dir = newMockDirectory(); // empty commit - new IndexWriter(dir, new IndexWriterConfig(null)).close(); + new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)).close(); SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(0, sis.counter); @@ -321,7 +322,7 @@ public void testSegmentNameInflation() throws IOException { assertEquals(4, sis.counter); // ensure we write _4 segment next - IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); iw.addDocument(new Document()); iw.commit(); iw.close(); @@ -336,7 +337,7 @@ public void testGenerationInflation() throws IOException { MockDirectoryWrapper dir = newMockDirectory(); // initial commit - IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); iw.addDocument(new Document()); iw.commit(); iw.close(); @@ -365,7 +366,7 @@ public void testTrashyFile() throws IOException { dir.setCheckIndexOnClose(false); // TODO: maybe handle such trash better elsewhere... // empty commit - new IndexWriter(dir, new IndexWriterConfig(null)).close(); + new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)).close(); SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(1, sis.getGeneration()); @@ -384,7 +385,7 @@ public void testTrashyGenFile() throws IOException { MockDirectoryWrapper dir = newMockDirectory(); // initial commit - IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); iw.addDocument(new Document()); iw.commit(); iw.close(); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java index 04adc5035db4..b7ec9cc46342 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java @@ -2054,7 +2054,7 @@ public void testGetCommitData() throws Exception { public void testGetCommitDataFromOldSnapshot() throws Exception { Directory dir = newDirectory(); - IndexWriter writer = new IndexWriter(dir, newSnapshotIndexWriterConfig(null)); + IndexWriter writer = new IndexWriter(dir, newSnapshotIndexWriterConfig((Analyzer) null)); writer.setLiveCommitData( new HashMap() { { @@ -2069,7 +2069,7 @@ public void testGetCommitDataFromOldSnapshot() throws Exception { writer.close(); // Modify the commit data and commit on close so the most recent commit data is different - writer = new IndexWriter(dir, newSnapshotIndexWriterConfig(null)); + writer = new IndexWriter(dir, newSnapshotIndexWriterConfig((Analyzer) null)); writer.setLiveCommitData( new HashMap() { { @@ -2084,7 +2084,7 @@ public void testGetCommitDataFromOldSnapshot() throws Exception { writer = new IndexWriter( dir, - newSnapshotIndexWriterConfig(null) + newSnapshotIndexWriterConfig((Analyzer) null) .setOpenMode(OpenMode.APPEND) .setIndexCommit(indexCommit)); assertEquals("value", getLiveCommitData(writer).get("key")); @@ -2620,7 +2620,7 @@ public void testCloseDuringCommit() throws Exception { final CountDownLatch finishCommit = new CountDownLatch(1); Directory dir = newDirectory(); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); // use an InfoStream that "takes a long time" to commit final IndexWriter iw = RandomIndexWriter.mockIndexWriter( @@ -3216,6 +3216,12 @@ public void onChange( control.setApplyAllDeletes(); } } + + @Override + public void flushWriter( + IndexWriterRAMManager ramManager, + IndexWriterRAMManager.PerWriterIndexWriterRAMManager perWriterRamManager) + throws IOException {} }); try (IndexWriter w = new IndexWriter(dir, indexWriterConfig)) { assertEquals(0, w.docWriter.flushControl.getDeleteBytesUsed()); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java index 7f4258d18fa4..87add0c8f74d 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java @@ -88,6 +88,7 @@ public void testDefaults() throws Exception { getters.add("getUseCompoundFile"); getters.add("isCheckPendingFlushOnUpdate"); getters.add("getSoftDeletesField"); + getters.add("getIndexWriterRAMManager"); for (Method m : IndexWriterConfig.class.getDeclaredMethods()) { if (m.getDeclaringClass() == IndexWriterConfig.class && m.getName().startsWith("get")) { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java index 2f92606ba7d0..bf25f8f8b63e 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java @@ -1994,7 +1994,7 @@ public void close() throws IOException {} }; Directory dir = newMockDirectory(); // we want to ensure we don't leak any locks or file handles - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); iwc.setInfoStream(evilInfoStream); // TODO: cutover to RandomIndexWriter.mockIndexWriter? IndexWriter iw = @@ -2067,7 +2067,7 @@ public void eval(MockDirectoryWrapper dir) throws IOException { } }); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); IndexWriter iw = new IndexWriter(dir, iwc); Document doc = new Document(); for (int i = 0; i < 10; i++) { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java index 48aac56a22d0..ef00d890880b 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.concurrent.CountDownLatch; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.search.IndexSearcher; @@ -47,7 +48,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase { @Monster("takes over two hours") public void testExactlyAtTrueLimit() throws Exception { Directory dir = newFSDirectory(createTempDir("2BDocs3")); - IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); Document doc = new Document(); doc.add(newStringField("field", "text", Field.Store.NO)); for (int i = 0; i < IndexWriter.MAX_DOCS; i++) { @@ -93,7 +94,7 @@ public void testAddDocument() throws Exception { setIndexWriterMaxDocs(10); try { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < 10; i++) { w.addDocument(new Document()); } @@ -116,7 +117,7 @@ public void testAddDocuments() throws Exception { setIndexWriterMaxDocs(10); try { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < 10; i++) { w.addDocument(new Document()); } @@ -139,7 +140,7 @@ public void testUpdateDocument() throws Exception { setIndexWriterMaxDocs(10); try { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < 10; i++) { w.addDocument(new Document()); } @@ -162,7 +163,7 @@ public void testUpdateDocuments() throws Exception { setIndexWriterMaxDocs(10); try { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < 10; i++) { w.addDocument(new Document()); } @@ -185,7 +186,7 @@ public void testReclaimedDeletes() throws Exception { setIndexWriterMaxDocs(10); try { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < 10; i++) { Document doc = new Document(); doc.add(newStringField("id", "" + i, Field.Store.NO)); @@ -226,7 +227,7 @@ public void testReclaimedDeletesWholeSegments() throws Exception { setIndexWriterMaxDocs(10); try { Directory dir = newDirectory(); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); iwc.setMergePolicy(NoMergePolicy.INSTANCE); IndexWriter w = new IndexWriter(dir, iwc); for (int i = 0; i < 10; i++) { @@ -271,14 +272,14 @@ public void testAddIndexes() throws Exception { setIndexWriterMaxDocs(10); try { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < 10; i++) { w.addDocument(new Document()); } w.close(); Directory dir2 = newDirectory(); - IndexWriter w2 = new IndexWriter(dir2, new IndexWriterConfig(null)); + IndexWriter w2 = new IndexWriter(dir2, new IndexWriterConfig((Analyzer) null)); w2.addDocument(new Document()); expectThrows( IllegalArgumentException.class, @@ -307,7 +308,7 @@ public void testAddIndexes() throws Exception { public void testMultiReaderExactLimit() throws Exception { Directory dir = newDirectory(); Document doc = new Document(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < 100000; i++) { w.addDocument(doc); } @@ -315,7 +316,7 @@ public void testMultiReaderExactLimit() throws Exception { int remainder = IndexWriter.MAX_DOCS % 100000; Directory dir2 = newDirectory(); - w = new IndexWriter(dir2, new IndexWriterConfig(null)); + w = new IndexWriter(dir2, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < remainder; i++) { w.addDocument(doc); } @@ -342,7 +343,7 @@ public void testMultiReaderExactLimit() throws Exception { public void testMultiReaderBeyondLimit() throws Exception { Directory dir = newDirectory(); Document doc = new Document(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < 100000; i++) { w.addDocument(doc); } @@ -354,7 +355,7 @@ public void testMultiReaderBeyondLimit() throws Exception { remainder++; Directory dir2 = newDirectory(); - w = new IndexWriter(dir2, new IndexWriterConfig(null)); + w = new IndexWriter(dir2, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < remainder; i++) { w.addDocument(doc); } @@ -387,7 +388,7 @@ public void testAddTooManyIndexesDir() throws Exception { // we cheat and add the same one over again... IW wants a write lock on each Directory dir = newDirectory(random(), NoLockFactory.INSTANCE); Document doc = new Document(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < 100000; i++) { w.addDocument(doc); } @@ -397,7 +398,7 @@ public void testAddTooManyIndexesDir() throws Exception { // wrap this with disk full, so test fails faster and doesn't fill up real disks. MockDirectoryWrapper dir2 = newMockDirectory(); - w = new IndexWriter(dir2, new IndexWriterConfig(null)); + w = new IndexWriter(dir2, new IndexWriterConfig((Analyzer) null)); w.commit(); // don't confuse checkindex dir2.setMaxSizeInBytes(dir2.sizeInBytes() + 65536); // 64KB Directory[] dirs = new Directory[1 + (IndexWriter.MAX_DOCS / 100000)]; @@ -437,7 +438,7 @@ public void testAddTooManyIndexesCodecReader() throws Exception { // we cheat and add the same one over again... IW wants a write lock on each Directory dir = newDirectory(random(), NoLockFactory.INSTANCE); Document doc = new Document(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); for (int i = 0; i < 100000; i++) { w.addDocument(doc); } @@ -447,7 +448,7 @@ public void testAddTooManyIndexesCodecReader() throws Exception { // wrap this with disk full, so test fails faster and doesn't fill up real disks. MockDirectoryWrapper dir2 = newMockDirectory(); - w = new IndexWriter(dir2, new IndexWriterConfig(null)); + w = new IndexWriter(dir2, new IndexWriterConfig((Analyzer) null)); w.commit(); // don't confuse checkindex dir2.setMaxSizeInBytes(dir2.sizeInBytes() + 65536); // 64KB IndexReader r = DirectoryReader.open(dir); @@ -498,7 +499,7 @@ public void testDeleteAll() throws Exception { setIndexWriterMaxDocs(1); try { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); w.addDocument(new Document()); expectThrows( IllegalArgumentException.class, @@ -526,7 +527,7 @@ public void testDeleteAllAfterFlush() throws Exception { setIndexWriterMaxDocs(2); try { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); w.addDocument(new Document()); DirectoryReader.open(w).close(); w.addDocument(new Document()); @@ -557,7 +558,7 @@ public void testDeleteAllAfterCommit() throws Exception { setIndexWriterMaxDocs(2); try { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); w.addDocument(new Document()); w.commit(); w.addDocument(new Document()); @@ -589,7 +590,7 @@ public void testDeleteAllMultipleThreads() throws Exception { setIndexWriterMaxDocs(limit); try { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); CountDownLatch startingGun = new CountDownLatch(1); Thread[] threads = new Thread[limit]; @@ -643,11 +644,11 @@ public void testDeleteAllAfterClose() throws Exception { setIndexWriterMaxDocs(2); try { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); w.addDocument(new Document()); w.close(); - IndexWriter w2 = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w2 = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); w2.addDocument(new Document()); expectThrows( IllegalArgumentException.class, @@ -676,10 +677,10 @@ public void testAcrossTwoIndexWriters() throws Exception { setIndexWriterMaxDocs(1); try { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); w.addDocument(new Document()); w.close(); - IndexWriter w2 = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w2 = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); expectThrows( IllegalArgumentException.class, () -> { @@ -696,7 +697,7 @@ public void testAcrossTwoIndexWriters() throws Exception { // LUCENE-6299 public void testCorruptIndexExceptionTooLarge() throws Exception { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); w.addDocument(new Document()); w.addDocument(new Document()); w.close(); @@ -718,7 +719,7 @@ public void testCorruptIndexExceptionTooLarge() throws Exception { // LUCENE-6299 public void testCorruptIndexExceptionTooLargeWriter() throws Exception { Directory dir = newDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); w.addDocument(new Document()); w.addDocument(new Document()); w.close(); @@ -728,7 +729,7 @@ public void testCorruptIndexExceptionTooLargeWriter() throws Exception { expectThrows( CorruptIndexException.class, () -> { - new IndexWriter(dir, new IndexWriterConfig(null)); + new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); }); } finally { restoreIndexWriterMaxDocs(); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterRAMManager.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterRAMManager.java new file mode 100644 index 000000000000..b54978d1f572 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterRAMManager.java @@ -0,0 +1,435 @@ +/* + * 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.index; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.FieldType; +import org.apache.lucene.document.TextField; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.util.LuceneTestCase; + +public class TestIndexWriterRAMManager extends LuceneTestCase { + + private static final FieldType storedTextType = new FieldType(TextField.TYPE_NOT_STORED); + + public void testSingleWriter() throws IOException { + try (Directory dir = newDirectory()) { + IndexWriterConfig indexWriterConfig = new IndexWriterConfig(); + TestFlushPolicy flushPolicy = new TestFlushPolicy(); + indexWriterConfig.setFlushPolicy(flushPolicy); + indexWriterConfig.setRAMBufferSizeMB(1); + try (IndexWriter w = new IndexWriter(dir, indexWriterConfig)) { + assertEquals(0, w.ramBytesUsed()); + int i = 0; + int errorLimit = 100000; // prevents loop from iterating forever + while (flushPolicy.flushedWriters.isEmpty()) { + Document doc = new Document(); + doc.add(newField("id", String.valueOf(i), storedTextType)); + w.addDocument(doc); + i += 1; + if (i == errorLimit) { + fail("Writer has not flushed when expected"); + } + } + assertEquals(1, flushPolicy.flushedWriters.size()); + // suppresses null pointer warning in the next line, this will always return true + assert flushPolicy.flushedWriters.size() == 1; + assertEquals(1, (int) flushPolicy.flushedWriters.poll()); + } + } + } + + public void testMultipleWriters() throws IOException { + try (Directory dir = newDirectory()) { + try (Directory dir2 = newDirectory()) { + IndexWriterRAMManager indexWriterRAMManager = new IndexWriterRAMManager(1); + // Writers share the same buffer, so we pass in the same ram manager + IndexWriterConfig indexWriterConfig = new IndexWriterConfig(indexWriterRAMManager); + TestFlushPolicy flushPolicy = new TestFlushPolicy(); + indexWriterConfig.setFlushPolicy(flushPolicy); + IndexWriterConfig indexWriterConfig2 = new IndexWriterConfig(indexWriterRAMManager); + indexWriterConfig2.setFlushPolicy(flushPolicy); + + try (IndexWriter w = new IndexWriter(dir, indexWriterConfig)) { + try (IndexWriter w2 = new IndexWriter(dir2, indexWriterConfig2)) { + assertEquals(0, w.ramBytesUsed()); + assertEquals(0, w2.ramBytesUsed()); + int i = 0; + int errorLimit = 100000; // prevents loop from iterating forever + boolean addToWriter1 = true; + while (flushPolicy.flushedWriters.size() < 2) { + Document doc = new Document(); + doc.add(newField("id", String.valueOf(i), storedTextType)); + if (addToWriter1) { + w.addDocument(doc); + } else { + w2.addDocument(doc); + } + addToWriter1 = !addToWriter1; + i += 1; + if (i == errorLimit) { + fail("Writers have not flushed when expected"); + } + } + assertEquals(2, flushPolicy.flushedWriters.size()); + assert flushPolicy.flushedWriters.size() == 2; + assertEquals(1, (int) flushPolicy.flushedWriters.poll()); + assert flushPolicy.flushedWriters.size() == 1; + assertEquals(2, (int) flushPolicy.flushedWriters.poll()); + } + } + } + } + } + + public void testMultipleWritersWithRemoval() throws IOException { + try (Directory dir = newDirectory()) { + try (Directory dir2 = newDirectory()) { + IndexWriterRAMManager indexWriterRAMManager = new IndexWriterRAMManager(1); + // Writers share the same buffer, so we pass in the same ram manager + IndexWriterConfig indexWriterConfig = new IndexWriterConfig(indexWriterRAMManager); + TestFlushPolicy flushPolicy = new TestFlushPolicy(); + indexWriterConfig.setFlushPolicy(flushPolicy); + IndexWriterConfig indexWriterConfig2 = new IndexWriterConfig(indexWriterRAMManager); + indexWriterConfig2.setFlushPolicy(flushPolicy); + + IndexWriter w = new IndexWriter(dir, indexWriterConfig); + IndexWriter w2 = new IndexWriter(dir2, indexWriterConfig2); + assertEquals(0, w.ramBytesUsed()); + assertEquals(0, w2.ramBytesUsed()); + int i = 0; + int errorLimit = 100000; // prevents loop from iterating forever + boolean addToWriter1 = true; + boolean w2Close = false; + while (flushPolicy.flushedWriters.size() < 4) { + if (w2Close == false && flushPolicy.flushedWriters.size() == 2) { + w2.close(); + w2Close = true; + } + Document doc = new Document(); + doc.add(newField("id", String.valueOf(i), storedTextType)); + if (addToWriter1 || w2Close) { + w.addDocument(doc); + } else { + w2.addDocument(doc); + } + addToWriter1 = !addToWriter1; + i += 1; + if (i == errorLimit) { + if (w.isOpen()) { + w.close(); + ; + } + if (w2.isOpen()) { + w2.close(); + } + w2.close(); + fail("Writers have not flushed when expected"); + } + } + if (w.isOpen()) { + w.close(); + ; + } + if (w2.isOpen()) { + w2.close(); + } + // we expect 1 flushed, then 2, then 1 then 1 since 2 was removed + assertEquals(4, flushPolicy.flushedWriters.size()); + assert flushPolicy.flushedWriters.size() == 4; + assertEquals(1, (int) flushPolicy.flushedWriters.poll()); + assert flushPolicy.flushedWriters.size() == 3; + assertEquals(2, (int) flushPolicy.flushedWriters.poll()); + assert flushPolicy.flushedWriters.size() == 2; + assertEquals(1, (int) flushPolicy.flushedWriters.poll()); + assert flushPolicy.flushedWriters.size() == 1; + assertEquals(1, (int) flushPolicy.flushedWriters.poll()); + } + } + } + + public void testMultipleWritersWithAdding() throws IOException { + try (Directory dir = newDirectory()) { + try (Directory dir2 = newDirectory()) { + try (Directory dir3 = newDirectory()) { + IndexWriterRAMManager indexWriterRAMManager = new IndexWriterRAMManager(1); + // Writers share the same buffer, so we pass in the same ram manager + IndexWriterConfig indexWriterConfig = new IndexWriterConfig(indexWriterRAMManager); + TestFlushPolicy flushPolicy = new TestFlushPolicy(); + indexWriterConfig.setFlushPolicy(flushPolicy); + IndexWriterConfig indexWriterConfig2 = new IndexWriterConfig(indexWriterRAMManager); + indexWriterConfig2.setFlushPolicy(flushPolicy); + IndexWriterConfig indexWriterConfig3 = new IndexWriterConfig(indexWriterRAMManager); + indexWriterConfig3.setFlushPolicy(flushPolicy); + + IndexWriter w = new IndexWriter(dir, indexWriterConfig); + IndexWriter w2 = new IndexWriter(dir2, indexWriterConfig2); + IndexWriter w3 = null; // don't init this right now + assertEquals(0, w.ramBytesUsed()); + assertEquals(0, w2.ramBytesUsed()); + int i = 0; + int errorLimit = 100000; // prevents loop from iterating forever + while (flushPolicy.flushedWriters.size() < 5) { + if (w3 == null && flushPolicy.flushedWriters.size() == 3) { + w3 = new IndexWriter(dir3, indexWriterConfig3); + assertEquals(0, w3.ramBytesUsed()); + } + Document doc = new Document(); + doc.add(newField("id", String.valueOf(i), storedTextType)); + if (i % 3 == 0) { + w.addDocument(doc); + } else if (i % 3 == 1) { + w2.addDocument(doc); + } else if (w3 != null) { + w3.addDocument(doc); + } + i += 1; + if (i == errorLimit) { + if (w.isOpen()) { + w.close(); + ; + } + if (w2.isOpen()) { + w2.close(); + } + if (w3 != null && w3.isOpen()) { + w3.close(); + } + w2.close(); + fail("Writers have not flushed when expected"); + } + } + if (w.isOpen()) { + w.close(); + ; + } + if (w2.isOpen()) { + w2.close(); + } + if (w3 != null && w3.isOpen()) { + w3.close(); + } + // we expect 1 flushed, then 2, then 1, then 2, then 3 since 3 was added + assertEquals(5, flushPolicy.flushedWriters.size()); + assert flushPolicy.flushedWriters.size() == 5; + assertEquals(1, (int) flushPolicy.flushedWriters.poll()); + assert flushPolicy.flushedWriters.size() == 4; + assertEquals(2, (int) flushPolicy.flushedWriters.poll()); + assert flushPolicy.flushedWriters.size() == 3; + assertEquals(1, (int) flushPolicy.flushedWriters.poll()); + assert flushPolicy.flushedWriters.size() == 2; + assertEquals(2, (int) flushPolicy.flushedWriters.poll()); + assert flushPolicy.flushedWriters.size() == 1; + assertEquals(3, (int) flushPolicy.flushedWriters.poll()); + } + } + } + } + + public void testRandom() throws IOException { + for (int i = 0; i < 20; i++) { + randomTest(); + } + } + + private static void randomTest() throws IOException { + int numWriters = random().nextInt(1, 100); + double ramBufferSize = random().nextDouble(); + List directories = new ArrayList<>(); + List configs = new ArrayList<>(); + List writers = new ArrayList<>(); + TestFlushPolicy flushPolicy = new TestFlushPolicy(); + TestEventRecordingIndexWriterRAMManager ramManager = + new TestEventRecordingIndexWriterRAMManager(ramBufferSize); + for (int i = 0; i < numWriters; i++) { + directories.add(newDirectory()); + configs.add(new IndexWriterConfig(ramManager)); + configs.get(i).setFlushPolicy(flushPolicy); + writers.add(new IndexWriter(directories.get(i), configs.get(i))); + assertEquals(0, writers.get(i).ramBytesUsed()); + } + + int flushedLimit = numWriters * 2; + int docId = 0; + while (ramManager.flushCount.get() < flushedLimit) { + boolean changeWriters = random().nextDouble() < 0.2; + if (changeWriters) { + boolean addWriter = random().nextBoolean(); + if (addWriter) { + directories.add(newDirectory()); + configs.add(new IndexWriterConfig(ramManager).setFlushPolicy(flushPolicy)); + writers.add(new IndexWriter(directories.getLast(), configs.getLast())); + } else { + int closeWriter = random().nextInt(writers.size()); + if (writers.get(closeWriter).isOpen()) { + writers.get(closeWriter).close(); + directories.get(closeWriter).close(); + } + } + } else { + Document doc = new Document(); + doc.add(newField("id", String.valueOf(docId++), storedTextType)); + int addDocWriter = random().nextInt(writers.size()); + if (writers.get(addDocWriter).isOpen()) { + writers.get(addDocWriter).addDocument(doc); + } + } + } + + verifyEvents(ramManager.events); + + for (int i = 0; i < writers.size(); i++) { + if (writers.get(i).isOpen()) { + writers.get(i).close(); + directories.get(i).close(); + } + } + } + + private static void verifyEvents( + Queue events) { + + TestEventRecordingIndexWriterRAMManager.TestEventAndId event = events.poll(); + int lastFlush = -1; + int maxValidWriter = 1; + Set removedWriters = new HashSet<>(); + while (event != null) { + if (event.event.equals(TestEventRecordingIndexWriterRAMManager.TestEvent.REMOVE)) { + removedWriters.add(event.id); + while (removedWriters.contains(maxValidWriter)) { + maxValidWriter--; + } + while (removedWriters.contains(lastFlush)) { + if (lastFlush == 1) { + lastFlush = maxValidWriter; + } else { + lastFlush--; + } + } + } else if (event.event.equals(TestEventRecordingIndexWriterRAMManager.TestEvent.ADD)) { + if (event.id > maxValidWriter) { + maxValidWriter = event.id; + } + } else if (event.event.equals(TestEventRecordingIndexWriterRAMManager.TestEvent.FLUSH)) { + int flushedId = event.id; + assertFalse("Flushed ID after removing it", removedWriters.contains(flushedId)); + if (lastFlush == -1) { + if (removedWriters.contains(1) == false) { + assertEquals("Must start flushing at the first id", 1, flushedId); + } + } else { + int nextValidFlush = lastFlush + 1; + while (removedWriters.contains(nextValidFlush) || nextValidFlush > maxValidWriter) { + if (nextValidFlush > maxValidWriter) { + nextValidFlush = 1; + } else { + nextValidFlush++; + } + } + + assertEquals("Flushed in the wrong order", nextValidFlush, event.id); + } + lastFlush = flushedId; + } + event = events.poll(); + } + } + + /** + * Flush policy used for testing that keeps track of all the writers that were flushed and what + * order they were flushed in + */ + private static class TestFlushPolicy extends FlushPolicy { + + ConcurrentLinkedQueue flushedWriters = new ConcurrentLinkedQueue<>(); + + @Override + public void onChange(DocumentsWriterFlushControl control, DocumentsWriterPerThread perThread) {} + + @Override + public void flushWriter( + IndexWriterRAMManager ramManager, + IndexWriterRAMManager.PerWriterIndexWriterRAMManager perWriterRamManager) + throws IOException { + long totalBytes = perWriterRamManager.getTotalBufferBytesUsed(); + if (totalBytes > ramManager.getRamBufferSizeMB() * 1024 * 1024) { + int flushedId = ramManager.flushRoundRobin(); + flushedWriters.add(flushedId); + } + } + } + + private static class TestEventRecordingIndexWriterRAMManager extends IndexWriterRAMManager { + + public enum TestEvent { + ADD, + REMOVE, + FLUSH; + } + + record TestEventAndId(TestEvent event, int id) { + + @Override + public String toString() { + return event + " " + id; + } + } + + ConcurrentLinkedQueue events = new ConcurrentLinkedQueue<>(); + + AtomicInteger flushCount = new AtomicInteger(); + + /** + * Default constructor + * + * @param ramBufferSizeMB the RAM buffer size to use between all registered {@link IndexWriter} + * instances + */ + TestEventRecordingIndexWriterRAMManager(double ramBufferSizeMB) { + super(ramBufferSizeMB); + } + + @Override + public int flushRoundRobin() throws IOException { + int flushed = super.flushRoundRobin(); + events.add(new TestEventAndId(TestEvent.FLUSH, flushed)); + flushCount.incrementAndGet(); + return flushed; + } + + @Override + protected int registerWriter(IndexWriter writer) { + int id = super.registerWriter(writer); + events.add(new TestEventAndId(TestEvent.ADD, id)); + return id; + } + + @Override + protected void removeWriter(int id) { + super.removeWriter(id); + events.add(new TestEventAndId(TestEvent.REMOVE, id)); + } + } +} diff --git a/lucene/core/src/test/org/apache/lucene/index/TestInfoStream.java b/lucene/core/src/test/org/apache/lucene/index/TestInfoStream.java index 80681ee126bd..82d241595bbe 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestInfoStream.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestInfoStream.java @@ -18,6 +18,7 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.util.LuceneTestCase; @@ -29,7 +30,7 @@ public class TestInfoStream extends LuceneTestCase { /** we shouldn't have test points unless we ask */ public void testTestPointsOff() throws Exception { Directory dir = newDirectory(); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); iwc.setInfoStream( new InfoStream() { @Override @@ -55,7 +56,7 @@ public boolean isEnabled(String component) { /** but they should work when we need */ public void testTestPointsOn() throws Exception { Directory dir = newDirectory(); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); AtomicBoolean seenTestPoint = new AtomicBoolean(); iwc.setInfoStream( new InfoStream() { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java index ef29c9b0f624..af8cd9e421db 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java @@ -20,6 +20,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.Codec; import org.apache.lucene.document.BinaryPoint; import org.apache.lucene.document.Document; @@ -566,7 +567,7 @@ public void testDeleteAllPointDocs() throws Exception { public void testPointsFieldMissingFromOneSegment() throws Exception { Directory dir = FSDirectory.open(createTempDir()); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); IndexWriter w = new IndexWriter(dir, iwc); Document doc = new Document(); doc.add(new StringField("id", "0", Field.Store.NO)); @@ -631,7 +632,7 @@ public void testSparsePoints() throws Exception { public void testCheckIndexIncludesPoints() throws Exception { Directory dir = new ByteBuffersDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); Document doc = new Document(); doc.add(new IntPoint("int1", 17)); w.addDocument(doc); @@ -669,7 +670,8 @@ public void testMergedStatsEmptyReader() throws IOException { public void testMergedStatsOneSegmentWithoutPoints() throws IOException { Directory dir = new ByteBuffersDirectory(); IndexWriter w = - new IndexWriter(dir, new IndexWriterConfig(null).setMergePolicy(NoMergePolicy.INSTANCE)); + new IndexWriter( + dir, new IndexWriterConfig((Analyzer) null).setMergePolicy(NoMergePolicy.INSTANCE)); w.addDocument(new Document()); DirectoryReader.open(w).close(); Document doc = new Document(); @@ -690,7 +692,7 @@ public void testMergedStatsOneSegmentWithoutPoints() throws IOException { public void testMergedStatsAllPointsDeleted() throws IOException { Directory dir = new ByteBuffersDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); w.addDocument(new Document()); Document doc = new Document(); doc.add(new IntPoint("field", Integer.MIN_VALUE)); @@ -728,7 +730,7 @@ private void doTestMergedStats() throws IOException { final int numDims = TestUtil.nextInt(random(), 1, 8); final int numBytesPerDim = TestUtil.nextInt(random(), 1, 16); Directory dir = new ByteBuffersDirectory(); - IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); final int numDocs = TestUtil.nextInt(random(), 10, 20); for (int i = 0; i < numDocs; ++i) { Document doc = new Document(); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java index 2c40a4e74860..2ae7bfa54f7e 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java @@ -22,6 +22,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.TextField; @@ -469,7 +470,7 @@ public IndexSearcher newSearcher(IndexReader ignored, IndexReader previous) { public void testListenerCalled() throws Exception { Directory dir = newDirectory(); - IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); final AtomicBoolean afterRefreshCalled = new AtomicBoolean(false); SearcherManager sm = new SearcherManager(iw, new SearcherFactory()); sm.addListener( diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java index e44855afe870..ebaf258852a0 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java @@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.index.ConcurrentMergeScheduler; @@ -321,7 +322,7 @@ public void run() { public void testCloseTwice() throws Exception { // test that we can close SM twice (per Closeable's contract). Directory dir = newDirectory(); - new IndexWriter(dir, new IndexWriterConfig(null)).close(); + new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)).close(); SearcherManager sm = new SearcherManager(dir, null); sm.close(); sm.close(); @@ -362,7 +363,7 @@ public void testReferenceDecrementIllegally() throws Exception { public void testEnsureOpen() throws Exception { Directory dir = newDirectory(); - new IndexWriter(dir, new IndexWriterConfig(null)).close(); + new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)).close(); SearcherManager sm = new SearcherManager(dir, null); IndexSearcher s = sm.acquire(); sm.close(); @@ -389,7 +390,7 @@ public void testEnsureOpen() throws Exception { public void testListenerCalled() throws Exception { Directory dir = newDirectory(); - IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); final AtomicBoolean afterRefreshCalled = new AtomicBoolean(false); SearcherManager sm = new SearcherManager(iw, false, false, new SearcherFactory()); sm.addListener( diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java index bd4934fbb046..0d08e89c4cca 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java @@ -28,6 +28,7 @@ import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.BinaryDocValuesField; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; @@ -237,7 +238,7 @@ protected IndexWriterConfig createIndexWriterConfig(OpenMode openMode) { // Make sure we use a MergePolicy which always merges adjacent segments and thus // keeps the doc IDs ordered as well (this is crucial for the taxonomy index). - return new IndexWriterConfig(null) + return new IndexWriterConfig((Analyzer) null) .setOpenMode(openMode) .setMergePolicy(new LogByteSizeMergePolicy()); } diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java index e8b2f9076b6f..6eaacb0653be 100644 --- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java +++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyWriter.java @@ -23,6 +23,7 @@ import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.facet.DrillDownQuery; import org.apache.lucene.facet.FacetField; @@ -274,7 +275,7 @@ public void testBackwardsCompatibility() throws Exception { Directory dir = newDirectory(); // create an empty index first, so that DirTaxoWriter initializes indexEpoch to 1. - new IndexWriter(dir, new IndexWriterConfig(null)).close(); + new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)).close(); DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(dir, OpenMode.CREATE_OR_APPEND, NO_OP_CACHE); diff --git a/lucene/misc/src/java/org/apache/lucene/misc/IndexMergeTool.java b/lucene/misc/src/java/org/apache/lucene/misc/IndexMergeTool.java index 0465f4e96b98..bfa3de0a8f1b 100644 --- a/lucene/misc/src/java/org/apache/lucene/misc/IndexMergeTool.java +++ b/lucene/misc/src/java/org/apache/lucene/misc/IndexMergeTool.java @@ -17,6 +17,7 @@ package org.apache.lucene.misc; import java.nio.file.Paths; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.IndexWriterConfig.OpenMode; @@ -48,7 +49,7 @@ public class IndexMergeTool { static class Options { String mergedIndexPath; String[] indexPaths; - IndexWriterConfig config = new IndexWriterConfig(null).setOpenMode(OpenMode.CREATE); + IndexWriterConfig config = new IndexWriterConfig((Analyzer) null).setOpenMode(OpenMode.CREATE); int maxSegments = 0; static Options parse(String[] args) throws ReflectiveOperationException { diff --git a/lucene/misc/src/java/org/apache/lucene/misc/index/MultiPassIndexSplitter.java b/lucene/misc/src/java/org/apache/lucene/misc/index/MultiPassIndexSplitter.java index c3c63ac860e9..547439962654 100644 --- a/lucene/misc/src/java/org/apache/lucene/misc/index/MultiPassIndexSplitter.java +++ b/lucene/misc/src/java/org/apache/lucene/misc/index/MultiPassIndexSplitter.java @@ -22,6 +22,7 @@ import java.nio.file.Paths; import java.util.ArrayList; import java.util.List; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.index.BaseCompositeReader; import org.apache.lucene.index.CodecReader; import org.apache.lucene.index.DirectoryReader; @@ -106,7 +107,8 @@ public void split(IndexReader in, Directory[] outputs, boolean seq) throws IOExc } } IndexWriter w = - new IndexWriter(outputs[i], new IndexWriterConfig(null).setOpenMode(OpenMode.CREATE)); + new IndexWriter( + outputs[i], new IndexWriterConfig((Analyzer) null).setOpenMode(OpenMode.CREATE)); System.err.println("Writing part " + (i + 1) + " ..."); // pass the subreaders directly, as our wrapper's numDocs/hasDeletetions are not up-to-date final List sr = input.getSequentialSubReadersWrapper(); diff --git a/lucene/misc/src/java/org/apache/lucene/misc/index/PKIndexSplitter.java b/lucene/misc/src/java/org/apache/lucene/misc/index/PKIndexSplitter.java index 789db6646d66..6a181b22284b 100644 --- a/lucene/misc/src/java/org/apache/lucene/misc/index/PKIndexSplitter.java +++ b/lucene/misc/src/java/org/apache/lucene/misc/index/PKIndexSplitter.java @@ -18,6 +18,7 @@ import java.io.IOException; import java.util.List; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.index.CodecReader; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.FilterCodecReader; @@ -57,7 +58,7 @@ public PKIndexSplitter(Directory input, Directory dir1, Directory dir2, Query do } private static IndexWriterConfig newDefaultConfig() { - return new IndexWriterConfig(null).setOpenMode(OpenMode.CREATE); + return new IndexWriterConfig((Analyzer) null).setOpenMode(OpenMode.CREATE); } public PKIndexSplitter( diff --git a/lucene/misc/src/test/org/apache/lucene/misc/index/TestBinaryDocValueSelector.java b/lucene/misc/src/test/org/apache/lucene/misc/index/TestBinaryDocValueSelector.java index 4576c571ba3b..37cdfd065637 100644 --- a/lucene/misc/src/test/org/apache/lucene/misc/index/TestBinaryDocValueSelector.java +++ b/lucene/misc/src/test/org/apache/lucene/misc/index/TestBinaryDocValueSelector.java @@ -18,6 +18,7 @@ package org.apache.lucene.misc.index; import java.io.IOException; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.BinaryDocValuesField; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; @@ -93,7 +94,7 @@ private static void assertSequentialIndex(Directory dir, int docNum, int segNum) } private static IndexWriterConfig getIndexWriterConfig() { - return new IndexWriterConfig(null) + return new IndexWriterConfig((Analyzer) null) .setOpenMode(IndexWriterConfig.OpenMode.CREATE) .setMergePolicy(NoMergePolicy.INSTANCE) .setIndexSort(new Sort(new SortField("ord", SortField.Type.INT))); diff --git a/lucene/misc/src/test/org/apache/lucene/misc/index/TestIndexRearranger.java b/lucene/misc/src/test/org/apache/lucene/misc/index/TestIndexRearranger.java index 3907b19f13c5..972236550a9c 100644 --- a/lucene/misc/src/test/org/apache/lucene/misc/index/TestIndexRearranger.java +++ b/lucene/misc/src/test/org/apache/lucene/misc/index/TestIndexRearranger.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; @@ -207,7 +208,7 @@ public void testDeleteNothing() throws Exception { } private static IndexWriterConfig getIndexWriterConfig() { - return new IndexWriterConfig(null) + return new IndexWriterConfig((Analyzer) null) .setOpenMode(IndexWriterConfig.OpenMode.CREATE) .setMergePolicy(NoMergePolicy.INSTANCE) .setIndexSort(new Sort(new SortField("ord", SortField.Type.INT))); diff --git a/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionRangeQuery.java b/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionRangeQuery.java index 4f6844c4b9b5..67391eca39b6 100644 --- a/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionRangeQuery.java +++ b/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionRangeQuery.java @@ -17,6 +17,7 @@ package org.apache.lucene.queries.function; import java.io.IOException; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; @@ -100,7 +101,7 @@ public void testDeletedMultiValued() throws IOException { private void doTestDeleted(ValueSource valueSource) throws IOException { // We delete doc with #3. Note we don't commit it to disk; we search using a near real-time // reader. - IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); try { writer.deleteDocuments( new FunctionRangeQuery(valueSource, 3, 3, true, true)); // delete the one with #3 diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/TestSpatialExample.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/TestSpatialExample.java index aadaff32ee03..23807b871ab4 100644 --- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/TestSpatialExample.java +++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/TestSpatialExample.java @@ -17,6 +17,7 @@ package org.apache.lucene.spatial; import java.io.IOException; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; @@ -97,7 +98,7 @@ protected void init() { } private void indexPoints() throws Exception { - IndexWriterConfig iwConfig = new IndexWriterConfig(null); + IndexWriterConfig iwConfig = new IndexWriterConfig((Analyzer) null); IndexWriter indexWriter = new IndexWriter(directory, iwConfig); // Spatial4j is x-y order for arguments diff --git a/lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java b/lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java index 60202bcb10af..1624c54b17f3 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.Comparator; import java.util.List; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.FieldType; @@ -158,7 +159,7 @@ public void setSpellIndex(Directory spellIndexDir) throws IOException { synchronized (modifyCurrentIndexLock) { ensureOpen(); if (!DirectoryReader.indexExists(spellIndexDir)) { - IndexWriter writer = new IndexWriter(spellIndexDir, new IndexWriterConfig(null)); + IndexWriter writer = new IndexWriter(spellIndexDir, new IndexWriterConfig((Analyzer) null)); writer.close(); } swapSearcher(spellIndexDir); @@ -453,7 +454,7 @@ public void clearIndex() throws IOException { ensureOpen(); final Directory dir = this.spellIndex; final IndexWriter writer = - new IndexWriter(dir, new IndexWriterConfig(null).setOpenMode(OpenMode.CREATE)); + new IndexWriter(dir, new IndexWriterConfig((Analyzer) null).setOpenMode(OpenMode.CREATE)); writer.close(); swapSearcher(dir); } diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePostingsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePostingsFormatTestCase.java index 8e0292b3f8db..cf4642c32eaf 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePostingsFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePostingsFormatTestCase.java @@ -752,7 +752,7 @@ protected void assertReused(String field, PostingsEnum p1, PostingsEnum p2) { public void testPostingsEnumDocsOnly() throws Exception { Directory dir = newDirectory(); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); IndexWriter iw = new IndexWriter(dir, iwc); Document doc = new Document(); doc.add(new StringField("foo", "bar", Field.Store.NO)); @@ -1269,7 +1269,7 @@ protected TokenStreamComponents createComponents(String fieldName) { public void testPostingsEnumPayloads() throws Exception { Directory dir = newDirectory(); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); IndexWriter iw = new IndexWriter(dir, iwc); Document doc = new Document(); Token token1 = new Token("bar", 0, 3); @@ -1474,7 +1474,7 @@ public void testPostingsEnumPayloads() throws Exception { public void testPostingsEnumAll() throws Exception { Directory dir = newDirectory(); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); IndexWriter iw = new IndexWriter(dir, iwc); Document doc = new Document(); Token token1 = new Token("bar", 0, 3); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseStoredFieldsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseStoredFieldsFormatTestCase.java index 84fdcde81732..f42804dfaa44 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseStoredFieldsFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseStoredFieldsFormatTestCase.java @@ -32,6 +32,7 @@ import java.util.Random; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.simpletext.SimpleTextCodec; @@ -832,7 +833,7 @@ public void testMismatchedFields() throws Exception { Directory[] dirs = new Directory[10]; for (int i = 0; i < dirs.length; i++) { Directory dir = newDirectory(); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); IndexWriter iw = new IndexWriter(dir, iwc); Document doc = new Document(); for (int j = 0; j < 10; j++) { @@ -849,7 +850,7 @@ public void testMismatchedFields() throws Exception { reader = new MismatchedDirectoryReader(reader, random()); } dirs[i] = newDirectory(); - IndexWriter adder = new IndexWriter(dirs[i], new IndexWriterConfig(null)); + IndexWriter adder = new IndexWriter(dirs[i], new IndexWriterConfig((Analyzer) null)); TestUtil.addIndexesSlowly(adder, reader); adder.commit(); adder.close(); @@ -858,7 +859,7 @@ public void testMismatchedFields() throws Exception { } Directory everything = newDirectory(); - IndexWriter iw = new IndexWriter(everything, new IndexWriterConfig(null)); + IndexWriter iw = new IndexWriter(everything, new IndexWriterConfig((Analyzer) null)); iw.addIndexes(dirs); iw.forceMerge(1); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseTermVectorsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseTermVectorsFormatTestCase.java index 53df7671d85f..bc05d539b2ec 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseTermVectorsFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseTermVectorsFormatTestCase.java @@ -1484,7 +1484,7 @@ protected TokenStreamComponents createComponents(String fieldName) { public void testPostingsEnumPayloads() throws Exception { Directory dir = newDirectory(); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); IndexWriter iw = new IndexWriter(dir, iwc); Document doc = new Document(); Token token1 = new Token("bar", 0, 3); @@ -1686,7 +1686,7 @@ public void testPostingsEnumPayloads() throws Exception { public void testPostingsEnumAll() throws Exception { Directory dir = newDirectory(); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); IndexWriter iw = new IndexWriter(dir, iwc); Document doc = new Document(); Token token1 = new Token("bar", 0, 3); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/LegacyBaseDocValuesFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/LegacyBaseDocValuesFormatTestCase.java index a77050a981c5..1b26badf72a9 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/LegacyBaseDocValuesFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/LegacyBaseDocValuesFormatTestCase.java @@ -3179,7 +3179,7 @@ public void testOneSortedNumber() throws IOException { public void testOneSortedNumberOneMissing() throws IOException { Directory directory = newDirectory(); - IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(null)); + IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig((Analyzer) null)); Document doc = new Document(); doc.add(new SortedNumericDocValuesField("dv", 5)); writer.addDocument(doc); @@ -3273,7 +3273,7 @@ public void testTwoSortedNumberSameValue() throws IOException { public void testTwoSortedNumberOneMissing() throws IOException { Directory directory = newDirectory(); - IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(null)); + IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig((Analyzer) null)); Document doc = new Document(); doc.add(new SortedNumericDocValuesField("dv", 11)); doc.add(new SortedNumericDocValuesField("dv", -5)); @@ -3297,7 +3297,7 @@ public void testTwoSortedNumberOneMissing() throws IOException { public void testSortedNumberMerge() throws IOException { Directory directory = newDirectory(); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); iwc.setMergePolicy(newLogMergePolicy()); IndexWriter writer = new IndexWriter(directory, iwc); Document doc = new Document(); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/store/MockDirectoryWrapper.java b/lucene/test-framework/src/java/org/apache/lucene/tests/store/MockDirectoryWrapper.java index 5f329209d804..a287ad82ca66 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/store/MockDirectoryWrapper.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/store/MockDirectoryWrapper.java @@ -40,6 +40,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Matcher; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexWriter; @@ -929,7 +930,7 @@ public synchronized void close() throws IOException { // now look for unreferenced files: discount ones that we tried to delete but could not Set allFiles = new HashSet<>(Arrays.asList(listAll())); String[] startFiles = allFiles.toArray(new String[0]); - IndexWriterConfig iwc = new IndexWriterConfig(null); + IndexWriterConfig iwc = new IndexWriterConfig((Analyzer) null); iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE); // We must do this before opening writer otherwise writer will be angry if there are @@ -983,7 +984,8 @@ public synchronized void close() throws IOException { int numDocs1 = ir1.numDocs(); ir1.close(); // Don't commit on close, so that no merges will be scheduled. - new IndexWriter(this, new IndexWriterConfig(null).setCommitOnClose(false)).close(); + new IndexWriter(this, new IndexWriterConfig((Analyzer) null).setCommitOnClose(false)) + .close(); DirectoryReader ir2 = DirectoryReader.open(this); int numDocs2 = ir2.numDocs(); ir2.close(); diff --git a/lucene/test-framework/src/test/org/apache/lucene/tests/util/TestFailIfUnreferencedFiles.java b/lucene/test-framework/src/test/org/apache/lucene/tests/util/TestFailIfUnreferencedFiles.java index 403ec45637f1..d172cbcd5e29 100644 --- a/lucene/test-framework/src/test/org/apache/lucene/tests/util/TestFailIfUnreferencedFiles.java +++ b/lucene/test-framework/src/test/org/apache/lucene/tests/util/TestFailIfUnreferencedFiles.java @@ -18,6 +18,7 @@ import com.carrotsearch.randomizedtesting.RandomizedTest; import java.util.Collections; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; @@ -40,7 +41,7 @@ public static class Nested1 extends WithNestedTests.AbstractNestedTest { public void testDummy() throws Exception { MockDirectoryWrapper dir = LuceneTestCase.newMockDirectory(); dir.setAssertNoUnrefencedFilesOnClose(true); - IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null)); + IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig((Analyzer) null)); iw.addDocument(new Document()); iw.close(); IndexOutput output = dir.createOutput("_hello.world", IOContext.DEFAULT);