You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2022/01/06 08:27:21 UTC

[lucene] branch branch_9x updated (7572352 -> a87cd9a)

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

jpountz pushed a change to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git.


    from 7572352  LUCENE-10291: Bug fix.
     new 6c1b9f7  LUCENE-10354: Clarify contract of codec APIs with missing/disabled fields. (#583)
     new a87cd9a  Fix path of docs for import into the website. (#524)

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 dev-tools/scripts/releaseWizard.yaml                 |  2 +-
 .../org/apache/lucene/codecs/DocValuesProducer.java  | 18 +++++++++++++-----
 .../org/apache/lucene/codecs/KnnVectorsReader.java   | 10 +++++++++-
 .../java/org/apache/lucene/codecs/NormsProducer.java |  3 ++-
 .../java/org/apache/lucene/codecs/PointsReader.java  |  6 +++++-
 .../codecs/lucene90/Lucene90HnswVectorsReader.java   |  8 +-------
 .../java/org/apache/lucene/index/CodecReader.java    | 12 +++++-------
 .../codecs/asserting/AssertingKnnVectorsFormat.java  | 20 +++++++++++++-------
 .../codecs/asserting/AssertingPointsFormat.java      | 12 +++++++++---
 9 files changed, 58 insertions(+), 33 deletions(-)

[lucene] 01/02: LUCENE-10354: Clarify contract of codec APIs with missing/disabled fields. (#583)

Posted by jp...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 6c1b9f74e855b32c40be892abcd9c143e35bc30b
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Wed Jan 5 18:47:35 2022 +0100

    LUCENE-10354: Clarify contract of codec APIs with missing/disabled fields. (#583)
---
 .../org/apache/lucene/codecs/DocValuesProducer.java  | 18 +++++++++++++-----
 .../org/apache/lucene/codecs/KnnVectorsReader.java   | 10 +++++++++-
 .../java/org/apache/lucene/codecs/NormsProducer.java |  3 ++-
 .../java/org/apache/lucene/codecs/PointsReader.java  |  6 +++++-
 .../codecs/lucene90/Lucene90HnswVectorsReader.java   |  8 +-------
 .../java/org/apache/lucene/index/CodecReader.java    | 12 +++++-------
 .../codecs/asserting/AssertingKnnVectorsFormat.java  | 20 +++++++++++++-------
 .../codecs/asserting/AssertingPointsFormat.java      | 12 +++++++++---
 8 files changed, 57 insertions(+), 32 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
index 96d381a..2731bcc 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs;
 import java.io.Closeable;
 import java.io.IOException;
 import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
@@ -37,31 +38,38 @@ public abstract class DocValuesProducer implements Closeable {
 
   /**
    * Returns {@link NumericDocValues} for this field. The returned instance need not be thread-safe:
-   * it will only be used by a single thread.
+   * it will only be used by a single thread. The behavior is undefined if the doc values type of
+   * the given field is not {@link DocValuesType#NUMERIC}. The return value is never {@code null}.
    */
   public abstract NumericDocValues getNumeric(FieldInfo field) throws IOException;
 
   /**
    * Returns {@link BinaryDocValues} for this field. The returned instance need not be thread-safe:
-   * it will only be used by a single thread.
+   * it will only be used by a single thread. The behavior is undefined if the doc values type of
+   * the given field is not {@link DocValuesType#BINARY}. The return value is never {@code null}.
    */
   public abstract BinaryDocValues getBinary(FieldInfo field) throws IOException;
 
   /**
    * Returns {@link SortedDocValues} for this field. The returned instance need not be thread-safe:
-   * it will only be used by a single thread.
+   * it will only be used by a single thread. The behavior is undefined if the doc values type of
+   * the given field is not {@link DocValuesType#SORTED}. The return value is never {@code null}.
    */
   public abstract SortedDocValues getSorted(FieldInfo field) throws IOException;
 
   /**
    * Returns {@link SortedNumericDocValues} for this field. The returned instance need not be
-   * thread-safe: it will only be used by a single thread.
+   * thread-safe: it will only be used by a single thread. The behavior is undefined if the doc
+   * values type of the given field is not {@link DocValuesType#SORTED_NUMERIC}. The return value is
+   * never {@code null}.
    */
   public abstract SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException;
 
   /**
    * Returns {@link SortedSetDocValues} for this field. The returned instance need not be
-   * thread-safe: it will only be used by a single thread.
+   * thread-safe: it will only be used by a single thread. The behavior is undefined if the doc
+   * values type of the given field is not {@link DocValuesType#SORTED_SET}. The return value is
+   * never {@code null}.
    */
   public abstract SortedSetDocValues getSortedSet(FieldInfo field) throws IOException;
 
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java
index d89e5c7..6407559 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs;
 
 import java.io.Closeable;
 import java.io.IOException;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.VectorValues;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.util.Accountable;
@@ -40,7 +41,11 @@ public abstract class KnnVectorsReader implements Closeable, Accountable {
    */
   public abstract void checkIntegrity() throws IOException;
 
-  /** Returns the {@link VectorValues} for the given {@code field} */
+  /**
+   * Returns the {@link VectorValues} for the given {@code field}. The behavior is undefined if the
+   * given field doesn't have KNN vectors enabled on its {@link FieldInfo}. The return value is
+   * never {@code null}.
+   */
   public abstract VectorValues getVectorValues(String field) throws IOException;
 
   /**
@@ -53,6 +58,9 @@ public abstract class KnnVectorsReader implements Closeable, Accountable {
    * true k closest neighbors. For large values of k (for example when k is close to the total
    * number of documents), the search may also retrieve fewer than k documents.
    *
+   * <p>The behavior is undefined if the given field doesn't have KNN vectors enabled on its {@link
+   * FieldInfo}. The return value is never {@code null}.
+   *
    * @param field the vector field to search
    * @param target the vector-valued query
    * @param k the number of docs to return
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/NormsProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/NormsProducer.java
index 6df7d7e..064e841 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/NormsProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/NormsProducer.java
@@ -33,7 +33,8 @@ public abstract class NormsProducer implements Closeable {
 
   /**
    * Returns {@link NumericDocValues} for this field. The returned instance need not be thread-safe:
-   * it will only be used by a single thread.
+   * it will only be used by a single thread. The behavior is undefined if the given field doesn't
+   * have norms enabled on its {@link FieldInfo}. The return value is never {@code null}.
    */
   public abstract NumericDocValues getNorms(FieldInfo field) throws IOException;
 
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PointsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/PointsReader.java
index 241c6dc..cc85dc0 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/PointsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PointsReader.java
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs;
 
 import java.io.Closeable;
 import java.io.IOException;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.PointValues;
 
 /**
@@ -40,7 +41,10 @@ public abstract class PointsReader implements Closeable {
    */
   public abstract void checkIntegrity() throws IOException;
 
-  /** Return {@link PointValues} for the given {@code field}. */
+  /**
+   * Return {@link PointValues} for the given {@code field}. The behavior is undefined if the given
+   * field doesn't have points enabled on its {@link FieldInfo}.
+   */
   public abstract PointValues getValues(String field) throws IOException;
 
   /**
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java
index b0ac8a9..2ae1bc4 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorsReader.java
@@ -226,19 +226,13 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader {
   @Override
   public VectorValues getVectorValues(String field) throws IOException {
     FieldEntry fieldEntry = fields.get(field);
-    if (fieldEntry == null || fieldEntry.dimension == 0) {
-      return null;
-    }
-
     return getOffHeapVectorValues(fieldEntry);
   }
 
   @Override
   public TopDocs search(String field, float[] target, int k, Bits acceptDocs) throws IOException {
     FieldEntry fieldEntry = fields.get(field);
-    if (fieldEntry == null || fieldEntry.dimension == 0) {
-      return null;
-    }
+
     if (fieldEntry.size() == 0) {
       return new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), new ScoreDoc[0]);
     }
diff --git a/lucene/core/src/java/org/apache/lucene/index/CodecReader.java b/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
index 064cdbf..9c05098 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
@@ -105,15 +105,13 @@ public abstract class CodecReader extends LeafReader {
 
   @Override
   public final Terms terms(String field) throws IOException {
-    // ensureOpen(); no; getPostingsReader calls this
-    FieldsProducer fieldsProducer = getPostingsReader();
-    if (fieldsProducer == null) {
+    ensureOpen();
+    FieldInfo fi = getFieldInfos().fieldInfo(field);
+    if (fi == null || fi.getIndexOptions() == IndexOptions.NONE) {
+      // Field does not exist or does not index postings
       return null;
     }
-    // We could check the FieldInfo IndexOptions but there's no point since
-    //   PostingsReader will simply return null for fields that don't exist or that have no terms
-    // index.
-    return fieldsProducer.terms(field);
+    return getPostingsReader().terms(field);
   }
 
   // returns the FieldInfo that corresponds to the given field and type, or
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/asserting/AssertingKnnVectorsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/asserting/AssertingKnnVectorsFormat.java
index 6b17d36..55b17d5 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/asserting/AssertingKnnVectorsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/asserting/AssertingKnnVectorsFormat.java
@@ -22,6 +22,7 @@ import org.apache.lucene.codecs.KnnVectorsFormat;
 import org.apache.lucene.codecs.KnnVectorsReader;
 import org.apache.lucene.codecs.KnnVectorsWriter;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.VectorValues;
@@ -45,7 +46,7 @@ public class AssertingKnnVectorsFormat extends KnnVectorsFormat {
 
   @Override
   public KnnVectorsReader fieldsReader(SegmentReadState state) throws IOException {
-    return new AssertingKnnVectorsReader(delegate.fieldsReader(state));
+    return new AssertingKnnVectorsReader(delegate.fieldsReader(state), state.fieldInfos);
   }
 
   static class AssertingKnnVectorsWriter extends KnnVectorsWriter {
@@ -76,10 +77,12 @@ public class AssertingKnnVectorsFormat extends KnnVectorsFormat {
 
   static class AssertingKnnVectorsReader extends KnnVectorsReader {
     final KnnVectorsReader delegate;
+    final FieldInfos fis;
 
-    AssertingKnnVectorsReader(KnnVectorsReader delegate) {
+    AssertingKnnVectorsReader(KnnVectorsReader delegate, FieldInfos fis) {
       assert delegate != null;
       this.delegate = delegate;
+      this.fis = fis;
     }
 
     @Override
@@ -89,17 +92,20 @@ public class AssertingKnnVectorsFormat extends KnnVectorsFormat {
 
     @Override
     public VectorValues getVectorValues(String field) throws IOException {
+      FieldInfo fi = fis.fieldInfo(field);
+      assert fi != null && fi.getVectorDimension() > 0;
       VectorValues values = delegate.getVectorValues(field);
-      if (values != null) {
-        assert values.docID() == -1;
-        assert values.size() >= 0;
-        assert values.dimension() > 0;
-      }
+      assert values != null;
+      assert values.docID() == -1;
+      assert values.size() >= 0;
+      assert values.dimension() > 0;
       return values;
     }
 
     @Override
     public TopDocs search(String field, float[] target, int k, Bits acceptDocs) throws IOException {
+      FieldInfo fi = fis.fieldInfo(field);
+      assert fi != null && fi.getVectorDimension() > 0;
       TopDocs hits = delegate.search(field, target, k, acceptDocs);
       assert hits != null;
       assert hits.scoreDocs.length <= k;
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/asserting/AssertingPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/asserting/AssertingPointsFormat.java
index 2b68a59..eada1e8 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/asserting/AssertingPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/asserting/AssertingPointsFormat.java
@@ -21,6 +21,7 @@ import org.apache.lucene.codecs.PointsFormat;
 import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.PointsWriter;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.PointValues;
 import org.apache.lucene.index.SegmentReadState;
@@ -54,18 +55,21 @@ public final class AssertingPointsFormat extends PointsFormat {
 
   @Override
   public PointsReader fieldsReader(SegmentReadState state) throws IOException {
-    return new AssertingPointsReader(state.segmentInfo.maxDoc(), in.fieldsReader(state), false);
+    return new AssertingPointsReader(
+        state.segmentInfo.maxDoc(), in.fieldsReader(state), state.fieldInfos, false);
   }
 
   static class AssertingPointsReader extends PointsReader {
     private final PointsReader in;
     private final int maxDoc;
+    private final FieldInfos fis;
     private final boolean merging;
     private final Thread creationThread;
 
-    AssertingPointsReader(int maxDoc, PointsReader in, boolean merging) {
+    AssertingPointsReader(int maxDoc, PointsReader in, FieldInfos fis, boolean merging) {
       this.in = in;
       this.maxDoc = maxDoc;
+      this.fis = fis;
       this.merging = merging;
       this.creationThread = Thread.currentThread();
       // do a few simple checks on init
@@ -80,6 +84,8 @@ public final class AssertingPointsFormat extends PointsFormat {
 
     @Override
     public PointValues getValues(String field) throws IOException {
+      FieldInfo fi = fis.fieldInfo(field);
+      assert fi != null && fi.getPointDimensionCount() > 0;
       if (merging) {
         AssertingCodec.assertThread("PointsReader", creationThread);
       }
@@ -97,7 +103,7 @@ public final class AssertingPointsFormat extends PointsFormat {
 
     @Override
     public PointsReader getMergeInstance() {
-      return new AssertingPointsReader(maxDoc, in.getMergeInstance(), true);
+      return new AssertingPointsReader(maxDoc, in.getMergeInstance(), fis, true);
     }
 
     @Override

[lucene] 02/02: Fix path of docs for import into the website. (#524)

Posted by jp...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit a87cd9ae16bdd49ed154524cadaa138b16be2005
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Thu Jan 6 09:26:45 2022 +0100

    Fix path of docs for import into the website. (#524)
    
    The current `svn import` looks for docs where they used to be produced by the
    `Ant` build, but `Gradle` now puts them in a different place.
---
 dev-tools/scripts/releaseWizard.yaml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-tools/scripts/releaseWizard.yaml b/dev-tools/scripts/releaseWizard.yaml
index cc31447..a36363e 100644
--- a/dev-tools/scripts/releaseWizard.yaml
+++ b/dev-tools/scripts/releaseWizard.yaml
@@ -977,7 +977,7 @@ groups:
         cmd: "{{ gradle_cmd }} documentation -Dversion.release={{ release_version }}"
         comment: Build documentation
       - !Command
-        cmd: svn -m "Add docs, changes and javadocs for Lucene {{ release_version }}"  import {{ git_checkout_folder }}/lucene/build/docs  https://svn.apache.org/repos/infra/sites/lucene/core/{{ version }}
+        cmd: svn -m "Add docs, changes and javadocs for Lucene {{ release_version }}"  import {{ git_checkout_folder }}/lucene/documentation/build/site  https://svn.apache.org/repos/infra/sites/lucene/core/{{ version }}
         logfile: add-docs-lucene.log
         comment: Add docs for Lucene
   - !Todo