jpountz commented on PR #13424: URL: https://github.com/apache/lucene/pull/13424#issuecomment-2133420215
Like for previous changes, I wrote a synthetic benchmark to make sure that this new API actually helps. <details> <summary>This benchmark simulates fetching 20 random stored documents in parallel. The index it creates is 39GB while my page cache only has a capacity of 25GB.</summary> ```java import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; import java.util.List; import java.util.Random; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import org.apache.lucene.document.Document; import org.apache.lucene.document.StoredField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.FilterMergePolicy; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.StoredFields; import org.apache.lucene.index.TieredMergePolicy; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; public class StoredFieldsPrefetchBench { public static int DUMMY; public static void main(String[] args) throws Exception { Path dirPath = Paths.get(args[0]); Directory dir = FSDirectory.open(dirPath); if (DirectoryReader.indexExists(dir) == false) { MergePolicy mergePolicy = new FilterMergePolicy(new TieredMergePolicy()) { @Override public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, MergeContext mergeContext) throws IOException { return false; } }; try (IndexWriter w = new IndexWriter(dir, new IndexWriterConfig().setUseCompoundFile(false).setMergePolicy(mergePolicy))) { ExecutorService executor = Executors.newFixedThreadPool(4); AtomicLong indexed = new AtomicLong(0); for (int task = 0; task < 1000; ++task) { executor.execute(() -> { Random r = ThreadLocalRandom.current(); for (int i = 0; i < 40_000; ++i) { Document doc = new Document(); byte[] bytes = new byte[1024]; r.nextBytes(bytes); doc.add(new StoredField("content", bytes)); try { w.addDocument(doc); } catch (IOException e) { throw new UncheckedIOException(e); } final long actualIndexed = indexed.incrementAndGet(); if (actualIndexed % 1_000_000 == 0) { System.out.println("Indexed: " + actualIndexed); try { DirectoryReader.open(w).close(); } catch (IOException e) { throw new UncheckedIOException(e); } } } }); } executor.shutdown(); executor.awaitTermination(1, TimeUnit.DAYS); w.commit(); System.out.println("Done indexing"); } } List<Long> latencies = new ArrayList<>(); try (IndexReader reader = DirectoryReader.open(dir)) { Random r = ThreadLocalRandom.current(); for (int i = 0; i < 10_000; ++i) { StoredFields storedFields = reader.storedFields(); long start = System.nanoTime(); int[] ids = new int[20]; for (int j = 0; j < ids.length; ++j) { ids[j] = r.nextInt(reader.maxDoc()); } for (int doc : ids) { storedFields.prefetch(doc); } for (int doc : ids) { DUMMY += storedFields.document(doc).getBinaryValue("content").hashCode(); } long end = System.nanoTime(); latencies.add((end - start) / 1000); } } latencies.sort(null); System.out.println("P50: " + latencies.get(latencies.size() / 2)); System.out.println("P90: " + latencies.get(latencies.size() * 9 / 10)); System.out.println("P99: " + latencies.get(latencies.size() * 99 / 100)); } } ``` </details> Before the change: P50: 2942 P90: 3900 P99: 4726 After the change: P50: 650 P90: 801 P99: 970 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org