jpountz commented on PR #13359:
URL: https://github.com/apache/lucene/pull/13359#issuecomment-2105658311

   This is a draft as I need to do more work on tests and making sure that this 
new method cannot corrupt the state of the `SegmentTermsEnum`.
   
   <details>
   <summary>But I created a benchmark that starts looking like running a Lucene 
query that is encouraging. It creates an index with many terms that have very 
short postings lists, so that running boolean queries on these terms is heavy 
on terms dictionary lookups rather than reading postings. Then it manually runs 
a disjunction over 3 terms (some of these terms may not exist in the index as 
they are created randomly), computing how long it takes to evaluate all hits. 
To work properly when running a query, we'd need to move `#bulkScorer` from 
`Weight` to `ScorerSupplier`, which I intend to do as a follow-up.</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.Field.Store;
   import org.apache.lucene.document.StringField;
   import org.apache.lucene.index.DirectoryReader;
   import org.apache.lucene.index.IndexReader;
   import org.apache.lucene.index.IndexWriter;
   import org.apache.lucene.index.IndexWriterConfig;
   import org.apache.lucene.index.Term;
   import org.apache.lucene.index.TieredMergePolicy;
   import org.apache.lucene.search.BooleanClause.Occur;
   import org.apache.lucene.search.BooleanQuery;
   import org.apache.lucene.search.DocIdSetIterator;
   import org.apache.lucene.search.IndexSearcher;
   import org.apache.lucene.search.ScoreMode;
   import org.apache.lucene.search.Scorer;
   import org.apache.lucene.search.ScorerSupplier;
   import org.apache.lucene.search.TermQuery;
   import org.apache.lucene.search.Weight;
   import org.apache.lucene.store.Directory;
   import org.apache.lucene.store.FSDirectory;
   
   public class TermsEnumPrefetchBench {
   
     private static final int NUM_TERMS = 3;
     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) {
         TieredMergePolicy mp = new TieredMergePolicy();
         mp.setSegmentsPerTier(100);
         mp.setMaxMergeAtOnce(100);
         mp.setMaxMergedSegmentMB(1024);
         try (IndexWriter w = new IndexWriter(dir, new IndexWriterConfig()
             .setMergePolicy(mp)
             .setRAMBufferSizeMB(1024))) {
           ExecutorService executor = 
Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
           AtomicLong indexed = new AtomicLong(0);
           for (int task = 0; task < 1000; ++task) {
             executor.execute(() -> {
               Random r = ThreadLocalRandom.current();
               for (int i = 0; i < 1_000; ++i) {
                 Document doc = new Document();
                 for (int j = 0; j < 10_000; ++j) {
                   doc.add(new StringField("f", 
Long.toString(r.nextLong(20_000_000_000L)), Store.NO));
                 }
                 try {
                   w.addDocument(doc);
                 } catch (IOException e) {
                   throw new UncheckedIOException(e);
                 }
                 final long actualIndexed = indexed.incrementAndGet(); 
                 if (actualIndexed % 10_000 == 0) {
                   System.out.println("Indexed: " + actualIndexed);
                 }
               }
             });
           }
   
           executor.shutdown();
           executor.awaitTermination(1, TimeUnit.DAYS);
           w.commit();
           System.out.println("Start force merging");
           w.forceMerge(1);
           System.out.println("Done force merging");
           w.commit();
         }
       }
       List<Long> latencies = new ArrayList<>();
       try (IndexReader reader = DirectoryReader.open(dir)) {
         IndexSearcher searcher = new IndexSearcher(reader);
   
         Random r = ThreadLocalRandom.current();
         for (int i = 0; i < 10_000; ++i) {
           long start = System.nanoTime();
           BooleanQuery.Builder query = new BooleanQuery.Builder();
           for (int t = 0; t < NUM_TERMS; ++t) {
             query.add(new TermQuery(new Term("f", 
Long.toString(r.nextLong(20_000_000_000L)))), Occur.SHOULD);
           }
           Weight weight = 
searcher.createWeight(searcher.rewrite(query.build()), 
ScoreMode.COMPLETE_NO_SCORES, 1f);
           ScorerSupplier ss = weight.scorerSupplier(reader.leaves().get(0));
           if (ss != null) {
             Scorer scorer = ss.get(Long.MAX_VALUE);
             DocIdSetIterator iter = scorer.iterator();
             for (int d = iter.nextDoc(); d != DocIdSetIterator.NO_MORE_DOCS; d 
= iter.nextDoc()) {
               DUMMY++;
             }
             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>
   
   Without the change:
   P50: 286
   P90: 403
   P99: 532
   
   With the change:
   P50: 148
   P90: 246
   P99: 368


-- 
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

Reply via email to