You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ju...@apache.org on 2021/02/03 21:28:56 UTC

[lucene-solr] branch branch_8x updated (3a47e07 -> fcb922b)

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

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


    from 3a47e07  LUCENE-9680: remove deprecation from getFieldNames (#2298)
     new 3698623  Improve backwards compatibility tests for sorted indexes. (#2276)
     new fcb922b  Remove write logic from Lucene70NormsFormat. (#2287)

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:
 .../lucene/codecs/lucene70/Lucene70Codec.java      |  66 +++---
 .../codecs/lucene70/Lucene70NormsFormat.java       | 106 +++++----
 .../codecs/lucene70/Lucene70NormsConsumer.java     |  30 ++-
 .../lucene/codecs/lucene70/Lucene70RWCodec.java    |  23 +-
 .../Lucene70RWNormsFormat.java}                    |  17 +-
 .../lucene/index/TestBackwardsCompatibility.java   | 257 +++++++++++----------
 6 files changed, 267 insertions(+), 232 deletions(-)
 rename lucene/backward-codecs/src/{java => test}/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java (84%)
 copy lucene/backward-codecs/src/test/org/apache/lucene/codecs/{lucene60/Lucene60RWPointsFormat.java => lucene70/Lucene70RWNormsFormat.java} (69%)


[lucene-solr] 02/02: Remove write logic from Lucene70NormsFormat. (#2287)

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

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

commit fcb922bea18fde00d0d657947b1554addb49b47e
Author: Julie Tibshirani <ju...@elastic.co>
AuthorDate: Wed Feb 3 09:28:48 2021 -0800

    Remove write logic from Lucene70NormsFormat. (#2287)
    
    Our policy is to not maintain write logic for old formats that can't be written
    to. The write logic is moved to the test folder to support unit testing.
---
 .../lucene/codecs/lucene70/Lucene70Codec.java      |  66 ++++++-------
 .../codecs/lucene70/Lucene70NormsFormat.java       | 106 +++++++++++----------
 .../codecs/lucene70/Lucene70NormsConsumer.java     |  30 ++++--
 .../lucene/codecs/lucene70/Lucene70RWCodec.java    |  23 +++--
 .../codecs/lucene70/Lucene70RWNormsFormat.java     |  30 ++++++
 5 files changed, 153 insertions(+), 102 deletions(-)

diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java
index 5107e07..b033777 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java
@@ -30,23 +30,21 @@ import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
 import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
 import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
 import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
-import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
 import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
 import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 
 /**
- * Implements the Lucene 7.0 index format, with configurable per-field postings
- * and docvalues formats.
- * <p>
- * If you want to reuse functionality of this codec in another codec, extend
- * {@link FilterCodec}.
+ * Implements the Lucene 7.0 index format, with configurable per-field postings and docvalues
+ * formats.
  *
- * @see org.apache.lucene.codecs.lucene70 package documentation for file format details.
+ * <p>If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}.
  *
+ * @see org.apache.lucene.codecs.lucene70 package documentation for file format details.
  * @lucene.experimental
  */
 public class Lucene70Codec extends Codec {
@@ -57,34 +55,36 @@ public class Lucene70Codec extends Codec {
   private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
   private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene70");
 
-  private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
-    @Override
-    public PostingsFormat getPostingsFormatForField(String field) {
-      throw new IllegalStateException("This codec should only be used for reading, not writing");
-    }
-  };
-  
-  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
-    @Override
-    public DocValuesFormat getDocValuesFormatForField(String field) {
-      return defaultDVFormat;
-    }
-  };
-  
-  private final StoredFieldsFormat storedFieldsFormat = new Lucene50StoredFieldsFormat(Mode.BEST_SPEED);
-
-  /** 
-   * Instantiates a new codec.
-   */
+  private final PostingsFormat postingsFormat =
+      new PerFieldPostingsFormat() {
+        @Override
+        public PostingsFormat getPostingsFormatForField(String field) {
+          throw new IllegalStateException(
+              "This codec should only be used for reading, not writing");
+        }
+      };
+
+  private final DocValuesFormat docValuesFormat =
+      new PerFieldDocValuesFormat() {
+        @Override
+        public DocValuesFormat getDocValuesFormatForField(String field) {
+          return defaultDVFormat;
+        }
+      };
+
+  private final StoredFieldsFormat storedFieldsFormat =
+      new Lucene50StoredFieldsFormat(Mode.BEST_SPEED);
+
+  /** Instantiates a new codec. */
   public Lucene70Codec() {
     super("Lucene70");
   }
-  
+
   @Override
   public StoredFieldsFormat storedFieldsFormat() {
     return storedFieldsFormat;
   }
-  
+
   @Override
   public final TermVectorsFormat termVectorsFormat() {
     return vectorsFormat;
@@ -94,17 +94,17 @@ public class Lucene70Codec extends Codec {
   public PostingsFormat postingsFormat() {
     return postingsFormat;
   }
-  
+
   @Override
   public final FieldInfosFormat fieldInfosFormat() {
     return fieldInfosFormat;
   }
-  
+
   @Override
   public SegmentInfoFormat segmentInfoFormat() {
     return segmentInfosFormat;
   }
-  
+
   @Override
   public final LiveDocsFormat liveDocsFormat() {
     return liveDocsFormat;
@@ -119,7 +119,7 @@ public class Lucene70Codec extends Codec {
   public final PointsFormat pointsFormat() {
     return new Lucene60PointsFormat();
   }
-  
+
   @Override
   public final DocValuesFormat docValuesFormat() {
     return docValuesFormat;
@@ -128,7 +128,7 @@ public class Lucene70Codec extends Codec {
   private final NormsFormat normsFormat = new Lucene70NormsFormat();
 
   @Override
-  public final NormsFormat normsFormat() {
+  public NormsFormat normsFormat() {
     return normsFormat;
   }
 }
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsFormat.java
index 7c764fe..3c42cac 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsFormat.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsFormat.java
@@ -17,7 +17,6 @@
 package org.apache.lucene.codecs.lucene70;
 
 import java.io.IOException;
-
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.NormsConsumer;
 import org.apache.lucene.codecs.NormsFormat;
@@ -28,72 +27,79 @@ import org.apache.lucene.store.DataOutput;
 
 /**
  * Lucene 7.0 Score normalization format.
- * <p>
- * Encodes normalization values by encoding each value with the minimum
- * number of bytes needed to represent the range (which can be zero).
- * <p>
- * Files:
+ *
+ * <p>Encodes normalization values by encoding each value with the minimum number of bytes needed to
+ * represent the range (which can be zero).
+ *
+ * <p>Files:
+ *
  * <ol>
- *   <li><tt>.nvd</tt>: Norms data</li>
- *   <li><tt>.nvm</tt>: Norms metadata</li>
+ *   <li><code>.nvd</code>: Norms data
+ *   <li><code>.nvm</code>: Norms metadata
  * </ol>
+ *
  * <ol>
- *   <li><a name="nvm"></a>
- *   <p>The Norms metadata or .nvm file.</p>
- *   <p>For each norms field, this stores metadata, such as the offset into the 
- *      Norms data (.nvd)</p>
- *   <p>Norms metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
- *   <ul>
- *     <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
- *     <li>Entry --&gt; FieldNumber, DocsWithFieldAddress, DocsWithFieldLength, NumDocsWithField, BytesPerNorm, NormsAddress</li>
- *     <li>FieldNumber --&gt; {@link DataOutput#writeInt Int32}</li>
- *     <li>DocsWithFieldAddress --&gt; {@link DataOutput#writeLong Int64}</li>
- *     <li>DocsWithFieldLength --&gt; {@link DataOutput#writeLong Int64}</li>
- *     <li>NumDocsWithField --&gt; {@link DataOutput#writeInt Int32}</li>
- *     <li>BytesPerNorm --&gt; {@link DataOutput#writeByte byte}</li>
- *     <li>NormsAddress --&gt; {@link DataOutput#writeLong Int64}</li>
- *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- *   </ul>
- *   <p>FieldNumber of -1 indicates the end of metadata.</p>
- *   <p>NormsAddress is the pointer to the start of the data in the norms data (.nvd), or the singleton value 
- *      when BytesPerValue = 0. If BytesPerValue is different from 0 then there are NumDocsWithField values
- *      to read at that offset.</p>
- *   <p>DocsWithFieldAddress is the pointer to the start of the bit set containing documents that have a norm
- *      in the norms data (.nvd), or -2 if no documents have a norm value, or -1 if all documents have a norm
- *      value.</p>
- *   <p>DocsWithFieldLength is the number of bytes used to encode the set of documents that have a norm.</p>
- *   <li><a name="nvd"></a>
- *   <p>The Norms data or .nvd file.</p>
- *   <p>For each Norms field, this stores the actual per-document data (the heavy-lifting)</p>
- *   <p>Norms data (.nvd) --&gt; Header,&lt; Data &gt;<sup>NumFields</sup>,Footer</p>
- *   <ul>
- *     <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
- *     <li>DocsWithFieldData --&gt; Bit set of MaxDoc bits</li>
- *     <li>NormsData --&gt; {@link DataOutput#writeByte(byte) byte}<sup>NumDocsWithField * BytesPerValue</sup></li>
- *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- *   </ul>
+ *   <li><a id="nvm"></a>
+ *       <p>The Norms metadata or .nvm file.
+ *       <p>For each norms field, this stores metadata, such as the offset into the Norms data
+ *       (.nvd)
+ *       <p>Norms metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer
+ *       <ul>
+ *         <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
+ *         <li>Entry --&gt; FieldNumber, DocsWithFieldAddress, DocsWithFieldLength,
+ *             NumDocsWithField, BytesPerNorm, NormsAddress
+ *         <li>FieldNumber --&gt; {@link DataOutput#writeInt Int32}
+ *         <li>DocsWithFieldAddress --&gt; {@link DataOutput#writeLong Int64}
+ *         <li>DocsWithFieldLength --&gt; {@link DataOutput#writeLong Int64}
+ *         <li>NumDocsWithField --&gt; {@link DataOutput#writeInt Int32}
+ *         <li>BytesPerNorm --&gt; {@link DataOutput#writeByte byte}
+ *         <li>NormsAddress --&gt; {@link DataOutput#writeLong Int64}
+ *         <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
+ *       </ul>
+ *       <p>FieldNumber of -1 indicates the end of metadata.
+ *       <p>NormsAddress is the pointer to the start of the data in the norms data (.nvd), or the
+ *       singleton value when BytesPerValue = 0. If BytesPerValue is different from 0 then there are
+ *       NumDocsWithField values to read at that offset.
+ *       <p>DocsWithFieldAddress is the pointer to the start of the bit set containing documents
+ *       that have a norm in the norms data (.nvd), or -2 if no documents have a norm value, or -1
+ *       if all documents have a norm value.
+ *       <p>DocsWithFieldLength is the number of bytes used to encode the set of documents that have
+ *       a norm.
+ *   <li><a id="nvd"></a>
+ *       <p>The Norms data or .nvd file.
+ *       <p>For each Norms field, this stores the actual per-document data (the heavy-lifting)
+ *       <p>Norms data (.nvd) --&gt; Header,&lt; Data &gt;<sup>NumFields</sup>,Footer
+ *       <ul>
+ *         <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}
+ *         <li>DocsWithFieldData --&gt; Bit set of MaxDoc bits
+ *         <li>NormsData --&gt; {@link DataOutput#writeByte(byte) byte}<sup>NumDocsWithField *
+ *             BytesPerValue</sup>
+ *         <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}
+ *       </ul>
  * </ol>
+ *
  * @lucene.experimental
  */
 public class Lucene70NormsFormat extends NormsFormat {
 
   /** Sole Constructor */
   public Lucene70NormsFormat() {}
-  
+
   @Override
   public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
-    return new Lucene70NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+    throw new UnsupportedOperationException("Old codecs may only be used for reading");
   }
 
   @Override
   public NormsProducer normsProducer(SegmentReadState state) throws IOException {
-    return new Lucene70NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+    return new Lucene70NormsProducer(
+        state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
   }
-  
-  private static final String DATA_CODEC = "Lucene70NormsData";
-  private static final String DATA_EXTENSION = "nvd";
-  private static final String METADATA_CODEC = "Lucene70NormsMetadata";
-  private static final String METADATA_EXTENSION = "nvm";
+
+  static final String DATA_CODEC = "Lucene70NormsData";
+  static final String DATA_EXTENSION = "nvd";
+  static final String METADATA_CODEC = "Lucene70NormsMetadata";
+  static final String METADATA_EXTENSION = "nvm";
   static final int VERSION_START = 0;
   static final int VERSION_CURRENT = VERSION_START;
 }
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java
similarity index 84%
rename from lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java
rename to lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java
index d79e246..3f840e7 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70NormsConsumer.java
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.lucene70;
 import static org.apache.lucene.codecs.lucene70.Lucene70NormsFormat.VERSION_CURRENT;
 
 import java.io.IOException;
-
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.NormsConsumer;
 import org.apache.lucene.codecs.NormsProducer;
@@ -31,22 +30,32 @@ import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.IOUtils;
 
-/**
- * Writer for {@link Lucene70NormsFormat}
- */
+/** Writer for {@link Lucene70NormsFormat} */
 final class Lucene70NormsConsumer extends NormsConsumer {
   IndexOutput data, meta;
   final int maxDoc;
 
-  Lucene70NormsConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+  Lucene70NormsConsumer(
+      SegmentWriteState state,
+      String dataCodec,
+      String dataExtension,
+      String metaCodec,
+      String metaExtension)
+      throws IOException {
     boolean success = false;
     try {
-      String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+      String dataName =
+          IndexFileNames.segmentFileName(
+              state.segmentInfo.name, state.segmentSuffix, dataExtension);
       data = state.directory.createOutput(dataName, state.context);
-      CodecUtil.writeIndexHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
-      String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+      CodecUtil.writeIndexHeader(
+          data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      String metaName =
+          IndexFileNames.segmentFileName(
+              state.segmentInfo.name, state.segmentSuffix, metaExtension);
       meta = state.directory.createOutput(metaName, state.context);
-      CodecUtil.writeIndexHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      CodecUtil.writeIndexHeader(
+          meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
       maxDoc = state.segmentInfo.maxDoc();
       success = true;
     } finally {
@@ -135,7 +144,8 @@ final class Lucene70NormsConsumer extends NormsConsumer {
     }
   }
 
-  private void writeValues(NumericDocValues values, int numBytesPerValue, IndexOutput out) throws IOException, AssertionError {
+  private void writeValues(NumericDocValues values, int numBytesPerValue, IndexOutput out)
+      throws IOException, AssertionError {
     for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
       long value = values.longValue();
       switch (numBytesPerValue) {
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70RWCodec.java
index a9bafc4..5847ca7 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70RWCodec.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70RWCodec.java
@@ -16,6 +16,7 @@
  */
 package org.apache.lucene.codecs.lucene70;
 
+import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
@@ -23,18 +24,17 @@ import org.apache.lucene.codecs.lucene50.Lucene50RWPostingsFormat;
 import org.apache.lucene.codecs.lucene50.Lucene50RWStoredFieldsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 
-/**
- * RW impersonation of {@link Lucene70Codec}.
- */
+/** RW impersonation of {@link Lucene70Codec}. */
 public final class Lucene70RWCodec extends Lucene70Codec {
 
   private final PostingsFormat defaultPF = new Lucene50RWPostingsFormat();
-  private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
-    @Override
-    public PostingsFormat getPostingsFormatForField(String field) {
-      return defaultPF;
-    }
-  };
+  private final PostingsFormat postingsFormat =
+      new PerFieldPostingsFormat() {
+        @Override
+        public PostingsFormat getPostingsFormatForField(String field) {
+          return defaultPF;
+        }
+      };
 
   /** Sole constructor. */
   public Lucene70RWCodec() {}
@@ -45,6 +45,11 @@ public final class Lucene70RWCodec extends Lucene70Codec {
   }
 
   @Override
+  public NormsFormat normsFormat() {
+    return new Lucene70RWNormsFormat();
+  }
+
+  @Override
   public StoredFieldsFormat storedFieldsFormat() {
     return new Lucene50RWStoredFieldsFormat();
   }
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70RWNormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70RWNormsFormat.java
new file mode 100644
index 0000000..7ec1101
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene70/Lucene70RWNormsFormat.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.codecs.lucene70;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.NormsConsumer;
+import org.apache.lucene.index.SegmentWriteState;
+
+public class Lucene70RWNormsFormat extends Lucene70NormsFormat {
+
+  @Override
+  public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
+    return new Lucene70NormsConsumer(
+        state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+  }
+}


[lucene-solr] 01/02: Improve backwards compatibility tests for sorted indexes. (#2276)

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

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

commit 369862349950506fc1b2a23cccbd80d2467a2dda
Author: Julie Tibshirani <ju...@elastic.co>
AuthorDate: Wed Feb 3 09:55:51 2021 -0800

    Improve backwards compatibility tests for sorted indexes. (#2276)
    
    This PR also cleans up some old checks that only applied to pre-6.0 indices.
---
 .../lucene/index/TestBackwardsCompatibility.java   | 257 +++++++++++----------
 1 file changed, 138 insertions(+), 119 deletions(-)

diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
index 666adab..188350f 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
@@ -61,11 +61,16 @@ import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DocValuesFieldExistsQuery;
+import org.apache.lucene.search.FieldDoc;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.NormsFieldExistsQuery;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.BaseDirectoryWrapper;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
@@ -949,19 +954,9 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     IndexSearcher searcher = newSearcher(reader);
 
     TestUtil.checkIndex(dir);
-    
-    // true if this is a 4.0+ index
-    final boolean is40Index = FieldInfos.getMergedFieldInfos(reader).fieldInfo("content5") != null;
-    // true if this is a 4.2+ index
-    final boolean is42Index = FieldInfos.getMergedFieldInfos(reader).fieldInfo("dvSortedSet") != null;
-    // true if this is a 4.9+ index
-    final boolean is49Index = FieldInfos.getMergedFieldInfos(reader).fieldInfo("dvSortedNumeric") != null;
-    // true if this index has points (>= 6.0)
-    final boolean hasPoints = FieldInfos.getMergedFieldInfos(reader).fieldInfo("intPoint1d") != null;
-
-    assert is40Index;
 
     final Bits liveDocs = MultiBits.getLiveDocs(reader);
+    assertNotNull(liveDocs);
 
     for(int i=0;i<35;i++) {
       if (liveDocs.get(i)) {
@@ -969,8 +964,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
         List<IndexableField> fields = d.getFields();
         boolean isProxDoc = d.getField("content3") == null;
         if (isProxDoc) {
-          final int numFields = is40Index ? 7 : 5;
-          assertEquals(numFields, fields.size());
+          assertEquals(7, fields.size());
           IndexableField f = d.getField("id");
           assertEquals(""+i, f.stringValue());
 
@@ -997,84 +991,74 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
       }
     }
 
-    if (is40Index) {
-      // check docvalues fields
-      NumericDocValues dvByte = MultiDocValues.getNumericValues(reader, "dvByte");
-      BinaryDocValues dvBytesDerefFixed = MultiDocValues.getBinaryValues(reader, "dvBytesDerefFixed");
-      BinaryDocValues dvBytesDerefVar = MultiDocValues.getBinaryValues(reader, "dvBytesDerefVar");
-      SortedDocValues dvBytesSortedFixed = MultiDocValues.getSortedValues(reader, "dvBytesSortedFixed");
-      SortedDocValues dvBytesSortedVar = MultiDocValues.getSortedValues(reader, "dvBytesSortedVar");
-      BinaryDocValues dvBytesStraightFixed = MultiDocValues.getBinaryValues(reader, "dvBytesStraightFixed");
-      BinaryDocValues dvBytesStraightVar = MultiDocValues.getBinaryValues(reader, "dvBytesStraightVar");
-      NumericDocValues dvDouble = MultiDocValues.getNumericValues(reader, "dvDouble");
-      NumericDocValues dvFloat = MultiDocValues.getNumericValues(reader, "dvFloat");
-      NumericDocValues dvInt = MultiDocValues.getNumericValues(reader, "dvInt");
-      NumericDocValues dvLong = MultiDocValues.getNumericValues(reader, "dvLong");
-      NumericDocValues dvPacked = MultiDocValues.getNumericValues(reader, "dvPacked");
-      NumericDocValues dvShort = MultiDocValues.getNumericValues(reader, "dvShort");
-      SortedSetDocValues dvSortedSet = null;
-      if (is42Index) {
-        dvSortedSet = MultiDocValues.getSortedSetValues(reader, "dvSortedSet");
-      }
-      SortedNumericDocValues dvSortedNumeric = null;
-      if (is49Index) {
-        dvSortedNumeric = MultiDocValues.getSortedNumericValues(reader, "dvSortedNumeric");
-      }
-      
-      for (int i=0;i<35;i++) {
-        int id = Integer.parseInt(reader.document(i).get("id"));
-        assertEquals(i, dvByte.nextDoc());
-        assertEquals(id, dvByte.longValue());
-        
-        byte bytes[] = new byte[] {
-            (byte)(id >>> 24), (byte)(id >>> 16),(byte)(id >>> 8),(byte)id
-        };
-        BytesRef expectedRef = new BytesRef(bytes);
-        
-        assertEquals(i, dvBytesDerefFixed.nextDoc());
-        BytesRef term = dvBytesDerefFixed.binaryValue();
-        assertEquals(expectedRef, term);
-        assertEquals(i, dvBytesDerefVar.nextDoc());
-        term = dvBytesDerefVar.binaryValue();
-        assertEquals(expectedRef, term);
-        assertEquals(i, dvBytesSortedFixed.nextDoc());
-        term = dvBytesSortedFixed.binaryValue();
-        assertEquals(expectedRef, term);
-        assertEquals(i, dvBytesSortedVar.nextDoc());
-        term = dvBytesSortedVar.binaryValue();
-        assertEquals(expectedRef, term);
-        assertEquals(i, dvBytesStraightFixed.nextDoc());
-        term = dvBytesStraightFixed.binaryValue();
-        assertEquals(expectedRef, term);
-        assertEquals(i, dvBytesStraightVar.nextDoc());
-        term = dvBytesStraightVar.binaryValue();
-        assertEquals(expectedRef, term);
-        
-        assertEquals(i, dvDouble.nextDoc());
-        assertEquals((double)id, Double.longBitsToDouble(dvDouble.longValue()), 0D);
-        assertEquals(i, dvFloat.nextDoc());
-        assertEquals((float)id, Float.intBitsToFloat((int)dvFloat.longValue()), 0F);
-        assertEquals(i, dvInt.nextDoc());
-        assertEquals(id, dvInt.longValue());
-        assertEquals(i, dvLong.nextDoc());
-        assertEquals(id, dvLong.longValue());
-        assertEquals(i, dvPacked.nextDoc());
-        assertEquals(id, dvPacked.longValue());
-        assertEquals(i, dvShort.nextDoc());
-        assertEquals(id, dvShort.longValue());
-        if (is42Index) {
-          assertEquals(i, dvSortedSet.nextDoc());
-          long ord = dvSortedSet.nextOrd();
-          assertEquals(SortedSetDocValues.NO_MORE_ORDS, dvSortedSet.nextOrd());
-          term = dvSortedSet.lookupOrd(ord);
-          assertEquals(expectedRef, term);
-        }
-        if (is49Index) {
-          assertEquals(i, dvSortedNumeric.nextDoc());
-          assertEquals(1, dvSortedNumeric.docValueCount());
-          assertEquals(id, dvSortedNumeric.nextValue());
-        }
-      }
+    // check docvalues fields
+    NumericDocValues dvByte = MultiDocValues.getNumericValues(reader, "dvByte");
+    BinaryDocValues dvBytesDerefFixed = MultiDocValues.getBinaryValues(reader, "dvBytesDerefFixed");
+    BinaryDocValues dvBytesDerefVar = MultiDocValues.getBinaryValues(reader, "dvBytesDerefVar");
+    SortedDocValues dvBytesSortedFixed = MultiDocValues.getSortedValues(reader, "dvBytesSortedFixed");
+    SortedDocValues dvBytesSortedVar = MultiDocValues.getSortedValues(reader, "dvBytesSortedVar");
+    BinaryDocValues dvBytesStraightFixed = MultiDocValues.getBinaryValues(reader, "dvBytesStraightFixed");
+    BinaryDocValues dvBytesStraightVar = MultiDocValues.getBinaryValues(reader, "dvBytesStraightVar");
+    NumericDocValues dvDouble = MultiDocValues.getNumericValues(reader, "dvDouble");
+    NumericDocValues dvFloat = MultiDocValues.getNumericValues(reader, "dvFloat");
+    NumericDocValues dvInt = MultiDocValues.getNumericValues(reader, "dvInt");
+    NumericDocValues dvLong = MultiDocValues.getNumericValues(reader, "dvLong");
+    NumericDocValues dvPacked = MultiDocValues.getNumericValues(reader, "dvPacked");
+    NumericDocValues dvShort = MultiDocValues.getNumericValues(reader, "dvShort");
+    SortedSetDocValues dvSortedSet = MultiDocValues.getSortedSetValues(reader, "dvSortedSet");
+    SortedNumericDocValues dvSortedNumeric = MultiDocValues.getSortedNumericValues(reader, "dvSortedNumeric");
+
+    for (int i=0;i<35;i++) {
+      int id = Integer.parseInt(reader.document(i).get("id"));
+      assertEquals(i, dvByte.nextDoc());
+      assertEquals(id, dvByte.longValue());
+
+      byte bytes[] = new byte[] {
+          (byte)(id >>> 24), (byte)(id >>> 16),(byte)(id >>> 8),(byte)id
+      };
+      BytesRef expectedRef = new BytesRef(bytes);
+
+      assertEquals(i, dvBytesDerefFixed.nextDoc());
+      BytesRef term = dvBytesDerefFixed.binaryValue();
+      assertEquals(expectedRef, term);
+      assertEquals(i, dvBytesDerefVar.nextDoc());
+      term = dvBytesDerefVar.binaryValue();
+      assertEquals(expectedRef, term);
+      assertEquals(i, dvBytesSortedFixed.nextDoc());
+      term = dvBytesSortedFixed.binaryValue();
+      assertEquals(expectedRef, term);
+      assertEquals(i, dvBytesSortedVar.nextDoc());
+      term = dvBytesSortedVar.binaryValue();
+      assertEquals(expectedRef, term);
+      assertEquals(i, dvBytesStraightFixed.nextDoc());
+      term = dvBytesStraightFixed.binaryValue();
+      assertEquals(expectedRef, term);
+      assertEquals(i, dvBytesStraightVar.nextDoc());
+      term = dvBytesStraightVar.binaryValue();
+      assertEquals(expectedRef, term);
+
+      assertEquals(i, dvDouble.nextDoc());
+      assertEquals((double)id, Double.longBitsToDouble(dvDouble.longValue()), 0D);
+      assertEquals(i, dvFloat.nextDoc());
+      assertEquals((float)id, Float.intBitsToFloat((int)dvFloat.longValue()), 0F);
+      assertEquals(i, dvInt.nextDoc());
+      assertEquals(id, dvInt.longValue());
+      assertEquals(i, dvLong.nextDoc());
+      assertEquals(id, dvLong.longValue());
+      assertEquals(i, dvPacked.nextDoc());
+      assertEquals(id, dvPacked.longValue());
+      assertEquals(i, dvShort.nextDoc());
+      assertEquals(id, dvShort.longValue());
+
+      assertEquals(i, dvSortedSet.nextDoc());
+      long ord = dvSortedSet.nextOrd();
+      assertEquals(SortedSetDocValues.NO_MORE_ORDS, dvSortedSet.nextOrd());
+      term = dvSortedSet.lookupOrd(ord);
+      assertEquals(expectedRef, term);
+
+      assertEquals(i, dvSortedNumeric.nextDoc());
+      assertEquals(1, dvSortedNumeric.docValueCount());
+      assertEquals(id, dvSortedNumeric.nextValue());
     }
     
     ScoreDoc[] hits = searcher.search(new TermQuery(new Term(new String("content"), "aaa")), 1000).scoreDocs;
@@ -1084,16 +1068,14 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     assertEquals("didn't get the right document first", "0", d.get("id"));
 
     doTestHits(hits, 34, searcher.getIndexReader());
-    
-    if (is40Index) {
-      hits = searcher.search(new TermQuery(new Term(new String("content5"), "aaa")), 1000).scoreDocs;
 
-      doTestHits(hits, 34, searcher.getIndexReader());
-    
-      hits = searcher.search(new TermQuery(new Term(new String("content6"), "aaa")), 1000).scoreDocs;
+    hits = searcher.search(new TermQuery(new Term(new String("content5"), "aaa")), 1000).scoreDocs;
 
-      doTestHits(hits, 34, searcher.getIndexReader());
-    }
+    doTestHits(hits, 34, searcher.getIndexReader());
+
+    hits = searcher.search(new TermQuery(new Term(new String("content6"), "aaa")), 1000).scoreDocs;
+
+    doTestHits(hits, 34, searcher.getIndexReader());
 
     hits = searcher.search(new TermQuery(new Term("utf8", "\u0000")), 1000).scoreDocs;
     assertEquals(34, hits.length);
@@ -1102,22 +1084,20 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     hits = searcher.search(new TermQuery(new Term("utf8", "ab\ud917\udc17cd")), 1000).scoreDocs;
     assertEquals(34, hits.length);
 
-    if (hasPoints) {
-      doTestHits(searcher.search(IntPoint.newRangeQuery("intPoint1d", 0, 34), 1000).scoreDocs, 34, searcher.getIndexReader());
-      doTestHits(searcher.search(IntPoint.newRangeQuery("intPoint2d", new int[] {0, 0}, new int[] {34, 68}), 1000).scoreDocs, 34, searcher.getIndexReader());
-      doTestHits(searcher.search(FloatPoint.newRangeQuery("floatPoint1d", 0f, 34f), 1000).scoreDocs, 34, searcher.getIndexReader());
-      doTestHits(searcher.search(FloatPoint.newRangeQuery("floatPoint2d", new float[] {0f, 0f}, new float[] {34f, 68f}), 1000).scoreDocs, 34, searcher.getIndexReader());
-      doTestHits(searcher.search(LongPoint.newRangeQuery("longPoint1d", 0, 34), 1000).scoreDocs, 34, searcher.getIndexReader());
-      doTestHits(searcher.search(LongPoint.newRangeQuery("longPoint2d", new long[] {0, 0}, new long[] {34, 68}), 1000).scoreDocs, 34, searcher.getIndexReader());
-      doTestHits(searcher.search(DoublePoint.newRangeQuery("doublePoint1d", 0.0, 34.0), 1000).scoreDocs, 34, searcher.getIndexReader());
-      doTestHits(searcher.search(DoublePoint.newRangeQuery("doublePoint2d", new double[] {0.0, 0.0}, new double[] {34.0, 68.0}), 1000).scoreDocs, 34, searcher.getIndexReader());
-      
-      byte[] bytes1 = new byte[4];
-      byte[] bytes2 = new byte[] {0, 0, 0, (byte) 34};
-      doTestHits(searcher.search(BinaryPoint.newRangeQuery("binaryPoint1d", bytes1, bytes2), 1000).scoreDocs, 34, searcher.getIndexReader());
-      byte[] bytes3 = new byte[] {0, 0, 0, (byte) 68};
-      doTestHits(searcher.search(BinaryPoint.newRangeQuery("binaryPoint2d", new byte[][] {bytes1, bytes1}, new byte[][] {bytes2, bytes3}), 1000).scoreDocs, 34, searcher.getIndexReader());
-    }
+    doTestHits(searcher.search(IntPoint.newRangeQuery("intPoint1d", 0, 34), 1000).scoreDocs, 34, searcher.getIndexReader());
+    doTestHits(searcher.search(IntPoint.newRangeQuery("intPoint2d", new int[] {0, 0}, new int[] {34, 68}), 1000).scoreDocs, 34, searcher.getIndexReader());
+    doTestHits(searcher.search(FloatPoint.newRangeQuery("floatPoint1d", 0f, 34f), 1000).scoreDocs, 34, searcher.getIndexReader());
+    doTestHits(searcher.search(FloatPoint.newRangeQuery("floatPoint2d", new float[] {0f, 0f}, new float[] {34f, 68f}), 1000).scoreDocs, 34, searcher.getIndexReader());
+    doTestHits(searcher.search(LongPoint.newRangeQuery("longPoint1d", 0, 34), 1000).scoreDocs, 34, searcher.getIndexReader());
+    doTestHits(searcher.search(LongPoint.newRangeQuery("longPoint2d", new long[] {0, 0}, new long[] {34, 68}), 1000).scoreDocs, 34, searcher.getIndexReader());
+    doTestHits(searcher.search(DoublePoint.newRangeQuery("doublePoint1d", 0.0, 34.0), 1000).scoreDocs, 34, searcher.getIndexReader());
+    doTestHits(searcher.search(DoublePoint.newRangeQuery("doublePoint2d", new double[] {0.0, 0.0}, new double[] {34.0, 68.0}), 1000).scoreDocs, 34, searcher.getIndexReader());
+
+    byte[] bytes1 = new byte[4];
+    byte[] bytes2 = new byte[] {0, 0, 0, (byte) 34};
+    doTestHits(searcher.search(BinaryPoint.newRangeQuery("binaryPoint1d", bytes1, bytes2), 1000).scoreDocs, 34, searcher.getIndexReader());
+    byte[] bytes3 = new byte[] {0, 0, 0, (byte) 68};
+    doTestHits(searcher.search(BinaryPoint.newRangeQuery("binaryPoint2d", new byte[][] {bytes1, bytes1}, new byte[][] {bytes2, bytes3}), 1000).scoreDocs, 34, searcher.getIndexReader());
 
     reader.close();
   }
@@ -1606,9 +1586,13 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
     Path oldIndexDir = createTempDir("moreterms");
     TestUtil.unzip(getDataInputStream(moreTermsIndex), oldIndexDir);
     Directory dir = newFSDirectory(oldIndexDir);
+    DirectoryReader reader = DirectoryReader.open(dir);
+
     verifyUsesDefaultCodec(dir, moreTermsIndex);
-    // TODO: more tests
     TestUtil.checkIndex(dir);
+    searchExampleIndex(reader);
+
+    reader.close();
     dir.close();
   }
 
@@ -1787,22 +1771,57 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
       assertNotNull("Sorted index index " + name + " not found", resource);
       TestUtil.unzip(resource, path);
 
-      // TODO: more tests
       Directory dir = newFSDirectory(path);
-
       DirectoryReader reader = DirectoryReader.open(dir);
+
       assertEquals(1, reader.leaves().size());
       Sort sort = reader.leaves().get(0).reader().getMetaData().getSort();
       assertNotNull(sort);
       assertEquals("<long: \"dateDV\">!", sort.toString());
       reader.close();
 
-      // this will confirm the docs really are sorted:
+      // This will confirm the docs are really sorted
       TestUtil.checkIndex(dir);
       dir.close();
     }
   }
 
+  private void searchExampleIndex(DirectoryReader reader) throws IOException {
+    IndexSearcher searcher = newSearcher(reader);
+
+    TopDocs topDocs = searcher.search(new NormsFieldExistsQuery("titleTokenized"), 10);
+    assertEquals(50, topDocs.totalHits.value);
+
+    topDocs = searcher.search(new DocValuesFieldExistsQuery("titleDV"), 10);
+    assertEquals(50, topDocs.totalHits.value);
+
+    topDocs = searcher.search(new TermQuery(new Term("body", "ja")), 10);
+    assertTrue(topDocs.totalHits.value > 0);
+
+    topDocs =
+            searcher.search(
+                    IntPoint.newRangeQuery("docid_int", 42, 44),
+                    10,
+                    new Sort(new SortField("docid_intDV", SortField.Type.INT)));
+    assertEquals(3, topDocs.totalHits.value);
+    assertEquals(3, topDocs.scoreDocs.length);
+    assertEquals(42, ((FieldDoc) topDocs.scoreDocs[0]).fields[0]);
+    assertEquals(43, ((FieldDoc) topDocs.scoreDocs[1]).fields[0]);
+    assertEquals(44, ((FieldDoc) topDocs.scoreDocs[2]).fields[0]);
+
+    topDocs = searcher.search(new TermQuery(new Term("body", "the")), 5);
+    assertTrue(topDocs.totalHits.value > 0);
+
+    topDocs =
+            searcher.search(
+                    new MatchAllDocsQuery(), 5, new Sort(new SortField("dateDV", SortField.Type.LONG)));
+    assertEquals(50, topDocs.totalHits.value);
+    assertEquals(5, topDocs.scoreDocs.length);
+    long firstDate = (Long) ((FieldDoc) topDocs.scoreDocs[0]).fields[0];
+    long lastDate = (Long) ((FieldDoc) topDocs.scoreDocs[4]).fields[0];
+    assertTrue(firstDate <= lastDate);
+  }
+
   /**
    * Tests that {@link CheckIndex} can detect invalid sort on sorted indices created
    * before https://issues.apache.org/jira/browse/LUCENE-8592.