You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dn...@apache.org on 2020/10/02 22:45:00 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9554: Expose IndexWriter#pendingNumDocs (#1941)

This is an automated email from the ASF dual-hosted git repository.

dnhatn pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 77396db  LUCENE-9554: Expose IndexWriter#pendingNumDocs (#1941)
77396db is described below

commit 77396dbf339445333302814bc993c3c9db84b974
Author: Nhat Nguyen <nh...@elastic.co>
AuthorDate: Fri Oct 2 18:44:43 2020 -0400

    LUCENE-9554: Expose IndexWriter#pendingNumDocs (#1941)
    
    Some applications can use the pendingNumDocs from IndexWriter to
    estimate that the number of documents of an index is very close to the
    hard limit so that it can reject writes without constructing Lucene
    documents.
---
 .../java/org/apache/lucene/index/IndexWriter.java    |  8 ++++++++
 .../org/apache/lucene/index/TestIndexWriter.java     | 20 ++++++++++++++++++++
 2 files changed, 28 insertions(+)

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 259b001..06ab80e 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -5480,6 +5480,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
     throw new IllegalArgumentException("number of documents in the index cannot exceed " + actualMaxDocs + " (current document count is " + pendingNumDocs.get() + "; added numDocs is " + addedNumDocs + ")");
   }
 
+  /**
+   * Returns the number of documents in the index including documents are being added (i.e., reserved).
+   * @lucene.experimental
+   */
+  public long getPendingNumDocs() {
+    return pendingNumDocs.get();
+  }
+
   /** Returns the highest <a href="#sequence_number">sequence number</a> across
    *  all completed operations, or 0 if no operations have finished yet.  Still
    *  in-flight operations (in other threads) are not counted until they finish.
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 2ef1a4f..1761e1c 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -4239,4 +4239,24 @@ public class TestIndexWriter extends LuceneTestCase {
     }
     IOUtils.close(w, dir);
   }
+
+  public void testPendingNumDocs() throws Exception {
+    try (Directory dir = newDirectory()) {
+      int numDocs = random().nextInt(100);
+      try (IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig())) {
+        for (int i = 0; i < numDocs; i++) {
+          Document d = new Document();
+          d.add(new StringField("id", Integer.toString(i), Field.Store.YES));
+          writer.addDocument(d);
+          assertEquals(i + 1L, writer.getPendingNumDocs());
+        }
+        assertEquals(numDocs, writer.getPendingNumDocs());
+        writer.flush();
+        assertEquals(numDocs, writer.getPendingNumDocs());
+      }
+      try (IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig())) {
+        assertEquals(numDocs, writer.getPendingNumDocs());
+      }
+    }
+  }
 }