You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2021/11/17 23:00:33 UTC

[lucene] branch branch_9_0 updated: LUCENE-10122 Use NumericDocValues to store taxonomy parent array instead of custom term positions (#442)

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

mikemccand pushed a commit to branch branch_9_0
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/branch_9_0 by this push:
     new 1ac5188  LUCENE-10122 Use NumericDocValues to store taxonomy parent array instead of custom term positions (#442)
1ac5188 is described below

commit 1ac5188891601d6f02940c73dd9c6160ce9d4eea
Author: Patrick Zhai <zh...@users.noreply.github.com>
AuthorDate: Wed Nov 17 15:00:25 2021 -0800

    LUCENE-10122 Use NumericDocValues to store taxonomy parent array instead of custom term positions (#442)
---
 lucene/CHANGES.txt                                 | 14 +++---
 .../lucene/facet/taxonomy/directory/Consts.java    |  6 +++
 .../directory/DirectoryTaxonomyWriter.java         | 52 ++++++++++++-------
 .../taxonomy/directory/TaxonomyIndexArrays.java    | 58 ++++++++++++++++++----
 4 files changed, 94 insertions(+), 36 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 801c2d3..1c6d089 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -191,7 +191,7 @@ Improvements
 * LUCENE-9633: Improve match highlighter behavior for degenerate intervals (on non-existing positions).
   (Dawid Weiss)
 
-* LUCENE-9618: Do not call IntervalIterator.nextInterval after NO_MORE_DOCS is returned. (Haoyu Zhai)
+* LUCENE-9618: Do not call IntervalIterator.nextInterval after NO_MORE_DOCS is returned. (Patrick Zhai)
 
 * LUCENE-9576: Improve ConcurrentMergeScheduler settings by default, assuming modern I/O.
   Previously Lucene was too conservative, jumping through hoops to detect if disks were SSD-backed.
@@ -398,6 +398,8 @@ Build
 Other
 ---------------------
 
+* LUCENE-10122: Use NumericDocValues to store taxonomy parent array (Patrick Zhai)
+
 * LUCENE-10136: allow 'var' declarations in source code (Dawid Weiss)
 
 * LUCENE-9570, LUCENE-9564: Apply google java format and enforce it on source Java files.
@@ -406,7 +408,7 @@ Other
 
 * LUCENE-9631: Properly override slice() on subclasses of OffsetRange. (Dawid Weiss)
 
-* LUCENE-9391: Upgrade HPPC to 0.8.2. (Haoyu Zhai)
+* LUCENE-9391: Upgrade HPPC to 0.8.2. (Patrick Zhai)
 
 * LUCENE-10021: Upgrade HPPC to 0.9.0. Replace usage of ...ScatterMap to ...HashMap. (Patrick Zhai)
 
@@ -467,7 +469,7 @@ Improvements
 * LUCENE-9662: Make CheckIndex concurrent by parallelizing index check across segments.
   (Zach Chen, Mike McCandless, Dawid Weiss, Robert Muir)
 
-* LUCENE-10103: Make QueryCache respect Accountable queries. (Haoyu Zhai)
+* LUCENE-10103: Make QueryCache respect Accountable queries. (Patrick Zhai)
 
 Optimizations
 ---------------------
@@ -677,7 +679,7 @@ New Features
   (Cameron VandenBerg)
 
 * LUCENE-9694: New tool for creating a deterministic index to enable benchmarking changes
-  on a consistent multi-segment index even when they require re-indexing. (Haoyu Zhai)
+  on a consistent multi-segment index even when they require re-indexing. (Patrick Zhai)
 
 * LUCENE-9385: Add FacetsConfig option to control which drill-down
   terms are indexed for a FacetLabel (Zachary Chen)
@@ -880,7 +882,7 @@ Improvements
 
 * LUCENE-8574: Add a new ExpressionValueSource which will enforce only one value per name
   per hit in dependencies, ExpressionFunctionValues will no longer
-  recompute already computed values (Haoyu Zhai)
+  recompute already computed values (Patrick Zhai)
 
 * LUCENE-9416: Fix CheckIndex to print an invalid non-zero norm as
   unsigned long when detecting corruption.
@@ -953,7 +955,7 @@ Bug Fixes
 Documentation
 ---------------------
 
-* LUCENE-9424: Add a performance warning to AttributeSource.captureState javadocs (Haoyu Zhai)
+* LUCENE-9424: Add a performance warning to AttributeSource.captureState javadocs (Patrick Zhai)
 
 Changes in Runtime Behavior
 ---------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/Consts.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/Consts.java
index 104bfdf..570c76f 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/Consts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/Consts.java
@@ -21,7 +21,13 @@ import org.apache.lucene.util.BytesRef;
 /** @lucene.experimental */
 abstract class Consts {
   static final String FULL = "$full_path$";
+
+  // TODO: Remove following 3 lines in Lucene 10, those are only used when
+  //       parent facet ordinals are stored using term position, and is deprecated
+  //       and migrated to NumericDocValues in Lucene 9
   static final String FIELD_PAYLOADS = "$payloads$";
   static final String PAYLOAD_PARENT = "p";
   static final BytesRef PAYLOAD_PARENT_BYTES_REF = new BytesRef(PAYLOAD_PARENT);
+
+  static final String FIELD_PARENT_ORDINAL_NDV = "$parent_ndv$";
 }
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 90f1d4e..f1e2ad9 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
@@ -34,6 +34,7 @@ import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.facet.FacetsConfig;
@@ -91,16 +92,18 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
 
   private final Directory dir;
   private final IndexWriter indexWriter;
-  private final boolean useOlderStoredFieldIndex;
+  private final boolean useOlderFormat;
   private final TaxonomyWriterCache cache;
   private final AtomicInteger cacheMisses = new AtomicInteger(0);
 
   // Records the taxonomy index epoch, updated on replaceTaxonomy as well.
   private long indexEpoch;
 
+  // TODO: remove following 2 fields in Lucene 10
   private SinglePositionTokenStream parentStream =
       new SinglePositionTokenStream(Consts.PAYLOAD_PARENT);
   private Field parentStreamField;
+
   private Field fullPathField;
   private int cacheMissesUntilFill = 11;
   private boolean shouldFillCache = true;
@@ -160,13 +163,13 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
     if (DirectoryReader.indexExists(directory) == false) {
       indexEpoch = 1;
       // no commit exists so we can safely use the new BinaryDocValues field
-      useOlderStoredFieldIndex = false;
+      useOlderFormat = false;
     } else {
       String epochStr = null;
 
       SegmentInfos infos = SegmentInfos.readLatestCommit(dir);
       /* a previous commit exists, so check the version of the last commit */
-      useOlderStoredFieldIndex = infos.getIndexCreatedVersionMajor() <= 8;
+      useOlderFormat = infos.getIndexCreatedVersionMajor() <= 8;
 
       Map<String, String> commitData = infos.getUserData();
       if (commitData != null) {
@@ -181,12 +184,17 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
       ++indexEpoch;
     }
 
-    FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
-    ft.setOmitNorms(true);
-    parentStreamField = new Field(Consts.FIELD_PAYLOADS, parentStream, ft);
-    if (useOlderStoredFieldIndex) {
+    if (useOlderFormat) {
+      // parent ordinal field
+      FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
+      ft.setOmitNorms(true);
+      parentStreamField = new Field(Consts.FIELD_PAYLOADS, parentStream, ft);
+
+      // full path field
       fullPathField = new StringField(Consts.FULL, "", Field.Store.YES);
     } else {
+      parentStreamField = null;
+
       fullPathField = new StringField(Consts.FULL, "", Field.Store.NO);
     }
 
@@ -466,23 +474,28 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
    * effectively synchronized as well.
    */
   private int addCategoryDocument(FacetLabel categoryPath, int parent) throws IOException {
-    // Before Lucene 2.9, position increments >=0 were supported, so we
-    // added 1 to parent to allow the parent -1 (the parent of the root).
-    // Unfortunately, starting with Lucene 2.9, after LUCENE-1542, this is
-    // no longer enough, since 0 is not encoded consistently either (see
-    // comment in SinglePositionTokenStream). But because we must be
-    // backward-compatible with existing indexes, we can't just fix what
-    // we write here (e.g., to write parent+2), and need to do a workaround
-    // in the reader (which knows that anyway only category 0 has a parent
-    // -1).
-    parentStream.set(Math.max(parent + 1, 1));
     Document d = new Document();
-    d.add(parentStreamField);
+    if (useOlderFormat) {
+      // Before Lucene 2.9, position increments >=0 were supported, so we
+      // added 1 to parent to allow the parent -1 (the parent of the root).
+      // Unfortunately, starting with Lucene 2.9, after LUCENE-1542, this is
+      // no longer enough, since 0 is not encoded consistently either (see
+      // comment in SinglePositionTokenStream). But because we must be
+      // backward-compatible with existing indexes, we can't just fix what
+      // we write here (e.g., to write parent+2), and need to do a workaround
+      // in the reader (which knows that anyway only category 0 has a parent
+      // -1).
+      assert parentStreamField != null;
+      parentStream.set(Math.max(parent + 1, 1));
+      d.add(parentStreamField);
+    } else {
+      d.add(new NumericDocValuesField(Consts.FIELD_PARENT_ORDINAL_NDV, parent));
+    }
 
     String fieldPath = FacetsConfig.pathToString(categoryPath.components, categoryPath.length);
     fullPathField.setStringValue(fieldPath);
 
-    if (useOlderStoredFieldIndex == false) {
+    if (useOlderFormat == false) {
       /* Lucene 9 switches to BinaryDocValuesField for storing taxonomy categories */
       d.add(new BinaryDocValuesField(Consts.FULL, new BytesRef(fieldPath)));
     }
@@ -508,6 +521,7 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
     return id;
   }
 
+  // TODO: remove this class in Lucene 10
   private static class SinglePositionTokenStream extends TokenStream {
     private CharTermAttribute termAtt;
     private PositionIncrementAttribute posIncrAtt;
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java
index 35c0495..a73b9a3 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java
@@ -25,7 +25,9 @@ import org.apache.lucene.facet.taxonomy.ParallelTaxonomyArrays;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiTerms;
+import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.Accountable;
@@ -58,13 +60,6 @@ class TaxonomyIndexArrays extends ParallelTaxonomyArrays implements Accountable
     parents = new int[reader.maxDoc()];
     if (parents.length > 0) {
       initParents(reader, 0);
-      // Starting Lucene 2.9, following the change LUCENE-1542, we can
-      // no longer reliably read the parent "-1" (see comment in
-      // LuceneTaxonomyWriter.SinglePositionTokenStream). We have no way
-      // to fix this in indexing without breaking backward-compatibility
-      // with existing indexes, so what we'll do instead is just
-      // hard-code the parent of ordinal 0 to be -1, and assume (as is
-      // indeed the case) that no other parent can be -1.
       parents[0] = TaxonomyReader.INVALID_ORDINAL;
     }
   }
@@ -130,6 +125,46 @@ class TaxonomyIndexArrays extends ParallelTaxonomyArrays implements Accountable
       return;
     }
 
+    if (getMajorVersion(reader) <= 8) {
+      loadParentUsingTermPosition(reader, first);
+      return;
+    }
+
+    for (LeafReaderContext leafContext : reader.leaves()) {
+      int leafDocNum = leafContext.reader().maxDoc();
+      if (leafContext.docBase + leafDocNum <= first) {
+        // skip this leaf if it does not contain new categories
+        continue;
+      }
+      NumericDocValues parentValues =
+          leafContext.reader().getNumericDocValues(Consts.FIELD_PARENT_ORDINAL_NDV);
+      if (parentValues == null) {
+        throw new CorruptIndexException(
+            "Parent data field " + Consts.FIELD_PARENT_ORDINAL_NDV + " not exists",
+            leafContext.reader().toString());
+      }
+
+      for (int doc = Math.max(first - leafContext.docBase, 0); doc < leafDocNum; doc++) {
+        if (parentValues.advanceExact(doc) == false) {
+          throw new CorruptIndexException(
+              "Missing parent data for category " + (doc + leafContext.docBase), reader.toString());
+        }
+        // we're putting an int and converting it back so it should be safe
+        parents[doc + leafContext.docBase] = Math.toIntExact(parentValues.longValue());
+      }
+    }
+  }
+
+  private static int getMajorVersion(IndexReader reader) {
+    return reader.leaves().get(0).reader().getMetaData().getCreatedVersionMajor();
+  }
+
+  /**
+   * Try loading the old way of storing parent ordinal first, return true if the parent array is
+   * loaded Or false if not, and we will try loading using NumericDocValues
+   */
+  // TODO: Remove in Lucene 10, this is only for back-compatibility
+  private void loadParentUsingTermPosition(IndexReader reader, int first) throws IOException {
     // it's ok to use MultiTerms because we only iterate on one posting list.
     // breaking it to loop over the leaves() only complicates code for no
     // apparent gain.
@@ -140,7 +175,7 @@ class TaxonomyIndexArrays extends ParallelTaxonomyArrays implements Accountable
     // shouldn't really happen, if it does, something's wrong
     if (positions == null || positions.advance(first) == DocIdSetIterator.NO_MORE_DOCS) {
       throw new CorruptIndexException(
-          "Missing parent data for category " + first, reader.toString());
+          "[Lucene 8] Missing parent data for category " + first, reader.toString());
     }
 
     int num = reader.maxDoc();
@@ -148,7 +183,7 @@ class TaxonomyIndexArrays extends ParallelTaxonomyArrays implements Accountable
       if (positions.docID() == i) {
         if (positions.freq() == 0) { // shouldn't happen
           throw new CorruptIndexException(
-              "Missing parent data for category " + i, reader.toString());
+              "[Lucene 8] Missing parent data for category " + i, reader.toString());
         }
 
         parents[i] = positions.nextPosition();
@@ -156,12 +191,13 @@ class TaxonomyIndexArrays extends ParallelTaxonomyArrays implements Accountable
         if (positions.nextDoc() == DocIdSetIterator.NO_MORE_DOCS) {
           if (i + 1 < num) {
             throw new CorruptIndexException(
-                "Missing parent data for category " + (i + 1), reader.toString());
+                "[Lucene 8] Missing parent data for category " + (i + 1), reader.toString());
           }
           break;
         }
       } else { // this shouldn't happen
-        throw new CorruptIndexException("Missing parent data for category " + i, reader.toString());
+        throw new CorruptIndexException(
+            "[Lucene 8] Missing parent data for category " + i, reader.toString());
       }
     }
   }