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:58 UTC

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

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);
+  }
+}