From a4ac3fe3ef94bef652ddf47ee3f3ca8f7e045c89 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 14 Oct 2019 18:47:58 +0200 Subject: [PATCH] LUCENE-9003: Compute numDocs() lazily. (#939) --- lucene/CHANGES.txt | 6 +- .../lucene/index/BaseCompositeReader.java | 24 +++++-- .../org/apache/lucene/index/IndexReader.java | 7 +- .../index/TestFilterDirectoryReader.java | 72 +++++++++++++++++++ 4 files changed, 102 insertions(+), 7 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 041719eba18..ae2b66b785c 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -79,12 +79,16 @@ Improvements (No changes) Optimizations ---------------------- * LUCENE-8928: When building a kd-tree for dimensions n > 2, compute exact bounds for an inner node every N splits to improve the quality of the tree. N is defined by SPLITS_BEFORE_EXACT_BOUNDS which is set to 4. (Ignacio Vera, Adrien Grand) +* BaseDirectoryReader no longer sums up the `LeafReader#numDocs` of its leaves + eagerly. This especially helps when creating views of readers that hide + documents, since computing the number of live documents is an expensive + operation. (Adrien Grand) + Bug Fixes * LUCENE-9001: Fix race condition in SetOnce. (Przemko Robakowski) diff --git a/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java b/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java index 8a19fe14120..5d32a1a06a4 100644 --- a/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; /** Base class for implementing {@link CompositeReader}s based on an array * of sub-readers. The implementing class has to add code for @@ -50,7 +51,7 @@ public abstract class BaseCompositeReader extends Composi private final R[] subReaders; private final int[] starts; // 1st docno for each reader private final int maxDoc; - private final int numDocs; + private AtomicInteger numDocs = new AtomicInteger(-1); // computed lazily /** List view solely for {@link #getSequentialSubReaders()}, * for effectiveness the array is used internally. */ @@ -68,12 +69,11 @@ public abstract class BaseCompositeReader extends Composi this.subReaders = subReaders; this.subReadersList = Collections.unmodifiableList(Arrays.asList(subReaders)); starts = new int[subReaders.length + 1]; // build starts array - long maxDoc = 0, numDocs = 0; + long maxDoc = 0; for (int i = 0; i < subReaders.length; i++) { starts[i] = (int) maxDoc; final IndexReader r = subReaders[i]; maxDoc += r.maxDoc(); // compute maxDocs - numDocs += r.numDocs(); // compute numDocs r.registerParentReader(this); } @@ -89,7 +89,6 @@ public abstract class BaseCompositeReader extends Composi this.maxDoc = Math.toIntExact(maxDoc); starts[subReaders.length] = this.maxDoc; - this.numDocs = Math.toIntExact(numDocs); } @Override @@ -102,6 +101,23 @@ public abstract class BaseCompositeReader extends Composi @Override public final int numDocs() { // Don't call ensureOpen() here (it could affect performance) + // We want to compute numDocs() lazily so that creating a wrapper that hides + // some documents isn't slow at wrapping time, but on the first time that + // numDocs() is called. This can help as there are lots of use-cases of a + // reader that don't involve calling numDocs(). + // However it's not crucial to make sure that we don't call numDocs() more + // than once on the sub readers, since they likely cache numDocs() anyway, + // hence the opaque read. + // http://gee.cs.oswego.edu/dl/html/j9mm.html#opaquesec. + int numDocs = this.numDocs.getOpaque(); + if (numDocs == -1) { + numDocs = 0; + for (IndexReader r : subReaders) { + numDocs += r.numDocs(); + } + assert numDocs >= 0; + this.numDocs.set(numDocs); + } return numDocs; } diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexReader.java b/lucene/core/src/java/org/apache/lucene/index/IndexReader.java index c87f5daeaeb..ff241446026 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexReader.java @@ -301,7 +301,9 @@ public abstract class IndexReader implements Closeable { return vectors.terms(field); } - /** Returns the number of documents in this index. */ + /** Returns the number of documents in this index. + *

NOTE: This operation may run in O(maxDoc). Implementations that + * can't return this number in constant-time should cache it. */ public abstract int numDocs(); /** Returns one greater than the largest possible document number. @@ -310,7 +312,8 @@ public abstract class IndexReader implements Closeable { */ public abstract int maxDoc(); - /** Returns the number of deleted documents. */ + /** Returns the number of deleted documents. + *

NOTE: This operation may run in O(maxDoc). */ public final int numDeletedDocs() { return maxDoc() - numDocs(); } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFilterDirectoryReader.java b/lucene/core/src/test/org/apache/lucene/index/TestFilterDirectoryReader.java index 62a42945581..2a00b299426 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestFilterDirectoryReader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestFilterDirectoryReader.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.atomic.AtomicLong; import org.apache.lucene.document.Document; import org.apache.lucene.index.FilterDirectoryReader.SubReaderWrapper; @@ -76,4 +77,75 @@ public class TestFilterDirectoryReader extends LuceneTestCase { dir.close(); } + private static class NumDocsCountingSubReaderWrapper extends SubReaderWrapper { + + private final AtomicLong numDocsCallCount; + + NumDocsCountingSubReaderWrapper(AtomicLong numDocsCallCount) { + this.numDocsCallCount = numDocsCallCount; + } + + @Override + public LeafReader wrap(LeafReader reader) { + return new FilterLeafReader(reader) { + @Override + public int numDocs() { + numDocsCallCount.incrementAndGet(); + return super.numDocs(); + } + + @Override + public CacheHelper getCoreCacheHelper() { + return in.getCoreCacheHelper(); + } + + @Override + public CacheHelper getReaderCacheHelper() { + return in.getReaderCacheHelper(); + } + }; + } + + } + + private static class NumDocsCountingFilterDirectoryReader extends FilterDirectoryReader { + + private final AtomicLong numDocsCallCount; + + public NumDocsCountingFilterDirectoryReader(DirectoryReader in, AtomicLong numDocsCallCount) throws IOException { + super(in, new NumDocsCountingSubReaderWrapper(numDocsCallCount)); + this.numDocsCallCount = numDocsCallCount; + } + + @Override + protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { + return new NumDocsCountingFilterDirectoryReader(in, numDocsCallCount); + } + + @Override + public CacheHelper getReaderCacheHelper() { + return in.getReaderCacheHelper(); + } + + } + + public void testFilterDirectoryReaderNumDocsIsLazy() throws IOException { + Directory dir = newDirectory(); + IndexWriter w = new IndexWriter(dir, newIndexWriterConfig()); + w.addDocument(new Document()); + DirectoryReader directoryReader = DirectoryReader.open(w); + w.close(); + + AtomicLong numDocsCallCount = new AtomicLong(); + DirectoryReader directoryReaderWrapper = new NumDocsCountingFilterDirectoryReader(directoryReader, numDocsCallCount); + assertEquals(0L, numDocsCallCount.get()); + assertEquals(1, directoryReaderWrapper.numDocs()); + assertEquals(1L, numDocsCallCount.get()); // one segment, so called once + assertEquals(1, directoryReaderWrapper.numDocs()); + assertEquals(1L, numDocsCallCount.get()); + + directoryReader.close(); + dir.close(); + } + }